release commit
diff --git a/.gitignore b/.gitignore
new file mode 100644
index 0000000..9ca9e43
--- /dev/null
+++ b/.gitignore
@@ -0,0 +1,21 @@
+/classes
+/lib
+deploy/lib
+deploy/logs
+.emacs-project
+*.jar
+bin/jzmq
+.DS_Store
+pom.xml
+deploy/classes
+*.fyc
+*.rbc
+*.pyc
+CHILD
+CHILDMAKER
+NANNY
+\#project.clj\#
+.\#project.clj
+.lein-failures
+_release
+*.zip
diff --git a/LICENSE.html b/LICENSE.html
new file mode 100644
index 0000000..aaa5ca7
--- /dev/null
+++ b/LICENSE.html
@@ -0,0 +1,261 @@
+<?xml version="1.0" encoding="ISO-8859-1" ?>
+<!DOCTYPE html PUBLIC "-//W3C//DTD XHTML 1.0 Transitional//EN" "http://www.w3.org/TR/xhtml1/DTD/xhtml1-transitional.dtd">
+<html xmlns="http://www.w3.org/1999/xhtml">
+
+<head>
+<meta http-equiv="Content-Type" content="text/html; charset=ISO-8859-1" />
+<title>Eclipse Public License - Version 1.0</title>
+<style type="text/css">
+  body {
+    size: 8.5in 11.0in;
+    margin: 0.25in 0.5in 0.25in 0.5in;
+    tab-interval: 0.5in;
+    }
+  p {  	
+    margin-left: auto;
+    margin-top:  0.5em;
+    margin-bottom: 0.5em;
+    }
+  p.list {
+  	margin-left: 0.5in;
+    margin-top:  0.05em;
+    margin-bottom: 0.05em;
+    }
+  </style>
+
+</head>
+
+<body lang="EN-US">
+<p>Copyright (c) Nathan Marz. All rights reserved.</p>
+<p align=center><b>Eclipse Public License - v 1.0</b></p>
+
+<p>THE ACCOMPANYING PROGRAM IS PROVIDED UNDER THE TERMS OF THIS ECLIPSE
+PUBLIC LICENSE (&quot;AGREEMENT&quot;). ANY USE, REPRODUCTION OR
+DISTRIBUTION OF THE PROGRAM CONSTITUTES RECIPIENT'S ACCEPTANCE OF THIS
+AGREEMENT.</p>
+
+<p><b>1. DEFINITIONS</b></p>
+
+<p>&quot;Contribution&quot; means:</p>
+
+<p class="list">a) in the case of the initial Contributor, the initial
+code and documentation distributed under this Agreement, and</p>
+<p class="list">b) in the case of each subsequent Contributor:</p>
+<p class="list">i) changes to the Program, and</p>
+<p class="list">ii) additions to the Program;</p>
+<p class="list">where such changes and/or additions to the Program
+originate from and are distributed by that particular Contributor. A
+Contribution 'originates' from a Contributor if it was added to the
+Program by such Contributor itself or anyone acting on such
+Contributor's behalf. Contributions do not include additions to the
+Program which: (i) are separate modules of software distributed in
+conjunction with the Program under their own license agreement, and (ii)
+are not derivative works of the Program.</p>
+
+<p>&quot;Contributor&quot; means any person or entity that distributes
+the Program.</p>
+
+<p>&quot;Licensed Patents&quot; mean patent claims licensable by a
+Contributor which are necessarily infringed by the use or sale of its
+Contribution alone or when combined with the Program.</p>
+
+<p>&quot;Program&quot; means the Contributions distributed in accordance
+with this Agreement.</p>
+
+<p>&quot;Recipient&quot; means anyone who receives the Program under
+this Agreement, including all Contributors.</p>
+
+<p><b>2. GRANT OF RIGHTS</b></p>
+
+<p class="list">a) Subject to the terms of this Agreement, each
+Contributor hereby grants Recipient a non-exclusive, worldwide,
+royalty-free copyright license to reproduce, prepare derivative works
+of, publicly display, publicly perform, distribute and sublicense the
+Contribution of such Contributor, if any, and such derivative works, in
+source code and object code form.</p>
+
+<p class="list">b) Subject to the terms of this Agreement, each
+Contributor hereby grants Recipient a non-exclusive, worldwide,
+royalty-free patent license under Licensed Patents to make, use, sell,
+offer to sell, import and otherwise transfer the Contribution of such
+Contributor, if any, in source code and object code form. This patent
+license shall apply to the combination of the Contribution and the
+Program if, at the time the Contribution is added by the Contributor,
+such addition of the Contribution causes such combination to be covered
+by the Licensed Patents. The patent license shall not apply to any other
+combinations which include the Contribution. No hardware per se is
+licensed hereunder.</p>
+
+<p class="list">c) Recipient understands that although each Contributor
+grants the licenses to its Contributions set forth herein, no assurances
+are provided by any Contributor that the Program does not infringe the
+patent or other intellectual property rights of any other entity. Each
+Contributor disclaims any liability to Recipient for claims brought by
+any other entity based on infringement of intellectual property rights
+or otherwise. As a condition to exercising the rights and licenses
+granted hereunder, each Recipient hereby assumes sole responsibility to
+secure any other intellectual property rights needed, if any. For
+example, if a third party patent license is required to allow Recipient
+to distribute the Program, it is Recipient's responsibility to acquire
+that license before distributing the Program.</p>
+
+<p class="list">d) Each Contributor represents that to its knowledge it
+has sufficient copyright rights in its Contribution, if any, to grant
+the copyright license set forth in this Agreement.</p>
+
+<p><b>3. REQUIREMENTS</b></p>
+
+<p>A Contributor may choose to distribute the Program in object code
+form under its own license agreement, provided that:</p>
+
+<p class="list">a) it complies with the terms and conditions of this
+Agreement; and</p>
+
+<p class="list">b) its license agreement:</p>
+
+<p class="list">i) effectively disclaims on behalf of all Contributors
+all warranties and conditions, express and implied, including warranties
+or conditions of title and non-infringement, and implied warranties or
+conditions of merchantability and fitness for a particular purpose;</p>
+
+<p class="list">ii) effectively excludes on behalf of all Contributors
+all liability for damages, including direct, indirect, special,
+incidental and consequential damages, such as lost profits;</p>
+
+<p class="list">iii) states that any provisions which differ from this
+Agreement are offered by that Contributor alone and not by any other
+party; and</p>
+
+<p class="list">iv) states that source code for the Program is available
+from such Contributor, and informs licensees how to obtain it in a
+reasonable manner on or through a medium customarily used for software
+exchange.</p>
+
+<p>When the Program is made available in source code form:</p>
+
+<p class="list">a) it must be made available under this Agreement; and</p>
+
+<p class="list">b) a copy of this Agreement must be included with each
+copy of the Program.</p>
+
+<p>Contributors may not remove or alter any copyright notices contained
+within the Program.</p>
+
+<p>Each Contributor must identify itself as the originator of its
+Contribution, if any, in a manner that reasonably allows subsequent
+Recipients to identify the originator of the Contribution.</p>
+
+<p><b>4. COMMERCIAL DISTRIBUTION</b></p>
+
+<p>Commercial distributors of software may accept certain
+responsibilities with respect to end users, business partners and the
+like. While this license is intended to facilitate the commercial use of
+the Program, the Contributor who includes the Program in a commercial
+product offering should do so in a manner which does not create
+potential liability for other Contributors. Therefore, if a Contributor
+includes the Program in a commercial product offering, such Contributor
+(&quot;Commercial Contributor&quot;) hereby agrees to defend and
+indemnify every other Contributor (&quot;Indemnified Contributor&quot;)
+against any losses, damages and costs (collectively &quot;Losses&quot;)
+arising from claims, lawsuits and other legal actions brought by a third
+party against the Indemnified Contributor to the extent caused by the
+acts or omissions of such Commercial Contributor in connection with its
+distribution of the Program in a commercial product offering. The
+obligations in this section do not apply to any claims or Losses
+relating to any actual or alleged intellectual property infringement. In
+order to qualify, an Indemnified Contributor must: a) promptly notify
+the Commercial Contributor in writing of such claim, and b) allow the
+Commercial Contributor to control, and cooperate with the Commercial
+Contributor in, the defense and any related settlement negotiations. The
+Indemnified Contributor may participate in any such claim at its own
+expense.</p>
+
+<p>For example, a Contributor might include the Program in a commercial
+product offering, Product X. That Contributor is then a Commercial
+Contributor. If that Commercial Contributor then makes performance
+claims, or offers warranties related to Product X, those performance
+claims and warranties are such Commercial Contributor's responsibility
+alone. Under this section, the Commercial Contributor would have to
+defend claims against the other Contributors related to those
+performance claims and warranties, and if a court requires any other
+Contributor to pay any damages as a result, the Commercial Contributor
+must pay those damages.</p>
+
+<p><b>5. NO WARRANTY</b></p>
+
+<p>EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, THE PROGRAM IS
+PROVIDED ON AN &quot;AS IS&quot; BASIS, WITHOUT WARRANTIES OR CONDITIONS
+OF ANY KIND, EITHER EXPRESS OR IMPLIED INCLUDING, WITHOUT LIMITATION,
+ANY WARRANTIES OR CONDITIONS OF TITLE, NON-INFRINGEMENT, MERCHANTABILITY
+OR FITNESS FOR A PARTICULAR PURPOSE. Each Recipient is solely
+responsible for determining the appropriateness of using and
+distributing the Program and assumes all risks associated with its
+exercise of rights under this Agreement , including but not limited to
+the risks and costs of program errors, compliance with applicable laws,
+damage to or loss of data, programs or equipment, and unavailability or
+interruption of operations.</p>
+
+<p><b>6. DISCLAIMER OF LIABILITY</b></p>
+
+<p>EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, NEITHER RECIPIENT
+NOR ANY CONTRIBUTORS SHALL HAVE ANY LIABILITY FOR ANY DIRECT, INDIRECT,
+INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING
+WITHOUT LIMITATION LOST PROFITS), 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 OR
+DISTRIBUTION OF THE PROGRAM OR THE EXERCISE OF ANY RIGHTS GRANTED
+HEREUNDER, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGES.</p>
+
+<p><b>7. GENERAL</b></p>
+
+<p>If any provision of this Agreement is invalid or unenforceable under
+applicable law, it shall not affect the validity or enforceability of
+the remainder of the terms of this Agreement, and without further action
+by the parties hereto, such provision shall be reformed to the minimum
+extent necessary to make such provision valid and enforceable.</p>
+
+<p>If Recipient institutes patent litigation against any entity
+(including a cross-claim or counterclaim in a lawsuit) alleging that the
+Program itself (excluding combinations of the Program with other
+software or hardware) infringes such Recipient's patent(s), then such
+Recipient's rights granted under Section 2(b) shall terminate as of the
+date such litigation is filed.</p>
+
+<p>All Recipient's rights under this Agreement shall terminate if it
+fails to comply with any of the material terms or conditions of this
+Agreement and does not cure such failure in a reasonable period of time
+after becoming aware of such noncompliance. If all Recipient's rights
+under this Agreement terminate, Recipient agrees to cease use and
+distribution of the Program as soon as reasonably practicable. However,
+Recipient's obligations under this Agreement and any licenses granted by
+Recipient relating to the Program shall continue and survive.</p>
+
+<p>Everyone is permitted to copy and distribute copies of this
+Agreement, but in order to avoid inconsistency the Agreement is
+copyrighted and may only be modified in the following manner. The
+Agreement Steward reserves the right to publish new versions (including
+revisions) of this Agreement from time to time. No one other than the
+Agreement Steward has the right to modify this Agreement. The Eclipse
+Foundation is the initial Agreement Steward. The Eclipse Foundation may
+assign the responsibility to serve as the Agreement Steward to a
+suitable separate entity. Each new version of the Agreement will be
+given a distinguishing version number. The Program (including
+Contributions) may always be distributed subject to the version of the
+Agreement under which it was received. In addition, after a new version
+of the Agreement is published, Contributor may elect to distribute the
+Program (including its Contributions) under the new version. Except as
+expressly stated in Sections 2(a) and 2(b) above, Recipient receives no
+rights or licenses to the intellectual property of any Contributor under
+this Agreement, whether expressly, by implication, estoppel or
+otherwise. All rights in the Program not expressly granted under this
+Agreement are reserved.</p>
+
+<p>This Agreement is governed by the laws of the State of New York and
+the intellectual property laws of the United States of America. No party
+to this Agreement will bring a legal action under this Agreement more
+than one year after the cause of action arose. Each party waives its
+rights to a jury trial in any resulting litigation.</p>
+
+</body>
+
+</html>
diff --git a/README.markdown b/README.markdown
new file mode 100644
index 0000000..372054a
--- /dev/null
+++ b/README.markdown
@@ -0,0 +1,26 @@
+Storm is a distributed realtime computation system. Similar to how Hadoop provides a set of general primitives for doing batch processing, Storm provides a set of general primitives for doing realtime computation. Storm is simple, can be used with any programming language, and is a lot of fun to use!
+
+## Documentation
+
+Documentation and tutorials can be found on the [Storm wiki](http://github.com/nathanmarz/storm/wiki).
+
+## Getting help
+
+Feel free to ask questions on Storm's mailing list: http://groups.google.com/group/storm-user
+
+You can also come to the #storm-user room on [freenode](http://freenode.net/). You can usually find a Storm developer there to help you out.
+
+## License
+
+The use and distribution terms for this software are covered by the
+Eclipse Public License 1.0 (http://opensource.org/licenses/eclipse-1.0.php)
+which can be found in the file LICENSE.html at the root of this distribution.
+By using this software in any fashion, you are agreeing to be bound by
+the terms of this license.
+You must not remove this notice, or any other, from this software.
+
+## Contributors
+
+* Nathan Marz ([@nathanmarz](http://twitter.com/nathanmarz))
+* Jason Jackson ([@jason_j](http://twitter.com/jason_j))
+* Christopher Bertels ([@bakkdoor](http://twitter.com/bakkdoor))
diff --git a/TODO b/TODO
new file mode 100644
index 0000000..23c8837
--- /dev/null
+++ b/TODO
@@ -0,0 +1,178 @@
+Use cases:
+
+1. number of steps between 2 people in a graph (topology with cycles?)
+
+
+#################
+
+* Repackage jzmq and zmq as a leiningen "native dep"
+       - this might be good, since the native dep can package builds for all different systems/os's?
+
+
+* Deploy design:
+
+- storm swap {name} {jar} {class}
+- it's allowed to use resources equal to current running topology plus number of free resources
+- starts in deactivated mode
+- add TOPOLOGY_STARTUP_TIME config for the delay until nimbus activates a topology after launching it
+- for swap, after the startup time, deactivate the other topology, wait the TOPOLOGY_MESSAGE_TIMEOUT_SECS, and then activate the other topology
+- should be able to decrease the message timeout for killing or swapping (add optional thrift parameter) -- or just make it part of the config?
+- add killWithOptions, swap, swapWithOptions
+
+* Storm UI, stats, debugging, diagnosis tools
+-- need to be able to hide system streams/components from the calculations (another query param and should be default)
+-- need to optimize (slowness is probably on nimbus end of querying zk, consider adding heartbeat caching into nimbus)
+-- add margins
+-- add titles so its easier to distinguish the various pages
+-- right align all table columns except for the leftmost
+		
+* Unit test the core pieces that have stabilized their APIs
+
+- process simulator
+- virtual ports
+- supervisor
+- utils
+- test worker/tasks
+
+* implement pseudo-distributed mode -- this is for testing the distributed parts of the code
+  - perhaps i can use pallet/vmfest for this
+
+* Need integration tests that run on an actual storm cluster (scp code/process code/zookeeper code not tested in unit tests)
+
+* bolts with none grouping can be pushed into a bolt. e.g. A -> B -> C
+     A -> D -> E
+     
+If A -> B and A -> D are shuffle grouping = none, and B -> C and D -> E are not, then both can be run in A, b's branch goes to C and D's branch goes to E
+
+
+* Failure design
+
+Add fail method to outputcollector
+Fail sends fail message to Acker for those anchors, which sends fail message back to spout.
+Whenever spout fails a tuple, it emits it in its failure stream...
+
+Add fail method to drpc... Causes blocked thread to throw exception
+
+* Have worker heartbeat with its task ids, nimbus verifies - if wrong, reassign tasks?
+- detect and ignore stray tasks
+Each worker can choose a unique id for itself when heart beating
+- nimbus deletes those that aren't in topology
+
+* Subscriptions design
+
+-- new kind of spout: "subscription spout"
+   --> goal is to sync it's data across the tasks that subscribe to its streams
+   --> after doing a grouping, remembers what task it sent the tuple to (regardless of grouping). if a task dies, it knows its subscriptions and asks to be resynced
+   --> normal operation is to push to tasks, but pull done when a task starts up (b/c previous task died or something)
+   --> need to be able to add tuples to subscription or take tuples away (this is protocol with who you're subscribing to - e.g. rocket)
+   --> subscriptions can only happen in a spout because it requires persistent state
+   --> when subscription spout task dies, it polls the source (e.g. rocket) for all the subscription info
+   --> ideally you'd set things up to have one subscription spout per rocket server
+   --> TODO: Need some way to delete subscriptions -> part of tuple or extra metadata on tuple (extra metadata seems cleaner)
+        --> add isSubscription() method to Tuple as well as a getSubscriptionType() [which returns ADD or REMOVE]
+   --> when a spout starts up, it also needs to push all of its subscription info
+   --> acks are irrelevant for subscription tuples -- how should acks be managed as an abstraction?
+        -- maybe the synchronized state is done for you -- you just access the state directly and receive a callback whenever it changes?
+        -- so don't use tuples...
+   --> subscriptions break all the abstractions, perhaps I should generalize spouts and factor acking as a library on top of storm. subscriptions would just be another kind of library? -> no, it seems to break abstractions anyway (like keeping task -> tuples in memory)
+   --> maybe call it "syncspout"
+   --> if just do syncing (don't expose tuples directly?)
+   --> have a "SubscribedState" class that takes care of indexing/etc. --> expose it through topologycontext?
+      -- need a way to distinguish between states of different streams
+      -- has "add" and "remove" methods
+      -- bolt can give a statemanager object that implements add and remove in the prepare method
+      -- add(Tuple tuple)
+      -- remove(Tuple tuple)
+   --> synchronize protocol (when spout or source of data dies):
+      --> send how many tuples are going to be sent
+      --> send the tuples
+      --> OR: pack everything together into a single message (could be hard b/c where tuples are supposed to go is abstracted away)
+      --> tie everything together with a unique ID
+      --> once task receives everything, has info needed to remove tuples
+   --> statespout should do long-polling with timeout
+   --> to do subscriptions, the state should contain something like [url, subscriber]. some bolt appends subscriber to tuples, group by subscriber, and send info back
+        --> how to to fields grouping with an even distribution?
+   -->  ********* tasks need to block on startup until they're synchronized *********
+          --> send sync messages in a loop until it's synchronized
+          --> add a task.synchronize.poll.freq.secs config (default to 10 seconds)
+          --> need to buffer other messages as topology is waiting for synchronization messages (use disk?)
+   --> could use acking system to know if a piece of state gets fully synchronized and communicate this with user
+      --> perhaps expose this through a special stream? (the state status stream -> similar to failure streams)
+   --> should be able to do updates of existing state
+      --> use case: have a knob that you can set externally
+      --> this isn't really any better than just using zookeeper directly
+   
+   
+_myState = context.setSubscribedState(_myState)
+
+StateSpout {
+  //does a timeout long poll and emits new add or remove state tuples (add and remove on the output collector)
+  nextTuple(StateSpoutOutputCollector) //collector has add and remove methods add(id, tuple). remove(id)
+  //emits all the tuples into the output collector (in the background, will also send ids and counts to tasks so they know how to synchronize)
+  //called on startup
+  //collector can have a synchronize method in case the source of data (e.g., rocket) craps out
+  synchronize(SynchronizationOutputCollector) //collector only has add(id, tuple) method
+}
+
+//task startup (in prepare method) [this is automatic]
+for(int taskId: statespoutids) {
+  emitDirect(SYNC_STREAM, tuple())
+}
+
+statespout synchronization():
+  id = uuid()
+  //getAlLStateTuples calls synchronize on the spout to get the tuples
+  for(Tuple t: getAllStateTuplesFromSource()) {
+    List tasks = emit(cons(id, t));
+    .. keep track of id -> tasks -> count
+    for(task: all output tasks) {
+      emitDirect(task, id, count)
+    } 
+  }
+
+for synchronization to work, task needs to keep track of which tasks sent it tuples, and compare against only that set on synchronization
+
+Need a way to propogate information back up the topology - "subscriptions"
+e.g. browser -> rocket -> bolt -> bolt -> bolt. 
+
+example: #retweets for a subscribed set of tweet ids
+
+storm topology
+
+ -> tweet spout (A) -> group on original id -> count (B) -> rocket
+
+subscriptions: rocket -> count (B) tweet id (need to group) -> spout (need to go to all)
+
+-- how does it work when stuff dies downstream or upstream? do people ask what the subscriptions are? or do you push your subscriptions up? a combination?
+
+-- maybe subscriptions are a "constant" spout? e..g, continuously emits and refreshes to make sure every task has the tuple. this seem amporphous and hard to implement... nimbus would need to refire all constant spouts whenever there's a reassignment that affects the flow of data. subscriptions seem more natural
+
+-- subscriptions are a special kind of stream that are driven by being asked to send it. e..g, rocket is a spout that emits subscription/unsubscription tuples. they only send it when they get something new, or are asked as to what all the subscriptions are
+
+-- maybe you just need a system stream to know when tasks are created. when you see that a downstream task is created, you know to fire subscriptions to it if its subscribed to your subscriptions stream? - how does this interplay with all the grouping types... you almost want to do a grouping and only send what to tasks that would have received. spouts would need to be able to subscribe to streams as well
+
+(use 'backtype.storm.testing)
+;;(start-simulating-time!)
+(def cluster (mk-local-storm-cluster))
+(use 'backtype.storm.bootstrap) (bootstrap)
+(import '[backtype.storm.testing TestWordCounter TestWordSpout TestGlobalCount TestAggregatesCounter])
+(def spout (feeder-spout ["word"]))
+(def topology (thrift/mk-topology
+                    {1 (thrift/mk-spout-spec spout :parallelism-hint 3)}
+                    {2 (thrift/mk-bolt-spec {1 ["word"]} (TestWordCounter.) :parallelism-hint 4)
+                     3 (thrift/mk-bolt-spec {1 :global} (TestGlobalCount.))
+                     4 (thrift/mk-bolt-spec {2 :global} (TestAggregatesCounter.))
+                     }))
+(submit-local-topology (:nimbus cluster) "test" {TOPOLOGY-WORKERS 4 TOPOLOGY-DEBUG true} topology)
+
+
+* clean up project
+  - remove log4j dir and instead generate it in the deploy (it's only used in bin/storm -> create a console one and put into bin/)
+  - include system component / stream information in the topologycontext and clean up system specific code all over the place
+
+* Very rare errors
+
+weird nullptr exceptions:
+(tasks i) on send-fn
+no virtual port socket for outbound task (in worker)
+
diff --git a/bin/build_release.sh b/bin/build_release.sh
new file mode 100644
index 0000000..7b5dcb2
--- /dev/null
+++ b/bin/build_release.sh
@@ -0,0 +1,41 @@
+#!/bin/bash
+
+RELEASE=`head -1 project.clj | awk '{print $3}' | sed -e 's/\"//' | sed -e 's/\"//'`
+
+echo Making release $RELEASE
+
+DIR=_release/storm-$RELEASE
+
+rm -rf _release
+export LEIN_ROOT=1
+rm *jar
+lein clean
+lein deps
+lein compile
+mv conf/log4j.properties conf/storm.log.properties
+lein jar
+mv conf/storm.log.properties conf/log4j.properties
+mkdir -p $DIR
+mkdir $DIR/lib
+cp storm*jar $DIR/
+cp lib/*.jar $DIR/lib
+
+cp -R log4j $DIR/
+mkdir $DIR/logs
+
+mkdir $DIR/conf
+cp conf/storm.yaml.example $DIR/conf/storm.yaml
+
+cp -R src/ui/public $DIR/
+
+cp -R bin $DIR/
+
+cp README.markdown $DIR/
+cp LICENSE.html $DIR/
+
+cd _release
+zip -r storm-$RELEASE.zip *
+cd ..
+mv _release/storm-*.zip .
+rm -rf _release
+
diff --git a/bin/install_zmq.sh b/bin/install_zmq.sh
new file mode 100755
index 0000000..b7a871b
--- /dev/null
+++ b/bin/install_zmq.sh
@@ -0,0 +1,29 @@
+#!/bin/bash
+export JAVA_HOME=$(/usr/libexec/java_home)
+
+if [ ! -d "$JAVA_HOME/include" ]; then
+    echo "
+Looks like you're missing your 'include' directory. If you're using Mac OS X, You'll need to install the Java dev package.
+
+- Navigate to http://goo.gl/D8lI
+- Click the Java tab on the right
+- Install the appropriate version and try again.
+"
+    exit -1;
+fi
+
+#install zeromq
+wget http://download.zeromq.org/historic/zeromq-2.1.7.tar.gz
+tar -xzf zeromq-2.1.7.tar.gz
+cd zeromq-2.1.7
+./configure
+make
+sudo make install
+
+#install jzmq (both native and into local maven cache)
+git clone https://github.com/nathanmarz/jzmq.git
+cd jzmq
+./autogen.sh
+./configure
+make
+sudo make install
diff --git a/bin/javadoc.sh b/bin/javadoc.sh
new file mode 100644
index 0000000..8f7905b
--- /dev/null
+++ b/bin/javadoc.sh
@@ -0,0 +1,2 @@
+mkdir -p doc
+javadoc -d doc/ `find src -name "*.java" | grep -v generated`
diff --git a/bin/storm b/bin/storm
new file mode 100755
index 0000000..a9e5c26
--- /dev/null
+++ b/bin/storm
@@ -0,0 +1,80 @@
+#!/usr/bin/python
+
+import os
+import sys
+import random
+import subprocess as sub
+
+CONF_DIR = os.path.expanduser("~/.storm")
+STORM_DIR = "/".join(os.path.abspath( __file__ ).split("/")[:-2])
+
+def get_jars_full(adir):
+    files = os.listdir(adir)
+    ret = []
+    for f in files:
+        if f.endswith(".jar"):
+            ret.append(adir + "/" + f)
+    return ret
+
+def get_classpath(extrajars):
+    ret = get_jars_full(STORM_DIR)
+    ret.extend(get_jars_full(STORM_DIR + "/lib"))
+    ret.extend(extrajars)
+    return ":".join(ret)
+
+def confvalue(name):
+    cp = get_classpath([])
+    command = ["java", "-client", "-cp", cp, "backtype.storm.command.config_value", name]
+    p = sub.Popen(command,stdout=sub.PIPE)
+    output, errors = p.communicate()
+    lines = output.split("\n")
+    for line in lines:
+        tokens = line.split(" ")
+        if tokens[0] == "VALUE:":
+            return tokens[1]
+
+def exec_storm_class(klass, jvmtype="-server", childopts="", extrajars=[], args=[], prefix=""):
+    nativepath = confvalue("java.library.path")
+    command = prefix + " java " + jvmtype + " -Djava.library.path=" + nativepath + " " + childopts + " -cp " + get_classpath(extrajars) + " " + klass + " " + " ".join(args)
+    print "Running: " + command    
+    os.system(command)
+
+COMMAND = sys.argv[1]
+ARGS = sys.argv[2:]
+
+def jar(jarfile, klass, *args):
+   exec_storm_class(
+        klass,
+        childopts="-Dlog4j.configuration=storm.log.properties",
+        jvmtype="-client",
+        extrajars=[jarfile, CONF_DIR, STORM_DIR + "/bin"],
+        args=args,
+        prefix="export STORM_JAR=" + jarfile + ";")
+
+def kill(name):
+    exec_storm_class("backtype.storm.command.kill_topology", args=[name], jvmtype="-client", extrajars=[CONF_DIR, STORM_DIR + "/bin"], childopts="-Dlog4j.configuration=storm.log.properties")
+
+def shell(resourcesdir, command, *args):
+    tmpjarpath = "stormshell" + str(random.randint(0, 10000000)) + ".jar"
+    os.system("jar cf %s %s" % (tmpjarpath, resourcesdir))
+    runnerargs = [tmpjarpath, command]
+    runnerargs.extend(args)
+    exec_storm_class("backtype.storm.command.shell_submission", args=runnerargs, jvmtype="-client", extrajars=[CONF_DIR], childopts="-Dlog4j.configuration=storm.log.properties")
+    os.system("rm " + tmpjarpath)
+
+def nimbus():
+    childopts = confvalue("nimbus.childopts") + " -Dlogfile.name=nimbus.log"
+    exec_storm_class("backtype.storm.daemon.nimbus", jvmtype="-server", extrajars=[STORM_DIR + "/log4j", STORM_DIR + "/conf"], childopts=childopts)
+
+def supervisor():
+    childopts = confvalue("nimbus.childopts") + " -Dlogfile.name=supervisor.log"
+    exec_storm_class("backtype.storm.daemon.supervisor", jvmtype="-server", extrajars=[STORM_DIR + "/log4j", STORM_DIR + "/conf"], childopts=childopts)
+
+def ui():
+    childopts = "-Xmx768m -Dlogfile.name=ui.log"
+    exec_storm_class("backtype.storm.ui.core", jvmtype="-server", childopts=childopts, extrajars=[STORM_DIR + "/log4j", STORM_DIR, STORM_DIR + "/conf"])
+
+
+COMMANDS = {"jar": jar, "kill": kill, "shell": shell, "nimbus": nimbus, "ui": ui, "supervisor": supervisor}
+
+COMMANDS[COMMAND](*ARGS)
diff --git a/conf/defaults.yaml b/conf/defaults.yaml
new file mode 100644
index 0000000..d222a38
--- /dev/null
+++ b/conf/defaults.yaml
@@ -0,0 +1,64 @@
+########### These all have default values as shown
+########### Additional configuration goes into storm.yaml
+
+java.library.path: "/usr/local/lib:/opt/local/lib:/usr/lib"
+
+### 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"
+storm.zookeeper.session.timeout: 10000
+storm.cluster.mode: "distributed" # can be distributed or local
+
+### nimbus.* configs are for the master
+nimbus.thrift.port: 6627
+nimbus.childopts: "-Xmx1024m"
+nimbus.task.timeout.secs: 30
+nimbus.supervisor.timeout.secs: 60
+nimbus.monitor.freq.secs: 10
+nimbus.task.launch.secs: 90
+nimbus.reassign: true
+nimbus.file.copy.expiration.secs: 600
+
+### supervisor.* configs are for node supervisors
+# Define the amount of workers that can be run on this machine. Each worker is assigned a port to use for communication
+supervisor.slots.ports:
+    - 6700
+    - 6701
+    - 6702
+    - 6703
+supervisor.childopts: "-Xmx1024m"
+#how long supervisor will wait to ensure that a worker process is started
+supervisor.worker.start.timeout.secs: 20
+#how long between heartbeats until supervisor considers that worker dead and tries to restart it
+supervisor.worker.timeout.secs: 25
+#how frequently the supervisor checks on the status of the processes it's monitoring and restarts if necessary
+supervisor.monitor.frequency.secs: 3
+#how frequently the supervisor heartbeats to the cluster state (for nimbus)
+supervisor.heartbeat.frequency.secs: 5
+supervisor.enable: true
+
+### worker.* configs are for task workers
+worker.childopts: "-Xmx768m"
+worker.heartbeat.frequency.secs: 1
+
+task.heartbeat.frequency.secs: 3
+task.refresh.poll.secs: 10
+
+zmq.threads: 1
+zmq.linger.millis: 5000
+
+### topology.* configs are for specific executing storms
+topology.debug: false
+topology.optimize: true
+topology.workers: 1
+topology.ackers: 1
+# maximum amount of time a message has to complete before it's considered failed
+topology.message.timeout.secs: 30
+topology.skip.missing.serializations: false
+topology.max.task.parallelism: null
+topology.max.spout.pending: null
+topology.state.synchronization.timeout.secs: 60
+topology.stats.sample.rate: 0.05
+
diff --git a/conf/log4j.properties b/conf/log4j.properties
new file mode 100644
index 0000000..f1c14b4
--- /dev/null
+++ b/conf/log4j.properties
@@ -0,0 +1,10 @@
+#This file should be deleted when deployed to server (workaround to leiningen classpath putting dev resources on path)
+#This file is needed for tests
+
+log4j.rootLogger=INFO, A1
+
+log4j.appender.A1=org.apache.log4j.ConsoleAppender
+
+log4j.appender.A1.layout=org.apache.log4j.PatternLayout
+log4j.appender.A1.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n
+log4j.category.org.apache.zookeeper=warn
diff --git a/conf/storm.yaml.example b/conf/storm.yaml.example
new file mode 100644
index 0000000..2401e02
--- /dev/null
+++ b/conf/storm.yaml.example
@@ -0,0 +1,16 @@
+########### These MUST be filled in for a storm configuration
+# storm.zookeeper.servers:
+#     - "server1"
+#     - "server2"
+# 
+# nimbus.host: "nimbus"
+# 
+# 
+# ##### These may optionally be filled in:
+#     
+## Map of tokens to a serialization class. tokens less than 32 are reserved by storm.
+## Tokens are written on the wire to identify the field.
+# topology.serializations: 
+#     33: "org.mycompany.MyObjectSerialization"
+#     34: "org.mycompany.MyOtherObjectSerialization"
+
diff --git a/log4j/log4j.properties b/log4j/log4j.properties
new file mode 100644
index 0000000..8cd69bd
--- /dev/null
+++ b/log4j/log4j.properties
@@ -0,0 +1,9 @@
+log4j.rootLogger=INFO, A1
+
+
+log4j.appender.A1 = org.apache.log4j.DailyRollingFileAppender
+log4j.appender.A1.File = logs/${logfile.name}
+log4j.appender.A1.Append = true
+log4j.appender.A1.DatePattern = '.'yyy-MM-dd
+log4j.appender.A1.layout = org.apache.log4j.PatternLayout
+log4j.appender.A1.layout.ConversionPattern = %d{yyyy-MM-dd HH:mm:ss} %c{1} [%p] %m%n
\ No newline at end of file
diff --git a/project.clj b/project.clj
new file mode 100644
index 0000000..5650086
--- /dev/null
+++ b/project.clj
@@ -0,0 +1,32 @@
+(defproject storm "0.5.0"
+  :source-path "src/clj"
+  :test-path "test/clj"
+  :java-source-path "src/jvm"
+  :javac-options {:debug "true" :fork "true"}
+  :resources-path "conf"
+  :dev-resources-path "src/dev"
+  :dependencies [[org.clojure/clojure "1.2.0"]
+                 [org.clojure/clojure-contrib "1.2.0"]
+                 [commons-io "1.4"]
+                 [org.apache.commons/commons-exec "1.1"]
+                 [jvyaml "1.0.0"]
+                 [backtype/thriftjava "1.0.0"]
+                 [clj-time "0.3.0"]
+                 [log4j/log4j "1.2.16"]
+                 [org.apache.zookeeper/zookeeper "3.3.2"]
+                 [backtype/jzmq "2.1.0"]
+                 [com.googlecode.json-simple/json-simple "1.1"]
+                 [compojure "0.6.4"]
+                 [hiccup "0.3.6"]
+                 [ring/ring-jetty-adapter "0.3.11"]
+                 ]
+  :uberjar-exclusions [#"META-INF.*"]
+  :dev-dependencies [
+                     [swank-clojure "1.2.1"]
+                     [lein-ring "0.4.5"]
+                    ]
+  :jvm-opts ["-Djava.library.path=/usr/local/lib:/opt/local/lib:/usr/lib"]
+  :ring {:handler backtype.storm.ui.core/app}
+  :extra-classpath-dirs ["src/ui"]
+  :aot :all
+)
diff --git a/src/clj/backtype/storm/LocalCluster.clj b/src/clj/backtype/storm/LocalCluster.clj
new file mode 100644
index 0000000..6acb075
--- /dev/null
+++ b/src/clj/backtype/storm/LocalCluster.clj
@@ -0,0 +1,26 @@
+(ns backtype.storm.LocalCluster
+  (:use [backtype.storm testing])
+  (:gen-class
+   :init init
+   :implements [backtype.storm.ILocalCluster]
+   :constructors {[] []}
+   :state state ))
+
+(defn -init []
+  (let [ret (mk-local-storm-cluster)]
+    [[] ret]
+    ))
+
+(defn -submitTopology [this name conf topology]
+  (submit-local-topology (:nimbus (. this state))
+                      name
+                      conf
+                      topology))
+
+(defn -shutdown [this]
+  (kill-local-storm-cluster (. this state))
+  )
+
+(defn -killTopology [this name]
+  (.killTopology (:nimbus (. this state)) name)
+  )
\ No newline at end of file
diff --git a/src/clj/backtype/storm/bootstrap.clj b/src/clj/backtype/storm/bootstrap.clj
new file mode 100644
index 0000000..cac2b45
--- /dev/null
+++ b/src/clj/backtype/storm/bootstrap.clj
@@ -0,0 +1,38 @@
+(ns backtype.storm.bootstrap)
+
+(defmacro bootstrap []
+  '(do
+      (import (quote [backtype.storm Constants]))
+      (import (quote [backtype.storm.testing FeederSpout TestPlannerBolt TestPlannerSpout AckFailDelegate AckTracker]))
+      (import (quote [backtype.storm.utils Utils LocalState Time TimeCacheMap
+                      TimeCacheMap$ExpiredCallback BufferFileInputStream]))
+      (import (quote [backtype.storm.serialization TupleSerializer TupleDeserializer]))
+      (import (quote [backtype.storm.spout ISpout SpoutOutputCollector ISpoutOutputCollector ShellSpout]))
+      (import (quote [backtype.storm.tuple Tuple Fields MessageId]))
+      (import (quote [backtype.storm.task IBolt IOutputCollector
+                      OutputCollector OutputCollectorImpl IInternalOutputCollector
+                      TopologyContext ShellBolt
+                      CoordinatedBolt CoordinatedBolt$SourceArgs KeyedFairBolt]))
+      (import (quote [backtype.storm.daemon Shutdownable]))
+      (use (quote [backtype.storm config util log clojure]))
+      (use (quote [clojure.contrib.seq :only [find-first]]))
+      (require (quote [backtype.storm [thrift :as thrift] [cluster :as cluster]
+                                      [event :as event] [process-simulator :as psim]]))
+      (require (quote [clojure.set :as set]))
+      (require (quote [zilch [mq :as mq]]))
+      (require (quote [zilch [virtual-port :as mqvp]]))
+      (require (quote [backtype.storm [stats :as stats]]))
+      (import (quote [org.apache.log4j PropertyConfigurator Logger]))
+
+      (import (quote [backtype.storm.generated Nimbus Nimbus$Processor Nimbus$Iface StormTopology ShellComponent
+                        NotAliveException AlreadyAliveException InvalidTopologyException
+                        ClusterSummary TopologyInfo TopologySummary TaskSummary TaskStats TaskSpecificStats
+                        SpoutStats BoltStats ErrorInfo SupervisorSummary]))
+      (import (quote [backtype.storm.daemon.common StormBase Assignment
+                      TaskInfo SupervisorInfo WorkerHeartbeat TaskHeartbeat]))
+      (import (quote [java.io File FileOutputStream FileInputStream]))
+      (import (quote [java.util List Random Map HashMap]))
+      (import (quote [org.apache.commons.io FileUtils]))
+      (import (quote [java.util ArrayList]))
+      (mq/zeromq-imports)
+      ))
diff --git a/src/clj/backtype/storm/clojure.clj b/src/clj/backtype/storm/clojure.clj
new file mode 100644
index 0000000..390d2aa
--- /dev/null
+++ b/src/clj/backtype/storm/clojure.clj
@@ -0,0 +1,71 @@
+(ns backtype.storm.clojure
+  (:use backtype.storm.bootstrap)
+  (:import [backtype.storm.generated StreamInfo])
+  (:import [backtype.storm.tuple Tuple])
+  (:import [backtype.storm.task OutputCollector])
+  (:import backtype.storm.clojure.ClojureBolt)
+  (:require [backtype.storm [thrift :as thrift]]))
+
+(defn direct-stream [fields]
+  (StreamInfo. fields true))
+
+(defn clojure-bolt* [output-spec fn-var & args]
+  (let [m (meta fn-var)]
+    (ClojureBolt. (str (:ns m)) (str (:name m)) args (thrift/mk-output-spec output-spec))
+    ))
+
+(defmacro clojure-bolt [output-spec fn-sym & args]
+  `(clojure-bolt* ~output-spec (var ~fn-sym) ~@args))
+
+(defmacro defbolt [name output-spec [tuple-sym collector-sym] & body]
+  (let [worker-name (symbol (str name "__"))]
+    `(do
+       (defn ~worker-name []
+         (fn [^Tuple ~tuple-sym ^OutputCollector ~collector-sym]
+           ~@body
+           ))
+       (def ~name (clojure-bolt ~output-spec ~worker-name))
+       )))
+
+(defn hint [sym class-sym]
+  (with-meta sym {:tag class-sym})
+  )
+
+(defmulti hinted-args (fn [kw args] kw))
+
+(defmethod hinted-args :prepare [_ [conf context collector]]
+           [(hint conf 'java.util.Map)
+            (hint context 'backtype.storm.task.TopologyContext)
+            (hint collector 'backtype.storm.bolt.OutputCollector)]
+           )
+
+(defmethod hinted-args :execute [_ [tuple collector]]
+           [(hint tuple 'backtype.storm.tuple.Tuple)
+            (hint collector 'backtype.storm.task.OutputCollector)]
+           )
+
+(defmethod hinted-args :cleanup [_ [collector]]
+           [(hint collector 'backtype.storm.task.OutputCollector)]
+           )
+
+(defmacro defboltfull [name output-spec & kwargs]
+  (let [opts (apply hash-map kwargs)
+        worker-name (symbol (str name "__"))
+        let-bindings (:let opts)
+        hof-args (:params opts)
+        definer (if hof-args
+                  `(defn ~name [& args#]
+                     (apply clojure-bolt* ~output-spec (var ~worker-name) args#))
+                  `(def ~name (clojure-bolt ~output-spec ~worker-name)))
+        fns (select-keys opts [:prepare :execute :cleanup])
+        fns (into {}
+                  (for [[fn-kw [args & impl]] fns]
+                    [fn-kw `(fn ~(hinted-args fn-kw args) ~@impl)]
+                    ))]
+    `(do
+       (defn ~worker-name [~@hof-args]
+         (let [~@let-bindings]
+           ~fns
+           ))
+       ~definer
+       )))
diff --git a/src/clj/backtype/storm/cluster.clj b/src/clj/backtype/storm/cluster.clj
new file mode 100644
index 0000000..09482e7
--- /dev/null
+++ b/src/clj/backtype/storm/cluster.clj
@@ -0,0 +1,422 @@
+(ns backtype.storm.cluster
+  (:import [org.apache.zookeeper.data Stat])
+  (:import [backtype.storm.utils Utils])
+  (:use [backtype.storm util log config])
+  (:use [clojure.contrib.core :only [dissoc-in]])
+  (:require [backtype.storm [zookeeper :as zk]])
+  )
+
+(defprotocol ClusterState
+  (set-ephemeral-node [this path data])
+  (delete-node [this path])
+  (set-data [this path data])  ;; if node does not exist, create persistent with this data 
+  (get-data [this path watch?])
+  (get-children [this path watch?])
+  (mkdirs [this path])
+  (close [this])
+  (register [this callback])
+  (unregister [this id])
+  )
+
+(defn mk-distributed-cluster-state [conf]
+  (let [zk (zk/mk-client (mk-zk-connect-string (assoc conf STORM-ZOOKEEPER-ROOT "/")))]
+    (zk/mkdirs zk (conf STORM-ZOOKEEPER-ROOT))
+    (.close zk)
+    )
+  (let [callbacks (atom {})
+        active (atom true)
+        mk-zk #(zk/mk-client (mk-zk-connect-string conf)
+                             (conf STORM-ZOOKEEPER-SESSION-TIMEOUT)
+                             %)
+        zk (atom nil)
+        watcher (fn this [state type path]
+                    (when @active
+                      (when-not (= :connected state)
+                        (log-message "Zookeeper disconnected. Attempting to reconnect")
+                        (reset! zk (mk-zk this))
+                        )
+                      (when-not (= :none type)
+                        (doseq [callback (vals @callbacks)]                          
+                          (callback type path))))
+                      )]
+    (reset! zk (mk-zk watcher))
+    (reify
+     ClusterState
+     (register [this callback]
+               (let [id (uuid)]
+                 (swap! callbacks assoc id callback)
+                 id
+                 ))
+     (unregister [this id]
+                 (swap! callbacks dissoc id))
+     (set-ephemeral-node [this path data]
+                         (zk/mkdirs @zk (parent-path path))
+                         (if (zk/exists @zk path false)
+                           (zk/set-data @zk path data) ; should verify that it's ephemeral
+                           (zk/create-node @zk path data :ephemeral)
+                           ))
+     
+     (set-data [this path data]
+               ;; note: this does not turn off any existing watches
+               (if (zk/exists @zk path false)
+                 (zk/set-data @zk path data)
+                 (do
+                   (zk/mkdirs @zk (parent-path path))
+                   (zk/create-node @zk path data :persistent)
+                   )))
+     
+     (delete-node [this path]
+                  (zk/delete-recursive @zk path)
+                  )
+     
+     (get-data [this path watch?]
+               (zk/get-data @zk path watch?)
+               )
+     
+     (get-children [this path watch?]
+                   (zk/get-children @zk path watch?))
+     
+     (mkdirs [this path]
+             (zk/mkdirs @zk path))
+     
+     (close [this]
+            (reset! active false)
+            (.close @zk))
+     )))
+
+(defprotocol StormClusterState
+  (assignments [this callback])
+  (assignment-info [this storm-id callback])
+  (active-storms [this])
+  (storm-base [this storm-id callback])
+
+  (task-storms [this])
+  (task-ids [this storm-id])
+  (task-info [this storm-id task-id])
+  (task-heartbeat [this storm-id task-id]) ;; returns nil if doesn't exist
+  (supervisors [this callback])
+  (supervisor-info [this supervisor-id])  ;; returns nil if doesn't exist
+
+  (setup-heartbeats! [this storm-id])
+  (teardown-heartbeats! [this storm-id])
+  (teardown-task-errors! [this storm-id])
+  (heartbeat-storms [this])
+  (task-error-storms [this])
+  (heartbeat-tasks [this storm-id])
+
+  (set-task! [this storm-id task-id info])
+  (task-heartbeat! [this storm-id task-id info])
+  (remove-task-heartbeat! [this storm-id task-id])
+  (supervisor-heartbeat! [this supervisor-id info])
+  (activate-storm! [this storm-id storm-base])
+  (deactivate-storm! [this storm-id])
+  (set-assignment! [this storm-id info])
+  (remove-storm! [this storm-id])
+  (report-task-error [this storm-id task-id error])
+  (task-errors [this storm-id task-id])
+
+  (disconnect [this])
+  )
+
+
+(def ASSIGNMENTS-ROOT "assignments")
+(def TASKS-ROOT "tasks")
+(def CODE-ROOT "code")
+(def STORMS-ROOT "storms")
+(def SUPERVISORS-ROOT "supervisors")
+(def TASKBEATS-ROOT "taskbeats")
+(def TASKERRORS-ROOT "taskerrors")
+
+(def ASSIGNMENTS-SUBTREE (str "/" ASSIGNMENTS-ROOT))
+(def TASKS-SUBTREE (str "/" TASKS-ROOT))
+(def STORMS-SUBTREE (str "/" STORMS-ROOT))
+(def SUPERVISORS-SUBTREE (str "/" SUPERVISORS-ROOT))
+(def TASKBEATS-SUBTREE (str "/" TASKBEATS-ROOT))
+(def TASKERRORS-SUBTREE (str "/" TASKERRORS-ROOT))
+
+(defn supervisor-path [id]
+  (str SUPERVISORS-SUBTREE "/" id))
+
+(defn assignment-path [id]
+  (str ASSIGNMENTS-SUBTREE "/" id))
+
+(defn storm-path [id]
+  (str STORMS-SUBTREE "/" id))
+
+(defn storm-task-root [storm-id]
+  (str TASKS-SUBTREE "/" storm-id))
+
+(defn task-path [storm-id task-id]
+  (str (storm-task-root storm-id) "/" task-id))
+
+(defn taskbeat-storm-root [storm-id]
+  (str TASKBEATS-SUBTREE "/" storm-id))
+
+(defn taskbeat-path [storm-id task-id]
+  (str (taskbeat-storm-root storm-id) "/" task-id))
+
+(defn taskerror-storm-root [storm-id]
+  (str TASKERRORS-SUBTREE "/" storm-id))
+
+(defn taskerror-path [storm-id task-id]
+  (str (taskerror-storm-root storm-id) "/" task-id))
+
+
+(defn- issue-callback! [cb-atom]
+  (let [cb @cb-atom]
+    (reset! cb-atom nil)
+    (when cb
+      (cb))
+    ))
+
+(defn- issue-map-callback! [cb-atom id]
+  (let [cb (@cb-atom id)]
+    (swap! cb-atom dissoc id)
+    (when cb
+      (cb id))
+    ))
+
+(defn- maybe-deserialize [ser]
+  (when ser
+    (Utils/deserialize ser)))
+
+(defstruct TaskError :error :time-secs)
+
+(defn mk-storm-cluster-state [cluster-state-spec]
+  (let [[solo? cluster-state] (if (satisfies? ClusterState cluster-state-spec)
+                                [false cluster-state-spec]
+                                [true (mk-distributed-cluster-state cluster-state-spec)])
+        assignment-info-callback (atom {})
+        supervisors-callback (atom nil)
+        assignments-callback (atom nil)
+        storm-base-callback (atom {})
+        state-id (register
+                  cluster-state
+                  (fn [type path]
+                    (let [[subtree & args] (tokenize-path path)]
+                      (condp = subtree
+                          ASSIGNMENTS-ROOT (if (empty? args)
+                                             (issue-callback! assignments-callback)
+                                             (issue-map-callback! assignment-info-callback (first args)))
+                          SUPERVISORS-ROOT (issue-callback! supervisors-callback)
+                          STORMS-ROOT (issue-map-callback! storm-base-callback (first args))
+                          ;; this should never happen
+                          (halt-process! 30 "Unknown callback for subtree " subtree args)
+                          )
+                      )))]
+    (doseq [p [ASSIGNMENTS-SUBTREE TASKS-SUBTREE STORMS-SUBTREE SUPERVISORS-SUBTREE TASKBEATS-SUBTREE TASKERRORS-SUBTREE]]
+      (mkdirs cluster-state p))
+    (reify
+     StormClusterState
+     
+     (assignments [this callback]
+        (when callback
+          (reset! assignments-callback callback))
+        (get-children cluster-state ASSIGNMENTS-SUBTREE (not-nil? callback)))
+      
+      (assignment-info [this storm-id callback]
+        (when callback
+          (swap! assignment-info-callback assoc storm-id callback))
+        (maybe-deserialize (get-data cluster-state (assignment-path storm-id) (not-nil? callback)))
+        )
+
+      (active-storms [this]
+        (get-children cluster-state STORMS-SUBTREE false)
+        )
+
+      (heartbeat-storms [this]
+        (get-children cluster-state TASKBEATS-SUBTREE false)
+        )
+
+      (task-error-storms [this]
+         (get-children cluster-state TASKERRORS-SUBTREE false)
+         )
+      
+      (heartbeat-tasks [this storm-id]
+        (get-children cluster-state (taskbeat-storm-root storm-id) false)
+        )
+
+      (task-storms [this]
+        (get-children cluster-state TASKS-SUBTREE false)
+        )
+
+      (task-ids [this storm-id]
+        (map parse-int
+          (get-children cluster-state (storm-task-root storm-id) false)
+          ))
+
+      (task-info [this storm-id task-id]
+        (maybe-deserialize (get-data cluster-state (task-path storm-id task-id) false))
+        )
+
+      ;; TODO: add a callback here so that nimbus can respond immediately when it goes down? 
+      (task-heartbeat [this storm-id task-id]
+        (maybe-deserialize (get-data cluster-state (taskbeat-path storm-id task-id) false))
+        )
+
+      (supervisors [this callback]
+        (when callback
+          (reset! supervisors-callback callback))
+        (get-children cluster-state SUPERVISORS-SUBTREE (not-nil? callback))
+        )
+
+      (supervisor-info [this supervisor-id]
+        (maybe-deserialize (get-data cluster-state (supervisor-path supervisor-id) false))
+        )
+
+      (set-task! [this storm-id task-id info]
+        (set-data cluster-state (task-path storm-id task-id) (Utils/serialize info))
+        )
+
+      (task-heartbeat! [this storm-id task-id info]
+        (set-ephemeral-node cluster-state (taskbeat-path storm-id task-id) (Utils/serialize info))
+        )
+
+      (remove-task-heartbeat! [this storm-id task-id]
+        (delete-node cluster-state (taskbeat-path storm-id task-id))
+        )
+
+      (setup-heartbeats! [this storm-id]
+        (mkdirs cluster-state (taskbeat-storm-root storm-id)))
+
+      (teardown-heartbeats! [this storm-id]
+        (delete-node cluster-state (taskbeat-storm-root storm-id)))
+
+      (teardown-task-errors! [this storm-id]
+        (delete-node cluster-state (taskerror-storm-root storm-id)))
+
+      (supervisor-heartbeat! [this supervisor-id info]
+        (set-ephemeral-node cluster-state (supervisor-path supervisor-id) (Utils/serialize info))
+        )
+
+      (activate-storm! [this storm-id storm-base]
+        (set-data cluster-state (storm-path storm-id) (Utils/serialize storm-base))
+        )
+
+      (storm-base [this storm-id callback]
+        (when callback
+          (swap! storm-base-callback assoc storm-id callback))
+        (maybe-deserialize (get-data cluster-state (storm-path storm-id) (not-nil? callback)))
+        )
+
+      (deactivate-storm! [this storm-id]
+        (delete-node cluster-state (storm-path storm-id))
+        )
+
+      (set-assignment! [this storm-id info]
+        (set-data cluster-state (assignment-path storm-id) (Utils/serialize info))
+        )
+
+      (remove-storm! [this storm-id]
+        ;; rmr the task related info. must remove assignment last
+        (delete-node cluster-state (storm-task-root storm-id))
+        (delete-node cluster-state (assignment-path storm-id))
+        )
+
+      (report-task-error [this storm-id task-id error]
+                         (let [path (taskerror-path storm-id task-id)
+                               _ (mkdirs cluster-state path)
+                               children (get-children cluster-state path false)
+                               times (sort (map #(Integer/parseInt %) children))                               
+                               ]
+                           (if (>= (count times) 10)
+                             (delete-node cluster-state (str path "/" (first times)))
+                             )
+                           (set-data cluster-state
+                                      (str path "/" (current-time-secs))
+                                      (.getBytes ^String (stringify-error error)))
+                           ))
+
+      (task-errors [this storm-id task-id]
+                   (let [path (taskerror-path storm-id task-id)
+                         _ (mkdirs cluster-state path)
+                         children (get-children cluster-state path false)
+                         errors (dofor [c children]
+                                       (let [^bytes v (get-data cluster-state (str path "/" c) false)]
+                                         (when v
+                                           (struct TaskError (String. v) (Integer/parseInt c))
+                                           )))
+                         ]
+                     (->> (filter not-nil? errors)
+                          (sort-by :time-secs)
+                          )                     
+                     ))
+      
+      (disconnect [this]
+        (unregister cluster-state state-id)
+        (when solo?
+          (close cluster-state)))
+      )))
+
+;; daemons have a single thread that will respond to events
+;; start with initialize event
+;; callbacks add events to the thread's queue
+
+;; keeps in memory cache of the state, only for what client subscribes to. Any subscription is automatically kept in sync, and when there are changes, client is notified.
+;; master gives orders through state, and client records status in state (ephemerally)
+
+;; master tells nodes what workers to launch
+
+;; master writes this. supervisors and workers subscribe to this to understand complete topology. each storm is a map from nodes to workers to tasks to ports whenever topology changes everyone will be notified
+;; master includes timestamp of each assignment so that appropriate time can be given to each worker to start up
+;; /assignments/{storm id}
+
+;; which tasks they talk to, etc. (immutable until shutdown)
+;; everyone reads this in full to understand structure
+;; /tasks/{storm id}/{task id} ; just contains bolt id
+
+
+;; supervisors send heartbeats here, master doesn't subscribe but checks asynchronously
+;; /supervisors/status/{ephemeral node ids}  ;; node metadata such as port ranges are kept here 
+
+;; tasks send heartbeats here, master doesn't subscribe, just checks asynchronously
+;; /taskbeats/{storm id}/{ephemeral task id}
+
+;; contains data about whether it's started or not, tasks and workers subscribe to specific storm here to know when to shutdown
+;; master manipulates
+;; /storms/{storm id}
+
+
+
+;; Zookeeper flows:
+
+;; Master:
+;; job submit:
+;; 1. read which nodes are available
+;; 2. set up the worker/{storm}/{task} stuff (static)
+;; 3. set assignments
+;; 4. start storm - necessary in case master goes down, when goes back up can remember to take down the storm (2 states: on or off)
+
+;; Monitoring (or by checking when nodes go down or heartbeats aren't received):
+;; 1. read assignment
+;; 2. see which tasks/nodes are up
+;; 3. make new assignment to fix any problems
+;; 4. if a storm exists but is not taken down fully, ensure that storm takedown is launched (step by step remove tasks and finally remove assignments)
+
+
+;; masters only possible watches is on ephemeral nodes and tasks, and maybe not even
+
+;; Supervisor:
+;; 1. monitor /storms/* and assignments
+;; 2. local state about which workers are local
+;; 3. when storm is on, check that workers are running locally & start/kill if different than assignments
+;; 4. when storm is off, monitor tasks for workers - when they all die or don't hearbeat, kill the process and cleanup
+
+;; Worker:
+;; 1. On startup, start the tasks if the storm is on
+
+;; Task:
+;; 1. monitor assignments, reroute when assignments change
+;; 2. monitor storm (when storm turns off, error if assignments change) - take down tasks as master turns them off
+
+
+
+;; locally on supervisor: workers write pids locally on startup, supervisor deletes it on shutdown (associates pid with worker name)
+;; supervisor periodically checks to make sure processes are alive
+;; {rootdir}/workers/{storm id}/{worker id}   ;; contains pid inside
+
+;; all tasks in a worker share the same cluster state
+;; workers, supervisors, and tasks subscribes to storm to know when it's started or stopped
+;; on stopped, master removes records in order (tasks need to subscribe to themselves to see if they disappear)
+;; when a master removes a worker, the supervisor should kill it (and escalate to kill -9)
+;; on shutdown, tasks subscribe to tasks that send data to them to wait for them to die. when node disappears, they can die
diff --git a/src/clj/backtype/storm/command/config_value.clj b/src/clj/backtype/storm/command/config_value.clj
new file mode 100644
index 0000000..113a641
--- /dev/null
+++ b/src/clj/backtype/storm/command/config_value.clj
@@ -0,0 +1,9 @@
+(ns backtype.storm.command.config-value
+  (:use [backtype.storm config log])
+  (:gen-class))
+
+
+(defn -main [^String name]
+  (let [conf (read-storm-config)]
+    (println "VALUE:" (conf name))
+    ))
diff --git a/src/clj/backtype/storm/command/kill_topology.clj b/src/clj/backtype/storm/command/kill_topology.clj
new file mode 100644
index 0000000..a0d0a11
--- /dev/null
+++ b/src/clj/backtype/storm/command/kill_topology.clj
@@ -0,0 +1,13 @@
+(ns backtype.storm.command.kill-topology
+  (:use [backtype.storm thrift config log])
+  (:gen-class))
+
+
+(defn -main [^String name]
+  (let [conf (read-storm-config)
+        host (conf NIMBUS-HOST)
+        port (conf NIMBUS-THRIFT-PORT)]
+    (with-nimbus-connection [nimbus host port]
+      (.killTopology nimbus name)
+      (log-message "Killed storm: " name)
+      )))
diff --git a/src/clj/backtype/storm/command/shell_submission.clj b/src/clj/backtype/storm/command/shell_submission.clj
new file mode 100644
index 0000000..4a3a759
--- /dev/null
+++ b/src/clj/backtype/storm/command/shell_submission.clj
@@ -0,0 +1,15 @@
+(ns backtype.storm.command.shell-submission
+  (:import [backtype.storm StormSubmitter])
+  (:use [backtype.storm thrift util config log])
+  (:require [clojure.string :as str])
+  (:gen-class))
+
+
+(defn -main [^String tmpjarpath & args]
+  (let [conf (read-storm-config)
+        host (conf NIMBUS-HOST)
+        port (conf NIMBUS-THRIFT-PORT)
+        jarpath (StormSubmitter/submitJar conf tmpjarpath)
+        args (concat args [host port jarpath])]
+    (exec-command! (str/join " " args))
+    ))
diff --git a/src/clj/backtype/storm/config.clj b/src/clj/backtype/storm/config.clj
new file mode 100644
index 0000000..ba43a14
--- /dev/null
+++ b/src/clj/backtype/storm/config.clj
@@ -0,0 +1,139 @@
+(ns backtype.storm.config
+  (:import [org.jvyaml YAML])
+  (:import [java.io FileReader File])
+  (:import [backtype.storm Config])
+  (:import [backtype.storm.utils Utils LocalState])
+  (:import [org.apache.commons.io FileUtils])
+  (:require [clojure.contrib [str-utils2 :as str]])
+  (:use [backtype.storm util])
+  )
+
+(def RESOURCES-SUBDIR "resources")
+
+;; define clojure constants for every configuration parameter
+(doseq [f (seq (.getFields Config))]
+  (let [name (.getName f)
+        new-name (.replace (.toUpperCase name) "_" "-")]
+    (eval
+      `(def ~(symbol new-name) (. Config ~(symbol name))))
+      ))
+
+(defn cluster-mode [conf & args]
+  (keyword (conf STORM-CLUSTER-MODE)))
+
+(defn local-mode? [conf]
+  (let [mode (conf STORM-CLUSTER-MODE)]
+    (condp = mode
+      "local" true
+      "distributed" false
+      (throw (IllegalArgumentException.
+                (str "Illegal cluster mode in conf: " mode)))
+      )))
+
+(defn sampling-rate [conf]
+  (->> (conf TOPOLOGY-STATS-SAMPLE-RATE)
+       (/ 1)
+       int))
+
+(defn mk-stats-sampler [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 mk-zk-connect-string [conf]
+  (let [servers (conf STORM-ZOOKEEPER-SERVERS)
+        port (conf STORM-ZOOKEEPER-PORT)
+        root (conf STORM-ZOOKEEPER-ROOT)]
+    (str
+      (str/join ","
+        (for [s servers]
+          (str s ":" port)))
+      root)
+    ))
+
+(defn read-default-config []
+  (clojurify-structure (Utils/readDefaultConfig)))
+
+(defn read-storm-config []
+  (clojurify-structure (Utils/readStormConfig)))
+
+(defn read-yaml-config [name]
+  (clojurify-structure (Utils/findAndReadConfigFile name true)))
+
+(defn master-stormdist-root [conf storm-id]
+  (str (conf STORM-LOCAL-DIR) "/stormdist/" storm-id))
+
+(defn master-stormjar-path [stormroot]
+  (str stormroot "/stormjar.jar"))
+
+(defn master-stormcode-path [stormroot]
+  (str stormroot "/stormcode.ser"))
+
+(defn master-stormconf-path [stormroot]
+  (str stormroot "/stormconf.ser"))
+
+(defn master-inbox [conf]
+  (let [ret (str (conf STORM-LOCAL-DIR) "/inbox")]
+    (FileUtils/forceMkdir (File. ret))
+    ret ))
+
+(defn supervisor-stormdist-root
+  ([conf] (str (conf STORM-LOCAL-DIR) "/stormdist"))
+  ([conf storm-id]
+      (str (supervisor-stormdist-root conf) "/" storm-id)))
+
+(defn supervisor-stormjar-path [stormroot]
+  (str stormroot "/stormjar.jar"))
+
+(defn supervisor-stormcode-path [stormroot]
+  (str stormroot "/stormcode.ser"))
+
+(defn supervisor-stormconf-path [stormroot]
+  (str stormroot "/stormconf.ser"))
+
+(defn supervisor-tmp-dir [conf]
+  (let [ret (str (conf STORM-LOCAL-DIR) "/tmp")]
+    (FileUtils/forceMkdir (File. ret))
+    ret ))
+
+(defn supervisor-storm-resources-path [stormroot]
+  (str stormroot "/" RESOURCES-SUBDIR))
+
+(defn ^LocalState supervisor-state [conf]
+  (LocalState. (str (conf STORM-LOCAL-DIR) "/localstate")))
+
+
+(defn worker-root
+  ([conf]
+     (str (conf STORM-LOCAL-DIR) "/workers"))
+  ([conf id]
+     (str (worker-root conf) "/" id)))
+
+(defn worker-pids-root
+  [conf id]
+  (str (worker-root conf id) "/pids"))
+
+(defn worker-pid-path [conf id pid]
+  (str (worker-pids-root conf id) "/" pid))
+
+(defn worker-heartbeats-root
+  [conf id]
+  (str (worker-root conf id) "/heartbeats"))
+
+;; workers heartbeat here with pid and timestamp
+;; if supervisor stops receiving heartbeat, it kills and restarts the process
+;; in local mode, keep a global map of ids to threads for simulating process management
+(defn ^LocalState worker-state  [conf id]
+  (LocalState. (worker-heartbeats-root conf id)))
diff --git a/src/clj/backtype/storm/daemon/common.clj b/src/clj/backtype/storm/daemon/common.clj
new file mode 100644
index 0000000..06de2b9
--- /dev/null
+++ b/src/clj/backtype/storm/daemon/common.clj
@@ -0,0 +1,96 @@
+(ns backtype.storm.daemon.common
+  (:use [clojure.contrib.seq-utils :only [find-first]])
+  (:use [backtype.storm log config util])
+  )
+
+(def ACKER-COMPONENT-ID -1)
+(def ACKER-INIT-STREAM-ID -1)
+(def ACKER-ACK-STREAM-ID -2)
+(def ACKER-FAIL-STREAM-ID -3)
+
+
+(defn system-component? [id]
+  (< id 0))
+
+;; 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 task->node+port task->start-time-secs])
+
+(defrecord StormBase [storm-name launch-time-secs])
+
+(defrecord SupervisorInfo [time-secs hostname worker-ports uptime-secs])
+
+(defrecord TaskInfo [component-id])
+
+(defprotocol DaemonCommon
+  (waiting? [this]))
+
+(def LS-WORKER-HEARTBEAT "worker-heartbeat")
+
+;; LocalState constants
+(def LS-ID "supervisor-id")
+(def LS-LOCAL-ASSIGNMENTS "local-assignments")
+(def LS-APPROVED-WORKERS "approved-workers")
+
+
+
+(defrecord WorkerHeartbeat [time-secs storm-id task-ids port])
+
+;; should include stats in here
+;; TODO: want to know how many it has processed from every source
+;; component/stream pair
+;; TODO: want to know how many it has emitted to every stream
+(defrecord TaskStats [^long processed
+                      ^long acked
+                      ^long emitted
+                      ^long transferred
+                      ^long failed])
+
+(defrecord TaskHeartbeat [time-secs uptime-secs stats])
+
+(defn new-task-stats []
+  (TaskStats. 0 0 0 0 0))
+
+;technically this is only active task ids
+(defn storm-task-ids [storm-cluster-state storm-id]
+  (keys (:task->node+port (.assignment-info storm-cluster-state storm-id nil))))
+
+(defn storm-task-info
+  "Returns map from task -> component id"
+  [storm-cluster-state storm-id]
+  (let [task-ids (.task-ids storm-cluster-state storm-id)]
+    (into {}
+      (dofor [id task-ids]
+        [id (:component-id (.task-info storm-cluster-state storm-id id))]
+        ))))
+
+(defn get-storm-id [storm-cluster-state storm-name]
+  (let [active-storms (.active-storms storm-cluster-state)]
+    (find-first
+      #(= storm-name (:storm-name (.storm-base storm-cluster-state % nil)))
+      active-storms)
+    ))
+
+(defn topology-bases [storm-cluster-state]
+  (let [active-topologies (.active-storms storm-cluster-state)]
+    (into {}
+          (dofor [id active-topologies]
+                 [id (.storm-base storm-cluster-state id nil)]
+                 ))
+    ))
+
+(defn validate-distributed-mode! [conf]
+  (if (local-mode? conf)
+      (throw
+        (IllegalArgumentException. "Cannot start server in local mode!"))))
+
+(defmacro defserverfn [name & body]
+  `(let [exec-fn# (fn ~@body)]
+    (defn ~name [& args#]
+      (try
+        (apply exec-fn# args#)
+      (catch Throwable t#
+        (log-error t# "Error on initialization of server " ~(str name))
+        (halt-process! 13 "Error on initialization")
+        )))))
diff --git a/src/clj/backtype/storm/daemon/drpc.clj b/src/clj/backtype/storm/daemon/drpc.clj
new file mode 100644
index 0000000..17dfded
--- /dev/null
+++ b/src/clj/backtype/storm/daemon/drpc.clj
@@ -0,0 +1,81 @@
+(ns backtype.storm.daemon.drpc
+  (:import [org.apache.thrift.server THsHaServer THsHaServer$Options])
+  (:import [org.apache.thrift.protocol TBinaryProtocol TBinaryProtocol$Factory])
+  (:import [org.apache.thrift TException])
+  (:import [org.apache.thrift.transport TNonblockingServerTransport TNonblockingServerSocket])
+  (:import [backtype.storm.generated DistributedRPC DistributedRPC$Iface DistributedRPC$Processor])
+  (:import [java.util.concurrent Semaphore])
+  (:import [backtype.storm.drpc SpoutAdder])
+  (:import [java.net InetAddress])
+  (:use [backtype.storm bootstrap])
+  (:gen-class))
+
+(bootstrap)
+
+
+(def DEFAULT-PORT 3772)  ; "drpc"
+(def REQUEST-TIMEOUT-SECS 600)
+(def TIMEOUT-CHECK-SECS 60)
+
+;; TODO: change this to use TimeCacheMap
+(defn service-handler [^SpoutAdder spout-adder port]
+  (let [ctr (atom 0)
+        id->sem (atom {})
+        id->result (atom {})
+        id->start (atom {})
+        cleanup (fn [id] (swap! id->sem dissoc id)
+                         (swap! id->result dissoc id)
+                         (swap! id->start dissoc id))
+        my-ip (.getHostAddress (InetAddress/getLocalHost))
+        ]
+    (async-loop
+      (fn []
+        (doseq [[id start] @id->start]
+          (when (> (time-delta start) REQUEST-TIMEOUT-SECS)
+            (if-let [sem (@id->sem id)]
+              (.release sem))
+            (cleanup id)
+            ))
+        TIMEOUT-CHECK-SECS
+        ))
+    (reify DistributedRPC$Iface
+      (^String execute [this ^String function ^String args]
+        (let [id (str (swap! ctr (fn [v] (mod (inc v) 1000000000))))
+              ^Semaphore sem (Semaphore. 0)
+              return-info (to-json {"ip" my-ip "port" port "id" id})
+              ]
+          (swap! id->start assoc id (current-time-secs))
+          (swap! id->sem assoc id sem)
+          (.add spout-adder function args return-info)
+          (.acquire sem)
+          (let [result (@id->result id)]
+            (cleanup id)
+            result
+            )))
+      (^void result [this ^String id ^String result]
+        (let [^Semaphore sem (@id->sem id)]
+          (when sem
+            (swap! id->result assoc id result)
+            (.release sem)
+            )))
+      )))
+
+(defn launch-server!
+  ([spout-adder]
+    (launch-server! DEFAULT-PORT spout-adder))
+  ([port spout-adder]
+    (let [options (THsHaServer$Options.)
+         _ (set! (. options maxWorkerThreads) 64)
+         service-handler (service-handler spout-adder port)
+         server (THsHaServer.
+                 (DistributedRPC$Processor. service-handler)
+                 (TNonblockingServerSocket. port)
+                 (TBinaryProtocol$Factory.) options)]
+      (.addShutdownHook (Runtime/getRuntime) (Thread. (fn [] (.stop server))))
+      (log-message "Starting Distributed RPC server...")
+      (.serve server))))
+
+(defn -main [spout-adder-class & args]
+  (let [form (concat ['new (symbol spout-adder-class)] args)]
+    (launch-server! (eval form))
+    ))
diff --git a/src/clj/backtype/storm/daemon/nimbus.clj b/src/clj/backtype/storm/daemon/nimbus.clj
new file mode 100644
index 0000000..5bd4392
--- /dev/null
+++ b/src/clj/backtype/storm/daemon/nimbus.clj
@@ -0,0 +1,638 @@
+(ns backtype.storm.daemon.nimbus
+  (:import [org.apache.thrift.server THsHaServer THsHaServer$Options])
+  (:import [org.apache.thrift.protocol TBinaryProtocol TBinaryProtocol$Factory])
+  (:import [org.apache.thrift TException])
+  (:import [org.apache.thrift.transport TNonblockingServerTransport TNonblockingServerSocket])
+  (:use [backtype.storm bootstrap])
+  (:use [backtype.storm.daemon common])
+  (:gen-class))
+
+(bootstrap)
+
+(defmulti setup-jar cluster-mode)
+
+
+;; Master:
+;; job submit:
+;; 1. read which nodes are available
+;; 2. set up the worker/{storm}/{task} stuff (static)
+;; 3. set assignments
+;; 4. start storm - necessary in case master goes down, when goes back up can remember to take down the storm (2 states: on or off)
+
+;; Monitoring (or by checking when nodes go down or heartbeats aren't received):
+;; 1. read assignment
+;; 2. see which tasks/nodes are up
+;; 3. make new assignment to fix any problems
+;; 4. if a storm exists but is not taken down fully, ensure that storm takedown is launched (step by step remove tasks and finally remove assignments)
+
+(defn- assigned-slots
+  "Returns a map from node-id to a set of ports"
+  [storm-cluster-state]
+  (let [assignments (.assignments storm-cluster-state nil)
+        ]
+    (defaulted
+      (apply merge-with set/union
+             (for [a assignments
+                   [_ [node port]] (-> (.assignment-info storm-cluster-state a nil) :task->node+port)]
+               {node #{port}}
+               ))
+      {})
+    ))
+
+(defn- all-supervisor-info
+  ([storm-cluster-state] (all-supervisor-info storm-cluster-state nil))
+  ([storm-cluster-state callback]
+     (let [supervisor-ids (.supervisors storm-cluster-state callback)]
+       (into {}
+             (mapcat
+              (fn [id]
+                (if-let [info (.supervisor-info storm-cluster-state id)]
+                  [[id info]]
+                  ))
+              supervisor-ids))
+       )))
+
+(defn- available-slots
+  [conf storm-cluster-state callback]
+  (let [supervisor-ids (.supervisors storm-cluster-state callback)
+        supervisor-infos (all-supervisor-info storm-cluster-state callback)
+        ;; TODO: this is broken. need to maintain a map since last time
+        ;; supervisor hearbeats like is done for tasks
+        ;; maybe it's ok to trust ephemeral nodes here?
+        ;;[[id info]]
+        ;; (when (< (time-delta (:time-secs info))
+        ;;          (conf NIMBUS-SUPERVISOR-TIMEOUT-SECS))
+        ;;   [[id info]]
+        ;;   )        
+        all-slots (map-val (comp set :worker-ports) supervisor-infos)
+        existing-slots (assigned-slots storm-cluster-state)
+        ]
+    [(map-val :hostname supervisor-infos)
+     (mapcat
+       (fn [[id slots]]
+         (for [s (set/difference slots (existing-slots id))]
+           [id s]))
+       all-slots)
+      ]))
+
+(defn state-spout-parallelism [state-spout-spec]
+  (-> state-spout-spec .get_common thrift/parallelism-hint))
+
+(defn- spout-parallelism [spout-spec]
+  (if (.is_distributed spout-spec)
+    (-> spout-spec .get_common thrift/parallelism-hint)
+    1 ))
+
+(defn bolt-parallelism [bolt-spec]
+  (let [hint (-> bolt-spec .get_common thrift/parallelism-hint)
+        fully-global? (every?
+                       thrift/global-grouping?
+                       (vals (.get_inputs bolt-spec)))]
+    (if fully-global?
+      1
+      hint
+      )))
+
+(defn- optimize-topology [topology]
+  ;; TODO: create new topology by collapsing bolts into CompoundSpout
+  ;; and CompoundBolt
+  ;; need to somehow maintain stream/component ids inside tuples
+  topology)
+
+(defn mk-task-maker [max-parallelism parallelism-func id-counter]
+  (fn [[component-id spec]]
+    (let [parallelism (parallelism-func spec)
+          parallelism (if max-parallelism (min parallelism max-parallelism) parallelism)
+          num-tasks (max 1 parallelism)]
+      (for-times num-tasks
+                 [(id-counter) component-id])
+      )))
+
+(defn- setup-storm-code [conf storm-id tmp-jar-location storm-conf topology]
+  (let [stormroot (master-stormdist-root conf storm-id)]
+   (FileUtils/forceMkdir (File. stormroot))
+   (FileUtils/cleanDirectory (File. stormroot))
+   (setup-jar conf tmp-jar-location stormroot)
+   (FileUtils/writeByteArrayToFile (File. (master-stormcode-path stormroot)) (Utils/serialize topology))
+   (FileUtils/writeByteArrayToFile (File. (master-stormconf-path stormroot)) (Utils/serialize storm-conf))
+   ))
+
+
+(defn- read-storm-conf [conf storm-id]
+  (let [stormroot (master-stormdist-root conf storm-id)]
+    (merge conf
+           (Utils/deserialize
+            (FileUtils/readFileToByteArray
+             (File. (master-stormconf-path stormroot))
+             )))))
+
+(defn- read-storm-topology [conf storm-id]
+  (let [stormroot (master-stormdist-root conf storm-id)]
+    (Utils/deserialize
+      (FileUtils/readFileToByteArray
+        (File. (master-stormcode-path stormroot))
+        ))))
+
+
+(defn max-message-timeout-time [conf storm-ids]
+  (apply max
+    (for [id storm-ids]
+      ((read-storm-conf conf id) TOPOLOGY-MESSAGE-TIMEOUT-SECS)
+      )))
+
+
+(defn task-dead? [conf storm-cluster-state storm-id task-id]
+  (let [info (.task-heartbeat storm-cluster-state storm-id task-id)]
+    (or (not info)
+        (> (time-delta (:time-secs info))
+           (conf NIMBUS-TASK-TIMEOUT-SECS)))
+    ))
+
+;; public so it can be mocked in tests
+(defn mk-task-component-assignments [conf storm-id]
+  (let [storm-conf (read-storm-conf conf storm-id)
+        max-parallelism (storm-conf TOPOLOGY-MAX-TASK-PARALLELISM)
+        topology (read-storm-topology conf storm-id)
+        slots-to-use (storm-conf TOPOLOGY-WORKERS)
+        counter (mk-counter)
+        tasks (concat
+               (mapcat (mk-task-maker max-parallelism bolt-parallelism counter)
+                       (.get_bolts topology))
+               (mapcat (mk-task-maker max-parallelism spout-parallelism counter)
+                       (.get_spouts topology))
+               (mapcat (mk-task-maker max-parallelism state-spout-parallelism counter)
+                       (.get_state_spouts topology))
+               (repeatedly (storm-conf TOPOLOGY-ACKERS)
+                           (fn [] [(counter) ACKER-COMPONENT-ID]))
+               )]
+    (into {}
+      tasks)
+    ))
+
+(defn- setup-storm-static [conf storm-id storm-cluster-state]
+  (doseq [[task-id component-id] (mk-task-component-assignments conf storm-id)]
+    (.set-task! storm-cluster-state storm-id task-id (TaskInfo. component-id))
+    ))
+
+
+;; Does not assume that clocks are synchronized. Task heartbeat is only used so that
+;; nimbus knows when it's received a new heartbeat. All timing is done by nimbus and
+;; tracked through task-heartbeat-cache
+(defn- alive-tasks [conf storm-id storm-cluster-state task-ids task-start-times task-heartbeats-cache]
+  (doall
+    (filter
+      (fn [task-id]
+        (let [heartbeat (.task-heartbeat storm-cluster-state storm-id task-id)
+              reported-time (:time-secs heartbeat)
+              {last-nimbus-time :nimbus-time
+               last-reported-time :task-reported-time} (get-in @task-heartbeats-cache
+                                                               [storm-id task-id])
+              task-start-time (get task-start-times task-id)
+              nimbus-time (if (or (not last-nimbus-time)
+                                  (not= last-reported-time reported-time))
+                            (current-time-secs)
+                            last-nimbus-time
+                            )
+              ]          
+          (swap! task-heartbeats-cache
+                 assoc-in [storm-id task-id]
+                 {:nimbus-time nimbus-time
+                  :task-reported-time reported-time})
+          (if (and task-start-time
+                   (or
+                    (< (time-delta task-start-time)
+                       (conf NIMBUS-TASK-LAUNCH-SECS))
+                    (not nimbus-time)
+                    (< (time-delta nimbus-time)
+                       (conf NIMBUS-TASK-TIMEOUT-SECS))
+                    ))
+            true
+            (do
+              (log-message "Task " storm-id ":" task-id " timed out")
+              false)
+            )))
+      task-ids
+      )))
+
+(defn- keeper-slots [existing-slots num-task-ids num-workers]
+  (if (= 0 num-workers)
+    {}
+    (let [distribution (atom (integer-divided num-task-ids num-workers))
+          keepers (atom {})]
+      (doseq [[node+port task-list] existing-slots :let [task-count (count task-list)]]
+        (when (pos? (get @distribution task-count 0))
+          (swap! keepers assoc node+port task-list)
+          (swap! distribution update-in [task-count] dec)
+          ))
+      @keepers
+      )))
+
+
+(defn sort-slots [all-slots]
+  (let [split-up (vals (group-by first all-slots))]
+    (apply interleave-all split-up)
+    ))
+
+;; NEW NOTES
+;; only assign to supervisors who are there and haven't timed out
+;; need to reassign workers with tasks that have timed out (will this make it brittle?)
+;; need to read in the topology and storm-conf from disk
+;; if no slots available and no slots used by this storm, just skip and do nothing
+;; otherwise, package rest of tasks into available slots (up to how much it needs)
+
+;; in the future could allocate tasks intelligently (so that "close" tasks reside on same machine)
+
+
+;; TODO: slots that have dead task should be reused as long as supervisor is active
+
+;; public so it can be mocked out
+(defn compute-new-task->node+port [conf storm-id existing-assignment storm-cluster-state available-slots callback task-heartbeats-cache]
+  (let [existing-assigned (reverse-map (:task->node+port existing-assignment))
+        storm-conf (read-storm-conf conf storm-id)
+        all-task-ids (set (.task-ids storm-cluster-state storm-id))
+        alive-ids (set (alive-tasks conf storm-id storm-cluster-state
+                                    all-task-ids (:task->start-time-secs existing-assignment) task-heartbeats-cache))
+        alive-assigned (filter-val (partial every? alive-ids) existing-assigned)
+        alive-node-ids (map first (keys alive-assigned))
+        total-slots-to-use (min (storm-conf TOPOLOGY-WORKERS)
+                                (+ (count available-slots) (count alive-assigned)))
+        keep-assigned (keeper-slots alive-assigned (count all-task-ids) total-slots-to-use)
+        freed-slots (keys (apply dissoc alive-assigned (keys keep-assigned)))
+        reassign-slots (take (- total-slots-to-use (count keep-assigned))
+                             (sort-slots (concat available-slots freed-slots)))
+        reassign-ids (sort (set/difference all-task-ids (set (apply concat (vals keep-assigned)))))
+        reassignment (into {}
+                           (map vector
+                                reassign-ids
+                                ;; for some reason it goes into infinite loop without limiting the repeat-seq
+                                (repeat-seq (count reassign-ids) reassign-slots)))
+        stay-assignment (into {} (mapcat (fn [[node+port task-ids]] (for [id task-ids] [id node+port])) keep-assigned))]
+    (when-not (empty? reassignment)
+      (log-message "Reassigning " storm-id " to " total-slots-to-use " slots")
+      (log-message "Reassign ids: " (vec reassign-ids))
+      (log-message "Available slots: " (pr-str available-slots))
+      )
+    (merge stay-assignment reassignment)
+    ))
+
+
+(defn changed-ids [task->node+port new-task->node+port]
+  (let [slot-assigned (reverse-map task->node+port)
+        new-slot-assigned (reverse-map new-task->node+port)
+        brand-new-slots (map-diff slot-assigned new-slot-assigned)]
+    (apply concat (vals brand-new-slots))
+    ))
+
+;; get existing assignment (just the task->node+port map) -> default to {}
+;; filter out ones which have a task timeout
+;; figure out available slots on cluster. add to that the used valid slots to get total slots. figure out how many tasks should be in each slot (e.g., 4, 4, 4, 5)
+;; only keep existing slots that satisfy one of those slots. for rest, reassign them across remaining slots
+;; edge case for slots with no task timeout but with supervisor timeout... just treat these as valid slots that can be reassigned to. worst comes to worse the task will timeout and won't assign here next time around
+(defn- mk-assignments [conf storm-id storm-cluster-state callback task-heartbeats-cache]
+  (log-debug "Determining assignment for " storm-id)
+  (let [existing-assignment (.assignment-info storm-cluster-state storm-id nil)
+        [node->host available-slots] (available-slots conf storm-cluster-state callback)
+        task->node+port (compute-new-task->node+port conf storm-id existing-assignment
+                          storm-cluster-state available-slots callback task-heartbeats-cache)
+        all-node->host (merge (:node->host existing-assignment) node->host)
+        reassign-ids (changed-ids (:task->node+port existing-assignment) task->node+port)
+        now-secs (current-time-secs)
+        start-times (merge (:task->start-time-secs existing-assignment)
+                           (into {}
+                             (for [id reassign-ids]
+                               [id now-secs]
+                               )))
+
+        assignment (Assignment.
+                    (master-stormdist-root conf storm-id)
+                    (select-keys all-node->host (map first (vals task->node+port)))
+                    task->node+port
+                    start-times
+                    )
+        ]
+    ;; tasks figure out what tasks to talk to by looking at topology at runtime
+    ;; only log/set when there's been a change to the assignment
+    (if (= existing-assignment assignment)
+      (log-debug "Assignment for " storm-id " hasn't changed")
+      (do
+        (log-message "Setting new assignment for storm id " storm-id ": " (pr-str assignment))
+        (.set-assignment! storm-cluster-state storm-id assignment)
+        ))
+    ))
+
+(defn- start-storm [storm-name storm-cluster-state storm-id]
+  (log-message "Activating " storm-name ": " storm-id)
+  (.activate-storm! storm-cluster-state storm-id (StormBase. storm-name (current-time-secs)))
+  )
+
+;; Master:
+;; job submit:
+;; 1. read which nodes are available
+;; 2. set up the worker/{storm}/{task} stuff (static)
+;; 3. set assignments
+;; 4. start storm - necessary in case master goes down, when goes back up can remember to take down the storm (2 states: on or off)
+
+(defn storm-active? [storm-cluster-state storm-name]
+  (not-nil? (get-storm-id storm-cluster-state storm-name)))
+
+(defn inactive-storm-ids [storm-cluster-state]
+  (let [assigned-ids (set (.assignments storm-cluster-state nil))
+        active-ids (set (.active-storms storm-cluster-state))]
+        (set/difference assigned-ids active-ids)
+        ))
+
+(defn cleanup-storm-ids [conf storm-cluster-state]
+  (let [heartbeat-ids (set (.heartbeat-storms storm-cluster-state))
+        error-ids (set (.task-error-storms storm-cluster-state))
+        assigned-ids (set (.assignments storm-cluster-state nil))
+        storm-ids (set/difference (set/union heartbeat-ids error-ids) assigned-ids)]
+    (filter
+      (fn [storm-id]
+        (every?
+          (partial task-dead? conf storm-cluster-state storm-id)
+          (.heartbeat-tasks storm-cluster-state storm-id)
+          ))
+      storm-ids
+      )))
+
+(defn validate-topology! [topology]
+  (let [bolt-ids (keys (.get_bolts topology))
+        spout-ids (keys (.get_spouts topology))
+        state-spout-ids (keys (.get_state_spouts topology))
+        common (any-intersection bolt-ids spout-ids state-spout-ids)]
+    (when-not (empty? common)
+      (throw
+       (InvalidTopologyException.
+        (str "Cannot use same component id for both spout and bolt: " (vec common))
+        )))
+    (when-not (every? #(> % 0) (concat bolt-ids spout-ids state-spout-ids))
+      (throw
+       (InvalidTopologyException.
+        "All component ids must be positive")))
+    ;; TODO: validate that every declared stream is positive
+    ))
+
+(defn file-cache-map [conf]
+  (TimeCacheMap.
+   (int (conf NIMBUS-FILE-COPY-EXPIRATION-SECS))
+   (reify TimeCacheMap$ExpiredCallback
+          (expire [this id stream]
+                  (.close stream)
+                  ))
+   ))
+
+(defserverfn service-handler [conf]
+  (let [submitted-count (atom 0)
+        active (atom true)
+        conf (merge (read-storm-config) conf) ;; useful when testing
+        storm-cluster-state (cluster/mk-storm-cluster-state conf)
+        [event-manager cleanup-manager :as managers] [(event/event-manager false) (event/event-manager false)]
+        inbox (master-inbox conf)
+        storm-submit-lock (Object.)
+        task-heartbeats-cache (atom {}) ; map from storm id -> task id -> {:nimbus-time :task-reported-time}
+        downloaders (file-cache-map conf)
+        uploaders (file-cache-map conf)
+        uptime (uptime-computer)
+        
+        cleanup-fn (fn []
+                      (let [to-kill-ids (locking storm-submit-lock (inactive-storm-ids storm-cluster-state))]
+                        (when-not (empty? to-kill-ids)
+                          (let [sleep-amt (max-message-timeout-time conf to-kill-ids)]
+                            (log-message "Waiting for " sleep-amt " seconds to kill topologies " (pr-str to-kill-ids))
+                            ;; sleep to let the storm finish processing whatever messages are still inside it
+                            (sleep-secs sleep-amt)
+                            (doseq [id to-kill-ids]
+                              ;; technically a supervisor could still think there's an assignment and try to d/l
+                              ;; this will cause supervisor to go down and come back up... eventually it should sync
+                              ;; TODO: removing code locally should be done separately (since topology that doesn't start will still have code)
+                              (rmr (master-stormdist-root conf id))
+                              (.remove-storm! storm-cluster-state id))
+                            (log-message "Killed topologies: " to-kill-ids))))
+                      (let [to-cleanup-ids (locking storm-submit-lock (cleanup-storm-ids conf storm-cluster-state))]
+                        (when-not (empty? to-cleanup-ids)
+                          (doseq [id to-cleanup-ids]
+                            (.teardown-heartbeats! storm-cluster-state id)
+                            (.teardown-task-errors! storm-cluster-state id)
+                            (swap! task-heartbeats-cache dissoc id)
+                            )
+                          (log-message "Cleaned up topology task heartbeats: " (pr-str to-cleanup-ids))
+                          )))
+        reassign-fn (fn this []
+                      (when (conf NIMBUS-REASSIGN)
+                        (locking storm-submit-lock
+                          (let [callback (fn [& ignored] (.add event-manager this))
+                                active-storm-ids (.active-storms storm-cluster-state)]
+                            (doseq [storm-id active-storm-ids]
+                              (let [base (.storm-base storm-cluster-state storm-id nil)]
+                                (mk-assignments conf storm-id storm-cluster-state callback task-heartbeats-cache)))
+                              ))))
+        threads [(async-loop
+                   (fn []
+                     (.add event-manager reassign-fn)
+                     (.add cleanup-manager cleanup-fn)
+                     (when @active (conf NIMBUS-MONITOR-FREQ-SECS))
+                     ))
+                   ]]
+
+    (reify Nimbus$Iface
+      (^void submitTopology
+             [this ^String storm-name ^String uploadedJarLocation ^String serializedConf ^StormTopology topology]
+             (when (storm-active? storm-cluster-state storm-name)
+               (throw (AlreadyAliveException. storm-name)))
+             (validate-topology! topology)
+             (swap! submitted-count inc)
+             (let [storm-id (str storm-name "-" @submitted-count "-" (current-time-secs))
+                   storm-conf (from-json serializedConf)
+                   storm-conf (assoc storm-conf STORM-ID storm-id)
+
+                   total-storm-conf (merge conf storm-conf)
+                   topology (if (total-storm-conf TOPOLOGY-OPTIMIZE) (optimize-topology topology) topology)]
+               (log-message "Received topology submission for " storm-name " with conf " storm-conf)
+               (setup-storm-code conf storm-id uploadedJarLocation storm-conf topology)
+               ;; protects against multiple storms being submitted at once and cleanup thread killing storm in b/w
+               ;; assignment and starting the storm
+               (locking storm-submit-lock
+                 (.setup-heartbeats! storm-cluster-state storm-id)
+                 (setup-storm-static conf storm-id storm-cluster-state)
+                 (mk-assignments conf storm-id storm-cluster-state (fn [& ignored] (.add event-manager reassign-fn)) task-heartbeats-cache)
+                 (start-storm storm-name storm-cluster-state storm-id))
+               ))
+
+      (^void killTopology [this ^String storm-name]
+        (let [storm-id (get-storm-id storm-cluster-state storm-name)]
+          (when-not storm-id
+            (throw (NotAliveException. storm-name)))
+          (.deactivate-storm! storm-cluster-state storm-id)
+          (.add cleanup-manager cleanup-fn)
+          (log-message "Deactivated " storm-name " and scheduled to be killed")
+          ))
+
+      (beginFileUpload [this]
+        (let [fileloc (str inbox "/stormjar-" (uuid) ".jar")]
+          (.put uploaders fileloc (FileOutputStream. fileloc))
+          (log-message "Uploading file from client to " fileloc)
+          fileloc
+          ))
+      
+      (^void uploadChunk [this ^String location ^bytes chunk]
+             (let [^FileOutputStream os (.get uploaders location)]
+               (when-not os
+                 (throw (RuntimeException.
+                         "File for that location does not exist (or timed out)")))
+               (.write os chunk)
+               (.put uploaders location os)
+               ))
+
+      (^void finishFileUpload [this ^String location]
+             (let [^FileOutputStream os (.get uploaders location)]
+               (when-not os
+                 (throw (RuntimeException.
+                         "File for that location does not exist (or timed out)")))
+               (.close os)
+               (log-message "Finished uploading file from client: " location)
+               (.remove uploaders location)
+               ))
+
+      (^String beginFileDownload [this ^String file]
+               (let [is (BufferFileInputStream. file)
+                     id (uuid)]
+                 (.put downloaders id is)
+                 id
+                 ))
+
+      (^bytes downloadChunk [this ^String id]
+              (let [^BufferFileInputStream is (.get downloaders id)]
+                (when-not is
+                  (throw (RuntimeException.
+                          "Could not find input stream for that id")))
+                (let [ret (.read is)]
+                  (.put downloaders id is)
+                  (when (empty? ret)
+                    (.remove downloaders id))
+                  ret
+                  )))
+
+      (^String getTopologyConf [this ^String id]
+               (to-json (read-storm-conf conf id)))
+
+      (^StormTopology getTopology [this ^String id]
+                      (read-storm-topology conf id))
+      
+      (^ClusterSummary getClusterInfo [this]
+        (let [assigned (assigned-slots storm-cluster-state)
+              supervisor-infos (all-supervisor-info storm-cluster-state)
+              supervisor-summaries (dofor [[id info] supervisor-infos]
+                                          (let [ports (set (:worker-ports info))
+                                                ]
+                                            (SupervisorSummary. (:hostname info)
+                                                                (:uptime-secs info)
+                                                                (count ports)
+                                                                (count (assigned id)))
+                                            ))
+              nimbus-uptime (uptime)
+              bases (topology-bases storm-cluster-state)
+              topology-summaries (dofor [[id base] bases]
+                                        (let [assignment (.assignment-info storm-cluster-state id nil)]
+                                          (TopologySummary. id
+                                                            (:storm-name base)
+                                                            (-> (:task->node+port assignment)
+                                                                keys
+                                                                count)
+                                                            (-> (:task->node+port assignment)
+                                                                vals
+                                                                set
+                                                                count)
+                                                            (time-delta (:launch-time-secs base))
+                                                            )
+                                          ))
+              ]
+          (ClusterSummary. supervisor-summaries
+                           nimbus-uptime
+                           topology-summaries)
+          ))
+      
+      (^TopologyInfo getTopologyInfo [this ^String storm-id]
+        (let [task-info (storm-task-info storm-cluster-state storm-id)
+              base (.storm-base storm-cluster-state storm-id nil)
+              assignment (.assignment-info storm-cluster-state storm-id nil)
+              task-summaries (dofor [[task component] task-info]
+                                    (let [[node port] (get-in assignment [:task->node+port task])
+                                          host (-> assignment :node->host (get node))
+                                          heartbeat (.task-heartbeat storm-cluster-state storm-id task)
+                                          errors (.task-errors storm-cluster-state storm-id task)
+                                          errors (dofor [e errors] (ErrorInfo. (:error e) (:time-secs e)))
+                                          stats (:stats heartbeat)
+                                          stats (if stats
+                                                  (stats/thriftify-task-stats stats))]
+                                      (doto
+                                          (TaskSummary. task
+                                                        component
+                                                        host
+                                                        port
+                                                        (nil-to-zero
+                                                         (:uptime-secs heartbeat))
+                                                        errors
+                                                        )
+                                        (.set_stats stats))
+                                      ))
+              ]
+          (TopologyInfo. storm-id
+                         (:storm-name base)
+                         (time-delta (:launch-time-secs base))
+                         task-summaries
+                         )
+          ))
+      
+      Shutdownable
+        (shutdown [this]
+          (log-message "Shutting down master")
+          (reset! active false)
+          (doseq [t threads]
+                 (.interrupt t)
+                 (.join t))
+          (.shutdown event-manager)
+          (.shutdown cleanup-manager)
+          (.disconnect storm-cluster-state)
+          (log-message "Shut down master")
+          )
+     DaemonCommon
+       (waiting? [this]
+         (and
+          (every? (memfn sleeping?) threads)
+          (every? (memfn waiting?) managers)
+          )))))
+
+(defn launch-server! [conf]
+  (validate-distributed-mode! conf)
+  (let [options (THsHaServer$Options.)
+       _ (set! (. options maxWorkerThreads) 64)
+       service-handler (service-handler conf)
+       server (THsHaServer.
+               (Nimbus$Processor. service-handler)
+               (TNonblockingServerSocket. (int (conf NIMBUS-THRIFT-PORT)))
+               (TBinaryProtocol$Factory.) options)]
+    (.addShutdownHook (Runtime/getRuntime) (Thread. (fn [] (.shutdown service-handler) (.stop server))))
+    (log-message "Starting Nimbus server...")
+    (.serve server)))
+
+
+;; distributed implementation
+
+(defmethod setup-jar :distributed [conf tmp-jar-location stormroot]
+           (let [src-file (File. tmp-jar-location)]
+             (if-not (.exists src-file)
+               (throw
+                (IllegalArgumentException.
+                 (str tmp-jar-location " to copy to " stormroot " does not exist!"))))
+             (FileUtils/copyFile src-file (File. (master-stormjar-path stormroot)))
+             ))
+
+;; local implementation
+
+(defmethod setup-jar :local [conf & args]
+  nil
+  )
+
+
+(defn -main []
+  (launch-server! (read-storm-config)))
diff --git a/src/clj/backtype/storm/daemon/supervisor.clj b/src/clj/backtype/storm/daemon/supervisor.clj
new file mode 100644
index 0000000..11decf3
--- /dev/null
+++ b/src/clj/backtype/storm/daemon/supervisor.clj
@@ -0,0 +1,399 @@
+(ns backtype.storm.daemon.supervisor
+  (:use [backtype.storm bootstrap])
+  (:use [backtype.storm.daemon common])
+  (:require [backtype.storm.daemon [worker :as worker]])
+  (:gen-class))
+
+(bootstrap)
+
+(defmulti download-storm-code cluster-mode)
+(defmulti launch-worker cluster-mode)
+
+;; used as part of a map from port to this
+(defrecord LocalAssignment [storm-id task-ids])
+
+(defprotocol SupervisorDaemon
+  (get-id [this])
+  (get-conf [this])
+  (shutdown-all-workers [this])
+  )
+
+
+(defn- read-my-tasks [storm-cluster-state storm-id supervisor-id callback]
+  (let [assignment (.assignment-info storm-cluster-state storm-id callback)
+        my-tasks (filter (fn [[_ [node _]]] (= node supervisor-id))
+                         (:task->node+port assignment))
+        port-tasks (apply merge-with
+                          concat
+                          (for [[task-id [_ port]] my-tasks]
+                            {port [task-id]}
+                            ))]
+    (into {} (for [[port task-ids] port-tasks]
+               ;; need to cast to int b/c it might be a long (due to how yaml parses things)
+               [(int port) (LocalAssignment. storm-id task-ids)]
+               ))
+    ))
+
+(defn- read-assignments
+  "Returns map from port to struct containing :storm-id and :task-ids and :master-code-dir"
+  [storm-cluster-state supervisor-id callback]
+  (let [storm-ids (.assignments storm-cluster-state callback)]
+    (apply merge-with
+           (fn [& ignored]
+             (throw (RuntimeException.
+                     "Should not have multiple storms assigned to one port")))
+           (dofor [sid storm-ids] (read-my-tasks storm-cluster-state sid supervisor-id callback))
+           )))
+
+(defn- read-storm-code-locations
+  [storm-cluster-state callback]
+  (let [storm-ids (.assignments storm-cluster-state callback)]
+    (into {}
+      (dofor [sid storm-ids]
+        [sid (:master-code-dir (.assignment-info storm-cluster-state sid callback))]
+        ))
+    ))
+
+
+(defn- read-downloaded-storm-ids [conf]
+  (read-dir-contents (supervisor-stormdist-root conf))
+  )
+
+(defn read-worker-heartbeat [conf id]
+  (let [local-state (worker-state conf id)]
+    (.get local-state LS-WORKER-HEARTBEAT)
+    ))
+
+
+(defn my-worker-ids [conf]
+  (read-dir-contents (worker-root conf)))
+
+(defn read-worker-heartbeats
+  "Returns map from worker id to heartbeat"
+  [conf]
+  (let [ids (my-worker-ids conf)]
+    (into {}
+      (dofor [id ids]
+        [id (read-worker-heartbeat conf id)]))
+    ))
+
+
+(defn matches-an-assignment? [worker-heartbeat assigned-tasks]
+  (let [local-assignment (assigned-tasks (:port worker-heartbeat))]
+    (and local-assignment
+         (= (:storm-id worker-heartbeat) (:storm-id local-assignment))
+         (= (set (:task-ids worker-heartbeat)) (set (:task-ids local-assignment))))
+    ))
+
+(defn read-allocated-workers
+  "Returns map from worker id to worker heartbeat. if the heartbeat is nil, then the worker is dead (timed out or never wrote heartbeat)"
+  [conf local-state assigned-tasks]
+  (let [now (current-time-secs)
+        id->heartbeat (read-worker-heartbeats conf)
+        approved-ids (set (keys (.get local-state LS-APPROVED-WORKERS)))]
+    (into
+     {}
+     (dofor [[id hb] id->heartbeat]
+            (let [state (cond
+                         (or (not (contains? approved-ids id))
+                             (not (matches-an-assignment? hb assigned-tasks)))
+                           :disallowed
+                         (not hb)
+                           :not-started
+                         (> (- now (:time-secs hb))
+                            (conf SUPERVISOR-WORKER-TIMEOUT-SECS))
+                           :timed-out
+                         true
+                           :valid)]
+              (log-debug "Worker " id " is " state ": " hb)
+              [id [state hb]]
+              ))
+     )))
+
+(defn- wait-for-worker-launch [conf id start-time]
+  (let [state (worker-state conf id)]    
+    (loop []
+      (let [hb (.get state LS-WORKER-HEARTBEAT)]
+        (when (and
+               (not hb)
+               (<
+                (- (current-time-secs) start-time)
+                (conf SUPERVISOR-WORKER-START-TIMEOUT-SECS)
+                ))
+          (log-message id " still hasn't started")
+          (Time/sleep 500)
+          (recur)
+          ))
+      )))
+
+(defn- wait-for-workers-launch [conf ids]
+  (let [start-time (current-time-secs)]
+    (doseq [id ids]
+      (wait-for-worker-launch conf id start-time))
+    ))
+
+(defn generate-supervisor-id []
+  (uuid))
+
+(defn try-cleanup-worker [conf id]
+  (try
+    (rmr (worker-heartbeats-root conf id))
+    ;; this avoids a race condition with worker or subprocess writing pid around same time
+    (rmpath (worker-pids-root conf id))
+    (rmpath (worker-root conf id))
+  (catch RuntimeException e
+    (log-error e "Failed to cleanup worker " id ". Will retry later")
+    )))
+
+(defn shutdown-worker [conf supervisor-id id worker-thread-pids-atom]
+  (log-message "Shutting down " supervisor-id ":" id)
+  (let [pids (read-dir-contents (worker-pids-root conf id))
+        thread-pid (@worker-thread-pids-atom id)]
+    (when thread-pid
+      (psim/kill-process thread-pid))
+    (doseq [pid pids]
+      (ensure-process-killed! pid)
+      (rmpath (worker-pid-path conf id pid))
+      )
+    (try-cleanup-worker conf id))
+  (log-message "Shut down " supervisor-id ":" id))
+
+;; in local state, supervisor stores who its current assignments are
+;; another thread launches events to restart any dead processes if necessary
+(defserverfn mk-supervisor [conf]
+  (FileUtils/cleanDirectory (File. (supervisor-tmp-dir conf)))
+  (let [active (atom true)
+        uptime (uptime-computer)
+        worker-thread-pids-atom (atom {})
+        storm-cluster-state (cluster/mk-storm-cluster-state conf)
+        local-state (supervisor-state conf)
+        my-hostname (local-hostname)
+        supervisor-id (if-let [id (.get local-state LS-ID)] id (generate-supervisor-id))
+        _ (.put local-state LS-ID supervisor-id)
+        [event-manager processes-event-manager :as managers] [(event/event-manager false) (event/event-manager false)]
+        sync-processes (fn []
+                         (let [assigned-tasks (defaulted (.get local-state LS-LOCAL-ASSIGNMENTS) {})
+                               allocated (read-allocated-workers conf local-state assigned-tasks)
+                               keepers (filter-map-val
+                                        (fn [[state _]] (= state :valid))
+                                        allocated)
+                               keep-ports (set (for [[id [_ hb]] keepers] (:port hb)))
+                               reassign-tasks (select-keys-pred (complement keep-ports) assigned-tasks)
+                               new-worker-ids (into
+                                               {}
+                                               (for [port (keys reassign-tasks)]
+                                                 [port (uuid)]))
+                               ]
+                           ;; 1. to kill are those in allocated that are dead or disallowed
+                           ;; 2. kill the ones that should be dead
+                           ;;     - read pids, kill -9 and individually remove file
+                           ;;     - rmr heartbeat dir, rmdir pid dir, rmdir id dir (catch exception and log)
+                           ;; 3. of the rest, figure out what assignments aren't yet satisfied
+                           ;; 4. generate new worker ids, write new "approved workers" to LS
+                           ;; 5. create local dir for worker id
+                           ;; 5. launch new workers (give worker-id, port, and supervisor-id)
+                           ;; 6. wait for workers launch
+                           
+                           (log-debug "Syncing processes")
+                           (log-debug "Assigned tasks: " assigned-tasks)
+                           (log-debug "Allocated: " allocated)
+                           (doseq [[id [state heartbeat]] allocated]
+                             (when (not= :valid state)
+                               (log-message
+                                "Shutting down and clearing state for id " id
+                                ". State: " state
+                                ", Heartbeat: " (pr-str heartbeat))
+                               (shutdown-worker conf supervisor-id id worker-thread-pids-atom)
+                               ))
+                           (doseq [id (vals new-worker-ids)]
+                             (local-mkdirs (worker-pids-root conf id)))
+                           (.put local-state LS-APPROVED-WORKERS
+                                 (merge
+                                  (select-keys (.get local-state LS-APPROVED-WORKERS)
+                                               (keys keepers))
+                                  (zipmap (vals new-worker-ids) (keys new-worker-ids))
+                                  ))
+                           (wait-for-workers-launch
+                            conf
+                            (dofor [[port assignment] reassign-tasks]
+                              (let [id (new-worker-ids port)]
+                                (log-message "Launching worker with assignment "
+                                             (pr-str assignment)
+                                             " for this supervisor "
+                                             supervisor-id
+                                             " on port "
+                                             port
+                                             " with id "
+                                             id
+                                             )
+                                (launch-worker conf
+                                               (:storm-id assignment)
+                                               supervisor-id
+                                               port
+                                               id
+                                               worker-thread-pids-atom)
+                                id)))
+                           ))
+        synchronize-supervisor (fn this []
+                                 (let [sync-callback (fn [& ignored] (.add event-manager this))
+                                       storm-code-map (read-storm-code-locations storm-cluster-state sync-callback)
+                                       assigned-storm-ids (set (keys storm-code-map))
+                                       downloaded-storm-ids (set (read-downloaded-storm-ids conf))
+                                       new-assignment (read-assignments
+                                                        storm-cluster-state
+                                                        supervisor-id
+                                                        sync-callback)]
+                                   (log-debug "Synchronizing supervisor")
+                                   (log-debug "Storm code map: " storm-code-map)
+                                   (log-debug "Downloaded storm ids: " downloaded-storm-ids)
+                                   (log-debug "New assignment: " new-assignment)
+                                   ;; download code first
+                                   ;; This might take awhile
+                                   ;;   - should this be done separately from usual monitoring?
+                                   ;; should we only download when storm is assigned to this supervisor?
+                                   (doseq [[storm-id master-code-dir] storm-code-map]
+                                     (when-not (downloaded-storm-ids storm-id)
+                                       (log-message "Downloading code for storm id "
+                                          storm-id
+                                          " from "
+                                          master-code-dir)
+                                       (download-storm-code conf storm-id master-code-dir)
+                                       (log-message "Finished downloading code for storm id "
+                                          storm-id
+                                          " from "
+                                          master-code-dir)
+                                       ))
+                                   ;; remove any downloaded code that's no longer assigned or active
+                                   (doseq [storm-id downloaded-storm-ids]
+                                     (when-not (assigned-storm-ids storm-id)
+                                       (log-message "Removing code for storm id "
+                                                    storm-id)
+                                       (rmr (supervisor-stormdist-root conf storm-id))
+                                       ))
+                                   (log-debug "Writing new assignment "
+                                              (pr-str new-assignment))
+                                   (.put local-state
+                                         LS-LOCAL-ASSIGNMENTS
+                                         new-assignment
+                                         )
+                                   (.add processes-event-manager sync-processes)
+                                   ))
+        heartbeat-fn (fn [] (.supervisor-heartbeat!
+                               storm-cluster-state
+                               supervisor-id
+                               (SupervisorInfo. (current-time-secs)
+                                                my-hostname
+                                                (conf SUPERVISOR-SLOTS-PORTS)
+                                                (uptime))))
+        _ (heartbeat-fn)
+        ;; should synchronize supervisor so it doesn't launch anything after being down (optimization)
+        threads (concat
+                  [(async-loop
+                     (fn []
+                       (heartbeat-fn)
+                       (when @active (conf SUPERVISOR-HEARTBEAT-FREQUENCY-SECS))
+                       )
+                     :priority Thread/MAX_PRIORITY)]
+                   ;; This isn't strictly necessary, but it doesn't hurt and ensures that the machine stays up
+                   ;; to date even if callbacks don't all work exactly right
+                   (when (conf SUPERVISOR-ENABLE)
+                     [(async-loop
+                       (fn []
+                         (.add event-manager synchronize-supervisor)
+                         (when @active 10)
+                         ))
+                      (async-loop
+                         (fn []
+                           (.add processes-event-manager sync-processes)
+                           (when @active (conf SUPERVISOR-MONITOR-FREQUENCY-SECS))
+                           )
+                         :priority Thread/MAX_PRIORITY)]))]
+    (reify
+     Shutdownable
+     (shutdown [this]
+               (log-message "Shutting down supervisor " supervisor-id)
+               (reset! active false)
+               (doseq [t threads]
+                 (.interrupt t)
+                 (.join t))
+               (.shutdown event-manager)
+               (.shutdown processes-event-manager)
+               (.disconnect storm-cluster-state))
+     SupervisorDaemon
+     (get-conf [this]
+       conf)
+     (get-id [this]
+       supervisor-id )
+     (shutdown-all-workers [this]
+       (let [ids (my-worker-ids conf)]
+         (doseq [id ids]
+           (shutdown-worker conf supervisor-id id worker-thread-pids-atom)
+           )))
+     DaemonCommon
+     (waiting? [this]
+       (or (not @active)
+           (and
+            (every? (memfn sleeping?) threads)
+            (every? (memfn waiting?) managers)))
+           ))))
+
+(defn kill-supervisor [supervisor]
+  (.shutdown supervisor)
+  )
+
+;; distributed implementation
+
+(defmethod download-storm-code
+    :distributed [conf storm-id master-code-dir]
+    ;; Downloading to permanent location is atomic
+    (let [tmproot (str (supervisor-tmp-dir conf) "/" (uuid))
+          stormroot (supervisor-stormdist-root conf storm-id)]
+      (FileUtils/forceMkdir (File. tmproot))      
+      
+      (Utils/downloadFromMaster conf (master-stormjar-path master-code-dir) (supervisor-stormjar-path tmproot))
+      (Utils/downloadFromMaster conf (master-stormcode-path master-code-dir) (supervisor-stormcode-path tmproot))
+      (Utils/downloadFromMaster conf (master-stormconf-path master-code-dir) (supervisor-stormconf-path tmproot))
+      (extract-dir-from-jar (supervisor-stormjar-path tmproot) RESOURCES-SUBDIR tmproot)
+      (FileUtils/moveDirectory (File. tmproot) (File. stormroot))
+      ))
+
+
+(defmethod launch-worker
+    :distributed [conf storm-id supervisor-id port worker-id worker-thread-pids-atom]
+    (let [stormroot (supervisor-stormdist-root conf storm-id)
+          stormjar (supervisor-stormjar-path stormroot)
+          classpath (add-to-classpath (current-classpath) [stormjar])
+          childopts (conf WORKER-CHILDOPTS)
+          logfilename (str "worker-" port ".log")
+          command (str "java -server " childopts
+                       " -Djava.library.path=" (conf JAVA-LIBRARY-PATH)
+                       " -Dlogfile.name=" logfilename
+                       " -cp " classpath " backtype.storm.daemon.worker "
+                       storm-id " " supervisor-id " " port " " worker-id)]
+      (launch-process command)
+      ))
+
+;; local implementation
+
+(defmethod download-storm-code
+    :local [conf storm-id master-code-dir]
+  (let [stormroot (supervisor-stormdist-root conf storm-id)]
+      (FileUtils/copyDirectory (File. master-code-dir) (File. stormroot))
+      (let [classloader (.getContextClassLoader (Thread/currentThread))
+            url (.getResource classloader RESOURCES-SUBDIR)]
+            (when url
+              (FileUtils/copyDirectory (File. (.getFile url)) (File. (str stormroot "/" RESOURCES-SUBDIR)))
+              ))))
+
+(defmethod launch-worker
+    :local [conf storm-id supervisor-id port worker-id worker-thread-pids-atom]
+    (let [pid (uuid)
+          worker (worker/mk-worker conf storm-id supervisor-id port worker-id)]
+      (psim/register-process pid worker)
+      (swap! worker-thread-pids-atom assoc worker-id pid)
+      ))
+
+(defn -main []
+  (let [conf (read-storm-config)]
+    (validate-distributed-mode! conf)
+    (mk-supervisor conf)))
diff --git a/src/clj/backtype/storm/daemon/task.clj b/src/clj/backtype/storm/daemon/task.clj
new file mode 100644
index 0000000..f6cdc17
--- /dev/null
+++ b/src/clj/backtype/storm/daemon/task.clj
@@ -0,0 +1,498 @@
+(ns backtype.storm.daemon.task
+  (:use [backtype.storm.daemon common])
+  (:use [backtype.storm bootstrap])
+  (:import [java.util.concurrent ConcurrentLinkedQueue ConcurrentHashMap])
+  (:require [backtype.storm [tuple :as tuple]]))
+
+(bootstrap)
+
+(defn- mk-fields-grouper [^Fields out-fields ^Fields group-fields num-tasks]
+  (fn [^Tuple tuple]
+    (mod (tuple/list-hash-code (.select out-fields group-fields (.getValues tuple)))
+         num-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."
+  [^Fields out-fields thrift-grouping num-tasks]
+  (let [random (Random.)]
+    (condp = (thrift/grouping-type thrift-grouping)
+      :fields
+        (if (thrift/global-grouping? thrift-grouping)
+          (fn [^Tuple tuple]
+            ;; It's possible for target to have multiple tasks if it reads multiple sources
+            0 )
+          (let [group-fields (Fields. (thrift/field-grouping thrift-grouping))]
+            (mk-fields-grouper out-fields group-fields num-tasks)
+            ))
+      :all
+        (fn [^Tuple tuple]
+          (range num-tasks))
+      :shuffle
+        (let [choices (rotating-random-range num-tasks)]
+          (fn [^Tuple tuple]
+            (acquire-random-range-id choices num-tasks)
+            ))
+      :none
+        (fn [^Tuple tuple]
+          (mod (.nextInt random) num-tasks))
+      :direct
+        :direct
+      )))
+
+(defn- update-ack [curr-entry val]
+  (let [old (get curr-entry :val 0)]
+    (assoc curr-entry :val (bit-xor old val))
+    ))
+
+(defn- acker-emit-direct [^OutputCollector collector ^Integer task ^Integer stream ^List values]
+  (.emitDirect collector task stream values)
+  )
+
+(defn mk-acker-bolt []
+  (let [output-collector (atom nil)
+        pending (atom nil)]
+    (reify IBolt
+      (^void prepare [this ^Map storm-conf ^TopologyContext context ^OutputCollector collector]
+               (reset! output-collector collector)
+               (reset! pending (TimeCacheMap. (int (storm-conf TOPOLOGY-MESSAGE-TIMEOUT-SECS))))
+               )
+      (^void execute [this ^Tuple tuple]
+             (let [id (.getValue tuple 0)
+                   ^TimeCacheMap pending @pending
+                   curr (.get pending id)
+                   curr (condp = (.getSourceStreamId tuple)
+                            ACKER-INIT-STREAM-ID (-> curr
+                                                     (update-ack id)
+                                                     (assoc :spout-task (.getValue tuple 1)))
+                            ACKER-ACK-STREAM-ID (update-ack curr (.getValue tuple 1))
+                            ACKER-FAIL-STREAM-ID (assoc curr :failed true))]
+               (.put pending id curr)
+               (when (and curr
+                          (:spout-task curr))
+                 (cond (= 0 (:val curr))
+                       (do
+                         (.remove pending id)
+                         (acker-emit-direct @output-collector
+                                            (:spout-task curr)
+                                            ACKER-ACK-STREAM-ID
+                                            [id]
+                                            ))
+                       (:failed curr)
+                       (do
+                         (.remove pending id)
+                         (acker-emit-direct @output-collector
+                                            (:spout-task curr)
+                                            ACKER-FAIL-STREAM-ID
+                                            [id]
+                                            ))
+                       ))
+               (.ack ^OutputCollector @output-collector tuple)
+               ))
+      (^void cleanup [this]
+             )
+      )))
+
+(defn- get-task-object [topology component-id]
+  (if (= ACKER-COMPONENT-ID component-id)
+    (mk-acker-bolt)
+    (let [spouts (.get_spouts topology)
+          bolts (.get_bolts topology)
+          obj (Utils/getSetComponentObject
+               (cond
+                (contains? spouts component-id) (.get_spout_object (get spouts component-id))
+                (contains? bolts component-id) (.get_bolt_object (get bolts component-id))
+                true (throw (RuntimeException. (str "Could not find " component-id " in " topology)))))
+          obj (if (instance? ShellComponent obj)
+                (if (contains? spouts component-id)
+                  (ShellSpout. obj)
+                  (ShellBolt. obj))
+                obj )]
+      obj
+      )))
+
+
+(defn outbound-components
+  "Returns map of stream id to component id to grouper"
+  [topology-context]
+  (let [output-groupings (clojurify-structure (.getThisTargets topology-context))
+        acker-task-amt (count (.getComponentTasks topology-context ACKER-COMPONENT-ID))]
+    (merge
+     {
+      ACKER-INIT-STREAM-ID
+        {ACKER-COMPONENT-ID (mk-fields-grouper (Fields. ["id" "spout-task"])
+                                               (Fields. ["id"])
+                                               acker-task-amt)}
+      ACKER-ACK-STREAM-ID
+        {ACKER-COMPONENT-ID (mk-fields-grouper (Fields. ["id" "ack-val"])
+                                               (Fields. ["id"])
+                                               acker-task-amt)}
+      ACKER-FAIL-STREAM-ID
+        {ACKER-COMPONENT-ID (mk-fields-grouper (Fields. ["id"]) ;; TODO: add failure msg here later...
+                                               (Fields. ["id"])
+                                               acker-task-amt)}
+      }
+     (into {}
+           (for [[stream-id component->grouping] output-groupings
+                 :let [out-fields (.getThisOutputFields topology-context stream-id)]]
+             [stream-id
+              (into {}
+                    (for [[component tgrouping] component->grouping]
+                      [component (mk-grouper out-fields
+                                             tgrouping
+                                             (count (.getComponentTasks topology-context component))
+                                             )]
+                      ))])))
+    ))
+
+
+
+(defmulti mk-executors class-selector)
+(defmulti close-component class-selector)
+(defmulti mk-task-stats class-selector)
+
+(defn- get-readable-name [topology-context]
+  (let [component-id (.getThisComponentId topology-context)]
+    (if (system-component? component-id)
+      ({ACKER-COMPONENT-ID "Acker"} component-id)
+      ;; TODO: change this so that can get better name for nested bolts 
+      (str (class (get-task-object (.getRawTopology topology-context) component-id)))
+      )))
+
+(defn- send-ack [^TopologyContext topology-context ^Tuple input-tuple
+                 ^List generated-ids send-fn]
+  (let [ack-val (bit-xor-vals generated-ids)]
+    (doseq [[anchor id] (.. input-tuple getMessageId getAnchorsToIds)]
+      (send-fn (Tuple. topology-context
+                       [anchor (bit-xor ack-val id)]
+                       (.getThisTaskId topology-context)
+                       ACKER-ACK-STREAM-ID))
+      )))
+
+(defn mk-task [conf storm-conf topology-context storm-id zmq-context cluster-state storm-active-atom transfer-fn]
+  (let [task-id (.getThisTaskId topology-context)
+        component-id (.getThisComponentId topology-context)
+        task-info (.getTaskToComponent topology-context)
+        active (atom true)
+        uptime (uptime-computer)
+        storm-cluster-state (cluster/mk-storm-cluster-state cluster-state)
+        
+        task-object (get-task-object (.getRawTopology topology-context)
+                                     (.getThisComponentId topology-context))
+        task-stats (mk-task-stats task-object (sampling-rate storm-conf))
+
+        report-error (fn [error]
+                       (.report-task-error storm-cluster-state storm-id task-id error)
+                       (halt-process! 1 "Task died!"))
+
+        ;; heartbeat ASAP so nimbus doesn't reassign
+        heartbeat-thread (async-loop
+                          (fn []
+                            (.task-heartbeat! storm-cluster-state storm-id task-id
+                                              (TaskHeartbeat. (current-time-secs)
+                                                              (uptime)
+                                                              (stats/render-stats! task-stats)))
+                            (when @active (storm-conf TASK-HEARTBEAT-FREQUENCY-SECS))
+                            )
+                          :priority Thread/MAX_PRIORITY
+                          :kill-fn report-error)
+
+        stream->component->grouper (outbound-components topology-context)
+        component->tasks (reverse-map task-info)
+        ;; important it binds to virtual port before function returns
+        ^ZMQ$Socket puller (-> zmq-context (mq/socket mq/pull) (mqvp/virtual-bind task-id))
+
+        ;; TODO: consider DRYing things up and moving stats / tuple -> multiple components code here
+        task-transfer-fn (fn [task ^Tuple tuple]
+                           (transfer-fn task tuple)
+                           )
+        task-readable-name (get-readable-name topology-context)
+
+        emit-sampler (mk-stats-sampler storm-conf)
+        send-fn (fn this
+                  ([^Integer out-task-id ^Tuple tuple]
+                     (when (= true (storm-conf TOPOLOGY-DEBUG))
+                       (log-message "Emitting direct: " out-task-id "; " task-readable-name " " tuple))
+                     (let [target-component (.getComponentId topology-context out-task-id)
+                           component->grouping (stream->component->grouper (.getSourceStreamId tuple))
+                           grouping (get component->grouping target-component)
+                           out-task-id (if (or
+                                            ;; This makes streams to/from system
+                                            ;; component (like ackers) implicit
+                                            (system-component? component-id)
+                                            (system-component? target-component)
+                                            grouping)
+                                         out-task-id)]
+                       (when (and (not-nil? grouping) (not= :direct grouping))
+                         (throw (IllegalArgumentException. "Cannot emitDirect to a task expecting a regular grouping")))
+                       (when out-task-id
+                         (task-transfer-fn out-task-id tuple))
+                       (when (emit-sampler)
+                         (stats/emitted-tuple! task-stats (.getSourceStreamId tuple))
+                         (stats/transferred-tuples! task-stats (.getSourceStreamId tuple) 1)
+                         )
+                       [out-task-id]
+                       ))
+                  ([^Tuple tuple]
+                     (when (= true (storm-conf TOPOLOGY-DEBUG))
+                       (log-message "Emitting: " task-readable-name " " tuple))
+                     (let [stream (.getSourceStreamId tuple)
+                           ;; TODO: this doesn't seem to be very fast
+                           ;; and seems to be the current bottleneck
+                           out-tasks (mapcat
+                                      (fn [[out-component grouper]]
+                                        (when (= :direct grouper)
+                                          ;;  TODO: this is wrong, need to check how the stream was declared
+                                          (throw (IllegalArgumentException. "Cannot do regular emit to direct stream")))
+                                        (let [tasks (component->tasks out-component)
+                                              indices (collectify (grouper tuple))]
+                                          (for [i indices] (tasks i))))
+                                      (stream->component->grouper stream))
+                           num-out-tasks (count out-tasks)]
+                       (when (emit-sampler)
+                         (stats/emitted-tuple! task-stats (.getSourceStreamId tuple))
+                         (stats/transferred-tuples! task-stats (.getSourceStreamId tuple) num-out-tasks)
+                         )
+                       (if (= num-out-tasks 1)
+                         (task-transfer-fn (first out-tasks) tuple)
+                         ;;TODO: optimize the out-tasks = 0 case by
+                         ;; not including this tuple in the ack list
+                         ;; for previous tuple
+                         ;; TODO: need to create the new ids, and then create the tuples, and then ack
+                         (let [out-ids (repeatedly (count out-tasks) #(MessageId/generateId))]
+                           (dorun
+                            (map (fn [id t]
+                                   (task-transfer-fn t (.copyWithNewId tuple id)))
+                                 out-ids
+                                 out-tasks))
+                           (send-ack topology-context
+                                     tuple
+                                     out-ids
+                                     this)
+                           ))
+                       out-tasks)))
+        executor-threads (dofor
+                          [exec (mk-executors task-object storm-conf puller send-fn
+                                              storm-active-atom topology-context
+                                              task-stats report-error)]
+                          (async-loop (fn [] (exec) (when @active 0))
+                                      :kill-fn report-error))
+        system-threads [heartbeat-thread]
+        all-threads  (concat executor-threads system-threads)]
+    (reify
+       Shutdownable
+       (shutdown
+        [this]
+        (log-message "Shutting down task " storm-id ":" task-id)
+        (reset! active false)
+        (let [pusher (-> zmq-context (mq/socket mq/push) (mqvp/virtual-connect task-id))]
+          ;; empty messages are skip messages (this unblocks the socket)
+          (mq/send pusher (mq/barr))
+          (.close pusher))
+        (doseq [t all-threads]
+          (.interrupt t)
+          (.join t))
+        (.remove-task-heartbeat! storm-cluster-state storm-id task-id)
+        (.disconnect storm-cluster-state)
+        (.close puller)
+        (close-component task-object)
+        (log-message "Shut down task " storm-id ":" task-id))
+       DaemonCommon
+       (waiting? [this]
+                 ;; executor threads are independent since they don't sleep
+                 ;; -> they block on zeromq
+                 (every? (memfn sleeping?) system-threads)
+                 ))))
+
+(defn- fail-spout-msg [^ISpout spout storm-conf msg-id ^Tuple tuple time-delta task-stats]
+  (log-message "Failing message " msg-id ": " tuple)
+  (.fail spout msg-id)
+  (when time-delta
+    (stats/spout-failed-tuple! task-stats (.getSourceStreamId tuple) time-delta)
+    ))
+
+(defn- ack-spout-msg [^ISpout spout storm-conf msg-id ^Tuple tuple time-delta task-stats]
+  (when (= true (storm-conf TOPOLOGY-DEBUG))
+    (log-message "Acking message " msg-id))
+  (.ack spout msg-id)
+  (when time-delta
+    (stats/spout-acked-tuple! task-stats (.getSourceStreamId tuple) time-delta)
+    ))
+
+(defmethod mk-executors ISpout [^ISpout spout storm-conf ^ZMQ$Socket puller send-fn storm-active-atom
+                                ^TopologyContext topology-context task-stats report-error-fn]
+  (let [wait-fn (fn [] @storm-active-atom)
+        max-spout-pending (storm-conf TOPOLOGY-MAX-SPOUT-PENDING)
+        deserializer (TupleDeserializer. storm-conf topology-context)
+        event-queue (ConcurrentLinkedQueue.)
+        sampler (mk-stats-sampler storm-conf)
+        pending (TimeCacheMap.
+                 (int (storm-conf TOPOLOGY-MESSAGE-TIMEOUT-SECS))
+                 (reify TimeCacheMap$ExpiredCallback
+                        (expire [this msg-id [spout-id tuple start-time-ms]]
+                                (let [time-delta (if start-time-ms (time-delta-ms start-time-ms))]
+                                  (.add event-queue #(fail-spout-msg spout storm-conf spout-id tuple time-delta task-stats)))
+                                )))
+        send-spout-msg (fn [out-stream-id values message-id out-task-id]
+                         (let [task-id (.getThisTaskId topology-context)
+                               gen-id (MessageId/generateId)
+                               tuple-id (if message-id
+                                          (MessageId/makeRootId gen-id)
+                                          (MessageId/makeUnanchored))
+                               tuple (Tuple. topology-context
+                                             values
+                                             task-id
+                                             out-stream-id
+                                             tuple-id)
+                               out-tasks (if out-task-id
+                                           (send-fn out-task-id tuple)
+                                           (send-fn tuple))]
+                           (if (= 0 (storm-conf TOPOLOGY-ACKERS))
+                             (.add event-queue #(ack-spout-msg spout storm-conf message-id tuple nil task-stats))
+                             (when message-id
+                               (.put pending gen-id [message-id
+                                                    tuple
+                                                    (if (sampler) (System/currentTimeMillis))])
+                               (send-fn (Tuple. topology-context
+                                                [gen-id task-id]
+                                                task-id
+                                                ACKER-INIT-STREAM-ID))
+                               ))
+                           out-tasks
+                           ))
+        output-collector (reify ISpoutOutputCollector
+                                (^List emit [this ^int stream-id ^List tuple ^Object message-id]
+                                       (send-spout-msg stream-id tuple message-id nil)
+                                       )
+                                (^void emitDirect [this ^int out-task-id ^int stream-id
+                                                   ^List tuple ^Object message-id]
+                                       (send-spout-msg stream-id tuple message-id out-task-id)
+                                       ))]
+    (.open spout storm-conf topology-context (SpoutOutputCollector. output-collector))
+    [(fn []
+       ;; This design requires that spouts be non-blocking
+       (loop []
+         (when-let [event (.poll event-queue)]
+           (event)
+           (recur)
+           ))
+       (if (or (not max-spout-pending)
+               (< (.size pending) max-spout-pending))
+         (if (wait-fn)
+           (.nextTuple spout)
+           (Time/sleep 100))
+         ;; TODO: log that it's getting throttled
+         ))
+      (fn []
+        (let [^bytes ser-msg (mq/recv puller)]
+          ;; skip empty messages (used during shutdown)
+          (when-not (empty? ser-msg)
+            (let [tuple (.deserialize deserializer ser-msg)
+                  id (.getValue tuple 0)
+                  [spout-id tuple-finished start-time-ms] (.remove pending id)
+                  time-delta (if start-time-ms (time-delta-ms start-time-ms))]
+              (when spout-id
+                (condp = (.getSourceStreamId tuple)
+                    ACKER-ACK-STREAM-ID (.add event-queue #(ack-spout-msg spout storm-conf spout-id
+                                                                          tuple-finished time-delta task-stats))
+                    ACKER-FAIL-STREAM-ID (.add event-queue #(fail-spout-msg spout storm-conf spout-id
+                                                                            tuple-finished time-delta task-stats))
+                    )))
+            ;; TODO: on failure, emit tuple to failure stream
+            )))
+    ]
+    ))
+
+(defn- tuple-time-delta! [^Map start-times ^Tuple tuple]
+  (let [start-time (.remove start-times tuple)]
+    (if start-time
+      (time-delta-ms start-time))
+    ))
+
+(defmethod mk-executors IBolt [^IBolt bolt storm-conf ^ZMQ$Socket puller send-fn storm-active-atom
+                               ^TopologyContext topology-context task-stats report-error-fn]
+  (let [deserializer (TupleDeserializer. storm-conf topology-context)
+        task-id (.getThisTaskId topology-context)
+        component-id (.getThisComponentId topology-context)
+        tuple-start-times (ConcurrentHashMap.)
+        sampler (mk-stats-sampler storm-conf)
+        output-collector (reify IInternalOutputCollector
+                          (^List emit [this ^Tuple output]
+                                 (send-fn output)
+                                 )
+                          (^void emitDirect [this ^int task-id ^Tuple output]
+                                 (send-fn task-id output)
+                                 )
+                          
+                          (^void ack [this ^Tuple input-tuple ^List generated-ids]
+                                 (send-ack topology-context
+                                           input-tuple
+                                           generated-ids
+                                           send-fn)
+                                 (let [delta (tuple-time-delta! tuple-start-times input-tuple)]
+                                   (when delta
+                                     (stats/bolt-acked-tuple! task-stats
+                                                              (.getSourceComponent input-tuple)
+                                                              (.getSourceStreamId input-tuple)
+                                                              delta)
+                                     )))
+                          (^void fail [this ^Tuple input-tuple]
+                                 (doseq [anchor (.. input-tuple getMessageId getAnchors)]
+                                     (send-fn (Tuple. topology-context
+                                                      [anchor]
+                                                      task-id
+                                                      ACKER-FAIL-STREAM-ID))
+                                     )
+                                 (let [delta (tuple-time-delta! tuple-start-times input-tuple)]
+                                   (when delta
+                                     (stats/bolt-failed-tuple! task-stats
+                                                               (.getSourceComponent input-tuple)
+                                                               (.getSourceStreamId input-tuple)
+                                                               delta)
+                                     )))
+                          (^void reportError [this ^Throwable error]
+                                 (report-error-fn error)
+                                 ))]
+    (.prepare bolt
+              storm-conf
+              topology-context
+              (OutputCollectorImpl. topology-context output-collector))
+    ;; TODO: can get any SubscribedState objects out of the context now
+    [(fn []
+       ;; 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
+       ;; TODO: for state sync, need to first send sync messages in a loop and receive tuples until synchronization
+       ;; buffer other tuples until fully synchronized, then process all of those tuples
+       ;; then go into normal loop
+       ;; spill to disk?
+       ;; could be receiving incremental updates while waiting for sync or even a partial sync because of another failed task
+       ;; should remember sync requests and include a random sync id in the request. drop anything not related to active sync requests
+       ;; or just timeout the sync messages that are coming in until full sync is hit from that task
+       ;; need to drop incremental updates from tasks where waiting for sync. otherwise, buffer the incremental updates
+       (let [^bytes ser (mq/recv puller)]
+         (when-not (empty? ser)  ; skip empty messages (used during shutdown)
+           (log-debug "Processing message")
+           (let [tuple (.deserialize deserializer ser)]
+             ;; TODO: for state sync, need to check if tuple comes from state spout. if so, update state
+             ;; TODO: how to handle incremental updates as well as synchronizations at same time
+             ;; TODO: need to version tuples somehow
+             (log-debug "Received tuple " tuple " at task " (.getThisTaskId topology-context))
+             (when (sampler)
+               (.put tuple-start-times tuple (System/currentTimeMillis)))
+             
+             (.execute bolt tuple)             
+             ))))]
+    ))
+
+
+(defmethod close-component ISpout [spout]
+  (.close spout))
+
+(defmethod close-component IBolt [bolt]
+  (.cleanup bolt))
+
+(defmethod mk-task-stats ISpout [_ rate]
+  (stats/mk-spout-stats rate))
+
+(defmethod mk-task-stats IBolt [_ rate]
+  (stats/mk-bolt-stats rate))
diff --git a/src/clj/backtype/storm/daemon/worker.clj b/src/clj/backtype/storm/daemon/worker.clj
new file mode 100644
index 0000000..6f15cf5
--- /dev/null
+++ b/src/clj/backtype/storm/daemon/worker.clj
@@ -0,0 +1,254 @@
+(ns backtype.storm.daemon.worker
+  (:use [backtype.storm.daemon common])
+  (:use [backtype.storm bootstrap])
+  (:import [java.util.concurrent LinkedBlockingQueue])
+  (:require [backtype.storm.daemon [task :as task]])
+  (:gen-class))
+
+(bootstrap)
+
+
+(defmulti virtual-port-url cluster-mode)
+(defmulti connect-url cluster-mode)
+
+
+(defn read-worker-task-ids [storm-cluster-state storm-id supervisor-id port]
+  (let [assignment (:task->node+port (.assignment-info storm-cluster-state storm-id nil))]
+    (doall
+      (mapcat (fn [[task-id loc]]
+              (if (= loc [supervisor-id port])
+                [task-id]
+                ))
+            assignment))
+    ))
+
+(defn- read-storm-cache [conf storm-id]
+  (let [stormroot (supervisor-stormdist-root conf storm-id)
+        conf-path (supervisor-stormconf-path stormroot)
+        topology-path (supervisor-stormcode-path stormroot)]
+    [(merge conf (Utils/deserialize (FileUtils/readFileToByteArray (File. conf-path))))
+     (Utils/deserialize (FileUtils/readFileToByteArray (File. topology-path)))]
+    ))
+
+(defn do-heartbeat [conf worker-id port storm-id task-ids]
+  (.put (worker-state conf worker-id)
+        LS-WORKER-HEARTBEAT
+        (WorkerHeartbeat.
+          (current-time-secs)
+          storm-id
+          task-ids
+          port)))
+
+(defn worker-outbound-tasks
+  "Returns seq of task-ids that receive messages from this worker"
+  ;; if this is an acker, needs to talk to the spouts
+  [task->component mk-topology-context task-ids]
+  (let [topology-context (mk-topology-context (first task-ids))
+        spout-components (-> topology-context
+                             .getRawTopology
+                             .get_spouts
+                             keys)
+        contains-acker? (some? (fn [tid]
+                                 (= ACKER-COMPONENT-ID
+                                    (.getComponentId topology-context tid)))
+                               task-ids)
+        components (concat
+                    [ACKER-COMPONENT-ID]
+                    (if contains-acker? spout-components)
+                    (mapcat
+                     (fn [task-id]
+                       (let [context (mk-topology-context task-id)]
+                         (->> (.getThisTargets context)
+                              vals
+                              (map keys)
+                              (apply concat))
+                         ))
+                     task-ids))]
+    (set
+     (apply concat
+            ;; fix this
+            (-> (reverse-map task->component) (select-keys components) vals)))
+    ))
+
+;; TODO: should worker even take the storm-id as input? this should be
+;; deducable from cluster state (by searching through assignments)
+;; what about if there's inconsistency in assignments? -> but nimbus
+;; should guarantee this consistency
+;; TODO: consider doing worker heartbeating rather than task heartbeating to reduce the load on zookeeper
+(defserverfn mk-worker [conf storm-id supervisor-id port worker-id]
+  (log-message "Launching worker for " storm-id " on " supervisor-id ":" port " with id " worker-id)
+  (let [active (atom true)
+        storm-active-atom (atom false)
+        cluster-state (cluster/mk-distributed-cluster-state conf)
+        storm-cluster-state (cluster/mk-storm-cluster-state cluster-state)
+        task-ids (read-worker-task-ids storm-cluster-state storm-id supervisor-id port)
+        ;; because in local mode, its not a separate
+        ;; process. supervisor will register it in this case
+        _ (when (= :distributed (cluster-mode conf))
+            (touch (worker-pid-path conf worker-id (process-pid))))
+        heartbeat-fn #(do-heartbeat conf worker-id port storm-id task-ids)
+        ;; do this here so that the worker process dies if this fails
+        ;; it's important that worker heartbeat to supervisor ASAP when launching so that the supervisor knows it's running (and can move on)
+        _ (heartbeat-fn)
+        [storm-conf topology] (read-storm-cache conf storm-id)
+        event-manager (event/event-manager true)
+        
+        task->component (storm-task-info storm-cluster-state storm-id)
+        mk-topology-context #(TopologyContext. topology
+                                               task->component
+                                               storm-id
+                                               (supervisor-storm-resources-path
+                                                 (supervisor-stormdist-root conf storm-id))
+                                               (worker-pids-root conf worker-id)
+                                               %)
+
+        zmq-context (mq/context (storm-conf ZMQ-THREADS))
+        outbound-tasks (worker-outbound-tasks task->component mk-topology-context task-ids)
+        endpoint-socket-lock (mk-rw-lock)
+        node+port->socket (atom {})
+        task->node+port (atom {})
+
+        transfer-queue (LinkedBlockingQueue.) ; possibly bound the size of it
+        
+        transfer-fn (fn [task ^Tuple tuple]
+                      (.put transfer-queue [task tuple])
+                      )
+        refresh-connections (fn this
+                              ([]
+                                (this (fn [& ignored] (.add event-manager this))))
+                              ([callback]
+                                (let [assignment (.assignment-info storm-cluster-state storm-id callback)
+                                      my-assignment (select-keys (:task->node+port assignment) outbound-tasks)
+                                      needed-connections (set (vals my-assignment))
+                                      current-connections (set (keys @node+port->socket))
+                                      new-connections (set/difference needed-connections current-connections)
+                                      remove-connections (set/difference current-connections needed-connections)]
+                                      (swap! node+port->socket
+                                             merge
+                                             (into {}
+                                               (dofor [[node port :as endpoint] new-connections]
+                                                 [endpoint
+                                                  (-> zmq-context
+                                                      (mq/socket mq/push)
+                                                      (mq/set-linger (storm-conf ZMQ-LINGER-MILLIS))
+                                                      (mq/connect
+                                                       (connect-url conf
+                                                                    ((:node->host assignment) node)
+                                                                    port)))
+                                                  ]
+                                                 )))
+                                      (write-locked endpoint-socket-lock
+                                        (reset! task->node+port my-assignment))
+                                      (doseq [endpoint remove-connections]
+                                        (.close (@node+port->socket endpoint)))
+                                      (apply swap!
+                                             node+port->socket
+                                             dissoc
+                                             remove-connections)
+                                  )))
+
+        refresh-storm-active (fn this
+                               ([]
+                                  (this (fn [& ignored] (.add event-manager this))))
+                               ([callback]
+                                  (reset!
+                                   storm-active-atom
+                                   (not-nil? (.storm-base storm-cluster-state storm-id callback)))
+                                  ))      
+        _ (refresh-connections nil)
+        _ (refresh-storm-active nil)
+
+        heartbeat-thread (async-loop
+                          (fn []
+                            ;; this @active check handles the case where it's started after shutdown* joins to the thread
+                            ;; if the thread is started after the join, then @active must be false. So there's no risk 
+                            ;; of writing heartbeat after it's been shut down.
+                            (when @active (heartbeat-fn) (conf WORKER-HEARTBEAT-FREQUENCY-SECS))
+                            )
+                          :priority Thread/MAX_PRIORITY)        
+        tasks (dofor [tid task-ids] (task/mk-task conf storm-conf (mk-topology-context tid) storm-id zmq-context cluster-state storm-active-atom transfer-fn))
+        threads [(async-loop
+                  (fn []
+                    (.add event-manager refresh-connections)
+                    (.add event-manager refresh-storm-active)
+                    (when @active (storm-conf TASK-REFRESH-POLL-SECS))
+                    ))
+                 (async-loop
+                  (fn [^ArrayList drainer ^TupleSerializer serializer]
+                    (let [felem (.take transfer-queue)]
+                      (.add drainer felem)
+                      (.drainTo transfer-queue drainer))
+                    (read-locked endpoint-socket-lock
+                      (let [node+port->socket @node+port->socket
+                            task->node+port @task->node+port]
+                        (doseq [[task ^Tuple tuple] drainer]
+                          (let [socket (node+port->socket (task->node+port task))
+                                ser-tuple (.serialize serializer tuple)]
+                            (mqvp/virtual-send socket task ser-tuple)
+                            ))
+                        ))
+                    (.clear drainer)
+                    0 )
+                  :args-fn (fn [] [(ArrayList.) (TupleSerializer. storm-conf)]))
+                 heartbeat-thread]
+        _ (log-message "Launching virtual port for " supervisor-id ":" port)
+        virtual-port-shutdown (mqvp/launch-virtual-port! zmq-context
+                                                         (virtual-port-url conf port)
+                                                         :kill-fn (fn [] (halt-process! 11))
+                                                         :valid-ports task-ids)
+        _ (log-message "Launched virtual port for " supervisor-id ":" port)
+        shutdown* (fn []
+                    (log-message "Shutting down worker " storm-id " " supervisor-id " " port)
+                    (reset! active false)
+                    (doseq [task tasks] (.shutdown task))
+                    (doseq [[_ socket] @node+port->socket]
+                      ;; this will do best effort flushing since the linger period
+                      ;; was set on creation
+                      (.close socket))
+                    (virtual-port-shutdown)
+                    (log-message "Terminating zmq context")
+                    (.term zmq-context)
+                    (log-message "Disconnecting from storm cluster state context")
+                    (log-message "Waiting for heartbeat thread to die")
+                    (doseq [t threads]
+                      (.interrupt t)
+                      (.join t))
+                    (.shutdown event-manager)
+                    (.disconnect storm-cluster-state)
+                    (.close cluster-state)
+                    (log-message "Shut down worker " storm-id " " supervisor-id " " port))
+        ret (reify
+             Shutdownable
+             (shutdown
+              [this]
+              (shutdown*))
+             DaemonCommon
+             (waiting? [this]
+                       (and
+                        (.waiting? event-manager)
+                        (every? (memfn waiting?) tasks)
+                        (.sleeping? heartbeat-thread)))
+             )]
+    (log-message "Worker " worker-id " for storm " storm-id " on " supervisor-id ":" port " has finished loading")
+    ret
+    ))
+
+
+
+(defmethod virtual-port-url :local [conf port]
+           (str "ipc://" port ".ipc"))
+
+(defmethod virtual-port-url :distributed [conf port]
+           (str "tcp://*:" port))
+
+(defmethod connect-url :local [conf host port]
+           (str "ipc://" port ".ipc"))
+
+(defmethod connect-url :distributed [conf host port]
+           (str "tcp://" host ":" port))
+
+
+(defn -main [storm-id supervisor-id port-str worker-id]  
+  (let [conf (read-storm-config)]
+    (validate-distributed-mode! conf)
+    (mk-worker conf storm-id supervisor-id (Integer/parseInt port-str) worker-id)))
diff --git a/src/clj/backtype/storm/event.clj b/src/clj/backtype/storm/event.clj
new file mode 100644
index 0000000..08fb12e
--- /dev/null
+++ b/src/clj/backtype/storm/event.clj
@@ -0,0 +1,56 @@
+(ns backtype.storm.event
+  (:use [backtype.storm log util])
+  (:import [backtype.storm.utils Time Utils])
+  (:import [java.util.concurrent LinkedBlockingQueue TimeUnit])
+  )
+
+(defprotocol EventManager
+  (add [this event-fn])
+  (waiting? [this])
+  (shutdown [this]))
+
+(defn event-manager
+  "Creates a thread to respond to events. Any error will cause process to halt"
+  [daemon?]
+  (let [added (atom 0)
+        processed (atom 0)
+        ^LinkedBlockingQueue queue (LinkedBlockingQueue.)
+        running (atom true)
+        runner (Thread.
+                  (fn []
+                    (while @running
+                      (try
+                        (let [r (.take queue)]
+                          (try
+                            (r)
+                            (swap! processed inc)
+                          (catch InterruptedException t
+                            (throw t))
+                          (catch Throwable t
+                            (log-error t "Error when processing event " r)
+                            (halt-process! 20 "Error when processing an event"))
+                            ))
+                      (catch InterruptedException t
+                          (log-message "Event manager interrupted")))
+                          )))]
+    (.setDaemon runner daemon?)
+    (.start runner)
+    (reify
+      EventManager
+      (add [this event-fn]
+        ;; should keep track of total added and processed to know if this is finished yet
+        (when-not @running
+          (throw (RuntimeException. "Cannot add events to a shutdown event manager")))
+        (swap! added inc)
+        (.put queue event-fn)
+        )
+      (waiting? [this]
+        (or (Time/isThreadWaiting runner)
+            (= @processed @added)
+            ))
+      (shutdown [this]
+        (reset! running false)
+        (.interrupt runner)
+        (.join runner)
+        )
+        )))
diff --git a/src/clj/backtype/storm/log.clj b/src/clj/backtype/storm/log.clj
new file mode 100644
index 0000000..3c8eb94
--- /dev/null
+++ b/src/clj/backtype/storm/log.clj
@@ -0,0 +1,11 @@
+(ns backtype.storm.log
+  (:require [clojure.contrib [logging :as log]]))
+
+(defmacro log-message [& args]
+  `(log/info (str ~@args)))
+
+(defmacro log-error [e & args]
+  `(log/error (str ~@args) ~e))
+
+(defmacro log-debug [& args]
+  `(log/debug (str ~@args)))
diff --git a/src/clj/backtype/storm/process_simulator.clj b/src/clj/backtype/storm/process_simulator.clj
new file mode 100644
index 0000000..708e6b0
--- /dev/null
+++ b/src/clj/backtype/storm/process_simulator.clj
@@ -0,0 +1,33 @@
+(ns backtype.storm.process-simulator
+  (:use [backtype.storm log util])
+  )
+
+(def pid-counter (mk-counter))
+
+(def process-map (atom {}))
+
+(def kill-lock (Object.))
+
+(defn register-process [pid shutdownable]
+  (swap! process-map assoc pid shutdownable))
+
+(defn process-handle [pid]
+  (@process-map pid))
+
+(defn all-processes []
+  (vals @process-map))
+
+(defn kill-process [pid]
+  (locking kill-lock ; in case cluster shuts down while supervisor is
+                     ; killing a task
+    (log-message "Killing process " pid)
+    (let [shutdownable (process-handle pid)]
+      (swap! process-map dissoc pid)
+      (when shutdownable
+        (.shutdown shutdownable))
+      )))
+
+(defn kill-all-processes []
+  (doseq [pid (keys @process-map)]
+    (kill-process pid)
+    ))
diff --git a/src/clj/backtype/storm/stats.clj b/src/clj/backtype/storm/stats.clj
new file mode 100644
index 0000000..c579c59
--- /dev/null
+++ b/src/clj/backtype/storm/stats.clj
@@ -0,0 +1,306 @@
+(ns backtype.storm.stats
+  (:import [backtype.storm.generated Nimbus Nimbus$Processor Nimbus$Iface StormTopology ShellComponent
+            NotAliveException AlreadyAliveException InvalidTopologyException GlobalStreamId
+            ClusterSummary TopologyInfo TopologySummary TaskSummary TaskStats TaskSpecificStats
+            SpoutStats BoltStats ErrorInfo SupervisorSummary])
+  (:use [backtype.storm util])
+  (:use [clojure.contrib.seq-utils :only [find-first]])
+  (:use [clojure.contrib.math :only [ceil]]))
+
+;;TODO: consider replacing this with some sort of RRD
+
+(defn curr-time-bucket [^Integer time-secs ^Integer bucket-size-secs]
+  (* bucket-size-secs (unchecked-divide time-secs bucket-size-secs))
+  )
+
+(defrecord RollingWindow [updater merger extractor bucket-size-secs num-buckets buckets])
+
+(defn rolling-window [updater merger extractor bucket-size-secs num-buckets]
+  (RollingWindow. updater merger extractor bucket-size-secs num-buckets {}))
+
+(defn update-rolling-window
+  ([^RollingWindow rw time-secs & args]
+     ;; this is 2.5x faster than using update-in...
+     (let [time-bucket (curr-time-bucket time-secs (:bucket-size-secs rw))
+           buckets (:buckets rw)
+           curr (get buckets time-bucket)           
+           curr (apply (:updater rw) curr args)
+           ]
+       (assoc rw :buckets (assoc buckets time-bucket curr))
+       )))
+
+(defn value-rolling-window [^RollingWindow rw]
+  ((:extractor rw)
+   (let [values (vals (:buckets rw))]
+     (apply (:merger rw) values)
+     )))
+
+(defn cleanup-rolling-window [^RollingWindow rw]
+  (let [buckets (:buckets rw)
+        cutoff (- (current-time-secs)
+                  (* (:num-buckets rw)
+                     (:bucket-size-secs rw)))
+        to-remove (filter #(< % cutoff) (keys buckets))
+        buckets (apply dissoc buckets to-remove)]
+    (assoc rw :buckets buckets)
+    ))
+
+(defn rolling-window-size [^RollingWindow rw]
+  (* (:bucket-size-secs rw) (:num-buckets rw)))
+
+(defrecord RollingWindowSet [updater extractor windows all-time])
+
+(defn rolling-window-set [updater merger extractor num-buckets & bucket-sizes]
+  (RollingWindowSet. updater extractor (dofor [s bucket-sizes] (rolling-window updater merger extractor s num-buckets)) nil)
+  )
+
+(defn update-rolling-window-set
+  ([^RollingWindowSet rws & args]
+     (let [now (current-time-secs)
+           new-windows (dofor [w (:windows rws)]
+                         (apply update-rolling-window w now args))]
+       (assoc rws :windows new-windows :all-time (apply (:updater rws) (:all-time rws) args))
+       )))
+
+(defn cleanup-rolling-window-set
+  ([^RollingWindowSet rws]
+     (let [windows (:windows rws)]
+       (assoc rws :windows (map cleanup-rolling-window windows))
+       )))
+
+(defn value-rolling-window-set [^RollingWindowSet rws]
+  (merge
+   (into {}
+         (for [w (:windows rws)]
+           {(rolling-window-size w) (value-rolling-window w)}
+           ))
+   {:all-time ((:extractor rws) (:all-time rws))}))
+
+(defn- incr-val
+  ([amap key]
+     (incr-val amap key 1))
+  ([amap key amt]
+     (let [val (get amap key (long 0))]
+       (assoc amap key (+ val amt))
+       )))
+
+(defn- update-avg [curr val]
+  (if curr
+    [(+ (first curr) val) (inc (second curr))]
+    [val (long 1)]
+    ))
+
+(defn- merge-avg [& avg]
+  [(apply + (map first avg))
+   (apply + (map second avg))
+   ])
+
+(defn- extract-avg [pair]
+  (double (/ (first pair) (second pair))))
+
+(defn- update-keyed-avg [amap key val]
+  (assoc amap key (update-avg (get amap key) val)))
+
+(defn- merge-keyed-avg [& vals]
+  (apply merge-with merge-avg vals))
+
+(defn- extract-keyed-avg [vals]
+  (map-val extract-avg vals))
+
+(defn- counter-extract [v]
+  (if v v {}))
+
+(defn keyed-counter-rolling-window-set [num-buckets & bucket-sizes]
+  (apply rolling-window-set incr-val (partial merge-with +) counter-extract num-buckets bucket-sizes))
+
+(defn avg-rolling-window-set [num-buckets & bucket-sizes]
+  (apply rolling-window-set update-avg merge-avg extract-avg num-buckets bucket-sizes)
+  )
+
+(defn keyed-avg-rolling-window-set [num-buckets & bucket-sizes]
+  (apply rolling-window-set update-keyed-avg merge-keyed-avg extract-keyed-avg num-buckets bucket-sizes))
+
+;; (defn choose-bucket [val buckets]
+;;   (let [ret (find-first #(<= val %) buckets)]
+;;     (if ret
+;;       ret
+;;       (* 10 (first buckets)))
+;;     ))
+
+;; ;; buckets must be between 1 and 9
+;; (defn to-proportional-bucket
+;;   "Maps to a bucket in the values order of magnitude. So if buckets are [1 2 5],
+;;    3 -> 5
+;;    7 -> 10
+;;    1234 -> 2000
+;;    etc."
+;;   [val buckets]
+;;   (cond (= 0 val) 0
+;;         (between? val 1 9) (choose-bucket val buckets)
+;;         :else (* 10 (to-proportional-bucket (ceil (/ val 10))
+;;                                             buckets))))
+
+(def COMMON-FIELDS [:emitted :transferred])
+(defrecord CommonStats [emitted transferred rate])
+
+(def BOLT-FIELDS [:acked :failed :process-latencies])
+;;acked and failed count individual tuples
+(defrecord BoltTaskStats [common acked failed process-latencies])
+
+(def SPOUT-FIELDS [:acked :failed :complete-latencies])
+;;acked and failed count tuple completion
+(defrecord SpoutTaskStats [common acked failed complete-latencies])
+
+(def NUM-STAT-BUCKETS 20)
+;; 10 minutes, 3 hours, 1 day
+(def STAT-BUCKETS [30 540 4320])
+
+(defn- mk-common-stats [rate]
+  (CommonStats. (atom (apply keyed-counter-rolling-window-set NUM-STAT-BUCKETS STAT-BUCKETS))
+                (atom (apply keyed-counter-rolling-window-set NUM-STAT-BUCKETS STAT-BUCKETS))
+                rate
+                ))
+
+(defn mk-bolt-stats [rate]
+  (BoltTaskStats. (mk-common-stats rate)
+                  (atom (apply keyed-counter-rolling-window-set NUM-STAT-BUCKETS STAT-BUCKETS))
+                  (atom (apply keyed-counter-rolling-window-set NUM-STAT-BUCKETS STAT-BUCKETS))
+                  (atom (apply keyed-avg-rolling-window-set NUM-STAT-BUCKETS STAT-BUCKETS))
+                  ))
+
+(defn mk-spout-stats [rate]
+  (SpoutTaskStats. (mk-common-stats rate)
+                   (atom (apply keyed-counter-rolling-window-set NUM-STAT-BUCKETS STAT-BUCKETS))
+                   (atom (apply keyed-counter-rolling-window-set NUM-STAT-BUCKETS STAT-BUCKETS))
+                   (atom (apply keyed-avg-rolling-window-set NUM-STAT-BUCKETS STAT-BUCKETS))
+                   ))
+
+(defmacro update-task-stat! [stats path & args]
+  (let [path (collectify path)]
+    `(swap! (-> ~stats ~@path) update-rolling-window-set ~@args)
+    ))
+
+(defmacro stats-rate [stats]
+  `(-> ~stats :common :rate))
+
+(defn emitted-tuple! [stats stream]
+  (update-task-stat! stats [:common :emitted] stream (stats-rate stats)))
+
+(defn transferred-tuples! [stats stream amt]
+  (update-task-stat! stats [:common :transferred] stream (* (stats-rate stats) amt)))
+
+(defn bolt-acked-tuple! [^BoltTaskStats stats component stream latency-ms]
+  (let [key [component stream]]
+    (update-task-stat! stats :acked key (stats-rate stats))
+    (update-task-stat! stats :process-latencies key latency-ms)
+    ))
+
+(defn bolt-failed-tuple! [^BoltTaskStats stats component stream latency-ms]
+  (let [key [component stream]]
+    (update-task-stat! stats :failed key (stats-rate stats))
+    ))
+
+(defn spout-acked-tuple! [^SpoutTaskStats stats stream latency-ms]
+  (update-task-stat! stats :acked stream (stats-rate stats))
+  (update-task-stat! stats :complete-latencies stream latency-ms)
+  )
+
+(defn spout-failed-tuple! [^SpoutTaskStats stats stream latency-ms]
+  (update-task-stat! stats :failed stream (stats-rate stats))
+  )
+
+(defn- cleanup-stat! [stat]
+  (swap! stat cleanup-rolling-window-set))
+
+(defn- cleanup-common-stats! [^CommonStats stats]
+  (doseq [f COMMON-FIELDS]
+    (cleanup-stat! (f stats))
+    ))
+
+(defn cleanup-bolt-stats! [^BoltTaskStats stats]
+  (cleanup-common-stats! (:common stats))
+  (doseq [f BOLT-FIELDS]
+    (cleanup-stat! (f stats))
+    ))
+
+(defn cleanup-spout-stats! [^SpoutTaskStats stats]
+  (cleanup-common-stats! (:common stats))
+  (doseq [f SPOUT-FIELDS]
+    (cleanup-stat! (f stats))
+    ))
+
+(defn- value-stats [stats fields]
+  (into
+   {}
+   (dofor [f fields]
+          [f (value-rolling-window-set @(f stats))]
+          )))
+
+(defn- value-common-stats [^CommonStats stats]
+  (merge
+   (value-stats stats COMMON-FIELDS)
+   {:rate (:rate stats)}))
+
+(defn value-bolt-stats! [^BoltTaskStats stats]
+  (cleanup-bolt-stats! stats)
+  (merge (value-common-stats (:common stats))
+         (value-stats stats BOLT-FIELDS)
+         {:type :bolt}))
+
+(defn value-spout-stats! [^SpoutTaskStats stats]
+  (cleanup-spout-stats! stats)
+  (merge (value-common-stats (:common stats))
+         (value-stats stats SPOUT-FIELDS)
+         {:type :spout}))
+
+
+(defmulti render-stats! class-selector)
+
+(defmethod render-stats! SpoutTaskStats [stats]
+  (value-spout-stats! stats))
+
+(defmethod render-stats! BoltTaskStats [stats]
+  (value-bolt-stats! stats))
+
+(defmulti thriftify-specific-stats :type)
+
+(defn window-set-converter
+  ([stats key-fn]
+     ;; make the first key a string,
+     (into {}
+           (for [[k v] stats]
+             [(str k)
+              (into {}
+                    (for [[k2 v2] v]
+                      [(key-fn k2) v2]))]
+             )
+           ))
+  ([stats]
+     (window-set-converter stats identity)))
+
+(defn to-global-stream-id [[component stream]]
+  (GlobalStreamId. component stream)
+  )
+
+(defmethod thriftify-specific-stats :bolt
+  [stats]
+  (TaskSpecificStats/bolt
+   (BoltStats. (window-set-converter (:acked stats) to-global-stream-id)
+               (window-set-converter (:failed stats) to-global-stream-id)
+               (window-set-converter (:process-latencies stats) to-global-stream-id)))
+  )
+
+(defmethod thriftify-specific-stats :spout
+  [stats]
+  (TaskSpecificStats/spout
+   (SpoutStats. (window-set-converter (:acked stats))
+                (window-set-converter (:failed stats))
+                (window-set-converter (:complete-latencies stats)))
+   ))
+
+(defn thriftify-task-stats [stats]
+  (let [specific-stats (thriftify-specific-stats stats)]
+    (TaskStats. (window-set-converter (:emitted stats))
+                (window-set-converter (:transferred stats))
+                specific-stats)
+    ))
diff --git a/src/clj/backtype/storm/testing.clj b/src/clj/backtype/storm/testing.clj
new file mode 100644
index 0000000..42058f3
--- /dev/null
+++ b/src/clj/backtype/storm/testing.clj
@@ -0,0 +1,432 @@
+(ns backtype.storm.testing
+  (:require [backtype.storm.daemon
+             [nimbus :as nimbus]
+             [supervisor :as supervisor]
+             [common :as common]
+             [task :as task]])
+  (:require [backtype.storm [process-simulator :as psim]])
+  (:import [org.apache.commons.io FileUtils])
+  (:import [java.io File])
+  (:import [backtype.storm.utils Time Utils])
+  (:import [backtype.storm.tuple Fields])
+  (:import [backtype.storm.generated GlobalStreamId Bolt])
+  (:import [backtype.storm.testing FeederSpout FixedTupleSpout FixedTuple TupleCaptureBolt
+            SpoutTracker BoltTracker TrackerAggregator])
+  (:require [backtype.storm [zookeeper :as zk]])
+  (:use [clojure.contrib.def :only [defnk]])
+  (:use [clojure.contrib.seq :only [find-first]])
+  (:use [backtype.storm cluster util thrift config log]))
+
+(defn feeder-spout [fields]
+  (FeederSpout. (Fields. fields)))
+
+(defn local-temp-path []
+  (str (System/getProperty "java.io.tmpdir") "/" (uuid)))
+
+(defn delete-all [paths]
+  (dorun
+    (for [t paths]
+      (if (.exists (File. t))
+        (FileUtils/forceDelete (File. t))
+        ))))
+
+(defmacro with-local-tmp [[& tmp-syms] & body]
+  (let [tmp-paths (mapcat (fn [t] [t `(local-temp-path)]) tmp-syms)]
+    `(let [~@tmp-paths]
+      (try
+        ~@body
+      (finally
+       (delete-all ~(vec tmp-syms)))
+      ))
+    ))
+
+(defn start-simulating-time! []
+  (Time/startSimulating))
+
+(defn stop-simulating-time! []
+  (Time/stopSimulating))
+
+(defmacro with-simulated-time [& body]
+  `(do
+     (start-simulating-time!)
+     (let [ret# (do ~@body)]
+       (stop-simulating-time!)
+       ret#
+       )))
+
+(defn advance-time-ms! [ms]
+  (Time/advanceTime ms))
+
+(defn advance-time-secs! [secs]
+  (advance-time-ms! (* (long secs) 1000)))
+
+
+(defnk add-supervisor [cluster-map :ports 2 :conf {} :id nil]
+  (let [tmp-dir (local-temp-path)
+        port-ids (if (sequential? ports) ports (doall (repeatedly ports (:port-counter cluster-map))))
+        supervisor-conf (merge (:daemon-conf cluster-map)
+                               conf
+                               {STORM-LOCAL-DIR tmp-dir
+                                SUPERVISOR-SLOTS-PORTS port-ids
+                               })
+        id-fn (if id (fn [] id) supervisor/generate-supervisor-id)
+        daemon (with-var-roots [supervisor/generate-supervisor-id id-fn] (supervisor/mk-supervisor supervisor-conf))]
+    (swap! (:supervisors cluster-map) conj daemon)
+    (swap! (:tmp-dirs cluster-map) conj tmp-dir)
+    daemon
+    ))
+
+;; returns map containing cluster info
+;; local dir is always overridden in maps
+;; can customize the supervisors (except for ports) by passing in map for :supervisors parameter
+;; if need to customize amt of ports more, can use add-supervisor calls afterwards
+(defnk mk-local-storm-cluster [:supervisors 2 :ports-per-supervisor 3 :daemon-conf {}]
+  (let [zk-port 2181
+        daemon-conf (merge (read-storm-config)
+                           {TOPOLOGY-SKIP-MISSING-SERIALIZATIONS true
+                            ZMQ-LINGER-MILLIS 0
+                            }
+                           daemon-conf
+                           {STORM-CLUSTER-MODE "local"
+                            STORM-ZOOKEEPER-PORT zk-port})
+        nimbus-tmp (local-temp-path)
+        zk-tmp (local-temp-path)
+        zk-handle (zk/mk-inprocess-zookeeper zk-tmp zk-port)
+        port-counter (mk-counter)
+        nimbus (nimbus/service-handler
+                (assoc daemon-conf STORM-LOCAL-DIR nimbus-tmp))
+        cluster-map {:nimbus nimbus
+                     :port-counter port-counter
+                     :daemon-conf daemon-conf
+                     :supervisors (atom [])
+                     :state (mk-distributed-cluster-state daemon-conf)
+                     :storm-cluster-state (mk-storm-cluster-state daemon-conf)
+                     :tmp-dirs (atom [nimbus-tmp zk-tmp])
+                     :zookeeper zk-handle}
+        supervisor-confs (if (sequential? supervisors)
+                           supervisors
+                           (repeat supervisors {}))]
+    (doseq [sc supervisor-confs]
+      (add-supervisor cluster-map :ports ports-per-supervisor :conf sc))
+    cluster-map
+    ))
+
+(defn get-supervisor [cluster-map supervisor-id]
+  (let [finder-fn #(= (.get-id %) supervisor-id)]
+    (find-first finder-fn @(:supervisors cluster-map))
+    ))
+
+(defn kill-supervisor [cluster-map supervisor-id]
+  (let [finder-fn #(= (.get-id %) supervisor-id)
+        supervisors @(:supervisors cluster-map)
+        sup (find-first finder-fn
+                        supervisors)]
+    ;; tmp-dir will be taken care of by shutdown
+    (reset! (:supervisors cluster-map) (remove-first finder-fn supervisors))
+    (.shutdown sup)
+    ))
+
+(defn kill-local-storm-cluster [cluster-map]
+  (.shutdown (:nimbus cluster-map))
+  (.close (:state cluster-map))
+  (.disconnect (:storm-cluster-state cluster-map))
+  (doseq [s @(:supervisors cluster-map)]
+    (.shutdown-all-workers s)
+    (supervisor/kill-supervisor s))
+  (psim/kill-all-processes)
+  (log-message "Shutting down in process zookeeper")
+  (zk/shutdown-inprocess-zookeeper (:zookeeper cluster-map))
+  (log-message "Done shutting down in process zookeeper")
+  (doseq [t @(:tmp-dirs cluster-map)]
+    (log-message "Deleting temporary path " t)
+    (rmr t)
+    ))
+
+
+(defn wait-until-cluster-waiting
+  "Wait until the cluster is idle. Should be used with time simulation."
+  [cluster-map]
+  ;; wait until all workers, supervisors, and nimbus is waiting
+  (let [supervisors @(:supervisors cluster-map)
+        workers (filter (partial satisfies? common/DaemonCommon) (psim/all-processes))
+        daemons (concat
+                  [(:nimbus cluster-map)]
+                  supervisors
+                  workers) ; because a worker may already be dead
+        ]
+    (while (not (every? (memfn waiting?) daemons))
+      (Thread/sleep 10)
+      )))
+
+(defn advance-cluster-time
+  ([cluster-map secs increment-secs]
+    (loop [left secs]
+      (when (> left 0)
+        (let [diff (min left increment-secs)]
+          (advance-time-secs! diff)
+          (wait-until-cluster-waiting cluster-map)
+          (recur (- left diff))
+          ))))
+  ([cluster-map secs]
+    (advance-cluster-time cluster-map secs 1)
+    ))
+
+(defmacro with-local-cluster [[cluster-sym & args] & body]
+  `(let [~cluster-sym (mk-local-storm-cluster ~@args)]
+     (try
+       ~@body
+     (catch Throwable t#
+       (log-error t# "Error in cluster")
+       )
+     (finally
+       (kill-local-storm-cluster ~cluster-sym)))
+       ))
+
+(defmacro with-simulated-time-local-cluster [& args]
+  `(with-simulated-time
+    (with-local-cluster ~@args)))
+
+(defmacro with-inprocess-zookeeper [port & body]
+  `(with-local-tmp [tmp#]
+     (let [zks# (zk/mk-inprocess-zookeeper tmp# ~port)]
+       (try
+         ~@body
+       (finally
+         (zk/shutdown-inprocess-zookeeper zks#)
+         ))
+       )))
+
+(defn submit-local-topology [nimbus storm-name conf topology]
+  (.submitTopology nimbus storm-name nil (to-json conf) topology))
+
+(defn submit-mocked-assignment [nimbus storm-name conf topology task->component task->node+port]
+  (with-var-roots [nimbus/mk-task-component-assignments (fn [& ignored] task->component)
+                   nimbus/compute-new-task->node+port (fn [& ignored] task->node+port)]
+    (submit-local-topology nimbus storm-name conf topology)
+    ))
+
+(defn mk-capture-launch-fn [capture-atom]
+  (fn [conf storm-id supervisor-id port worker-id _]
+    (let [existing (get @capture-atom [supervisor-id port] [])]
+      (swap! capture-atom assoc [supervisor-id port] (conj existing storm-id))
+      )))
+
+(defn find-worker-id [supervisor-conf port]
+  (let [supervisor-state (supervisor-state supervisor-conf)
+        worker->port (.get supervisor-state common/LS-APPROVED-WORKERS)]
+    (first ((reverse-map worker->port) port))
+    ))
+
+(defn find-worker-port [supervisor-conf worker-id]
+  (let [supervisor-state (supervisor-state supervisor-conf)
+        worker->port (.get supervisor-state common/LS-APPROVED-WORKERS)
+        ]
+    (worker->port worker-id)
+    ))
+
+(defn mk-capture-shutdown-fn [capture-atom]
+  (let [existing-fn supervisor/shutdown-worker]
+    (fn [conf supervisor-id worker-id worker-thread-pids-atom]
+      (let [port (find-worker-port conf worker-id)
+            existing (get @capture-atom [supervisor-id port] 0)]      
+        (swap! capture-atom assoc [supervisor-id port] (inc existing))
+        (existing-fn conf supervisor-id worker-id worker-thread-pids-atom)
+        ))))
+
+(defmacro capture-changed-workers [& body]
+  `(let [launch-captured# (atom {})
+         shutdown-captured# (atom {})]
+    (with-var-roots [supervisor/launch-worker (mk-capture-launch-fn launch-captured#)
+                     supervisor/shutdown-worker (mk-capture-shutdown-fn shutdown-captured#)]
+      ~@body
+      {:launched @launch-captured#
+       :shutdown @shutdown-captured#}
+      )))
+
+(defmacro capture-launched-workers [& body]
+  `(:launched (capture-changed-workers ~@body)))
+
+(defmacro capture-shutdown-workers [& body]
+  `(:shutdown (capture-changed-workers ~@body)))
+
+(defnk aggregated-stat [cluster-map storm-name stat-key :component-ids nil]
+  (let [state (:storm-cluster-state cluster-map)
+        storm-id (common/get-storm-id state storm-name)
+        component->tasks (reverse-map
+                          (common/storm-task-info
+                           state
+                           storm-id))
+        component->tasks (if component-ids
+                           (select-keys component->tasks component-ids)
+                           component->tasks)
+        task-ids (apply concat (vals component->tasks))
+        heartbeats (dofor [id task-ids] (.task-heartbeat state storm-id id))
+        stats (dofor [hb heartbeats] (if hb (stat-key (:stats hb)) 0))]
+    (reduce + stats)
+    ))
+
+(defn emitted-spout-tuples [cluster-map topology storm-name]
+  (aggregated-stat cluster-map
+                   storm-name
+                   :emitted
+                   :component-ids (keys (.get_spouts topology))))
+
+(defn transferred-tuples [cluster-map storm-name]
+  (aggregated-stat cluster-map storm-name :transferred))
+
+(defn acked-tuples [cluster-map storm-name]
+  (aggregated-stat cluster-map storm-name :acked))
+
+(defn simulate-wait [cluster-map]
+  (if (Time/isSimulating)
+    (advance-cluster-time cluster-map 10)
+    (Thread/sleep 100)
+    ))
+
+
+;; TODO: mock-sources needs to be able to mock out state spouts as well
+(defnk complete-topology [cluster-map topology :mock-sources {} :storm-conf {}]
+  (let [storm-name (str "topologytest-" (uuid))
+        state (:storm-cluster-state cluster-map)
+        spouts (.get_spouts topology)
+        bolts (.get_bolts topology)
+        replacements (map-val (fn [v]
+                                (FixedTupleSpout.
+                                 (for [tup v]
+                                   (if (map? tup)
+                                     (FixedTuple. (:stream tup) (:values tup))
+                                     tup))))
+                              mock-sources)
+        all-ids (concat (keys spouts) (keys bolts))
+        all-streams (apply concat
+                           (for [[id spec] (merge (clojurify-structure spouts) (clojurify-structure bolts))]
+                             (for [[stream _] (.. spec get_common get_streams)]
+                               (GlobalStreamId. id stream))))
+        max-id (apply max all-ids)
+        capturer (TupleCaptureBolt. storm-name)
+        ]
+    (doseq [[id spout] replacements]
+      (let [spout-spec (get spouts id)]
+        (.set_spout_object spout-spec (serialize-component-object spout))
+        ))
+    (doseq [[_ spout-spec] (clojurify-structure spouts)]
+      (when-not (instance? FixedTupleSpout (deserialized-component-object (.get_spout_object spout-spec)))
+        (throw (RuntimeException. "Cannot complete topology unless every spout is a FixedTupleSpout (or mocked to be)"))
+        ))
+    
+    (.set_bolts topology
+                (assoc (clojurify-structure bolts)
+                  (inc max-id)
+                  (Bolt.
+                   (into {} (for [id all-streams] [id (mk-global-grouping)]))
+                   (serialize-component-object capturer)
+                   (mk-plain-component-common {} nil))
+                  ))
+    (submit-local-topology (:nimbus cluster-map) storm-name storm-conf topology)
+
+    
+    
+    (let [num-source-tuples (reduce +
+                                    (for [[_ spout-spec] spouts]
+                                      (-> (.get_spout_object spout-spec)
+                                          deserialized-component-object
+                                          .getSourceTuples
+                                          count)
+                                      ))
+          storm-id (common/get-storm-id state storm-name)]
+      (while (< (+ (FixedTupleSpout/getNumAcked storm-id)
+                   (FixedTupleSpout/getNumFailed storm-id))
+                num-source-tuples)
+        (simulate-wait cluster-map))
+
+      (.killTopology (:nimbus cluster-map) storm-name)
+      (while (.assignment-info state storm-id nil)
+        (simulate-wait cluster-map))
+      (FixedTupleSpout/clear storm-id))
+
+    (.getResults capturer)
+    ))
+
+(defn read-tuples
+  ([results component-id stream-id]
+     (let [fixed-tuples (get results component-id [])]
+       (mapcat
+        (fn [ft]
+          (if (= stream-id (. ft stream))
+            [(vec (. ft values))]))
+        fixed-tuples)
+       ))
+  ([results component-id]
+     (read-tuples results component-id Utils/DEFAULT_STREAM_ID)
+     ))
+
+(defn ms= [& args]  
+  (apply = (map multi-set args)))
+
+(def TRACKER-BOLT-ID 9999)
+
+(defn mk-tracked-topology
+  "Spouts are of form [spout & options], bolts are of form [inputs bolt & options]"
+  [spouts-map bolts-map]
+  (let [tracker (TrackerAggregator.)
+        spouts-map (into {}
+                         (for [[id [spout & options]] spouts-map]
+                           [id
+                            (apply mk-spout-spec
+                                   (SpoutTracker. spout)
+                                   options)]))
+        bolts-map (into {}
+                        (for [[id [inputs bolt & options]] bolts-map]
+                          [id
+                           (apply mk-bolt-spec
+                                  inputs
+                                  (BoltTracker. bolt)
+                                  options)]))
+        all-ids (concat (keys spouts-map) (keys bolts-map))
+        tracker-inputs (into {}
+                             (for [key all-ids]
+                               [[key TrackerAggregator/TRACK_STREAM] :global]
+                               ))
+        bolts-map (assoc bolts-map
+                    TRACKER-BOLT-ID
+                    (mk-bolt-spec
+                     tracker-inputs
+                     tracker
+                     ))
+        ]
+    {:topology (mk-topology spouts-map bolts-map)
+     :last-spout-emit (atom 0)
+     :tracker tracker
+     }))
+
+(defmacro with-tracked-cluster [cluster-args & body]
+  `(with-var-roots [task/outbound-components (let [old# task/outbound-components]
+                                               (fn [& args#]
+                                                 (merge (apply old# args#)
+                                                        {TrackerAggregator/TRACK_STREAM
+                                                         {TRACKER-BOLT-ID (fn [& args#] 0)}}
+                                                        )))
+                    task/mk-acker-bolt (let [old# task/mk-acker-bolt]
+                                         (fn [& args#]
+                                           (BoltTracker. (apply old# args#))
+                                           ))
+                    ]
+     (with-local-cluster ~cluster-args
+       ~@body
+       )))
+
+(defn tracked-wait
+  "Waits until topology is idle and 'amt' more tuples have been emitted by spouts."
+  ([tracked-topology]
+     (tracked-wait tracked-topology 1))
+  ([tracked-topology amt]
+      (let [target (+ amt @(:last-spout-emit tracked-topology))
+            tracker (:tracker tracked-topology)
+            waiting? (fn []
+                       (or (not= target (.getSpoutEmitted tracker))
+                           (not= (.getTransferred tracker) (.getProcessed tracker))
+                           ))]
+        (while (waiting?)
+          (Thread/sleep 5))
+        (reset! (:last-spout-emit tracked-topology) target)
+        )))
diff --git a/src/clj/backtype/storm/thrift.clj b/src/clj/backtype/storm/thrift.clj
new file mode 100644
index 0000000..14bbd76
--- /dev/null
+++ b/src/clj/backtype/storm/thrift.clj
@@ -0,0 +1,168 @@
+(ns backtype.storm.thrift
+  (:import [backtype.storm.generated Grouping Nimbus StormTopology Bolt Nimbus$Client Nimbus$Iface ComponentCommon Grouping$_Fields SpoutSpec NullStruct StreamInfo GlobalStreamId ComponentObject ComponentObject$_Fields ShellComponent])
+  (:import [backtype.storm.utils Utils])
+  (:import [backtype.storm Constants])
+  (:import [backtype.storm.task CoordinatedBolt CoordinatedBolt$SourceArgs])
+  (:import [backtype.storm.topology OutputFieldsGetter IBasicBolt BasicBoltExecutor])
+  (:import [org.apache.thrift.protocol TBinaryProtocol TProtocol])
+  (:import [org.apache.thrift.transport TTransport TFramedTransport TSocket])
+  (:use [backtype.storm util])
+  (:use [clojure.contrib.def :only [defnk]])
+  )
+
+(def grouping-constants
+  {Grouping$_Fields/FIELDS :fields
+   Grouping$_Fields/SHUFFLE :shuffle
+   Grouping$_Fields/ALL :all
+   Grouping$_Fields/NONE :none
+   Grouping$_Fields/DIRECT :direct
+  })
+
+(defn grouping-type [^Grouping grouping]
+  (grouping-constants (.getSetField grouping)))
+
+(defn field-grouping [^Grouping grouping]
+  (when-not (= (grouping-type grouping) :fields)
+    (throw (IllegalArgumentException. "Tried to get grouping fields from non fields grouping")))
+  (.get_fields grouping))
+
+(defn global-grouping? [^Grouping grouping]
+  (and (= :fields (grouping-type grouping))
+       (empty? (field-grouping grouping))
+       ))
+
+(defn parallelism-hint [^ComponentCommon component-common]
+  (let [phint (.get_parallelism_hint component-common)]
+    (if (= phint 0) 1 phint)
+    ))
+
+(defn nimbus-client-and-conn [host port]
+  (let [transport (TFramedTransport. (TSocket. host port))
+        prot (TBinaryProtocol. transport)
+        client (Nimbus$Client. prot)]
+        (.open transport)
+        [client transport] ))
+
+(defmacro with-nimbus-connection [[client-sym host port] & body]
+  `(let [[^Nimbus$Client ~client-sym ^TTransport conn#] (nimbus-client-and-conn ~host ~port)]
+      (try
+        ~@body
+      (finally (.close conn#)))
+      ))
+
+(defn mk-component-common [component parallelism-hint]
+  (let [getter (OutputFieldsGetter.)
+        _ (.declareOutputFields component getter)
+        ret (ComponentCommon. (.getFieldsDeclaration getter))]
+    (when parallelism-hint
+      (.set_parallelism_hint ret parallelism-hint))
+    ret
+    ))
+
+(defn direct-output-fields [fields]
+  (StreamInfo. fields true))
+
+(defn mk-output-spec [output-spec]
+  (let [output-spec (if (map? output-spec) output-spec {Utils/DEFAULT_STREAM_ID output-spec})]
+    (map-val
+      (fn [out]
+        (if (instance? StreamInfo out)
+          out
+          (StreamInfo. out false)
+          ))
+      output-spec
+      )))
+
+(defn mk-plain-component-common [output-spec parallelism-hint]
+  (let [ret (ComponentCommon. (mk-output-spec output-spec))]
+    (when parallelism-hint
+      (.set_parallelism_hint ret parallelism-hint))
+    ret
+    ))
+
+(defnk mk-spout-spec [spout :parallelism-hint nil]
+  (SpoutSpec. (ComponentObject/serialized_java (Utils/serialize spout))
+              (mk-component-common spout parallelism-hint)
+              (.isDistributed spout))
+  )
+
+(defn mk-shuffle-grouping []
+  (Grouping/shuffle (NullStruct.)))
+
+(defn mk-fields-grouping [fields]
+  (Grouping/fields fields))
+
+(defn mk-global-grouping []
+  (mk-fields-grouping []))
+
+(defn mk-direct-grouping []
+  (Grouping/direct (NullStruct.)))
+
+(defn mk-all-grouping []
+  (Grouping/all (NullStruct.)))
+
+(defn mk-none-grouping []
+  (Grouping/none (NullStruct.)))
+
+(defn deserialized-component-object [^ComponentObject obj]
+  (when (not= (.getSetField obj) ComponentObject$_Fields/SERIALIZED_JAVA)
+    (throw (RuntimeException. "Cannot deserialize non-java-serialized object")))
+  (Utils/deserialize (.get_serialized_java obj))
+  )
+
+(defn serialize-component-object [obj]
+  (ComponentObject/serialized_java (Utils/serialize obj)))
+
+(defn mk-grouping [grouping-spec]
+  (cond (nil? grouping-spec) (mk-none-grouping)
+        (instance? Grouping grouping-spec) grouping-spec
+        (sequential? grouping-spec) (mk-fields-grouping grouping-spec)
+        (= grouping-spec :shuffle) (mk-shuffle-grouping)
+        (= grouping-spec :none) (mk-none-grouping)
+        (= grouping-spec :all) (mk-all-grouping)
+        (= grouping-spec :global) (mk-global-grouping)
+        (= grouping-spec :direct) (mk-direct-grouping)
+        true (throw (IllegalArgumentException. (str grouping-spec " is not a valid grouping")))
+        ))
+
+(defn- mk-inputs [inputs]
+  (into {}
+    (for [[stream-id grouping-spec] inputs]
+      [(if (sequential? stream-id)
+         (GlobalStreamId. (first stream-id) (second stream-id))
+         (GlobalStreamId. stream-id (Utils/DEFAULT_STREAM_ID)))
+       (mk-grouping grouping-spec)]
+      )))
+
+(defnk mk-bolt-spec [inputs bolt :parallelism-hint nil]
+  (let [bolt (if (instance? IBasicBolt bolt) (BasicBoltExecutor. bolt) bolt)]
+    (Bolt.
+     (mk-inputs inputs)
+     (ComponentObject/serialized_java (Utils/serialize bolt))
+     (mk-component-common bolt parallelism-hint)
+     )))
+
+(defnk mk-shell-bolt-spec [inputs command script output-spec :parallelism-hint nil]
+  (Bolt.
+    (mk-inputs inputs)
+    (ComponentObject/shell (ShellComponent. command script))
+    (mk-plain-component-common output-spec parallelism-hint)
+    ))
+
+(defn mk-topology
+  ([spout-map bolt-map]
+     (StormTopology. spout-map bolt-map {}))
+  ([spout-map bolt-map state-spout-map]
+     (StormTopology. spout-map bolt-map state-spout-map)))
+
+(defnk coordinated-bolt [bolt :type nil :all-out false]
+  (let [source (condp = type
+                   nil nil
+                   :all (CoordinatedBolt$SourceArgs/all)
+                   :single (CoordinatedBolt$SourceArgs/single))]
+    (CoordinatedBolt. bolt source all-out)
+    ))
+
+(def COORD-STREAM Constants/COORDINATED_STREAM_ID)
+
+
diff --git a/src/clj/backtype/storm/tuple.clj b/src/clj/backtype/storm/tuple.clj
new file mode 100644
index 0000000..ce30ce9
--- /dev/null
+++ b/src/clj/backtype/storm/tuple.clj
@@ -0,0 +1,12 @@
+(ns backtype.storm.tuple
+  (:use [backtype.storm bootstrap])
+  )
+
+(bootstrap)
+
+(defn tuple-hash-code [^Tuple tuple]
+  (.hashCode (.getValues tuple))
+  )
+
+(defn list-hash-code [^List alist]
+  (.hashCode alist))
diff --git a/src/clj/backtype/storm/ui/core.clj b/src/clj/backtype/storm/ui/core.clj
new file mode 100644
index 0000000..2c9248a
--- /dev/null
+++ b/src/clj/backtype/storm/ui/core.clj
@@ -0,0 +1,682 @@
+(ns backtype.storm.ui.core
+  (:use compojure.core)
+  (:use [hiccup core page-helpers])
+  (:use [backtype.storm config util])
+  (:use [backtype.storm.ui helpers])
+  (:use [backtype.storm.daemon [common :only [ACKER-COMPONENT-ID]]])
+  (:use [clojure.contrib.def :only [defnk]])
+  (:use [clojure.contrib.seq-utils :only [find-first]])
+  (:use [ring.adapter.jetty :only [run-jetty]])
+  (:import [backtype.storm.generated TaskSpecificStats
+            TaskStats TaskSummary TopologyInfo SpoutStats BoltStats
+            ErrorInfo ClusterSummary SupervisorSummary TopologySummary
+            Nimbus$Client StormTopology GlobalStreamId])
+  (:require [compojure.route :as route]
+            [compojure.handler :as handler]
+            [backtype.storm [thrift :as thrift]])
+  (:gen-class))
+
+(def *STORM-CONF* (read-storm-config))
+
+(defmacro with-nimbus [nimbus-sym & body]
+  `(thrift/with-nimbus-connection [~nimbus-sym "localhost" (*STORM-CONF* NIMBUS-THRIFT-PORT)]
+     ~@body
+     ))
+
+(defn get-filled-stats [summs]
+  (->> summs
+       (map #(.get_stats ^TaskSummary %))
+       (filter not-nil?)))
+
+(defn ui-template [body]
+  (html
+   [:head
+    [:title "Storm UI"]
+    (include-css "/css/bootstrap-1.1.0.css")
+    (include-js "/js/jquery-1.6.2.min.js")
+    (include-js "/js/jquery.tablesorter.min.js")
+    ]
+   [:script "$.tablesorter.addParser({ 
+        id: 'stormtimestr', 
+        is: function(s) { 
+            return false; 
+        }, 
+        format: function(s) {
+            if(s.search('All time')!=-1) {
+              return 1000000000;
+            }
+            var total = 0;
+            $.each(s.split(' '), function(i, v) {
+              var amt = parseInt(v);
+              if(v.search('ms')!=-1) {
+                total += amt;
+              } else if (v.search('s')!=-1) {
+                total += amt * 1000;
+              } else if (v.search('m')!=-1) {
+                total += amt * 1000 * 60;
+              } else if (v.search('h')!=-1) {
+                total += amt * 1000 * 60 * 60;
+              } else if (v.search('d')!=-1) {
+                total += amt * 1000 * 60 * 60 * 24;
+              }
+            });
+            return total;
+        }, 
+        type: 'numeric' 
+    }); "]
+   [:body
+    [:h1 (link-to "/" "Storm UI")]
+    (seq body)
+    ]))
+
+(defn cluster-summary-table [^ClusterSummary summ]
+  (let [sups (.get_supervisors summ)
+        used-slots (reduce + (map #(.get_num_used_workers ^SupervisorSummary %) sups))
+        total-slots (reduce + (map #(.get_num_workers ^SupervisorSummary %) sups))
+        free-slots (- total-slots used-slots)
+        total-tasks (->> (.get_topologies summ)
+                         (map #(.get_num_tasks ^TopologySummary %))
+                         (reduce +))]
+    (table ["Nimbus uptime" "Supervisors" "Used slots" "Free slots" "Total slots" "Running tasks"]
+           [[(pretty-uptime-sec (.get_nimbus_uptime_secs summ))
+             (count sups)
+             used-slots
+             free-slots
+             total-slots
+             total-tasks]])
+    ))
+
+(defn topology-link
+  ([id] (topology-link id id))
+  ([id content]
+     (link-to (format "/topology/%s" id) content)))
+
+(defn main-topology-summary-table [summs]
+  ;; make the id clickable
+  ;; make the table sortable
+  (sorted-table
+   ["Name" "Id" "Uptime" "Num workers" "Num tasks"]
+   (for [^TopologySummary t summs]
+     [(topology-link (.get_id t) (.get_name t))
+      (.get_id t)
+      (pretty-uptime-sec (.get_uptime_secs t))
+      (.get_num_workers t)
+      (.get_num_tasks t)
+      ])
+   :time-cols [2]
+   :sort-list "[[2,1]]"
+   ))
+
+(defn supervisor-summary-table [summs]
+  (sorted-table 
+   ["Host" "Uptime" "Slots" "Used slots"]
+   (for [^SupervisorSummary s summs]
+     [(.get_host s)
+      (pretty-uptime-sec (.get_uptime_secs s))
+      (.get_num_workers s)
+      (.get_num_used_workers s)])
+   :time-cols [1]))
+
+(defn main-page []
+  (with-nimbus nimbus
+    (let [summ (.getClusterInfo ^Nimbus$Client nimbus)]
+      (concat
+       [[:h2 "Cluster Summary"]]
+       [(cluster-summary-table summ)]
+       [[:h2 "Topology summary"]]
+       (main-topology-summary-table (.get_topologies summ))
+       [[:h2 "Supervisor summary"]]
+       (supervisor-summary-table (.get_supervisors summ))       
+       ))))
+
+(defn component-type [^StormTopology topology id]
+  (let [bolts (.get_bolts topology)
+        spouts (.get_spouts topology)]
+    (cond
+     (.containsKey bolts id) :bolt
+     (.containsKey spouts id) :spout
+     (= ACKER-COMPONENT-ID id) :bolt
+     )))
+
+(defn task-summary-type [topology ^TaskSummary s]
+  (component-type topology (.get_component_id s)))
+
+(defn add-pairs
+  ([] [0 0])
+  ([[a1 a2] [b1 b2]]
+      [(+ a1 b1) (+ a2 b2)]))
+
+(defn expand-averages [avg counts]
+  (let [avg (clojurify-structure avg)
+        counts (clojurify-structure counts)]
+    (into {}
+          (for [[slice streams] counts]
+            [slice
+             (into {}
+                   (for [[stream c] streams]
+                     [stream
+                      [(* c (get-in avg [slice stream]))
+                       c]]
+                     ))]
+            ))))
+
+
+(defn expand-averages-seq [average-seq counts-seq]
+  (->> (map vector average-seq counts-seq)
+       (map #(apply expand-averages %))
+       (apply merge-with
+              (fn [s1 s2]
+                (merge-with
+                 add-pairs
+                 s1
+                 s2)))
+       ))
+
+(defn- val-avg [[t c]]
+  (if (= t 0) 0
+      (double (/ t c))))
+
+(defn aggregate-averages [average-seq counts-seq]
+  (->> (expand-averages-seq average-seq counts-seq)
+       (map-val
+        (fn [s]
+          (map-val val-avg s)
+          ))
+       ))
+
+(defn aggregate-counts [counts-seq]
+  (->> counts-seq
+       (map clojurify-structure)
+       (apply merge-with
+              (fn [s1 s2]
+                (merge-with + s1 s2))
+              )))
+
+(defn aggregate-avg-streams [avg counts]
+  (let [expanded (expand-averages avg counts)]
+    (->> expanded
+         (map-val #(reduce add-pairs (vals %)))
+         (map-val val-avg)
+         )))
+
+(defn aggregate-count-streams [stats]
+  (->> stats
+       (map-val #(reduce + (vals %)))))
+
+(defn aggregate-common-stats [stats-seq]
+  {:emitted (aggregate-counts (map #(.get_emitted ^TaskStats %) stats-seq))
+   :transferred (aggregate-counts (map #(.get_transferred ^TaskStats %) stats-seq))}
+  )
+
+(defn aggregate-bolt-stats [stats-seq]
+  (let [stats-seq (collectify stats-seq)]
+    (merge (aggregate-common-stats stats-seq)
+           {:acked
+            (aggregate-counts (map #(.. ^TaskStats % get_specific get_bolt get_acked)
+                                   stats-seq))
+            :failed
+            (aggregate-counts (map #(.. ^TaskStats % get_specific get_bolt get_failed)
+                                   stats-seq))
+            :process-latencies
+            (aggregate-averages (map #(.. ^TaskStats % get_specific get_bolt get_process_ms_avg)
+                                     stats-seq)
+                                (map #(.. ^TaskStats % get_specific get_bolt get_acked)
+                                     stats-seq))}
+           )))
+
+(defn aggregate-spout-stats [stats-seq]
+  (let [stats-seq (collectify stats-seq)]
+    (merge (aggregate-common-stats stats-seq)
+           {:acked
+            (aggregate-counts (map #(.. ^TaskStats % get_specific get_spout get_acked)
+                                   stats-seq))
+            :failed
+            (aggregate-counts (map #(.. ^TaskStats % get_specific get_spout get_failed)
+                                   stats-seq))
+            :complete-latencies
+            (aggregate-averages (map #(.. ^TaskStats % get_specific get_spout get_complete_ms_avg)
+                                     stats-seq)
+                                (map #(.. ^TaskStats % get_specific get_spout get_acked)
+                                     stats-seq))
+            }
+           )))
+
+(defn aggregate-bolt-streams [stats]
+  {:acked (aggregate-count-streams (:acked stats))
+   :failed (aggregate-count-streams (:failed stats))
+   :emitted (aggregate-count-streams (:emitted stats))
+   :transferred (aggregate-count-streams (:transferred stats))
+   :process-latencies (aggregate-avg-streams (:process-latencies stats)
+                                             (:acked stats))
+   })
+
+(defn aggregate-spout-streams [stats]
+  {:acked (aggregate-count-streams (:acked stats))
+   :failed (aggregate-count-streams (:failed stats))
+   :emitted (aggregate-count-streams (:emitted stats))
+   :transferred (aggregate-count-streams (:transferred stats))
+   :complete-latencies (aggregate-avg-streams (:complete-latencies stats)
+                                              (:acked stats))
+   })
+
+(defn spout-summary? [topology s]
+  (= :spout (task-summary-type topology s)))
+
+(defn bolt-summary? [topology s]
+  (= :bolt (task-summary-type topology s)))
+
+(defn topology-summary-table [^TopologyInfo summ]
+  (let [tasks (.get_tasks summ)
+        workers (set (for [^TaskSummary t tasks] [(.get_host t) (.get_port t)]))]
+    (table ["Name" "Id" "Uptime" "Num workers" "Num tasks"]
+           [[(.get_name summ)
+             (.get_id summ)
+             (pretty-uptime-sec (.get_uptime_secs summ))
+             (count workers)
+             (count tasks)
+             ]]
+           )))
+
+(defn total-aggregate-stats [spout-summs bolt-summs]
+  (let [spout-stats (get-filled-stats spout-summs)
+        bolt-stats (get-filled-stats bolt-summs)
+        agg-spout-stats (-> spout-stats
+                            aggregate-spout-stats
+                            aggregate-spout-streams)
+        agg-bolt-stats (-> bolt-stats
+                           aggregate-bolt-stats
+                           aggregate-bolt-streams)]
+    (merge-with
+     (fn [s1 s2]
+       (merge-with + s1 s2))
+     (select-keys agg-bolt-stats [:emitted :transferred])
+     agg-spout-stats
+     )))
+
+(defn stats-times [stats-map]
+  (sort-by #(Integer/parseInt %)
+           (-> stats-map
+               clojurify-structure
+               (dissoc ":all-time")
+               keys)))
+
+(defn topology-stats-table [id window stats]
+  (let [times (stats-times (:emitted stats))
+        display-map (into {} (for [t times] [t pretty-uptime-sec]))
+        display-map (assoc display-map ":all-time" (fn [_] "All time"))]
+    (sorted-table
+     ["Window" "Emitted" "Transferred" "Complete latency (ms)" "Acked" "Failed"]
+     (for [k (concat times [":all-time"])
+           :let [disp ((display-map k) k)]]
+       [(link-to (if (= k window) {:class "red"} {})
+                 (format "/topology/%s?window=%s" id k)
+                 disp)
+        (get-in stats [:emitted k])
+        (get-in stats [:transferred k])
+        (float-str (get-in stats [:complete-latencies k]))
+        (get-in stats [:acked k])
+        (get-in stats [:failed k])
+        ]
+       )
+     :time-cols [0]
+     )))
+
+(defn group-by-comp [summs]
+  (let [ret (group-by #(.get_component_id ^TaskSummary %) summs)]
+    (into (sorted-map) ret )))
+
+(defn error-subset [error-str]
+  (apply str (take 200 error-str)))
+
+(defn most-recent-error [summs]
+  (let [summs (collectify summs)
+        error (->> summs
+                   (mapcat #(clojurify-structure (.get_errors ^TaskSummary %)))
+                   (sort-by #(.get_error_time_secs ^ErrorInfo %))
+                   reverse
+                   first)]
+    (if error
+      [:span (if (< (time-delta (.get_error_time_secs ^ErrorInfo error))
+                    (* 60 30))
+               {:class "red"}
+               {})
+       (error-subset (.get_error ^ErrorInfo error))]
+      )))
+
+(defn component-link [storm-id id]
+  (link-to (format "/topology/%s/component/%s" storm-id id) id))
+
+(defn spout-comp-table [top-id summ-map window]
+  (sorted-table
+   ["Id" "Parallelism" "Emitted" "Transferred" "Complete latency (ms)"
+    "Acked" "Failed" "Last error"]
+   (for [[id summs] summ-map
+         :let [stats-seq (get-filled-stats summs)
+               stats (aggregate-spout-streams
+                      (aggregate-spout-stats
+                       stats-seq))]]
+     [(component-link top-id id)
+      (count summs)
+      (get-in stats [:emitted window])
+      (get-in stats [:transferred window])
+      (float-str (get-in stats [:complete-latencies window]))
+      (get-in stats [:acked window])
+      (get-in stats [:failed window])
+      (most-recent-error summs)
+      ]
+     )))
+
+(defn bolt-comp-table [top-id summ-map window]
+  (sorted-table
+   ["Id" "Parallelism" "Emitted" "Transferred" "Process latency (ms)"
+    "Acked" "Failed" "Last error"]
+   (for [[id summs] summ-map
+         :let [stats-seq (get-filled-stats summs)
+               stats (aggregate-bolt-streams
+                      (aggregate-bolt-stats
+                       stats-seq))
+               ]]
+     [(component-link top-id id)
+      (count summs)
+      (get-in stats [:emitted window])
+      (get-in stats [:transferred window])
+      (float-str (get-in stats [:process-latencies window]))
+      (get-in stats [:acked window])
+      (get-in stats [:failed window])
+      (most-recent-error summs)
+      ]
+     )))
+
+(defn window-hint [window]
+  (if (= window ":all-time")
+    "All time"
+    (pretty-uptime-sec window)))
+
+(defn topology-page [id window]
+  (with-nimbus nimbus
+    (let [window (if window window ":all-time")
+          window-hint (window-hint window)
+          summ (.getTopologyInfo ^Nimbus$Client nimbus id)
+          topology (.getTopology ^Nimbus$Client nimbus id)
+          spout-summs (filter (partial spout-summary? topology) (.get_tasks summ))
+          bolt-summs (filter (partial bolt-summary? topology) (.get_tasks summ))
+          spout-comp-summs (group-by-comp spout-summs)
+          bolt-comp-summs (group-by-comp bolt-summs)
+          ]
+      (concat
+       [[:h2 "Topology summary"]]
+       [(topology-summary-table summ)]
+       [[:h2 "Topology stats"]]
+       (topology-stats-table id window (total-aggregate-stats spout-summs bolt-summs))
+       [[:h2 "Spouts (" window-hint ")"]]
+       (spout-comp-table id spout-comp-summs window)
+       [[:h2 "Bolts (" window-hint ")"]]
+       (bolt-comp-table id bolt-comp-summs window)
+       ))))
+
+(defn component-task-summs [^TopologyInfo summ topology id]
+  (let [spout-summs (filter (partial spout-summary? topology) (.get_tasks summ))
+        bolt-summs (filter (partial bolt-summary? topology) (.get_tasks summ))
+        spout-comp-summs (group-by-comp spout-summs)
+        bolt-comp-summs (group-by-comp bolt-summs)
+        ret (if (contains? spout-comp-summs id)
+              (spout-comp-summs id)
+              (bolt-comp-summs id))]
+    (sort-by #(.get_task_id ^TaskSummary %) ret)
+    ))
+
+(defnk task-link [topology-id id :suffix ""]
+  (link-to (format "/topology/%s/task/%s%s" topology-id id suffix)
+           id))
+
+(defn spout-summary-table [topology-id id stats window]
+  (let [times (stats-times (:emitted stats))
+        display-map (into {} (for [t times] [t pretty-uptime-sec]))
+        display-map (assoc display-map ":all-time" (fn [_] "All time"))]
+    (sorted-table
+     ["Window" "Emitted" "Transferred" "Complete latency (ms)" "Acked" "Failed"]
+     (for [k (concat times [":all-time"])
+           :let [disp ((display-map k) k)]]
+       [(link-to (if (= k window) {:class "red"} {})
+                 (format "/topology/%s/component/%s?window=%s" topology-id id k)
+                 disp)
+        (get-in stats [:emitted k])
+        (get-in stats [:transferred k])
+        (float-str (get-in stats [:complete-latencies k]))
+        (get-in stats [:acked k])
+        (get-in stats [:failed k])
+        ])
+     :time-cols [0])))
+
+(defn spout-output-summary-table [stream-summary window]
+  (let [stream-summary (map-val swap-map-order (swap-map-order stream-summary))]
+    (sorted-table
+     ["Stream" "Emitted" "Transferred" "Complete latency (ms)" "Acked" "Failed"]
+     (for [[s stats] (stream-summary window)]
+       [s
+        (nil-to-zero (:emitted stats))
+        (nil-to-zero (:transferred stats))
+        (float-str (:complete-latencies stats))
+        (nil-to-zero (:acked stats))
+        (nil-to-zero (:failed stats))])
+     )))
+
+(defn spout-task-table [topology-id tasks window]
+  (sorted-table
+   ["Id" "Uptime" "Host" "Port" "Emitted" "Transferred"
+    "Complete latency (ms)" "Acked" "Failed" "Last error"]
+   (for [^TaskSummary t tasks
+         :let [stats (.get_stats t)
+               stats (if stats
+                       (-> stats
+                           aggregate-spout-stats
+                           aggregate-spout-streams
+                           swap-map-order
+                           (get window)))]]
+     [(task-link topology-id (.get_task_id t))
+      (pretty-uptime-sec (.get_uptime_secs t))
+      (.get_host t)
+      (.get_port t)
+      (nil-to-zero (:emitted stats))
+      (nil-to-zero (:transferred stats))
+      (float-str (:complete-latencies stats))
+      (nil-to-zero (:acked stats))
+      (nil-to-zero (:failed stats))
+      (most-recent-error t)
+      ]
+     )
+   :time-cols [1]
+   ))
+
+(defn spout-page [window ^TopologyInfo topology-info component tasks]
+  (let [window-hint (str " (" (window-hint window) ")")
+        stats (get-filled-stats tasks)
+        stream-summary (-> stats aggregate-spout-stats)
+        summary (-> stream-summary aggregate-spout-streams)]
+    (concat
+     [[:h2 "Spout stats"]]
+     (spout-summary-table (.get_id topology-info) component summary window)
+     [[:h2 "Output stats" window-hint]]
+     (spout-output-summary-table stream-summary window)
+     [[:h2 "Tasks" window-hint]]
+     (spout-task-table (.get_id topology-info) tasks window)
+     ;; task id, task uptime, stream aggregated stats, last error
+     )))
+
+(defn bolt-output-summary-table [stream-summary window]
+  (let [stream-summary (-> stream-summary
+                           swap-map-order
+                           (get window)
+                           (select-keys [:emitted :transferred])
+                           swap-map-order)]
+    (sorted-table
+     ["Stream" "Emitted" "Transferred"]
+     (for [[s stats] stream-summary]
+       [s
+        (nil-to-zero (:emitted stats))
+        (nil-to-zero (:transferred stats))
+        ])
+     )))
+
+(defn bolt-input-summary-table [stream-summary window]
+  (let [stream-summary (-> stream-summary
+                           swap-map-order
+                           (get window)
+                           (select-keys [:acked :failed :process-latencies])
+                           swap-map-order)]
+    (sorted-table
+     ["Component" "Stream" "Process latency (ms)" "Acked" "Failed"]
+     (for [[^GlobalStreamId s stats] stream-summary]
+       [(.get_componentId s)
+        (.get_streamId s)
+        (float-str (:process-latencies stats))
+        (nil-to-zero (:acked stats))
+        (nil-to-zero (:failed stats))
+        ])
+     )))
+
+(defn bolt-task-table [topology-id tasks window]
+  (sorted-table
+   ["Id" "Uptime" "Host" "Port" "Emitted" "Transferred"
+    "Process latency (ms)" "Acked" "Failed" "Last error"]
+   (for [^TaskSummary t tasks
+         :let [stats (.get_stats t)
+               stats (if stats
+                       (-> stats
+                           aggregate-bolt-stats
+                           aggregate-bolt-streams
+                           swap-map-order
+                           (get window)))]]
+     [(task-link topology-id (.get_task_id t))
+      (pretty-uptime-sec (.get_uptime_secs t))
+      (.get_host t)
+      (.get_port t)
+      (nil-to-zero (:emitted stats))
+      (nil-to-zero (:transferred stats))
+      (float-str (:process-latencies stats))
+      (nil-to-zero (:acked stats))
+      (nil-to-zero (:failed stats))
+      (most-recent-error t)
+      ]
+     )
+   :time-cols [1]
+   ))
+
+(defn bolt-summary-table [topology-id id stats window]
+  (let [times (stats-times (:emitted stats))
+        display-map (into {} (for [t times] [t pretty-uptime-sec]))
+        display-map (assoc display-map ":all-time" (fn [_] "All time"))]
+    (sorted-table
+     ["Window" "Emitted" "Transferred" "Process latency (ms)" "Acked" "Failed"]
+     (for [k (concat times [":all-time"])
+           :let [disp ((display-map k) k)]]
+       [(link-to (if (= k window) {:class "red"} {})
+                 (format "/topology/%s/component/%s?window=%s" topology-id id k)
+                 disp)
+        (get-in stats [:emitted k])
+        (get-in stats [:transferred k])
+        (float-str (get-in stats [:process-latencies k]))
+        (get-in stats [:acked k])
+        (get-in stats [:failed k])
+        ])
+     :time-cols [0])))
+
+(defn bolt-page [window ^TopologyInfo topology-info component tasks]
+  (let [window-hint (str " (" (window-hint window) ")")
+        stats (get-filled-stats tasks)
+        stream-summary (-> stats aggregate-bolt-stats)
+        summary (-> stream-summary aggregate-bolt-streams)]
+    (concat
+     [[:h2 "Bolt stats"]]
+     (bolt-summary-table (.get_id topology-info) component summary window)
+
+     [[:h2 "Input stats" window-hint]]
+     (bolt-input-summary-table stream-summary window)
+     
+     [[:h2 "Output stats" window-hint]]
+     (bolt-output-summary-table stream-summary window)
+
+     [[:h2 "Tasks"]]
+     (bolt-task-table (.get_id topology-info) tasks window)
+     )))
+
+(defn component-page [topology-id component window]
+  (with-nimbus nimbus
+    (let [window (if window window ":all-time")
+          summ (.getTopologyInfo ^Nimbus$Client nimbus topology-id)
+          topology (.getTopology ^Nimbus$Client nimbus topology-id)
+          type (component-type topology component)
+          summs (component-task-summs summ topology component)
+          spec (cond (= type :spout) (spout-page window summ component summs)
+                     (= type :bolt) (bolt-page window summ component summs))]
+      (concat
+       [[:h2 "Component summary"]
+        (table ["Id" "Topology" "Parallelism"]
+               [[component
+                 (topology-link (.get_id summ) (.get_name summ))
+                 (count summs)
+                 ]])]
+       spec
+       ))))
+
+(defn errors-table [^TaskSummary task]
+  (let [errors (->> task
+                    .get_errors
+                    (sort-by #(.get_error_time_secs ^ErrorInfo %))
+                    reverse)]
+    (sorted-table
+     ["Time" "Error"]
+     (for [^ErrorInfo e errors]
+       [(date-str (.get_error_time_secs e))
+        [:pre (.get_error e)]])
+     :sort-list "[[0,1]]"
+     )))
+
+(defn task-summary-table [^TaskSummary task ^TopologyInfo summ]
+  (table ["Id" "Topology" "Component" "Uptime" "Host" "Port"]
+         [[(.get_task_id task)
+            (topology-link (.get_id summ) (.get_name summ))
+            (component-link (.get_id summ) (.get_component_id task))
+            (pretty-uptime-sec (.get_uptime_secs task))
+            (.get_host task)
+            (.get_port task)]]
+         ))
+
+(defn task-page [topology-id task-id window]
+  (with-nimbus nimbus
+    (let [summ (.getTopologyInfo ^Nimbus$Client nimbus topology-id)
+          topology (.getTopology ^Nimbus$Client nimbus topology-id)
+          task (->> summ
+                    .get_tasks
+                    (find-first #(= (.get_task_id ^TaskSummary %) task-id)))]
+      (concat
+       [[:h2 "Task summary"]]
+       [(task-summary-table task summ)]
+       ;; TODO: overall stats -> window, ...
+       ;; TODO: inputs/outputs stream stats
+       [[:h2 "Errors"]]
+       (errors-table task)
+       ))))
+
+
+(defroutes main-routes
+  (GET "/" []
+       (-> (main-page)
+           ui-template))
+  (GET "/topology/:id" [id & m]
+       (-> (topology-page id (:window m))
+           ui-template))
+  (GET "/topology/:id/component/:component" [id component & m]
+       (-> (component-page id (Integer/parseInt component) (:window m))
+           ui-template))
+  (GET "/topology/:id/task/:task" [id task & m]
+       (-> (task-page id (Integer/parseInt task) (:window m))
+           ui-template))
+  (route/resources "/")
+  (route/not-found "Page not found"))
+
+(def app
+  (handler/site main-routes))
+
+(defn -main []
+  (run-jetty app {:port 8080}))
diff --git a/src/clj/backtype/storm/ui/helpers.clj b/src/clj/backtype/storm/ui/helpers.clj
new file mode 100644
index 0000000..bf49988
--- /dev/null
+++ b/src/clj/backtype/storm/ui/helpers.clj
@@ -0,0 +1,115 @@
+(ns backtype.storm.ui.helpers
+  (:use compojure.core)
+  (:use [hiccup core page-helpers])
+  (:use [clojure.contrib
+         [str-utils2 :only [join]]
+         [def :only [defnk]]])
+  (:use [backtype.storm.util :only [uuid]])
+  (:use [clj-time coerce format])
+  (:require [compojure.route :as route]
+            [compojure.handler :as handler]))
+
+(defn split-divide [val divider]
+  [(int (/ val divider)) (mod val divider)]
+  )
+
+(def PRETTY-SEC-DIVIDERS
+     [["s" 60]
+      ["m" 60]
+      ["h" 24]
+      ["d" nil]])
+
+(def PRETTY-MS-DIVIDERS
+     (cons ["ms" 1000]
+           PRETTY-SEC-DIVIDERS))
+
+(defn pretty-uptime-str* [val dividers]
+  (let [val (if (string? val) (Integer/parseInt val) val)
+        vals (reduce (fn [[state val] [_ divider]]
+                       (if (pos? val)
+                         (let [[divided mod] (if divider
+                                               (split-divide val divider)
+                                               [nil val])]
+                           [(concat state [mod])
+                            divided]
+                           )
+                         [state val]
+                         ))
+                     [[] val]
+                     dividers)
+        strs (->>
+              (first vals)
+              (map
+               (fn [[suffix _] val]
+                 (str val suffix))
+               dividers
+               ))]
+    (join " " (reverse strs))
+    ))
+
+(defn pretty-uptime-sec [secs]
+  (pretty-uptime-str* secs PRETTY-SEC-DIVIDERS))
+
+(defn pretty-uptime-ms [ms]
+  (pretty-uptime-str* ms PRETTY-MS-DIVIDERS))
+
+
+(defelem table [headers data]
+  [:table
+   [:thead
+    [:tr
+     (for [h headers]
+       [:th h])
+     ]]
+   [:tbody
+    (for [row data]
+      [:tr
+       (for [col row]
+         [:td col]
+         )]
+      )]
+   ])
+
+(defnk sort-table [id :sort-list "[[0,0]]" :time-cols []]
+  (let [strs (for [c time-cols] (format "%s: { sorter: 'stormtimestr'}" c))
+        sorters (join ", " strs)]
+    [:script
+     (format  "$(document).ready(function() {
+$(\"table#%s\").each(function(i) { $(this).tablesorter({ sortList: %s, headers: {%s}}); });
+});"
+              id
+              sort-list
+              sorters)]))
+
+(defn float-str [n]
+  (if n
+    (format "%.3f" (float n))
+    "0"
+    ))
+
+(defn swap-map-order [m]
+  (->> m
+       (map (fn [[k v]]
+              (into
+               {}
+               (for [[k2 v2] v]
+                 [k2 {k v2}]
+                 ))
+              ))
+       (apply merge-with merge)
+       ))
+
+(defn sorted-table [headers data & args]
+  (let [id (uuid)]
+    (concat
+     [(table {:class "zebra-striped" :id id}
+             headers
+             data)]
+     (if-not (empty? data)
+       [(apply sort-table id args)])
+     )))
+
+(defn date-str [secs]
+  (let [dt (from-long (* 1000 (long secs)))]
+    (unparse (:rfc822 formatters) dt)
+    ))
diff --git a/src/clj/backtype/storm/util.clj b/src/clj/backtype/storm/util.clj
new file mode 100644
index 0000000..cb582a9
--- /dev/null
+++ b/src/clj/backtype/storm/util.clj
@@ -0,0 +1,499 @@
+(ns backtype.storm.util
+  (:import [java.net InetAddress])
+  (:import [java.util Map List Collection])
+  (:import [java.io FileReader])
+  (:import [backtype.storm Config])
+  (:import [backtype.storm.utils Time])
+  (:import [java.util UUID])
+  (:import [java.util.concurrent.locks ReentrantReadWriteLock])
+  (:import [java.io File RandomAccessFile StringWriter PrintWriter])
+  (:import [java.lang.management ManagementFactory])
+  (:import [org.apache.commons.exec DefaultExecutor CommandLine])
+  (:import [org.apache.commons.io FileUtils])
+  (:import [org.apache.commons.exec ExecuteException])
+  (:import [org.json.simple JSONValue])
+  (:require [clojure.contrib [str-utils2 :as str]])
+  (:require [clojure [set :as set]])
+  (:use [clojure walk])
+  (:use [backtype.storm log])
+  (:use [clojure.contrib.def :only [defnk]])
+  )
+
+(defn local-hostname []
+  (.getCanonicalHostName (InetAddress/getLocalHost)))
+
+(defn uuid []
+  (str (UUID/randomUUID)))
+
+(defn current-time-secs []
+  (int (unchecked-divide (Time/currentTimeMillis) (long 1000))))
+
+(defn clojurify-structure [s]
+  (prewalk (fn [x]
+              (cond (instance? Map x) (into {} x)
+                    (instance? List x) (vec x)
+                    true x))
+           s))
+
+(defmacro with-file-lock [path & body]
+  `(let [f# (File. ~path)
+         _# (.createNewFile f#)
+         rf# (RandomAccessFile. f# "rw")
+         lock# (.. rf# (getChannel) (lock))]
+      (try
+        ~@body
+        (finally
+          (.release lock#)
+          (.close rf#))
+        )))
+
+(defn tokenize-path [^String path]
+  (let [toks (.split path "/")]
+    (vec (filter (complement empty?) toks))
+    ))
+
+(defn assoc-conj [m k v]
+  (merge-with concat m {k [v]}))
+
+;; returns [ones in first set not in second, ones in second set not in first]
+(defn set-delta [old curr]
+  (let [s1 (set old)
+        s2 (set curr)]
+    [(set/difference s1 s2) (set/difference s2 s1)]
+    ))
+
+(defn parent-path [path]
+  (let [toks (tokenize-path path)]
+    (str "/" (str/join "/" (butlast toks)))
+    ))
+
+(defn toks->path [toks]
+  (str "/" (str/join "/" toks))
+  )
+
+(defn normalize-path [^String path]
+  (toks->path (tokenize-path path)))
+
+(defn map-val [afn amap]
+  (into {}
+    (for [[k v] amap]
+      [k (afn v)]
+      )))
+
+(defn filter-val [afn amap]
+  (into {}
+    (filter
+      (fn [[k v]]
+        (afn v))
+       amap
+       )))
+
+(defn full-path [parent name]
+  (let [toks (tokenize-path parent)]
+    (toks->path (conj toks name))
+    ))
+
+(defn not-nil? [o]
+  (not (nil? o)))
+
+(defn barr [& vals]
+  (byte-array (map byte vals)))
+
+(defn halt-process! [val & msg]
+  (log-message "Halting process: " msg)
+  (Thread/sleep 1000)
+  (.halt (Runtime/getRuntime) val)
+  )
+
+(defn sum [vals]
+  (reduce + vals))
+
+(defn repeat-seq
+  ([aseq]
+    (apply concat (repeat aseq)))
+  ([amt aseq]
+    (apply concat (repeat amt aseq))
+    ))
+
+(defn div
+  "Perform floating point division on the arguments."
+  [f & rest] (apply / (double f) rest))
+
+(defn defaulted [val default]
+  (if val val default))
+
+(defn mk-counter []
+  (let [val (atom 0)]
+    (fn []
+      (swap! val inc))))
+
+(defmacro for-times [times & body]
+  `(for [i# (range ~times)]
+     ~@body
+     ))
+
+(defmacro dofor [& body]
+  `(doall (for ~@body)))
+
+(defn reverse-map
+  "{:a 1 :b 1 :c 2} -> {1 [:a :b] 2 :c}"
+  [amap]
+  (reduce (fn [m [k v]]
+    (let [existing (get m v [])]
+      (assoc m v (conj existing k))))
+    {} amap))
+
+(defmacro print-vars [& vars]
+  (let [prints (for [v vars] `(println ~(str v) ~v))]
+    `(do ~@prints)))
+
+(defn process-pid
+  "Gets the pid of this JVM. Hacky because Java doesn't provide a real way to do this."
+  []
+  (let [name (.getName (ManagementFactory/getRuntimeMXBean))
+        split (.split name "@")]
+    (when-not (= 2 (count split))
+      (throw (RuntimeException. (str "Got unexpected process name: " name))))
+    (first split)
+    ))
+
+(defn exec-command! [command]
+  (let [[comm-str & args] (seq (.split command " "))
+        command (CommandLine. comm-str)]
+    (doseq [a args]
+      (.addArgument command a))
+    (.execute (DefaultExecutor.) command)
+    ))
+
+(defn extract-dir-from-jar [jarpath dir destdir]
+  (try
+    (exec-command! (str "unzip -qq " jarpath " " dir "/** -d " destdir))
+  (catch ExecuteException e
+    (log-message "Error when trying to extract " dir " from " jarpath))
+  ))
+
+(defn ensure-process-killed! [pid]
+  ;; TODO: should probably do a ps ax of some sort to make sure it was killed
+  (try
+    (exec-command! (str "kill -9 " pid))
+  (catch ExecuteException e
+    (log-message "Error when trying to kill " pid ". Process is probably already dead."))
+    ))
+
+(defn launch-process [command]
+  (let [command (seq (.split command " "))
+        builder (ProcessBuilder. (cons "nohup" command))]
+    (.start builder)
+    ))
+
+(defn sleep-secs [secs]
+  (Time/sleep (* (long secs) 1000)))
+
+(defn sleep-until-secs [target-secs]
+  (Time/sleepUntil (* (long target-secs) 1000)))
+
+(defprotocol SmartThread
+  (start [this])
+  (join [this])
+  (interrupt [this])
+  (sleeping? [this]))
+
+;; afn returns amount of time to sleep
+(defnk async-loop [afn
+                   :daemon false
+                   :kill-fn (fn [error] (halt-process! 1 "Async loop died!"))
+                   :priority Thread/NORM_PRIORITY
+                   :args-fn (fn [] [])
+                   :start true]
+  (let [thread (Thread.
+                (fn []
+                  (try
+                    (let [args (args-fn)]
+                      (loop []
+                        (let [sleep-time (apply afn args)]
+                          (when-not (nil? sleep-time)
+                            (sleep-secs sleep-time)
+                            (recur))
+                          )))
+                    (catch InterruptedException e
+                      (log-message "Async loop interrupted!")
+                      )
+                    (catch Throwable t
+                      ;; work around clojure wrapping exceptions
+                      (if (instance? InterruptedException (.getCause t))
+                        (log-message "Async loop interrupted!")
+                        (do
+                          (log-error t "Async loop died!")
+                          (kill-fn t)
+                          ))
+                      ))
+                  ))]
+    (.setDaemon thread daemon)
+    (.setPriority thread priority)
+    (when start
+      (.start thread))
+    ;; should return object that supports stop, interrupt, join, and waiting?
+    (reify SmartThread
+      (start [this]
+        (.start thread))
+      (join [this]
+        (.join thread))
+      (interrupt [this]
+        (.interrupt thread))
+      (sleeping? [this]
+        (Time/isThreadWaiting thread)
+        ))
+      ))
+
+(defn filter-map-val [afn amap]
+  (into {} (filter (fn [[k v]] (afn v)) amap)))
+
+(defn exists-file? [path]
+  (.exists (File. path)))
+
+(defn rmr [path]
+  (when (exists-file? path)
+    (FileUtils/forceDelete (File. path))))
+
+(defn rmpath
+  "Removes file or directory at the path. Not recursive. Throws exception on failure"
+  [path]
+  (let [deleted? (.delete (File. path))]
+    (when-not deleted?
+      (throw (RuntimeException. (str "Failed to delete " path))))
+    ))
+
+(defn local-mkdirs
+  [path]
+  (FileUtils/forceMkdir (File. path)))
+
+(defn touch [path]
+  (let [success? (.createNewFile (File. path))]
+    (when-not success?
+      (throw (RuntimeException. (str "Failed to touch " path))))
+    ))
+
+(defn read-dir-contents [dir]
+  (if (exists-file? dir)
+    (let [content-files (.listFiles (File. dir))]
+      (map #(.getName ^File %) content-files))
+    [] ))
+
+(defn compact [aseq]
+  (filter (complement nil?) aseq))
+
+(defn current-classpath []
+  (System/getProperty "java.class.path"))
+
+(defn add-to-classpath [classpath paths]
+  (str/join ":" (cons classpath paths)))
+
+(defn ^ReentrantReadWriteLock mk-rw-lock []
+  (ReentrantReadWriteLock.))
+
+(defmacro read-locked [rw-lock & body]
+  `(let [rlock# (.readLock ~rw-lock)]
+      (try
+        (.lock rlock#)
+        ~@body
+      (finally (.unlock rlock#)))))
+
+(defmacro write-locked [rw-lock & body]
+  `(let [wlock# (.writeLock ~rw-lock)]
+      (try
+        (.lock wlock#)
+        ~@body
+      (finally (.unlock wlock#)))))
+
+(defn wait-for-condition [apredicate]
+  (while (not (apredicate))
+    (Time/sleep 100)
+    ))
+
+(defn some? [pred aseq]
+  ((complement nil?) (some pred aseq)))
+
+(defn time-delta [time-secs]
+  (- (current-time-secs) time-secs))
+
+(defn time-delta-ms [time-ms]
+  (- (System/currentTimeMillis) time-ms))
+
+(defn parse-int [str]
+  (Integer/parseInt str))
+
+(defn integer-divided [sum num-pieces]
+  (let [base (int (/ sum num-pieces))
+        num-inc (mod sum num-pieces)
+        num-bases (- num-pieces num-inc)]
+    (if (= num-inc 0)
+      {base num-bases}
+      {base num-bases (inc base) num-inc}
+      )))
+
+(defn collectify [obj]
+  (if (or (sequential? obj) (instance? Collection obj)) obj [obj]))
+
+(defn to-json [^Map m]
+  (JSONValue/toJSONString m))
+
+(defn from-json [^String str]
+  (clojurify-structure
+    (JSONValue/parse str)))
+
+(defmacro letlocals [& body]
+   (let [[tobind lexpr] (split-at (dec (count body)) body)
+         binded (vec (mapcat (fn [e]
+                  (if (and (list? e) (= 'bind (first e)))
+                     [(second e) (last e)]
+                     ['_ e]
+                     ))
+                  tobind ))]
+     `(let ~binded
+         ~(first lexpr)
+      )))
+
+(defn remove-first [pred aseq]
+  (let [[b e] (split-with (complement pred) aseq)]
+    (when (empty? e)
+      (throw (IllegalArgumentException. "Nothing to remove")))
+    (concat b (rest e))
+    ))
+
+(defn multi-set
+  "Returns a map of elem to count"
+  [aseq]
+  (apply merge-with +
+         (map #(hash-map % 1) aseq)))
+
+(defn set-var-root* [avar val]
+  (alter-var-root avar (fn [avar] val)))
+
+(defmacro set-var-root [var-sym val]
+  `(set-var-root* (var ~var-sym) ~val))
+
+(defmacro with-var-roots [bindings & body]
+  (let [settings (partition 2 bindings)
+        tmpvars (repeatedly (count settings) (partial gensym "old"))
+        vars (map first settings)
+        savevals (vec (mapcat (fn [t v] [t v]) tmpvars vars))
+        setters (for [[v s] settings] `(set-var-root ~v ~s))
+        restorers (map (fn [v s] `(set-var-root ~v ~s)) vars tmpvars)
+        ]
+    `(let ~savevals
+      ~@setters
+      (try
+        ~@body
+      (finally
+        ~@restorers))
+      )))
+
+(defn map-diff
+  "Returns mappings in m2 that aren't in m1"
+  [m1 m2]
+  (into {}
+    (filter
+      (fn [[k v]] (not= v (m1 k)))
+      m2
+      )))
+
+
+(defn select-keys-pred [pred amap]
+  (into {}
+        (filter
+         (fn [[k v]]
+           (pred k))
+         amap)))
+
+
+(defn rotating-random-range [amt]
+  (ref (shuffle (range amt))))
+
+(defn acquire-random-range-id [rr amt]
+  (dosync
+   (let [ret (first @rr)]
+     (alter
+      rr
+      (fn [rr]
+        (if (= 1 (count rr))
+          (shuffle (range amt))
+          (next rr))
+        ))
+     ret
+     )))
+
+; this can be rewritten to be tail recursive
+(defn interleave-all [& colls]
+  (if (empty? colls)
+    []
+    (let [colls (filter (complement empty?) colls)
+          my-elems (map first colls)
+          rest-elems (apply interleave-all (map rest colls))]
+      (concat my-elems rest-elems)
+      )))
+
+(defn update [m k afn]
+  (assoc m k (afn (get m k))))
+
+(defn any-intersection [& sets]
+  (let [elem->count (multi-set (apply concat sets))]
+    (-> (filter-val #(> % 1) elem->count)
+        keys
+        )))
+
+(defn between?
+  "val >= lower and val <= upper"
+  [val lower upper]
+  (and (>= val lower)
+       (<= val upper)))
+
+(defmacro benchmark [& body]
+  `(time
+    (doseq [i# (range 1000000)]
+      ~@body)))
+
+(defn rand-sampler [freq]
+  (let [r (java.util.Random.)]
+    (fn []
+      (= 0 (.nextInt r freq)))
+    ))
+
+(defn even-sampler [freq]
+  (let [r (java.util.Random.)
+        state (atom [-1 (.nextInt r freq)])
+        updater (fn [[i target]]
+                  (let [i (inc i)]
+                    (if (>= i freq)
+                      [0 (.nextInt r freq)]
+                      [i target]
+                      )))]
+    (with-meta
+      (fn []
+        (let [[i target] (swap! state updater)]
+          (= i target)
+          ))
+      {:rate freq})))
+
+(defn sampler-rate [sampler]
+  (:rate (meta sampler)))
+
+(defn class-selector [obj & args] (class obj))
+
+(defn uptime-computer []
+  (let [start-time (current-time-secs)]
+    (fn []
+      (time-delta start-time)
+      )))
+
+(defn stringify-error [error]
+  (let [result (StringWriter.)
+        printer (PrintWriter. result)]
+    (.printStackTrace error printer)
+    (.toString result)
+    ))
+
+(defn nil-to-zero [v]
+  (if v v 0))
+
+(defn bit-xor-vals [vals]
+  (reduce bit-xor 0 vals))
diff --git a/src/clj/backtype/storm/zookeeper.clj b/src/clj/backtype/storm/zookeeper.clj
new file mode 100644
index 0000000..8a74131
--- /dev/null
+++ b/src/clj/backtype/storm/zookeeper.clj
@@ -0,0 +1,110 @@
+(ns backtype.storm.zookeeper
+  (:import [org.apache.zookeeper ZooKeeper Watcher KeeperException$NoNodeException
+            ZooDefs ZooDefs$Ids CreateMode WatchedEvent Watcher$Event Watcher$Event$KeeperState
+            Watcher$Event$EventType])
+  (:import [org.apache.zookeeper.data Stat])
+  (:import [org.apache.zookeeper.server ZooKeeperServer NIOServerCnxn$Factory])
+  (:import [java.net InetSocketAddress])
+  (:import [java.io File])
+  (:use [backtype.storm util log config]))
+
+(def zk-keeper-states
+  {Watcher$Event$KeeperState/Disconnected :disconnected
+   Watcher$Event$KeeperState/SyncConnected :connected
+   Watcher$Event$KeeperState/AuthFailed :auth-failed
+   Watcher$Event$KeeperState/Expired :expired
+  })
+
+(def zk-event-types
+  {Watcher$Event$EventType/None :none
+   Watcher$Event$EventType/NodeCreated :node-created
+   Watcher$Event$EventType/NodeDeleted :node-deleted
+   Watcher$Event$EventType/NodeDataChanged :node-data-changed
+   Watcher$Event$EventType/NodeChildrenChanged :node-children-changed
+  })
+
+
+;; TODO: make this block until session is established (wait until a flag is triggered by watcher)
+(defn mk-client
+  ([conn-str session-timeout watcher]
+    (ZooKeeper.
+      conn-str
+      session-timeout
+      (reify Watcher
+        (^void process [this ^WatchedEvent event]
+          (watcher (zk-keeper-states (.getState event))
+                   (zk-event-types (.getType event))
+                   (.getPath event))
+          ))))
+  ([conn-str watcher]
+    (mk-client conn-str 10000 watcher))
+  ([conn-str]
+    ;; this constructor is intended for debugging
+    (mk-client
+      conn-str
+      (fn [state type path]
+        (log-message "Zookeeper state update: " state type path)))
+      ))
+
+(def zk-create-modes
+  {:ephemeral CreateMode/EPHEMERAL
+   :persistent CreateMode/PERSISTENT})
+
+(defn create-node
+  ([^ZooKeeper zk ^String path ^bytes data mode]
+    (.create zk (normalize-path path) data ZooDefs$Ids/OPEN_ACL_UNSAFE (zk-create-modes mode)))
+  ([^ZooKeeper zk ^String path ^bytes data]
+    (create-node zk path data :persistent)))
+
+(defn exists-node? [^ZooKeeper zk ^String path watch?]
+  ((complement nil?) (.exists zk (normalize-path path) watch?)))
+
+(defn delete-node [^ZooKeeper zk ^String path]
+  (.delete zk (normalize-path path) -1))
+
+(defn mkdirs [^ZooKeeper zk ^String path]
+  (let [path (normalize-path path)]
+    (when-not (or (= path "/") (exists-node? zk path false))
+      (mkdirs zk (parent-path path))
+      (create-node zk path (barr 7) :persistent)
+      )))
+
+(defn get-data [^ZooKeeper zk ^String path watch?]
+  (let [path (normalize-path path)]
+    (try
+      (if (.exists zk path watch?)
+        (.getData zk path watch? (Stat.)))
+    (catch KeeperException$NoNodeException e
+      ;; this is fine b/c we still have a watch from the successful exists call
+      nil ))))
+
+(defn get-children [^ZooKeeper zk ^String path watch?]
+  (.getChildren zk (normalize-path path) watch?)
+  )
+
+(defn set-data [^ZooKeeper zk ^String path ^bytes data]
+  (.setData zk (normalize-path path) data -1))
+
+(defn exists [^ZooKeeper zk ^String path watch?]
+  (.exists zk (normalize-path path) watch?)
+  )
+
+(defn delete-recursive [^ZooKeeper zk ^String path]
+  (let [path (normalize-path path)]
+    (when (exists-node? zk path false)
+      (let [children (get-children zk path false)]
+        (doseq [c children]
+          (delete-recursive zk (full-path path c)))
+        (delete-node zk path)
+        ))))
+
+(defn mk-inprocess-zookeeper [localdir port]
+  (let [localfile (File. localdir)
+        zk (ZooKeeperServer. localfile localfile 2000)
+        factory (NIOServerCnxn$Factory. (InetSocketAddress. port))]
+    (.startup factory zk)
+    factory
+    ))
+
+(defn shutdown-inprocess-zookeeper [handle]
+  (.shutdown handle))
diff --git a/src/clj/zilch/mq.clj b/src/clj/zilch/mq.clj
new file mode 100644
index 0000000..6c0240b
--- /dev/null
+++ b/src/clj/zilch/mq.clj
@@ -0,0 +1,93 @@
+;; Copyright 2011 Tim Dysinger
+
+;;    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.
+
+(ns zilch.mq
+  (:refer-clojure :exclude [send])
+  )
+
+(defmacro zeromq-imports []
+  '(do
+    (import '[org.zeromq ZMQ ZMQ$Context ZMQ$Socket])
+    ))
+
+(zeromq-imports)
+
+(defn ^ZMQ$Context context [threads]
+  (ZMQ/context threads))
+
+(defmacro with-context
+  [id threads & body]
+  `(let [~id (context ~threads)]
+     (try ~@body
+          (finally (.term ~id)))))
+
+(def sndmore ZMQ/SNDMORE)
+
+(def req ZMQ/REQ)
+(def rep ZMQ/REP)
+(def xreq ZMQ/XREQ)
+(def xrep ZMQ/XREP)
+(def pub ZMQ/PUB)
+(def sub ZMQ/SUB)
+(def pair ZMQ/PAIR)
+(def push ZMQ/PUSH)
+(def pull ZMQ/PULL)
+
+(defn ^bytes barr [& arr]
+  (byte-array (map byte arr)))
+
+(defn ^ZMQ$Socket socket
+  [^ZMQ$Context context type]
+  (.socket context type))
+
+(defn set-linger
+  [^ZMQ$Socket socket linger-ms]
+  (doto socket
+    (.setLinger (long linger-ms))))
+
+(defn bind
+  [^ZMQ$Socket socket url]
+  (doto socket
+    (.bind url)))
+
+(defn connect
+  [^ZMQ$Socket socket url]
+  (doto socket
+    (.connect url)))
+
+(defn subscribe
+  ([^ZMQ$Socket socket ^bytes topic]
+     (doto socket
+       (.subscribe topic)))
+  ([^ZMQ$Socket socket]
+     (subscribe socket (byte-array []))))
+
+(defn unsubscribe
+  ([^ZMQ$Socket socket ^bytes topic]
+     (doto socket
+       (.unsubscribe (.getBytes topic))))
+  ([^ZMQ$Socket socket]
+     (unsubscribe socket "")))
+
+(defn send
+  ([^ZMQ$Socket socket ^bytes message flags]
+     (.send socket message flags))
+  ([^ZMQ$Socket socket ^bytes message]
+     (send socket message ZMQ/NOBLOCK)))
+
+(defn recv
+  ([^ZMQ$Socket socket flags]
+     (.recv socket flags))
+  ([^ZMQ$Socket socket]
+     (recv socket 0)))
diff --git a/src/clj/zilch/virtual_port.clj b/src/clj/zilch/virtual_port.clj
new file mode 100644
index 0000000..e567865
--- /dev/null
+++ b/src/clj/zilch/virtual_port.clj
@@ -0,0 +1,96 @@
+(ns zilch.virtual-port
+  (:use [clojure.contrib.def :only [defnk]])
+  (:use [backtype.storm util log])
+  (:require [zilch [mq :as mq]])
+  (:import [java.nio ByteBuffer])
+  (:import [java.util.concurrent Semaphore]))
+
+(mq/zeromq-imports)
+
+(defn mk-packet [virtual-port ^bytes message]
+  (let [bb (ByteBuffer/allocate (+ 2 (count message)))]
+    (.putShort bb (short virtual-port))
+    (.put bb message)
+    (.array bb)
+    ))
+
+(defn parse-packet [^bytes packet]
+  (let [bb (ByteBuffer/wrap packet)
+        port (.getShort bb)
+        msg (byte-array (- (count packet) 2))]
+    (.get bb msg)
+    [port msg]
+    ))
+
+(defn virtual-url [port]
+  (str "inproc://" port))
+
+(defn- get-virtual-socket! [context mapping-atom port]
+  (when-not (contains? @mapping-atom port)
+    (log-message "Connecting to virtual port " port)
+    (swap! mapping-atom
+           assoc
+           port
+           (-> context (mq/socket mq/push) (mq/connect (virtual-url port)))
+           ))
+  (@mapping-atom port))
+
+(defn close-virtual-sockets! [mapping-atom]
+  (doseq [[_ virtual-socket] @mapping-atom]
+    (.close virtual-socket))
+  (reset! mapping-atom {}))
+
+(defn virtual-send
+  ([^ZMQ$Socket socket virtual-port ^bytes message flags]
+     (mq/send socket (mk-packet virtual-port message) flags))
+  ([^ZMQ$Socket socket virtual-port ^bytes message]
+     (virtual-send socket virtual-port message ZMQ/NOBLOCK)))
+
+(defnk launch-virtual-port!
+  [context url :daemon true
+               :kill-fn (fn [] (System/exit 1))
+               :priority Thread/NORM_PRIORITY
+               :valid-ports nil]
+  (let [valid-ports (set (map short valid-ports))
+        vthread (async-loop
+                  (fn [^ZMQ$Socket socket virtual-mapping]
+                        (let [[port msg] (parse-packet (mq/recv socket))]
+                          (if (= port -1)
+                            (do
+                              (log-message "Virtual port " url " received shutdown notice")
+                              (close-virtual-sockets! virtual-mapping)
+                              (.close socket)
+                              nil )
+                            (if (or (nil? valid-ports) (contains? valid-ports port))
+                              (let [^ZMQ$Socket virtual-socket (get-virtual-socket! context virtual-mapping port)]
+                                ;; TODO: probably need to handle multi-part messages here or something
+                                (mq/send virtual-socket msg)
+                                0
+                                )
+                              (log-message "Received invalid message directed at port " port ". Dropping...")
+                              ))))
+                  :args-fn (fn [] [(-> context (mq/socket mq/pull) (mq/bind url)) (atom {})])
+                  :daemon daemon
+                  :kill-fn kill-fn
+                  :priority priority)]
+    (fn []
+      (let [kill-socket (-> context (mq/socket mq/push) (mq/connect url))]
+        (log-message "Shutting down virtual port at url: " url)
+        (virtual-send kill-socket
+                      -1
+                      (mq/barr 1))
+        (.close kill-socket)
+        (log-message "Waiting for virtual port at url " url " to die")
+        (.join vthread)
+        (log-message "Shutdown virtual port at url: " url)
+        ))))
+
+(defn virtual-bind
+  [^ZMQ$Socket socket virtual-port]
+  (mq/bind socket (virtual-url virtual-port))
+  )
+
+(defn virtual-connect
+  [^ZMQ$Socket socket virtual-port]
+  (mq/connect socket (virtual-url virtual-port))
+  )
diff --git a/src/dev/resources/storm.fy b/src/dev/resources/storm.fy
new file mode 120000
index 0000000..d354fb1
--- /dev/null
+++ b/src/dev/resources/storm.fy
@@ -0,0 +1 @@
+../../multilang/fy/storm.fy
\ No newline at end of file
diff --git a/src/dev/resources/storm.py b/src/dev/resources/storm.py
new file mode 120000
index 0000000..5e73111
--- /dev/null
+++ b/src/dev/resources/storm.py
@@ -0,0 +1 @@
+../../multilang/py/storm.py
\ No newline at end of file
diff --git a/src/dev/resources/storm.rb b/src/dev/resources/storm.rb
new file mode 120000
index 0000000..96db018
--- /dev/null
+++ b/src/dev/resources/storm.rb
@@ -0,0 +1 @@
+../../multilang/rb/storm.rb
\ No newline at end of file
diff --git a/src/dev/resources/tester.fy b/src/dev/resources/tester.fy
new file mode 100644
index 0000000..bad0429
--- /dev/null
+++ b/src/dev/resources/tester.fy
@@ -0,0 +1,10 @@
+require: "storm"
+
+class TesterBolt : Storm Bolt {
+  def process: tuple {
+    emit: [tuple values first + "lalala"]
+    ack: tuple
+  }
+}
+
+TesterBolt new run
\ No newline at end of file
diff --git a/src/dev/resources/tester.py b/src/dev/resources/tester.py
new file mode 100644
index 0000000..b38007c
--- /dev/null
+++ b/src/dev/resources/tester.py
@@ -0,0 +1,8 @@
+import storm
+
+class TesterBolt(storm.Bolt):
+    def process(self, tup):
+        storm.emit([tup.values[0]+"lalala"])
+        storm.ack(tup)
+
+TesterBolt().run()
\ No newline at end of file
diff --git a/src/dev/resources/tester.rb b/src/dev/resources/tester.rb
new file mode 100644
index 0000000..a59a93c
--- /dev/null
+++ b/src/dev/resources/tester.rb
@@ -0,0 +1,10 @@
+require File.expand_path("storm", File.dirname(__FILE__))
+
+class TesterBolt < Storm::Bolt
+  def process(tuple)
+    emit [tuple.values[0] + "lalala"]
+    ack tuple
+  end
+end
+
+TesterBolt.new.run
diff --git a/src/genthrift.sh b/src/genthrift.sh
new file mode 100644
index 0000000..593bf3a
--- /dev/null
+++ b/src/genthrift.sh
@@ -0,0 +1,6 @@
+rm -rf gen-javabean gen-py py
+rm -rf jvm/backtype/storm/generated
+thrift --gen java:beans,hashcode,nocamel --gen py:utf8strings storm.thrift
+mv gen-javabean/backtype/storm/generated jvm/backtype/storm/generated
+mv gen-py py
+rm -rf gen-javabean
diff --git a/src/jvm/backtype/storm/Config.java b/src/jvm/backtype/storm/Config.java
new file mode 100644
index 0000000..a53e793
--- /dev/null
+++ b/src/jvm/backtype/storm/Config.java
@@ -0,0 +1,299 @@
+package backtype.storm;
+
+/**
+ * This class provides constants for all the configurations possible on a Storm
+ * cluster and Storm topology. Default values for these configs can be found in
+ * defaults.yaml.
+ *
+ * <p>Note that you may put other configurations in the configuration maps. Storm
+ * will ignore anything it doesn't recognize, but your topologies are free to make
+ * use of them.</p>
+ */
+public class Config {
+    
+    /**
+     * A list of hosts of ZooKeeper servers used to manage the cluster.
+     */
+    public static String STORM_ZOOKEEPER_SERVERS = "storm.zookeeper.servers";
+
+    /**
+     * The port Storm will use to connect to each of the ZooKeeper servers.
+     */
+    public static String STORM_ZOOKEEPER_PORT = "storm.zookeeper.port";
+
+    /**
+     * A directory on the local filesystem used by Storm for any local
+     * filesystem usage it needs. The directory must exist and the Storm daemons must
+     * have permission to read/write from this location.
+     */
+    public static String STORM_LOCAL_DIR = "storm.local.dir";
+
+
+    /**
+     * The mode this Storm cluster is running in. Either "distributed" or "local".
+     */
+    public static String STORM_CLUSTER_MODE = "storm.cluster.mode";
+
+    /**
+     * The root location at which Storm stores data in ZooKeeper.
+     */
+    public static String STORM_ZOOKEEPER_ROOT = "storm.zookeeper.root";
+
+    /**
+     * The timeout for clients to ZooKeeper.
+     */
+    public static String STORM_ZOOKEEPER_SESSION_TIMEOUT = "storm.zookeeper.session.timeout";
+
+    /**
+     * The id assigned to a running topology. The id is the storm name with a unique nonce appended.
+     */
+    public static String STORM_ID = "storm.id";
+    
+    /**
+     * The host that the master server is running on.
+     */
+    public static String NIMBUS_HOST = "nimbus.host";
+
+    /**
+     * Which port the Thrift interface of Nimbus should run on. Clients should
+     * connect to this port to upload jars and submit topologies.
+     */
+    public static String NIMBUS_THRIFT_PORT = "nimbus.thrift.port";
+
+
+    /**
+     * This parameter is used by the storm-deploy project to configure the
+     * jvm options for the nimbus daemon.
+     */
+    public static String NIMBUS_CHILDOPTS = "nimbus.childopts";
+
+
+    /**
+     * How long without heartbeating a task can go before nimbus will consider the
+     * task dead and reassign it to another location.
+     */
+    public static String NIMBUS_TASK_TIMEOUT_SECS = "nimbus.task.timeout.secs";
+
+
+    /**
+     * How often nimbus should wake up to check heartbeats and do reassignments. Note
+     * that if a machine ever goes down Nimbus will immediately wake up and take action.
+     * This parameter is for checking for failures when there's no explicit event like that
+     * occuring.
+     */
+    public static String NIMBUS_MONITOR_FREQ_SECS = "nimbus.monitor.freq.secs";
+
+
+    /**
+     * How long before a supervisor can go without heartbeating before nimbus considers it dead
+     * and stops assigning new work to it.
+     */
+    public static String NIMBUS_SUPERVISOR_TIMEOUT_SECS = "nimbus.supervisor.timeout.secs";
+
+    /**
+     * A special timeout used when a task is initially launched. During launch, this is the timeout
+     * used until the first heartbeat, overriding nimbus.task.timeout.secs.
+     *
+     * <p>A separate timeout exists for launch because there can be quite a bit of overhead
+     * to launching new JVM's and configuring them.</p>
+     */
+    public static String NIMBUS_TASK_LAUNCH_SECS = "nimbus.task.launch.secs";
+
+    /**
+     * Whether or not nimbus should reassign tasks if it detects that a task goes down. 
+     * Defaults to true, and it's not recommended to change this value.
+     */
+    public static String NIMBUS_REASSIGN = "nimbus.reassign";
+
+    /**
+     * During upload/download with the master, how long an upload or download connection is idle
+     * before nimbus considers it dead and drops the connection.
+     */
+    public static String NIMBUS_FILE_COPY_EXPIRATION_SECS = "nimbus.file.copy.expiration.secs";
+
+    /**
+     * A list of ports that can run workers on this supervisor. Each worker uses one port, and
+     * the supervisor will only run one worker per port. Use this configuration to tune
+     * how many workers run on each machine.
+     */
+    public static String SUPERVISOR_SLOTS_PORTS = "supervisor.slots.ports";
+
+
+
+    /**
+     * This parameter is used by the storm-deploy project to configure the
+     * jvm options for the supervisor daemon.
+     */
+    public static String SUPERVISOR_CHILDOPTS = "supervisor.childopts";
+
+
+    /**
+     * How long a worker can go without heartbeating before the supervisor tries to
+     * restart the worker process.
+     */
+    public static String SUPERVISOR_WORKER_TIMEOUT_SECS = "supervisor.worker.timeout.secs";
+
+
+    /**
+     * How long a worker can go without heartbeating during the initial launch before
+     * the supervisor tries to restart the worker process. This value override
+     * supervisor.worker.timeout.secs during launch because there is additional
+     * overhead to starting and configuring the JVM on launch.
+     */
+    public static String SUPERVISOR_WORKER_START_TIMEOUT_SECS = "supervisor.worker.start.timeout.secs";
+
+
+    /**
+     * Whether or not the supervisor should launch workers assigned to it. Defaults
+     * to true -- and you should probably never change this value. This configuration
+     * is used in the Storm unit tests.
+     */
+    public static String SUPERVISOR_ENABLE = "supervisor.enable";
+
+
+    /**
+     * how often the supervisor sends a heartbeat to the master.
+     */
+    public static String SUPERVISOR_HEARTBEAT_FREQUENCY_SECS = "supervisor.heartbeat.frequency.secs";
+
+
+    /**
+     * How often the supervisor checks the worker heartbeats to see if any of them
+     * need to be restarted.
+     */
+    public static String SUPERVISOR_MONITOR_FREQUENCY_SECS = "supervisor.monitor.frequency.secs";
+    
+    /**
+     * The jvm opts provided to workers launched by this supervisor.
+     */
+    public static String WORKER_CHILDOPTS = "worker.childopts";
+
+
+    /**
+     * How often this worker should heartbeat to the supervisor.
+     */
+    public static String WORKER_HEARTBEAT_FREQUENCY_SECS = "worker.heartbeat.frequency.secs";
+
+    /**
+     * How often a task should heartbeat its status to the master.
+     */
+    public static String TASK_HEARTBEAT_FREQUENCY_SECS = "task.heartbeat.frequency.secs";
+
+
+    /**
+     * How often a task should sync its connections with other tasks (if a task is
+     * reassigned, the other tasks sending messages to it need to refresh their connections).
+     * In general though, when a reassignment happens other tasks will be notified
+     * almost immediately. This configuration is here just in case that notification doesn't
+     * come through.
+     */
+    public static String TASK_REFRESH_POLL_SECS = "task.refresh.poll.secs";
+
+    
+    /**
+     * When set to true, Storm will log every message that's emitted.
+     */
+    public static String TOPOLOGY_DEBUG = "topology.debug";
+
+
+    /**
+     * Whether or not the master should optimize topologies by running multiple
+     * tasks in a single thread where appropriate.
+     */
+    public static String TOPOLOGY_OPTIMIZE = "topology.optimize";
+
+    /**
+     * How many processes should be spawned around the cluster to execute this
+     * topology. Each process will execute some number of tasks as threads within
+     * them. This parameter should be used in conjunction with the parallelism hints
+     * on each component in the topology to tune the performance of a topology.
+     */
+    public static String TOPOLOGY_WORKERS = "topology.workers";
+
+    /**
+     * How many acker tasks should be spawned for the topology. An acker task keeps
+     * track of a subset of the tuples emitted by spouts and detects when a spout
+     * tuple is fully processed. When an acker task detects that a spout tuple
+     * is finished, it sends a message to the spout to acknowledge the tuple. The
+     * number of ackers should be scaled with the amount of throughput going
+     * through the cluster for the topology. Typically, you don't need that many
+     * ackers though.
+     *
+     * <p>If this is set to 0, then Storm will immediately ack tuples as soon
+     * as they come off the spout, effectively disabling reliability.</p>
+     */
+    public static String TOPOLOGY_ACKERS = "topology.ackers";
+
+
+    /**
+     * The maximum amount of time given to the topology to fully process a message
+     * emitted by a spout. If the message is not acked within this time frame, Storm
+     * will fail the message on the spout. Some spouts implementations will then replay
+     * the message at a later time.
+     */
+    public static String TOPOLOGY_MESSAGE_TIMEOUT_SECS = "topology.message.timeout.secs";
+
+    /**
+     * A map from unique tokens to the name of classes that implement custom serializations.
+     * Custom serializations are implemented using the {@link backtype.storm.serialization.ISerialization}
+     * interface. The unique tokens you provide are what are serialized on the wire to identify fields. This
+     * is much more efficient than writing the classname. These serializations will be used
+     * automatically when appropriate.
+     */
+    public static String TOPOLOGY_SERIALIZATIONS = "topology.serializations";
+
+    /**
+     * Whether or not Storm should skip the loading of a serialization for which it
+     * does not contain the code. Otherwise, the task will fail to load and will throw
+     * an error at runtime.
+     */
+    public static String TOPOLOGY_SKIP_MISSING_SERIALIZATIONS= "topology.skip.missing.serializations";
+
+
+    /**
+     * The maximum parallelism allowed for a component in this topology. This configuration is
+     * typically used in testing to limit the number of threads spawned in local mode.
+     */
+    public static String TOPOLOGY_MAX_TASK_PARALLELISM="topology.max.task.parallelism";
+
+
+    /**
+     * The maximum parallelism allowed for a component in this topology. This configuration is
+     * typically used in testing to limit the number of threads spawned in local mode.
+     */
+    public static String TOPOLOGY_MAX_SPOUT_PENDING="topology.max.spout.pending";
+
+
+    /**
+     * The maximum amount of time a component gives a source of state to synchronize before it requests
+     * synchronization again.
+     */
+    public static String TOPOLOGY_STATE_SYNCHRONIZATION_TIMEOUT_SECS="topology.state.synchronization.timeout.secs";
+
+    /**
+     * The percentage of tuples to sample to produce stats for a task.
+     */
+    public static String TOPOLOGY_STATS_SAMPLE_RATE="topology.stats.sample.rate";
+
+    /**
+     * The number of threads that should be used by the zeromq context in each worker process.
+     */
+    public static String ZMQ_THREADS = "zmq.threads";
+
+
+    /**
+     * How long a connection should retry sending messages to a target host when
+     * the connection is closed. This is an advanced configuration and can almost
+     * certainly be ignored.
+     */
+    public static String ZMQ_LINGER_MILLIS = "zmq.linger.millis";
+
+    /**
+     * This value is passed to spawned JVMs (e.g., Nimbus, Supervisor, and Workers)
+     * for the java.library.path value. java.library.path tells the JVM where 
+     * to look for native libraries. It is necessary to set this config correctly since
+     * Storm uses the ZeroMQ and JZMQ native libs. 
+     */
+    public static String JAVA_LIBRARY_PATH = "java.library.path";
+
+}
diff --git a/src/jvm/backtype/storm/Constants.java b/src/jvm/backtype/storm/Constants.java
new file mode 100644
index 0000000..acd6fab
--- /dev/null
+++ b/src/jvm/backtype/storm/Constants.java
@@ -0,0 +1,6 @@
+package backtype.storm;
+
+
+public class Constants {
+    public static final int COORDINATED_STREAM_ID = 100;
+}
diff --git a/src/jvm/backtype/storm/ILocalCluster.java b/src/jvm/backtype/storm/ILocalCluster.java
new file mode 100644
index 0000000..3e305aa
--- /dev/null
+++ b/src/jvm/backtype/storm/ILocalCluster.java
@@ -0,0 +1,16 @@
+package backtype.storm;
+
+import backtype.storm.generated.AlreadyAliveException;
+import backtype.storm.generated.InvalidTopologyException;
+import backtype.storm.generated.NotAliveException;
+import backtype.storm.generated.StormTopology;
+import java.util.Map;
+
+
+public interface ILocalCluster {
+    void submitTopology(String topologyName, Map conf, StormTopology topology)
+            throws AlreadyAliveException, InvalidTopologyException;
+    void killTopology(String topologyName)
+            throws NotAliveException;
+    void shutdown();
+}
diff --git a/src/jvm/backtype/storm/StormSubmitter.java b/src/jvm/backtype/storm/StormSubmitter.java
new file mode 100644
index 0000000..f31d04a
--- /dev/null
+++ b/src/jvm/backtype/storm/StormSubmitter.java
@@ -0,0 +1,96 @@
+package backtype.storm;
+
+import backtype.storm.generated.AlreadyAliveException;
+import backtype.storm.generated.InvalidTopologyException;
+import backtype.storm.generated.Nimbus;
+import backtype.storm.generated.StormTopology;
+import backtype.storm.utils.BufferFileInputStream;
+import backtype.storm.utils.NimbusClient;
+import backtype.storm.utils.Utils;
+import java.util.Map;
+import org.apache.log4j.Logger;
+import org.apache.thrift.TException;
+import org.json.simple.JSONValue;
+
+/**
+ * Use this class to submit topologies to run on the Storm cluster. You should run your program
+ * with the "storm jar" command from the command-line, and then use this class to
+ * submit your topologies.
+ */
+public class StormSubmitter {
+    public static Logger LOG = Logger.getLogger(StormSubmitter.class);    
+
+    private static Nimbus.Iface localNimbus = null;
+
+    public static void setLocalNimbus(Nimbus.Iface localNimbusHandler) {
+        StormSubmitter.localNimbus = localNimbusHandler;
+    }
+
+    /**
+     * Submits a topology to run on the cluster. A topology runs forever or until 
+     * explicitly killed.
+     *
+     *
+     * @param name the name of the storm.
+     * @param stormConf the topology-specific configuration. See {@link Config}. 
+     * @param topology the processing to execute.
+     * @throws AlreadyAliveException if a topology with this name is already running
+     * @throws InvalidTopologyException if an invalid topology was submitted
+     */
+    public static void submitTopology(String name, Map stormConf, StormTopology topology) throws AlreadyAliveException, InvalidTopologyException {
+        Map conf = Utils.readStormConfig();
+        conf.putAll(stormConf);
+        try {
+            String serConf = JSONValue.toJSONString(stormConf);
+            if(localNimbus!=null) {
+                LOG.info("Submitting topology " + name + " in local mode");
+                localNimbus.submitTopology(name, null, serConf, topology);
+            } else {
+                submitJar(conf);
+                NimbusClient client = NimbusClient.getConfiguredClient(conf);
+                try {
+                    LOG.info("Submitting topology " +  name + " in distributed mode with conf " + serConf);
+                    client.getClient().submitTopology(name, submittedJar, serConf, topology);
+                } finally {
+                    client.close();
+                }
+            }
+            LOG.info("Finished submitting topology: " +  name);
+        } catch(TException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    private static String submittedJar = null;
+    
+    private static void submitJar(Map conf) {
+        if(submittedJar==null) {
+            LOG.info("Jar not uploaded to master yet. Submitting jar...");
+            String localJar = System.getenv("STORM_JAR");
+            submittedJar = submitJar(conf, localJar);
+        } else {
+            LOG.info("Jar already uploaded to master. Not submitting jar.");
+        }
+    }
+    
+    public static String submitJar(Map conf, String localJar) {
+        NimbusClient client = NimbusClient.getConfiguredClient(conf);
+        try {
+            String uploadLocation = client.getClient().beginFileUpload();
+            LOG.info("Uploading topology jar " + localJar + " to assigned location: " + uploadLocation);
+            BufferFileInputStream is = new BufferFileInputStream(localJar);
+            while(true) {
+                byte[] toSubmit = is.read();
+                if(toSubmit.length==0) break;
+                client.getClient().uploadChunk(uploadLocation, toSubmit);
+            }
+            client.getClient().finishFileUpload(uploadLocation);
+            LOG.info("Successfully uploaded topology jar to assigned location: " + uploadLocation);
+            return uploadLocation;
+        } catch(Exception e) {
+            throw new RuntimeException(e);            
+        } finally {
+            client.close();
+        } 
+    }
+}
diff --git a/src/jvm/backtype/storm/clojure/ClojureBolt.java b/src/jvm/backtype/storm/clojure/ClojureBolt.java
new file mode 100644
index 0000000..5dd6cf5
--- /dev/null
+++ b/src/jvm/backtype/storm/clojure/ClojureBolt.java
@@ -0,0 +1,94 @@
+package backtype.storm.clojure;
+
+import backtype.storm.generated.StreamInfo;
+import backtype.storm.task.OutputCollector;
+import backtype.storm.task.TopologyContext;
+import backtype.storm.topology.IRichBolt;
+import backtype.storm.topology.OutputFieldsDeclarer;
+import backtype.storm.tuple.Fields;
+import backtype.storm.tuple.Tuple;
+import clojure.lang.IFn;
+import clojure.lang.Keyword;
+import clojure.lang.RT;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+
+public class ClojureBolt implements IRichBolt {
+    Map<Integer, StreamInfo> _fields;
+    String _namespace;
+    String _fnName;
+    List<Object> _args;
+    
+    IFn _execute;
+    IFn _cleanup;
+    
+    OutputCollector _collector;
+    
+    
+    public ClojureBolt(String namespace, String fnName, Map<Integer, StreamInfo> fields) {
+        this(namespace, fnName, new ArrayList<Object>(), fields);
+    }
+    
+    public ClojureBolt(String namespace, String fnName, List<Object> args, Map<Integer, StreamInfo> fields) {
+        _namespace = namespace;
+        _fnName = fnName;
+        _args = args;
+        _fields = fields;
+    }
+
+    @Override
+    public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
+        _collector = collector;
+        try {
+          clojure.lang.Compiler.eval(RT.readString("(require '" + _namespace + ")"));
+        } catch (Exception e) {
+          //if playing from the repl and defining functions, file won't exist
+        }
+        IFn hof = (IFn) RT.var(_namespace, _fnName).deref();
+        try {
+            Object fns = hof.applyTo(RT.seq(_args));
+            if(fns instanceof Map) {
+                Map fnMap = (Map) fns;
+                IFn prepare = (IFn) fnMap.get(Keyword.intern("prepare"));
+                if(prepare!=null)
+                    prepare.invoke(stormConf, context, collector);
+                _execute = (IFn) fnMap.get(Keyword.intern("execute"));
+                _cleanup = (IFn) fnMap.get(Keyword.intern("cleanup"));
+            } else {
+                _execute = (IFn) fns;   
+            }
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    @Override
+    public void execute(Tuple input) {
+        try {
+            _execute.invoke(input, _collector);
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }    
+    }
+
+    @Override
+    public void cleanup() {
+        if(_cleanup!=null) {
+            try {
+                _cleanup.invoke(_collector);
+            } catch (Exception e) {
+                throw new RuntimeException(e);
+            }
+        }
+    }
+
+    @Override
+    public void declareOutputFields(OutputFieldsDeclarer declarer) {
+        for(Integer stream: _fields.keySet()) {
+            StreamInfo info = _fields.get(stream);
+            declarer.declareStream(stream, info.is_direct(), new Fields(info.get_output_fields()));
+        }
+    }
+}
diff --git a/src/jvm/backtype/storm/daemon/Shutdownable.java b/src/jvm/backtype/storm/daemon/Shutdownable.java
new file mode 100644
index 0000000..07b1fdd
--- /dev/null
+++ b/src/jvm/backtype/storm/daemon/Shutdownable.java
@@ -0,0 +1,5 @@
+package backtype.storm.daemon;
+
+public interface Shutdownable {
+    public void shutdown();
+}
\ No newline at end of file
diff --git a/src/jvm/backtype/storm/drpc/DRPCScheme.java b/src/jvm/backtype/storm/drpc/DRPCScheme.java
new file mode 100644
index 0000000..3d8955c
--- /dev/null
+++ b/src/jvm/backtype/storm/drpc/DRPCScheme.java
@@ -0,0 +1,25 @@
+package backtype.storm.drpc;
+
+import backtype.storm.spout.Scheme;
+import backtype.storm.tuple.Fields;
+import java.io.UnsupportedEncodingException;
+import java.util.List;
+import java.util.Map;
+import org.json.simple.JSONValue;
+import static backtype.storm.utils.Utils.tuple;
+
+public class DRPCScheme implements Scheme {
+    public List<Object> deserialize(byte[] bytes) {
+        try {
+            Map obj = (Map) JSONValue.parse(new String(bytes, "UTF-8"));
+            return tuple(obj.get("args"), obj.get("return"));
+        } catch (UnsupportedEncodingException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    public Fields getOutputFields() {
+        return new Fields("args", "return");
+    }
+
+}
diff --git a/src/jvm/backtype/storm/drpc/ReturnResults.java b/src/jvm/backtype/storm/drpc/ReturnResults.java
new file mode 100644
index 0000000..e9ed213
--- /dev/null
+++ b/src/jvm/backtype/storm/drpc/ReturnResults.java
@@ -0,0 +1,48 @@
+package backtype.storm.drpc;
+
+import backtype.storm.task.OutputCollector;
+import backtype.storm.task.TopologyContext;
+import backtype.storm.topology.IRichBolt;
+import backtype.storm.topology.OutputFieldsDeclarer;
+import backtype.storm.tuple.Tuple;
+import backtype.storm.utils.DRPCClient;
+import java.util.Map;
+import org.apache.thrift.TException;
+import org.json.simple.JSONValue;
+
+
+public class ReturnResults implements IRichBolt {
+
+    OutputCollector _collector;
+
+    public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
+        _collector = collector;
+    }
+
+    public void execute(Tuple input) {
+        String result = (String) input.getValue(0);
+        String returnInfo = (String) input.getValue(1);
+        if(returnInfo!=null) {
+            Map retMap = (Map) JSONValue.parse(returnInfo);
+            String ip = (String) retMap.get("ip");
+            Long port = (Long) retMap.get("port");
+            String id = (String) retMap.get("id");
+            try {
+                DRPCClient client = new DRPCClient(ip, (int) port.longValue());
+                client.result(id, result);
+                client.close();
+                _collector.ack(input);
+            } catch(TException e) {
+                _collector.fail(input);
+            }
+        }
+    }
+
+    public void cleanup() {
+
+    }
+
+    public void declareOutputFields(OutputFieldsDeclarer declarer) {
+    }
+
+}
diff --git a/src/jvm/backtype/storm/drpc/SpoutAdder.java b/src/jvm/backtype/storm/drpc/SpoutAdder.java
new file mode 100644
index 0000000..b4028e5
--- /dev/null
+++ b/src/jvm/backtype/storm/drpc/SpoutAdder.java
@@ -0,0 +1,6 @@
+package backtype.storm.drpc;
+
+
+public interface SpoutAdder {
+    public void add(String function, String args, String returnInfo);
+}
diff --git a/src/jvm/backtype/storm/generated/AlreadyAliveException.java b/src/jvm/backtype/storm/generated/AlreadyAliveException.java
new file mode 100644
index 0000000..04a4326
--- /dev/null
+++ b/src/jvm/backtype/storm/generated/AlreadyAliveException.java
@@ -0,0 +1,320 @@
+/**
+ * Autogenerated by Thrift
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ */
+package backtype.storm.generated;
+
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.util.Arrays;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.thrift.*;
+import org.apache.thrift.async.*;
+import org.apache.thrift.meta_data.*;
+import org.apache.thrift.transport.*;
+import org.apache.thrift.protocol.*;
+
+public class AlreadyAliveException extends Exception implements TBase<AlreadyAliveException, AlreadyAliveException._Fields>, java.io.Serializable, Cloneable {
+  private static final TStruct STRUCT_DESC = new TStruct("AlreadyAliveException");
+
+  private static final TField MSG_FIELD_DESC = new TField("msg", TType.STRING, (short)1);
+
+  private String msg;
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements TFieldIdEnum {
+    MSG((short)1, "msg");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // MSG
+          return MSG;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+
+  public static final Map<_Fields, FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, FieldMetaData> tmpMap = new EnumMap<_Fields, FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.MSG, new FieldMetaData("msg", TFieldRequirementType.REQUIRED, 
+        new FieldValueMetaData(TType.STRING)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    FieldMetaData.addStructMetaDataMap(AlreadyAliveException.class, metaDataMap);
+  }
+
+  public AlreadyAliveException() {
+  }
+
+  public AlreadyAliveException(
+    String msg)
+  {
+    this();
+    this.msg = msg;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public AlreadyAliveException(AlreadyAliveException other) {
+    if (other.is_set_msg()) {
+      this.msg = other.msg;
+    }
+  }
+
+  public AlreadyAliveException deepCopy() {
+    return new AlreadyAliveException(this);
+  }
+
+  @Deprecated
+  public AlreadyAliveException clone() {
+    return new AlreadyAliveException(this);
+  }
+
+  public String get_msg() {
+    return this.msg;
+  }
+
+  public void set_msg(String msg) {
+    this.msg = msg;
+  }
+
+  public void unset_msg() {
+    this.msg = null;
+  }
+
+  /** Returns true if field msg is set (has been asigned a value) and false otherwise */
+  public boolean is_set_msg() {
+    return this.msg != null;
+  }
+
+  public void set_msg_isSet(boolean value) {
+    if (!value) {
+      this.msg = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case MSG:
+      if (value == null) {
+        unset_msg();
+      } else {
+        set_msg((String)value);
+      }
+      break;
+
+    }
+  }
+
+  public void setFieldValue(int fieldID, Object value) {
+    setFieldValue(_Fields.findByThriftIdOrThrow(fieldID), value);
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case MSG:
+      return get_msg();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  public Object getFieldValue(int fieldId) {
+    return getFieldValue(_Fields.findByThriftIdOrThrow(fieldId));
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been asigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    switch (field) {
+    case MSG:
+      return is_set_msg();
+    }
+    throw new IllegalStateException();
+  }
+
+  public boolean isSet(int fieldID) {
+    return isSet(_Fields.findByThriftIdOrThrow(fieldID));
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof AlreadyAliveException)
+      return this.equals((AlreadyAliveException)that);
+    return false;
+  }
+
+  public boolean equals(AlreadyAliveException that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_msg = true && this.is_set_msg();
+    boolean that_present_msg = true && that.is_set_msg();
+    if (this_present_msg || that_present_msg) {
+      if (!(this_present_msg && that_present_msg))
+        return false;
+      if (!this.msg.equals(that.msg))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    HashCodeBuilder builder = new HashCodeBuilder();
+
+    boolean present_msg = true && (is_set_msg());
+    builder.append(present_msg);
+    if (present_msg)
+      builder.append(msg);
+
+    return builder.toHashCode();
+  }
+
+  public int compareTo(AlreadyAliveException other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+    AlreadyAliveException typedOther = (AlreadyAliveException)other;
+
+    lastComparison = Boolean.valueOf(is_set_msg()).compareTo(typedOther.is_set_msg());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_msg()) {      lastComparison = TBaseHelper.compareTo(this.msg, typedOther.msg);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public void read(TProtocol iprot) throws TException {
+    TField field;
+    iprot.readStructBegin();
+    while (true)
+    {
+      field = iprot.readFieldBegin();
+      if (field.type == TType.STOP) { 
+        break;
+      }
+      switch (field.id) {
+        case 1: // MSG
+          if (field.type == TType.STRING) {
+            this.msg = iprot.readString();
+          } else { 
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        default:
+          TProtocolUtil.skip(iprot, field.type);
+      }
+      iprot.readFieldEnd();
+    }
+    iprot.readStructEnd();
+    validate();
+  }
+
+  public void write(TProtocol oprot) throws TException {
+    validate();
+
+    oprot.writeStructBegin(STRUCT_DESC);
+    if (this.msg != null) {
+      oprot.writeFieldBegin(MSG_FIELD_DESC);
+      oprot.writeString(this.msg);
+      oprot.writeFieldEnd();
+    }
+    oprot.writeFieldStop();
+    oprot.writeStructEnd();
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("AlreadyAliveException(");
+    boolean first = true;
+
+    sb.append("msg:");
+    if (this.msg == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.msg);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws TException {
+    // check for required fields
+    if (!is_set_msg()) {
+      throw new TProtocolException("Required field 'msg' is unset! Struct:" + toString());
+    }
+
+  }
+
+}
+
diff --git a/src/jvm/backtype/storm/generated/Bolt.java b/src/jvm/backtype/storm/generated/Bolt.java
new file mode 100644
index 0000000..fa41601
--- /dev/null
+++ b/src/jvm/backtype/storm/generated/Bolt.java
@@ -0,0 +1,559 @@
+/**
+ * Autogenerated by Thrift
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ */
+package backtype.storm.generated;
+
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.util.Arrays;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.thrift.*;
+import org.apache.thrift.async.*;
+import org.apache.thrift.meta_data.*;
+import org.apache.thrift.transport.*;
+import org.apache.thrift.protocol.*;
+
+public class Bolt implements TBase<Bolt, Bolt._Fields>, java.io.Serializable, Cloneable {
+  private static final TStruct STRUCT_DESC = new TStruct("Bolt");
+
+  private static final TField INPUTS_FIELD_DESC = new TField("inputs", TType.MAP, (short)1);
+  private static final TField BOLT_OBJECT_FIELD_DESC = new TField("bolt_object", TType.STRUCT, (short)2);
+  private static final TField COMMON_FIELD_DESC = new TField("common", TType.STRUCT, (short)3);
+
+  private Map<GlobalStreamId,Grouping> inputs;
+  private ComponentObject bolt_object;
+  private ComponentCommon common;
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements TFieldIdEnum {
+    INPUTS((short)1, "inputs"),
+    BOLT_OBJECT((short)2, "bolt_object"),
+    COMMON((short)3, "common");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // INPUTS
+          return INPUTS;
+        case 2: // BOLT_OBJECT
+          return BOLT_OBJECT;
+        case 3: // COMMON
+          return COMMON;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+
+  public static final Map<_Fields, FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, FieldMetaData> tmpMap = new EnumMap<_Fields, FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.INPUTS, new FieldMetaData("inputs", TFieldRequirementType.REQUIRED, 
+        new MapMetaData(TType.MAP, 
+            new StructMetaData(TType.STRUCT, GlobalStreamId.class), 
+            new StructMetaData(TType.STRUCT, Grouping.class))));
+    tmpMap.put(_Fields.BOLT_OBJECT, new FieldMetaData("bolt_object", TFieldRequirementType.REQUIRED, 
+        new StructMetaData(TType.STRUCT, ComponentObject.class)));
+    tmpMap.put(_Fields.COMMON, new FieldMetaData("common", TFieldRequirementType.REQUIRED, 
+        new StructMetaData(TType.STRUCT, ComponentCommon.class)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    FieldMetaData.addStructMetaDataMap(Bolt.class, metaDataMap);
+  }
+
+  public Bolt() {
+  }
+
+  public Bolt(
+    Map<GlobalStreamId,Grouping> inputs,
+    ComponentObject bolt_object,
+    ComponentCommon common)
+  {
+    this();
+    this.inputs = inputs;
+    this.bolt_object = bolt_object;
+    this.common = common;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public Bolt(Bolt other) {
+    if (other.is_set_inputs()) {
+      Map<GlobalStreamId,Grouping> __this__inputs = new HashMap<GlobalStreamId,Grouping>();
+      for (Map.Entry<GlobalStreamId, Grouping> other_element : other.inputs.entrySet()) {
+
+        GlobalStreamId other_element_key = other_element.getKey();
+        Grouping other_element_value = other_element.getValue();
+
+        GlobalStreamId __this__inputs_copy_key = new GlobalStreamId(other_element_key);
+
+        Grouping __this__inputs_copy_value = new Grouping(other_element_value);
+
+        __this__inputs.put(__this__inputs_copy_key, __this__inputs_copy_value);
+      }
+      this.inputs = __this__inputs;
+    }
+    if (other.is_set_bolt_object()) {
+      this.bolt_object = new ComponentObject(other.bolt_object);
+    }
+    if (other.is_set_common()) {
+      this.common = new ComponentCommon(other.common);
+    }
+  }
+
+  public Bolt deepCopy() {
+    return new Bolt(this);
+  }
+
+  @Deprecated
+  public Bolt clone() {
+    return new Bolt(this);
+  }
+
+  public int get_inputs_size() {
+    return (this.inputs == null) ? 0 : this.inputs.size();
+  }
+
+  public void put_to_inputs(GlobalStreamId key, Grouping val) {
+    if (this.inputs == null) {
+      this.inputs = new HashMap<GlobalStreamId,Grouping>();
+    }
+    this.inputs.put(key, val);
+  }
+
+  public Map<GlobalStreamId,Grouping> get_inputs() {
+    return this.inputs;
+  }
+
+  public void set_inputs(Map<GlobalStreamId,Grouping> inputs) {
+    this.inputs = inputs;
+  }
+
+  public void unset_inputs() {
+    this.inputs = null;
+  }
+
+  /** Returns true if field inputs is set (has been asigned a value) and false otherwise */
+  public boolean is_set_inputs() {
+    return this.inputs != null;
+  }
+
+  public void set_inputs_isSet(boolean value) {
+    if (!value) {
+      this.inputs = null;
+    }
+  }
+
+  public ComponentObject get_bolt_object() {
+    return this.bolt_object;
+  }
+
+  public void set_bolt_object(ComponentObject bolt_object) {
+    this.bolt_object = bolt_object;
+  }
+
+  public void unset_bolt_object() {
+    this.bolt_object = null;
+  }
+
+  /** Returns true if field bolt_object is set (has been asigned a value) and false otherwise */
+  public boolean is_set_bolt_object() {
+    return this.bolt_object != null;
+  }
+
+  public void set_bolt_object_isSet(boolean value) {
+    if (!value) {
+      this.bolt_object = null;
+    }
+  }
+
+  public ComponentCommon get_common() {
+    return this.common;
+  }
+
+  public void set_common(ComponentCommon common) {
+    this.common = common;
+  }
+
+  public void unset_common() {
+    this.common = null;
+  }
+
+  /** Returns true if field common is set (has been asigned a value) and false otherwise */
+  public boolean is_set_common() {
+    return this.common != null;
+  }
+
+  public void set_common_isSet(boolean value) {
+    if (!value) {
+      this.common = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case INPUTS:
+      if (value == null) {
+        unset_inputs();
+      } else {
+        set_inputs((Map<GlobalStreamId,Grouping>)value);
+      }
+      break;
+
+    case BOLT_OBJECT:
+      if (value == null) {
+        unset_bolt_object();
+      } else {
+        set_bolt_object((ComponentObject)value);
+      }
+      break;
+
+    case COMMON:
+      if (value == null) {
+        unset_common();
+      } else {
+        set_common((ComponentCommon)value);
+      }
+      break;
+
+    }
+  }
+
+  public void setFieldValue(int fieldID, Object value) {
+    setFieldValue(_Fields.findByThriftIdOrThrow(fieldID), value);
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case INPUTS:
+      return get_inputs();
+
+    case BOLT_OBJECT:
+      return get_bolt_object();
+
+    case COMMON:
+      return get_common();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  public Object getFieldValue(int fieldId) {
+    return getFieldValue(_Fields.findByThriftIdOrThrow(fieldId));
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been asigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    switch (field) {
+    case INPUTS:
+      return is_set_inputs();
+    case BOLT_OBJECT:
+      return is_set_bolt_object();
+    case COMMON:
+      return is_set_common();
+    }
+    throw new IllegalStateException();
+  }
+
+  public boolean isSet(int fieldID) {
+    return isSet(_Fields.findByThriftIdOrThrow(fieldID));
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof Bolt)
+      return this.equals((Bolt)that);
+    return false;
+  }
+
+  public boolean equals(Bolt that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_inputs = true && this.is_set_inputs();
+    boolean that_present_inputs = true && that.is_set_inputs();
+    if (this_present_inputs || that_present_inputs) {
+      if (!(this_present_inputs && that_present_inputs))
+        return false;
+      if (!this.inputs.equals(that.inputs))
+        return false;
+    }
+
+    boolean this_present_bolt_object = true && this.is_set_bolt_object();
+    boolean that_present_bolt_object = true && that.is_set_bolt_object();
+    if (this_present_bolt_object || that_present_bolt_object) {
+      if (!(this_present_bolt_object && that_present_bolt_object))
+        return false;
+      if (!this.bolt_object.equals(that.bolt_object))
+        return false;
+    }
+
+    boolean this_present_common = true && this.is_set_common();
+    boolean that_present_common = true && that.is_set_common();
+    if (this_present_common || that_present_common) {
+      if (!(this_present_common && that_present_common))
+        return false;
+      if (!this.common.equals(that.common))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    HashCodeBuilder builder = new HashCodeBuilder();
+
+    boolean present_inputs = true && (is_set_inputs());
+    builder.append(present_inputs);
+    if (present_inputs)
+      builder.append(inputs);
+
+    boolean present_bolt_object = true && (is_set_bolt_object());
+    builder.append(present_bolt_object);
+    if (present_bolt_object)
+      builder.append(bolt_object);
+
+    boolean present_common = true && (is_set_common());
+    builder.append(present_common);
+    if (present_common)
+      builder.append(common);
+
+    return builder.toHashCode();
+  }
+
+  public int compareTo(Bolt other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+    Bolt typedOther = (Bolt)other;
+
+    lastComparison = Boolean.valueOf(is_set_inputs()).compareTo(typedOther.is_set_inputs());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_inputs()) {      lastComparison = TBaseHelper.compareTo(this.inputs, typedOther.inputs);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_bolt_object()).compareTo(typedOther.is_set_bolt_object());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_bolt_object()) {      lastComparison = TBaseHelper.compareTo(this.bolt_object, typedOther.bolt_object);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_common()).compareTo(typedOther.is_set_common());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_common()) {      lastComparison = TBaseHelper.compareTo(this.common, typedOther.common);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public void read(TProtocol iprot) throws TException {
+    TField field;
+    iprot.readStructBegin();
+    while (true)
+    {
+      field = iprot.readFieldBegin();
+      if (field.type == TType.STOP) { 
+        break;
+      }
+      switch (field.id) {
+        case 1: // INPUTS
+          if (field.type == TType.MAP) {
+            {
+              TMap _map13 = iprot.readMapBegin();
+              this.inputs = new HashMap<GlobalStreamId,Grouping>(2*_map13.size);
+              for (int _i14 = 0; _i14 < _map13.size; ++_i14)
+              {
+                GlobalStreamId _key15;
+                Grouping _val16;
+                _key15 = new GlobalStreamId();
+                _key15.read(iprot);
+                _val16 = new Grouping();
+                _val16.read(iprot);
+                this.inputs.put(_key15, _val16);
+              }
+              iprot.readMapEnd();
+            }
+          } else { 
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 2: // BOLT_OBJECT
+          if (field.type == TType.STRUCT) {
+            this.bolt_object = new ComponentObject();
+            this.bolt_object.read(iprot);
+          } else { 
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 3: // COMMON
+          if (field.type == TType.STRUCT) {
+            this.common = new ComponentCommon();
+            this.common.read(iprot);
+          } else { 
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        default:
+          TProtocolUtil.skip(iprot, field.type);
+      }
+      iprot.readFieldEnd();
+    }
+    iprot.readStructEnd();
+    validate();
+  }
+
+  public void write(TProtocol oprot) throws TException {
+    validate();
+
+    oprot.writeStructBegin(STRUCT_DESC);
+    if (this.inputs != null) {
+      oprot.writeFieldBegin(INPUTS_FIELD_DESC);
+      {
+        oprot.writeMapBegin(new TMap(TType.STRUCT, TType.STRUCT, this.inputs.size()));
+        for (Map.Entry<GlobalStreamId, Grouping> _iter17 : this.inputs.entrySet())
+        {
+          _iter17.getKey().write(oprot);
+          _iter17.getValue().write(oprot);
+        }
+        oprot.writeMapEnd();
+      }
+      oprot.writeFieldEnd();
+    }
+    if (this.bolt_object != null) {
+      oprot.writeFieldBegin(BOLT_OBJECT_FIELD_DESC);
+      this.bolt_object.write(oprot);
+      oprot.writeFieldEnd();
+    }
+    if (this.common != null) {
+      oprot.writeFieldBegin(COMMON_FIELD_DESC);
+      this.common.write(oprot);
+      oprot.writeFieldEnd();
+    }
+    oprot.writeFieldStop();
+    oprot.writeStructEnd();
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("Bolt(");
+    boolean first = true;
+
+    sb.append("inputs:");
+    if (this.inputs == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.inputs);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("bolt_object:");
+    if (this.bolt_object == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.bolt_object);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("common:");
+    if (this.common == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.common);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws TException {
+    // check for required fields
+    if (!is_set_inputs()) {
+      throw new TProtocolException("Required field 'inputs' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_bolt_object()) {
+      throw new TProtocolException("Required field 'bolt_object' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_common()) {
+      throw new TProtocolException("Required field 'common' is unset! Struct:" + toString());
+    }
+
+  }
+
+}
+
diff --git a/src/jvm/backtype/storm/generated/BoltStats.java b/src/jvm/backtype/storm/generated/BoltStats.java
new file mode 100644
index 0000000..41462c4
--- /dev/null
+++ b/src/jvm/backtype/storm/generated/BoltStats.java
@@ -0,0 +1,747 @@
+/**
+ * Autogenerated by Thrift
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ */
+package backtype.storm.generated;
+
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.util.Arrays;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.thrift.*;
+import org.apache.thrift.async.*;
+import org.apache.thrift.meta_data.*;
+import org.apache.thrift.transport.*;
+import org.apache.thrift.protocol.*;
+
+public class BoltStats implements TBase<BoltStats, BoltStats._Fields>, java.io.Serializable, Cloneable {
+  private static final TStruct STRUCT_DESC = new TStruct("BoltStats");
+
+  private static final TField ACKED_FIELD_DESC = new TField("acked", TType.MAP, (short)1);
+  private static final TField FAILED_FIELD_DESC = new TField("failed", TType.MAP, (short)2);
+  private static final TField PROCESS_MS_AVG_FIELD_DESC = new TField("process_ms_avg", TType.MAP, (short)3);
+
+  private Map<String,Map<GlobalStreamId,Long>> acked;
+  private Map<String,Map<GlobalStreamId,Long>> failed;
+  private Map<String,Map<GlobalStreamId,Double>> process_ms_avg;
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements TFieldIdEnum {
+    ACKED((short)1, "acked"),
+    FAILED((short)2, "failed"),
+    PROCESS_MS_AVG((short)3, "process_ms_avg");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // ACKED
+          return ACKED;
+        case 2: // FAILED
+          return FAILED;
+        case 3: // PROCESS_MS_AVG
+          return PROCESS_MS_AVG;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+
+  public static final Map<_Fields, FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, FieldMetaData> tmpMap = new EnumMap<_Fields, FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.ACKED, new FieldMetaData("acked", TFieldRequirementType.REQUIRED, 
+        new MapMetaData(TType.MAP, 
+            new FieldValueMetaData(TType.STRING), 
+            new MapMetaData(TType.MAP, 
+                new StructMetaData(TType.STRUCT, GlobalStreamId.class), 
+                new FieldValueMetaData(TType.I64)))));
+    tmpMap.put(_Fields.FAILED, new FieldMetaData("failed", TFieldRequirementType.REQUIRED, 
+        new MapMetaData(TType.MAP, 
+            new FieldValueMetaData(TType.STRING), 
+            new MapMetaData(TType.MAP, 
+                new StructMetaData(TType.STRUCT, GlobalStreamId.class), 
+                new FieldValueMetaData(TType.I64)))));
+    tmpMap.put(_Fields.PROCESS_MS_AVG, new FieldMetaData("process_ms_avg", TFieldRequirementType.REQUIRED, 
+        new MapMetaData(TType.MAP, 
+            new FieldValueMetaData(TType.STRING), 
+            new MapMetaData(TType.MAP, 
+                new StructMetaData(TType.STRUCT, GlobalStreamId.class), 
+                new FieldValueMetaData(TType.DOUBLE)))));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    FieldMetaData.addStructMetaDataMap(BoltStats.class, metaDataMap);
+  }
+
+  public BoltStats() {
+  }
+
+  public BoltStats(
+    Map<String,Map<GlobalStreamId,Long>> acked,
+    Map<String,Map<GlobalStreamId,Long>> failed,
+    Map<String,Map<GlobalStreamId,Double>> process_ms_avg)
+  {
+    this();
+    this.acked = acked;
+    this.failed = failed;
+    this.process_ms_avg = process_ms_avg;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public BoltStats(BoltStats other) {
+    if (other.is_set_acked()) {
+      Map<String,Map<GlobalStreamId,Long>> __this__acked = new HashMap<String,Map<GlobalStreamId,Long>>();
+      for (Map.Entry<String, Map<GlobalStreamId,Long>> other_element : other.acked.entrySet()) {
+
+        String other_element_key = other_element.getKey();
+        Map<GlobalStreamId,Long> other_element_value = other_element.getValue();
+
+        String __this__acked_copy_key = other_element_key;
+
+        Map<GlobalStreamId,Long> __this__acked_copy_value = new HashMap<GlobalStreamId,Long>();
+        for (Map.Entry<GlobalStreamId, Long> other_element_value_element : other_element_value.entrySet()) {
+
+          GlobalStreamId other_element_value_element_key = other_element_value_element.getKey();
+          Long other_element_value_element_value = other_element_value_element.getValue();
+
+          GlobalStreamId __this__acked_copy_value_copy_key = new GlobalStreamId(other_element_value_element_key);
+
+          Long __this__acked_copy_value_copy_value = other_element_value_element_value;
+
+          __this__acked_copy_value.put(__this__acked_copy_value_copy_key, __this__acked_copy_value_copy_value);
+        }
+
+        __this__acked.put(__this__acked_copy_key, __this__acked_copy_value);
+      }
+      this.acked = __this__acked;
+    }
+    if (other.is_set_failed()) {
+      Map<String,Map<GlobalStreamId,Long>> __this__failed = new HashMap<String,Map<GlobalStreamId,Long>>();
+      for (Map.Entry<String, Map<GlobalStreamId,Long>> other_element : other.failed.entrySet()) {
+
+        String other_element_key = other_element.getKey();
+        Map<GlobalStreamId,Long> other_element_value = other_element.getValue();
+
+        String __this__failed_copy_key = other_element_key;
+
+        Map<GlobalStreamId,Long> __this__failed_copy_value = new HashMap<GlobalStreamId,Long>();
+        for (Map.Entry<GlobalStreamId, Long> other_element_value_element : other_element_value.entrySet()) {
+
+          GlobalStreamId other_element_value_element_key = other_element_value_element.getKey();
+          Long other_element_value_element_value = other_element_value_element.getValue();
+
+          GlobalStreamId __this__failed_copy_value_copy_key = new GlobalStreamId(other_element_value_element_key);
+
+          Long __this__failed_copy_value_copy_value = other_element_value_element_value;
+
+          __this__failed_copy_value.put(__this__failed_copy_value_copy_key, __this__failed_copy_value_copy_value);
+        }
+
+        __this__failed.put(__this__failed_copy_key, __this__failed_copy_value);
+      }
+      this.failed = __this__failed;
+    }
+    if (other.is_set_process_ms_avg()) {
+      Map<String,Map<GlobalStreamId,Double>> __this__process_ms_avg = new HashMap<String,Map<GlobalStreamId,Double>>();
+      for (Map.Entry<String, Map<GlobalStreamId,Double>> other_element : other.process_ms_avg.entrySet()) {
+
+        String other_element_key = other_element.getKey();
+        Map<GlobalStreamId,Double> other_element_value = other_element.getValue();
+
+        String __this__process_ms_avg_copy_key = other_element_key;
+
+        Map<GlobalStreamId,Double> __this__process_ms_avg_copy_value = new HashMap<GlobalStreamId,Double>();
+        for (Map.Entry<GlobalStreamId, Double> other_element_value_element : other_element_value.entrySet()) {
+
+          GlobalStreamId other_element_value_element_key = other_element_value_element.getKey();
+          Double other_element_value_element_value = other_element_value_element.getValue();
+
+          GlobalStreamId __this__process_ms_avg_copy_value_copy_key = new GlobalStreamId(other_element_value_element_key);
+
+          Double __this__process_ms_avg_copy_value_copy_value = other_element_value_element_value;
+
+          __this__process_ms_avg_copy_value.put(__this__process_ms_avg_copy_value_copy_key, __this__process_ms_avg_copy_value_copy_value);
+        }
+
+        __this__process_ms_avg.put(__this__process_ms_avg_copy_key, __this__process_ms_avg_copy_value);
+      }
+      this.process_ms_avg = __this__process_ms_avg;
+    }
+  }
+
+  public BoltStats deepCopy() {
+    return new BoltStats(this);
+  }
+
+  @Deprecated
+  public BoltStats clone() {
+    return new BoltStats(this);
+  }
+
+  public int get_acked_size() {
+    return (this.acked == null) ? 0 : this.acked.size();
+  }
+
+  public void put_to_acked(String key, Map<GlobalStreamId,Long> val) {
+    if (this.acked == null) {
+      this.acked = new HashMap<String,Map<GlobalStreamId,Long>>();
+    }
+    this.acked.put(key, val);
+  }
+
+  public Map<String,Map<GlobalStreamId,Long>> get_acked() {
+    return this.acked;
+  }
+
+  public void set_acked(Map<String,Map<GlobalStreamId,Long>> acked) {
+    this.acked = acked;
+  }
+
+  public void unset_acked() {
+    this.acked = null;
+  }
+
+  /** Returns true if field acked is set (has been asigned a value) and false otherwise */
+  public boolean is_set_acked() {
+    return this.acked != null;
+  }
+
+  public void set_acked_isSet(boolean value) {
+    if (!value) {
+      this.acked = null;
+    }
+  }
+
+  public int get_failed_size() {
+    return (this.failed == null) ? 0 : this.failed.size();
+  }
+
+  public void put_to_failed(String key, Map<GlobalStreamId,Long> val) {
+    if (this.failed == null) {
+      this.failed = new HashMap<String,Map<GlobalStreamId,Long>>();
+    }
+    this.failed.put(key, val);
+  }
+
+  public Map<String,Map<GlobalStreamId,Long>> get_failed() {
+    return this.failed;
+  }
+
+  public void set_failed(Map<String,Map<GlobalStreamId,Long>> failed) {
+    this.failed = failed;
+  }
+
+  public void unset_failed() {
+    this.failed = null;
+  }
+
+  /** Returns true if field failed is set (has been asigned a value) and false otherwise */
+  public boolean is_set_failed() {
+    return this.failed != null;
+  }
+
+  public void set_failed_isSet(boolean value) {
+    if (!value) {
+      this.failed = null;
+    }
+  }
+
+  public int get_process_ms_avg_size() {
+    return (this.process_ms_avg == null) ? 0 : this.process_ms_avg.size();
+  }
+
+  public void put_to_process_ms_avg(String key, Map<GlobalStreamId,Double> val) {
+    if (this.process_ms_avg == null) {
+      this.process_ms_avg = new HashMap<String,Map<GlobalStreamId,Double>>();
+    }
+    this.process_ms_avg.put(key, val);
+  }
+
+  public Map<String,Map<GlobalStreamId,Double>> get_process_ms_avg() {
+    return this.process_ms_avg;
+  }
+
+  public void set_process_ms_avg(Map<String,Map<GlobalStreamId,Double>> process_ms_avg) {
+    this.process_ms_avg = process_ms_avg;
+  }
+
+  public void unset_process_ms_avg() {
+    this.process_ms_avg = null;
+  }
+
+  /** Returns true if field process_ms_avg is set (has been asigned a value) and false otherwise */
+  public boolean is_set_process_ms_avg() {
+    return this.process_ms_avg != null;
+  }
+
+  public void set_process_ms_avg_isSet(boolean value) {
+    if (!value) {
+      this.process_ms_avg = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case ACKED:
+      if (value == null) {
+        unset_acked();
+      } else {
+        set_acked((Map<String,Map<GlobalStreamId,Long>>)value);
+      }
+      break;
+
+    case FAILED:
+      if (value == null) {
+        unset_failed();
+      } else {
+        set_failed((Map<String,Map<GlobalStreamId,Long>>)value);
+      }
+      break;
+
+    case PROCESS_MS_AVG:
+      if (value == null) {
+        unset_process_ms_avg();
+      } else {
+        set_process_ms_avg((Map<String,Map<GlobalStreamId,Double>>)value);
+      }
+      break;
+
+    }
+  }
+
+  public void setFieldValue(int fieldID, Object value) {
+    setFieldValue(_Fields.findByThriftIdOrThrow(fieldID), value);
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case ACKED:
+      return get_acked();
+
+    case FAILED:
+      return get_failed();
+
+    case PROCESS_MS_AVG:
+      return get_process_ms_avg();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  public Object getFieldValue(int fieldId) {
+    return getFieldValue(_Fields.findByThriftIdOrThrow(fieldId));
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been asigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    switch (field) {
+    case ACKED:
+      return is_set_acked();
+    case FAILED:
+      return is_set_failed();
+    case PROCESS_MS_AVG:
+      return is_set_process_ms_avg();
+    }
+    throw new IllegalStateException();
+  }
+
+  public boolean isSet(int fieldID) {
+    return isSet(_Fields.findByThriftIdOrThrow(fieldID));
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof BoltStats)
+      return this.equals((BoltStats)that);
+    return false;
+  }
+
+  public boolean equals(BoltStats that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_acked = true && this.is_set_acked();
+    boolean that_present_acked = true && that.is_set_acked();
+    if (this_present_acked || that_present_acked) {
+      if (!(this_present_acked && that_present_acked))
+        return false;
+      if (!this.acked.equals(that.acked))
+        return false;
+    }
+
+    boolean this_present_failed = true && this.is_set_failed();
+    boolean that_present_failed = true && that.is_set_failed();
+    if (this_present_failed || that_present_failed) {
+      if (!(this_present_failed && that_present_failed))
+        return false;
+      if (!this.failed.equals(that.failed))
+        return false;
+    }
+
+    boolean this_present_process_ms_avg = true && this.is_set_process_ms_avg();
+    boolean that_present_process_ms_avg = true && that.is_set_process_ms_avg();
+    if (this_present_process_ms_avg || that_present_process_ms_avg) {
+      if (!(this_present_process_ms_avg && that_present_process_ms_avg))
+        return false;
+      if (!this.process_ms_avg.equals(that.process_ms_avg))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    HashCodeBuilder builder = new HashCodeBuilder();
+
+    boolean present_acked = true && (is_set_acked());
+    builder.append(present_acked);
+    if (present_acked)
+      builder.append(acked);
+
+    boolean present_failed = true && (is_set_failed());
+    builder.append(present_failed);
+    if (present_failed)
+      builder.append(failed);
+
+    boolean present_process_ms_avg = true && (is_set_process_ms_avg());
+    builder.append(present_process_ms_avg);
+    if (present_process_ms_avg)
+      builder.append(process_ms_avg);
+
+    return builder.toHashCode();
+  }
+
+  public int compareTo(BoltStats other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+    BoltStats typedOther = (BoltStats)other;
+
+    lastComparison = Boolean.valueOf(is_set_acked()).compareTo(typedOther.is_set_acked());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_acked()) {      lastComparison = TBaseHelper.compareTo(this.acked, typedOther.acked);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_failed()).compareTo(typedOther.is_set_failed());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_failed()) {      lastComparison = TBaseHelper.compareTo(this.failed, typedOther.failed);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_process_ms_avg()).compareTo(typedOther.is_set_process_ms_avg());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_process_ms_avg()) {      lastComparison = TBaseHelper.compareTo(this.process_ms_avg, typedOther.process_ms_avg);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public void read(TProtocol iprot) throws TException {
+    TField field;
+    iprot.readStructBegin();
+    while (true)
+    {
+      field = iprot.readFieldBegin();
+      if (field.type == TType.STOP) { 
+        break;
+      }
+      switch (field.id) {
+        case 1: // ACKED
+          if (field.type == TType.MAP) {
+            {
+              TMap _map41 = iprot.readMapBegin();
+              this.acked = new HashMap<String,Map<GlobalStreamId,Long>>(2*_map41.size);
+              for (int _i42 = 0; _i42 < _map41.size; ++_i42)
+              {
+                String _key43;
+                Map<GlobalStreamId,Long> _val44;
+                _key43 = iprot.readString();
+                {
+                  TMap _map45 = iprot.readMapBegin();
+                  _val44 = new HashMap<GlobalStreamId,Long>(2*_map45.size);
+                  for (int _i46 = 0; _i46 < _map45.size; ++_i46)
+                  {
+                    GlobalStreamId _key47;
+                    long _val48;
+                    _key47 = new GlobalStreamId();
+                    _key47.read(iprot);
+                    _val48 = iprot.readI64();
+                    _val44.put(_key47, _val48);
+                  }
+                  iprot.readMapEnd();
+                }
+                this.acked.put(_key43, _val44);
+              }
+              iprot.readMapEnd();
+            }
+          } else { 
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 2: // FAILED
+          if (field.type == TType.MAP) {
+            {
+              TMap _map49 = iprot.readMapBegin();
+              this.failed = new HashMap<String,Map<GlobalStreamId,Long>>(2*_map49.size);
+              for (int _i50 = 0; _i50 < _map49.size; ++_i50)
+              {
+                String _key51;
+                Map<GlobalStreamId,Long> _val52;
+                _key51 = iprot.readString();
+                {
+                  TMap _map53 = iprot.readMapBegin();
+                  _val52 = new HashMap<GlobalStreamId,Long>(2*_map53.size);
+                  for (int _i54 = 0; _i54 < _map53.size; ++_i54)
+                  {
+                    GlobalStreamId _key55;
+                    long _val56;
+                    _key55 = new GlobalStreamId();
+                    _key55.read(iprot);
+                    _val56 = iprot.readI64();
+                    _val52.put(_key55, _val56);
+                  }
+                  iprot.readMapEnd();
+                }
+                this.failed.put(_key51, _val52);
+              }
+              iprot.readMapEnd();
+            }
+          } else { 
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 3: // PROCESS_MS_AVG
+          if (field.type == TType.MAP) {
+            {
+              TMap _map57 = iprot.readMapBegin();
+              this.process_ms_avg = new HashMap<String,Map<GlobalStreamId,Double>>(2*_map57.size);
+              for (int _i58 = 0; _i58 < _map57.size; ++_i58)
+              {
+                String _key59;
+                Map<GlobalStreamId,Double> _val60;
+                _key59 = iprot.readString();
+                {
+                  TMap _map61 = iprot.readMapBegin();
+                  _val60 = new HashMap<GlobalStreamId,Double>(2*_map61.size);
+                  for (int _i62 = 0; _i62 < _map61.size; ++_i62)
+                  {
+                    GlobalStreamId _key63;
+                    double _val64;
+                    _key63 = new GlobalStreamId();
+                    _key63.read(iprot);
+                    _val64 = iprot.readDouble();
+                    _val60.put(_key63, _val64);
+                  }
+                  iprot.readMapEnd();
+                }
+                this.process_ms_avg.put(_key59, _val60);
+              }
+              iprot.readMapEnd();
+            }
+          } else { 
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        default:
+          TProtocolUtil.skip(iprot, field.type);
+      }
+      iprot.readFieldEnd();
+    }
+    iprot.readStructEnd();
+    validate();
+  }
+
+  public void write(TProtocol oprot) throws TException {
+    validate();
+
+    oprot.writeStructBegin(STRUCT_DESC);
+    if (this.acked != null) {
+      oprot.writeFieldBegin(ACKED_FIELD_DESC);
+      {
+        oprot.writeMapBegin(new TMap(TType.STRING, TType.MAP, this.acked.size()));
+        for (Map.Entry<String, Map<GlobalStreamId,Long>> _iter65 : this.acked.entrySet())
+        {
+          oprot.writeString(_iter65.getKey());
+          {
+            oprot.writeMapBegin(new TMap(TType.STRUCT, TType.I64, _iter65.getValue().size()));
+            for (Map.Entry<GlobalStreamId, Long> _iter66 : _iter65.getValue().entrySet())
+            {
+              _iter66.getKey().write(oprot);
+              oprot.writeI64(_iter66.getValue());
+            }
+            oprot.writeMapEnd();
+          }
+        }
+        oprot.writeMapEnd();
+      }
+      oprot.writeFieldEnd();
+    }
+    if (this.failed != null) {
+      oprot.writeFieldBegin(FAILED_FIELD_DESC);
+      {
+        oprot.writeMapBegin(new TMap(TType.STRING, TType.MAP, this.failed.size()));
+        for (Map.Entry<String, Map<GlobalStreamId,Long>> _iter67 : this.failed.entrySet())
+        {
+          oprot.writeString(_iter67.getKey());
+          {
+            oprot.writeMapBegin(new TMap(TType.STRUCT, TType.I64, _iter67.getValue().size()));
+            for (Map.Entry<GlobalStreamId, Long> _iter68 : _iter67.getValue().entrySet())
+            {
+              _iter68.getKey().write(oprot);
+              oprot.writeI64(_iter68.getValue());
+            }
+            oprot.writeMapEnd();
+          }
+        }
+        oprot.writeMapEnd();
+      }
+      oprot.writeFieldEnd();
+    }
+    if (this.process_ms_avg != null) {
+      oprot.writeFieldBegin(PROCESS_MS_AVG_FIELD_DESC);
+      {
+        oprot.writeMapBegin(new TMap(TType.STRING, TType.MAP, this.process_ms_avg.size()));
+        for (Map.Entry<String, Map<GlobalStreamId,Double>> _iter69 : this.process_ms_avg.entrySet())
+        {
+          oprot.writeString(_iter69.getKey());
+          {
+            oprot.writeMapBegin(new TMap(TType.STRUCT, TType.DOUBLE, _iter69.getValue().size()));
+            for (Map.Entry<GlobalStreamId, Double> _iter70 : _iter69.getValue().entrySet())
+            {
+              _iter70.getKey().write(oprot);
+              oprot.writeDouble(_iter70.getValue());
+            }
+            oprot.writeMapEnd();
+          }
+        }
+        oprot.writeMapEnd();
+      }
+      oprot.writeFieldEnd();
+    }
+    oprot.writeFieldStop();
+    oprot.writeStructEnd();
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("BoltStats(");
+    boolean first = true;
+
+    sb.append("acked:");
+    if (this.acked == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.acked);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("failed:");
+    if (this.failed == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.failed);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("process_ms_avg:");
+    if (this.process_ms_avg == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.process_ms_avg);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws TException {
+    // check for required fields
+    if (!is_set_acked()) {
+      throw new TProtocolException("Required field 'acked' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_failed()) {
+      throw new TProtocolException("Required field 'failed' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_process_ms_avg()) {
+      throw new TProtocolException("Required field 'process_ms_avg' is unset! Struct:" + toString());
+    }
+
+  }
+
+}
+
diff --git a/src/jvm/backtype/storm/generated/ClusterSummary.java b/src/jvm/backtype/storm/generated/ClusterSummary.java
new file mode 100644
index 0000000..666a9aa
--- /dev/null
+++ b/src/jvm/backtype/storm/generated/ClusterSummary.java
@@ -0,0 +1,583 @@
+/**
+ * Autogenerated by Thrift
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ */
+package backtype.storm.generated;
+
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.util.Arrays;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.thrift.*;
+import org.apache.thrift.async.*;
+import org.apache.thrift.meta_data.*;
+import org.apache.thrift.transport.*;
+import org.apache.thrift.protocol.*;
+
+public class ClusterSummary implements TBase<ClusterSummary, ClusterSummary._Fields>, java.io.Serializable, Cloneable {
+  private static final TStruct STRUCT_DESC = new TStruct("ClusterSummary");
+
+  private static final TField SUPERVISORS_FIELD_DESC = new TField("supervisors", TType.LIST, (short)1);
+  private static final TField NIMBUS_UPTIME_SECS_FIELD_DESC = new TField("nimbus_uptime_secs", TType.I32, (short)2);
+  private static final TField TOPOLOGIES_FIELD_DESC = new TField("topologies", TType.LIST, (short)3);
+
+  private List<SupervisorSummary> supervisors;
+  private int nimbus_uptime_secs;
+  private List<TopologySummary> topologies;
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements TFieldIdEnum {
+    SUPERVISORS((short)1, "supervisors"),
+    NIMBUS_UPTIME_SECS((short)2, "nimbus_uptime_secs"),
+    TOPOLOGIES((short)3, "topologies");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // SUPERVISORS
+          return SUPERVISORS;
+        case 2: // NIMBUS_UPTIME_SECS
+          return NIMBUS_UPTIME_SECS;
+        case 3: // TOPOLOGIES
+          return TOPOLOGIES;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __NIMBUS_UPTIME_SECS_ISSET_ID = 0;
+  private BitSet __isset_bit_vector = new BitSet(1);
+
+  public static final Map<_Fields, FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, FieldMetaData> tmpMap = new EnumMap<_Fields, FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.SUPERVISORS, new FieldMetaData("supervisors", TFieldRequirementType.REQUIRED, 
+        new ListMetaData(TType.LIST, 
+            new StructMetaData(TType.STRUCT, SupervisorSummary.class))));
+    tmpMap.put(_Fields.NIMBUS_UPTIME_SECS, new FieldMetaData("nimbus_uptime_secs", TFieldRequirementType.REQUIRED, 
+        new FieldValueMetaData(TType.I32)));
+    tmpMap.put(_Fields.TOPOLOGIES, new FieldMetaData("topologies", TFieldRequirementType.REQUIRED, 
+        new ListMetaData(TType.LIST, 
+            new StructMetaData(TType.STRUCT, TopologySummary.class))));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    FieldMetaData.addStructMetaDataMap(ClusterSummary.class, metaDataMap);
+  }
+
+  public ClusterSummary() {
+  }
+
+  public ClusterSummary(
+    List<SupervisorSummary> supervisors,
+    int nimbus_uptime_secs,
+    List<TopologySummary> topologies)
+  {
+    this();
+    this.supervisors = supervisors;
+    this.nimbus_uptime_secs = nimbus_uptime_secs;
+    set_nimbus_uptime_secs_isSet(true);
+    this.topologies = topologies;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public ClusterSummary(ClusterSummary other) {
+    __isset_bit_vector.clear();
+    __isset_bit_vector.or(other.__isset_bit_vector);
+    if (other.is_set_supervisors()) {
+      List<SupervisorSummary> __this__supervisors = new ArrayList<SupervisorSummary>();
+      for (SupervisorSummary other_element : other.supervisors) {
+        __this__supervisors.add(new SupervisorSummary(other_element));
+      }
+      this.supervisors = __this__supervisors;
+    }
+    this.nimbus_uptime_secs = other.nimbus_uptime_secs;
+    if (other.is_set_topologies()) {
+      List<TopologySummary> __this__topologies = new ArrayList<TopologySummary>();
+      for (TopologySummary other_element : other.topologies) {
+        __this__topologies.add(new TopologySummary(other_element));
+      }
+      this.topologies = __this__topologies;
+    }
+  }
+
+  public ClusterSummary deepCopy() {
+    return new ClusterSummary(this);
+  }
+
+  @Deprecated
+  public ClusterSummary clone() {
+    return new ClusterSummary(this);
+  }
+
+  public int get_supervisors_size() {
+    return (this.supervisors == null) ? 0 : this.supervisors.size();
+  }
+
+  public java.util.Iterator<SupervisorSummary> get_supervisors_iterator() {
+    return (this.supervisors == null) ? null : this.supervisors.iterator();
+  }
+
+  public void add_to_supervisors(SupervisorSummary elem) {
+    if (this.supervisors == null) {
+      this.supervisors = new ArrayList<SupervisorSummary>();
+    }
+    this.supervisors.add(elem);
+  }
+
+  public List<SupervisorSummary> get_supervisors() {
+    return this.supervisors;
+  }
+
+  public void set_supervisors(List<SupervisorSummary> supervisors) {
+    this.supervisors = supervisors;
+  }
+
+  public void unset_supervisors() {
+    this.supervisors = null;
+  }
+
+  /** Returns true if field supervisors is set (has been asigned a value) and false otherwise */
+  public boolean is_set_supervisors() {
+    return this.supervisors != null;
+  }
+
+  public void set_supervisors_isSet(boolean value) {
+    if (!value) {
+      this.supervisors = null;
+    }
+  }
+
+  public int get_nimbus_uptime_secs() {
+    return this.nimbus_uptime_secs;
+  }
+
+  public void set_nimbus_uptime_secs(int nimbus_uptime_secs) {
+    this.nimbus_uptime_secs = nimbus_uptime_secs;
+    set_nimbus_uptime_secs_isSet(true);
+  }
+
+  public void unset_nimbus_uptime_secs() {
+    __isset_bit_vector.clear(__NIMBUS_UPTIME_SECS_ISSET_ID);
+  }
+
+  /** Returns true if field nimbus_uptime_secs is set (has been asigned a value) and false otherwise */
+  public boolean is_set_nimbus_uptime_secs() {
+    return __isset_bit_vector.get(__NIMBUS_UPTIME_SECS_ISSET_ID);
+  }
+
+  public void set_nimbus_uptime_secs_isSet(boolean value) {
+    __isset_bit_vector.set(__NIMBUS_UPTIME_SECS_ISSET_ID, value);
+  }
+
+  public int get_topologies_size() {
+    return (this.topologies == null) ? 0 : this.topologies.size();
+  }
+
+  public java.util.Iterator<TopologySummary> get_topologies_iterator() {
+    return (this.topologies == null) ? null : this.topologies.iterator();
+  }
+
+  public void add_to_topologies(TopologySummary elem) {
+    if (this.topologies == null) {
+      this.topologies = new ArrayList<TopologySummary>();
+    }
+    this.topologies.add(elem);
+  }
+
+  public List<TopologySummary> get_topologies() {
+    return this.topologies;
+  }
+
+  public void set_topologies(List<TopologySummary> topologies) {
+    this.topologies = topologies;
+  }
+
+  public void unset_topologies() {
+    this.topologies = null;
+  }
+
+  /** Returns true if field topologies is set (has been asigned a value) and false otherwise */
+  public boolean is_set_topologies() {
+    return this.topologies != null;
+  }
+
+  public void set_topologies_isSet(boolean value) {
+    if (!value) {
+      this.topologies = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case SUPERVISORS:
+      if (value == null) {
+        unset_supervisors();
+      } else {
+        set_supervisors((List<SupervisorSummary>)value);
+      }
+      break;
+
+    case NIMBUS_UPTIME_SECS:
+      if (value == null) {
+        unset_nimbus_uptime_secs();
+      } else {
+        set_nimbus_uptime_secs((Integer)value);
+      }
+      break;
+
+    case TOPOLOGIES:
+      if (value == null) {
+        unset_topologies();
+      } else {
+        set_topologies((List<TopologySummary>)value);
+      }
+      break;
+
+    }
+  }
+
+  public void setFieldValue(int fieldID, Object value) {
+    setFieldValue(_Fields.findByThriftIdOrThrow(fieldID), value);
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case SUPERVISORS:
+      return get_supervisors();
+
+    case NIMBUS_UPTIME_SECS:
+      return new Integer(get_nimbus_uptime_secs());
+
+    case TOPOLOGIES:
+      return get_topologies();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  public Object getFieldValue(int fieldId) {
+    return getFieldValue(_Fields.findByThriftIdOrThrow(fieldId));
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been asigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    switch (field) {
+    case SUPERVISORS:
+      return is_set_supervisors();
+    case NIMBUS_UPTIME_SECS:
+      return is_set_nimbus_uptime_secs();
+    case TOPOLOGIES:
+      return is_set_topologies();
+    }
+    throw new IllegalStateException();
+  }
+
+  public boolean isSet(int fieldID) {
+    return isSet(_Fields.findByThriftIdOrThrow(fieldID));
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof ClusterSummary)
+      return this.equals((ClusterSummary)that);
+    return false;
+  }
+
+  public boolean equals(ClusterSummary that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_supervisors = true && this.is_set_supervisors();
+    boolean that_present_supervisors = true && that.is_set_supervisors();
+    if (this_present_supervisors || that_present_supervisors) {
+      if (!(this_present_supervisors && that_present_supervisors))
+        return false;
+      if (!this.supervisors.equals(that.supervisors))
+        return false;
+    }
+
+    boolean this_present_nimbus_uptime_secs = true;
+    boolean that_present_nimbus_uptime_secs = true;
+    if (this_present_nimbus_uptime_secs || that_present_nimbus_uptime_secs) {
+      if (!(this_present_nimbus_uptime_secs && that_present_nimbus_uptime_secs))
+        return false;
+      if (this.nimbus_uptime_secs != that.nimbus_uptime_secs)
+        return false;
+    }
+
+    boolean this_present_topologies = true && this.is_set_topologies();
+    boolean that_present_topologies = true && that.is_set_topologies();
+    if (this_present_topologies || that_present_topologies) {
+      if (!(this_present_topologies && that_present_topologies))
+        return false;
+      if (!this.topologies.equals(that.topologies))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    HashCodeBuilder builder = new HashCodeBuilder();
+
+    boolean present_supervisors = true && (is_set_supervisors());
+    builder.append(present_supervisors);
+    if (present_supervisors)
+      builder.append(supervisors);
+
+    boolean present_nimbus_uptime_secs = true;
+    builder.append(present_nimbus_uptime_secs);
+    if (present_nimbus_uptime_secs)
+      builder.append(nimbus_uptime_secs);
+
+    boolean present_topologies = true && (is_set_topologies());
+    builder.append(present_topologies);
+    if (present_topologies)
+      builder.append(topologies);
+
+    return builder.toHashCode();
+  }
+
+  public int compareTo(ClusterSummary other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+    ClusterSummary typedOther = (ClusterSummary)other;
+
+    lastComparison = Boolean.valueOf(is_set_supervisors()).compareTo(typedOther.is_set_supervisors());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_supervisors()) {      lastComparison = TBaseHelper.compareTo(this.supervisors, typedOther.supervisors);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_nimbus_uptime_secs()).compareTo(typedOther.is_set_nimbus_uptime_secs());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_nimbus_uptime_secs()) {      lastComparison = TBaseHelper.compareTo(this.nimbus_uptime_secs, typedOther.nimbus_uptime_secs);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_topologies()).compareTo(typedOther.is_set_topologies());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_topologies()) {      lastComparison = TBaseHelper.compareTo(this.topologies, typedOther.topologies);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public void read(TProtocol iprot) throws TException {
+    TField field;
+    iprot.readStructBegin();
+    while (true)
+    {
+      field = iprot.readFieldBegin();
+      if (field.type == TType.STOP) { 
+        break;
+      }
+      switch (field.id) {
+        case 1: // SUPERVISORS
+          if (field.type == TType.LIST) {
+            {
+              TList _list33 = iprot.readListBegin();
+              this.supervisors = new ArrayList<SupervisorSummary>(_list33.size);
+              for (int _i34 = 0; _i34 < _list33.size; ++_i34)
+              {
+                SupervisorSummary _elem35;
+                _elem35 = new SupervisorSummary();
+                _elem35.read(iprot);
+                this.supervisors.add(_elem35);
+              }
+              iprot.readListEnd();
+            }
+          } else { 
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 2: // NIMBUS_UPTIME_SECS
+          if (field.type == TType.I32) {
+            this.nimbus_uptime_secs = iprot.readI32();
+            set_nimbus_uptime_secs_isSet(true);
+          } else { 
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 3: // TOPOLOGIES
+          if (field.type == TType.LIST) {
+            {
+              TList _list36 = iprot.readListBegin();
+              this.topologies = new ArrayList<TopologySummary>(_list36.size);
+              for (int _i37 = 0; _i37 < _list36.size; ++_i37)
+              {
+                TopologySummary _elem38;
+                _elem38 = new TopologySummary();
+                _elem38.read(iprot);
+                this.topologies.add(_elem38);
+              }
+              iprot.readListEnd();
+            }
+          } else { 
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        default:
+          TProtocolUtil.skip(iprot, field.type);
+      }
+      iprot.readFieldEnd();
+    }
+    iprot.readStructEnd();
+    validate();
+  }
+
+  public void write(TProtocol oprot) throws TException {
+    validate();
+
+    oprot.writeStructBegin(STRUCT_DESC);
+    if (this.supervisors != null) {
+      oprot.writeFieldBegin(SUPERVISORS_FIELD_DESC);
+      {
+        oprot.writeListBegin(new TList(TType.STRUCT, this.supervisors.size()));
+        for (SupervisorSummary _iter39 : this.supervisors)
+        {
+          _iter39.write(oprot);
+        }
+        oprot.writeListEnd();
+      }
+      oprot.writeFieldEnd();
+    }
+    oprot.writeFieldBegin(NIMBUS_UPTIME_SECS_FIELD_DESC);
+    oprot.writeI32(this.nimbus_uptime_secs);
+    oprot.writeFieldEnd();
+    if (this.topologies != null) {
+      oprot.writeFieldBegin(TOPOLOGIES_FIELD_DESC);
+      {
+        oprot.writeListBegin(new TList(TType.STRUCT, this.topologies.size()));
+        for (TopologySummary _iter40 : this.topologies)
+        {
+          _iter40.write(oprot);
+        }
+        oprot.writeListEnd();
+      }
+      oprot.writeFieldEnd();
+    }
+    oprot.writeFieldStop();
+    oprot.writeStructEnd();
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("ClusterSummary(");
+    boolean first = true;
+
+    sb.append("supervisors:");
+    if (this.supervisors == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.supervisors);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("nimbus_uptime_secs:");
+    sb.append(this.nimbus_uptime_secs);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("topologies:");
+    if (this.topologies == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.topologies);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws TException {
+    // check for required fields
+    if (!is_set_supervisors()) {
+      throw new TProtocolException("Required field 'supervisors' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_nimbus_uptime_secs()) {
+      throw new TProtocolException("Required field 'nimbus_uptime_secs' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_topologies()) {
+      throw new TProtocolException("Required field 'topologies' is unset! Struct:" + toString());
+    }
+
+  }
+
+}
+
diff --git a/src/jvm/backtype/storm/generated/ComponentCommon.java b/src/jvm/backtype/storm/generated/ComponentCommon.java
new file mode 100644
index 0000000..206c7c1
--- /dev/null
+++ b/src/jvm/backtype/storm/generated/ComponentCommon.java
@@ -0,0 +1,455 @@
+/**
+ * Autogenerated by Thrift
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ */
+package backtype.storm.generated;
+
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.util.Arrays;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.thrift.*;
+import org.apache.thrift.async.*;
+import org.apache.thrift.meta_data.*;
+import org.apache.thrift.transport.*;
+import org.apache.thrift.protocol.*;
+
+public class ComponentCommon implements TBase<ComponentCommon, ComponentCommon._Fields>, java.io.Serializable, Cloneable {
+  private static final TStruct STRUCT_DESC = new TStruct("ComponentCommon");
+
+  private static final TField STREAMS_FIELD_DESC = new TField("streams", TType.MAP, (short)1);
+  private static final TField PARALLELISM_HINT_FIELD_DESC = new TField("parallelism_hint", TType.I32, (short)2);
+
+  private Map<Integer,StreamInfo> streams;
+  private int parallelism_hint;
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements TFieldIdEnum {
+    STREAMS((short)1, "streams"),
+    PARALLELISM_HINT((short)2, "parallelism_hint");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // STREAMS
+          return STREAMS;
+        case 2: // PARALLELISM_HINT
+          return PARALLELISM_HINT;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __PARALLELISM_HINT_ISSET_ID = 0;
+  private BitSet __isset_bit_vector = new BitSet(1);
+
+  public static final Map<_Fields, FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, FieldMetaData> tmpMap = new EnumMap<_Fields, FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.STREAMS, new FieldMetaData("streams", TFieldRequirementType.REQUIRED, 
+        new MapMetaData(TType.MAP, 
+            new FieldValueMetaData(TType.I32), 
+            new StructMetaData(TType.STRUCT, StreamInfo.class))));
+    tmpMap.put(_Fields.PARALLELISM_HINT, new FieldMetaData("parallelism_hint", TFieldRequirementType.OPTIONAL, 
+        new FieldValueMetaData(TType.I32)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    FieldMetaData.addStructMetaDataMap(ComponentCommon.class, metaDataMap);
+  }
+
+  public ComponentCommon() {
+  }
+
+  public ComponentCommon(
+    Map<Integer,StreamInfo> streams)
+  {
+    this();
+    this.streams = streams;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public ComponentCommon(ComponentCommon other) {
+    __isset_bit_vector.clear();
+    __isset_bit_vector.or(other.__isset_bit_vector);
+    if (other.is_set_streams()) {
+      Map<Integer,StreamInfo> __this__streams = new HashMap<Integer,StreamInfo>();
+      for (Map.Entry<Integer, StreamInfo> other_element : other.streams.entrySet()) {
+
+        Integer other_element_key = other_element.getKey();
+        StreamInfo other_element_value = other_element.getValue();
+
+        Integer __this__streams_copy_key = other_element_key;
+
+        StreamInfo __this__streams_copy_value = new StreamInfo(other_element_value);
+
+        __this__streams.put(__this__streams_copy_key, __this__streams_copy_value);
+      }
+      this.streams = __this__streams;
+    }
+    this.parallelism_hint = other.parallelism_hint;
+  }
+
+  public ComponentCommon deepCopy() {
+    return new ComponentCommon(this);
+  }
+
+  @Deprecated
+  public ComponentCommon clone() {
+    return new ComponentCommon(this);
+  }
+
+  public int get_streams_size() {
+    return (this.streams == null) ? 0 : this.streams.size();
+  }
+
+  public void put_to_streams(int key, StreamInfo val) {
+    if (this.streams == null) {
+      this.streams = new HashMap<Integer,StreamInfo>();
+    }
+    this.streams.put(key, val);
+  }
+
+  public Map<Integer,StreamInfo> get_streams() {
+    return this.streams;
+  }
+
+  public void set_streams(Map<Integer,StreamInfo> streams) {
+    this.streams = streams;
+  }
+
+  public void unset_streams() {
+    this.streams = null;
+  }
+
+  /** Returns true if field streams is set (has been asigned a value) and false otherwise */
+  public boolean is_set_streams() {
+    return this.streams != null;
+  }
+
+  public void set_streams_isSet(boolean value) {
+    if (!value) {
+      this.streams = null;
+    }
+  }
+
+  public int get_parallelism_hint() {
+    return this.parallelism_hint;
+  }
+
+  public void set_parallelism_hint(int parallelism_hint) {
+    this.parallelism_hint = parallelism_hint;
+    set_parallelism_hint_isSet(true);
+  }
+
+  public void unset_parallelism_hint() {
+    __isset_bit_vector.clear(__PARALLELISM_HINT_ISSET_ID);
+  }
+
+  /** Returns true if field parallelism_hint is set (has been asigned a value) and false otherwise */
+  public boolean is_set_parallelism_hint() {
+    return __isset_bit_vector.get(__PARALLELISM_HINT_ISSET_ID);
+  }
+
+  public void set_parallelism_hint_isSet(boolean value) {
+    __isset_bit_vector.set(__PARALLELISM_HINT_ISSET_ID, value);
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case STREAMS:
+      if (value == null) {
+        unset_streams();
+      } else {
+        set_streams((Map<Integer,StreamInfo>)value);
+      }
+      break;
+
+    case PARALLELISM_HINT:
+      if (value == null) {
+        unset_parallelism_hint();
+      } else {
+        set_parallelism_hint((Integer)value);
+      }
+      break;
+
+    }
+  }
+
+  public void setFieldValue(int fieldID, Object value) {
+    setFieldValue(_Fields.findByThriftIdOrThrow(fieldID), value);
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case STREAMS:
+      return get_streams();
+
+    case PARALLELISM_HINT:
+      return new Integer(get_parallelism_hint());
+
+    }
+    throw new IllegalStateException();
+  }
+
+  public Object getFieldValue(int fieldId) {
+    return getFieldValue(_Fields.findByThriftIdOrThrow(fieldId));
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been asigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    switch (field) {
+    case STREAMS:
+      return is_set_streams();
+    case PARALLELISM_HINT:
+      return is_set_parallelism_hint();
+    }
+    throw new IllegalStateException();
+  }
+
+  public boolean isSet(int fieldID) {
+    return isSet(_Fields.findByThriftIdOrThrow(fieldID));
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof ComponentCommon)
+      return this.equals((ComponentCommon)that);
+    return false;
+  }
+
+  public boolean equals(ComponentCommon that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_streams = true && this.is_set_streams();
+    boolean that_present_streams = true && that.is_set_streams();
+    if (this_present_streams || that_present_streams) {
+      if (!(this_present_streams && that_present_streams))
+        return false;
+      if (!this.streams.equals(that.streams))
+        return false;
+    }
+
+    boolean this_present_parallelism_hint = true && this.is_set_parallelism_hint();
+    boolean that_present_parallelism_hint = true && that.is_set_parallelism_hint();
+    if (this_present_parallelism_hint || that_present_parallelism_hint) {
+      if (!(this_present_parallelism_hint && that_present_parallelism_hint))
+        return false;
+      if (this.parallelism_hint != that.parallelism_hint)
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    HashCodeBuilder builder = new HashCodeBuilder();
+
+    boolean present_streams = true && (is_set_streams());
+    builder.append(present_streams);
+    if (present_streams)
+      builder.append(streams);
+
+    boolean present_parallelism_hint = true && (is_set_parallelism_hint());
+    builder.append(present_parallelism_hint);
+    if (present_parallelism_hint)
+      builder.append(parallelism_hint);
+
+    return builder.toHashCode();
+  }
+
+  public int compareTo(ComponentCommon other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+    ComponentCommon typedOther = (ComponentCommon)other;
+
+    lastComparison = Boolean.valueOf(is_set_streams()).compareTo(typedOther.is_set_streams());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_streams()) {      lastComparison = TBaseHelper.compareTo(this.streams, typedOther.streams);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_parallelism_hint()).compareTo(typedOther.is_set_parallelism_hint());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_parallelism_hint()) {      lastComparison = TBaseHelper.compareTo(this.parallelism_hint, typedOther.parallelism_hint);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public void read(TProtocol iprot) throws TException {
+    TField field;
+    iprot.readStructBegin();
+    while (true)
+    {
+      field = iprot.readFieldBegin();
+      if (field.type == TType.STOP) { 
+        break;
+      }
+      switch (field.id) {
+        case 1: // STREAMS
+          if (field.type == TType.MAP) {
+            {
+              TMap _map8 = iprot.readMapBegin();
+              this.streams = new HashMap<Integer,StreamInfo>(2*_map8.size);
+              for (int _i9 = 0; _i9 < _map8.size; ++_i9)
+              {
+                int _key10;
+                StreamInfo _val11;
+                _key10 = iprot.readI32();
+                _val11 = new StreamInfo();
+                _val11.read(iprot);
+                this.streams.put(_key10, _val11);
+              }
+              iprot.readMapEnd();
+            }
+          } else { 
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 2: // PARALLELISM_HINT
+          if (field.type == TType.I32) {
+            this.parallelism_hint = iprot.readI32();
+            set_parallelism_hint_isSet(true);
+          } else { 
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        default:
+          TProtocolUtil.skip(iprot, field.type);
+      }
+      iprot.readFieldEnd();
+    }
+    iprot.readStructEnd();
+    validate();
+  }
+
+  public void write(TProtocol oprot) throws TException {
+    validate();
+
+    oprot.writeStructBegin(STRUCT_DESC);
+    if (this.streams != null) {
+      oprot.writeFieldBegin(STREAMS_FIELD_DESC);
+      {
+        oprot.writeMapBegin(new TMap(TType.I32, TType.STRUCT, this.streams.size()));
+        for (Map.Entry<Integer, StreamInfo> _iter12 : this.streams.entrySet())
+        {
+          oprot.writeI32(_iter12.getKey());
+          _iter12.getValue().write(oprot);
+        }
+        oprot.writeMapEnd();
+      }
+      oprot.writeFieldEnd();
+    }
+    if (is_set_parallelism_hint()) {
+      oprot.writeFieldBegin(PARALLELISM_HINT_FIELD_DESC);
+      oprot.writeI32(this.parallelism_hint);
+      oprot.writeFieldEnd();
+    }
+    oprot.writeFieldStop();
+    oprot.writeStructEnd();
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("ComponentCommon(");
+    boolean first = true;
+
+    sb.append("streams:");
+    if (this.streams == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.streams);
+    }
+    first = false;
+    if (is_set_parallelism_hint()) {
+      if (!first) sb.append(", ");
+      sb.append("parallelism_hint:");
+      sb.append(this.parallelism_hint);
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws TException {
+    // check for required fields
+    if (!is_set_streams()) {
+      throw new TProtocolException("Required field 'streams' is unset! Struct:" + toString());
+    }
+
+  }
+
+}
+
diff --git a/src/jvm/backtype/storm/generated/ComponentObject.java b/src/jvm/backtype/storm/generated/ComponentObject.java
new file mode 100644
index 0000000..0852f20
--- /dev/null
+++ b/src/jvm/backtype/storm/generated/ComponentObject.java
@@ -0,0 +1,301 @@
+/**
+ * Autogenerated by Thrift
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ */
+package backtype.storm.generated;
+
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.util.Arrays;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.thrift.*;
+import org.apache.thrift.async.*;
+import org.apache.thrift.meta_data.*;
+import org.apache.thrift.transport.*;
+import org.apache.thrift.protocol.*;
+
+public class ComponentObject extends TUnion<ComponentObject, ComponentObject._Fields> {
+  private static final TStruct STRUCT_DESC = new TStruct("ComponentObject");
+  private static final TField SERIALIZED_JAVA_FIELD_DESC = new TField("serialized_java", TType.STRING, (short)1);
+  private static final TField SHELL_FIELD_DESC = new TField("shell", TType.STRUCT, (short)2);
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements TFieldIdEnum {
+    SERIALIZED_JAVA((short)1, "serialized_java"),
+    SHELL((short)2, "shell");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // SERIALIZED_JAVA
+          return SERIALIZED_JAVA;
+        case 2: // SHELL
+          return SHELL;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  public static final Map<_Fields, FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, FieldMetaData> tmpMap = new EnumMap<_Fields, FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.SERIALIZED_JAVA, new FieldMetaData("serialized_java", TFieldRequirementType.DEFAULT, 
+        new FieldValueMetaData(TType.STRING)));
+    tmpMap.put(_Fields.SHELL, new FieldMetaData("shell", TFieldRequirementType.DEFAULT, 
+        new StructMetaData(TType.STRUCT, ShellComponent.class)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    FieldMetaData.addStructMetaDataMap(ComponentObject.class, metaDataMap);
+  }
+
+  public ComponentObject() {
+    super();
+  }
+
+  public ComponentObject(_Fields setField, Object value) {
+    super(setField, value);
+  }
+
+  public ComponentObject(ComponentObject other) {
+    super(other);
+  }
+  public ComponentObject deepCopy() {
+    return new ComponentObject(this);
+  }
+
+  public static ComponentObject serialized_java(byte[] value) {
+    ComponentObject x = new ComponentObject();
+    x.set_serialized_java(value);
+    return x;
+  }
+
+  public static ComponentObject shell(ShellComponent value) {
+    ComponentObject x = new ComponentObject();
+    x.set_shell(value);
+    return x;
+  }
+
+
+  @Override
+  protected void checkType(_Fields setField, Object value) throws ClassCastException {
+    switch (setField) {
+      case SERIALIZED_JAVA:
+        if (value instanceof byte[]) {
+          break;
+        }
+        throw new ClassCastException("Was expecting value of type byte[] for field 'serialized_java', but got " + value.getClass().getSimpleName());
+      case SHELL:
+        if (value instanceof ShellComponent) {
+          break;
+        }
+        throw new ClassCastException("Was expecting value of type ShellComponent for field 'shell', but got " + value.getClass().getSimpleName());
+      default:
+        throw new IllegalArgumentException("Unknown field id " + setField);
+    }
+  }
+
+  @Override
+  protected Object readValue(TProtocol iprot, TField field) throws TException {
+    _Fields setField = _Fields.findByThriftId(field.id);
+    if (setField != null) {
+      switch (setField) {
+        case SERIALIZED_JAVA:
+          if (field.type == SERIALIZED_JAVA_FIELD_DESC.type) {
+            byte[] serialized_java;
+            serialized_java = iprot.readBinary();
+            return serialized_java;
+          } else {
+            TProtocolUtil.skip(iprot, field.type);
+            return null;
+          }
+        case SHELL:
+          if (field.type == SHELL_FIELD_DESC.type) {
+            ShellComponent shell;
+            shell = new ShellComponent();
+            shell.read(iprot);
+            return shell;
+          } else {
+            TProtocolUtil.skip(iprot, field.type);
+            return null;
+          }
+        default:
+          throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!");
+      }
+    } else {
+      TProtocolUtil.skip(iprot, field.type);
+      return null;
+    }
+  }
+
+  @Override
+  protected void writeValue(TProtocol oprot) throws TException {
+    switch (setField_) {
+      case SERIALIZED_JAVA:
+        byte[] serialized_java = (byte[])value_;
+        oprot.writeBinary(serialized_java);
+        return;
+      case SHELL:
+        ShellComponent shell = (ShellComponent)value_;
+        shell.write(oprot);
+        return;
+      default:
+        throw new IllegalStateException("Cannot write union with unknown field " + setField_);
+    }
+  }
+
+  @Override
+  protected TField getFieldDesc(_Fields setField) {
+    switch (setField) {
+      case SERIALIZED_JAVA:
+        return SERIALIZED_JAVA_FIELD_DESC;
+      case SHELL:
+        return SHELL_FIELD_DESC;
+      default:
+        throw new IllegalArgumentException("Unknown field id " + setField);
+    }
+  }
+
+  @Override
+  protected TStruct getStructDesc() {
+    return STRUCT_DESC;
+  }
+
+  @Override
+  protected _Fields enumForId(short id) {
+    return _Fields.findByThriftIdOrThrow(id);
+  }
+
+  public byte[] get_serialized_java() {
+    if (getSetField() == _Fields.SERIALIZED_JAVA) {
+      return (byte[])getFieldValue();
+    } else {
+      throw new RuntimeException("Cannot get field 'serialized_java' because union is currently set to " + getFieldDesc(getSetField()).name);
+    }
+  }
+
+  public void set_serialized_java(byte[] value) {
+    if (value == null) throw new NullPointerException();
+    setField_ = _Fields.SERIALIZED_JAVA;
+    value_ = value;
+  }
+
+  public ShellComponent get_shell() {
+    if (getSetField() == _Fields.SHELL) {
+      return (ShellComponent)getFieldValue();
+    } else {
+      throw new RuntimeException("Cannot get field 'shell' because union is currently set to " + getFieldDesc(getSetField()).name);
+    }
+  }
+
+  public void set_shell(ShellComponent value) {
+    if (value == null) throw new NullPointerException();
+    setField_ = _Fields.SHELL;
+    value_ = value;
+  }
+
+  public boolean equals(Object other) {
+    if (other instanceof ComponentObject) {
+      return equals((ComponentObject)other);
+    } else {
+      return false;
+    }
+  }
+
+  public boolean equals(ComponentObject other) {
+    return other != null && getSetField() == other.getSetField() && ((value_ instanceof byte[]) ? 
+      Arrays.equals((byte[])getFieldValue(), (byte[])other.getFieldValue()) : getFieldValue().equals(other.getFieldValue()));
+  }
+
+  @Override
+  public int compareTo(ComponentObject other) {
+    int lastComparison = TBaseHelper.compareTo(getSetField(), other.getSetField());
+    if (lastComparison == 0) {
+      Object myValue = getFieldValue();
+      if (myValue instanceof byte[]) {
+        return TBaseHelper.compareTo((byte[])myValue, (byte[])other.getFieldValue());
+      } else if (myValue instanceof List) {
+        return TBaseHelper.compareTo((List)myValue, (List)other.getFieldValue());
+      } else if (myValue instanceof Set) {
+        return TBaseHelper.compareTo((Set)myValue, (Set)other.getFieldValue());
+      } else if (myValue instanceof Map){
+        return TBaseHelper.compareTo((Map)myValue, (Map)other.getFieldValue());
+      } else {
+        return TBaseHelper.compareTo((Comparable)myValue, (Comparable)other.getFieldValue());
+      }
+    }
+    return lastComparison;
+  }
+
+
+  @Override
+  public int hashCode() {
+    HashCodeBuilder hcb = new HashCodeBuilder();
+    hcb.append(this.getClass().getName());
+    TFieldIdEnum setField = getSetField();
+    if (setField != null) {
+      hcb.append(setField.getThriftFieldId());
+      Object value = getFieldValue();
+      if (value instanceof TEnum) {
+        hcb.append(((TEnum)getFieldValue()).getValue());
+      } else {
+        hcb.append(value);
+      }
+    }
+    return hcb.toHashCode();
+  }
+}
diff --git a/src/jvm/backtype/storm/generated/DistributedRPC.java b/src/jvm/backtype/storm/generated/DistributedRPC.java
new file mode 100644
index 0000000..60a3eea
--- /dev/null
+++ b/src/jvm/backtype/storm/generated/DistributedRPC.java
@@ -0,0 +1,1566 @@
+/**
+ * Autogenerated by Thrift
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ */
+package backtype.storm.generated;
+
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.util.Arrays;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.thrift.*;
+import org.apache.thrift.async.*;
+import org.apache.thrift.meta_data.*;
+import org.apache.thrift.transport.*;
+import org.apache.thrift.protocol.*;
+
+public class DistributedRPC {
+
+  public interface Iface {
+
+    public String execute(String functionName, String funcArgs) throws TException;
+
+    public void result(String id, String result) throws TException;
+
+  }
+
+  public interface AsyncIface {
+
+    public void execute(String functionName, String funcArgs, AsyncMethodCallback<AsyncClient.execute_call> resultHandler) throws TException;
+
+    public void result(String id, String result, AsyncMethodCallback<AsyncClient.result_call> resultHandler) throws TException;
+
+  }
+
+  public static class Client implements TServiceClient, Iface {
+    public static class Factory implements TServiceClientFactory<Client> {
+      public Factory() {}
+      public Client getClient(TProtocol prot) {
+        return new Client(prot);
+      }
+      public Client getClient(TProtocol iprot, TProtocol oprot) {
+        return new Client(iprot, oprot);
+      }
+    }
+
+    public Client(TProtocol prot)
+    {
+      this(prot, prot);
+    }
+
+    public Client(TProtocol iprot, TProtocol oprot)
+    {
+      iprot_ = iprot;
+      oprot_ = oprot;
+    }
+
+    protected TProtocol iprot_;
+    protected TProtocol oprot_;
+
+    protected int seqid_;
+
+    public TProtocol getInputProtocol()
+    {
+      return this.iprot_;
+    }
+
+    public TProtocol getOutputProtocol()
+    {
+      return this.oprot_;
+    }
+
+    public String execute(String functionName, String funcArgs) throws TException
+    {
+      send_execute(functionName, funcArgs);
+      return recv_execute();
+    }
+
+    public void send_execute(String functionName, String funcArgs) throws TException
+    {
+      oprot_.writeMessageBegin(new TMessage("execute", TMessageType.CALL, ++seqid_));
+      execute_args args = new execute_args();
+      args.set_functionName(functionName);
+      args.set_funcArgs(funcArgs);
+      args.write(oprot_);
+      oprot_.writeMessageEnd();
+      oprot_.getTransport().flush();
+    }
+
+    public String recv_execute() throws TException
+    {
+      TMessage msg = iprot_.readMessageBegin();
+      if (msg.type == TMessageType.EXCEPTION) {
+        TApplicationException x = TApplicationException.read(iprot_);
+        iprot_.readMessageEnd();
+        throw x;
+      }
+      if (msg.seqid != seqid_) {
+        throw new TApplicationException(TApplicationException.BAD_SEQUENCE_ID, "execute failed: out of sequence response");
+      }
+      execute_result result = new execute_result();
+      result.read(iprot_);
+      iprot_.readMessageEnd();
+      if (result.is_set_success()) {
+        return result.success;
+      }
+      throw new TApplicationException(TApplicationException.MISSING_RESULT, "execute failed: unknown result");
+    }
+
+    public void result(String id, String result) throws TException
+    {
+      send_result(id, result);
+      recv_result();
+    }
+
+    public void send_result(String id, String result) throws TException
+    {
+      oprot_.writeMessageBegin(new TMessage("result", TMessageType.CALL, ++seqid_));
+      result_args args = new result_args();
+      args.set_id(id);
+      args.set_result(result);
+      args.write(oprot_);
+      oprot_.writeMessageEnd();
+      oprot_.getTransport().flush();
+    }
+
+    public void recv_result() throws TException
+    {
+      TMessage msg = iprot_.readMessageBegin();
+      if (msg.type == TMessageType.EXCEPTION) {
+        TApplicationException x = TApplicationException.read(iprot_);
+        iprot_.readMessageEnd();
+        throw x;
+      }
+      if (msg.seqid != seqid_) {
+        throw new TApplicationException(TApplicationException.BAD_SEQUENCE_ID, "result failed: out of sequence response");
+      }
+      result_result result = new result_result();
+      result.read(iprot_);
+      iprot_.readMessageEnd();
+      return;
+    }
+
+  }
+  public static class AsyncClient extends TAsyncClient implements AsyncIface {
+    public static class Factory implements TAsyncClientFactory<AsyncClient> {
+      private TAsyncClientManager clientManager;
+      private TProtocolFactory protocolFactory;
+      public Factory(TAsyncClientManager clientManager, TProtocolFactory protocolFactory) {
+        this.clientManager = clientManager;
+        this.protocolFactory = protocolFactory;
+      }
+      public AsyncClient getAsyncClient(TNonblockingTransport transport) {
+        return new AsyncClient(protocolFactory, clientManager, transport);
+      }
+    }
+
+    public AsyncClient(TProtocolFactory protocolFactory, TAsyncClientManager clientManager, TNonblockingTransport transport) {
+      super(protocolFactory, clientManager, transport);
+    }
+
+    public void execute(String functionName, String funcArgs, AsyncMethodCallback<execute_call> resultHandler) throws TException {
+      checkReady();
+      execute_call method_call = new execute_call(functionName, funcArgs, resultHandler, this, protocolFactory, transport);
+      manager.call(method_call);
+    }
+
+    public static class execute_call extends TAsyncMethodCall {
+      private String functionName;
+      private String funcArgs;
+      public execute_call(String functionName, String funcArgs, AsyncMethodCallback<execute_call> resultHandler, TAsyncClient client, TProtocolFactory protocolFactory, TNonblockingTransport transport) throws TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.functionName = functionName;
+        this.funcArgs = funcArgs;
+      }
+
+      public void write_args(TProtocol prot) throws TException {
+        prot.writeMessageBegin(new TMessage("execute", TMessageType.CALL, 0));
+        execute_args args = new execute_args();
+        args.set_functionName(functionName);
+        args.set_funcArgs(funcArgs);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public String getResult() throws TException {
+        if (getState() != State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        TMemoryInputTransport memoryTransport = new TMemoryInputTransport(getFrameBuffer().array());
+        TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_execute();
+      }
+    }
+
+    public void result(String id, String result, AsyncMethodCallback<result_call> resultHandler) throws TException {
+      checkReady();
+      result_call method_call = new result_call(id, result, resultHandler, this, protocolFactory, transport);
+      manager.call(method_call);
+    }
+
+    public static class result_call extends TAsyncMethodCall {
+      private String id;
+      private String result;
+      public result_call(String id, String result, AsyncMethodCallback<result_call> resultHandler, TAsyncClient client, TProtocolFactory protocolFactory, TNonblockingTransport transport) throws TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.id = id;
+        this.result = result;
+      }
+
+      public void write_args(TProtocol prot) throws TException {
+        prot.writeMessageBegin(new TMessage("result", TMessageType.CALL, 0));
+        result_args args = new result_args();
+        args.set_id(id);
+        args.set_result(result);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public void getResult() throws TException {
+        if (getState() != State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        TMemoryInputTransport memoryTransport = new TMemoryInputTransport(getFrameBuffer().array());
+        TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        (new Client(prot)).recv_result();
+      }
+    }
+
+  }
+
+  public static class Processor implements TProcessor {
+    private static final Logger LOGGER = LoggerFactory.getLogger(Processor.class.getName());
+    public Processor(Iface iface)
+    {
+      iface_ = iface;
+      processMap_.put("execute", new execute());
+      processMap_.put("result", new result());
+    }
+
+    protected static interface ProcessFunction {
+      public void process(int seqid, TProtocol iprot, TProtocol oprot) throws TException;
+    }
+
+    private Iface iface_;
+    protected final HashMap<String,ProcessFunction> processMap_ = new HashMap<String,ProcessFunction>();
+
+    public boolean process(TProtocol iprot, TProtocol oprot) throws TException
+    {
+      TMessage msg = iprot.readMessageBegin();
+      ProcessFunction fn = processMap_.get(msg.name);
+      if (fn == null) {
+        TProtocolUtil.skip(iprot, TType.STRUCT);
+        iprot.readMessageEnd();
+        TApplicationException x = new TApplicationException(TApplicationException.UNKNOWN_METHOD, "Invalid method name: '"+msg.name+"'");
+        oprot.writeMessageBegin(new TMessage(msg.name, TMessageType.EXCEPTION, msg.seqid));
+        x.write(oprot);
+        oprot.writeMessageEnd();
+        oprot.getTransport().flush();
+        return true;
+      }
+      fn.process(msg.seqid, iprot, oprot);
+      return true;
+    }
+
+    private class execute implements ProcessFunction {
+      public void process(int seqid, TProtocol iprot, TProtocol oprot) throws TException
+      {
+        execute_args args = new execute_args();
+        try {
+          args.read(iprot);
+        } catch (TProtocolException e) {
+          iprot.readMessageEnd();
+          TApplicationException x = new TApplicationException(TApplicationException.PROTOCOL_ERROR, e.getMessage());
+          oprot.writeMessageBegin(new TMessage("execute", TMessageType.EXCEPTION, seqid));
+          x.write(oprot);
+          oprot.writeMessageEnd();
+          oprot.getTransport().flush();
+          return;
+        }
+        iprot.readMessageEnd();
+        execute_result result = new execute_result();
+        result.success = iface_.execute(args.functionName, args.funcArgs);
+        oprot.writeMessageBegin(new TMessage("execute", TMessageType.REPLY, seqid));
+        result.write(oprot);
+        oprot.writeMessageEnd();
+        oprot.getTransport().flush();
+      }
+
+    }
+
+    private class result implements ProcessFunction {
+      public void process(int seqid, TProtocol iprot, TProtocol oprot) throws TException
+      {
+        result_args args = new result_args();
+        try {
+          args.read(iprot);
+        } catch (TProtocolException e) {
+          iprot.readMessageEnd();
+          TApplicationException x = new TApplicationException(TApplicationException.PROTOCOL_ERROR, e.getMessage());
+          oprot.writeMessageBegin(new TMessage("result", TMessageType.EXCEPTION, seqid));
+          x.write(oprot);
+          oprot.writeMessageEnd();
+          oprot.getTransport().flush();
+          return;
+        }
+        iprot.readMessageEnd();
+        result_result result = new result_result();
+        iface_.result(args.id, args.result);
+        oprot.writeMessageBegin(new TMessage("result", TMessageType.REPLY, seqid));
+        result.write(oprot);
+        oprot.writeMessageEnd();
+        oprot.getTransport().flush();
+      }
+
+    }
+
+  }
+
+  public static class execute_args implements TBase<execute_args, execute_args._Fields>, java.io.Serializable, Cloneable   {
+    private static final TStruct STRUCT_DESC = new TStruct("execute_args");
+
+    private static final TField FUNCTION_NAME_FIELD_DESC = new TField("functionName", TType.STRING, (short)1);
+    private static final TField FUNC_ARGS_FIELD_DESC = new TField("funcArgs", TType.STRING, (short)2);
+
+    private String functionName;
+    private String funcArgs;
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements TFieldIdEnum {
+      FUNCTION_NAME((short)1, "functionName"),
+      FUNC_ARGS((short)2, "funcArgs");
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 1: // FUNCTION_NAME
+            return FUNCTION_NAME;
+          case 2: // FUNC_ARGS
+            return FUNC_ARGS;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+
+    public static final Map<_Fields, FieldMetaData> metaDataMap;
+    static {
+      Map<_Fields, FieldMetaData> tmpMap = new EnumMap<_Fields, FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.FUNCTION_NAME, new FieldMetaData("functionName", TFieldRequirementType.DEFAULT, 
+          new FieldValueMetaData(TType.STRING)));
+      tmpMap.put(_Fields.FUNC_ARGS, new FieldMetaData("funcArgs", TFieldRequirementType.DEFAULT, 
+          new FieldValueMetaData(TType.STRING)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      FieldMetaData.addStructMetaDataMap(execute_args.class, metaDataMap);
+    }
+
+    public execute_args() {
+    }
+
+    public execute_args(
+      String functionName,
+      String funcArgs)
+    {
+      this();
+      this.functionName = functionName;
+      this.funcArgs = funcArgs;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public execute_args(execute_args other) {
+      if (other.is_set_functionName()) {
+        this.functionName = other.functionName;
+      }
+      if (other.is_set_funcArgs()) {
+        this.funcArgs = other.funcArgs;
+      }
+    }
+
+    public execute_args deepCopy() {
+      return new execute_args(this);
+    }
+
+    @Deprecated
+    public execute_args clone() {
+      return new execute_args(this);
+    }
+
+    public String get_functionName() {
+      return this.functionName;
+    }
+
+    public void set_functionName(String functionName) {
+      this.functionName = functionName;
+    }
+
+    public void unset_functionName() {
+      this.functionName = null;
+    }
+
+    /** Returns true if field functionName is set (has been asigned a value) and false otherwise */
+    public boolean is_set_functionName() {
+      return this.functionName != null;
+    }
+
+    public void set_functionName_isSet(boolean value) {
+      if (!value) {
+        this.functionName = null;
+      }
+    }
+
+    public String get_funcArgs() {
+      return this.funcArgs;
+    }
+
+    public void set_funcArgs(String funcArgs) {
+      this.funcArgs = funcArgs;
+    }
+
+    public void unset_funcArgs() {
+      this.funcArgs = null;
+    }
+
+    /** Returns true if field funcArgs is set (has been asigned a value) and false otherwise */
+    public boolean is_set_funcArgs() {
+      return this.funcArgs != null;
+    }
+
+    public void set_funcArgs_isSet(boolean value) {
+      if (!value) {
+        this.funcArgs = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case FUNCTION_NAME:
+        if (value == null) {
+          unset_functionName();
+        } else {
+          set_functionName((String)value);
+        }
+        break;
+
+      case FUNC_ARGS:
+        if (value == null) {
+          unset_funcArgs();
+        } else {
+          set_funcArgs((String)value);
+        }
+        break;
+
+      }
+    }
+
+    public void setFieldValue(int fieldID, Object value) {
+      setFieldValue(_Fields.findByThriftIdOrThrow(fieldID), value);
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case FUNCTION_NAME:
+        return get_functionName();
+
+      case FUNC_ARGS:
+        return get_funcArgs();
+
+      }
+      throw new IllegalStateException();
+    }
+
+    public Object getFieldValue(int fieldId) {
+      return getFieldValue(_Fields.findByThriftIdOrThrow(fieldId));
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been asigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      switch (field) {
+      case FUNCTION_NAME:
+        return is_set_functionName();
+      case FUNC_ARGS:
+        return is_set_funcArgs();
+      }
+      throw new IllegalStateException();
+    }
+
+    public boolean isSet(int fieldID) {
+      return isSet(_Fields.findByThriftIdOrThrow(fieldID));
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof execute_args)
+        return this.equals((execute_args)that);
+      return false;
+    }
+
+    public boolean equals(execute_args that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_functionName = true && this.is_set_functionName();
+      boolean that_present_functionName = true && that.is_set_functionName();
+      if (this_present_functionName || that_present_functionName) {
+        if (!(this_present_functionName && that_present_functionName))
+          return false;
+        if (!this.functionName.equals(that.functionName))
+          return false;
+      }
+
+      boolean this_present_funcArgs = true && this.is_set_funcArgs();
+      boolean that_present_funcArgs = true && that.is_set_funcArgs();
+      if (this_present_funcArgs || that_present_funcArgs) {
+        if (!(this_present_funcArgs && that_present_funcArgs))
+          return false;
+        if (!this.funcArgs.equals(that.funcArgs))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      HashCodeBuilder builder = new HashCodeBuilder();
+
+      boolean present_functionName = true && (is_set_functionName());
+      builder.append(present_functionName);
+      if (present_functionName)
+        builder.append(functionName);
+
+      boolean present_funcArgs = true && (is_set_funcArgs());
+      builder.append(present_funcArgs);
+      if (present_funcArgs)
+        builder.append(funcArgs);
+
+      return builder.toHashCode();
+    }
+
+    public int compareTo(execute_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+      execute_args typedOther = (execute_args)other;
+
+      lastComparison = Boolean.valueOf(is_set_functionName()).compareTo(typedOther.is_set_functionName());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_functionName()) {        lastComparison = TBaseHelper.compareTo(this.functionName, typedOther.functionName);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(is_set_funcArgs()).compareTo(typedOther.is_set_funcArgs());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_funcArgs()) {        lastComparison = TBaseHelper.compareTo(this.funcArgs, typedOther.funcArgs);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    public void read(TProtocol iprot) throws TException {
+      TField field;
+      iprot.readStructBegin();
+      while (true)
+      {
+        field = iprot.readFieldBegin();
+        if (field.type == TType.STOP) { 
+          break;
+        }
+        switch (field.id) {
+          case 1: // FUNCTION_NAME
+            if (field.type == TType.STRING) {
+              this.functionName = iprot.readString();
+            } else { 
+              TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
+          case 2: // FUNC_ARGS
+            if (field.type == TType.STRING) {
+              this.funcArgs = iprot.readString();
+            } else { 
+              TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
+          default:
+            TProtocolUtil.skip(iprot, field.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      validate();
+    }
+
+    public void write(TProtocol oprot) throws TException {
+      validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (this.functionName != null) {
+        oprot.writeFieldBegin(FUNCTION_NAME_FIELD_DESC);
+        oprot.writeString(this.functionName);
+        oprot.writeFieldEnd();
+      }
+      if (this.funcArgs != null) {
+        oprot.writeFieldBegin(FUNC_ARGS_FIELD_DESC);
+        oprot.writeString(this.funcArgs);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("execute_args(");
+      boolean first = true;
+
+      sb.append("functionName:");
+      if (this.functionName == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.functionName);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("funcArgs:");
+      if (this.funcArgs == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.funcArgs);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws TException {
+      // check for required fields
+    }
+
+  }
+
+  public static class execute_result implements TBase<execute_result, execute_result._Fields>, java.io.Serializable, Cloneable   {
+    private static final TStruct STRUCT_DESC = new TStruct("execute_result");
+
+    private static final TField SUCCESS_FIELD_DESC = new TField("success", TType.STRING, (short)0);
+
+    private String success;
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements TFieldIdEnum {
+      SUCCESS((short)0, "success");
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 0: // SUCCESS
+            return SUCCESS;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+
+    public static final Map<_Fields, FieldMetaData> metaDataMap;
+    static {
+      Map<_Fields, FieldMetaData> tmpMap = new EnumMap<_Fields, FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.SUCCESS, new FieldMetaData("success", TFieldRequirementType.DEFAULT, 
+          new FieldValueMetaData(TType.STRING)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      FieldMetaData.addStructMetaDataMap(execute_result.class, metaDataMap);
+    }
+
+    public execute_result() {
+    }
+
+    public execute_result(
+      String success)
+    {
+      this();
+      this.success = success;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public execute_result(execute_result other) {
+      if (other.is_set_success()) {
+        this.success = other.success;
+      }
+    }
+
+    public execute_result deepCopy() {
+      return new execute_result(this);
+    }
+
+    @Deprecated
+    public execute_result clone() {
+      return new execute_result(this);
+    }
+
+    public String get_success() {
+      return this.success;
+    }
+
+    public void set_success(String success) {
+      this.success = success;
+    }
+
+    public void unset_success() {
+      this.success = null;
+    }
+
+    /** Returns true if field success is set (has been asigned a value) and false otherwise */
+    public boolean is_set_success() {
+      return this.success != null;
+    }
+
+    public void set_success_isSet(boolean value) {
+      if (!value) {
+        this.success = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case SUCCESS:
+        if (value == null) {
+          unset_success();
+        } else {
+          set_success((String)value);
+        }
+        break;
+
+      }
+    }
+
+    public void setFieldValue(int fieldID, Object value) {
+      setFieldValue(_Fields.findByThriftIdOrThrow(fieldID), value);
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case SUCCESS:
+        return get_success();
+
+      }
+      throw new IllegalStateException();
+    }
+
+    public Object getFieldValue(int fieldId) {
+      return getFieldValue(_Fields.findByThriftIdOrThrow(fieldId));
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been asigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      switch (field) {
+      case SUCCESS:
+        return is_set_success();
+      }
+      throw new IllegalStateException();
+    }
+
+    public boolean isSet(int fieldID) {
+      return isSet(_Fields.findByThriftIdOrThrow(fieldID));
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof execute_result)
+        return this.equals((execute_result)that);
+      return false;
+    }
+
+    public boolean equals(execute_result that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_success = true && this.is_set_success();
+      boolean that_present_success = true && that.is_set_success();
+      if (this_present_success || that_present_success) {
+        if (!(this_present_success && that_present_success))
+          return false;
+        if (!this.success.equals(that.success))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      HashCodeBuilder builder = new HashCodeBuilder();
+
+      boolean present_success = true && (is_set_success());
+      builder.append(present_success);
+      if (present_success)
+        builder.append(success);
+
+      return builder.toHashCode();
+    }
+
+    public int compareTo(execute_result other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+      execute_result typedOther = (execute_result)other;
+
+      lastComparison = Boolean.valueOf(is_set_success()).compareTo(typedOther.is_set_success());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_success()) {        lastComparison = TBaseHelper.compareTo(this.success, typedOther.success);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    public void read(TProtocol iprot) throws TException {
+      TField field;
+      iprot.readStructBegin();
+      while (true)
+      {
+        field = iprot.readFieldBegin();
+        if (field.type == TType.STOP) { 
+          break;
+        }
+        switch (field.id) {
+          case 0: // SUCCESS
+            if (field.type == TType.STRING) {
+              this.success = iprot.readString();
+            } else { 
+              TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
+          default:
+            TProtocolUtil.skip(iprot, field.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      validate();
+    }
+
+    public void write(TProtocol oprot) throws TException {
+      oprot.writeStructBegin(STRUCT_DESC);
+
+      if (this.is_set_success()) {
+        oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
+        oprot.writeString(this.success);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("execute_result(");
+      boolean first = true;
+
+      sb.append("success:");
+      if (this.success == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.success);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws TException {
+      // check for required fields
+    }
+
+  }
+
+  public static class result_args implements TBase<result_args, result_args._Fields>, java.io.Serializable, Cloneable   {
+    private static final TStruct STRUCT_DESC = new TStruct("result_args");
+
+    private static final TField ID_FIELD_DESC = new TField("id", TType.STRING, (short)1);
+    private static final TField RESULT_FIELD_DESC = new TField("result", TType.STRING, (short)2);
+
+    private String id;
+    private String result;
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements TFieldIdEnum {
+      ID((short)1, "id"),
+      RESULT((short)2, "result");
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 1: // ID
+            return ID;
+          case 2: // RESULT
+            return RESULT;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+
+    public static final Map<_Fields, FieldMetaData> metaDataMap;
+    static {
+      Map<_Fields, FieldMetaData> tmpMap = new EnumMap<_Fields, FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.ID, new FieldMetaData("id", TFieldRequirementType.DEFAULT, 
+          new FieldValueMetaData(TType.STRING)));
+      tmpMap.put(_Fields.RESULT, new FieldMetaData("result", TFieldRequirementType.DEFAULT, 
+          new FieldValueMetaData(TType.STRING)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      FieldMetaData.addStructMetaDataMap(result_args.class, metaDataMap);
+    }
+
+    public result_args() {
+    }
+
+    public result_args(
+      String id,
+      String result)
+    {
+      this();
+      this.id = id;
+      this.result = result;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public result_args(result_args other) {
+      if (other.is_set_id()) {
+        this.id = other.id;
+      }
+      if (other.is_set_result()) {
+        this.result = other.result;
+      }
+    }
+
+    public result_args deepCopy() {
+      return new result_args(this);
+    }
+
+    @Deprecated
+    public result_args clone() {
+      return new result_args(this);
+    }
+
+    public String get_id() {
+      return this.id;
+    }
+
+    public void set_id(String id) {
+      this.id = id;
+    }
+
+    public void unset_id() {
+      this.id = null;
+    }
+
+    /** Returns true if field id is set (has been asigned a value) and false otherwise */
+    public boolean is_set_id() {
+      return this.id != null;
+    }
+
+    public void set_id_isSet(boolean value) {
+      if (!value) {
+        this.id = null;
+      }
+    }
+
+    public String get_result() {
+      return this.result;
+    }
+
+    public void set_result(String result) {
+      this.result = result;
+    }
+
+    public void unset_result() {
+      this.result = null;
+    }
+
+    /** Returns true if field result is set (has been asigned a value) and false otherwise */
+    public boolean is_set_result() {
+      return this.result != null;
+    }
+
+    public void set_result_isSet(boolean value) {
+      if (!value) {
+        this.result = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case ID:
+        if (value == null) {
+          unset_id();
+        } else {
+          set_id((String)value);
+        }
+        break;
+
+      case RESULT:
+        if (value == null) {
+          unset_result();
+        } else {
+          set_result((String)value);
+        }
+        break;
+
+      }
+    }
+
+    public void setFieldValue(int fieldID, Object value) {
+      setFieldValue(_Fields.findByThriftIdOrThrow(fieldID), value);
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case ID:
+        return get_id();
+
+      case RESULT:
+        return get_result();
+
+      }
+      throw new IllegalStateException();
+    }
+
+    public Object getFieldValue(int fieldId) {
+      return getFieldValue(_Fields.findByThriftIdOrThrow(fieldId));
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been asigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      switch (field) {
+      case ID:
+        return is_set_id();
+      case RESULT:
+        return is_set_result();
+      }
+      throw new IllegalStateException();
+    }
+
+    public boolean isSet(int fieldID) {
+      return isSet(_Fields.findByThriftIdOrThrow(fieldID));
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof result_args)
+        return this.equals((result_args)that);
+      return false;
+    }
+
+    public boolean equals(result_args that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_id = true && this.is_set_id();
+      boolean that_present_id = true && that.is_set_id();
+      if (this_present_id || that_present_id) {
+        if (!(this_present_id && that_present_id))
+          return false;
+        if (!this.id.equals(that.id))
+          return false;
+      }
+
+      boolean this_present_result = true && this.is_set_result();
+      boolean that_present_result = true && that.is_set_result();
+      if (this_present_result || that_present_result) {
+        if (!(this_present_result && that_present_result))
+          return false;
+        if (!this.result.equals(that.result))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      HashCodeBuilder builder = new HashCodeBuilder();
+
+      boolean present_id = true && (is_set_id());
+      builder.append(present_id);
+      if (present_id)
+        builder.append(id);
+
+      boolean present_result = true && (is_set_result());
+      builder.append(present_result);
+      if (present_result)
+        builder.append(result);
+
+      return builder.toHashCode();
+    }
+
+    public int compareTo(result_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+      result_args typedOther = (result_args)other;
+
+      lastComparison = Boolean.valueOf(is_set_id()).compareTo(typedOther.is_set_id());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_id()) {        lastComparison = TBaseHelper.compareTo(this.id, typedOther.id);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(is_set_result()).compareTo(typedOther.is_set_result());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_result()) {        lastComparison = TBaseHelper.compareTo(this.result, typedOther.result);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    public void read(TProtocol iprot) throws TException {
+      TField field;
+      iprot.readStructBegin();
+      while (true)
+      {
+        field = iprot.readFieldBegin();
+        if (field.type == TType.STOP) { 
+          break;
+        }
+        switch (field.id) {
+          case 1: // ID
+            if (field.type == TType.STRING) {
+              this.id = iprot.readString();
+            } else { 
+              TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
+          case 2: // RESULT
+            if (field.type == TType.STRING) {
+              this.result = iprot.readString();
+            } else { 
+              TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
+          default:
+            TProtocolUtil.skip(iprot, field.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      validate();
+    }
+
+    public void write(TProtocol oprot) throws TException {
+      validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (this.id != null) {
+        oprot.writeFieldBegin(ID_FIELD_DESC);
+        oprot.writeString(this.id);
+        oprot.writeFieldEnd();
+      }
+      if (this.result != null) {
+        oprot.writeFieldBegin(RESULT_FIELD_DESC);
+        oprot.writeString(this.result);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("result_args(");
+      boolean first = true;
+
+      sb.append("id:");
+      if (this.id == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.id);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("result:");
+      if (this.result == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.result);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws TException {
+      // check for required fields
+    }
+
+  }
+
+  public static class result_result implements TBase<result_result, result_result._Fields>, java.io.Serializable, Cloneable   {
+    private static final TStruct STRUCT_DESC = new TStruct("result_result");
+
+
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements TFieldIdEnum {
+;
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+    public static final Map<_Fields, FieldMetaData> metaDataMap;
+    static {
+      Map<_Fields, FieldMetaData> tmpMap = new EnumMap<_Fields, FieldMetaData>(_Fields.class);
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      FieldMetaData.addStructMetaDataMap(result_result.class, metaDataMap);
+    }
+
+    public result_result() {
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public result_result(result_result other) {
+    }
+
+    public result_result deepCopy() {
+      return new result_result(this);
+    }
+
+    @Deprecated
+    public result_result clone() {
+      return new result_result(this);
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      }
+    }
+
+    public void setFieldValue(int fieldID, Object value) {
+      setFieldValue(_Fields.findByThriftIdOrThrow(fieldID), value);
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      }
+      throw new IllegalStateException();
+    }
+
+    public Object getFieldValue(int fieldId) {
+      return getFieldValue(_Fields.findByThriftIdOrThrow(fieldId));
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been asigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      switch (field) {
+      }
+      throw new IllegalStateException();
+    }
+
+    public boolean isSet(int fieldID) {
+      return isSet(_Fields.findByThriftIdOrThrow(fieldID));
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof result_result)
+        return this.equals((result_result)that);
+      return false;
+    }
+
+    public boolean equals(result_result that) {
+      if (that == null)
+        return false;
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      HashCodeBuilder builder = new HashCodeBuilder();
+
+      return builder.toHashCode();
+    }
+
+    public int compareTo(result_result other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+      result_result typedOther = (result_result)other;
+
+      return 0;
+    }
+
+    public void read(TProtocol iprot) throws TException {
+      TField field;
+      iprot.readStructBegin();
+      while (true)
+      {
+        field = iprot.readFieldBegin();
+        if (field.type == TType.STOP) { 
+          break;
+        }
+        switch (field.id) {
+          default:
+            TProtocolUtil.skip(iprot, field.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      validate();
+    }
+
+    public void write(TProtocol oprot) throws TException {
+      oprot.writeStructBegin(STRUCT_DESC);
+
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("result_result(");
+      boolean first = true;
+
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws TException {
+      // check for required fields
+    }
+
+  }
+
+}
diff --git a/src/jvm/backtype/storm/generated/ErrorInfo.java b/src/jvm/backtype/storm/generated/ErrorInfo.java
new file mode 100644
index 0000000..e54440c
--- /dev/null
+++ b/src/jvm/backtype/storm/generated/ErrorInfo.java
@@ -0,0 +1,412 @@
+/**
+ * Autogenerated by Thrift
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ */
+package backtype.storm.generated;
+
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.util.Arrays;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.thrift.*;
+import org.apache.thrift.async.*;
+import org.apache.thrift.meta_data.*;
+import org.apache.thrift.transport.*;
+import org.apache.thrift.protocol.*;
+
+public class ErrorInfo implements TBase<ErrorInfo, ErrorInfo._Fields>, java.io.Serializable, Cloneable {
+  private static final TStruct STRUCT_DESC = new TStruct("ErrorInfo");
+
+  private static final TField ERROR_FIELD_DESC = new TField("error", TType.STRING, (short)1);
+  private static final TField ERROR_TIME_SECS_FIELD_DESC = new TField("error_time_secs", TType.I32, (short)2);
+
+  private String error;
+  private int error_time_secs;
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements TFieldIdEnum {
+    ERROR((short)1, "error"),
+    ERROR_TIME_SECS((short)2, "error_time_secs");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // ERROR
+          return ERROR;
+        case 2: // ERROR_TIME_SECS
+          return ERROR_TIME_SECS;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __ERROR_TIME_SECS_ISSET_ID = 0;
+  private BitSet __isset_bit_vector = new BitSet(1);
+
+  public static final Map<_Fields, FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, FieldMetaData> tmpMap = new EnumMap<_Fields, FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.ERROR, new FieldMetaData("error", TFieldRequirementType.REQUIRED, 
+        new FieldValueMetaData(TType.STRING)));
+    tmpMap.put(_Fields.ERROR_TIME_SECS, new FieldMetaData("error_time_secs", TFieldRequirementType.REQUIRED, 
+        new FieldValueMetaData(TType.I32)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    FieldMetaData.addStructMetaDataMap(ErrorInfo.class, metaDataMap);
+  }
+
+  public ErrorInfo() {
+  }
+
+  public ErrorInfo(
+    String error,
+    int error_time_secs)
+  {
+    this();
+    this.error = error;
+    this.error_time_secs = error_time_secs;
+    set_error_time_secs_isSet(true);
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public ErrorInfo(ErrorInfo other) {
+    __isset_bit_vector.clear();
+    __isset_bit_vector.or(other.__isset_bit_vector);
+    if (other.is_set_error()) {
+      this.error = other.error;
+    }
+    this.error_time_secs = other.error_time_secs;
+  }
+
+  public ErrorInfo deepCopy() {
+    return new ErrorInfo(this);
+  }
+
+  @Deprecated
+  public ErrorInfo clone() {
+    return new ErrorInfo(this);
+  }
+
+  public String get_error() {
+    return this.error;
+  }
+
+  public void set_error(String error) {
+    this.error = error;
+  }
+
+  public void unset_error() {
+    this.error = null;
+  }
+
+  /** Returns true if field error is set (has been asigned a value) and false otherwise */
+  public boolean is_set_error() {
+    return this.error != null;
+  }
+
+  public void set_error_isSet(boolean value) {
+    if (!value) {
+      this.error = null;
+    }
+  }
+
+  public int get_error_time_secs() {
+    return this.error_time_secs;
+  }
+
+  public void set_error_time_secs(int error_time_secs) {
+    this.error_time_secs = error_time_secs;
+    set_error_time_secs_isSet(true);
+  }
+
+  public void unset_error_time_secs() {
+    __isset_bit_vector.clear(__ERROR_TIME_SECS_ISSET_ID);
+  }
+
+  /** Returns true if field error_time_secs is set (has been asigned a value) and false otherwise */
+  public boolean is_set_error_time_secs() {
+    return __isset_bit_vector.get(__ERROR_TIME_SECS_ISSET_ID);
+  }
+
+  public void set_error_time_secs_isSet(boolean value) {
+    __isset_bit_vector.set(__ERROR_TIME_SECS_ISSET_ID, value);
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case ERROR:
+      if (value == null) {
+        unset_error();
+      } else {
+        set_error((String)value);
+      }
+      break;
+
+    case ERROR_TIME_SECS:
+      if (value == null) {
+        unset_error_time_secs();
+      } else {
+        set_error_time_secs((Integer)value);
+      }
+      break;
+
+    }
+  }
+
+  public void setFieldValue(int fieldID, Object value) {
+    setFieldValue(_Fields.findByThriftIdOrThrow(fieldID), value);
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case ERROR:
+      return get_error();
+
+    case ERROR_TIME_SECS:
+      return new Integer(get_error_time_secs());
+
+    }
+    throw new IllegalStateException();
+  }
+
+  public Object getFieldValue(int fieldId) {
+    return getFieldValue(_Fields.findByThriftIdOrThrow(fieldId));
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been asigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    switch (field) {
+    case ERROR:
+      return is_set_error();
+    case ERROR_TIME_SECS:
+      return is_set_error_time_secs();
+    }
+    throw new IllegalStateException();
+  }
+
+  public boolean isSet(int fieldID) {
+    return isSet(_Fields.findByThriftIdOrThrow(fieldID));
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof ErrorInfo)
+      return this.equals((ErrorInfo)that);
+    return false;
+  }
+
+  public boolean equals(ErrorInfo that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_error = true && this.is_set_error();
+    boolean that_present_error = true && that.is_set_error();
+    if (this_present_error || that_present_error) {
+      if (!(this_present_error && that_present_error))
+        return false;
+      if (!this.error.equals(that.error))
+        return false;
+    }
+
+    boolean this_present_error_time_secs = true;
+    boolean that_present_error_time_secs = true;
+    if (this_present_error_time_secs || that_present_error_time_secs) {
+      if (!(this_present_error_time_secs && that_present_error_time_secs))
+        return false;
+      if (this.error_time_secs != that.error_time_secs)
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    HashCodeBuilder builder = new HashCodeBuilder();
+
+    boolean present_error = true && (is_set_error());
+    builder.append(present_error);
+    if (present_error)
+      builder.append(error);
+
+    boolean present_error_time_secs = true;
+    builder.append(present_error_time_secs);
+    if (present_error_time_secs)
+      builder.append(error_time_secs);
+
+    return builder.toHashCode();
+  }
+
+  public int compareTo(ErrorInfo other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+    ErrorInfo typedOther = (ErrorInfo)other;
+
+    lastComparison = Boolean.valueOf(is_set_error()).compareTo(typedOther.is_set_error());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_error()) {      lastComparison = TBaseHelper.compareTo(this.error, typedOther.error);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_error_time_secs()).compareTo(typedOther.is_set_error_time_secs());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_error_time_secs()) {      lastComparison = TBaseHelper.compareTo(this.error_time_secs, typedOther.error_time_secs);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public void read(TProtocol iprot) throws TException {
+    TField field;
+    iprot.readStructBegin();
+    while (true)
+    {
+      field = iprot.readFieldBegin();
+      if (field.type == TType.STOP) { 
+        break;
+      }
+      switch (field.id) {
+        case 1: // ERROR
+          if (field.type == TType.STRING) {
+            this.error = iprot.readString();
+          } else { 
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 2: // ERROR_TIME_SECS
+          if (field.type == TType.I32) {
+            this.error_time_secs = iprot.readI32();
+            set_error_time_secs_isSet(true);
+          } else { 
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        default:
+          TProtocolUtil.skip(iprot, field.type);
+      }
+      iprot.readFieldEnd();
+    }
+    iprot.readStructEnd();
+    validate();
+  }
+
+  public void write(TProtocol oprot) throws TException {
+    validate();
+
+    oprot.writeStructBegin(STRUCT_DESC);
+    if (this.error != null) {
+      oprot.writeFieldBegin(ERROR_FIELD_DESC);
+      oprot.writeString(this.error);
+      oprot.writeFieldEnd();
+    }
+    oprot.writeFieldBegin(ERROR_TIME_SECS_FIELD_DESC);
+    oprot.writeI32(this.error_time_secs);
+    oprot.writeFieldEnd();
+    oprot.writeFieldStop();
+    oprot.writeStructEnd();
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("ErrorInfo(");
+    boolean first = true;
+
+    sb.append("error:");
+    if (this.error == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.error);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("error_time_secs:");
+    sb.append(this.error_time_secs);
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws TException {
+    // check for required fields
+    if (!is_set_error()) {
+      throw new TProtocolException("Required field 'error' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_error_time_secs()) {
+      throw new TProtocolException("Required field 'error_time_secs' is unset! Struct:" + toString());
+    }
+
+  }
+
+}
+
diff --git a/src/jvm/backtype/storm/generated/GlobalStreamId.java b/src/jvm/backtype/storm/generated/GlobalStreamId.java
new file mode 100644
index 0000000..4643e1c
--- /dev/null
+++ b/src/jvm/backtype/storm/generated/GlobalStreamId.java
@@ -0,0 +1,406 @@
+/**
+ * Autogenerated by Thrift
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ */
+package backtype.storm.generated;
+
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.util.Arrays;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.thrift.*;
+import org.apache.thrift.async.*;
+import org.apache.thrift.meta_data.*;
+import org.apache.thrift.transport.*;
+import org.apache.thrift.protocol.*;
+
+public class GlobalStreamId implements TBase<GlobalStreamId, GlobalStreamId._Fields>, java.io.Serializable, Cloneable {
+  private static final TStruct STRUCT_DESC = new TStruct("GlobalStreamId");
+
+  private static final TField COMPONENT_ID_FIELD_DESC = new TField("componentId", TType.I32, (short)1);
+  private static final TField STREAM_ID_FIELD_DESC = new TField("streamId", TType.I32, (short)2);
+
+  private int componentId;
+  private int streamId;
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements TFieldIdEnum {
+    COMPONENT_ID((short)1, "componentId"),
+    STREAM_ID((short)2, "streamId");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // COMPONENT_ID
+          return COMPONENT_ID;
+        case 2: // STREAM_ID
+          return STREAM_ID;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __COMPONENTID_ISSET_ID = 0;
+  private static final int __STREAMID_ISSET_ID = 1;
+  private BitSet __isset_bit_vector = new BitSet(2);
+
+  public static final Map<_Fields, FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, FieldMetaData> tmpMap = new EnumMap<_Fields, FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.COMPONENT_ID, new FieldMetaData("componentId", TFieldRequirementType.REQUIRED, 
+        new FieldValueMetaData(TType.I32)));
+    tmpMap.put(_Fields.STREAM_ID, new FieldMetaData("streamId", TFieldRequirementType.REQUIRED, 
+        new FieldValueMetaData(TType.I32)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    FieldMetaData.addStructMetaDataMap(GlobalStreamId.class, metaDataMap);
+  }
+
+  public GlobalStreamId() {
+  }
+
+  public GlobalStreamId(
+    int componentId,
+    int streamId)
+  {
+    this();
+    this.componentId = componentId;
+    set_componentId_isSet(true);
+    this.streamId = streamId;
+    set_streamId_isSet(true);
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public GlobalStreamId(GlobalStreamId other) {
+    __isset_bit_vector.clear();
+    __isset_bit_vector.or(other.__isset_bit_vector);
+    this.componentId = other.componentId;
+    this.streamId = other.streamId;
+  }
+
+  public GlobalStreamId deepCopy() {
+    return new GlobalStreamId(this);
+  }
+
+  @Deprecated
+  public GlobalStreamId clone() {
+    return new GlobalStreamId(this);
+  }
+
+  public int get_componentId() {
+    return this.componentId;
+  }
+
+  public void set_componentId(int componentId) {
+    this.componentId = componentId;
+    set_componentId_isSet(true);
+  }
+
+  public void unset_componentId() {
+    __isset_bit_vector.clear(__COMPONENTID_ISSET_ID);
+  }
+
+  /** Returns true if field componentId is set (has been asigned a value) and false otherwise */
+  public boolean is_set_componentId() {
+    return __isset_bit_vector.get(__COMPONENTID_ISSET_ID);
+  }
+
+  public void set_componentId_isSet(boolean value) {
+    __isset_bit_vector.set(__COMPONENTID_ISSET_ID, value);
+  }
+
+  public int get_streamId() {
+    return this.streamId;
+  }
+
+  public void set_streamId(int streamId) {
+    this.streamId = streamId;
+    set_streamId_isSet(true);
+  }
+
+  public void unset_streamId() {
+    __isset_bit_vector.clear(__STREAMID_ISSET_ID);
+  }
+
+  /** Returns true if field streamId is set (has been asigned a value) and false otherwise */
+  public boolean is_set_streamId() {
+    return __isset_bit_vector.get(__STREAMID_ISSET_ID);
+  }
+
+  public void set_streamId_isSet(boolean value) {
+    __isset_bit_vector.set(__STREAMID_ISSET_ID, value);
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case COMPONENT_ID:
+      if (value == null) {
+        unset_componentId();
+      } else {
+        set_componentId((Integer)value);
+      }
+      break;
+
+    case STREAM_ID:
+      if (value == null) {
+        unset_streamId();
+      } else {
+        set_streamId((Integer)value);
+      }
+      break;
+
+    }
+  }
+
+  public void setFieldValue(int fieldID, Object value) {
+    setFieldValue(_Fields.findByThriftIdOrThrow(fieldID), value);
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case COMPONENT_ID:
+      return new Integer(get_componentId());
+
+    case STREAM_ID:
+      return new Integer(get_streamId());
+
+    }
+    throw new IllegalStateException();
+  }
+
+  public Object getFieldValue(int fieldId) {
+    return getFieldValue(_Fields.findByThriftIdOrThrow(fieldId));
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been asigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    switch (field) {
+    case COMPONENT_ID:
+      return is_set_componentId();
+    case STREAM_ID:
+      return is_set_streamId();
+    }
+    throw new IllegalStateException();
+  }
+
+  public boolean isSet(int fieldID) {
+    return isSet(_Fields.findByThriftIdOrThrow(fieldID));
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof GlobalStreamId)
+      return this.equals((GlobalStreamId)that);
+    return false;
+  }
+
+  public boolean equals(GlobalStreamId that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_componentId = true;
+    boolean that_present_componentId = true;
+    if (this_present_componentId || that_present_componentId) {
+      if (!(this_present_componentId && that_present_componentId))
+        return false;
+      if (this.componentId != that.componentId)
+        return false;
+    }
+
+    boolean this_present_streamId = true;
+    boolean that_present_streamId = true;
+    if (this_present_streamId || that_present_streamId) {
+      if (!(this_present_streamId && that_present_streamId))
+        return false;
+      if (this.streamId != that.streamId)
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    HashCodeBuilder builder = new HashCodeBuilder();
+
+    boolean present_componentId = true;
+    builder.append(present_componentId);
+    if (present_componentId)
+      builder.append(componentId);
+
+    boolean present_streamId = true;
+    builder.append(present_streamId);
+    if (present_streamId)
+      builder.append(streamId);
+
+    return builder.toHashCode();
+  }
+
+  public int compareTo(GlobalStreamId other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+    GlobalStreamId typedOther = (GlobalStreamId)other;
+
+    lastComparison = Boolean.valueOf(is_set_componentId()).compareTo(typedOther.is_set_componentId());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_componentId()) {      lastComparison = TBaseHelper.compareTo(this.componentId, typedOther.componentId);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_streamId()).compareTo(typedOther.is_set_streamId());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_streamId()) {      lastComparison = TBaseHelper.compareTo(this.streamId, typedOther.streamId);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public void read(TProtocol iprot) throws TException {
+    TField field;
+    iprot.readStructBegin();
+    while (true)
+    {
+      field = iprot.readFieldBegin();
+      if (field.type == TType.STOP) { 
+        break;
+      }
+      switch (field.id) {
+        case 1: // COMPONENT_ID
+          if (field.type == TType.I32) {
+            this.componentId = iprot.readI32();
+            set_componentId_isSet(true);
+          } else { 
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 2: // STREAM_ID
+          if (field.type == TType.I32) {
+            this.streamId = iprot.readI32();
+            set_streamId_isSet(true);
+          } else { 
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        default:
+          TProtocolUtil.skip(iprot, field.type);
+      }
+      iprot.readFieldEnd();
+    }
+    iprot.readStructEnd();
+    validate();
+  }
+
+  public void write(TProtocol oprot) throws TException {
+    validate();
+
+    oprot.writeStructBegin(STRUCT_DESC);
+    oprot.writeFieldBegin(COMPONENT_ID_FIELD_DESC);
+    oprot.writeI32(this.componentId);
+    oprot.writeFieldEnd();
+    oprot.writeFieldBegin(STREAM_ID_FIELD_DESC);
+    oprot.writeI32(this.streamId);
+    oprot.writeFieldEnd();
+    oprot.writeFieldStop();
+    oprot.writeStructEnd();
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("GlobalStreamId(");
+    boolean first = true;
+
+    sb.append("componentId:");
+    sb.append(this.componentId);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("streamId:");
+    sb.append(this.streamId);
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws TException {
+    // check for required fields
+    if (!is_set_componentId()) {
+      throw new TProtocolException("Required field 'componentId' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_streamId()) {
+      throw new TProtocolException("Required field 'streamId' is unset! Struct:" + toString());
+    }
+
+  }
+
+}
+
diff --git a/src/jvm/backtype/storm/generated/Grouping.java b/src/jvm/backtype/storm/generated/Grouping.java
new file mode 100644
index 0000000..1061413
--- /dev/null
+++ b/src/jvm/backtype/storm/generated/Grouping.java
@@ -0,0 +1,460 @@
+/**
+ * Autogenerated by Thrift
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ */
+package backtype.storm.generated;
+
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.util.Arrays;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.thrift.*;
+import org.apache.thrift.async.*;
+import org.apache.thrift.meta_data.*;
+import org.apache.thrift.transport.*;
+import org.apache.thrift.protocol.*;
+
+public class Grouping extends TUnion<Grouping, Grouping._Fields> {
+  private static final TStruct STRUCT_DESC = new TStruct("Grouping");
+  private static final TField FIELDS_FIELD_DESC = new TField("fields", TType.LIST, (short)1);
+  private static final TField SHUFFLE_FIELD_DESC = new TField("shuffle", TType.STRUCT, (short)2);
+  private static final TField ALL_FIELD_DESC = new TField("all", TType.STRUCT, (short)3);
+  private static final TField NONE_FIELD_DESC = new TField("none", TType.STRUCT, (short)4);
+  private static final TField DIRECT_FIELD_DESC = new TField("direct", TType.STRUCT, (short)5);
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements TFieldIdEnum {
+    FIELDS((short)1, "fields"),
+    SHUFFLE((short)2, "shuffle"),
+    ALL((short)3, "all"),
+    NONE((short)4, "none"),
+    DIRECT((short)5, "direct");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // FIELDS
+          return FIELDS;
+        case 2: // SHUFFLE
+          return SHUFFLE;
+        case 3: // ALL
+          return ALL;
+        case 4: // NONE
+          return NONE;
+        case 5: // DIRECT
+          return DIRECT;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  public static final Map<_Fields, FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, FieldMetaData> tmpMap = new EnumMap<_Fields, FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.FIELDS, new FieldMetaData("fields", TFieldRequirementType.DEFAULT, 
+        new ListMetaData(TType.LIST, 
+            new FieldValueMetaData(TType.STRING))));
+    tmpMap.put(_Fields.SHUFFLE, new FieldMetaData("shuffle", TFieldRequirementType.DEFAULT, 
+        new StructMetaData(TType.STRUCT, NullStruct.class)));
+    tmpMap.put(_Fields.ALL, new FieldMetaData("all", TFieldRequirementType.DEFAULT, 
+        new StructMetaData(TType.STRUCT, NullStruct.class)));
+    tmpMap.put(_Fields.NONE, new FieldMetaData("none", TFieldRequirementType.DEFAULT, 
+        new StructMetaData(TType.STRUCT, NullStruct.class)));
+    tmpMap.put(_Fields.DIRECT, new FieldMetaData("direct", TFieldRequirementType.DEFAULT, 
+        new StructMetaData(TType.STRUCT, NullStruct.class)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    FieldMetaData.addStructMetaDataMap(Grouping.class, metaDataMap);
+  }
+
+  public Grouping() {
+    super();
+  }
+
+  public Grouping(_Fields setField, Object value) {
+    super(setField, value);
+  }
+
+  public Grouping(Grouping other) {
+    super(other);
+  }
+  public Grouping deepCopy() {
+    return new Grouping(this);
+  }
+
+  public static Grouping fields(List<String> value) {
+    Grouping x = new Grouping();
+    x.set_fields(value);
+    return x;
+  }
+
+  public static Grouping shuffle(NullStruct value) {
+    Grouping x = new Grouping();
+    x.set_shuffle(value);
+    return x;
+  }
+
+  public static Grouping all(NullStruct value) {
+    Grouping x = new Grouping();
+    x.set_all(value);
+    return x;
+  }
+
+  public static Grouping none(NullStruct value) {
+    Grouping x = new Grouping();
+    x.set_none(value);
+    return x;
+  }
+
+  public static Grouping direct(NullStruct value) {
+    Grouping x = new Grouping();
+    x.set_direct(value);
+    return x;
+  }
+
+
+  @Override
+  protected void checkType(_Fields setField, Object value) throws ClassCastException {
+    switch (setField) {
+      case FIELDS:
+        if (value instanceof List) {
+          break;
+        }
+        throw new ClassCastException("Was expecting value of type List<String> for field 'fields', but got " + value.getClass().getSimpleName());
+      case SHUFFLE:
+        if (value instanceof NullStruct) {
+          break;
+        }
+        throw new ClassCastException("Was expecting value of type NullStruct for field 'shuffle', but got " + value.getClass().getSimpleName());
+      case ALL:
+        if (value instanceof NullStruct) {
+          break;
+        }
+        throw new ClassCastException("Was expecting value of type NullStruct for field 'all', but got " + value.getClass().getSimpleName());
+      case NONE:
+        if (value instanceof NullStruct) {
+          break;
+        }
+        throw new ClassCastException("Was expecting value of type NullStruct for field 'none', but got " + value.getClass().getSimpleName());
+      case DIRECT:
+        if (value instanceof NullStruct) {
+          break;
+        }
+        throw new ClassCastException("Was expecting value of type NullStruct for field 'direct', but got " + value.getClass().getSimpleName());
+      default:
+        throw new IllegalArgumentException("Unknown field id " + setField);
+    }
+  }
+
+  @Override
+  protected Object readValue(TProtocol iprot, TField field) throws TException {
+    _Fields setField = _Fields.findByThriftId(field.id);
+    if (setField != null) {
+      switch (setField) {
+        case FIELDS:
+          if (field.type == FIELDS_FIELD_DESC.type) {
+            List<String> fields;
+            {
+              TList _list0 = iprot.readListBegin();
+              fields = new ArrayList<String>(_list0.size);
+              for (int _i1 = 0; _i1 < _list0.size; ++_i1)
+              {
+                String _elem2;
+                _elem2 = iprot.readString();
+                fields.add(_elem2);
+              }
+              iprot.readListEnd();
+            }
+            return fields;
+          } else {
+            TProtocolUtil.skip(iprot, field.type);
+            return null;
+          }
+        case SHUFFLE:
+          if (field.type == SHUFFLE_FIELD_DESC.type) {
+            NullStruct shuffle;
+            shuffle = new NullStruct();
+            shuffle.read(iprot);
+            return shuffle;
+          } else {
+            TProtocolUtil.skip(iprot, field.type);
+            return null;
+          }
+        case ALL:
+          if (field.type == ALL_FIELD_DESC.type) {
+            NullStruct all;
+            all = new NullStruct();
+            all.read(iprot);
+            return all;
+          } else {
+            TProtocolUtil.skip(iprot, field.type);
+            return null;
+          }
+        case NONE:
+          if (field.type == NONE_FIELD_DESC.type) {
+            NullStruct none;
+            none = new NullStruct();
+            none.read(iprot);
+            return none;
+          } else {
+            TProtocolUtil.skip(iprot, field.type);
+            return null;
+          }
+        case DIRECT:
+          if (field.type == DIRECT_FIELD_DESC.type) {
+            NullStruct direct;
+            direct = new NullStruct();
+            direct.read(iprot);
+            return direct;
+          } else {
+            TProtocolUtil.skip(iprot, field.type);
+            return null;
+          }
+        default:
+          throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!");
+      }
+    } else {
+      TProtocolUtil.skip(iprot, field.type);
+      return null;
+    }
+  }
+
+  @Override
+  protected void writeValue(TProtocol oprot) throws TException {
+    switch (setField_) {
+      case FIELDS:
+        List<String> fields = (List<String>)value_;
+        {
+          oprot.writeListBegin(new TList(TType.STRING, fields.size()));
+          for (String _iter3 : fields)
+          {
+            oprot.writeString(_iter3);
+          }
+          oprot.writeListEnd();
+        }
+        return;
+      case SHUFFLE:
+        NullStruct shuffle = (NullStruct)value_;
+        shuffle.write(oprot);
+        return;
+      case ALL:
+        NullStruct all = (NullStruct)value_;
+        all.write(oprot);
+        return;
+      case NONE:
+        NullStruct none = (NullStruct)value_;
+        none.write(oprot);
+        return;
+      case DIRECT:
+        NullStruct direct = (NullStruct)value_;
+        direct.write(oprot);
+        return;
+      default:
+        throw new IllegalStateException("Cannot write union with unknown field " + setField_);
+    }
+  }
+
+  @Override
+  protected TField getFieldDesc(_Fields setField) {
+    switch (setField) {
+      case FIELDS:
+        return FIELDS_FIELD_DESC;
+      case SHUFFLE:
+        return SHUFFLE_FIELD_DESC;
+      case ALL:
+        return ALL_FIELD_DESC;
+      case NONE:
+        return NONE_FIELD_DESC;
+      case DIRECT:
+        return DIRECT_FIELD_DESC;
+      default:
+        throw new IllegalArgumentException("Unknown field id " + setField);
+    }
+  }
+
+  @Override
+  protected TStruct getStructDesc() {
+    return STRUCT_DESC;
+  }
+
+  @Override
+  protected _Fields enumForId(short id) {
+    return _Fields.findByThriftIdOrThrow(id);
+  }
+
+  public List<String> get_fields() {
+    if (getSetField() == _Fields.FIELDS) {
+      return (List<String>)getFieldValue();
+    } else {
+      throw new RuntimeException("Cannot get field 'fields' because union is currently set to " + getFieldDesc(getSetField()).name);
+    }
+  }
+
+  public void set_fields(List<String> value) {
+    if (value == null) throw new NullPointerException();
+    setField_ = _Fields.FIELDS;
+    value_ = value;
+  }
+
+  public NullStruct get_shuffle() {
+    if (getSetField() == _Fields.SHUFFLE) {
+      return (NullStruct)getFieldValue();
+    } else {
+      throw new RuntimeException("Cannot get field 'shuffle' because union is currently set to " + getFieldDesc(getSetField()).name);
+    }
+  }
+
+  public void set_shuffle(NullStruct value) {
+    if (value == null) throw new NullPointerException();
+    setField_ = _Fields.SHUFFLE;
+    value_ = value;
+  }
+
+  public NullStruct get_all() {
+    if (getSetField() == _Fields.ALL) {
+      return (NullStruct)getFieldValue();
+    } else {
+      throw new RuntimeException("Cannot get field 'all' because union is currently set to " + getFieldDesc(getSetField()).name);
+    }
+  }
+
+  public void set_all(NullStruct value) {
+    if (value == null) throw new NullPointerException();
+    setField_ = _Fields.ALL;
+    value_ = value;
+  }
+
+  public NullStruct get_none() {
+    if (getSetField() == _Fields.NONE) {
+      return (NullStruct)getFieldValue();
+    } else {
+      throw new RuntimeException("Cannot get field 'none' because union is currently set to " + getFieldDesc(getSetField()).name);
+    }
+  }
+
+  public void set_none(NullStruct value) {
+    if (value == null) throw new NullPointerException();
+    setField_ = _Fields.NONE;
+    value_ = value;
+  }
+
+  public NullStruct get_direct() {
+    if (getSetField() == _Fields.DIRECT) {
+      return (NullStruct)getFieldValue();
+    } else {
+      throw new RuntimeException("Cannot get field 'direct' because union is currently set to " + getFieldDesc(getSetField()).name);
+    }
+  }
+
+  public void set_direct(NullStruct value) {
+    if (value == null) throw new NullPointerException();
+    setField_ = _Fields.DIRECT;
+    value_ = value;
+  }
+
+  public boolean equals(Object other) {
+    if (other instanceof Grouping) {
+      return equals((Grouping)other);
+    } else {
+      return false;
+    }
+  }
+
+  public boolean equals(Grouping other) {
+    return other != null && getSetField() == other.getSetField() && ((value_ instanceof byte[]) ? 
+      Arrays.equals((byte[])getFieldValue(), (byte[])other.getFieldValue()) : getFieldValue().equals(other.getFieldValue()));
+  }
+
+  @Override
+  public int compareTo(Grouping other) {
+    int lastComparison = TBaseHelper.compareTo(getSetField(), other.getSetField());
+    if (lastComparison == 0) {
+      Object myValue = getFieldValue();
+      if (myValue instanceof byte[]) {
+        return TBaseHelper.compareTo((byte[])myValue, (byte[])other.getFieldValue());
+      } else if (myValue instanceof List) {
+        return TBaseHelper.compareTo((List)myValue, (List)other.getFieldValue());
+      } else if (myValue instanceof Set) {
+        return TBaseHelper.compareTo((Set)myValue, (Set)other.getFieldValue());
+      } else if (myValue instanceof Map){
+        return TBaseHelper.compareTo((Map)myValue, (Map)other.getFieldValue());
+      } else {
+        return TBaseHelper.compareTo((Comparable)myValue, (Comparable)other.getFieldValue());
+      }
+    }
+    return lastComparison;
+  }
+
+
+  @Override
+  public int hashCode() {
+    HashCodeBuilder hcb = new HashCodeBuilder();
+    hcb.append(this.getClass().getName());
+    TFieldIdEnum setField = getSetField();
+    if (setField != null) {
+      hcb.append(setField.getThriftFieldId());
+      Object value = getFieldValue();
+      if (value instanceof TEnum) {
+        hcb.append(((TEnum)getFieldValue()).getValue());
+      } else {
+        hcb.append(value);
+      }
+    }
+    return hcb.toHashCode();
+  }
+}
diff --git a/src/jvm/backtype/storm/generated/InvalidTopologyException.java b/src/jvm/backtype/storm/generated/InvalidTopologyException.java
new file mode 100644
index 0000000..72a175c
--- /dev/null
+++ b/src/jvm/backtype/storm/generated/InvalidTopologyException.java
@@ -0,0 +1,320 @@
+/**
+ * Autogenerated by Thrift
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ */
+package backtype.storm.generated;
+
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.util.Arrays;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.thrift.*;
+import org.apache.thrift.async.*;
+import org.apache.thrift.meta_data.*;
+import org.apache.thrift.transport.*;
+import org.apache.thrift.protocol.*;
+
+public class InvalidTopologyException extends Exception implements TBase<InvalidTopologyException, InvalidTopologyException._Fields>, java.io.Serializable, Cloneable {
+  private static final TStruct STRUCT_DESC = new TStruct("InvalidTopologyException");
+
+  private static final TField MSG_FIELD_DESC = new TField("msg", TType.STRING, (short)1);
+
+  private String msg;
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements TFieldIdEnum {
+    MSG((short)1, "msg");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // MSG
+          return MSG;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+
+  public static final Map<_Fields, FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, FieldMetaData> tmpMap = new EnumMap<_Fields, FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.MSG, new FieldMetaData("msg", TFieldRequirementType.REQUIRED, 
+        new FieldValueMetaData(TType.STRING)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    FieldMetaData.addStructMetaDataMap(InvalidTopologyException.class, metaDataMap);
+  }
+
+  public InvalidTopologyException() {
+  }
+
+  public InvalidTopologyException(
+    String msg)
+  {
+    this();
+    this.msg = msg;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public InvalidTopologyException(InvalidTopologyException other) {
+    if (other.is_set_msg()) {
+      this.msg = other.msg;
+    }
+  }
+
+  public InvalidTopologyException deepCopy() {
+    return new InvalidTopologyException(this);
+  }
+
+  @Deprecated
+  public InvalidTopologyException clone() {
+    return new InvalidTopologyException(this);
+  }
+
+  public String get_msg() {
+    return this.msg;
+  }
+
+  public void set_msg(String msg) {
+    this.msg = msg;
+  }
+
+  public void unset_msg() {
+    this.msg = null;
+  }
+
+  /** Returns true if field msg is set (has been asigned a value) and false otherwise */
+  public boolean is_set_msg() {
+    return this.msg != null;
+  }
+
+  public void set_msg_isSet(boolean value) {
+    if (!value) {
+      this.msg = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case MSG:
+      if (value == null) {
+        unset_msg();
+      } else {
+        set_msg((String)value);
+      }
+      break;
+
+    }
+  }
+
+  public void setFieldValue(int fieldID, Object value) {
+    setFieldValue(_Fields.findByThriftIdOrThrow(fieldID), value);
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case MSG:
+      return get_msg();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  public Object getFieldValue(int fieldId) {
+    return getFieldValue(_Fields.findByThriftIdOrThrow(fieldId));
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been asigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    switch (field) {
+    case MSG:
+      return is_set_msg();
+    }
+    throw new IllegalStateException();
+  }
+
+  public boolean isSet(int fieldID) {
+    return isSet(_Fields.findByThriftIdOrThrow(fieldID));
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof InvalidTopologyException)
+      return this.equals((InvalidTopologyException)that);
+    return false;
+  }
+
+  public boolean equals(InvalidTopologyException that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_msg = true && this.is_set_msg();
+    boolean that_present_msg = true && that.is_set_msg();
+    if (this_present_msg || that_present_msg) {
+      if (!(this_present_msg && that_present_msg))
+        return false;
+      if (!this.msg.equals(that.msg))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    HashCodeBuilder builder = new HashCodeBuilder();
+
+    boolean present_msg = true && (is_set_msg());
+    builder.append(present_msg);
+    if (present_msg)
+      builder.append(msg);
+
+    return builder.toHashCode();
+  }
+
+  public int compareTo(InvalidTopologyException other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+    InvalidTopologyException typedOther = (InvalidTopologyException)other;
+
+    lastComparison = Boolean.valueOf(is_set_msg()).compareTo(typedOther.is_set_msg());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_msg()) {      lastComparison = TBaseHelper.compareTo(this.msg, typedOther.msg);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public void read(TProtocol iprot) throws TException {
+    TField field;
+    iprot.readStructBegin();
+    while (true)
+    {
+      field = iprot.readFieldBegin();
+      if (field.type == TType.STOP) { 
+        break;
+      }
+      switch (field.id) {
+        case 1: // MSG
+          if (field.type == TType.STRING) {
+            this.msg = iprot.readString();
+          } else { 
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        default:
+          TProtocolUtil.skip(iprot, field.type);
+      }
+      iprot.readFieldEnd();
+    }
+    iprot.readStructEnd();
+    validate();
+  }
+
+  public void write(TProtocol oprot) throws TException {
+    validate();
+
+    oprot.writeStructBegin(STRUCT_DESC);
+    if (this.msg != null) {
+      oprot.writeFieldBegin(MSG_FIELD_DESC);
+      oprot.writeString(this.msg);
+      oprot.writeFieldEnd();
+    }
+    oprot.writeFieldStop();
+    oprot.writeStructEnd();
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("InvalidTopologyException(");
+    boolean first = true;
+
+    sb.append("msg:");
+    if (this.msg == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.msg);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws TException {
+    // check for required fields
+    if (!is_set_msg()) {
+      throw new TProtocolException("Required field 'msg' is unset! Struct:" + toString());
+    }
+
+  }
+
+}
+
diff --git a/src/jvm/backtype/storm/generated/Nimbus.java b/src/jvm/backtype/storm/generated/Nimbus.java
new file mode 100644
index 0000000..6fd9e17
--- /dev/null
+++ b/src/jvm/backtype/storm/generated/Nimbus.java
@@ -0,0 +1,7967 @@
+/**
+ * Autogenerated by Thrift
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ */
+package backtype.storm.generated;
+
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.util.Arrays;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.thrift.*;
+import org.apache.thrift.async.*;
+import org.apache.thrift.meta_data.*;
+import org.apache.thrift.transport.*;
+import org.apache.thrift.protocol.*;
+
+public class Nimbus {
+
+  public interface Iface {
+
+    public void submitTopology(String name, String uploadedJarLocation, String jsonConf, StormTopology topology) throws AlreadyAliveException, InvalidTopologyException, TException;
+
+    public void killTopology(String name) throws NotAliveException, TException;
+
+    public String beginFileUpload() throws TException;
+
+    public void uploadChunk(String location, byte[] chunk) throws TException;
+
+    public void finishFileUpload(String location) throws TException;
+
+    public String beginFileDownload(String file) throws TException;
+
+    public byte[] downloadChunk(String id) throws TException;
+
+    public ClusterSummary getClusterInfo() throws TException;
+
+    public TopologyInfo getTopologyInfo(String id) throws NotAliveException, TException;
+
+    public String getTopologyConf(String id) throws NotAliveException, TException;
+
+    public StormTopology getTopology(String id) throws NotAliveException, TException;
+
+  }
+
+  public interface AsyncIface {
+
+    public void submitTopology(String name, String uploadedJarLocation, String jsonConf, StormTopology topology, AsyncMethodCallback<AsyncClient.submitTopology_call> resultHandler) throws TException;
+
+    public void killTopology(String name, AsyncMethodCallback<AsyncClient.killTopology_call> resultHandler) throws TException;
+
+    public void beginFileUpload(AsyncMethodCallback<AsyncClient.beginFileUpload_call> resultHandler) throws TException;
+
+    public void uploadChunk(String location, byte[] chunk, AsyncMethodCallback<AsyncClient.uploadChunk_call> resultHandler) throws TException;
+
+    public void finishFileUpload(String location, AsyncMethodCallback<AsyncClient.finishFileUpload_call> resultHandler) throws TException;
+
+    public void beginFileDownload(String file, AsyncMethodCallback<AsyncClient.beginFileDownload_call> resultHandler) throws TException;
+
+    public void downloadChunk(String id, AsyncMethodCallback<AsyncClient.downloadChunk_call> resultHandler) throws TException;
+
+    public void getClusterInfo(AsyncMethodCallback<AsyncClient.getClusterInfo_call> resultHandler) throws TException;
+
+    public void getTopologyInfo(String id, AsyncMethodCallback<AsyncClient.getTopologyInfo_call> resultHandler) throws TException;
+
+    public void getTopologyConf(String id, AsyncMethodCallback<AsyncClient.getTopologyConf_call> resultHandler) throws TException;
+
+    public void getTopology(String id, AsyncMethodCallback<AsyncClient.getTopology_call> resultHandler) throws TException;
+
+  }
+
+  public static class Client implements TServiceClient, Iface {
+    public static class Factory implements TServiceClientFactory<Client> {
+      public Factory() {}
+      public Client getClient(TProtocol prot) {
+        return new Client(prot);
+      }
+      public Client getClient(TProtocol iprot, TProtocol oprot) {
+        return new Client(iprot, oprot);
+      }
+    }
+
+    public Client(TProtocol prot)
+    {
+      this(prot, prot);
+    }
+
+    public Client(TProtocol iprot, TProtocol oprot)
+    {
+      iprot_ = iprot;
+      oprot_ = oprot;
+    }
+
+    protected TProtocol iprot_;
+    protected TProtocol oprot_;
+
+    protected int seqid_;
+
+    public TProtocol getInputProtocol()
+    {
+      return this.iprot_;
+    }
+
+    public TProtocol getOutputProtocol()
+    {
+      return this.oprot_;
+    }
+
+    public void submitTopology(String name, String uploadedJarLocation, String jsonConf, StormTopology topology) throws AlreadyAliveException, InvalidTopologyException, TException
+    {
+      send_submitTopology(name, uploadedJarLocation, jsonConf, topology);
+      recv_submitTopology();
+    }
+
+    public void send_submitTopology(String name, String uploadedJarLocation, String jsonConf, StormTopology topology) throws TException
+    {
+      oprot_.writeMessageBegin(new TMessage("submitTopology", TMessageType.CALL, ++seqid_));
+      submitTopology_args args = new submitTopology_args();
+      args.set_name(name);
+      args.set_uploadedJarLocation(uploadedJarLocation);
+      args.set_jsonConf(jsonConf);
+      args.set_topology(topology);
+      args.write(oprot_);
+      oprot_.writeMessageEnd();
+      oprot_.getTransport().flush();
+    }
+
+    public void recv_submitTopology() throws AlreadyAliveException, InvalidTopologyException, TException
+    {
+      TMessage msg = iprot_.readMessageBegin();
+      if (msg.type == TMessageType.EXCEPTION) {
+        TApplicationException x = TApplicationException.read(iprot_);
+        iprot_.readMessageEnd();
+        throw x;
+      }
+      if (msg.seqid != seqid_) {
+        throw new TApplicationException(TApplicationException.BAD_SEQUENCE_ID, "submitTopology failed: out of sequence response");
+      }
+      submitTopology_result result = new submitTopology_result();
+      result.read(iprot_);
+      iprot_.readMessageEnd();
+      if (result.e != null) {
+        throw result.e;
+      }
+      if (result.ite != null) {
+        throw result.ite;
+      }
+      return;
+    }
+
+    public void killTopology(String name) throws NotAliveException, TException
+    {
+      send_killTopology(name);
+      recv_killTopology();
+    }
+
+    public void send_killTopology(String name) throws TException
+    {
+      oprot_.writeMessageBegin(new TMessage("killTopology", TMessageType.CALL, ++seqid_));
+      killTopology_args args = new killTopology_args();
+      args.set_name(name);
+      args.write(oprot_);
+      oprot_.writeMessageEnd();
+      oprot_.getTransport().flush();
+    }
+
+    public void recv_killTopology() throws NotAliveException, TException
+    {
+      TMessage msg = iprot_.readMessageBegin();
+      if (msg.type == TMessageType.EXCEPTION) {
+        TApplicationException x = TApplicationException.read(iprot_);
+        iprot_.readMessageEnd();
+        throw x;
+      }
+      if (msg.seqid != seqid_) {
+        throw new TApplicationException(TApplicationException.BAD_SEQUENCE_ID, "killTopology failed: out of sequence response");
+      }
+      killTopology_result result = new killTopology_result();
+      result.read(iprot_);
+      iprot_.readMessageEnd();
+      if (result.e != null) {
+        throw result.e;
+      }
+      return;
+    }
+
+    public String beginFileUpload() throws TException
+    {
+      send_beginFileUpload();
+      return recv_beginFileUpload();
+    }
+
+    public void send_beginFileUpload() throws TException
+    {
+      oprot_.writeMessageBegin(new TMessage("beginFileUpload", TMessageType.CALL, ++seqid_));
+      beginFileUpload_args args = new beginFileUpload_args();
+      args.write(oprot_);
+      oprot_.writeMessageEnd();
+      oprot_.getTransport().flush();
+    }
+
+    public String recv_beginFileUpload() throws TException
+    {
+      TMessage msg = iprot_.readMessageBegin();
+      if (msg.type == TMessageType.EXCEPTION) {
+        TApplicationException x = TApplicationException.read(iprot_);
+        iprot_.readMessageEnd();
+        throw x;
+      }
+      if (msg.seqid != seqid_) {
+        throw new TApplicationException(TApplicationException.BAD_SEQUENCE_ID, "beginFileUpload failed: out of sequence response");
+      }
+      beginFileUpload_result result = new beginFileUpload_result();
+      result.read(iprot_);
+      iprot_.readMessageEnd();
+      if (result.is_set_success()) {
+        return result.success;
+      }
+      throw new TApplicationException(TApplicationException.MISSING_RESULT, "beginFileUpload failed: unknown result");
+    }
+
+    public void uploadChunk(String location, byte[] chunk) throws TException
+    {
+      send_uploadChunk(location, chunk);
+      recv_uploadChunk();
+    }
+
+    public void send_uploadChunk(String location, byte[] chunk) throws TException
+    {
+      oprot_.writeMessageBegin(new TMessage("uploadChunk", TMessageType.CALL, ++seqid_));
+      uploadChunk_args args = new uploadChunk_args();
+      args.set_location(location);
+      args.set_chunk(chunk);
+      args.write(oprot_);
+      oprot_.writeMessageEnd();
+      oprot_.getTransport().flush();
+    }
+
+    public void recv_uploadChunk() throws TException
+    {
+      TMessage msg = iprot_.readMessageBegin();
+      if (msg.type == TMessageType.EXCEPTION) {
+        TApplicationException x = TApplicationException.read(iprot_);
+        iprot_.readMessageEnd();
+        throw x;
+      }
+      if (msg.seqid != seqid_) {
+        throw new TApplicationException(TApplicationException.BAD_SEQUENCE_ID, "uploadChunk failed: out of sequence response");
+      }
+      uploadChunk_result result = new uploadChunk_result();
+      result.read(iprot_);
+      iprot_.readMessageEnd();
+      return;
+    }
+
+    public void finishFileUpload(String location) throws TException
+    {
+      send_finishFileUpload(location);
+      recv_finishFileUpload();
+    }
+
+    public void send_finishFileUpload(String location) throws TException
+    {
+      oprot_.writeMessageBegin(new TMessage("finishFileUpload", TMessageType.CALL, ++seqid_));
+      finishFileUpload_args args = new finishFileUpload_args();
+      args.set_location(location);
+      args.write(oprot_);
+      oprot_.writeMessageEnd();
+      oprot_.getTransport().flush();
+    }
+
+    public void recv_finishFileUpload() throws TException
+    {
+      TMessage msg = iprot_.readMessageBegin();
+      if (msg.type == TMessageType.EXCEPTION) {
+        TApplicationException x = TApplicationException.read(iprot_);
+        iprot_.readMessageEnd();
+        throw x;
+      }
+      if (msg.seqid != seqid_) {
+        throw new TApplicationException(TApplicationException.BAD_SEQUENCE_ID, "finishFileUpload failed: out of sequence response");
+      }
+      finishFileUpload_result result = new finishFileUpload_result();
+      result.read(iprot_);
+      iprot_.readMessageEnd();
+      return;
+    }
+
+    public String beginFileDownload(String file) throws TException
+    {
+      send_beginFileDownload(file);
+      return recv_beginFileDownload();
+    }
+
+    public void send_beginFileDownload(String file) throws TException
+    {
+      oprot_.writeMessageBegin(new TMessage("beginFileDownload", TMessageType.CALL, ++seqid_));
+      beginFileDownload_args args = new beginFileDownload_args();
+      args.set_file(file);
+      args.write(oprot_);
+      oprot_.writeMessageEnd();
+      oprot_.getTransport().flush();
+    }
+
+    public String recv_beginFileDownload() throws TException
+    {
+      TMessage msg = iprot_.readMessageBegin();
+      if (msg.type == TMessageType.EXCEPTION) {
+        TApplicationException x = TApplicationException.read(iprot_);
+        iprot_.readMessageEnd();
+        throw x;
+      }
+      if (msg.seqid != seqid_) {
+        throw new TApplicationException(TApplicationException.BAD_SEQUENCE_ID, "beginFileDownload failed: out of sequence response");
+      }
+      beginFileDownload_result result = new beginFileDownload_result();
+      result.read(iprot_);
+      iprot_.readMessageEnd();
+      if (result.is_set_success()) {
+        return result.success;
+      }
+      throw new TApplicationException(TApplicationException.MISSING_RESULT, "beginFileDownload failed: unknown result");
+    }
+
+    public byte[] downloadChunk(String id) throws TException
+    {
+      send_downloadChunk(id);
+      return recv_downloadChunk();
+    }
+
+    public void send_downloadChunk(String id) throws TException
+    {
+      oprot_.writeMessageBegin(new TMessage("downloadChunk", TMessageType.CALL, ++seqid_));
+      downloadChunk_args args = new downloadChunk_args();
+      args.set_id(id);
+      args.write(oprot_);
+      oprot_.writeMessageEnd();
+      oprot_.getTransport().flush();
+    }
+
+    public byte[] recv_downloadChunk() throws TException
+    {
+      TMessage msg = iprot_.readMessageBegin();
+      if (msg.type == TMessageType.EXCEPTION) {
+        TApplicationException x = TApplicationException.read(iprot_);
+        iprot_.readMessageEnd();
+        throw x;
+      }
+      if (msg.seqid != seqid_) {
+        throw new TApplicationException(TApplicationException.BAD_SEQUENCE_ID, "downloadChunk failed: out of sequence response");
+      }
+      downloadChunk_result result = new downloadChunk_result();
+      result.read(iprot_);
+      iprot_.readMessageEnd();
+      if (result.is_set_success()) {
+        return result.success;
+      }
+      throw new TApplicationException(TApplicationException.MISSING_RESULT, "downloadChunk failed: unknown result");
+    }
+
+    public ClusterSummary getClusterInfo() throws TException
+    {
+      send_getClusterInfo();
+      return recv_getClusterInfo();
+    }
+
+    public void send_getClusterInfo() throws TException
+    {
+      oprot_.writeMessageBegin(new TMessage("getClusterInfo", TMessageType.CALL, ++seqid_));
+      getClusterInfo_args args = new getClusterInfo_args();
+      args.write(oprot_);
+      oprot_.writeMessageEnd();
+      oprot_.getTransport().flush();
+    }
+
+    public ClusterSummary recv_getClusterInfo() throws TException
+    {
+      TMessage msg = iprot_.readMessageBegin();
+      if (msg.type == TMessageType.EXCEPTION) {
+        TApplicationException x = TApplicationException.read(iprot_);
+        iprot_.readMessageEnd();
+        throw x;
+      }
+      if (msg.seqid != seqid_) {
+        throw new TApplicationException(TApplicationException.BAD_SEQUENCE_ID, "getClusterInfo failed: out of sequence response");
+      }
+      getClusterInfo_result result = new getClusterInfo_result();
+      result.read(iprot_);
+      iprot_.readMessageEnd();
+      if (result.is_set_success()) {
+        return result.success;
+      }
+      throw new TApplicationException(TApplicationException.MISSING_RESULT, "getClusterInfo failed: unknown result");
+    }
+
+    public TopologyInfo getTopologyInfo(String id) throws NotAliveException, TException
+    {
+      send_getTopologyInfo(id);
+      return recv_getTopologyInfo();
+    }
+
+    public void send_getTopologyInfo(String id) throws TException
+    {
+      oprot_.writeMessageBegin(new TMessage("getTopologyInfo", TMessageType.CALL, ++seqid_));
+      getTopologyInfo_args args = new getTopologyInfo_args();
+      args.set_id(id);
+      args.write(oprot_);
+      oprot_.writeMessageEnd();
+      oprot_.getTransport().flush();
+    }
+
+    public TopologyInfo recv_getTopologyInfo() throws NotAliveException, TException
+    {
+      TMessage msg = iprot_.readMessageBegin();
+      if (msg.type == TMessageType.EXCEPTION) {
+        TApplicationException x = TApplicationException.read(iprot_);
+        iprot_.readMessageEnd();
+        throw x;
+      }
+      if (msg.seqid != seqid_) {
+        throw new TApplicationException(TApplicationException.BAD_SEQUENCE_ID, "getTopologyInfo failed: out of sequence response");
+      }
+      getTopologyInfo_result result = new getTopologyInfo_result();
+      result.read(iprot_);
+      iprot_.readMessageEnd();
+      if (result.is_set_success()) {
+        return result.success;
+      }
+      if (result.e != null) {
+        throw result.e;
+      }
+      throw new TApplicationException(TApplicationException.MISSING_RESULT, "getTopologyInfo failed: unknown result");
+    }
+
+    public String getTopologyConf(String id) throws NotAliveException, TException
+    {
+      send_getTopologyConf(id);
+      return recv_getTopologyConf();
+    }
+
+    public void send_getTopologyConf(String id) throws TException
+    {
+      oprot_.writeMessageBegin(new TMessage("getTopologyConf", TMessageType.CALL, ++seqid_));
+      getTopologyConf_args args = new getTopologyConf_args();
+      args.set_id(id);
+      args.write(oprot_);
+      oprot_.writeMessageEnd();
+      oprot_.getTransport().flush();
+    }
+
+    public String recv_getTopologyConf() throws NotAliveException, TException
+    {
+      TMessage msg = iprot_.readMessageBegin();
+      if (msg.type == TMessageType.EXCEPTION) {
+        TApplicationException x = TApplicationException.read(iprot_);
+        iprot_.readMessageEnd();
+        throw x;
+      }
+      if (msg.seqid != seqid_) {
+        throw new TApplicationException(TApplicationException.BAD_SEQUENCE_ID, "getTopologyConf failed: out of sequence response");
+      }
+      getTopologyConf_result result = new getTopologyConf_result();
+      result.read(iprot_);
+      iprot_.readMessageEnd();
+      if (result.is_set_success()) {
+        return result.success;
+      }
+      if (result.e != null) {
+        throw result.e;
+      }
+      throw new TApplicationException(TApplicationException.MISSING_RESULT, "getTopologyConf failed: unknown result");
+    }
+
+    public StormTopology getTopology(String id) throws NotAliveException, TException
+    {
+      send_getTopology(id);
+      return recv_getTopology();
+    }
+
+    public void send_getTopology(String id) throws TException
+    {
+      oprot_.writeMessageBegin(new TMessage("getTopology", TMessageType.CALL, ++seqid_));
+      getTopology_args args = new getTopology_args();
+      args.set_id(id);
+      args.write(oprot_);
+      oprot_.writeMessageEnd();
+      oprot_.getTransport().flush();
+    }
+
+    public StormTopology recv_getTopology() throws NotAliveException, TException
+    {
+      TMessage msg = iprot_.readMessageBegin();
+      if (msg.type == TMessageType.EXCEPTION) {
+        TApplicationException x = TApplicationException.read(iprot_);
+        iprot_.readMessageEnd();
+        throw x;
+      }
+      if (msg.seqid != seqid_) {
+        throw new TApplicationException(TApplicationException.BAD_SEQUENCE_ID, "getTopology failed: out of sequence response");
+      }
+      getTopology_result result = new getTopology_result();
+      result.read(iprot_);
+      iprot_.readMessageEnd();
+      if (result.is_set_success()) {
+        return result.success;
+      }
+      if (result.e != null) {
+        throw result.e;
+      }
+      throw new TApplicationException(TApplicationException.MISSING_RESULT, "getTopology failed: unknown result");
+    }
+
+  }
+  public static class AsyncClient extends TAsyncClient implements AsyncIface {
+    public static class Factory implements TAsyncClientFactory<AsyncClient> {
+      private TAsyncClientManager clientManager;
+      private TProtocolFactory protocolFactory;
+      public Factory(TAsyncClientManager clientManager, TProtocolFactory protocolFactory) {
+        this.clientManager = clientManager;
+        this.protocolFactory = protocolFactory;
+      }
+      public AsyncClient getAsyncClient(TNonblockingTransport transport) {
+        return new AsyncClient(protocolFactory, clientManager, transport);
+      }
+    }
+
+    public AsyncClient(TProtocolFactory protocolFactory, TAsyncClientManager clientManager, TNonblockingTransport transport) {
+      super(protocolFactory, clientManager, transport);
+    }
+
+    public void submitTopology(String name, String uploadedJarLocation, String jsonConf, StormTopology topology, AsyncMethodCallback<submitTopology_call> resultHandler) throws TException {
+      checkReady();
+      submitTopology_call method_call = new submitTopology_call(name, uploadedJarLocation, jsonConf, topology, resultHandler, this, protocolFactory, transport);
+      manager.call(method_call);
+    }
+
+    public static class submitTopology_call extends TAsyncMethodCall {
+      private String name;
+      private String uploadedJarLocation;
+      private String jsonConf;
+      private StormTopology topology;
+      public submitTopology_call(String name, String uploadedJarLocation, String jsonConf, StormTopology topology, AsyncMethodCallback<submitTopology_call> resultHandler, TAsyncClient client, TProtocolFactory protocolFactory, TNonblockingTransport transport) throws TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.name = name;
+        this.uploadedJarLocation = uploadedJarLocation;
+        this.jsonConf = jsonConf;
+        this.topology = topology;
+      }
+
+      public void write_args(TProtocol prot) throws TException {
+        prot.writeMessageBegin(new TMessage("submitTopology", TMessageType.CALL, 0));
+        submitTopology_args args = new submitTopology_args();
+        args.set_name(name);
+        args.set_uploadedJarLocation(uploadedJarLocation);
+        args.set_jsonConf(jsonConf);
+        args.set_topology(topology);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public void getResult() throws AlreadyAliveException, InvalidTopologyException, TException {
+        if (getState() != State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        TMemoryInputTransport memoryTransport = new TMemoryInputTransport(getFrameBuffer().array());
+        TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        (new Client(prot)).recv_submitTopology();
+      }
+    }
+
+    public void killTopology(String name, AsyncMethodCallback<killTopology_call> resultHandler) throws TException {
+      checkReady();
+      killTopology_call method_call = new killTopology_call(name, resultHandler, this, protocolFactory, transport);
+      manager.call(method_call);
+    }
+
+    public static class killTopology_call extends TAsyncMethodCall {
+      private String name;
+      public killTopology_call(String name, AsyncMethodCallback<killTopology_call> resultHandler, TAsyncClient client, TProtocolFactory protocolFactory, TNonblockingTransport transport) throws TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.name = name;
+      }
+
+      public void write_args(TProtocol prot) throws TException {
+        prot.writeMessageBegin(new TMessage("killTopology", TMessageType.CALL, 0));
+        killTopology_args args = new killTopology_args();
+        args.set_name(name);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public void getResult() throws NotAliveException, TException {
+        if (getState() != State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        TMemoryInputTransport memoryTransport = new TMemoryInputTransport(getFrameBuffer().array());
+        TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        (new Client(prot)).recv_killTopology();
+      }
+    }
+
+    public void beginFileUpload(AsyncMethodCallback<beginFileUpload_call> resultHandler) throws TException {
+      checkReady();
+      beginFileUpload_call method_call = new beginFileUpload_call(resultHandler, this, protocolFactory, transport);
+      manager.call(method_call);
+    }
+
+    public static class beginFileUpload_call extends TAsyncMethodCall {
+      public beginFileUpload_call(AsyncMethodCallback<beginFileUpload_call> resultHandler, TAsyncClient client, TProtocolFactory protocolFactory, TNonblockingTransport transport) throws TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+      }
+
+      public void write_args(TProtocol prot) throws TException {
+        prot.writeMessageBegin(new TMessage("beginFileUpload", TMessageType.CALL, 0));
+        beginFileUpload_args args = new beginFileUpload_args();
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public String getResult() throws TException {
+        if (getState() != State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        TMemoryInputTransport memoryTransport = new TMemoryInputTransport(getFrameBuffer().array());
+        TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_beginFileUpload();
+      }
+    }
+
+    public void uploadChunk(String location, byte[] chunk, AsyncMethodCallback<uploadChunk_call> resultHandler) throws TException {
+      checkReady();
+      uploadChunk_call method_call = new uploadChunk_call(location, chunk, resultHandler, this, protocolFactory, transport);
+      manager.call(method_call);
+    }
+
+    public static class uploadChunk_call extends TAsyncMethodCall {
+      private String location;
+      private byte[] chunk;
+      public uploadChunk_call(String location, byte[] chunk, AsyncMethodCallback<uploadChunk_call> resultHandler, TAsyncClient client, TProtocolFactory protocolFactory, TNonblockingTransport transport) throws TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.location = location;
+        this.chunk = chunk;
+      }
+
+      public void write_args(TProtocol prot) throws TException {
+        prot.writeMessageBegin(new TMessage("uploadChunk", TMessageType.CALL, 0));
+        uploadChunk_args args = new uploadChunk_args();
+        args.set_location(location);
+        args.set_chunk(chunk);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public void getResult() throws TException {
+        if (getState() != State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        TMemoryInputTransport memoryTransport = new TMemoryInputTransport(getFrameBuffer().array());
+        TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        (new Client(prot)).recv_uploadChunk();
+      }
+    }
+
+    public void finishFileUpload(String location, AsyncMethodCallback<finishFileUpload_call> resultHandler) throws TException {
+      checkReady();
+      finishFileUpload_call method_call = new finishFileUpload_call(location, resultHandler, this, protocolFactory, transport);
+      manager.call(method_call);
+    }
+
+    public static class finishFileUpload_call extends TAsyncMethodCall {
+      private String location;
+      public finishFileUpload_call(String location, AsyncMethodCallback<finishFileUpload_call> resultHandler, TAsyncClient client, TProtocolFactory protocolFactory, TNonblockingTransport transport) throws TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.location = location;
+      }
+
+      public void write_args(TProtocol prot) throws TException {
+        prot.writeMessageBegin(new TMessage("finishFileUpload", TMessageType.CALL, 0));
+        finishFileUpload_args args = new finishFileUpload_args();
+        args.set_location(location);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public void getResult() throws TException {
+        if (getState() != State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        TMemoryInputTransport memoryTransport = new TMemoryInputTransport(getFrameBuffer().array());
+        TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        (new Client(prot)).recv_finishFileUpload();
+      }
+    }
+
+    public void beginFileDownload(String file, AsyncMethodCallback<beginFileDownload_call> resultHandler) throws TException {
+      checkReady();
+      beginFileDownload_call method_call = new beginFileDownload_call(file, resultHandler, this, protocolFactory, transport);
+      manager.call(method_call);
+    }
+
+    public static class beginFileDownload_call extends TAsyncMethodCall {
+      private String file;
+      public beginFileDownload_call(String file, AsyncMethodCallback<beginFileDownload_call> resultHandler, TAsyncClient client, TProtocolFactory protocolFactory, TNonblockingTransport transport) throws TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.file = file;
+      }
+
+      public void write_args(TProtocol prot) throws TException {
+        prot.writeMessageBegin(new TMessage("beginFileDownload", TMessageType.CALL, 0));
+        beginFileDownload_args args = new beginFileDownload_args();
+        args.set_file(file);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public String getResult() throws TException {
+        if (getState() != State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        TMemoryInputTransport memoryTransport = new TMemoryInputTransport(getFrameBuffer().array());
+        TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_beginFileDownload();
+      }
+    }
+
+    public void downloadChunk(String id, AsyncMethodCallback<downloadChunk_call> resultHandler) throws TException {
+      checkReady();
+      downloadChunk_call method_call = new downloadChunk_call(id, resultHandler, this, protocolFactory, transport);
+      manager.call(method_call);
+    }
+
+    public static class downloadChunk_call extends TAsyncMethodCall {
+      private String id;
+      public downloadChunk_call(String id, AsyncMethodCallback<downloadChunk_call> resultHandler, TAsyncClient client, TProtocolFactory protocolFactory, TNonblockingTransport transport) throws TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.id = id;
+      }
+
+      public void write_args(TProtocol prot) throws TException {
+        prot.writeMessageBegin(new TMessage("downloadChunk", TMessageType.CALL, 0));
+        downloadChunk_args args = new downloadChunk_args();
+        args.set_id(id);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public byte[] getResult() throws TException {
+        if (getState() != State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        TMemoryInputTransport memoryTransport = new TMemoryInputTransport(getFrameBuffer().array());
+        TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_downloadChunk();
+      }
+    }
+
+    public void getClusterInfo(AsyncMethodCallback<getClusterInfo_call> resultHandler) throws TException {
+      checkReady();
+      getClusterInfo_call method_call = new getClusterInfo_call(resultHandler, this, protocolFactory, transport);
+      manager.call(method_call);
+    }
+
+    public static class getClusterInfo_call extends TAsyncMethodCall {
+      public getClusterInfo_call(AsyncMethodCallback<getClusterInfo_call> resultHandler, TAsyncClient client, TProtocolFactory protocolFactory, TNonblockingTransport transport) throws TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+      }
+
+      public void write_args(TProtocol prot) throws TException {
+        prot.writeMessageBegin(new TMessage("getClusterInfo", TMessageType.CALL, 0));
+        getClusterInfo_args args = new getClusterInfo_args();
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public ClusterSummary getResult() throws TException {
+        if (getState() != State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        TMemoryInputTransport memoryTransport = new TMemoryInputTransport(getFrameBuffer().array());
+        TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_getClusterInfo();
+      }
+    }
+
+    public void getTopologyInfo(String id, AsyncMethodCallback<getTopologyInfo_call> resultHandler) throws TException {
+      checkReady();
+      getTopologyInfo_call method_call = new getTopologyInfo_call(id, resultHandler, this, protocolFactory, transport);
+      manager.call(method_call);
+    }
+
+    public static class getTopologyInfo_call extends TAsyncMethodCall {
+      private String id;
+      public getTopologyInfo_call(String id, AsyncMethodCallback<getTopologyInfo_call> resultHandler, TAsyncClient client, TProtocolFactory protocolFactory, TNonblockingTransport transport) throws TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.id = id;
+      }
+
+      public void write_args(TProtocol prot) throws TException {
+        prot.writeMessageBegin(new TMessage("getTopologyInfo", TMessageType.CALL, 0));
+        getTopologyInfo_args args = new getTopologyInfo_args();
+        args.set_id(id);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public TopologyInfo getResult() throws NotAliveException, TException {
+        if (getState() != State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        TMemoryInputTransport memoryTransport = new TMemoryInputTransport(getFrameBuffer().array());
+        TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_getTopologyInfo();
+      }
+    }
+
+    public void getTopologyConf(String id, AsyncMethodCallback<getTopologyConf_call> resultHandler) throws TException {
+      checkReady();
+      getTopologyConf_call method_call = new getTopologyConf_call(id, resultHandler, this, protocolFactory, transport);
+      manager.call(method_call);
+    }
+
+    public static class getTopologyConf_call extends TAsyncMethodCall {
+      private String id;
+      public getTopologyConf_call(String id, AsyncMethodCallback<getTopologyConf_call> resultHandler, TAsyncClient client, TProtocolFactory protocolFactory, TNonblockingTransport transport) throws TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.id = id;
+      }
+
+      public void write_args(TProtocol prot) throws TException {
+        prot.writeMessageBegin(new TMessage("getTopologyConf", TMessageType.CALL, 0));
+        getTopologyConf_args args = new getTopologyConf_args();
+        args.set_id(id);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public String getResult() throws NotAliveException, TException {
+        if (getState() != State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        TMemoryInputTransport memoryTransport = new TMemoryInputTransport(getFrameBuffer().array());
+        TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_getTopologyConf();
+      }
+    }
+
+    public void getTopology(String id, AsyncMethodCallback<getTopology_call> resultHandler) throws TException {
+      checkReady();
+      getTopology_call method_call = new getTopology_call(id, resultHandler, this, protocolFactory, transport);
+      manager.call(method_call);
+    }
+
+    public static class getTopology_call extends TAsyncMethodCall {
+      private String id;
+      public getTopology_call(String id, AsyncMethodCallback<getTopology_call> resultHandler, TAsyncClient client, TProtocolFactory protocolFactory, TNonblockingTransport transport) throws TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.id = id;
+      }
+
+      public void write_args(TProtocol prot) throws TException {
+        prot.writeMessageBegin(new TMessage("getTopology", TMessageType.CALL, 0));
+        getTopology_args args = new getTopology_args();
+        args.set_id(id);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public StormTopology getResult() throws NotAliveException, TException {
+        if (getState() != State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        TMemoryInputTransport memoryTransport = new TMemoryInputTransport(getFrameBuffer().array());
+        TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_getTopology();
+      }
+    }
+
+  }
+
+  public static class Processor implements TProcessor {
+    private static final Logger LOGGER = LoggerFactory.getLogger(Processor.class.getName());
+    public Processor(Iface iface)
+    {
+      iface_ = iface;
+      processMap_.put("submitTopology", new submitTopology());
+      processMap_.put("killTopology", new killTopology());
+      processMap_.put("beginFileUpload", new beginFileUpload());
+      processMap_.put("uploadChunk", new uploadChunk());
+      processMap_.put("finishFileUpload", new finishFileUpload());
+      processMap_.put("beginFileDownload", new beginFileDownload());
+      processMap_.put("downloadChunk", new downloadChunk());
+      processMap_.put("getClusterInfo", new getClusterInfo());
+      processMap_.put("getTopologyInfo", new getTopologyInfo());
+      processMap_.put("getTopologyConf", new getTopologyConf());
+      processMap_.put("getTopology", new getTopology());
+    }
+
+    protected static interface ProcessFunction {
+      public void process(int seqid, TProtocol iprot, TProtocol oprot) throws TException;
+    }
+
+    private Iface iface_;
+    protected final HashMap<String,ProcessFunction> processMap_ = new HashMap<String,ProcessFunction>();
+
+    public boolean process(TProtocol iprot, TProtocol oprot) throws TException
+    {
+      TMessage msg = iprot.readMessageBegin();
+      ProcessFunction fn = processMap_.get(msg.name);
+      if (fn == null) {
+        TProtocolUtil.skip(iprot, TType.STRUCT);
+        iprot.readMessageEnd();
+        TApplicationException x = new TApplicationException(TApplicationException.UNKNOWN_METHOD, "Invalid method name: '"+msg.name+"'");
+        oprot.writeMessageBegin(new TMessage(msg.name, TMessageType.EXCEPTION, msg.seqid));
+        x.write(oprot);
+        oprot.writeMessageEnd();
+        oprot.getTransport().flush();
+        return true;
+      }
+      fn.process(msg.seqid, iprot, oprot);
+      return true;
+    }
+
+    private class submitTopology implements ProcessFunction {
+      public void process(int seqid, TProtocol iprot, TProtocol oprot) throws TException
+      {
+        submitTopology_args args = new submitTopology_args();
+        try {
+          args.read(iprot);
+        } catch (TProtocolException e) {
+          iprot.readMessageEnd();
+          TApplicationException x = new TApplicationException(TApplicationException.PROTOCOL_ERROR, e.getMessage());
+          oprot.writeMessageBegin(new TMessage("submitTopology", TMessageType.EXCEPTION, seqid));
+          x.write(oprot);
+          oprot.writeMessageEnd();
+          oprot.getTransport().flush();
+          return;
+        }
+        iprot.readMessageEnd();
+        submitTopology_result result = new submitTopology_result();
+        try {
+          iface_.submitTopology(args.name, args.uploadedJarLocation, args.jsonConf, args.topology);
+        } catch (AlreadyAliveException e) {
+          result.e = e;
+        } catch (InvalidTopologyException ite) {
+          result.ite = ite;
+        } catch (Throwable th) {
+          LOGGER.error("Internal error processing submitTopology", th);
+          TApplicationException x = new TApplicationException(TApplicationException.INTERNAL_ERROR, "Internal error processing submitTopology");
+          oprot.writeMessageBegin(new TMessage("submitTopology", TMessageType.EXCEPTION, seqid));
+          x.write(oprot);
+          oprot.writeMessageEnd();
+          oprot.getTransport().flush();
+          return;
+        }
+        oprot.writeMessageBegin(new TMessage("submitTopology", TMessageType.REPLY, seqid));
+        result.write(oprot);
+        oprot.writeMessageEnd();
+        oprot.getTransport().flush();
+      }
+
+    }
+
+    private class killTopology implements ProcessFunction {
+      public void process(int seqid, TProtocol iprot, TProtocol oprot) throws TException
+      {
+        killTopology_args args = new killTopology_args();
+        try {
+          args.read(iprot);
+        } catch (TProtocolException e) {
+          iprot.readMessageEnd();
+          TApplicationException x = new TApplicationException(TApplicationException.PROTOCOL_ERROR, e.getMessage());
+          oprot.writeMessageBegin(new TMessage("killTopology", TMessageType.EXCEPTION, seqid));
+          x.write(oprot);
+          oprot.writeMessageEnd();
+          oprot.getTransport().flush();
+          return;
+        }
+        iprot.readMessageEnd();
+        killTopology_result result = new killTopology_result();
+        try {
+          iface_.killTopology(args.name);
+        } catch (NotAliveException e) {
+          result.e = e;
+        } catch (Throwable th) {
+          LOGGER.error("Internal error processing killTopology", th);
+          TApplicationException x = new TApplicationException(TApplicationException.INTERNAL_ERROR, "Internal error processing killTopology");
+          oprot.writeMessageBegin(new TMessage("killTopology", TMessageType.EXCEPTION, seqid));
+          x.write(oprot);
+          oprot.writeMessageEnd();
+          oprot.getTransport().flush();
+          return;
+        }
+        oprot.writeMessageBegin(new TMessage("killTopology", TMessageType.REPLY, seqid));
+        result.write(oprot);
+        oprot.writeMessageEnd();
+        oprot.getTransport().flush();
+      }
+
+    }
+
+    private class beginFileUpload implements ProcessFunction {
+      public void process(int seqid, TProtocol iprot, TProtocol oprot) throws TException
+      {
+        beginFileUpload_args args = new beginFileUpload_args();
+        try {
+          args.read(iprot);
+        } catch (TProtocolException e) {
+          iprot.readMessageEnd();
+          TApplicationException x = new TApplicationException(TApplicationException.PROTOCOL_ERROR, e.getMessage());
+          oprot.writeMessageBegin(new TMessage("beginFileUpload", TMessageType.EXCEPTION, seqid));
+          x.write(oprot);
+          oprot.writeMessageEnd();
+          oprot.getTransport().flush();
+          return;
+        }
+        iprot.readMessageEnd();
+        beginFileUpload_result result = new beginFileUpload_result();
+        result.success = iface_.beginFileUpload();
+        oprot.writeMessageBegin(new TMessage("beginFileUpload", TMessageType.REPLY, seqid));
+        result.write(oprot);
+        oprot.writeMessageEnd();
+        oprot.getTransport().flush();
+      }
+
+    }
+
+    private class uploadChunk implements ProcessFunction {
+      public void process(int seqid, TProtocol iprot, TProtocol oprot) throws TException
+      {
+        uploadChunk_args args = new uploadChunk_args();
+        try {
+          args.read(iprot);
+        } catch (TProtocolException e) {
+          iprot.readMessageEnd();
+          TApplicationException x = new TApplicationException(TApplicationException.PROTOCOL_ERROR, e.getMessage());
+          oprot.writeMessageBegin(new TMessage("uploadChunk", TMessageType.EXCEPTION, seqid));
+          x.write(oprot);
+          oprot.writeMessageEnd();
+          oprot.getTransport().flush();
+          return;
+        }
+        iprot.readMessageEnd();
+        uploadChunk_result result = new uploadChunk_result();
+        iface_.uploadChunk(args.location, args.chunk);
+        oprot.writeMessageBegin(new TMessage("uploadChunk", TMessageType.REPLY, seqid));
+        result.write(oprot);
+        oprot.writeMessageEnd();
+        oprot.getTransport().flush();
+      }
+
+    }
+
+    private class finishFileUpload implements ProcessFunction {
+      public void process(int seqid, TProtocol iprot, TProtocol oprot) throws TException
+      {
+        finishFileUpload_args args = new finishFileUpload_args();
+        try {
+          args.read(iprot);
+        } catch (TProtocolException e) {
+          iprot.readMessageEnd();
+          TApplicationException x = new TApplicationException(TApplicationException.PROTOCOL_ERROR, e.getMessage());
+          oprot.writeMessageBegin(new TMessage("finishFileUpload", TMessageType.EXCEPTION, seqid));
+          x.write(oprot);
+          oprot.writeMessageEnd();
+          oprot.getTransport().flush();
+          return;
+        }
+        iprot.readMessageEnd();
+        finishFileUpload_result result = new finishFileUpload_result();
+        iface_.finishFileUpload(args.location);
+        oprot.writeMessageBegin(new TMessage("finishFileUpload", TMessageType.REPLY, seqid));
+        result.write(oprot);
+        oprot.writeMessageEnd();
+        oprot.getTransport().flush();
+      }
+
+    }
+
+    private class beginFileDownload implements ProcessFunction {
+      public void process(int seqid, TProtocol iprot, TProtocol oprot) throws TException
+      {
+        beginFileDownload_args args = new beginFileDownload_args();
+        try {
+          args.read(iprot);
+        } catch (TProtocolException e) {
+          iprot.readMessageEnd();
+          TApplicationException x = new TApplicationException(TApplicationException.PROTOCOL_ERROR, e.getMessage());
+          oprot.writeMessageBegin(new TMessage("beginFileDownload", TMessageType.EXCEPTION, seqid));
+          x.write(oprot);
+          oprot.writeMessageEnd();
+          oprot.getTransport().flush();
+          return;
+        }
+        iprot.readMessageEnd();
+        beginFileDownload_result result = new beginFileDownload_result();
+        result.success = iface_.beginFileDownload(args.file);
+        oprot.writeMessageBegin(new TMessage("beginFileDownload", TMessageType.REPLY, seqid));
+        result.write(oprot);
+        oprot.writeMessageEnd();
+        oprot.getTransport().flush();
+      }
+
+    }
+
+    private class downloadChunk implements ProcessFunction {
+      public void process(int seqid, TProtocol iprot, TProtocol oprot) throws TException
+      {
+        downloadChunk_args args = new downloadChunk_args();
+        try {
+          args.read(iprot);
+        } catch (TProtocolException e) {
+          iprot.readMessageEnd();
+          TApplicationException x = new TApplicationException(TApplicationException.PROTOCOL_ERROR, e.getMessage());
+          oprot.writeMessageBegin(new TMessage("downloadChunk", TMessageType.EXCEPTION, seqid));
+          x.write(oprot);
+          oprot.writeMessageEnd();
+          oprot.getTransport().flush();
+          return;
+        }
+        iprot.readMessageEnd();
+        downloadChunk_result result = new downloadChunk_result();
+        result.success = iface_.downloadChunk(args.id);
+        oprot.writeMessageBegin(new TMessage("downloadChunk", TMessageType.REPLY, seqid));
+        result.write(oprot);
+        oprot.writeMessageEnd();
+        oprot.getTransport().flush();
+      }
+
+    }
+
+    private class getClusterInfo implements ProcessFunction {
+      public void process(int seqid, TProtocol iprot, TProtocol oprot) throws TException
+      {
+        getClusterInfo_args args = new getClusterInfo_args();
+        try {
+          args.read(iprot);
+        } catch (TProtocolException e) {
+          iprot.readMessageEnd();
+          TApplicationException x = new TApplicationException(TApplicationException.PROTOCOL_ERROR, e.getMessage());
+          oprot.writeMessageBegin(new TMessage("getClusterInfo", TMessageType.EXCEPTION, seqid));
+          x.write(oprot);
+          oprot.writeMessageEnd();
+          oprot.getTransport().flush();
+          return;
+        }
+        iprot.readMessageEnd();
+        getClusterInfo_result result = new getClusterInfo_result();
+        result.success = iface_.getClusterInfo();
+        oprot.writeMessageBegin(new TMessage("getClusterInfo", TMessageType.REPLY, seqid));
+        result.write(oprot);
+        oprot.writeMessageEnd();
+        oprot.getTransport().flush();
+      }
+
+    }
+
+    private class getTopologyInfo implements ProcessFunction {
+      public void process(int seqid, TProtocol iprot, TProtocol oprot) throws TException
+      {
+        getTopologyInfo_args args = new getTopologyInfo_args();
+        try {
+          args.read(iprot);
+        } catch (TProtocolException e) {
+          iprot.readMessageEnd();
+          TApplicationException x = new TApplicationException(TApplicationException.PROTOCOL_ERROR, e.getMessage());
+          oprot.writeMessageBegin(new TMessage("getTopologyInfo", TMessageType.EXCEPTION, seqid));
+          x.write(oprot);
+          oprot.writeMessageEnd();
+          oprot.getTransport().flush();
+          return;
+        }
+        iprot.readMessageEnd();
+        getTopologyInfo_result result = new getTopologyInfo_result();
+        try {
+          result.success = iface_.getTopologyInfo(args.id);
+        } catch (NotAliveException e) {
+          result.e = e;
+        } catch (Throwable th) {
+          LOGGER.error("Internal error processing getTopologyInfo", th);
+          TApplicationException x = new TApplicationException(TApplicationException.INTERNAL_ERROR, "Internal error processing getTopologyInfo");
+          oprot.writeMessageBegin(new TMessage("getTopologyInfo", TMessageType.EXCEPTION, seqid));
+          x.write(oprot);
+          oprot.writeMessageEnd();
+          oprot.getTransport().flush();
+          return;
+        }
+        oprot.writeMessageBegin(new TMessage("getTopologyInfo", TMessageType.REPLY, seqid));
+        result.write(oprot);
+        oprot.writeMessageEnd();
+        oprot.getTransport().flush();
+      }
+
+    }
+
+    private class getTopologyConf implements ProcessFunction {
+      public void process(int seqid, TProtocol iprot, TProtocol oprot) throws TException
+      {
+        getTopologyConf_args args = new getTopologyConf_args();
+        try {
+          args.read(iprot);
+        } catch (TProtocolException e) {
+          iprot.readMessageEnd();
+          TApplicationException x = new TApplicationException(TApplicationException.PROTOCOL_ERROR, e.getMessage());
+          oprot.writeMessageBegin(new TMessage("getTopologyConf", TMessageType.EXCEPTION, seqid));
+          x.write(oprot);
+          oprot.writeMessageEnd();
+          oprot.getTransport().flush();
+          return;
+        }
+        iprot.readMessageEnd();
+        getTopologyConf_result result = new getTopologyConf_result();
+        try {
+          result.success = iface_.getTopologyConf(args.id);
+        } catch (NotAliveException e) {
+          result.e = e;
+        } catch (Throwable th) {
+          LOGGER.error("Internal error processing getTopologyConf", th);
+          TApplicationException x = new TApplicationException(TApplicationException.INTERNAL_ERROR, "Internal error processing getTopologyConf");
+          oprot.writeMessageBegin(new TMessage("getTopologyConf", TMessageType.EXCEPTION, seqid));
+          x.write(oprot);
+          oprot.writeMessageEnd();
+          oprot.getTransport().flush();
+          return;
+        }
+        oprot.writeMessageBegin(new TMessage("getTopologyConf", TMessageType.REPLY, seqid));
+        result.write(oprot);
+        oprot.writeMessageEnd();
+        oprot.getTransport().flush();
+      }
+
+    }
+
+    private class getTopology implements ProcessFunction {
+      public void process(int seqid, TProtocol iprot, TProtocol oprot) throws TException
+      {
+        getTopology_args args = new getTopology_args();
+        try {
+          args.read(iprot);
+        } catch (TProtocolException e) {
+          iprot.readMessageEnd();
+          TApplicationException x = new TApplicationException(TApplicationException.PROTOCOL_ERROR, e.getMessage());
+          oprot.writeMessageBegin(new TMessage("getTopology", TMessageType.EXCEPTION, seqid));
+          x.write(oprot);
+          oprot.writeMessageEnd();
+          oprot.getTransport().flush();
+          return;
+        }
+        iprot.readMessageEnd();
+        getTopology_result result = new getTopology_result();
+        try {
+          result.success = iface_.getTopology(args.id);
+        } catch (NotAliveException e) {
+          result.e = e;
+        } catch (Throwable th) {
+          LOGGER.error("Internal error processing getTopology", th);
+          TApplicationException x = new TApplicationException(TApplicationException.INTERNAL_ERROR, "Internal error processing getTopology");
+          oprot.writeMessageBegin(new TMessage("getTopology", TMessageType.EXCEPTION, seqid));
+          x.write(oprot);
+          oprot.writeMessageEnd();
+          oprot.getTransport().flush();
+          return;
+        }
+        oprot.writeMessageBegin(new TMessage("getTopology", TMessageType.REPLY, seqid));
+        result.write(oprot);
+        oprot.writeMessageEnd();
+        oprot.getTransport().flush();
+      }
+
+    }
+
+  }
+
+  public static class submitTopology_args implements TBase<submitTopology_args, submitTopology_args._Fields>, java.io.Serializable, Cloneable   {
+    private static final TStruct STRUCT_DESC = new TStruct("submitTopology_args");
+
+    private static final TField NAME_FIELD_DESC = new TField("name", TType.STRING, (short)1);
+    private static final TField UPLOADED_JAR_LOCATION_FIELD_DESC = new TField("uploadedJarLocation", TType.STRING, (short)2);
+    private static final TField JSON_CONF_FIELD_DESC = new TField("jsonConf", TType.STRING, (short)3);
+    private static final TField TOPOLOGY_FIELD_DESC = new TField("topology", TType.STRUCT, (short)4);
+
+    private String name;
+    private String uploadedJarLocation;
+    private String jsonConf;
+    private StormTopology topology;
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements TFieldIdEnum {
+      NAME((short)1, "name"),
+      UPLOADED_JAR_LOCATION((short)2, "uploadedJarLocation"),
+      JSON_CONF((short)3, "jsonConf"),
+      TOPOLOGY((short)4, "topology");
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 1: // NAME
+            return NAME;
+          case 2: // UPLOADED_JAR_LOCATION
+            return UPLOADED_JAR_LOCATION;
+          case 3: // JSON_CONF
+            return JSON_CONF;
+          case 4: // TOPOLOGY
+            return TOPOLOGY;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+
+    public static final Map<_Fields, FieldMetaData> metaDataMap;
+    static {
+      Map<_Fields, FieldMetaData> tmpMap = new EnumMap<_Fields, FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.NAME, new FieldMetaData("name", TFieldRequirementType.DEFAULT, 
+          new FieldValueMetaData(TType.STRING)));
+      tmpMap.put(_Fields.UPLOADED_JAR_LOCATION, new FieldMetaData("uploadedJarLocation", TFieldRequirementType.DEFAULT, 
+          new FieldValueMetaData(TType.STRING)));
+      tmpMap.put(_Fields.JSON_CONF, new FieldMetaData("jsonConf", TFieldRequirementType.DEFAULT, 
+          new FieldValueMetaData(TType.STRING)));
+      tmpMap.put(_Fields.TOPOLOGY, new FieldMetaData("topology", TFieldRequirementType.DEFAULT, 
+          new StructMetaData(TType.STRUCT, StormTopology.class)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      FieldMetaData.addStructMetaDataMap(submitTopology_args.class, metaDataMap);
+    }
+
+    public submitTopology_args() {
+    }
+
+    public submitTopology_args(
+      String name,
+      String uploadedJarLocation,
+      String jsonConf,
+      StormTopology topology)
+    {
+      this();
+      this.name = name;
+      this.uploadedJarLocation = uploadedJarLocation;
+      this.jsonConf = jsonConf;
+      this.topology = topology;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public submitTopology_args(submitTopology_args other) {
+      if (other.is_set_name()) {
+        this.name = other.name;
+      }
+      if (other.is_set_uploadedJarLocation()) {
+        this.uploadedJarLocation = other.uploadedJarLocation;
+      }
+      if (other.is_set_jsonConf()) {
+        this.jsonConf = other.jsonConf;
+      }
+      if (other.is_set_topology()) {
+        this.topology = new StormTopology(other.topology);
+      }
+    }
+
+    public submitTopology_args deepCopy() {
+      return new submitTopology_args(this);
+    }
+
+    @Deprecated
+    public submitTopology_args clone() {
+      return new submitTopology_args(this);
+    }
+
+    public String get_name() {
+      return this.name;
+    }
+
+    public void set_name(String name) {
+      this.name = name;
+    }
+
+    public void unset_name() {
+      this.name = null;
+    }
+
+    /** Returns true if field name is set (has been asigned a value) and false otherwise */
+    public boolean is_set_name() {
+      return this.name != null;
+    }
+
+    public void set_name_isSet(boolean value) {
+      if (!value) {
+        this.name = null;
+      }
+    }
+
+    public String get_uploadedJarLocation() {
+      return this.uploadedJarLocation;
+    }
+
+    public void set_uploadedJarLocation(String uploadedJarLocation) {
+      this.uploadedJarLocation = uploadedJarLocation;
+    }
+
+    public void unset_uploadedJarLocation() {
+      this.uploadedJarLocation = null;
+    }
+
+    /** Returns true if field uploadedJarLocation is set (has been asigned a value) and false otherwise */
+    public boolean is_set_uploadedJarLocation() {
+      return this.uploadedJarLocation != null;
+    }
+
+    public void set_uploadedJarLocation_isSet(boolean value) {
+      if (!value) {
+        this.uploadedJarLocation = null;
+      }
+    }
+
+    public String get_jsonConf() {
+      return this.jsonConf;
+    }
+
+    public void set_jsonConf(String jsonConf) {
+      this.jsonConf = jsonConf;
+    }
+
+    public void unset_jsonConf() {
+      this.jsonConf = null;
+    }
+
+    /** Returns true if field jsonConf is set (has been asigned a value) and false otherwise */
+    public boolean is_set_jsonConf() {
+      return this.jsonConf != null;
+    }
+
+    public void set_jsonConf_isSet(boolean value) {
+      if (!value) {
+        this.jsonConf = null;
+      }
+    }
+
+    public StormTopology get_topology() {
+      return this.topology;
+    }
+
+    public void set_topology(StormTopology topology) {
+      this.topology = topology;
+    }
+
+    public void unset_topology() {
+      this.topology = null;
+    }
+
+    /** Returns true if field topology is set (has been asigned a value) and false otherwise */
+    public boolean is_set_topology() {
+      return this.topology != null;
+    }
+
+    public void set_topology_isSet(boolean value) {
+      if (!value) {
+        this.topology = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case NAME:
+        if (value == null) {
+          unset_name();
+        } else {
+          set_name((String)value);
+        }
+        break;
+
+      case UPLOADED_JAR_LOCATION:
+        if (value == null) {
+          unset_uploadedJarLocation();
+        } else {
+          set_uploadedJarLocation((String)value);
+        }
+        break;
+
+      case JSON_CONF:
+        if (value == null) {
+          unset_jsonConf();
+        } else {
+          set_jsonConf((String)value);
+        }
+        break;
+
+      case TOPOLOGY:
+        if (value == null) {
+          unset_topology();
+        } else {
+          set_topology((StormTopology)value);
+        }
+        break;
+
+      }
+    }
+
+    public void setFieldValue(int fieldID, Object value) {
+      setFieldValue(_Fields.findByThriftIdOrThrow(fieldID), value);
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case NAME:
+        return get_name();
+
+      case UPLOADED_JAR_LOCATION:
+        return get_uploadedJarLocation();
+
+      case JSON_CONF:
+        return get_jsonConf();
+
+      case TOPOLOGY:
+        return get_topology();
+
+      }
+      throw new IllegalStateException();
+    }
+
+    public Object getFieldValue(int fieldId) {
+      return getFieldValue(_Fields.findByThriftIdOrThrow(fieldId));
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been asigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      switch (field) {
+      case NAME:
+        return is_set_name();
+      case UPLOADED_JAR_LOCATION:
+        return is_set_uploadedJarLocation();
+      case JSON_CONF:
+        return is_set_jsonConf();
+      case TOPOLOGY:
+        return is_set_topology();
+      }
+      throw new IllegalStateException();
+    }
+
+    public boolean isSet(int fieldID) {
+      return isSet(_Fields.findByThriftIdOrThrow(fieldID));
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof submitTopology_args)
+        return this.equals((submitTopology_args)that);
+      return false;
+    }
+
+    public boolean equals(submitTopology_args that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_name = true && this.is_set_name();
+      boolean that_present_name = true && that.is_set_name();
+      if (this_present_name || that_present_name) {
+        if (!(this_present_name && that_present_name))
+          return false;
+        if (!this.name.equals(that.name))
+          return false;
+      }
+
+      boolean this_present_uploadedJarLocation = true && this.is_set_uploadedJarLocation();
+      boolean that_present_uploadedJarLocation = true && that.is_set_uploadedJarLocation();
+      if (this_present_uploadedJarLocation || that_present_uploadedJarLocation) {
+        if (!(this_present_uploadedJarLocation && that_present_uploadedJarLocation))
+          return false;
+        if (!this.uploadedJarLocation.equals(that.uploadedJarLocation))
+          return false;
+      }
+
+      boolean this_present_jsonConf = true && this.is_set_jsonConf();
+      boolean that_present_jsonConf = true && that.is_set_jsonConf();
+      if (this_present_jsonConf || that_present_jsonConf) {
+        if (!(this_present_jsonConf && that_present_jsonConf))
+          return false;
+        if (!this.jsonConf.equals(that.jsonConf))
+          return false;
+      }
+
+      boolean this_present_topology = true && this.is_set_topology();
+      boolean that_present_topology = true && that.is_set_topology();
+      if (this_present_topology || that_present_topology) {
+        if (!(this_present_topology && that_present_topology))
+          return false;
+        if (!this.topology.equals(that.topology))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      HashCodeBuilder builder = new HashCodeBuilder();
+
+      boolean present_name = true && (is_set_name());
+      builder.append(present_name);
+      if (present_name)
+        builder.append(name);
+
+      boolean present_uploadedJarLocation = true && (is_set_uploadedJarLocation());
+      builder.append(present_uploadedJarLocation);
+      if (present_uploadedJarLocation)
+        builder.append(uploadedJarLocation);
+
+      boolean present_jsonConf = true && (is_set_jsonConf());
+      builder.append(present_jsonConf);
+      if (present_jsonConf)
+        builder.append(jsonConf);
+
+      boolean present_topology = true && (is_set_topology());
+      builder.append(present_topology);
+      if (present_topology)
+        builder.append(topology);
+
+      return builder.toHashCode();
+    }
+
+    public int compareTo(submitTopology_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+      submitTopology_args typedOther = (submitTopology_args)other;
+
+      lastComparison = Boolean.valueOf(is_set_name()).compareTo(typedOther.is_set_name());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_name()) {        lastComparison = TBaseHelper.compareTo(this.name, typedOther.name);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(is_set_uploadedJarLocation()).compareTo(typedOther.is_set_uploadedJarLocation());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_uploadedJarLocation()) {        lastComparison = TBaseHelper.compareTo(this.uploadedJarLocation, typedOther.uploadedJarLocation);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(is_set_jsonConf()).compareTo(typedOther.is_set_jsonConf());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_jsonConf()) {        lastComparison = TBaseHelper.compareTo(this.jsonConf, typedOther.jsonConf);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(is_set_topology()).compareTo(typedOther.is_set_topology());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_topology()) {        lastComparison = TBaseHelper.compareTo(this.topology, typedOther.topology);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    public void read(TProtocol iprot) throws TException {
+      TField field;
+      iprot.readStructBegin();
+      while (true)
+      {
+        field = iprot.readFieldBegin();
+        if (field.type == TType.STOP) { 
+          break;
+        }
+        switch (field.id) {
+          case 1: // NAME
+            if (field.type == TType.STRING) {
+              this.name = iprot.readString();
+            } else { 
+              TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
+          case 2: // UPLOADED_JAR_LOCATION
+            if (field.type == TType.STRING) {
+              this.uploadedJarLocation = iprot.readString();
+            } else { 
+              TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
+          case 3: // JSON_CONF
+            if (field.type == TType.STRING) {
+              this.jsonConf = iprot.readString();
+            } else { 
+              TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
+          case 4: // TOPOLOGY
+            if (field.type == TType.STRUCT) {
+              this.topology = new StormTopology();
+              this.topology.read(iprot);
+            } else { 
+              TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
+          default:
+            TProtocolUtil.skip(iprot, field.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      validate();
+    }
+
+    public void write(TProtocol oprot) throws TException {
+      validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (this.name != null) {
+        oprot.writeFieldBegin(NAME_FIELD_DESC);
+        oprot.writeString(this.name);
+        oprot.writeFieldEnd();
+      }
+      if (this.uploadedJarLocation != null) {
+        oprot.writeFieldBegin(UPLOADED_JAR_LOCATION_FIELD_DESC);
+        oprot.writeString(this.uploadedJarLocation);
+        oprot.writeFieldEnd();
+      }
+      if (this.jsonConf != null) {
+        oprot.writeFieldBegin(JSON_CONF_FIELD_DESC);
+        oprot.writeString(this.jsonConf);
+        oprot.writeFieldEnd();
+      }
+      if (this.topology != null) {
+        oprot.writeFieldBegin(TOPOLOGY_FIELD_DESC);
+        this.topology.write(oprot);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("submitTopology_args(");
+      boolean first = true;
+
+      sb.append("name:");
+      if (this.name == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.name);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("uploadedJarLocation:");
+      if (this.uploadedJarLocation == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.uploadedJarLocation);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("jsonConf:");
+      if (this.jsonConf == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.jsonConf);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("topology:");
+      if (this.topology == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.topology);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws TException {
+      // check for required fields
+    }
+
+  }
+
+  public static class submitTopology_result implements TBase<submitTopology_result, submitTopology_result._Fields>, java.io.Serializable, Cloneable   {
+    private static final TStruct STRUCT_DESC = new TStruct("submitTopology_result");
+
+    private static final TField E_FIELD_DESC = new TField("e", TType.STRUCT, (short)1);
+    private static final TField ITE_FIELD_DESC = new TField("ite", TType.STRUCT, (short)2);
+
+    private AlreadyAliveException e;
+    private InvalidTopologyException ite;
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements TFieldIdEnum {
+      E((short)1, "e"),
+      ITE((short)2, "ite");
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 1: // E
+            return E;
+          case 2: // ITE
+            return ITE;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+
+    public static final Map<_Fields, FieldMetaData> metaDataMap;
+    static {
+      Map<_Fields, FieldMetaData> tmpMap = new EnumMap<_Fields, FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.E, new FieldMetaData("e", TFieldRequirementType.DEFAULT, 
+          new FieldValueMetaData(TType.STRUCT)));
+      tmpMap.put(_Fields.ITE, new FieldMetaData("ite", TFieldRequirementType.DEFAULT, 
+          new FieldValueMetaData(TType.STRUCT)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      FieldMetaData.addStructMetaDataMap(submitTopology_result.class, metaDataMap);
+    }
+
+    public submitTopology_result() {
+    }
+
+    public submitTopology_result(
+      AlreadyAliveException e,
+      InvalidTopologyException ite)
+    {
+      this();
+      this.e = e;
+      this.ite = ite;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public submitTopology_result(submitTopology_result other) {
+      if (other.is_set_e()) {
+        this.e = new AlreadyAliveException(other.e);
+      }
+      if (other.is_set_ite()) {
+        this.ite = new InvalidTopologyException(other.ite);
+      }
+    }
+
+    public submitTopology_result deepCopy() {
+      return new submitTopology_result(this);
+    }
+
+    @Deprecated
+    public submitTopology_result clone() {
+      return new submitTopology_result(this);
+    }
+
+    public AlreadyAliveException get_e() {
+      return this.e;
+    }
+
+    public void set_e(AlreadyAliveException e) {
+      this.e = e;
+    }
+
+    public void unset_e() {
+      this.e = null;
+    }
+
+    /** Returns true if field e is set (has been asigned a value) and false otherwise */
+    public boolean is_set_e() {
+      return this.e != null;
+    }
+
+    public void set_e_isSet(boolean value) {
+      if (!value) {
+        this.e = null;
+      }
+    }
+
+    public InvalidTopologyException get_ite() {
+      return this.ite;
+    }
+
+    public void set_ite(InvalidTopologyException ite) {
+      this.ite = ite;
+    }
+
+    public void unset_ite() {
+      this.ite = null;
+    }
+
+    /** Returns true if field ite is set (has been asigned a value) and false otherwise */
+    public boolean is_set_ite() {
+      return this.ite != null;
+    }
+
+    public void set_ite_isSet(boolean value) {
+      if (!value) {
+        this.ite = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case E:
+        if (value == null) {
+          unset_e();
+        } else {
+          set_e((AlreadyAliveException)value);
+        }
+        break;
+
+      case ITE:
+        if (value == null) {
+          unset_ite();
+        } else {
+          set_ite((InvalidTopologyException)value);
+        }
+        break;
+
+      }
+    }
+
+    public void setFieldValue(int fieldID, Object value) {
+      setFieldValue(_Fields.findByThriftIdOrThrow(fieldID), value);
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case E:
+        return get_e();
+
+      case ITE:
+        return get_ite();
+
+      }
+      throw new IllegalStateException();
+    }
+
+    public Object getFieldValue(int fieldId) {
+      return getFieldValue(_Fields.findByThriftIdOrThrow(fieldId));
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been asigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      switch (field) {
+      case E:
+        return is_set_e();
+      case ITE:
+        return is_set_ite();
+      }
+      throw new IllegalStateException();
+    }
+
+    public boolean isSet(int fieldID) {
+      return isSet(_Fields.findByThriftIdOrThrow(fieldID));
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof submitTopology_result)
+        return this.equals((submitTopology_result)that);
+      return false;
+    }
+
+    public boolean equals(submitTopology_result that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_e = true && this.is_set_e();
+      boolean that_present_e = true && that.is_set_e();
+      if (this_present_e || that_present_e) {
+        if (!(this_present_e && that_present_e))
+          return false;
+        if (!this.e.equals(that.e))
+          return false;
+      }
+
+      boolean this_present_ite = true && this.is_set_ite();
+      boolean that_present_ite = true && that.is_set_ite();
+      if (this_present_ite || that_present_ite) {
+        if (!(this_present_ite && that_present_ite))
+          return false;
+        if (!this.ite.equals(that.ite))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      HashCodeBuilder builder = new HashCodeBuilder();
+
+      boolean present_e = true && (is_set_e());
+      builder.append(present_e);
+      if (present_e)
+        builder.append(e);
+
+      boolean present_ite = true && (is_set_ite());
+      builder.append(present_ite);
+      if (present_ite)
+        builder.append(ite);
+
+      return builder.toHashCode();
+    }
+
+    public int compareTo(submitTopology_result other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+      submitTopology_result typedOther = (submitTopology_result)other;
+
+      lastComparison = Boolean.valueOf(is_set_e()).compareTo(typedOther.is_set_e());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_e()) {        lastComparison = TBaseHelper.compareTo(this.e, typedOther.e);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(is_set_ite()).compareTo(typedOther.is_set_ite());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_ite()) {        lastComparison = TBaseHelper.compareTo(this.ite, typedOther.ite);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    public void read(TProtocol iprot) throws TException {
+      TField field;
+      iprot.readStructBegin();
+      while (true)
+      {
+        field = iprot.readFieldBegin();
+        if (field.type == TType.STOP) { 
+          break;
+        }
+        switch (field.id) {
+          case 1: // E
+            if (field.type == TType.STRUCT) {
+              this.e = new AlreadyAliveException();
+              this.e.read(iprot);
+            } else { 
+              TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
+          case 2: // ITE
+            if (field.type == TType.STRUCT) {
+              this.ite = new InvalidTopologyException();
+              this.ite.read(iprot);
+            } else { 
+              TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
+          default:
+            TProtocolUtil.skip(iprot, field.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      validate();
+    }
+
+    public void write(TProtocol oprot) throws TException {
+      oprot.writeStructBegin(STRUCT_DESC);
+
+      if (this.is_set_e()) {
+        oprot.writeFieldBegin(E_FIELD_DESC);
+        this.e.write(oprot);
+        oprot.writeFieldEnd();
+      } else if (this.is_set_ite()) {
+        oprot.writeFieldBegin(ITE_FIELD_DESC);
+        this.ite.write(oprot);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("submitTopology_result(");
+      boolean first = true;
+
+      sb.append("e:");
+      if (this.e == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.e);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("ite:");
+      if (this.ite == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.ite);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws TException {
+      // check for required fields
+    }
+
+  }
+
+  public static class killTopology_args implements TBase<killTopology_args, killTopology_args._Fields>, java.io.Serializable, Cloneable   {
+    private static final TStruct STRUCT_DESC = new TStruct("killTopology_args");
+
+    private static final TField NAME_FIELD_DESC = new TField("name", TType.STRING, (short)1);
+
+    private String name;
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements TFieldIdEnum {
+      NAME((short)1, "name");
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 1: // NAME
+            return NAME;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+
+    public static final Map<_Fields, FieldMetaData> metaDataMap;
+    static {
+      Map<_Fields, FieldMetaData> tmpMap = new EnumMap<_Fields, FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.NAME, new FieldMetaData("name", TFieldRequirementType.DEFAULT, 
+          new FieldValueMetaData(TType.STRING)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      FieldMetaData.addStructMetaDataMap(killTopology_args.class, metaDataMap);
+    }
+
+    public killTopology_args() {
+    }
+
+    public killTopology_args(
+      String name)
+    {
+      this();
+      this.name = name;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public killTopology_args(killTopology_args other) {
+      if (other.is_set_name()) {
+        this.name = other.name;
+      }
+    }
+
+    public killTopology_args deepCopy() {
+      return new killTopology_args(this);
+    }
+
+    @Deprecated
+    public killTopology_args clone() {
+      return new killTopology_args(this);
+    }
+
+    public String get_name() {
+      return this.name;
+    }
+
+    public void set_name(String name) {
+      this.name = name;
+    }
+
+    public void unset_name() {
+      this.name = null;
+    }
+
+    /** Returns true if field name is set (has been asigned a value) and false otherwise */
+    public boolean is_set_name() {
+      return this.name != null;
+    }
+
+    public void set_name_isSet(boolean value) {
+      if (!value) {
+        this.name = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case NAME:
+        if (value == null) {
+          unset_name();
+        } else {
+          set_name((String)value);
+        }
+        break;
+
+      }
+    }
+
+    public void setFieldValue(int fieldID, Object value) {
+      setFieldValue(_Fields.findByThriftIdOrThrow(fieldID), value);
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case NAME:
+        return get_name();
+
+      }
+      throw new IllegalStateException();
+    }
+
+    public Object getFieldValue(int fieldId) {
+      return getFieldValue(_Fields.findByThriftIdOrThrow(fieldId));
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been asigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      switch (field) {
+      case NAME:
+        return is_set_name();
+      }
+      throw new IllegalStateException();
+    }
+
+    public boolean isSet(int fieldID) {
+      return isSet(_Fields.findByThriftIdOrThrow(fieldID));
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof killTopology_args)
+        return this.equals((killTopology_args)that);
+      return false;
+    }
+
+    public boolean equals(killTopology_args that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_name = true && this.is_set_name();
+      boolean that_present_name = true && that.is_set_name();
+      if (this_present_name || that_present_name) {
+        if (!(this_present_name && that_present_name))
+          return false;
+        if (!this.name.equals(that.name))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      HashCodeBuilder builder = new HashCodeBuilder();
+
+      boolean present_name = true && (is_set_name());
+      builder.append(present_name);
+      if (present_name)
+        builder.append(name);
+
+      return builder.toHashCode();
+    }
+
+    public int compareTo(killTopology_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+      killTopology_args typedOther = (killTopology_args)other;
+
+      lastComparison = Boolean.valueOf(is_set_name()).compareTo(typedOther.is_set_name());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_name()) {        lastComparison = TBaseHelper.compareTo(this.name, typedOther.name);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    public void read(TProtocol iprot) throws TException {
+      TField field;
+      iprot.readStructBegin();
+      while (true)
+      {
+        field = iprot.readFieldBegin();
+        if (field.type == TType.STOP) { 
+          break;
+        }
+        switch (field.id) {
+          case 1: // NAME
+            if (field.type == TType.STRING) {
+              this.name = iprot.readString();
+            } else { 
+              TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
+          default:
+            TProtocolUtil.skip(iprot, field.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      validate();
+    }
+
+    public void write(TProtocol oprot) throws TException {
+      validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (this.name != null) {
+        oprot.writeFieldBegin(NAME_FIELD_DESC);
+        oprot.writeString(this.name);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("killTopology_args(");
+      boolean first = true;
+
+      sb.append("name:");
+      if (this.name == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.name);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws TException {
+      // check for required fields
+    }
+
+  }
+
+  public static class killTopology_result implements TBase<killTopology_result, killTopology_result._Fields>, java.io.Serializable, Cloneable   {
+    private static final TStruct STRUCT_DESC = new TStruct("killTopology_result");
+
+    private static final TField E_FIELD_DESC = new TField("e", TType.STRUCT, (short)1);
+
+    private NotAliveException e;
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements TFieldIdEnum {
+      E((short)1, "e");
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 1: // E
+            return E;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+
+    public static final Map<_Fields, FieldMetaData> metaDataMap;
+    static {
+      Map<_Fields, FieldMetaData> tmpMap = new EnumMap<_Fields, FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.E, new FieldMetaData("e", TFieldRequirementType.DEFAULT, 
+          new FieldValueMetaData(TType.STRUCT)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      FieldMetaData.addStructMetaDataMap(killTopology_result.class, metaDataMap);
+    }
+
+    public killTopology_result() {
+    }
+
+    public killTopology_result(
+      NotAliveException e)
+    {
+      this();
+      this.e = e;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public killTopology_result(killTopology_result other) {
+      if (other.is_set_e()) {
+        this.e = new NotAliveException(other.e);
+      }
+    }
+
+    public killTopology_result deepCopy() {
+      return new killTopology_result(this);
+    }
+
+    @Deprecated
+    public killTopology_result clone() {
+      return new killTopology_result(this);
+    }
+
+    public NotAliveException get_e() {
+      return this.e;
+    }
+
+    public void set_e(NotAliveException e) {
+      this.e = e;
+    }
+
+    public void unset_e() {
+      this.e = null;
+    }
+
+    /** Returns true if field e is set (has been asigned a value) and false otherwise */
+    public boolean is_set_e() {
+      return this.e != null;
+    }
+
+    public void set_e_isSet(boolean value) {
+      if (!value) {
+        this.e = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case E:
+        if (value == null) {
+          unset_e();
+        } else {
+          set_e((NotAliveException)value);
+        }
+        break;
+
+      }
+    }
+
+    public void setFieldValue(int fieldID, Object value) {
+      setFieldValue(_Fields.findByThriftIdOrThrow(fieldID), value);
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case E:
+        return get_e();
+
+      }
+      throw new IllegalStateException();
+    }
+
+    public Object getFieldValue(int fieldId) {
+      return getFieldValue(_Fields.findByThriftIdOrThrow(fieldId));
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been asigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      switch (field) {
+      case E:
+        return is_set_e();
+      }
+      throw new IllegalStateException();
+    }
+
+    public boolean isSet(int fieldID) {
+      return isSet(_Fields.findByThriftIdOrThrow(fieldID));
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof killTopology_result)
+        return this.equals((killTopology_result)that);
+      return false;
+    }
+
+    public boolean equals(killTopology_result that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_e = true && this.is_set_e();
+      boolean that_present_e = true && that.is_set_e();
+      if (this_present_e || that_present_e) {
+        if (!(this_present_e && that_present_e))
+          return false;
+        if (!this.e.equals(that.e))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      HashCodeBuilder builder = new HashCodeBuilder();
+
+      boolean present_e = true && (is_set_e());
+      builder.append(present_e);
+      if (present_e)
+        builder.append(e);
+
+      return builder.toHashCode();
+    }
+
+    public int compareTo(killTopology_result other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+      killTopology_result typedOther = (killTopology_result)other;
+
+      lastComparison = Boolean.valueOf(is_set_e()).compareTo(typedOther.is_set_e());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_e()) {        lastComparison = TBaseHelper.compareTo(this.e, typedOther.e);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    public void read(TProtocol iprot) throws TException {
+      TField field;
+      iprot.readStructBegin();
+      while (true)
+      {
+        field = iprot.readFieldBegin();
+        if (field.type == TType.STOP) { 
+          break;
+        }
+        switch (field.id) {
+          case 1: // E
+            if (field.type == TType.STRUCT) {
+              this.e = new NotAliveException();
+              this.e.read(iprot);
+            } else { 
+              TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
+          default:
+            TProtocolUtil.skip(iprot, field.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      validate();
+    }
+
+    public void write(TProtocol oprot) throws TException {
+      oprot.writeStructBegin(STRUCT_DESC);
+
+      if (this.is_set_e()) {
+        oprot.writeFieldBegin(E_FIELD_DESC);
+        this.e.write(oprot);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("killTopology_result(");
+      boolean first = true;
+
+      sb.append("e:");
+      if (this.e == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.e);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws TException {
+      // check for required fields
+    }
+
+  }
+
+  public static class beginFileUpload_args implements TBase<beginFileUpload_args, beginFileUpload_args._Fields>, java.io.Serializable, Cloneable   {
+    private static final TStruct STRUCT_DESC = new TStruct("beginFileUpload_args");
+
+
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements TFieldIdEnum {
+;
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+    public static final Map<_Fields, FieldMetaData> metaDataMap;
+    static {
+      Map<_Fields, FieldMetaData> tmpMap = new EnumMap<_Fields, FieldMetaData>(_Fields.class);
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      FieldMetaData.addStructMetaDataMap(beginFileUpload_args.class, metaDataMap);
+    }
+
+    public beginFileUpload_args() {
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public beginFileUpload_args(beginFileUpload_args other) {
+    }
+
+    public beginFileUpload_args deepCopy() {
+      return new beginFileUpload_args(this);
+    }
+
+    @Deprecated
+    public beginFileUpload_args clone() {
+      return new beginFileUpload_args(this);
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      }
+    }
+
+    public void setFieldValue(int fieldID, Object value) {
+      setFieldValue(_Fields.findByThriftIdOrThrow(fieldID), value);
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      }
+      throw new IllegalStateException();
+    }
+
+    public Object getFieldValue(int fieldId) {
+      return getFieldValue(_Fields.findByThriftIdOrThrow(fieldId));
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been asigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      switch (field) {
+      }
+      throw new IllegalStateException();
+    }
+
+    public boolean isSet(int fieldID) {
+      return isSet(_Fields.findByThriftIdOrThrow(fieldID));
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof beginFileUpload_args)
+        return this.equals((beginFileUpload_args)that);
+      return false;
+    }
+
+    public boolean equals(beginFileUpload_args that) {
+      if (that == null)
+        return false;
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      HashCodeBuilder builder = new HashCodeBuilder();
+
+      return builder.toHashCode();
+    }
+
+    public int compareTo(beginFileUpload_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+      beginFileUpload_args typedOther = (beginFileUpload_args)other;
+
+      return 0;
+    }
+
+    public void read(TProtocol iprot) throws TException {
+      TField field;
+      iprot.readStructBegin();
+      while (true)
+      {
+        field = iprot.readFieldBegin();
+        if (field.type == TType.STOP) { 
+          break;
+        }
+        switch (field.id) {
+          default:
+            TProtocolUtil.skip(iprot, field.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      validate();
+    }
+
+    public void write(TProtocol oprot) throws TException {
+      validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("beginFileUpload_args(");
+      boolean first = true;
+
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws TException {
+      // check for required fields
+    }
+
+  }
+
+  public static class beginFileUpload_result implements TBase<beginFileUpload_result, beginFileUpload_result._Fields>, java.io.Serializable, Cloneable   {
+    private static final TStruct STRUCT_DESC = new TStruct("beginFileUpload_result");
+
+    private static final TField SUCCESS_FIELD_DESC = new TField("success", TType.STRING, (short)0);
+
+    private String success;
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements TFieldIdEnum {
+      SUCCESS((short)0, "success");
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 0: // SUCCESS
+            return SUCCESS;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+
+    public static final Map<_Fields, FieldMetaData> metaDataMap;
+    static {
+      Map<_Fields, FieldMetaData> tmpMap = new EnumMap<_Fields, FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.SUCCESS, new FieldMetaData("success", TFieldRequirementType.DEFAULT, 
+          new FieldValueMetaData(TType.STRING)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      FieldMetaData.addStructMetaDataMap(beginFileUpload_result.class, metaDataMap);
+    }
+
+    public beginFileUpload_result() {
+    }
+
+    public beginFileUpload_result(
+      String success)
+    {
+      this();
+      this.success = success;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public beginFileUpload_result(beginFileUpload_result other) {
+      if (other.is_set_success()) {
+        this.success = other.success;
+      }
+    }
+
+    public beginFileUpload_result deepCopy() {
+      return new beginFileUpload_result(this);
+    }
+
+    @Deprecated
+    public beginFileUpload_result clone() {
+      return new beginFileUpload_result(this);
+    }
+
+    public String get_success() {
+      return this.success;
+    }
+
+    public void set_success(String success) {
+      this.success = success;
+    }
+
+    public void unset_success() {
+      this.success = null;
+    }
+
+    /** Returns true if field success is set (has been asigned a value) and false otherwise */
+    public boolean is_set_success() {
+      return this.success != null;
+    }
+
+    public void set_success_isSet(boolean value) {
+      if (!value) {
+        this.success = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case SUCCESS:
+        if (value == null) {
+          unset_success();
+        } else {
+          set_success((String)value);
+        }
+        break;
+
+      }
+    }
+
+    public void setFieldValue(int fieldID, Object value) {
+      setFieldValue(_Fields.findByThriftIdOrThrow(fieldID), value);
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case SUCCESS:
+        return get_success();
+
+      }
+      throw new IllegalStateException();
+    }
+
+    public Object getFieldValue(int fieldId) {
+      return getFieldValue(_Fields.findByThriftIdOrThrow(fieldId));
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been asigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      switch (field) {
+      case SUCCESS:
+        return is_set_success();
+      }
+      throw new IllegalStateException();
+    }
+
+    public boolean isSet(int fieldID) {
+      return isSet(_Fields.findByThriftIdOrThrow(fieldID));
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof beginFileUpload_result)
+        return this.equals((beginFileUpload_result)that);
+      return false;
+    }
+
+    public boolean equals(beginFileUpload_result that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_success = true && this.is_set_success();
+      boolean that_present_success = true && that.is_set_success();
+      if (this_present_success || that_present_success) {
+        if (!(this_present_success && that_present_success))
+          return false;
+        if (!this.success.equals(that.success))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      HashCodeBuilder builder = new HashCodeBuilder();
+
+      boolean present_success = true && (is_set_success());
+      builder.append(present_success);
+      if (present_success)
+        builder.append(success);
+
+      return builder.toHashCode();
+    }
+
+    public int compareTo(beginFileUpload_result other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+      beginFileUpload_result typedOther = (beginFileUpload_result)other;
+
+      lastComparison = Boolean.valueOf(is_set_success()).compareTo(typedOther.is_set_success());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_success()) {        lastComparison = TBaseHelper.compareTo(this.success, typedOther.success);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    public void read(TProtocol iprot) throws TException {
+      TField field;
+      iprot.readStructBegin();
+      while (true)
+      {
+        field = iprot.readFieldBegin();
+        if (field.type == TType.STOP) { 
+          break;
+        }
+        switch (field.id) {
+          case 0: // SUCCESS
+            if (field.type == TType.STRING) {
+              this.success = iprot.readString();
+            } else { 
+              TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
+          default:
+            TProtocolUtil.skip(iprot, field.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      validate();
+    }
+
+    public void write(TProtocol oprot) throws TException {
+      oprot.writeStructBegin(STRUCT_DESC);
+
+      if (this.is_set_success()) {
+        oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
+        oprot.writeString(this.success);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("beginFileUpload_result(");
+      boolean first = true;
+
+      sb.append("success:");
+      if (this.success == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.success);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws TException {
+      // check for required fields
+    }
+
+  }
+
+  public static class uploadChunk_args implements TBase<uploadChunk_args, uploadChunk_args._Fields>, java.io.Serializable, Cloneable   {
+    private static final TStruct STRUCT_DESC = new TStruct("uploadChunk_args");
+
+    private static final TField LOCATION_FIELD_DESC = new TField("location", TType.STRING, (short)1);
+    private static final TField CHUNK_FIELD_DESC = new TField("chunk", TType.STRING, (short)2);
+
+    private String location;
+    private byte[] chunk;
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements TFieldIdEnum {
+      LOCATION((short)1, "location"),
+      CHUNK((short)2, "chunk");
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 1: // LOCATION
+            return LOCATION;
+          case 2: // CHUNK
+            return CHUNK;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+
+    public static final Map<_Fields, FieldMetaData> metaDataMap;
+    static {
+      Map<_Fields, FieldMetaData> tmpMap = new EnumMap<_Fields, FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.LOCATION, new FieldMetaData("location", TFieldRequirementType.DEFAULT, 
+          new FieldValueMetaData(TType.STRING)));
+      tmpMap.put(_Fields.CHUNK, new FieldMetaData("chunk", TFieldRequirementType.DEFAULT, 
+          new FieldValueMetaData(TType.STRING)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      FieldMetaData.addStructMetaDataMap(uploadChunk_args.class, metaDataMap);
+    }
+
+    public uploadChunk_args() {
+    }
+
+    public uploadChunk_args(
+      String location,
+      byte[] chunk)
+    {
+      this();
+      this.location = location;
+      this.chunk = chunk;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public uploadChunk_args(uploadChunk_args other) {
+      if (other.is_set_location()) {
+        this.location = other.location;
+      }
+      if (other.is_set_chunk()) {
+        this.chunk = new byte[other.chunk.length];
+        System.arraycopy(other.chunk, 0, chunk, 0, other.chunk.length);
+      }
+    }
+
+    public uploadChunk_args deepCopy() {
+      return new uploadChunk_args(this);
+    }
+
+    @Deprecated
+    public uploadChunk_args clone() {
+      return new uploadChunk_args(this);
+    }
+
+    public String get_location() {
+      return this.location;
+    }
+
+    public void set_location(String location) {
+      this.location = location;
+    }
+
+    public void unset_location() {
+      this.location = null;
+    }
+
+    /** Returns true if field location is set (has been asigned a value) and false otherwise */
+    public boolean is_set_location() {
+      return this.location != null;
+    }
+
+    public void set_location_isSet(boolean value) {
+      if (!value) {
+        this.location = null;
+      }
+    }
+
+    public byte[] get_chunk() {
+      return this.chunk;
+    }
+
+    public void set_chunk(byte[] chunk) {
+      this.chunk = chunk;
+    }
+
+    public void unset_chunk() {
+      this.chunk = null;
+    }
+
+    /** Returns true if field chunk is set (has been asigned a value) and false otherwise */
+    public boolean is_set_chunk() {
+      return this.chunk != null;
+    }
+
+    public void set_chunk_isSet(boolean value) {
+      if (!value) {
+        this.chunk = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case LOCATION:
+        if (value == null) {
+          unset_location();
+        } else {
+          set_location((String)value);
+        }
+        break;
+
+      case CHUNK:
+        if (value == null) {
+          unset_chunk();
+        } else {
+          set_chunk((byte[])value);
+        }
+        break;
+
+      }
+    }
+
+    public void setFieldValue(int fieldID, Object value) {
+      setFieldValue(_Fields.findByThriftIdOrThrow(fieldID), value);
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case LOCATION:
+        return get_location();
+
+      case CHUNK:
+        return get_chunk();
+
+      }
+      throw new IllegalStateException();
+    }
+
+    public Object getFieldValue(int fieldId) {
+      return getFieldValue(_Fields.findByThriftIdOrThrow(fieldId));
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been asigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      switch (field) {
+      case LOCATION:
+        return is_set_location();
+      case CHUNK:
+        return is_set_chunk();
+      }
+      throw new IllegalStateException();
+    }
+
+    public boolean isSet(int fieldID) {
+      return isSet(_Fields.findByThriftIdOrThrow(fieldID));
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof uploadChunk_args)
+        return this.equals((uploadChunk_args)that);
+      return false;
+    }
+
+    public boolean equals(uploadChunk_args that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_location = true && this.is_set_location();
+      boolean that_present_location = true && that.is_set_location();
+      if (this_present_location || that_present_location) {
+        if (!(this_present_location && that_present_location))
+          return false;
+        if (!this.location.equals(that.location))
+          return false;
+      }
+
+      boolean this_present_chunk = true && this.is_set_chunk();
+      boolean that_present_chunk = true && that.is_set_chunk();
+      if (this_present_chunk || that_present_chunk) {
+        if (!(this_present_chunk && that_present_chunk))
+          return false;
+        if (!java.util.Arrays.equals(this.chunk, that.chunk))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      HashCodeBuilder builder = new HashCodeBuilder();
+
+      boolean present_location = true && (is_set_location());
+      builder.append(present_location);
+      if (present_location)
+        builder.append(location);
+
+      boolean present_chunk = true && (is_set_chunk());
+      builder.append(present_chunk);
+      if (present_chunk)
+        builder.append(chunk);
+
+      return builder.toHashCode();
+    }
+
+    public int compareTo(uploadChunk_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+      uploadChunk_args typedOther = (uploadChunk_args)other;
+
+      lastComparison = Boolean.valueOf(is_set_location()).compareTo(typedOther.is_set_location());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_location()) {        lastComparison = TBaseHelper.compareTo(this.location, typedOther.location);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(is_set_chunk()).compareTo(typedOther.is_set_chunk());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_chunk()) {        lastComparison = TBaseHelper.compareTo(this.chunk, typedOther.chunk);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    public void read(TProtocol iprot) throws TException {
+      TField field;
+      iprot.readStructBegin();
+      while (true)
+      {
+        field = iprot.readFieldBegin();
+        if (field.type == TType.STOP) { 
+          break;
+        }
+        switch (field.id) {
+          case 1: // LOCATION
+            if (field.type == TType.STRING) {
+              this.location = iprot.readString();
+            } else { 
+              TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
+          case 2: // CHUNK
+            if (field.type == TType.STRING) {
+              this.chunk = iprot.readBinary();
+            } else { 
+              TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
+          default:
+            TProtocolUtil.skip(iprot, field.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      validate();
+    }
+
+    public void write(TProtocol oprot) throws TException {
+      validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (this.location != null) {
+        oprot.writeFieldBegin(LOCATION_FIELD_DESC);
+        oprot.writeString(this.location);
+        oprot.writeFieldEnd();
+      }
+      if (this.chunk != null) {
+        oprot.writeFieldBegin(CHUNK_FIELD_DESC);
+        oprot.writeBinary(this.chunk);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("uploadChunk_args(");
+      boolean first = true;
+
+      sb.append("location:");
+      if (this.location == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.location);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("chunk:");
+      if (this.chunk == null) {
+        sb.append("null");
+      } else {
+          int __chunk_size = Math.min(this.chunk.length, 128);
+          for (int i = 0; i < __chunk_size; i++) {
+            if (i != 0) sb.append(" ");
+            sb.append(Integer.toHexString(this.chunk[i]).length() > 1 ? Integer.toHexString(this.chunk[i]).substring(Integer.toHexString(this.chunk[i]).length() - 2).toUpperCase() : "0" + Integer.toHexString(this.chunk[i]).toUpperCase());
+          }
+          if (this.chunk.length > 128) sb.append(" ...");
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws TException {
+      // check for required fields
+    }
+
+  }
+
+  public static class uploadChunk_result implements TBase<uploadChunk_result, uploadChunk_result._Fields>, java.io.Serializable, Cloneable   {
+    private static final TStruct STRUCT_DESC = new TStruct("uploadChunk_result");
+
+
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements TFieldIdEnum {
+;
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+    public static final Map<_Fields, FieldMetaData> metaDataMap;
+    static {
+      Map<_Fields, FieldMetaData> tmpMap = new EnumMap<_Fields, FieldMetaData>(_Fields.class);
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      FieldMetaData.addStructMetaDataMap(uploadChunk_result.class, metaDataMap);
+    }
+
+    public uploadChunk_result() {
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public uploadChunk_result(uploadChunk_result other) {
+    }
+
+    public uploadChunk_result deepCopy() {
+      return new uploadChunk_result(this);
+    }
+
+    @Deprecated
+    public uploadChunk_result clone() {
+      return new uploadChunk_result(this);
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      }
+    }
+
+    public void setFieldValue(int fieldID, Object value) {
+      setFieldValue(_Fields.findByThriftIdOrThrow(fieldID), value);
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      }
+      throw new IllegalStateException();
+    }
+
+    public Object getFieldValue(int fieldId) {
+      return getFieldValue(_Fields.findByThriftIdOrThrow(fieldId));
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been asigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      switch (field) {
+      }
+      throw new IllegalStateException();
+    }
+
+    public boolean isSet(int fieldID) {
+      return isSet(_Fields.findByThriftIdOrThrow(fieldID));
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof uploadChunk_result)
+        return this.equals((uploadChunk_result)that);
+      return false;
+    }
+
+    public boolean equals(uploadChunk_result that) {
+      if (that == null)
+        return false;
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      HashCodeBuilder builder = new HashCodeBuilder();
+
+      return builder.toHashCode();
+    }
+
+    public int compareTo(uploadChunk_result other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+      uploadChunk_result typedOther = (uploadChunk_result)other;
+
+      return 0;
+    }
+
+    public void read(TProtocol iprot) throws TException {
+      TField field;
+      iprot.readStructBegin();
+      while (true)
+      {
+        field = iprot.readFieldBegin();
+        if (field.type == TType.STOP) { 
+          break;
+        }
+        switch (field.id) {
+          default:
+            TProtocolUtil.skip(iprot, field.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      validate();
+    }
+
+    public void write(TProtocol oprot) throws TException {
+      oprot.writeStructBegin(STRUCT_DESC);
+
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("uploadChunk_result(");
+      boolean first = true;
+
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws TException {
+      // check for required fields
+    }
+
+  }
+
+  public static class finishFileUpload_args implements TBase<finishFileUpload_args, finishFileUpload_args._Fields>, java.io.Serializable, Cloneable   {
+    private static final TStruct STRUCT_DESC = new TStruct("finishFileUpload_args");
+
+    private static final TField LOCATION_FIELD_DESC = new TField("location", TType.STRING, (short)1);
+
+    private String location;
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements TFieldIdEnum {
+      LOCATION((short)1, "location");
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 1: // LOCATION
+            return LOCATION;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+
+    public static final Map<_Fields, FieldMetaData> metaDataMap;
+    static {
+      Map<_Fields, FieldMetaData> tmpMap = new EnumMap<_Fields, FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.LOCATION, new FieldMetaData("location", TFieldRequirementType.DEFAULT, 
+          new FieldValueMetaData(TType.STRING)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      FieldMetaData.addStructMetaDataMap(finishFileUpload_args.class, metaDataMap);
+    }
+
+    public finishFileUpload_args() {
+    }
+
+    public finishFileUpload_args(
+      String location)
+    {
+      this();
+      this.location = location;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public finishFileUpload_args(finishFileUpload_args other) {
+      if (other.is_set_location()) {
+        this.location = other.location;
+      }
+    }
+
+    public finishFileUpload_args deepCopy() {
+      return new finishFileUpload_args(this);
+    }
+
+    @Deprecated
+    public finishFileUpload_args clone() {
+      return new finishFileUpload_args(this);
+    }
+
+    public String get_location() {
+      return this.location;
+    }
+
+    public void set_location(String location) {
+      this.location = location;
+    }
+
+    public void unset_location() {
+      this.location = null;
+    }
+
+    /** Returns true if field location is set (has been asigned a value) and false otherwise */
+    public boolean is_set_location() {
+      return this.location != null;
+    }
+
+    public void set_location_isSet(boolean value) {
+      if (!value) {
+        this.location = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case LOCATION:
+        if (value == null) {
+          unset_location();
+        } else {
+          set_location((String)value);
+        }
+        break;
+
+      }
+    }
+
+    public void setFieldValue(int fieldID, Object value) {
+      setFieldValue(_Fields.findByThriftIdOrThrow(fieldID), value);
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case LOCATION:
+        return get_location();
+
+      }
+      throw new IllegalStateException();
+    }
+
+    public Object getFieldValue(int fieldId) {
+      return getFieldValue(_Fields.findByThriftIdOrThrow(fieldId));
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been asigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      switch (field) {
+      case LOCATION:
+        return is_set_location();
+      }
+      throw new IllegalStateException();
+    }
+
+    public boolean isSet(int fieldID) {
+      return isSet(_Fields.findByThriftIdOrThrow(fieldID));
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof finishFileUpload_args)
+        return this.equals((finishFileUpload_args)that);
+      return false;
+    }
+
+    public boolean equals(finishFileUpload_args that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_location = true && this.is_set_location();
+      boolean that_present_location = true && that.is_set_location();
+      if (this_present_location || that_present_location) {
+        if (!(this_present_location && that_present_location))
+          return false;
+        if (!this.location.equals(that.location))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      HashCodeBuilder builder = new HashCodeBuilder();
+
+      boolean present_location = true && (is_set_location());
+      builder.append(present_location);
+      if (present_location)
+        builder.append(location);
+
+      return builder.toHashCode();
+    }
+
+    public int compareTo(finishFileUpload_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+      finishFileUpload_args typedOther = (finishFileUpload_args)other;
+
+      lastComparison = Boolean.valueOf(is_set_location()).compareTo(typedOther.is_set_location());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_location()) {        lastComparison = TBaseHelper.compareTo(this.location, typedOther.location);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    public void read(TProtocol iprot) throws TException {
+      TField field;
+      iprot.readStructBegin();
+      while (true)
+      {
+        field = iprot.readFieldBegin();
+        if (field.type == TType.STOP) { 
+          break;
+        }
+        switch (field.id) {
+          case 1: // LOCATION
+            if (field.type == TType.STRING) {
+              this.location = iprot.readString();
+            } else { 
+              TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
+          default:
+            TProtocolUtil.skip(iprot, field.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      validate();
+    }
+
+    public void write(TProtocol oprot) throws TException {
+      validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (this.location != null) {
+        oprot.writeFieldBegin(LOCATION_FIELD_DESC);
+        oprot.writeString(this.location);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("finishFileUpload_args(");
+      boolean first = true;
+
+      sb.append("location:");
+      if (this.location == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.location);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws TException {
+      // check for required fields
+    }
+
+  }
+
+  public static class finishFileUpload_result implements TBase<finishFileUpload_result, finishFileUpload_result._Fields>, java.io.Serializable, Cloneable   {
+    private static final TStruct STRUCT_DESC = new TStruct("finishFileUpload_result");
+
+
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements TFieldIdEnum {
+;
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+    public static final Map<_Fields, FieldMetaData> metaDataMap;
+    static {
+      Map<_Fields, FieldMetaData> tmpMap = new EnumMap<_Fields, FieldMetaData>(_Fields.class);
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      FieldMetaData.addStructMetaDataMap(finishFileUpload_result.class, metaDataMap);
+    }
+
+    public finishFileUpload_result() {
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public finishFileUpload_result(finishFileUpload_result other) {
+    }
+
+    public finishFileUpload_result deepCopy() {
+      return new finishFileUpload_result(this);
+    }
+
+    @Deprecated
+    public finishFileUpload_result clone() {
+      return new finishFileUpload_result(this);
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      }
+    }
+
+    public void setFieldValue(int fieldID, Object value) {
+      setFieldValue(_Fields.findByThriftIdOrThrow(fieldID), value);
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      }
+      throw new IllegalStateException();
+    }
+
+    public Object getFieldValue(int fieldId) {
+      return getFieldValue(_Fields.findByThriftIdOrThrow(fieldId));
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been asigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      switch (field) {
+      }
+      throw new IllegalStateException();
+    }
+
+    public boolean isSet(int fieldID) {
+      return isSet(_Fields.findByThriftIdOrThrow(fieldID));
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof finishFileUpload_result)
+        return this.equals((finishFileUpload_result)that);
+      return false;
+    }
+
+    public boolean equals(finishFileUpload_result that) {
+      if (that == null)
+        return false;
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      HashCodeBuilder builder = new HashCodeBuilder();
+
+      return builder.toHashCode();
+    }
+
+    public int compareTo(finishFileUpload_result other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+      finishFileUpload_result typedOther = (finishFileUpload_result)other;
+
+      return 0;
+    }
+
+    public void read(TProtocol iprot) throws TException {
+      TField field;
+      iprot.readStructBegin();
+      while (true)
+      {
+        field = iprot.readFieldBegin();
+        if (field.type == TType.STOP) { 
+          break;
+        }
+        switch (field.id) {
+          default:
+            TProtocolUtil.skip(iprot, field.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      validate();
+    }
+
+    public void write(TProtocol oprot) throws TException {
+      oprot.writeStructBegin(STRUCT_DESC);
+
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("finishFileUpload_result(");
+      boolean first = true;
+
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws TException {
+      // check for required fields
+    }
+
+  }
+
+  public static class beginFileDownload_args implements TBase<beginFileDownload_args, beginFileDownload_args._Fields>, java.io.Serializable, Cloneable   {
+    private static final TStruct STRUCT_DESC = new TStruct("beginFileDownload_args");
+
+    private static final TField FILE_FIELD_DESC = new TField("file", TType.STRING, (short)1);
+
+    private String file;
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements TFieldIdEnum {
+      FILE((short)1, "file");
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 1: // FILE
+            return FILE;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+
+    public static final Map<_Fields, FieldMetaData> metaDataMap;
+    static {
+      Map<_Fields, FieldMetaData> tmpMap = new EnumMap<_Fields, FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.FILE, new FieldMetaData("file", TFieldRequirementType.DEFAULT, 
+          new FieldValueMetaData(TType.STRING)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      FieldMetaData.addStructMetaDataMap(beginFileDownload_args.class, metaDataMap);
+    }
+
+    public beginFileDownload_args() {
+    }
+
+    public beginFileDownload_args(
+      String file)
+    {
+      this();
+      this.file = file;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public beginFileDownload_args(beginFileDownload_args other) {
+      if (other.is_set_file()) {
+        this.file = other.file;
+      }
+    }
+
+    public beginFileDownload_args deepCopy() {
+      return new beginFileDownload_args(this);
+    }
+
+    @Deprecated
+    public beginFileDownload_args clone() {
+      return new beginFileDownload_args(this);
+    }
+
+    public String get_file() {
+      return this.file;
+    }
+
+    public void set_file(String file) {
+      this.file = file;
+    }
+
+    public void unset_file() {
+      this.file = null;
+    }
+
+    /** Returns true if field file is set (has been asigned a value) and false otherwise */
+    public boolean is_set_file() {
+      return this.file != null;
+    }
+
+    public void set_file_isSet(boolean value) {
+      if (!value) {
+        this.file = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case FILE:
+        if (value == null) {
+          unset_file();
+        } else {
+          set_file((String)value);
+        }
+        break;
+
+      }
+    }
+
+    public void setFieldValue(int fieldID, Object value) {
+      setFieldValue(_Fields.findByThriftIdOrThrow(fieldID), value);
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case FILE:
+        return get_file();
+
+      }
+      throw new IllegalStateException();
+    }
+
+    public Object getFieldValue(int fieldId) {
+      return getFieldValue(_Fields.findByThriftIdOrThrow(fieldId));
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been asigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      switch (field) {
+      case FILE:
+        return is_set_file();
+      }
+      throw new IllegalStateException();
+    }
+
+    public boolean isSet(int fieldID) {
+      return isSet(_Fields.findByThriftIdOrThrow(fieldID));
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof beginFileDownload_args)
+        return this.equals((beginFileDownload_args)that);
+      return false;
+    }
+
+    public boolean equals(beginFileDownload_args that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_file = true && this.is_set_file();
+      boolean that_present_file = true && that.is_set_file();
+      if (this_present_file || that_present_file) {
+        if (!(this_present_file && that_present_file))
+          return false;
+        if (!this.file.equals(that.file))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      HashCodeBuilder builder = new HashCodeBuilder();
+
+      boolean present_file = true && (is_set_file());
+      builder.append(present_file);
+      if (present_file)
+        builder.append(file);
+
+      return builder.toHashCode();
+    }
+
+    public int compareTo(beginFileDownload_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+      beginFileDownload_args typedOther = (beginFileDownload_args)other;
+
+      lastComparison = Boolean.valueOf(is_set_file()).compareTo(typedOther.is_set_file());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_file()) {        lastComparison = TBaseHelper.compareTo(this.file, typedOther.file);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    public void read(TProtocol iprot) throws TException {
+      TField field;
+      iprot.readStructBegin();
+      while (true)
+      {
+        field = iprot.readFieldBegin();
+        if (field.type == TType.STOP) { 
+          break;
+        }
+        switch (field.id) {
+          case 1: // FILE
+            if (field.type == TType.STRING) {
+              this.file = iprot.readString();
+            } else { 
+              TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
+          default:
+            TProtocolUtil.skip(iprot, field.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      validate();
+    }
+
+    public void write(TProtocol oprot) throws TException {
+      validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (this.file != null) {
+        oprot.writeFieldBegin(FILE_FIELD_DESC);
+        oprot.writeString(this.file);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("beginFileDownload_args(");
+      boolean first = true;
+
+      sb.append("file:");
+      if (this.file == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.file);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws TException {
+      // check for required fields
+    }
+
+  }
+
+  public static class beginFileDownload_result implements TBase<beginFileDownload_result, beginFileDownload_result._Fields>, java.io.Serializable, Cloneable   {
+    private static final TStruct STRUCT_DESC = new TStruct("beginFileDownload_result");
+
+    private static final TField SUCCESS_FIELD_DESC = new TField("success", TType.STRING, (short)0);
+
+    private String success;
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements TFieldIdEnum {
+      SUCCESS((short)0, "success");
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 0: // SUCCESS
+            return SUCCESS;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+
+    public static final Map<_Fields, FieldMetaData> metaDataMap;
+    static {
+      Map<_Fields, FieldMetaData> tmpMap = new EnumMap<_Fields, FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.SUCCESS, new FieldMetaData("success", TFieldRequirementType.DEFAULT, 
+          new FieldValueMetaData(TType.STRING)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      FieldMetaData.addStructMetaDataMap(beginFileDownload_result.class, metaDataMap);
+    }
+
+    public beginFileDownload_result() {
+    }
+
+    public beginFileDownload_result(
+      String success)
+    {
+      this();
+      this.success = success;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public beginFileDownload_result(beginFileDownload_result other) {
+      if (other.is_set_success()) {
+        this.success = other.success;
+      }
+    }
+
+    public beginFileDownload_result deepCopy() {
+      return new beginFileDownload_result(this);
+    }
+
+    @Deprecated
+    public beginFileDownload_result clone() {
+      return new beginFileDownload_result(this);
+    }
+
+    public String get_success() {
+      return this.success;
+    }
+
+    public void set_success(String success) {
+      this.success = success;
+    }
+
+    public void unset_success() {
+      this.success = null;
+    }
+
+    /** Returns true if field success is set (has been asigned a value) and false otherwise */
+    public boolean is_set_success() {
+      return this.success != null;
+    }
+
+    public void set_success_isSet(boolean value) {
+      if (!value) {
+        this.success = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case SUCCESS:
+        if (value == null) {
+          unset_success();
+        } else {
+          set_success((String)value);
+        }
+        break;
+
+      }
+    }
+
+    public void setFieldValue(int fieldID, Object value) {
+      setFieldValue(_Fields.findByThriftIdOrThrow(fieldID), value);
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case SUCCESS:
+        return get_success();
+
+      }
+      throw new IllegalStateException();
+    }
+
+    public Object getFieldValue(int fieldId) {
+      return getFieldValue(_Fields.findByThriftIdOrThrow(fieldId));
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been asigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      switch (field) {
+      case SUCCESS:
+        return is_set_success();
+      }
+      throw new IllegalStateException();
+    }
+
+    public boolean isSet(int fieldID) {
+      return isSet(_Fields.findByThriftIdOrThrow(fieldID));
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof beginFileDownload_result)
+        return this.equals((beginFileDownload_result)that);
+      return false;
+    }
+
+    public boolean equals(beginFileDownload_result that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_success = true && this.is_set_success();
+      boolean that_present_success = true && that.is_set_success();
+      if (this_present_success || that_present_success) {
+        if (!(this_present_success && that_present_success))
+          return false;
+        if (!this.success.equals(that.success))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      HashCodeBuilder builder = new HashCodeBuilder();
+
+      boolean present_success = true && (is_set_success());
+      builder.append(present_success);
+      if (present_success)
+        builder.append(success);
+
+      return builder.toHashCode();
+    }
+
+    public int compareTo(beginFileDownload_result other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+      beginFileDownload_result typedOther = (beginFileDownload_result)other;
+
+      lastComparison = Boolean.valueOf(is_set_success()).compareTo(typedOther.is_set_success());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_success()) {        lastComparison = TBaseHelper.compareTo(this.success, typedOther.success);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    public void read(TProtocol iprot) throws TException {
+      TField field;
+      iprot.readStructBegin();
+      while (true)
+      {
+        field = iprot.readFieldBegin();
+        if (field.type == TType.STOP) { 
+          break;
+        }
+        switch (field.id) {
+          case 0: // SUCCESS
+            if (field.type == TType.STRING) {
+              this.success = iprot.readString();
+            } else { 
+              TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
+          default:
+            TProtocolUtil.skip(iprot, field.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      validate();
+    }
+
+    public void write(TProtocol oprot) throws TException {
+      oprot.writeStructBegin(STRUCT_DESC);
+
+      if (this.is_set_success()) {
+        oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
+        oprot.writeString(this.success);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("beginFileDownload_result(");
+      boolean first = true;
+
+      sb.append("success:");
+      if (this.success == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.success);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws TException {
+      // check for required fields
+    }
+
+  }
+
+  public static class downloadChunk_args implements TBase<downloadChunk_args, downloadChunk_args._Fields>, java.io.Serializable, Cloneable   {
+    private static final TStruct STRUCT_DESC = new TStruct("downloadChunk_args");
+
+    private static final TField ID_FIELD_DESC = new TField("id", TType.STRING, (short)1);
+
+    private String id;
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements TFieldIdEnum {
+      ID((short)1, "id");
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 1: // ID
+            return ID;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+
+    public static final Map<_Fields, FieldMetaData> metaDataMap;
+    static {
+      Map<_Fields, FieldMetaData> tmpMap = new EnumMap<_Fields, FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.ID, new FieldMetaData("id", TFieldRequirementType.DEFAULT, 
+          new FieldValueMetaData(TType.STRING)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      FieldMetaData.addStructMetaDataMap(downloadChunk_args.class, metaDataMap);
+    }
+
+    public downloadChunk_args() {
+    }
+
+    public downloadChunk_args(
+      String id)
+    {
+      this();
+      this.id = id;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public downloadChunk_args(downloadChunk_args other) {
+      if (other.is_set_id()) {
+        this.id = other.id;
+      }
+    }
+
+    public downloadChunk_args deepCopy() {
+      return new downloadChunk_args(this);
+    }
+
+    @Deprecated
+    public downloadChunk_args clone() {
+      return new downloadChunk_args(this);
+    }
+
+    public String get_id() {
+      return this.id;
+    }
+
+    public void set_id(String id) {
+      this.id = id;
+    }
+
+    public void unset_id() {
+      this.id = null;
+    }
+
+    /** Returns true if field id is set (has been asigned a value) and false otherwise */
+    public boolean is_set_id() {
+      return this.id != null;
+    }
+
+    public void set_id_isSet(boolean value) {
+      if (!value) {
+        this.id = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case ID:
+        if (value == null) {
+          unset_id();
+        } else {
+          set_id((String)value);
+        }
+        break;
+
+      }
+    }
+
+    public void setFieldValue(int fieldID, Object value) {
+      setFieldValue(_Fields.findByThriftIdOrThrow(fieldID), value);
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case ID:
+        return get_id();
+
+      }
+      throw new IllegalStateException();
+    }
+
+    public Object getFieldValue(int fieldId) {
+      return getFieldValue(_Fields.findByThriftIdOrThrow(fieldId));
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been asigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      switch (field) {
+      case ID:
+        return is_set_id();
+      }
+      throw new IllegalStateException();
+    }
+
+    public boolean isSet(int fieldID) {
+      return isSet(_Fields.findByThriftIdOrThrow(fieldID));
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof downloadChunk_args)
+        return this.equals((downloadChunk_args)that);
+      return false;
+    }
+
+    public boolean equals(downloadChunk_args that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_id = true && this.is_set_id();
+      boolean that_present_id = true && that.is_set_id();
+      if (this_present_id || that_present_id) {
+        if (!(this_present_id && that_present_id))
+          return false;
+        if (!this.id.equals(that.id))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      HashCodeBuilder builder = new HashCodeBuilder();
+
+      boolean present_id = true && (is_set_id());
+      builder.append(present_id);
+      if (present_id)
+        builder.append(id);
+
+      return builder.toHashCode();
+    }
+
+    public int compareTo(downloadChunk_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+      downloadChunk_args typedOther = (downloadChunk_args)other;
+
+      lastComparison = Boolean.valueOf(is_set_id()).compareTo(typedOther.is_set_id());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_id()) {        lastComparison = TBaseHelper.compareTo(this.id, typedOther.id);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    public void read(TProtocol iprot) throws TException {
+      TField field;
+      iprot.readStructBegin();
+      while (true)
+      {
+        field = iprot.readFieldBegin();
+        if (field.type == TType.STOP) { 
+          break;
+        }
+        switch (field.id) {
+          case 1: // ID
+            if (field.type == TType.STRING) {
+              this.id = iprot.readString();
+            } else { 
+              TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
+          default:
+            TProtocolUtil.skip(iprot, field.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      validate();
+    }
+
+    public void write(TProtocol oprot) throws TException {
+      validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (this.id != null) {
+        oprot.writeFieldBegin(ID_FIELD_DESC);
+        oprot.writeString(this.id);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("downloadChunk_args(");
+      boolean first = true;
+
+      sb.append("id:");
+      if (this.id == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.id);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws TException {
+      // check for required fields
+    }
+
+  }
+
+  public static class downloadChunk_result implements TBase<downloadChunk_result, downloadChunk_result._Fields>, java.io.Serializable, Cloneable   {
+    private static final TStruct STRUCT_DESC = new TStruct("downloadChunk_result");
+
+    private static final TField SUCCESS_FIELD_DESC = new TField("success", TType.STRING, (short)0);
+
+    private byte[] success;
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements TFieldIdEnum {
+      SUCCESS((short)0, "success");
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 0: // SUCCESS
+            return SUCCESS;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+
+    public static final Map<_Fields, FieldMetaData> metaDataMap;
+    static {
+      Map<_Fields, FieldMetaData> tmpMap = new EnumMap<_Fields, FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.SUCCESS, new FieldMetaData("success", TFieldRequirementType.DEFAULT, 
+          new FieldValueMetaData(TType.STRING)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      FieldMetaData.addStructMetaDataMap(downloadChunk_result.class, metaDataMap);
+    }
+
+    public downloadChunk_result() {
+    }
+
+    public downloadChunk_result(
+      byte[] success)
+    {
+      this();
+      this.success = success;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public downloadChunk_result(downloadChunk_result other) {
+      if (other.is_set_success()) {
+        this.success = new byte[other.success.length];
+        System.arraycopy(other.success, 0, success, 0, other.success.length);
+      }
+    }
+
+    public downloadChunk_result deepCopy() {
+      return new downloadChunk_result(this);
+    }
+
+    @Deprecated
+    public downloadChunk_result clone() {
+      return new downloadChunk_result(this);
+    }
+
+    public byte[] get_success() {
+      return this.success;
+    }
+
+    public void set_success(byte[] success) {
+      this.success = success;
+    }
+
+    public void unset_success() {
+      this.success = null;
+    }
+
+    /** Returns true if field success is set (has been asigned a value) and false otherwise */
+    public boolean is_set_success() {
+      return this.success != null;
+    }
+
+    public void set_success_isSet(boolean value) {
+      if (!value) {
+        this.success = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case SUCCESS:
+        if (value == null) {
+          unset_success();
+        } else {
+          set_success((byte[])value);
+        }
+        break;
+
+      }
+    }
+
+    public void setFieldValue(int fieldID, Object value) {
+      setFieldValue(_Fields.findByThriftIdOrThrow(fieldID), value);
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case SUCCESS:
+        return get_success();
+
+      }
+      throw new IllegalStateException();
+    }
+
+    public Object getFieldValue(int fieldId) {
+      return getFieldValue(_Fields.findByThriftIdOrThrow(fieldId));
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been asigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      switch (field) {
+      case SUCCESS:
+        return is_set_success();
+      }
+      throw new IllegalStateException();
+    }
+
+    public boolean isSet(int fieldID) {
+      return isSet(_Fields.findByThriftIdOrThrow(fieldID));
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof downloadChunk_result)
+        return this.equals((downloadChunk_result)that);
+      return false;
+    }
+
+    public boolean equals(downloadChunk_result that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_success = true && this.is_set_success();
+      boolean that_present_success = true && that.is_set_success();
+      if (this_present_success || that_present_success) {
+        if (!(this_present_success && that_present_success))
+          return false;
+        if (!java.util.Arrays.equals(this.success, that.success))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      HashCodeBuilder builder = new HashCodeBuilder();
+
+      boolean present_success = true && (is_set_success());
+      builder.append(present_success);
+      if (present_success)
+        builder.append(success);
+
+      return builder.toHashCode();
+    }
+
+    public int compareTo(downloadChunk_result other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+      downloadChunk_result typedOther = (downloadChunk_result)other;
+
+      lastComparison = Boolean.valueOf(is_set_success()).compareTo(typedOther.is_set_success());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_success()) {        lastComparison = TBaseHelper.compareTo(this.success, typedOther.success);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    public void read(TProtocol iprot) throws TException {
+      TField field;
+      iprot.readStructBegin();
+      while (true)
+      {
+        field = iprot.readFieldBegin();
+        if (field.type == TType.STOP) { 
+          break;
+        }
+        switch (field.id) {
+          case 0: // SUCCESS
+            if (field.type == TType.STRING) {
+              this.success = iprot.readBinary();
+            } else { 
+              TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
+          default:
+            TProtocolUtil.skip(iprot, field.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      validate();
+    }
+
+    public void write(TProtocol oprot) throws TException {
+      oprot.writeStructBegin(STRUCT_DESC);
+
+      if (this.is_set_success()) {
+        oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
+        oprot.writeBinary(this.success);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("downloadChunk_result(");
+      boolean first = true;
+
+      sb.append("success:");
+      if (this.success == null) {
+        sb.append("null");
+      } else {
+          int __success_size = Math.min(this.success.length, 128);
+          for (int i = 0; i < __success_size; i++) {
+            if (i != 0) sb.append(" ");
+            sb.append(Integer.toHexString(this.success[i]).length() > 1 ? Integer.toHexString(this.success[i]).substring(Integer.toHexString(this.success[i]).length() - 2).toUpperCase() : "0" + Integer.toHexString(this.success[i]).toUpperCase());
+          }
+          if (this.success.length > 128) sb.append(" ...");
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws TException {
+      // check for required fields
+    }
+
+  }
+
+  public static class getClusterInfo_args implements TBase<getClusterInfo_args, getClusterInfo_args._Fields>, java.io.Serializable, Cloneable   {
+    private static final TStruct STRUCT_DESC = new TStruct("getClusterInfo_args");
+
+
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements TFieldIdEnum {
+;
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+    public static final Map<_Fields, FieldMetaData> metaDataMap;
+    static {
+      Map<_Fields, FieldMetaData> tmpMap = new EnumMap<_Fields, FieldMetaData>(_Fields.class);
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      FieldMetaData.addStructMetaDataMap(getClusterInfo_args.class, metaDataMap);
+    }
+
+    public getClusterInfo_args() {
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public getClusterInfo_args(getClusterInfo_args other) {
+    }
+
+    public getClusterInfo_args deepCopy() {
+      return new getClusterInfo_args(this);
+    }
+
+    @Deprecated
+    public getClusterInfo_args clone() {
+      return new getClusterInfo_args(this);
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      }
+    }
+
+    public void setFieldValue(int fieldID, Object value) {
+      setFieldValue(_Fields.findByThriftIdOrThrow(fieldID), value);
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      }
+      throw new IllegalStateException();
+    }
+
+    public Object getFieldValue(int fieldId) {
+      return getFieldValue(_Fields.findByThriftIdOrThrow(fieldId));
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been asigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      switch (field) {
+      }
+      throw new IllegalStateException();
+    }
+
+    public boolean isSet(int fieldID) {
+      return isSet(_Fields.findByThriftIdOrThrow(fieldID));
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof getClusterInfo_args)
+        return this.equals((getClusterInfo_args)that);
+      return false;
+    }
+
+    public boolean equals(getClusterInfo_args that) {
+      if (that == null)
+        return false;
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      HashCodeBuilder builder = new HashCodeBuilder();
+
+      return builder.toHashCode();
+    }
+
+    public int compareTo(getClusterInfo_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+      getClusterInfo_args typedOther = (getClusterInfo_args)other;
+
+      return 0;
+    }
+
+    public void read(TProtocol iprot) throws TException {
+      TField field;
+      iprot.readStructBegin();
+      while (true)
+      {
+        field = iprot.readFieldBegin();
+        if (field.type == TType.STOP) { 
+          break;
+        }
+        switch (field.id) {
+          default:
+            TProtocolUtil.skip(iprot, field.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      validate();
+    }
+
+    public void write(TProtocol oprot) throws TException {
+      validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("getClusterInfo_args(");
+      boolean first = true;
+
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws TException {
+      // check for required fields
+    }
+
+  }
+
+  public static class getClusterInfo_result implements TBase<getClusterInfo_result, getClusterInfo_result._Fields>, java.io.Serializable, Cloneable   {
+    private static final TStruct STRUCT_DESC = new TStruct("getClusterInfo_result");
+
+    private static final TField SUCCESS_FIELD_DESC = new TField("success", TType.STRUCT, (short)0);
+
+    private ClusterSummary success;
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements TFieldIdEnum {
+      SUCCESS((short)0, "success");
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 0: // SUCCESS
+            return SUCCESS;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+
+    public static final Map<_Fields, FieldMetaData> metaDataMap;
+    static {
+      Map<_Fields, FieldMetaData> tmpMap = new EnumMap<_Fields, FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.SUCCESS, new FieldMetaData("success", TFieldRequirementType.DEFAULT, 
+          new StructMetaData(TType.STRUCT, ClusterSummary.class)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      FieldMetaData.addStructMetaDataMap(getClusterInfo_result.class, metaDataMap);
+    }
+
+    public getClusterInfo_result() {
+    }
+
+    public getClusterInfo_result(
+      ClusterSummary success)
+    {
+      this();
+      this.success = success;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public getClusterInfo_result(getClusterInfo_result other) {
+      if (other.is_set_success()) {
+        this.success = new ClusterSummary(other.success);
+      }
+    }
+
+    public getClusterInfo_result deepCopy() {
+      return new getClusterInfo_result(this);
+    }
+
+    @Deprecated
+    public getClusterInfo_result clone() {
+      return new getClusterInfo_result(this);
+    }
+
+    public ClusterSummary get_success() {
+      return this.success;
+    }
+
+    public void set_success(ClusterSummary success) {
+      this.success = success;
+    }
+
+    public void unset_success() {
+      this.success = null;
+    }
+
+    /** Returns true if field success is set (has been asigned a value) and false otherwise */
+    public boolean is_set_success() {
+      return this.success != null;
+    }
+
+    public void set_success_isSet(boolean value) {
+      if (!value) {
+        this.success = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case SUCCESS:
+        if (value == null) {
+          unset_success();
+        } else {
+          set_success((ClusterSummary)value);
+        }
+        break;
+
+      }
+    }
+
+    public void setFieldValue(int fieldID, Object value) {
+      setFieldValue(_Fields.findByThriftIdOrThrow(fieldID), value);
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case SUCCESS:
+        return get_success();
+
+      }
+      throw new IllegalStateException();
+    }
+
+    public Object getFieldValue(int fieldId) {
+      return getFieldValue(_Fields.findByThriftIdOrThrow(fieldId));
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been asigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      switch (field) {
+      case SUCCESS:
+        return is_set_success();
+      }
+      throw new IllegalStateException();
+    }
+
+    public boolean isSet(int fieldID) {
+      return isSet(_Fields.findByThriftIdOrThrow(fieldID));
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof getClusterInfo_result)
+        return this.equals((getClusterInfo_result)that);
+      return false;
+    }
+
+    public boolean equals(getClusterInfo_result that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_success = true && this.is_set_success();
+      boolean that_present_success = true && that.is_set_success();
+      if (this_present_success || that_present_success) {
+        if (!(this_present_success && that_present_success))
+          return false;
+        if (!this.success.equals(that.success))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      HashCodeBuilder builder = new HashCodeBuilder();
+
+      boolean present_success = true && (is_set_success());
+      builder.append(present_success);
+      if (present_success)
+        builder.append(success);
+
+      return builder.toHashCode();
+    }
+
+    public int compareTo(getClusterInfo_result other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+      getClusterInfo_result typedOther = (getClusterInfo_result)other;
+
+      lastComparison = Boolean.valueOf(is_set_success()).compareTo(typedOther.is_set_success());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_success()) {        lastComparison = TBaseHelper.compareTo(this.success, typedOther.success);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    public void read(TProtocol iprot) throws TException {
+      TField field;
+      iprot.readStructBegin();
+      while (true)
+      {
+        field = iprot.readFieldBegin();
+        if (field.type == TType.STOP) { 
+          break;
+        }
+        switch (field.id) {
+          case 0: // SUCCESS
+            if (field.type == TType.STRUCT) {
+              this.success = new ClusterSummary();
+              this.success.read(iprot);
+            } else { 
+              TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
+          default:
+            TProtocolUtil.skip(iprot, field.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      validate();
+    }
+
+    public void write(TProtocol oprot) throws TException {
+      oprot.writeStructBegin(STRUCT_DESC);
+
+      if (this.is_set_success()) {
+        oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
+        this.success.write(oprot);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("getClusterInfo_result(");
+      boolean first = true;
+
+      sb.append("success:");
+      if (this.success == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.success);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws TException {
+      // check for required fields
+    }
+
+  }
+
+  public static class getTopologyInfo_args implements TBase<getTopologyInfo_args, getTopologyInfo_args._Fields>, java.io.Serializable, Cloneable   {
+    private static final TStruct STRUCT_DESC = new TStruct("getTopologyInfo_args");
+
+    private static final TField ID_FIELD_DESC = new TField("id", TType.STRING, (short)1);
+
+    private String id;
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements TFieldIdEnum {
+      ID((short)1, "id");
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 1: // ID
+            return ID;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+
+    public static final Map<_Fields, FieldMetaData> metaDataMap;
+    static {
+      Map<_Fields, FieldMetaData> tmpMap = new EnumMap<_Fields, FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.ID, new FieldMetaData("id", TFieldRequirementType.DEFAULT, 
+          new FieldValueMetaData(TType.STRING)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      FieldMetaData.addStructMetaDataMap(getTopologyInfo_args.class, metaDataMap);
+    }
+
+    public getTopologyInfo_args() {
+    }
+
+    public getTopologyInfo_args(
+      String id)
+    {
+      this();
+      this.id = id;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public getTopologyInfo_args(getTopologyInfo_args other) {
+      if (other.is_set_id()) {
+        this.id = other.id;
+      }
+    }
+
+    public getTopologyInfo_args deepCopy() {
+      return new getTopologyInfo_args(this);
+    }
+
+    @Deprecated
+    public getTopologyInfo_args clone() {
+      return new getTopologyInfo_args(this);
+    }
+
+    public String get_id() {
+      return this.id;
+    }
+
+    public void set_id(String id) {
+      this.id = id;
+    }
+
+    public void unset_id() {
+      this.id = null;
+    }
+
+    /** Returns true if field id is set (has been asigned a value) and false otherwise */
+    public boolean is_set_id() {
+      return this.id != null;
+    }
+
+    public void set_id_isSet(boolean value) {
+      if (!value) {
+        this.id = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case ID:
+        if (value == null) {
+          unset_id();
+        } else {
+          set_id((String)value);
+        }
+        break;
+
+      }
+    }
+
+    public void setFieldValue(int fieldID, Object value) {
+      setFieldValue(_Fields.findByThriftIdOrThrow(fieldID), value);
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case ID:
+        return get_id();
+
+      }
+      throw new IllegalStateException();
+    }
+
+    public Object getFieldValue(int fieldId) {
+      return getFieldValue(_Fields.findByThriftIdOrThrow(fieldId));
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been asigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      switch (field) {
+      case ID:
+        return is_set_id();
+      }
+      throw new IllegalStateException();
+    }
+
+    public boolean isSet(int fieldID) {
+      return isSet(_Fields.findByThriftIdOrThrow(fieldID));
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof getTopologyInfo_args)
+        return this.equals((getTopologyInfo_args)that);
+      return false;
+    }
+
+    public boolean equals(getTopologyInfo_args that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_id = true && this.is_set_id();
+      boolean that_present_id = true && that.is_set_id();
+      if (this_present_id || that_present_id) {
+        if (!(this_present_id && that_present_id))
+          return false;
+        if (!this.id.equals(that.id))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      HashCodeBuilder builder = new HashCodeBuilder();
+
+      boolean present_id = true && (is_set_id());
+      builder.append(present_id);
+      if (present_id)
+        builder.append(id);
+
+      return builder.toHashCode();
+    }
+
+    public int compareTo(getTopologyInfo_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+      getTopologyInfo_args typedOther = (getTopologyInfo_args)other;
+
+      lastComparison = Boolean.valueOf(is_set_id()).compareTo(typedOther.is_set_id());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_id()) {        lastComparison = TBaseHelper.compareTo(this.id, typedOther.id);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    public void read(TProtocol iprot) throws TException {
+      TField field;
+      iprot.readStructBegin();
+      while (true)
+      {
+        field = iprot.readFieldBegin();
+        if (field.type == TType.STOP) { 
+          break;
+        }
+        switch (field.id) {
+          case 1: // ID
+            if (field.type == TType.STRING) {
+              this.id = iprot.readString();
+            } else { 
+              TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
+          default:
+            TProtocolUtil.skip(iprot, field.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      validate();
+    }
+
+    public void write(TProtocol oprot) throws TException {
+      validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (this.id != null) {
+        oprot.writeFieldBegin(ID_FIELD_DESC);
+        oprot.writeString(this.id);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("getTopologyInfo_args(");
+      boolean first = true;
+
+      sb.append("id:");
+      if (this.id == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.id);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws TException {
+      // check for required fields
+    }
+
+  }
+
+  public static class getTopologyInfo_result implements TBase<getTopologyInfo_result, getTopologyInfo_result._Fields>, java.io.Serializable, Cloneable   {
+    private static final TStruct STRUCT_DESC = new TStruct("getTopologyInfo_result");
+
+    private static final TField SUCCESS_FIELD_DESC = new TField("success", TType.STRUCT, (short)0);
+    private static final TField E_FIELD_DESC = new TField("e", TType.STRUCT, (short)1);
+
+    private TopologyInfo success;
+    private NotAliveException e;
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements TFieldIdEnum {
+      SUCCESS((short)0, "success"),
+      E((short)1, "e");
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 0: // SUCCESS
+            return SUCCESS;
+          case 1: // E
+            return E;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+
+    public static final Map<_Fields, FieldMetaData> metaDataMap;
+    static {
+      Map<_Fields, FieldMetaData> tmpMap = new EnumMap<_Fields, FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.SUCCESS, new FieldMetaData("success", TFieldRequirementType.DEFAULT, 
+          new StructMetaData(TType.STRUCT, TopologyInfo.class)));
+      tmpMap.put(_Fields.E, new FieldMetaData("e", TFieldRequirementType.DEFAULT, 
+          new FieldValueMetaData(TType.STRUCT)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      FieldMetaData.addStructMetaDataMap(getTopologyInfo_result.class, metaDataMap);
+    }
+
+    public getTopologyInfo_result() {
+    }
+
+    public getTopologyInfo_result(
+      TopologyInfo success,
+      NotAliveException e)
+    {
+      this();
+      this.success = success;
+      this.e = e;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public getTopologyInfo_result(getTopologyInfo_result other) {
+      if (other.is_set_success()) {
+        this.success = new TopologyInfo(other.success);
+      }
+      if (other.is_set_e()) {
+        this.e = new NotAliveException(other.e);
+      }
+    }
+
+    public getTopologyInfo_result deepCopy() {
+      return new getTopologyInfo_result(this);
+    }
+
+    @Deprecated
+    public getTopologyInfo_result clone() {
+      return new getTopologyInfo_result(this);
+    }
+
+    public TopologyInfo get_success() {
+      return this.success;
+    }
+
+    public void set_success(TopologyInfo success) {
+      this.success = success;
+    }
+
+    public void unset_success() {
+      this.success = null;
+    }
+
+    /** Returns true if field success is set (has been asigned a value) and false otherwise */
+    public boolean is_set_success() {
+      return this.success != null;
+    }
+
+    public void set_success_isSet(boolean value) {
+      if (!value) {
+        this.success = null;
+      }
+    }
+
+    public NotAliveException get_e() {
+      return this.e;
+    }
+
+    public void set_e(NotAliveException e) {
+      this.e = e;
+    }
+
+    public void unset_e() {
+      this.e = null;
+    }
+
+    /** Returns true if field e is set (has been asigned a value) and false otherwise */
+    public boolean is_set_e() {
+      return this.e != null;
+    }
+
+    public void set_e_isSet(boolean value) {
+      if (!value) {
+        this.e = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case SUCCESS:
+        if (value == null) {
+          unset_success();
+        } else {
+          set_success((TopologyInfo)value);
+        }
+        break;
+
+      case E:
+        if (value == null) {
+          unset_e();
+        } else {
+          set_e((NotAliveException)value);
+        }
+        break;
+
+      }
+    }
+
+    public void setFieldValue(int fieldID, Object value) {
+      setFieldValue(_Fields.findByThriftIdOrThrow(fieldID), value);
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case SUCCESS:
+        return get_success();
+
+      case E:
+        return get_e();
+
+      }
+      throw new IllegalStateException();
+    }
+
+    public Object getFieldValue(int fieldId) {
+      return getFieldValue(_Fields.findByThriftIdOrThrow(fieldId));
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been asigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      switch (field) {
+      case SUCCESS:
+        return is_set_success();
+      case E:
+        return is_set_e();
+      }
+      throw new IllegalStateException();
+    }
+
+    public boolean isSet(int fieldID) {
+      return isSet(_Fields.findByThriftIdOrThrow(fieldID));
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof getTopologyInfo_result)
+        return this.equals((getTopologyInfo_result)that);
+      return false;
+    }
+
+    public boolean equals(getTopologyInfo_result that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_success = true && this.is_set_success();
+      boolean that_present_success = true && that.is_set_success();
+      if (this_present_success || that_present_success) {
+        if (!(this_present_success && that_present_success))
+          return false;
+        if (!this.success.equals(that.success))
+          return false;
+      }
+
+      boolean this_present_e = true && this.is_set_e();
+      boolean that_present_e = true && that.is_set_e();
+      if (this_present_e || that_present_e) {
+        if (!(this_present_e && that_present_e))
+          return false;
+        if (!this.e.equals(that.e))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      HashCodeBuilder builder = new HashCodeBuilder();
+
+      boolean present_success = true && (is_set_success());
+      builder.append(present_success);
+      if (present_success)
+        builder.append(success);
+
+      boolean present_e = true && (is_set_e());
+      builder.append(present_e);
+      if (present_e)
+        builder.append(e);
+
+      return builder.toHashCode();
+    }
+
+    public int compareTo(getTopologyInfo_result other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+      getTopologyInfo_result typedOther = (getTopologyInfo_result)other;
+
+      lastComparison = Boolean.valueOf(is_set_success()).compareTo(typedOther.is_set_success());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_success()) {        lastComparison = TBaseHelper.compareTo(this.success, typedOther.success);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(is_set_e()).compareTo(typedOther.is_set_e());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_e()) {        lastComparison = TBaseHelper.compareTo(this.e, typedOther.e);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    public void read(TProtocol iprot) throws TException {
+      TField field;
+      iprot.readStructBegin();
+      while (true)
+      {
+        field = iprot.readFieldBegin();
+        if (field.type == TType.STOP) { 
+          break;
+        }
+        switch (field.id) {
+          case 0: // SUCCESS
+            if (field.type == TType.STRUCT) {
+              this.success = new TopologyInfo();
+              this.success.read(iprot);
+            } else { 
+              TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
+          case 1: // E
+            if (field.type == TType.STRUCT) {
+              this.e = new NotAliveException();
+              this.e.read(iprot);
+            } else { 
+              TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
+          default:
+            TProtocolUtil.skip(iprot, field.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      validate();
+    }
+
+    public void write(TProtocol oprot) throws TException {
+      oprot.writeStructBegin(STRUCT_DESC);
+
+      if (this.is_set_success()) {
+        oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
+        this.success.write(oprot);
+        oprot.writeFieldEnd();
+      } else if (this.is_set_e()) {
+        oprot.writeFieldBegin(E_FIELD_DESC);
+        this.e.write(oprot);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("getTopologyInfo_result(");
+      boolean first = true;
+
+      sb.append("success:");
+      if (this.success == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.success);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("e:");
+      if (this.e == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.e);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws TException {
+      // check for required fields
+    }
+
+  }
+
+  public static class getTopologyConf_args implements TBase<getTopologyConf_args, getTopologyConf_args._Fields>, java.io.Serializable, Cloneable   {
+    private static final TStruct STRUCT_DESC = new TStruct("getTopologyConf_args");
+
+    private static final TField ID_FIELD_DESC = new TField("id", TType.STRING, (short)1);
+
+    private String id;
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements TFieldIdEnum {
+      ID((short)1, "id");
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 1: // ID
+            return ID;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+
+    public static final Map<_Fields, FieldMetaData> metaDataMap;
+    static {
+      Map<_Fields, FieldMetaData> tmpMap = new EnumMap<_Fields, FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.ID, new FieldMetaData("id", TFieldRequirementType.DEFAULT, 
+          new FieldValueMetaData(TType.STRING)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      FieldMetaData.addStructMetaDataMap(getTopologyConf_args.class, metaDataMap);
+    }
+
+    public getTopologyConf_args() {
+    }
+
+    public getTopologyConf_args(
+      String id)
+    {
+      this();
+      this.id = id;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public getTopologyConf_args(getTopologyConf_args other) {
+      if (other.is_set_id()) {
+        this.id = other.id;
+      }
+    }
+
+    public getTopologyConf_args deepCopy() {
+      return new getTopologyConf_args(this);
+    }
+
+    @Deprecated
+    public getTopologyConf_args clone() {
+      return new getTopologyConf_args(this);
+    }
+
+    public String get_id() {
+      return this.id;
+    }
+
+    public void set_id(String id) {
+      this.id = id;
+    }
+
+    public void unset_id() {
+      this.id = null;
+    }
+
+    /** Returns true if field id is set (has been asigned a value) and false otherwise */
+    public boolean is_set_id() {
+      return this.id != null;
+    }
+
+    public void set_id_isSet(boolean value) {
+      if (!value) {
+        this.id = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case ID:
+        if (value == null) {
+          unset_id();
+        } else {
+          set_id((String)value);
+        }
+        break;
+
+      }
+    }
+
+    public void setFieldValue(int fieldID, Object value) {
+      setFieldValue(_Fields.findByThriftIdOrThrow(fieldID), value);
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case ID:
+        return get_id();
+
+      }
+      throw new IllegalStateException();
+    }
+
+    public Object getFieldValue(int fieldId) {
+      return getFieldValue(_Fields.findByThriftIdOrThrow(fieldId));
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been asigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      switch (field) {
+      case ID:
+        return is_set_id();
+      }
+      throw new IllegalStateException();
+    }
+
+    public boolean isSet(int fieldID) {
+      return isSet(_Fields.findByThriftIdOrThrow(fieldID));
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof getTopologyConf_args)
+        return this.equals((getTopologyConf_args)that);
+      return false;
+    }
+
+    public boolean equals(getTopologyConf_args that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_id = true && this.is_set_id();
+      boolean that_present_id = true && that.is_set_id();
+      if (this_present_id || that_present_id) {
+        if (!(this_present_id && that_present_id))
+          return false;
+        if (!this.id.equals(that.id))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      HashCodeBuilder builder = new HashCodeBuilder();
+
+      boolean present_id = true && (is_set_id());
+      builder.append(present_id);
+      if (present_id)
+        builder.append(id);
+
+      return builder.toHashCode();
+    }
+
+    public int compareTo(getTopologyConf_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+      getTopologyConf_args typedOther = (getTopologyConf_args)other;
+
+      lastComparison = Boolean.valueOf(is_set_id()).compareTo(typedOther.is_set_id());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_id()) {        lastComparison = TBaseHelper.compareTo(this.id, typedOther.id);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    public void read(TProtocol iprot) throws TException {
+      TField field;
+      iprot.readStructBegin();
+      while (true)
+      {
+        field = iprot.readFieldBegin();
+        if (field.type == TType.STOP) { 
+          break;
+        }
+        switch (field.id) {
+          case 1: // ID
+            if (field.type == TType.STRING) {
+              this.id = iprot.readString();
+            } else { 
+              TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
+          default:
+            TProtocolUtil.skip(iprot, field.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      validate();
+    }
+
+    public void write(TProtocol oprot) throws TException {
+      validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (this.id != null) {
+        oprot.writeFieldBegin(ID_FIELD_DESC);
+        oprot.writeString(this.id);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("getTopologyConf_args(");
+      boolean first = true;
+
+      sb.append("id:");
+      if (this.id == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.id);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws TException {
+      // check for required fields
+    }
+
+  }
+
+  public static class getTopologyConf_result implements TBase<getTopologyConf_result, getTopologyConf_result._Fields>, java.io.Serializable, Cloneable   {
+    private static final TStruct STRUCT_DESC = new TStruct("getTopologyConf_result");
+
+    private static final TField SUCCESS_FIELD_DESC = new TField("success", TType.STRING, (short)0);
+    private static final TField E_FIELD_DESC = new TField("e", TType.STRUCT, (short)1);
+
+    private String success;
+    private NotAliveException e;
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements TFieldIdEnum {
+      SUCCESS((short)0, "success"),
+      E((short)1, "e");
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 0: // SUCCESS
+            return SUCCESS;
+          case 1: // E
+            return E;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+
+    public static final Map<_Fields, FieldMetaData> metaDataMap;
+    static {
+      Map<_Fields, FieldMetaData> tmpMap = new EnumMap<_Fields, FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.SUCCESS, new FieldMetaData("success", TFieldRequirementType.DEFAULT, 
+          new FieldValueMetaData(TType.STRING)));
+      tmpMap.put(_Fields.E, new FieldMetaData("e", TFieldRequirementType.DEFAULT, 
+          new FieldValueMetaData(TType.STRUCT)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      FieldMetaData.addStructMetaDataMap(getTopologyConf_result.class, metaDataMap);
+    }
+
+    public getTopologyConf_result() {
+    }
+
+    public getTopologyConf_result(
+      String success,
+      NotAliveException e)
+    {
+      this();
+      this.success = success;
+      this.e = e;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public getTopologyConf_result(getTopologyConf_result other) {
+      if (other.is_set_success()) {
+        this.success = other.success;
+      }
+      if (other.is_set_e()) {
+        this.e = new NotAliveException(other.e);
+      }
+    }
+
+    public getTopologyConf_result deepCopy() {
+      return new getTopologyConf_result(this);
+    }
+
+    @Deprecated
+    public getTopologyConf_result clone() {
+      return new getTopologyConf_result(this);
+    }
+
+    public String get_success() {
+      return this.success;
+    }
+
+    public void set_success(String success) {
+      this.success = success;
+    }
+
+    public void unset_success() {
+      this.success = null;
+    }
+
+    /** Returns true if field success is set (has been asigned a value) and false otherwise */
+    public boolean is_set_success() {
+      return this.success != null;
+    }
+
+    public void set_success_isSet(boolean value) {
+      if (!value) {
+        this.success = null;
+      }
+    }
+
+    public NotAliveException get_e() {
+      return this.e;
+    }
+
+    public void set_e(NotAliveException e) {
+      this.e = e;
+    }
+
+    public void unset_e() {
+      this.e = null;
+    }
+
+    /** Returns true if field e is set (has been asigned a value) and false otherwise */
+    public boolean is_set_e() {
+      return this.e != null;
+    }
+
+    public void set_e_isSet(boolean value) {
+      if (!value) {
+        this.e = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case SUCCESS:
+        if (value == null) {
+          unset_success();
+        } else {
+          set_success((String)value);
+        }
+        break;
+
+      case E:
+        if (value == null) {
+          unset_e();
+        } else {
+          set_e((NotAliveException)value);
+        }
+        break;
+
+      }
+    }
+
+    public void setFieldValue(int fieldID, Object value) {
+      setFieldValue(_Fields.findByThriftIdOrThrow(fieldID), value);
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case SUCCESS:
+        return get_success();
+
+      case E:
+        return get_e();
+
+      }
+      throw new IllegalStateException();
+    }
+
+    public Object getFieldValue(int fieldId) {
+      return getFieldValue(_Fields.findByThriftIdOrThrow(fieldId));
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been asigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      switch (field) {
+      case SUCCESS:
+        return is_set_success();
+      case E:
+        return is_set_e();
+      }
+      throw new IllegalStateException();
+    }
+
+    public boolean isSet(int fieldID) {
+      return isSet(_Fields.findByThriftIdOrThrow(fieldID));
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof getTopologyConf_result)
+        return this.equals((getTopologyConf_result)that);
+      return false;
+    }
+
+    public boolean equals(getTopologyConf_result that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_success = true && this.is_set_success();
+      boolean that_present_success = true && that.is_set_success();
+      if (this_present_success || that_present_success) {
+        if (!(this_present_success && that_present_success))
+          return false;
+        if (!this.success.equals(that.success))
+          return false;
+      }
+
+      boolean this_present_e = true && this.is_set_e();
+      boolean that_present_e = true && that.is_set_e();
+      if (this_present_e || that_present_e) {
+        if (!(this_present_e && that_present_e))
+          return false;
+        if (!this.e.equals(that.e))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      HashCodeBuilder builder = new HashCodeBuilder();
+
+      boolean present_success = true && (is_set_success());
+      builder.append(present_success);
+      if (present_success)
+        builder.append(success);
+
+      boolean present_e = true && (is_set_e());
+      builder.append(present_e);
+      if (present_e)
+        builder.append(e);
+
+      return builder.toHashCode();
+    }
+
+    public int compareTo(getTopologyConf_result other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+      getTopologyConf_result typedOther = (getTopologyConf_result)other;
+
+      lastComparison = Boolean.valueOf(is_set_success()).compareTo(typedOther.is_set_success());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_success()) {        lastComparison = TBaseHelper.compareTo(this.success, typedOther.success);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(is_set_e()).compareTo(typedOther.is_set_e());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_e()) {        lastComparison = TBaseHelper.compareTo(this.e, typedOther.e);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    public void read(TProtocol iprot) throws TException {
+      TField field;
+      iprot.readStructBegin();
+      while (true)
+      {
+        field = iprot.readFieldBegin();
+        if (field.type == TType.STOP) { 
+          break;
+        }
+        switch (field.id) {
+          case 0: // SUCCESS
+            if (field.type == TType.STRING) {
+              this.success = iprot.readString();
+            } else { 
+              TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
+          case 1: // E
+            if (field.type == TType.STRUCT) {
+              this.e = new NotAliveException();
+              this.e.read(iprot);
+            } else { 
+              TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
+          default:
+            TProtocolUtil.skip(iprot, field.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      validate();
+    }
+
+    public void write(TProtocol oprot) throws TException {
+      oprot.writeStructBegin(STRUCT_DESC);
+
+      if (this.is_set_success()) {
+        oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
+        oprot.writeString(this.success);
+        oprot.writeFieldEnd();
+      } else if (this.is_set_e()) {
+        oprot.writeFieldBegin(E_FIELD_DESC);
+        this.e.write(oprot);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("getTopologyConf_result(");
+      boolean first = true;
+
+      sb.append("success:");
+      if (this.success == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.success);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("e:");
+      if (this.e == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.e);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws TException {
+      // check for required fields
+    }
+
+  }
+
+  public static class getTopology_args implements TBase<getTopology_args, getTopology_args._Fields>, java.io.Serializable, Cloneable   {
+    private static final TStruct STRUCT_DESC = new TStruct("getTopology_args");
+
+    private static final TField ID_FIELD_DESC = new TField("id", TType.STRING, (short)1);
+
+    private String id;
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements TFieldIdEnum {
+      ID((short)1, "id");
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 1: // ID
+            return ID;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+
+    public static final Map<_Fields, FieldMetaData> metaDataMap;
+    static {
+      Map<_Fields, FieldMetaData> tmpMap = new EnumMap<_Fields, FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.ID, new FieldMetaData("id", TFieldRequirementType.DEFAULT, 
+          new FieldValueMetaData(TType.STRING)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      FieldMetaData.addStructMetaDataMap(getTopology_args.class, metaDataMap);
+    }
+
+    public getTopology_args() {
+    }
+
+    public getTopology_args(
+      String id)
+    {
+      this();
+      this.id = id;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public getTopology_args(getTopology_args other) {
+      if (other.is_set_id()) {
+        this.id = other.id;
+      }
+    }
+
+    public getTopology_args deepCopy() {
+      return new getTopology_args(this);
+    }
+
+    @Deprecated
+    public getTopology_args clone() {
+      return new getTopology_args(this);
+    }
+
+    public String get_id() {
+      return this.id;
+    }
+
+    public void set_id(String id) {
+      this.id = id;
+    }
+
+    public void unset_id() {
+      this.id = null;
+    }
+
+    /** Returns true if field id is set (has been asigned a value) and false otherwise */
+    public boolean is_set_id() {
+      return this.id != null;
+    }
+
+    public void set_id_isSet(boolean value) {
+      if (!value) {
+        this.id = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case ID:
+        if (value == null) {
+          unset_id();
+        } else {
+          set_id((String)value);
+        }
+        break;
+
+      }
+    }
+
+    public void setFieldValue(int fieldID, Object value) {
+      setFieldValue(_Fields.findByThriftIdOrThrow(fieldID), value);
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case ID:
+        return get_id();
+
+      }
+      throw new IllegalStateException();
+    }
+
+    public Object getFieldValue(int fieldId) {
+      return getFieldValue(_Fields.findByThriftIdOrThrow(fieldId));
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been asigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      switch (field) {
+      case ID:
+        return is_set_id();
+      }
+      throw new IllegalStateException();
+    }
+
+    public boolean isSet(int fieldID) {
+      return isSet(_Fields.findByThriftIdOrThrow(fieldID));
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof getTopology_args)
+        return this.equals((getTopology_args)that);
+      return false;
+    }
+
+    public boolean equals(getTopology_args that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_id = true && this.is_set_id();
+      boolean that_present_id = true && that.is_set_id();
+      if (this_present_id || that_present_id) {
+        if (!(this_present_id && that_present_id))
+          return false;
+        if (!this.id.equals(that.id))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      HashCodeBuilder builder = new HashCodeBuilder();
+
+      boolean present_id = true && (is_set_id());
+      builder.append(present_id);
+      if (present_id)
+        builder.append(id);
+
+      return builder.toHashCode();
+    }
+
+    public int compareTo(getTopology_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+      getTopology_args typedOther = (getTopology_args)other;
+
+      lastComparison = Boolean.valueOf(is_set_id()).compareTo(typedOther.is_set_id());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_id()) {        lastComparison = TBaseHelper.compareTo(this.id, typedOther.id);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    public void read(TProtocol iprot) throws TException {
+      TField field;
+      iprot.readStructBegin();
+      while (true)
+      {
+        field = iprot.readFieldBegin();
+        if (field.type == TType.STOP) { 
+          break;
+        }
+        switch (field.id) {
+          case 1: // ID
+            if (field.type == TType.STRING) {
+              this.id = iprot.readString();
+            } else { 
+              TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
+          default:
+            TProtocolUtil.skip(iprot, field.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      validate();
+    }
+
+    public void write(TProtocol oprot) throws TException {
+      validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (this.id != null) {
+        oprot.writeFieldBegin(ID_FIELD_DESC);
+        oprot.writeString(this.id);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("getTopology_args(");
+      boolean first = true;
+
+      sb.append("id:");
+      if (this.id == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.id);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws TException {
+      // check for required fields
+    }
+
+  }
+
+  public static class getTopology_result implements TBase<getTopology_result, getTopology_result._Fields>, java.io.Serializable, Cloneable   {
+    private static final TStruct STRUCT_DESC = new TStruct("getTopology_result");
+
+    private static final TField SUCCESS_FIELD_DESC = new TField("success", TType.STRUCT, (short)0);
+    private static final TField E_FIELD_DESC = new TField("e", TType.STRUCT, (short)1);
+
+    private StormTopology success;
+    private NotAliveException e;
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements TFieldIdEnum {
+      SUCCESS((short)0, "success"),
+      E((short)1, "e");
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 0: // SUCCESS
+            return SUCCESS;
+          case 1: // E
+            return E;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+
+    public static final Map<_Fields, FieldMetaData> metaDataMap;
+    static {
+      Map<_Fields, FieldMetaData> tmpMap = new EnumMap<_Fields, FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.SUCCESS, new FieldMetaData("success", TFieldRequirementType.DEFAULT, 
+          new StructMetaData(TType.STRUCT, StormTopology.class)));
+      tmpMap.put(_Fields.E, new FieldMetaData("e", TFieldRequirementType.DEFAULT, 
+          new FieldValueMetaData(TType.STRUCT)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      FieldMetaData.addStructMetaDataMap(getTopology_result.class, metaDataMap);
+    }
+
+    public getTopology_result() {
+    }
+
+    public getTopology_result(
+      StormTopology success,
+      NotAliveException e)
+    {
+      this();
+      this.success = success;
+      this.e = e;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public getTopology_result(getTopology_result other) {
+      if (other.is_set_success()) {
+        this.success = new StormTopology(other.success);
+      }
+      if (other.is_set_e()) {
+        this.e = new NotAliveException(other.e);
+      }
+    }
+
+    public getTopology_result deepCopy() {
+      return new getTopology_result(this);
+    }
+
+    @Deprecated
+    public getTopology_result clone() {
+      return new getTopology_result(this);
+    }
+
+    public StormTopology get_success() {
+      return this.success;
+    }
+
+    public void set_success(StormTopology success) {
+      this.success = success;
+    }
+
+    public void unset_success() {
+      this.success = null;
+    }
+
+    /** Returns true if field success is set (has been asigned a value) and false otherwise */
+    public boolean is_set_success() {
+      return this.success != null;
+    }
+
+    public void set_success_isSet(boolean value) {
+      if (!value) {
+        this.success = null;
+      }
+    }
+
+    public NotAliveException get_e() {
+      return this.e;
+    }
+
+    public void set_e(NotAliveException e) {
+      this.e = e;
+    }
+
+    public void unset_e() {
+      this.e = null;
+    }
+
+    /** Returns true if field e is set (has been asigned a value) and false otherwise */
+    public boolean is_set_e() {
+      return this.e != null;
+    }
+
+    public void set_e_isSet(boolean value) {
+      if (!value) {
+        this.e = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case SUCCESS:
+        if (value == null) {
+          unset_success();
+        } else {
+          set_success((StormTopology)value);
+        }
+        break;
+
+      case E:
+        if (value == null) {
+          unset_e();
+        } else {
+          set_e((NotAliveException)value);
+        }
+        break;
+
+      }
+    }
+
+    public void setFieldValue(int fieldID, Object value) {
+      setFieldValue(_Fields.findByThriftIdOrThrow(fieldID), value);
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case SUCCESS:
+        return get_success();
+
+      case E:
+        return get_e();
+
+      }
+      throw new IllegalStateException();
+    }
+
+    public Object getFieldValue(int fieldId) {
+      return getFieldValue(_Fields.findByThriftIdOrThrow(fieldId));
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been asigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      switch (field) {
+      case SUCCESS:
+        return is_set_success();
+      case E:
+        return is_set_e();
+      }
+      throw new IllegalStateException();
+    }
+
+    public boolean isSet(int fieldID) {
+      return isSet(_Fields.findByThriftIdOrThrow(fieldID));
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof getTopology_result)
+        return this.equals((getTopology_result)that);
+      return false;
+    }
+
+    public boolean equals(getTopology_result that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_success = true && this.is_set_success();
+      boolean that_present_success = true && that.is_set_success();
+      if (this_present_success || that_present_success) {
+        if (!(this_present_success && that_present_success))
+          return false;
+        if (!this.success.equals(that.success))
+          return false;
+      }
+
+      boolean this_present_e = true && this.is_set_e();
+      boolean that_present_e = true && that.is_set_e();
+      if (this_present_e || that_present_e) {
+        if (!(this_present_e && that_present_e))
+          return false;
+        if (!this.e.equals(that.e))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      HashCodeBuilder builder = new HashCodeBuilder();
+
+      boolean present_success = true && (is_set_success());
+      builder.append(present_success);
+      if (present_success)
+        builder.append(success);
+
+      boolean present_e = true && (is_set_e());
+      builder.append(present_e);
+      if (present_e)
+        builder.append(e);
+
+      return builder.toHashCode();
+    }
+
+    public int compareTo(getTopology_result other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+      getTopology_result typedOther = (getTopology_result)other;
+
+      lastComparison = Boolean.valueOf(is_set_success()).compareTo(typedOther.is_set_success());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_success()) {        lastComparison = TBaseHelper.compareTo(this.success, typedOther.success);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(is_set_e()).compareTo(typedOther.is_set_e());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (is_set_e()) {        lastComparison = TBaseHelper.compareTo(this.e, typedOther.e);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    public void read(TProtocol iprot) throws TException {
+      TField field;
+      iprot.readStructBegin();
+      while (true)
+      {
+        field = iprot.readFieldBegin();
+        if (field.type == TType.STOP) { 
+          break;
+        }
+        switch (field.id) {
+          case 0: // SUCCESS
+            if (field.type == TType.STRUCT) {
+              this.success = new StormTopology();
+              this.success.read(iprot);
+            } else { 
+              TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
+          case 1: // E
+            if (field.type == TType.STRUCT) {
+              this.e = new NotAliveException();
+              this.e.read(iprot);
+            } else { 
+              TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
+          default:
+            TProtocolUtil.skip(iprot, field.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      validate();
+    }
+
+    public void write(TProtocol oprot) throws TException {
+      oprot.writeStructBegin(STRUCT_DESC);
+
+      if (this.is_set_success()) {
+        oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
+        this.success.write(oprot);
+        oprot.writeFieldEnd();
+      } else if (this.is_set_e()) {
+        oprot.writeFieldBegin(E_FIELD_DESC);
+        this.e.write(oprot);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("getTopology_result(");
+      boolean first = true;
+
+      sb.append("success:");
+      if (this.success == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.success);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("e:");
+      if (this.e == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.e);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws TException {
+      // check for required fields
+    }
+
+  }
+
+}
diff --git a/src/jvm/backtype/storm/generated/NotAliveException.java b/src/jvm/backtype/storm/generated/NotAliveException.java
new file mode 100644
index 0000000..3252b94
--- /dev/null
+++ b/src/jvm/backtype/storm/generated/NotAliveException.java
@@ -0,0 +1,320 @@
+/**
+ * Autogenerated by Thrift
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ */
+package backtype.storm.generated;
+
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.util.Arrays;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.thrift.*;
+import org.apache.thrift.async.*;
+import org.apache.thrift.meta_data.*;
+import org.apache.thrift.transport.*;
+import org.apache.thrift.protocol.*;
+
+public class NotAliveException extends Exception implements TBase<NotAliveException, NotAliveException._Fields>, java.io.Serializable, Cloneable {
+  private static final TStruct STRUCT_DESC = new TStruct("NotAliveException");
+
+  private static final TField MSG_FIELD_DESC = new TField("msg", TType.STRING, (short)1);
+
+  private String msg;
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements TFieldIdEnum {
+    MSG((short)1, "msg");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // MSG
+          return MSG;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+
+  public static final Map<_Fields, FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, FieldMetaData> tmpMap = new EnumMap<_Fields, FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.MSG, new FieldMetaData("msg", TFieldRequirementType.REQUIRED, 
+        new FieldValueMetaData(TType.STRING)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    FieldMetaData.addStructMetaDataMap(NotAliveException.class, metaDataMap);
+  }
+
+  public NotAliveException() {
+  }
+
+  public NotAliveException(
+    String msg)
+  {
+    this();
+    this.msg = msg;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public NotAliveException(NotAliveException other) {
+    if (other.is_set_msg()) {
+      this.msg = other.msg;
+    }
+  }
+
+  public NotAliveException deepCopy() {
+    return new NotAliveException(this);
+  }
+
+  @Deprecated
+  public NotAliveException clone() {
+    return new NotAliveException(this);
+  }
+
+  public String get_msg() {
+    return this.msg;
+  }
+
+  public void set_msg(String msg) {
+    this.msg = msg;
+  }
+
+  public void unset_msg() {
+    this.msg = null;
+  }
+
+  /** Returns true if field msg is set (has been asigned a value) and false otherwise */
+  public boolean is_set_msg() {
+    return this.msg != null;
+  }
+
+  public void set_msg_isSet(boolean value) {
+    if (!value) {
+      this.msg = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case MSG:
+      if (value == null) {
+        unset_msg();
+      } else {
+        set_msg((String)value);
+      }
+      break;
+
+    }
+  }
+
+  public void setFieldValue(int fieldID, Object value) {
+    setFieldValue(_Fields.findByThriftIdOrThrow(fieldID), value);
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case MSG:
+      return get_msg();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  public Object getFieldValue(int fieldId) {
+    return getFieldValue(_Fields.findByThriftIdOrThrow(fieldId));
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been asigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    switch (field) {
+    case MSG:
+      return is_set_msg();
+    }
+    throw new IllegalStateException();
+  }
+
+  public boolean isSet(int fieldID) {
+    return isSet(_Fields.findByThriftIdOrThrow(fieldID));
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof NotAliveException)
+      return this.equals((NotAliveException)that);
+    return false;
+  }
+
+  public boolean equals(NotAliveException that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_msg = true && this.is_set_msg();
+    boolean that_present_msg = true && that.is_set_msg();
+    if (this_present_msg || that_present_msg) {
+      if (!(this_present_msg && that_present_msg))
+        return false;
+      if (!this.msg.equals(that.msg))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    HashCodeBuilder builder = new HashCodeBuilder();
+
+    boolean present_msg = true && (is_set_msg());
+    builder.append(present_msg);
+    if (present_msg)
+      builder.append(msg);
+
+    return builder.toHashCode();
+  }
+
+  public int compareTo(NotAliveException other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+    NotAliveException typedOther = (NotAliveException)other;
+
+    lastComparison = Boolean.valueOf(is_set_msg()).compareTo(typedOther.is_set_msg());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_msg()) {      lastComparison = TBaseHelper.compareTo(this.msg, typedOther.msg);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public void read(TProtocol iprot) throws TException {
+    TField field;
+    iprot.readStructBegin();
+    while (true)
+    {
+      field = iprot.readFieldBegin();
+      if (field.type == TType.STOP) { 
+        break;
+      }
+      switch (field.id) {
+        case 1: // MSG
+          if (field.type == TType.STRING) {
+            this.msg = iprot.readString();
+          } else { 
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        default:
+          TProtocolUtil.skip(iprot, field.type);
+      }
+      iprot.readFieldEnd();
+    }
+    iprot.readStructEnd();
+    validate();
+  }
+
+  public void write(TProtocol oprot) throws TException {
+    validate();
+
+    oprot.writeStructBegin(STRUCT_DESC);
+    if (this.msg != null) {
+      oprot.writeFieldBegin(MSG_FIELD_DESC);
+      oprot.writeString(this.msg);
+      oprot.writeFieldEnd();
+    }
+    oprot.writeFieldStop();
+    oprot.writeStructEnd();
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("NotAliveException(");
+    boolean first = true;
+
+    sb.append("msg:");
+    if (this.msg == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.msg);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws TException {
+    // check for required fields
+    if (!is_set_msg()) {
+      throw new TProtocolException("Required field 'msg' is unset! Struct:" + toString());
+    }
+
+  }
+
+}
+
diff --git a/src/jvm/backtype/storm/generated/NullStruct.java b/src/jvm/backtype/storm/generated/NullStruct.java
new file mode 100644
index 0000000..c545305
--- /dev/null
+++ b/src/jvm/backtype/storm/generated/NullStruct.java
@@ -0,0 +1,219 @@
+/**
+ * Autogenerated by Thrift
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ */
+package backtype.storm.generated;
+
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.util.Arrays;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.thrift.*;
+import org.apache.thrift.async.*;
+import org.apache.thrift.meta_data.*;
+import org.apache.thrift.transport.*;
+import org.apache.thrift.protocol.*;
+
+public class NullStruct implements TBase<NullStruct, NullStruct._Fields>, java.io.Serializable, Cloneable {
+  private static final TStruct STRUCT_DESC = new TStruct("NullStruct");
+
+
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements TFieldIdEnum {
+;
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+  public static final Map<_Fields, FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, FieldMetaData> tmpMap = new EnumMap<_Fields, FieldMetaData>(_Fields.class);
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    FieldMetaData.addStructMetaDataMap(NullStruct.class, metaDataMap);
+  }
+
+  public NullStruct() {
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public NullStruct(NullStruct other) {
+  }
+
+  public NullStruct deepCopy() {
+    return new NullStruct(this);
+  }
+
+  @Deprecated
+  public NullStruct clone() {
+    return new NullStruct(this);
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    }
+  }
+
+  public void setFieldValue(int fieldID, Object value) {
+    setFieldValue(_Fields.findByThriftIdOrThrow(fieldID), value);
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    }
+    throw new IllegalStateException();
+  }
+
+  public Object getFieldValue(int fieldId) {
+    return getFieldValue(_Fields.findByThriftIdOrThrow(fieldId));
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been asigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    switch (field) {
+    }
+    throw new IllegalStateException();
+  }
+
+  public boolean isSet(int fieldID) {
+    return isSet(_Fields.findByThriftIdOrThrow(fieldID));
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof NullStruct)
+      return this.equals((NullStruct)that);
+    return false;
+  }
+
+  public boolean equals(NullStruct that) {
+    if (that == null)
+      return false;
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    HashCodeBuilder builder = new HashCodeBuilder();
+
+    return builder.toHashCode();
+  }
+
+  public int compareTo(NullStruct other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+    NullStruct typedOther = (NullStruct)other;
+
+    return 0;
+  }
+
+  public void read(TProtocol iprot) throws TException {
+    TField field;
+    iprot.readStructBegin();
+    while (true)
+    {
+      field = iprot.readFieldBegin();
+      if (field.type == TType.STOP) { 
+        break;
+      }
+      switch (field.id) {
+        default:
+          TProtocolUtil.skip(iprot, field.type);
+      }
+      iprot.readFieldEnd();
+    }
+    iprot.readStructEnd();
+    validate();
+  }
+
+  public void write(TProtocol oprot) throws TException {
+    validate();
+
+    oprot.writeStructBegin(STRUCT_DESC);
+    oprot.writeFieldStop();
+    oprot.writeStructEnd();
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("NullStruct(");
+    boolean first = true;
+
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws TException {
+    // check for required fields
+  }
+
+}
+
diff --git a/src/jvm/backtype/storm/generated/ShellComponent.java b/src/jvm/backtype/storm/generated/ShellComponent.java
new file mode 100644
index 0000000..aa68893
--- /dev/null
+++ b/src/jvm/backtype/storm/generated/ShellComponent.java
@@ -0,0 +1,407 @@
+/**
+ * Autogenerated by Thrift
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ */
+package backtype.storm.generated;
+
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.util.Arrays;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.thrift.*;
+import org.apache.thrift.async.*;
+import org.apache.thrift.meta_data.*;
+import org.apache.thrift.transport.*;
+import org.apache.thrift.protocol.*;
+
+public class ShellComponent implements TBase<ShellComponent, ShellComponent._Fields>, java.io.Serializable, Cloneable {
+  private static final TStruct STRUCT_DESC = new TStruct("ShellComponent");
+
+  private static final TField EXECUTION_COMMAND_FIELD_DESC = new TField("execution_command", TType.STRING, (short)1);
+  private static final TField SCRIPT_FIELD_DESC = new TField("script", TType.STRING, (short)2);
+
+  private String execution_command;
+  private String script;
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements TFieldIdEnum {
+    EXECUTION_COMMAND((short)1, "execution_command"),
+    SCRIPT((short)2, "script");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // EXECUTION_COMMAND
+          return EXECUTION_COMMAND;
+        case 2: // SCRIPT
+          return SCRIPT;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+
+  public static final Map<_Fields, FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, FieldMetaData> tmpMap = new EnumMap<_Fields, FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.EXECUTION_COMMAND, new FieldMetaData("execution_command", TFieldRequirementType.DEFAULT, 
+        new FieldValueMetaData(TType.STRING)));
+    tmpMap.put(_Fields.SCRIPT, new FieldMetaData("script", TFieldRequirementType.DEFAULT, 
+        new FieldValueMetaData(TType.STRING)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    FieldMetaData.addStructMetaDataMap(ShellComponent.class, metaDataMap);
+  }
+
+  public ShellComponent() {
+  }
+
+  public ShellComponent(
+    String execution_command,
+    String script)
+  {
+    this();
+    this.execution_command = execution_command;
+    this.script = script;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public ShellComponent(ShellComponent other) {
+    if (other.is_set_execution_command()) {
+      this.execution_command = other.execution_command;
+    }
+    if (other.is_set_script()) {
+      this.script = other.script;
+    }
+  }
+
+  public ShellComponent deepCopy() {
+    return new ShellComponent(this);
+  }
+
+  @Deprecated
+  public ShellComponent clone() {
+    return new ShellComponent(this);
+  }
+
+  public String get_execution_command() {
+    return this.execution_command;
+  }
+
+  public void set_execution_command(String execution_command) {
+    this.execution_command = execution_command;
+  }
+
+  public void unset_execution_command() {
+    this.execution_command = null;
+  }
+
+  /** Returns true if field execution_command is set (has been asigned a value) and false otherwise */
+  public boolean is_set_execution_command() {
+    return this.execution_command != null;
+  }
+
+  public void set_execution_command_isSet(boolean value) {
+    if (!value) {
+      this.execution_command = null;
+    }
+  }
+
+  public String get_script() {
+    return this.script;
+  }
+
+  public void set_script(String script) {
+    this.script = script;
+  }
+
+  public void unset_script() {
+    this.script = null;
+  }
+
+  /** Returns true if field script is set (has been asigned a value) and false otherwise */
+  public boolean is_set_script() {
+    return this.script != null;
+  }
+
+  public void set_script_isSet(boolean value) {
+    if (!value) {
+      this.script = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case EXECUTION_COMMAND:
+      if (value == null) {
+        unset_execution_command();
+      } else {
+        set_execution_command((String)value);
+      }
+      break;
+
+    case SCRIPT:
+      if (value == null) {
+        unset_script();
+      } else {
+        set_script((String)value);
+      }
+      break;
+
+    }
+  }
+
+  public void setFieldValue(int fieldID, Object value) {
+    setFieldValue(_Fields.findByThriftIdOrThrow(fieldID), value);
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case EXECUTION_COMMAND:
+      return get_execution_command();
+
+    case SCRIPT:
+      return get_script();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  public Object getFieldValue(int fieldId) {
+    return getFieldValue(_Fields.findByThriftIdOrThrow(fieldId));
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been asigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    switch (field) {
+    case EXECUTION_COMMAND:
+      return is_set_execution_command();
+    case SCRIPT:
+      return is_set_script();
+    }
+    throw new IllegalStateException();
+  }
+
+  public boolean isSet(int fieldID) {
+    return isSet(_Fields.findByThriftIdOrThrow(fieldID));
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof ShellComponent)
+      return this.equals((ShellComponent)that);
+    return false;
+  }
+
+  public boolean equals(ShellComponent that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_execution_command = true && this.is_set_execution_command();
+    boolean that_present_execution_command = true && that.is_set_execution_command();
+    if (this_present_execution_command || that_present_execution_command) {
+      if (!(this_present_execution_command && that_present_execution_command))
+        return false;
+      if (!this.execution_command.equals(that.execution_command))
+        return false;
+    }
+
+    boolean this_present_script = true && this.is_set_script();
+    boolean that_present_script = true && that.is_set_script();
+    if (this_present_script || that_present_script) {
+      if (!(this_present_script && that_present_script))
+        return false;
+      if (!this.script.equals(that.script))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    HashCodeBuilder builder = new HashCodeBuilder();
+
+    boolean present_execution_command = true && (is_set_execution_command());
+    builder.append(present_execution_command);
+    if (present_execution_command)
+      builder.append(execution_command);
+
+    boolean present_script = true && (is_set_script());
+    builder.append(present_script);
+    if (present_script)
+      builder.append(script);
+
+    return builder.toHashCode();
+  }
+
+  public int compareTo(ShellComponent other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+    ShellComponent typedOther = (ShellComponent)other;
+
+    lastComparison = Boolean.valueOf(is_set_execution_command()).compareTo(typedOther.is_set_execution_command());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_execution_command()) {      lastComparison = TBaseHelper.compareTo(this.execution_command, typedOther.execution_command);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_script()).compareTo(typedOther.is_set_script());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_script()) {      lastComparison = TBaseHelper.compareTo(this.script, typedOther.script);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public void read(TProtocol iprot) throws TException {
+    TField field;
+    iprot.readStructBegin();
+    while (true)
+    {
+      field = iprot.readFieldBegin();
+      if (field.type == TType.STOP) { 
+        break;
+      }
+      switch (field.id) {
+        case 1: // EXECUTION_COMMAND
+          if (field.type == TType.STRING) {
+            this.execution_command = iprot.readString();
+          } else { 
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 2: // SCRIPT
+          if (field.type == TType.STRING) {
+            this.script = iprot.readString();
+          } else { 
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        default:
+          TProtocolUtil.skip(iprot, field.type);
+      }
+      iprot.readFieldEnd();
+    }
+    iprot.readStructEnd();
+    validate();
+  }
+
+  public void write(TProtocol oprot) throws TException {
+    validate();
+
+    oprot.writeStructBegin(STRUCT_DESC);
+    if (this.execution_command != null) {
+      oprot.writeFieldBegin(EXECUTION_COMMAND_FIELD_DESC);
+      oprot.writeString(this.execution_command);
+      oprot.writeFieldEnd();
+    }
+    if (this.script != null) {
+      oprot.writeFieldBegin(SCRIPT_FIELD_DESC);
+      oprot.writeString(this.script);
+      oprot.writeFieldEnd();
+    }
+    oprot.writeFieldStop();
+    oprot.writeStructEnd();
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("ShellComponent(");
+    boolean first = true;
+
+    sb.append("execution_command:");
+    if (this.execution_command == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.execution_command);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("script:");
+    if (this.script == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.script);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws TException {
+    // check for required fields
+  }
+
+}
+
diff --git a/src/jvm/backtype/storm/generated/SpoutSpec.java b/src/jvm/backtype/storm/generated/SpoutSpec.java
new file mode 100644
index 0000000..b88fb1c
--- /dev/null
+++ b/src/jvm/backtype/storm/generated/SpoutSpec.java
@@ -0,0 +1,509 @@
+/**
+ * Autogenerated by Thrift
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ */
+package backtype.storm.generated;
+
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.util.Arrays;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.thrift.*;
+import org.apache.thrift.async.*;
+import org.apache.thrift.meta_data.*;
+import org.apache.thrift.transport.*;
+import org.apache.thrift.protocol.*;
+
+public class SpoutSpec implements TBase<SpoutSpec, SpoutSpec._Fields>, java.io.Serializable, Cloneable {
+  private static final TStruct STRUCT_DESC = new TStruct("SpoutSpec");
+
+  private static final TField SPOUT_OBJECT_FIELD_DESC = new TField("spout_object", TType.STRUCT, (short)1);
+  private static final TField COMMON_FIELD_DESC = new TField("common", TType.STRUCT, (short)2);
+  private static final TField DISTRIBUTED_FIELD_DESC = new TField("distributed", TType.BOOL, (short)3);
+
+  private ComponentObject spout_object;
+  private ComponentCommon common;
+  private boolean distributed;
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements TFieldIdEnum {
+    SPOUT_OBJECT((short)1, "spout_object"),
+    COMMON((short)2, "common"),
+    DISTRIBUTED((short)3, "distributed");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // SPOUT_OBJECT
+          return SPOUT_OBJECT;
+        case 2: // COMMON
+          return COMMON;
+        case 3: // DISTRIBUTED
+          return DISTRIBUTED;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __DISTRIBUTED_ISSET_ID = 0;
+  private BitSet __isset_bit_vector = new BitSet(1);
+
+  public static final Map<_Fields, FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, FieldMetaData> tmpMap = new EnumMap<_Fields, FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.SPOUT_OBJECT, new FieldMetaData("spout_object", TFieldRequirementType.REQUIRED, 
+        new StructMetaData(TType.STRUCT, ComponentObject.class)));
+    tmpMap.put(_Fields.COMMON, new FieldMetaData("common", TFieldRequirementType.REQUIRED, 
+        new StructMetaData(TType.STRUCT, ComponentCommon.class)));
+    tmpMap.put(_Fields.DISTRIBUTED, new FieldMetaData("distributed", TFieldRequirementType.REQUIRED, 
+        new FieldValueMetaData(TType.BOOL)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    FieldMetaData.addStructMetaDataMap(SpoutSpec.class, metaDataMap);
+  }
+
+  public SpoutSpec() {
+  }
+
+  public SpoutSpec(
+    ComponentObject spout_object,
+    ComponentCommon common,
+    boolean distributed)
+  {
+    this();
+    this.spout_object = spout_object;
+    this.common = common;
+    this.distributed = distributed;
+    set_distributed_isSet(true);
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public SpoutSpec(SpoutSpec other) {
+    __isset_bit_vector.clear();
+    __isset_bit_vector.or(other.__isset_bit_vector);
+    if (other.is_set_spout_object()) {
+      this.spout_object = new ComponentObject(other.spout_object);
+    }
+    if (other.is_set_common()) {
+      this.common = new ComponentCommon(other.common);
+    }
+    this.distributed = other.distributed;
+  }
+
+  public SpoutSpec deepCopy() {
+    return new SpoutSpec(this);
+  }
+
+  @Deprecated
+  public SpoutSpec clone() {
+    return new SpoutSpec(this);
+  }
+
+  public ComponentObject get_spout_object() {
+    return this.spout_object;
+  }
+
+  public void set_spout_object(ComponentObject spout_object) {
+    this.spout_object = spout_object;
+  }
+
+  public void unset_spout_object() {
+    this.spout_object = null;
+  }
+
+  /** Returns true if field spout_object is set (has been asigned a value) and false otherwise */
+  public boolean is_set_spout_object() {
+    return this.spout_object != null;
+  }
+
+  public void set_spout_object_isSet(boolean value) {
+    if (!value) {
+      this.spout_object = null;
+    }
+  }
+
+  public ComponentCommon get_common() {
+    return this.common;
+  }
+
+  public void set_common(ComponentCommon common) {
+    this.common = common;
+  }
+
+  public void unset_common() {
+    this.common = null;
+  }
+
+  /** Returns true if field common is set (has been asigned a value) and false otherwise */
+  public boolean is_set_common() {
+    return this.common != null;
+  }
+
+  public void set_common_isSet(boolean value) {
+    if (!value) {
+      this.common = null;
+    }
+  }
+
+  public boolean is_distributed() {
+    return this.distributed;
+  }
+
+  public void set_distributed(boolean distributed) {
+    this.distributed = distributed;
+    set_distributed_isSet(true);
+  }
+
+  public void unset_distributed() {
+    __isset_bit_vector.clear(__DISTRIBUTED_ISSET_ID);
+  }
+
+  /** Returns true if field distributed is set (has been asigned a value) and false otherwise */
+  public boolean is_set_distributed() {
+    return __isset_bit_vector.get(__DISTRIBUTED_ISSET_ID);
+  }
+
+  public void set_distributed_isSet(boolean value) {
+    __isset_bit_vector.set(__DISTRIBUTED_ISSET_ID, value);
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case SPOUT_OBJECT:
+      if (value == null) {
+        unset_spout_object();
+      } else {
+        set_spout_object((ComponentObject)value);
+      }
+      break;
+
+    case COMMON:
+      if (value == null) {
+        unset_common();
+      } else {
+        set_common((ComponentCommon)value);
+      }
+      break;
+
+    case DISTRIBUTED:
+      if (value == null) {
+        unset_distributed();
+      } else {
+        set_distributed((Boolean)value);
+      }
+      break;
+
+    }
+  }
+
+  public void setFieldValue(int fieldID, Object value) {
+    setFieldValue(_Fields.findByThriftIdOrThrow(fieldID), value);
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case SPOUT_OBJECT:
+      return get_spout_object();
+
+    case COMMON:
+      return get_common();
+
+    case DISTRIBUTED:
+      return new Boolean(is_distributed());
+
+    }
+    throw new IllegalStateException();
+  }
+
+  public Object getFieldValue(int fieldId) {
+    return getFieldValue(_Fields.findByThriftIdOrThrow(fieldId));
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been asigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    switch (field) {
+    case SPOUT_OBJECT:
+      return is_set_spout_object();
+    case COMMON:
+      return is_set_common();
+    case DISTRIBUTED:
+      return is_set_distributed();
+    }
+    throw new IllegalStateException();
+  }
+
+  public boolean isSet(int fieldID) {
+    return isSet(_Fields.findByThriftIdOrThrow(fieldID));
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof SpoutSpec)
+      return this.equals((SpoutSpec)that);
+    return false;
+  }
+
+  public boolean equals(SpoutSpec that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_spout_object = true && this.is_set_spout_object();
+    boolean that_present_spout_object = true && that.is_set_spout_object();
+    if (this_present_spout_object || that_present_spout_object) {
+      if (!(this_present_spout_object && that_present_spout_object))
+        return false;
+      if (!this.spout_object.equals(that.spout_object))
+        return false;
+    }
+
+    boolean this_present_common = true && this.is_set_common();
+    boolean that_present_common = true && that.is_set_common();
+    if (this_present_common || that_present_common) {
+      if (!(this_present_common && that_present_common))
+        return false;
+      if (!this.common.equals(that.common))
+        return false;
+    }
+
+    boolean this_present_distributed = true;
+    boolean that_present_distributed = true;
+    if (this_present_distributed || that_present_distributed) {
+      if (!(this_present_distributed && that_present_distributed))
+        return false;
+      if (this.distributed != that.distributed)
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    HashCodeBuilder builder = new HashCodeBuilder();
+
+    boolean present_spout_object = true && (is_set_spout_object());
+    builder.append(present_spout_object);
+    if (present_spout_object)
+      builder.append(spout_object);
+
+    boolean present_common = true && (is_set_common());
+    builder.append(present_common);
+    if (present_common)
+      builder.append(common);
+
+    boolean present_distributed = true;
+    builder.append(present_distributed);
+    if (present_distributed)
+      builder.append(distributed);
+
+    return builder.toHashCode();
+  }
+
+  public int compareTo(SpoutSpec other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+    SpoutSpec typedOther = (SpoutSpec)other;
+
+    lastComparison = Boolean.valueOf(is_set_spout_object()).compareTo(typedOther.is_set_spout_object());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_spout_object()) {      lastComparison = TBaseHelper.compareTo(this.spout_object, typedOther.spout_object);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_common()).compareTo(typedOther.is_set_common());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_common()) {      lastComparison = TBaseHelper.compareTo(this.common, typedOther.common);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_distributed()).compareTo(typedOther.is_set_distributed());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_distributed()) {      lastComparison = TBaseHelper.compareTo(this.distributed, typedOther.distributed);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public void read(TProtocol iprot) throws TException {
+    TField field;
+    iprot.readStructBegin();
+    while (true)
+    {
+      field = iprot.readFieldBegin();
+      if (field.type == TType.STOP) { 
+        break;
+      }
+      switch (field.id) {
+        case 1: // SPOUT_OBJECT
+          if (field.type == TType.STRUCT) {
+            this.spout_object = new ComponentObject();
+            this.spout_object.read(iprot);
+          } else { 
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 2: // COMMON
+          if (field.type == TType.STRUCT) {
+            this.common = new ComponentCommon();
+            this.common.read(iprot);
+          } else { 
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 3: // DISTRIBUTED
+          if (field.type == TType.BOOL) {
+            this.distributed = iprot.readBool();
+            set_distributed_isSet(true);
+          } else { 
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        default:
+          TProtocolUtil.skip(iprot, field.type);
+      }
+      iprot.readFieldEnd();
+    }
+    iprot.readStructEnd();
+    validate();
+  }
+
+  public void write(TProtocol oprot) throws TException {
+    validate();
+
+    oprot.writeStructBegin(STRUCT_DESC);
+    if (this.spout_object != null) {
+      oprot.writeFieldBegin(SPOUT_OBJECT_FIELD_DESC);
+      this.spout_object.write(oprot);
+      oprot.writeFieldEnd();
+    }
+    if (this.common != null) {
+      oprot.writeFieldBegin(COMMON_FIELD_DESC);
+      this.common.write(oprot);
+      oprot.writeFieldEnd();
+    }
+    oprot.writeFieldBegin(DISTRIBUTED_FIELD_DESC);
+    oprot.writeBool(this.distributed);
+    oprot.writeFieldEnd();
+    oprot.writeFieldStop();
+    oprot.writeStructEnd();
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("SpoutSpec(");
+    boolean first = true;
+
+    sb.append("spout_object:");
+    if (this.spout_object == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.spout_object);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("common:");
+    if (this.common == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.common);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("distributed:");
+    sb.append(this.distributed);
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws TException {
+    // check for required fields
+    if (!is_set_spout_object()) {
+      throw new TProtocolException("Required field 'spout_object' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_common()) {
+      throw new TProtocolException("Required field 'common' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_distributed()) {
+      throw new TProtocolException("Required field 'distributed' is unset! Struct:" + toString());
+    }
+
+  }
+
+}
+
diff --git a/src/jvm/backtype/storm/generated/SpoutStats.java b/src/jvm/backtype/storm/generated/SpoutStats.java
new file mode 100644
index 0000000..c2b56b8
--- /dev/null
+++ b/src/jvm/backtype/storm/generated/SpoutStats.java
@@ -0,0 +1,744 @@
+/**
+ * Autogenerated by Thrift
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ */
+package backtype.storm.generated;
+
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.util.Arrays;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.thrift.*;
+import org.apache.thrift.async.*;
+import org.apache.thrift.meta_data.*;
+import org.apache.thrift.transport.*;
+import org.apache.thrift.protocol.*;
+
+public class SpoutStats implements TBase<SpoutStats, SpoutStats._Fields>, java.io.Serializable, Cloneable {
+  private static final TStruct STRUCT_DESC = new TStruct("SpoutStats");
+
+  private static final TField ACKED_FIELD_DESC = new TField("acked", TType.MAP, (short)1);
+  private static final TField FAILED_FIELD_DESC = new TField("failed", TType.MAP, (short)2);
+  private static final TField COMPLETE_MS_AVG_FIELD_DESC = new TField("complete_ms_avg", TType.MAP, (short)3);
+
+  private Map<String,Map<Integer,Long>> acked;
+  private Map<String,Map<Integer,Long>> failed;
+  private Map<String,Map<Integer,Double>> complete_ms_avg;
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements TFieldIdEnum {
+    ACKED((short)1, "acked"),
+    FAILED((short)2, "failed"),
+    COMPLETE_MS_AVG((short)3, "complete_ms_avg");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // ACKED
+          return ACKED;
+        case 2: // FAILED
+          return FAILED;
+        case 3: // COMPLETE_MS_AVG
+          return COMPLETE_MS_AVG;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+
+  public static final Map<_Fields, FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, FieldMetaData> tmpMap = new EnumMap<_Fields, FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.ACKED, new FieldMetaData("acked", TFieldRequirementType.REQUIRED, 
+        new MapMetaData(TType.MAP, 
+            new FieldValueMetaData(TType.STRING), 
+            new MapMetaData(TType.MAP, 
+                new FieldValueMetaData(TType.I32), 
+                new FieldValueMetaData(TType.I64)))));
+    tmpMap.put(_Fields.FAILED, new FieldMetaData("failed", TFieldRequirementType.REQUIRED, 
+        new MapMetaData(TType.MAP, 
+            new FieldValueMetaData(TType.STRING), 
+            new MapMetaData(TType.MAP, 
+                new FieldValueMetaData(TType.I32), 
+                new FieldValueMetaData(TType.I64)))));
+    tmpMap.put(_Fields.COMPLETE_MS_AVG, new FieldMetaData("complete_ms_avg", TFieldRequirementType.REQUIRED, 
+        new MapMetaData(TType.MAP, 
+            new FieldValueMetaData(TType.STRING), 
+            new MapMetaData(TType.MAP, 
+                new FieldValueMetaData(TType.I32), 
+                new FieldValueMetaData(TType.DOUBLE)))));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    FieldMetaData.addStructMetaDataMap(SpoutStats.class, metaDataMap);
+  }
+
+  public SpoutStats() {
+  }
+
+  public SpoutStats(
+    Map<String,Map<Integer,Long>> acked,
+    Map<String,Map<Integer,Long>> failed,
+    Map<String,Map<Integer,Double>> complete_ms_avg)
+  {
+    this();
+    this.acked = acked;
+    this.failed = failed;
+    this.complete_ms_avg = complete_ms_avg;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public SpoutStats(SpoutStats other) {
+    if (other.is_set_acked()) {
+      Map<String,Map<Integer,Long>> __this__acked = new HashMap<String,Map<Integer,Long>>();
+      for (Map.Entry<String, Map<Integer,Long>> other_element : other.acked.entrySet()) {
+
+        String other_element_key = other_element.getKey();
+        Map<Integer,Long> other_element_value = other_element.getValue();
+
+        String __this__acked_copy_key = other_element_key;
+
+        Map<Integer,Long> __this__acked_copy_value = new HashMap<Integer,Long>();
+        for (Map.Entry<Integer, Long> other_element_value_element : other_element_value.entrySet()) {
+
+          Integer other_element_value_element_key = other_element_value_element.getKey();
+          Long other_element_value_element_value = other_element_value_element.getValue();
+
+          Integer __this__acked_copy_value_copy_key = other_element_value_element_key;
+
+          Long __this__acked_copy_value_copy_value = other_element_value_element_value;
+
+          __this__acked_copy_value.put(__this__acked_copy_value_copy_key, __this__acked_copy_value_copy_value);
+        }
+
+        __this__acked.put(__this__acked_copy_key, __this__acked_copy_value);
+      }
+      this.acked = __this__acked;
+    }
+    if (other.is_set_failed()) {
+      Map<String,Map<Integer,Long>> __this__failed = new HashMap<String,Map<Integer,Long>>();
+      for (Map.Entry<String, Map<Integer,Long>> other_element : other.failed.entrySet()) {
+
+        String other_element_key = other_element.getKey();
+        Map<Integer,Long> other_element_value = other_element.getValue();
+
+        String __this__failed_copy_key = other_element_key;
+
+        Map<Integer,Long> __this__failed_copy_value = new HashMap<Integer,Long>();
+        for (Map.Entry<Integer, Long> other_element_value_element : other_element_value.entrySet()) {
+
+          Integer other_element_value_element_key = other_element_value_element.getKey();
+          Long other_element_value_element_value = other_element_value_element.getValue();
+
+          Integer __this__failed_copy_value_copy_key = other_element_value_element_key;
+
+          Long __this__failed_copy_value_copy_value = other_element_value_element_value;
+
+          __this__failed_copy_value.put(__this__failed_copy_value_copy_key, __this__failed_copy_value_copy_value);
+        }
+
+        __this__failed.put(__this__failed_copy_key, __this__failed_copy_value);
+      }
+      this.failed = __this__failed;
+    }
+    if (other.is_set_complete_ms_avg()) {
+      Map<String,Map<Integer,Double>> __this__complete_ms_avg = new HashMap<String,Map<Integer,Double>>();
+      for (Map.Entry<String, Map<Integer,Double>> other_element : other.complete_ms_avg.entrySet()) {
+
+        String other_element_key = other_element.getKey();
+        Map<Integer,Double> other_element_value = other_element.getValue();
+
+        String __this__complete_ms_avg_copy_key = other_element_key;
+
+        Map<Integer,Double> __this__complete_ms_avg_copy_value = new HashMap<Integer,Double>();
+        for (Map.Entry<Integer, Double> other_element_value_element : other_element_value.entrySet()) {
+
+          Integer other_element_value_element_key = other_element_value_element.getKey();
+          Double other_element_value_element_value = other_element_value_element.getValue();
+
+          Integer __this__complete_ms_avg_copy_value_copy_key = other_element_value_element_key;
+
+          Double __this__complete_ms_avg_copy_value_copy_value = other_element_value_element_value;
+
+          __this__complete_ms_avg_copy_value.put(__this__complete_ms_avg_copy_value_copy_key, __this__complete_ms_avg_copy_value_copy_value);
+        }
+
+        __this__complete_ms_avg.put(__this__complete_ms_avg_copy_key, __this__complete_ms_avg_copy_value);
+      }
+      this.complete_ms_avg = __this__complete_ms_avg;
+    }
+  }
+
+  public SpoutStats deepCopy() {
+    return new SpoutStats(this);
+  }
+
+  @Deprecated
+  public SpoutStats clone() {
+    return new SpoutStats(this);
+  }
+
+  public int get_acked_size() {
+    return (this.acked == null) ? 0 : this.acked.size();
+  }
+
+  public void put_to_acked(String key, Map<Integer,Long> val) {
+    if (this.acked == null) {
+      this.acked = new HashMap<String,Map<Integer,Long>>();
+    }
+    this.acked.put(key, val);
+  }
+
+  public Map<String,Map<Integer,Long>> get_acked() {
+    return this.acked;
+  }
+
+  public void set_acked(Map<String,Map<Integer,Long>> acked) {
+    this.acked = acked;
+  }
+
+  public void unset_acked() {
+    this.acked = null;
+  }
+
+  /** Returns true if field acked is set (has been asigned a value) and false otherwise */
+  public boolean is_set_acked() {
+    return this.acked != null;
+  }
+
+  public void set_acked_isSet(boolean value) {
+    if (!value) {
+      this.acked = null;
+    }
+  }
+
+  public int get_failed_size() {
+    return (this.failed == null) ? 0 : this.failed.size();
+  }
+
+  public void put_to_failed(String key, Map<Integer,Long> val) {
+    if (this.failed == null) {
+      this.failed = new HashMap<String,Map<Integer,Long>>();
+    }
+    this.failed.put(key, val);
+  }
+
+  public Map<String,Map<Integer,Long>> get_failed() {
+    return this.failed;
+  }
+
+  public void set_failed(Map<String,Map<Integer,Long>> failed) {
+    this.failed = failed;
+  }
+
+  public void unset_failed() {
+    this.failed = null;
+  }
+
+  /** Returns true if field failed is set (has been asigned a value) and false otherwise */
+  public boolean is_set_failed() {
+    return this.failed != null;
+  }
+
+  public void set_failed_isSet(boolean value) {
+    if (!value) {
+      this.failed = null;
+    }
+  }
+
+  public int get_complete_ms_avg_size() {
+    return (this.complete_ms_avg == null) ? 0 : this.complete_ms_avg.size();
+  }
+
+  public void put_to_complete_ms_avg(String key, Map<Integer,Double> val) {
+    if (this.complete_ms_avg == null) {
+      this.complete_ms_avg = new HashMap<String,Map<Integer,Double>>();
+    }
+    this.complete_ms_avg.put(key, val);
+  }
+
+  public Map<String,Map<Integer,Double>> get_complete_ms_avg() {
+    return this.complete_ms_avg;
+  }
+
+  public void set_complete_ms_avg(Map<String,Map<Integer,Double>> complete_ms_avg) {
+    this.complete_ms_avg = complete_ms_avg;
+  }
+
+  public void unset_complete_ms_avg() {
+    this.complete_ms_avg = null;
+  }
+
+  /** Returns true if field complete_ms_avg is set (has been asigned a value) and false otherwise */
+  public boolean is_set_complete_ms_avg() {
+    return this.complete_ms_avg != null;
+  }
+
+  public void set_complete_ms_avg_isSet(boolean value) {
+    if (!value) {
+      this.complete_ms_avg = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case ACKED:
+      if (value == null) {
+        unset_acked();
+      } else {
+        set_acked((Map<String,Map<Integer,Long>>)value);
+      }
+      break;
+
+    case FAILED:
+      if (value == null) {
+        unset_failed();
+      } else {
+        set_failed((Map<String,Map<Integer,Long>>)value);
+      }
+      break;
+
+    case COMPLETE_MS_AVG:
+      if (value == null) {
+        unset_complete_ms_avg();
+      } else {
+        set_complete_ms_avg((Map<String,Map<Integer,Double>>)value);
+      }
+      break;
+
+    }
+  }
+
+  public void setFieldValue(int fieldID, Object value) {
+    setFieldValue(_Fields.findByThriftIdOrThrow(fieldID), value);
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case ACKED:
+      return get_acked();
+
+    case FAILED:
+      return get_failed();
+
+    case COMPLETE_MS_AVG:
+      return get_complete_ms_avg();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  public Object getFieldValue(int fieldId) {
+    return getFieldValue(_Fields.findByThriftIdOrThrow(fieldId));
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been asigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    switch (field) {
+    case ACKED:
+      return is_set_acked();
+    case FAILED:
+      return is_set_failed();
+    case COMPLETE_MS_AVG:
+      return is_set_complete_ms_avg();
+    }
+    throw new IllegalStateException();
+  }
+
+  public boolean isSet(int fieldID) {
+    return isSet(_Fields.findByThriftIdOrThrow(fieldID));
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof SpoutStats)
+      return this.equals((SpoutStats)that);
+    return false;
+  }
+
+  public boolean equals(SpoutStats that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_acked = true && this.is_set_acked();
+    boolean that_present_acked = true && that.is_set_acked();
+    if (this_present_acked || that_present_acked) {
+      if (!(this_present_acked && that_present_acked))
+        return false;
+      if (!this.acked.equals(that.acked))
+        return false;
+    }
+
+    boolean this_present_failed = true && this.is_set_failed();
+    boolean that_present_failed = true && that.is_set_failed();
+    if (this_present_failed || that_present_failed) {
+      if (!(this_present_failed && that_present_failed))
+        return false;
+      if (!this.failed.equals(that.failed))
+        return false;
+    }
+
+    boolean this_present_complete_ms_avg = true && this.is_set_complete_ms_avg();
+    boolean that_present_complete_ms_avg = true && that.is_set_complete_ms_avg();
+    if (this_present_complete_ms_avg || that_present_complete_ms_avg) {
+      if (!(this_present_complete_ms_avg && that_present_complete_ms_avg))
+        return false;
+      if (!this.complete_ms_avg.equals(that.complete_ms_avg))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    HashCodeBuilder builder = new HashCodeBuilder();
+
+    boolean present_acked = true && (is_set_acked());
+    builder.append(present_acked);
+    if (present_acked)
+      builder.append(acked);
+
+    boolean present_failed = true && (is_set_failed());
+    builder.append(present_failed);
+    if (present_failed)
+      builder.append(failed);
+
+    boolean present_complete_ms_avg = true && (is_set_complete_ms_avg());
+    builder.append(present_complete_ms_avg);
+    if (present_complete_ms_avg)
+      builder.append(complete_ms_avg);
+
+    return builder.toHashCode();
+  }
+
+  public int compareTo(SpoutStats other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+    SpoutStats typedOther = (SpoutStats)other;
+
+    lastComparison = Boolean.valueOf(is_set_acked()).compareTo(typedOther.is_set_acked());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_acked()) {      lastComparison = TBaseHelper.compareTo(this.acked, typedOther.acked);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_failed()).compareTo(typedOther.is_set_failed());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_failed()) {      lastComparison = TBaseHelper.compareTo(this.failed, typedOther.failed);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_complete_ms_avg()).compareTo(typedOther.is_set_complete_ms_avg());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_complete_ms_avg()) {      lastComparison = TBaseHelper.compareTo(this.complete_ms_avg, typedOther.complete_ms_avg);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public void read(TProtocol iprot) throws TException {
+    TField field;
+    iprot.readStructBegin();
+    while (true)
+    {
+      field = iprot.readFieldBegin();
+      if (field.type == TType.STOP) { 
+        break;
+      }
+      switch (field.id) {
+        case 1: // ACKED
+          if (field.type == TType.MAP) {
+            {
+              TMap _map71 = iprot.readMapBegin();
+              this.acked = new HashMap<String,Map<Integer,Long>>(2*_map71.size);
+              for (int _i72 = 0; _i72 < _map71.size; ++_i72)
+              {
+                String _key73;
+                Map<Integer,Long> _val74;
+                _key73 = iprot.readString();
+                {
+                  TMap _map75 = iprot.readMapBegin();
+                  _val74 = new HashMap<Integer,Long>(2*_map75.size);
+                  for (int _i76 = 0; _i76 < _map75.size; ++_i76)
+                  {
+                    int _key77;
+                    long _val78;
+                    _key77 = iprot.readI32();
+                    _val78 = iprot.readI64();
+                    _val74.put(_key77, _val78);
+                  }
+                  iprot.readMapEnd();
+                }
+                this.acked.put(_key73, _val74);
+              }
+              iprot.readMapEnd();
+            }
+          } else { 
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 2: // FAILED
+          if (field.type == TType.MAP) {
+            {
+              TMap _map79 = iprot.readMapBegin();
+              this.failed = new HashMap<String,Map<Integer,Long>>(2*_map79.size);
+              for (int _i80 = 0; _i80 < _map79.size; ++_i80)
+              {
+                String _key81;
+                Map<Integer,Long> _val82;
+                _key81 = iprot.readString();
+                {
+                  TMap _map83 = iprot.readMapBegin();
+                  _val82 = new HashMap<Integer,Long>(2*_map83.size);
+                  for (int _i84 = 0; _i84 < _map83.size; ++_i84)
+                  {
+                    int _key85;
+                    long _val86;
+                    _key85 = iprot.readI32();
+                    _val86 = iprot.readI64();
+                    _val82.put(_key85, _val86);
+                  }
+                  iprot.readMapEnd();
+                }
+                this.failed.put(_key81, _val82);
+              }
+              iprot.readMapEnd();
+            }
+          } else { 
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 3: // COMPLETE_MS_AVG
+          if (field.type == TType.MAP) {
+            {
+              TMap _map87 = iprot.readMapBegin();
+              this.complete_ms_avg = new HashMap<String,Map<Integer,Double>>(2*_map87.size);
+              for (int _i88 = 0; _i88 < _map87.size; ++_i88)
+              {
+                String _key89;
+                Map<Integer,Double> _val90;
+                _key89 = iprot.readString();
+                {
+                  TMap _map91 = iprot.readMapBegin();
+                  _val90 = new HashMap<Integer,Double>(2*_map91.size);
+                  for (int _i92 = 0; _i92 < _map91.size; ++_i92)
+                  {
+                    int _key93;
+                    double _val94;
+                    _key93 = iprot.readI32();
+                    _val94 = iprot.readDouble();
+                    _val90.put(_key93, _val94);
+                  }
+                  iprot.readMapEnd();
+                }
+                this.complete_ms_avg.put(_key89, _val90);
+              }
+              iprot.readMapEnd();
+            }
+          } else { 
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        default:
+          TProtocolUtil.skip(iprot, field.type);
+      }
+      iprot.readFieldEnd();
+    }
+    iprot.readStructEnd();
+    validate();
+  }
+
+  public void write(TProtocol oprot) throws TException {
+    validate();
+
+    oprot.writeStructBegin(STRUCT_DESC);
+    if (this.acked != null) {
+      oprot.writeFieldBegin(ACKED_FIELD_DESC);
+      {
+        oprot.writeMapBegin(new TMap(TType.STRING, TType.MAP, this.acked.size()));
+        for (Map.Entry<String, Map<Integer,Long>> _iter95 : this.acked.entrySet())
+        {
+          oprot.writeString(_iter95.getKey());
+          {
+            oprot.writeMapBegin(new TMap(TType.I32, TType.I64, _iter95.getValue().size()));
+            for (Map.Entry<Integer, Long> _iter96 : _iter95.getValue().entrySet())
+            {
+              oprot.writeI32(_iter96.getKey());
+              oprot.writeI64(_iter96.getValue());
+            }
+            oprot.writeMapEnd();
+          }
+        }
+        oprot.writeMapEnd();
+      }
+      oprot.writeFieldEnd();
+    }
+    if (this.failed != null) {
+      oprot.writeFieldBegin(FAILED_FIELD_DESC);
+      {
+        oprot.writeMapBegin(new TMap(TType.STRING, TType.MAP, this.failed.size()));
+        for (Map.Entry<String, Map<Integer,Long>> _iter97 : this.failed.entrySet())
+        {
+          oprot.writeString(_iter97.getKey());
+          {
+            oprot.writeMapBegin(new TMap(TType.I32, TType.I64, _iter97.getValue().size()));
+            for (Map.Entry<Integer, Long> _iter98 : _iter97.getValue().entrySet())
+            {
+              oprot.writeI32(_iter98.getKey());
+              oprot.writeI64(_iter98.getValue());
+            }
+            oprot.writeMapEnd();
+          }
+        }
+        oprot.writeMapEnd();
+      }
+      oprot.writeFieldEnd();
+    }
+    if (this.complete_ms_avg != null) {
+      oprot.writeFieldBegin(COMPLETE_MS_AVG_FIELD_DESC);
+      {
+        oprot.writeMapBegin(new TMap(TType.STRING, TType.MAP, this.complete_ms_avg.size()));
+        for (Map.Entry<String, Map<Integer,Double>> _iter99 : this.complete_ms_avg.entrySet())
+        {
+          oprot.writeString(_iter99.getKey());
+          {
+            oprot.writeMapBegin(new TMap(TType.I32, TType.DOUBLE, _iter99.getValue().size()));
+            for (Map.Entry<Integer, Double> _iter100 : _iter99.getValue().entrySet())
+            {
+              oprot.writeI32(_iter100.getKey());
+              oprot.writeDouble(_iter100.getValue());
+            }
+            oprot.writeMapEnd();
+          }
+        }
+        oprot.writeMapEnd();
+      }
+      oprot.writeFieldEnd();
+    }
+    oprot.writeFieldStop();
+    oprot.writeStructEnd();
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("SpoutStats(");
+    boolean first = true;
+
+    sb.append("acked:");
+    if (this.acked == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.acked);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("failed:");
+    if (this.failed == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.failed);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("complete_ms_avg:");
+    if (this.complete_ms_avg == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.complete_ms_avg);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws TException {
+    // check for required fields
+    if (!is_set_acked()) {
+      throw new TProtocolException("Required field 'acked' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_failed()) {
+      throw new TProtocolException("Required field 'failed' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_complete_ms_avg()) {
+      throw new TProtocolException("Required field 'complete_ms_avg' is unset! Struct:" + toString());
+    }
+
+  }
+
+}
+
diff --git a/src/jvm/backtype/storm/generated/StateSpoutSpec.java b/src/jvm/backtype/storm/generated/StateSpoutSpec.java
new file mode 100644
index 0000000..41d295b
--- /dev/null
+++ b/src/jvm/backtype/storm/generated/StateSpoutSpec.java
@@ -0,0 +1,417 @@
+/**
+ * Autogenerated by Thrift
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ */
+package backtype.storm.generated;
+
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.util.Arrays;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.thrift.*;
+import org.apache.thrift.async.*;
+import org.apache.thrift.meta_data.*;
+import org.apache.thrift.transport.*;
+import org.apache.thrift.protocol.*;
+
+public class StateSpoutSpec implements TBase<StateSpoutSpec, StateSpoutSpec._Fields>, java.io.Serializable, Cloneable {
+  private static final TStruct STRUCT_DESC = new TStruct("StateSpoutSpec");
+
+  private static final TField STATE_SPOUT_OBJECT_FIELD_DESC = new TField("state_spout_object", TType.STRUCT, (short)1);
+  private static final TField COMMON_FIELD_DESC = new TField("common", TType.STRUCT, (short)2);
+
+  private ComponentObject state_spout_object;
+  private ComponentCommon common;
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements TFieldIdEnum {
+    STATE_SPOUT_OBJECT((short)1, "state_spout_object"),
+    COMMON((short)2, "common");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // STATE_SPOUT_OBJECT
+          return STATE_SPOUT_OBJECT;
+        case 2: // COMMON
+          return COMMON;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+
+  public static final Map<_Fields, FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, FieldMetaData> tmpMap = new EnumMap<_Fields, FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.STATE_SPOUT_OBJECT, new FieldMetaData("state_spout_object", TFieldRequirementType.REQUIRED, 
+        new StructMetaData(TType.STRUCT, ComponentObject.class)));
+    tmpMap.put(_Fields.COMMON, new FieldMetaData("common", TFieldRequirementType.REQUIRED, 
+        new StructMetaData(TType.STRUCT, ComponentCommon.class)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    FieldMetaData.addStructMetaDataMap(StateSpoutSpec.class, metaDataMap);
+  }
+
+  public StateSpoutSpec() {
+  }
+
+  public StateSpoutSpec(
+    ComponentObject state_spout_object,
+    ComponentCommon common)
+  {
+    this();
+    this.state_spout_object = state_spout_object;
+    this.common = common;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public StateSpoutSpec(StateSpoutSpec other) {
+    if (other.is_set_state_spout_object()) {
+      this.state_spout_object = new ComponentObject(other.state_spout_object);
+    }
+    if (other.is_set_common()) {
+      this.common = new ComponentCommon(other.common);
+    }
+  }
+
+  public StateSpoutSpec deepCopy() {
+    return new StateSpoutSpec(this);
+  }
+
+  @Deprecated
+  public StateSpoutSpec clone() {
+    return new StateSpoutSpec(this);
+  }
+
+  public ComponentObject get_state_spout_object() {
+    return this.state_spout_object;
+  }
+
+  public void set_state_spout_object(ComponentObject state_spout_object) {
+    this.state_spout_object = state_spout_object;
+  }
+
+  public void unset_state_spout_object() {
+    this.state_spout_object = null;
+  }
+
+  /** Returns true if field state_spout_object is set (has been asigned a value) and false otherwise */
+  public boolean is_set_state_spout_object() {
+    return this.state_spout_object != null;
+  }
+
+  public void set_state_spout_object_isSet(boolean value) {
+    if (!value) {
+      this.state_spout_object = null;
+    }
+  }
+
+  public ComponentCommon get_common() {
+    return this.common;
+  }
+
+  public void set_common(ComponentCommon common) {
+    this.common = common;
+  }
+
+  public void unset_common() {
+    this.common = null;
+  }
+
+  /** Returns true if field common is set (has been asigned a value) and false otherwise */
+  public boolean is_set_common() {
+    return this.common != null;
+  }
+
+  public void set_common_isSet(boolean value) {
+    if (!value) {
+      this.common = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case STATE_SPOUT_OBJECT:
+      if (value == null) {
+        unset_state_spout_object();
+      } else {
+        set_state_spout_object((ComponentObject)value);
+      }
+      break;
+
+    case COMMON:
+      if (value == null) {
+        unset_common();
+      } else {
+        set_common((ComponentCommon)value);
+      }
+      break;
+
+    }
+  }
+
+  public void setFieldValue(int fieldID, Object value) {
+    setFieldValue(_Fields.findByThriftIdOrThrow(fieldID), value);
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case STATE_SPOUT_OBJECT:
+      return get_state_spout_object();
+
+    case COMMON:
+      return get_common();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  public Object getFieldValue(int fieldId) {
+    return getFieldValue(_Fields.findByThriftIdOrThrow(fieldId));
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been asigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    switch (field) {
+    case STATE_SPOUT_OBJECT:
+      return is_set_state_spout_object();
+    case COMMON:
+      return is_set_common();
+    }
+    throw new IllegalStateException();
+  }
+
+  public boolean isSet(int fieldID) {
+    return isSet(_Fields.findByThriftIdOrThrow(fieldID));
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof StateSpoutSpec)
+      return this.equals((StateSpoutSpec)that);
+    return false;
+  }
+
+  public boolean equals(StateSpoutSpec that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_state_spout_object = true && this.is_set_state_spout_object();
+    boolean that_present_state_spout_object = true && that.is_set_state_spout_object();
+    if (this_present_state_spout_object || that_present_state_spout_object) {
+      if (!(this_present_state_spout_object && that_present_state_spout_object))
+        return false;
+      if (!this.state_spout_object.equals(that.state_spout_object))
+        return false;
+    }
+
+    boolean this_present_common = true && this.is_set_common();
+    boolean that_present_common = true && that.is_set_common();
+    if (this_present_common || that_present_common) {
+      if (!(this_present_common && that_present_common))
+        return false;
+      if (!this.common.equals(that.common))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    HashCodeBuilder builder = new HashCodeBuilder();
+
+    boolean present_state_spout_object = true && (is_set_state_spout_object());
+    builder.append(present_state_spout_object);
+    if (present_state_spout_object)
+      builder.append(state_spout_object);
+
+    boolean present_common = true && (is_set_common());
+    builder.append(present_common);
+    if (present_common)
+      builder.append(common);
+
+    return builder.toHashCode();
+  }
+
+  public int compareTo(StateSpoutSpec other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+    StateSpoutSpec typedOther = (StateSpoutSpec)other;
+
+    lastComparison = Boolean.valueOf(is_set_state_spout_object()).compareTo(typedOther.is_set_state_spout_object());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_state_spout_object()) {      lastComparison = TBaseHelper.compareTo(this.state_spout_object, typedOther.state_spout_object);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_common()).compareTo(typedOther.is_set_common());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_common()) {      lastComparison = TBaseHelper.compareTo(this.common, typedOther.common);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public void read(TProtocol iprot) throws TException {
+    TField field;
+    iprot.readStructBegin();
+    while (true)
+    {
+      field = iprot.readFieldBegin();
+      if (field.type == TType.STOP) { 
+        break;
+      }
+      switch (field.id) {
+        case 1: // STATE_SPOUT_OBJECT
+          if (field.type == TType.STRUCT) {
+            this.state_spout_object = new ComponentObject();
+            this.state_spout_object.read(iprot);
+          } else { 
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 2: // COMMON
+          if (field.type == TType.STRUCT) {
+            this.common = new ComponentCommon();
+            this.common.read(iprot);
+          } else { 
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        default:
+          TProtocolUtil.skip(iprot, field.type);
+      }
+      iprot.readFieldEnd();
+    }
+    iprot.readStructEnd();
+    validate();
+  }
+
+  public void write(TProtocol oprot) throws TException {
+    validate();
+
+    oprot.writeStructBegin(STRUCT_DESC);
+    if (this.state_spout_object != null) {
+      oprot.writeFieldBegin(STATE_SPOUT_OBJECT_FIELD_DESC);
+      this.state_spout_object.write(oprot);
+      oprot.writeFieldEnd();
+    }
+    if (this.common != null) {
+      oprot.writeFieldBegin(COMMON_FIELD_DESC);
+      this.common.write(oprot);
+      oprot.writeFieldEnd();
+    }
+    oprot.writeFieldStop();
+    oprot.writeStructEnd();
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("StateSpoutSpec(");
+    boolean first = true;
+
+    sb.append("state_spout_object:");
+    if (this.state_spout_object == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.state_spout_object);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("common:");
+    if (this.common == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.common);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws TException {
+    // check for required fields
+    if (!is_set_state_spout_object()) {
+      throw new TProtocolException("Required field 'state_spout_object' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_common()) {
+      throw new TProtocolException("Required field 'common' is unset! Struct:" + toString());
+    }
+
+  }
+
+}
+
diff --git a/src/jvm/backtype/storm/generated/StormTopology.java b/src/jvm/backtype/storm/generated/StormTopology.java
new file mode 100644
index 0000000..087ebb4
--- /dev/null
+++ b/src/jvm/backtype/storm/generated/StormTopology.java
@@ -0,0 +1,648 @@
+/**
+ * Autogenerated by Thrift
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ */
+package backtype.storm.generated;
+
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.util.Arrays;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.thrift.*;
+import org.apache.thrift.async.*;
+import org.apache.thrift.meta_data.*;
+import org.apache.thrift.transport.*;
+import org.apache.thrift.protocol.*;
+
+public class StormTopology implements TBase<StormTopology, StormTopology._Fields>, java.io.Serializable, Cloneable {
+  private static final TStruct STRUCT_DESC = new TStruct("StormTopology");
+
+  private static final TField SPOUTS_FIELD_DESC = new TField("spouts", TType.MAP, (short)1);
+  private static final TField BOLTS_FIELD_DESC = new TField("bolts", TType.MAP, (short)2);
+  private static final TField STATE_SPOUTS_FIELD_DESC = new TField("state_spouts", TType.MAP, (short)3);
+
+  private Map<Integer,SpoutSpec> spouts;
+  private Map<Integer,Bolt> bolts;
+  private Map<Integer,StateSpoutSpec> state_spouts;
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements TFieldIdEnum {
+    SPOUTS((short)1, "spouts"),
+    BOLTS((short)2, "bolts"),
+    STATE_SPOUTS((short)3, "state_spouts");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // SPOUTS
+          return SPOUTS;
+        case 2: // BOLTS
+          return BOLTS;
+        case 3: // STATE_SPOUTS
+          return STATE_SPOUTS;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+
+  public static final Map<_Fields, FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, FieldMetaData> tmpMap = new EnumMap<_Fields, FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.SPOUTS, new FieldMetaData("spouts", TFieldRequirementType.REQUIRED, 
+        new MapMetaData(TType.MAP, 
+            new FieldValueMetaData(TType.I32), 
+            new StructMetaData(TType.STRUCT, SpoutSpec.class))));
+    tmpMap.put(_Fields.BOLTS, new FieldMetaData("bolts", TFieldRequirementType.REQUIRED, 
+        new MapMetaData(TType.MAP, 
+            new FieldValueMetaData(TType.I32), 
+            new StructMetaData(TType.STRUCT, Bolt.class))));
+    tmpMap.put(_Fields.STATE_SPOUTS, new FieldMetaData("state_spouts", TFieldRequirementType.REQUIRED, 
+        new MapMetaData(TType.MAP, 
+            new FieldValueMetaData(TType.I32), 
+            new StructMetaData(TType.STRUCT, StateSpoutSpec.class))));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    FieldMetaData.addStructMetaDataMap(StormTopology.class, metaDataMap);
+  }
+
+  public StormTopology() {
+  }
+
+  public StormTopology(
+    Map<Integer,SpoutSpec> spouts,
+    Map<Integer,Bolt> bolts,
+    Map<Integer,StateSpoutSpec> state_spouts)
+  {
+    this();
+    this.spouts = spouts;
+    this.bolts = bolts;
+    this.state_spouts = state_spouts;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public StormTopology(StormTopology other) {
+    if (other.is_set_spouts()) {
+      Map<Integer,SpoutSpec> __this__spouts = new HashMap<Integer,SpoutSpec>();
+      for (Map.Entry<Integer, SpoutSpec> other_element : other.spouts.entrySet()) {
+
+        Integer other_element_key = other_element.getKey();
+        SpoutSpec other_element_value = other_element.getValue();
+
+        Integer __this__spouts_copy_key = other_element_key;
+
+        SpoutSpec __this__spouts_copy_value = new SpoutSpec(other_element_value);
+
+        __this__spouts.put(__this__spouts_copy_key, __this__spouts_copy_value);
+      }
+      this.spouts = __this__spouts;
+    }
+    if (other.is_set_bolts()) {
+      Map<Integer,Bolt> __this__bolts = new HashMap<Integer,Bolt>();
+      for (Map.Entry<Integer, Bolt> other_element : other.bolts.entrySet()) {
+
+        Integer other_element_key = other_element.getKey();
+        Bolt other_element_value = other_element.getValue();
+
+        Integer __this__bolts_copy_key = other_element_key;
+
+        Bolt __this__bolts_copy_value = new Bolt(other_element_value);
+
+        __this__bolts.put(__this__bolts_copy_key, __this__bolts_copy_value);
+      }
+      this.bolts = __this__bolts;
+    }
+    if (other.is_set_state_spouts()) {
+      Map<Integer,StateSpoutSpec> __this__state_spouts = new HashMap<Integer,StateSpoutSpec>();
+      for (Map.Entry<Integer, StateSpoutSpec> other_element : other.state_spouts.entrySet()) {
+
+        Integer other_element_key = other_element.getKey();
+        StateSpoutSpec other_element_value = other_element.getValue();
+
+        Integer __this__state_spouts_copy_key = other_element_key;
+
+        StateSpoutSpec __this__state_spouts_copy_value = new StateSpoutSpec(other_element_value);
+
+        __this__state_spouts.put(__this__state_spouts_copy_key, __this__state_spouts_copy_value);
+      }
+      this.state_spouts = __this__state_spouts;
+    }
+  }
+
+  public StormTopology deepCopy() {
+    return new StormTopology(this);
+  }
+
+  @Deprecated
+  public StormTopology clone() {
+    return new StormTopology(this);
+  }
+
+  public int get_spouts_size() {
+    return (this.spouts == null) ? 0 : this.spouts.size();
+  }
+
+  public void put_to_spouts(int key, SpoutSpec val) {
+    if (this.spouts == null) {
+      this.spouts = new HashMap<Integer,SpoutSpec>();
+    }
+    this.spouts.put(key, val);
+  }
+
+  public Map<Integer,SpoutSpec> get_spouts() {
+    return this.spouts;
+  }
+
+  public void set_spouts(Map<Integer,SpoutSpec> spouts) {
+    this.spouts = spouts;
+  }
+
+  public void unset_spouts() {
+    this.spouts = null;
+  }
+
+  /** Returns true if field spouts is set (has been asigned a value) and false otherwise */
+  public boolean is_set_spouts() {
+    return this.spouts != null;
+  }
+
+  public void set_spouts_isSet(boolean value) {
+    if (!value) {
+      this.spouts = null;
+    }
+  }
+
+  public int get_bolts_size() {
+    return (this.bolts == null) ? 0 : this.bolts.size();
+  }
+
+  public void put_to_bolts(int key, Bolt val) {
+    if (this.bolts == null) {
+      this.bolts = new HashMap<Integer,Bolt>();
+    }
+    this.bolts.put(key, val);
+  }
+
+  public Map<Integer,Bolt> get_bolts() {
+    return this.bolts;
+  }
+
+  public void set_bolts(Map<Integer,Bolt> bolts) {
+    this.bolts = bolts;
+  }
+
+  public void unset_bolts() {
+    this.bolts = null;
+  }
+
+  /** Returns true if field bolts is set (has been asigned a value) and false otherwise */
+  public boolean is_set_bolts() {
+    return this.bolts != null;
+  }
+
+  public void set_bolts_isSet(boolean value) {
+    if (!value) {
+      this.bolts = null;
+    }
+  }
+
+  public int get_state_spouts_size() {
+    return (this.state_spouts == null) ? 0 : this.state_spouts.size();
+  }
+
+  public void put_to_state_spouts(int key, StateSpoutSpec val) {
+    if (this.state_spouts == null) {
+      this.state_spouts = new HashMap<Integer,StateSpoutSpec>();
+    }
+    this.state_spouts.put(key, val);
+  }
+
+  public Map<Integer,StateSpoutSpec> get_state_spouts() {
+    return this.state_spouts;
+  }
+
+  public void set_state_spouts(Map<Integer,StateSpoutSpec> state_spouts) {
+    this.state_spouts = state_spouts;
+  }
+
+  public void unset_state_spouts() {
+    this.state_spouts = null;
+  }
+
+  /** Returns true if field state_spouts is set (has been asigned a value) and false otherwise */
+  public boolean is_set_state_spouts() {
+    return this.state_spouts != null;
+  }
+
+  public void set_state_spouts_isSet(boolean value) {
+    if (!value) {
+      this.state_spouts = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case SPOUTS:
+      if (value == null) {
+        unset_spouts();
+      } else {
+        set_spouts((Map<Integer,SpoutSpec>)value);
+      }
+      break;
+
+    case BOLTS:
+      if (value == null) {
+        unset_bolts();
+      } else {
+        set_bolts((Map<Integer,Bolt>)value);
+      }
+      break;
+
+    case STATE_SPOUTS:
+      if (value == null) {
+        unset_state_spouts();
+      } else {
+        set_state_spouts((Map<Integer,StateSpoutSpec>)value);
+      }
+      break;
+
+    }
+  }
+
+  public void setFieldValue(int fieldID, Object value) {
+    setFieldValue(_Fields.findByThriftIdOrThrow(fieldID), value);
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case SPOUTS:
+      return get_spouts();
+
+    case BOLTS:
+      return get_bolts();
+
+    case STATE_SPOUTS:
+      return get_state_spouts();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  public Object getFieldValue(int fieldId) {
+    return getFieldValue(_Fields.findByThriftIdOrThrow(fieldId));
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been asigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    switch (field) {
+    case SPOUTS:
+      return is_set_spouts();
+    case BOLTS:
+      return is_set_bolts();
+    case STATE_SPOUTS:
+      return is_set_state_spouts();
+    }
+    throw new IllegalStateException();
+  }
+
+  public boolean isSet(int fieldID) {
+    return isSet(_Fields.findByThriftIdOrThrow(fieldID));
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof StormTopology)
+      return this.equals((StormTopology)that);
+    return false;
+  }
+
+  public boolean equals(StormTopology that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_spouts = true && this.is_set_spouts();
+    boolean that_present_spouts = true && that.is_set_spouts();
+    if (this_present_spouts || that_present_spouts) {
+      if (!(this_present_spouts && that_present_spouts))
+        return false;
+      if (!this.spouts.equals(that.spouts))
+        return false;
+    }
+
+    boolean this_present_bolts = true && this.is_set_bolts();
+    boolean that_present_bolts = true && that.is_set_bolts();
+    if (this_present_bolts || that_present_bolts) {
+      if (!(this_present_bolts && that_present_bolts))
+        return false;
+      if (!this.bolts.equals(that.bolts))
+        return false;
+    }
+
+    boolean this_present_state_spouts = true && this.is_set_state_spouts();
+    boolean that_present_state_spouts = true && that.is_set_state_spouts();
+    if (this_present_state_spouts || that_present_state_spouts) {
+      if (!(this_present_state_spouts && that_present_state_spouts))
+        return false;
+      if (!this.state_spouts.equals(that.state_spouts))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    HashCodeBuilder builder = new HashCodeBuilder();
+
+    boolean present_spouts = true && (is_set_spouts());
+    builder.append(present_spouts);
+    if (present_spouts)
+      builder.append(spouts);
+
+    boolean present_bolts = true && (is_set_bolts());
+    builder.append(present_bolts);
+    if (present_bolts)
+      builder.append(bolts);
+
+    boolean present_state_spouts = true && (is_set_state_spouts());
+    builder.append(present_state_spouts);
+    if (present_state_spouts)
+      builder.append(state_spouts);
+
+    return builder.toHashCode();
+  }
+
+  public int compareTo(StormTopology other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+    StormTopology typedOther = (StormTopology)other;
+
+    lastComparison = Boolean.valueOf(is_set_spouts()).compareTo(typedOther.is_set_spouts());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_spouts()) {      lastComparison = TBaseHelper.compareTo(this.spouts, typedOther.spouts);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_bolts()).compareTo(typedOther.is_set_bolts());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_bolts()) {      lastComparison = TBaseHelper.compareTo(this.bolts, typedOther.bolts);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_state_spouts()).compareTo(typedOther.is_set_state_spouts());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_state_spouts()) {      lastComparison = TBaseHelper.compareTo(this.state_spouts, typedOther.state_spouts);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public void read(TProtocol iprot) throws TException {
+    TField field;
+    iprot.readStructBegin();
+    while (true)
+    {
+      field = iprot.readFieldBegin();
+      if (field.type == TType.STOP) { 
+        break;
+      }
+      switch (field.id) {
+        case 1: // SPOUTS
+          if (field.type == TType.MAP) {
+            {
+              TMap _map18 = iprot.readMapBegin();
+              this.spouts = new HashMap<Integer,SpoutSpec>(2*_map18.size);
+              for (int _i19 = 0; _i19 < _map18.size; ++_i19)
+              {
+                int _key20;
+                SpoutSpec _val21;
+                _key20 = iprot.readI32();
+                _val21 = new SpoutSpec();
+                _val21.read(iprot);
+                this.spouts.put(_key20, _val21);
+              }
+              iprot.readMapEnd();
+            }
+          } else { 
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 2: // BOLTS
+          if (field.type == TType.MAP) {
+            {
+              TMap _map22 = iprot.readMapBegin();
+              this.bolts = new HashMap<Integer,Bolt>(2*_map22.size);
+              for (int _i23 = 0; _i23 < _map22.size; ++_i23)
+              {
+                int _key24;
+                Bolt _val25;
+                _key24 = iprot.readI32();
+                _val25 = new Bolt();
+                _val25.read(iprot);
+                this.bolts.put(_key24, _val25);
+              }
+              iprot.readMapEnd();
+            }
+          } else { 
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 3: // STATE_SPOUTS
+          if (field.type == TType.MAP) {
+            {
+              TMap _map26 = iprot.readMapBegin();
+              this.state_spouts = new HashMap<Integer,StateSpoutSpec>(2*_map26.size);
+              for (int _i27 = 0; _i27 < _map26.size; ++_i27)
+              {
+                int _key28;
+                StateSpoutSpec _val29;
+                _key28 = iprot.readI32();
+                _val29 = new StateSpoutSpec();
+                _val29.read(iprot);
+                this.state_spouts.put(_key28, _val29);
+              }
+              iprot.readMapEnd();
+            }
+          } else { 
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        default:
+          TProtocolUtil.skip(iprot, field.type);
+      }
+      iprot.readFieldEnd();
+    }
+    iprot.readStructEnd();
+    validate();
+  }
+
+  public void write(TProtocol oprot) throws TException {
+    validate();
+
+    oprot.writeStructBegin(STRUCT_DESC);
+    if (this.spouts != null) {
+      oprot.writeFieldBegin(SPOUTS_FIELD_DESC);
+      {
+        oprot.writeMapBegin(new TMap(TType.I32, TType.STRUCT, this.spouts.size()));
+        for (Map.Entry<Integer, SpoutSpec> _iter30 : this.spouts.entrySet())
+        {
+          oprot.writeI32(_iter30.getKey());
+          _iter30.getValue().write(oprot);
+        }
+        oprot.writeMapEnd();
+      }
+      oprot.writeFieldEnd();
+    }
+    if (this.bolts != null) {
+      oprot.writeFieldBegin(BOLTS_FIELD_DESC);
+      {
+        oprot.writeMapBegin(new TMap(TType.I32, TType.STRUCT, this.bolts.size()));
+        for (Map.Entry<Integer, Bolt> _iter31 : this.bolts.entrySet())
+        {
+          oprot.writeI32(_iter31.getKey());
+          _iter31.getValue().write(oprot);
+        }
+        oprot.writeMapEnd();
+      }
+      oprot.writeFieldEnd();
+    }
+    if (this.state_spouts != null) {
+      oprot.writeFieldBegin(STATE_SPOUTS_FIELD_DESC);
+      {
+        oprot.writeMapBegin(new TMap(TType.I32, TType.STRUCT, this.state_spouts.size()));
+        for (Map.Entry<Integer, StateSpoutSpec> _iter32 : this.state_spouts.entrySet())
+        {
+          oprot.writeI32(_iter32.getKey());
+          _iter32.getValue().write(oprot);
+        }
+        oprot.writeMapEnd();
+      }
+      oprot.writeFieldEnd();
+    }
+    oprot.writeFieldStop();
+    oprot.writeStructEnd();
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("StormTopology(");
+    boolean first = true;
+
+    sb.append("spouts:");
+    if (this.spouts == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.spouts);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("bolts:");
+    if (this.bolts == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.bolts);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("state_spouts:");
+    if (this.state_spouts == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.state_spouts);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws TException {
+    // check for required fields
+    if (!is_set_spouts()) {
+      throw new TProtocolException("Required field 'spouts' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_bolts()) {
+      throw new TProtocolException("Required field 'bolts' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_state_spouts()) {
+      throw new TProtocolException("Required field 'state_spouts' is unset! Struct:" + toString());
+    }
+
+  }
+
+}
+
diff --git a/src/jvm/backtype/storm/generated/StreamInfo.java b/src/jvm/backtype/storm/generated/StreamInfo.java
new file mode 100644
index 0000000..ce04ce6
--- /dev/null
+++ b/src/jvm/backtype/storm/generated/StreamInfo.java
@@ -0,0 +1,449 @@
+/**
+ * Autogenerated by Thrift
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ */
+package backtype.storm.generated;
+
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.util.Arrays;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.thrift.*;
+import org.apache.thrift.async.*;
+import org.apache.thrift.meta_data.*;
+import org.apache.thrift.transport.*;
+import org.apache.thrift.protocol.*;
+
+public class StreamInfo implements TBase<StreamInfo, StreamInfo._Fields>, java.io.Serializable, Cloneable {
+  private static final TStruct STRUCT_DESC = new TStruct("StreamInfo");
+
+  private static final TField OUTPUT_FIELDS_FIELD_DESC = new TField("output_fields", TType.LIST, (short)1);
+  private static final TField DIRECT_FIELD_DESC = new TField("direct", TType.BOOL, (short)2);
+
+  private List<String> output_fields;
+  private boolean direct;
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements TFieldIdEnum {
+    OUTPUT_FIELDS((short)1, "output_fields"),
+    DIRECT((short)2, "direct");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // OUTPUT_FIELDS
+          return OUTPUT_FIELDS;
+        case 2: // DIRECT
+          return DIRECT;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __DIRECT_ISSET_ID = 0;
+  private BitSet __isset_bit_vector = new BitSet(1);
+
+  public static final Map<_Fields, FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, FieldMetaData> tmpMap = new EnumMap<_Fields, FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.OUTPUT_FIELDS, new FieldMetaData("output_fields", TFieldRequirementType.REQUIRED, 
+        new ListMetaData(TType.LIST, 
+            new FieldValueMetaData(TType.STRING))));
+    tmpMap.put(_Fields.DIRECT, new FieldMetaData("direct", TFieldRequirementType.REQUIRED, 
+        new FieldValueMetaData(TType.BOOL)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    FieldMetaData.addStructMetaDataMap(StreamInfo.class, metaDataMap);
+  }
+
+  public StreamInfo() {
+  }
+
+  public StreamInfo(
+    List<String> output_fields,
+    boolean direct)
+  {
+    this();
+    this.output_fields = output_fields;
+    this.direct = direct;
+    set_direct_isSet(true);
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public StreamInfo(StreamInfo other) {
+    __isset_bit_vector.clear();
+    __isset_bit_vector.or(other.__isset_bit_vector);
+    if (other.is_set_output_fields()) {
+      List<String> __this__output_fields = new ArrayList<String>();
+      for (String other_element : other.output_fields) {
+        __this__output_fields.add(other_element);
+      }
+      this.output_fields = __this__output_fields;
+    }
+    this.direct = other.direct;
+  }
+
+  public StreamInfo deepCopy() {
+    return new StreamInfo(this);
+  }
+
+  @Deprecated
+  public StreamInfo clone() {
+    return new StreamInfo(this);
+  }
+
+  public int get_output_fields_size() {
+    return (this.output_fields == null) ? 0 : this.output_fields.size();
+  }
+
+  public java.util.Iterator<String> get_output_fields_iterator() {
+    return (this.output_fields == null) ? null : this.output_fields.iterator();
+  }
+
+  public void add_to_output_fields(String elem) {
+    if (this.output_fields == null) {
+      this.output_fields = new ArrayList<String>();
+    }
+    this.output_fields.add(elem);
+  }
+
+  public List<String> get_output_fields() {
+    return this.output_fields;
+  }
+
+  public void set_output_fields(List<String> output_fields) {
+    this.output_fields = output_fields;
+  }
+
+  public void unset_output_fields() {
+    this.output_fields = null;
+  }
+
+  /** Returns true if field output_fields is set (has been asigned a value) and false otherwise */
+  public boolean is_set_output_fields() {
+    return this.output_fields != null;
+  }
+
+  public void set_output_fields_isSet(boolean value) {
+    if (!value) {
+      this.output_fields = null;
+    }
+  }
+
+  public boolean is_direct() {
+    return this.direct;
+  }
+
+  public void set_direct(boolean direct) {
+    this.direct = direct;
+    set_direct_isSet(true);
+  }
+
+  public void unset_direct() {
+    __isset_bit_vector.clear(__DIRECT_ISSET_ID);
+  }
+
+  /** Returns true if field direct is set (has been asigned a value) and false otherwise */
+  public boolean is_set_direct() {
+    return __isset_bit_vector.get(__DIRECT_ISSET_ID);
+  }
+
+  public void set_direct_isSet(boolean value) {
+    __isset_bit_vector.set(__DIRECT_ISSET_ID, value);
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case OUTPUT_FIELDS:
+      if (value == null) {
+        unset_output_fields();
+      } else {
+        set_output_fields((List<String>)value);
+      }
+      break;
+
+    case DIRECT:
+      if (value == null) {
+        unset_direct();
+      } else {
+        set_direct((Boolean)value);
+      }
+      break;
+
+    }
+  }
+
+  public void setFieldValue(int fieldID, Object value) {
+    setFieldValue(_Fields.findByThriftIdOrThrow(fieldID), value);
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case OUTPUT_FIELDS:
+      return get_output_fields();
+
+    case DIRECT:
+      return new Boolean(is_direct());
+
+    }
+    throw new IllegalStateException();
+  }
+
+  public Object getFieldValue(int fieldId) {
+    return getFieldValue(_Fields.findByThriftIdOrThrow(fieldId));
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been asigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    switch (field) {
+    case OUTPUT_FIELDS:
+      return is_set_output_fields();
+    case DIRECT:
+      return is_set_direct();
+    }
+    throw new IllegalStateException();
+  }
+
+  public boolean isSet(int fieldID) {
+    return isSet(_Fields.findByThriftIdOrThrow(fieldID));
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof StreamInfo)
+      return this.equals((StreamInfo)that);
+    return false;
+  }
+
+  public boolean equals(StreamInfo that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_output_fields = true && this.is_set_output_fields();
+    boolean that_present_output_fields = true && that.is_set_output_fields();
+    if (this_present_output_fields || that_present_output_fields) {
+      if (!(this_present_output_fields && that_present_output_fields))
+        return false;
+      if (!this.output_fields.equals(that.output_fields))
+        return false;
+    }
+
+    boolean this_present_direct = true;
+    boolean that_present_direct = true;
+    if (this_present_direct || that_present_direct) {
+      if (!(this_present_direct && that_present_direct))
+        return false;
+      if (this.direct != that.direct)
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    HashCodeBuilder builder = new HashCodeBuilder();
+
+    boolean present_output_fields = true && (is_set_output_fields());
+    builder.append(present_output_fields);
+    if (present_output_fields)
+      builder.append(output_fields);
+
+    boolean present_direct = true;
+    builder.append(present_direct);
+    if (present_direct)
+      builder.append(direct);
+
+    return builder.toHashCode();
+  }
+
+  public int compareTo(StreamInfo other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+    StreamInfo typedOther = (StreamInfo)other;
+
+    lastComparison = Boolean.valueOf(is_set_output_fields()).compareTo(typedOther.is_set_output_fields());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_output_fields()) {      lastComparison = TBaseHelper.compareTo(this.output_fields, typedOther.output_fields);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_direct()).compareTo(typedOther.is_set_direct());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_direct()) {      lastComparison = TBaseHelper.compareTo(this.direct, typedOther.direct);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public void read(TProtocol iprot) throws TException {
+    TField field;
+    iprot.readStructBegin();
+    while (true)
+    {
+      field = iprot.readFieldBegin();
+      if (field.type == TType.STOP) { 
+        break;
+      }
+      switch (field.id) {
+        case 1: // OUTPUT_FIELDS
+          if (field.type == TType.LIST) {
+            {
+              TList _list4 = iprot.readListBegin();
+              this.output_fields = new ArrayList<String>(_list4.size);
+              for (int _i5 = 0; _i5 < _list4.size; ++_i5)
+              {
+                String _elem6;
+                _elem6 = iprot.readString();
+                this.output_fields.add(_elem6);
+              }
+              iprot.readListEnd();
+            }
+          } else { 
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 2: // DIRECT
+          if (field.type == TType.BOOL) {
+            this.direct = iprot.readBool();
+            set_direct_isSet(true);
+          } else { 
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        default:
+          TProtocolUtil.skip(iprot, field.type);
+      }
+      iprot.readFieldEnd();
+    }
+    iprot.readStructEnd();
+    validate();
+  }
+
+  public void write(TProtocol oprot) throws TException {
+    validate();
+
+    oprot.writeStructBegin(STRUCT_DESC);
+    if (this.output_fields != null) {
+      oprot.writeFieldBegin(OUTPUT_FIELDS_FIELD_DESC);
+      {
+        oprot.writeListBegin(new TList(TType.STRING, this.output_fields.size()));
+        for (String _iter7 : this.output_fields)
+        {
+          oprot.writeString(_iter7);
+        }
+        oprot.writeListEnd();
+      }
+      oprot.writeFieldEnd();
+    }
+    oprot.writeFieldBegin(DIRECT_FIELD_DESC);
+    oprot.writeBool(this.direct);
+    oprot.writeFieldEnd();
+    oprot.writeFieldStop();
+    oprot.writeStructEnd();
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("StreamInfo(");
+    boolean first = true;
+
+    sb.append("output_fields:");
+    if (this.output_fields == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.output_fields);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("direct:");
+    sb.append(this.direct);
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws TException {
+    // check for required fields
+    if (!is_set_output_fields()) {
+      throw new TProtocolException("Required field 'output_fields' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_direct()) {
+      throw new TProtocolException("Required field 'direct' is unset! Struct:" + toString());
+    }
+
+  }
+
+}
+
diff --git a/src/jvm/backtype/storm/generated/SupervisorSummary.java b/src/jvm/backtype/storm/generated/SupervisorSummary.java
new file mode 100644
index 0000000..b027434
--- /dev/null
+++ b/src/jvm/backtype/storm/generated/SupervisorSummary.java
@@ -0,0 +1,590 @@
+/**
+ * Autogenerated by Thrift
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ */
+package backtype.storm.generated;
+
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.util.Arrays;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.thrift.*;
+import org.apache.thrift.async.*;
+import org.apache.thrift.meta_data.*;
+import org.apache.thrift.transport.*;
+import org.apache.thrift.protocol.*;
+
+public class SupervisorSummary implements TBase<SupervisorSummary, SupervisorSummary._Fields>, java.io.Serializable, Cloneable {
+  private static final TStruct STRUCT_DESC = new TStruct("SupervisorSummary");
+
+  private static final TField HOST_FIELD_DESC = new TField("host", TType.STRING, (short)1);
+  private static final TField UPTIME_SECS_FIELD_DESC = new TField("uptime_secs", TType.I32, (short)2);
+  private static final TField NUM_WORKERS_FIELD_DESC = new TField("num_workers", TType.I32, (short)3);
+  private static final TField NUM_USED_WORKERS_FIELD_DESC = new TField("num_used_workers", TType.I32, (short)4);
+
+  private String host;
+  private int uptime_secs;
+  private int num_workers;
+  private int num_used_workers;
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements TFieldIdEnum {
+    HOST((short)1, "host"),
+    UPTIME_SECS((short)2, "uptime_secs"),
+    NUM_WORKERS((short)3, "num_workers"),
+    NUM_USED_WORKERS((short)4, "num_used_workers");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // HOST
+          return HOST;
+        case 2: // UPTIME_SECS
+          return UPTIME_SECS;
+        case 3: // NUM_WORKERS
+          return NUM_WORKERS;
+        case 4: // NUM_USED_WORKERS
+          return NUM_USED_WORKERS;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __UPTIME_SECS_ISSET_ID = 0;
+  private static final int __NUM_WORKERS_ISSET_ID = 1;
+  private static final int __NUM_USED_WORKERS_ISSET_ID = 2;
+  private BitSet __isset_bit_vector = new BitSet(3);
+
+  public static final Map<_Fields, FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, FieldMetaData> tmpMap = new EnumMap<_Fields, FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.HOST, new FieldMetaData("host", TFieldRequirementType.REQUIRED, 
+        new FieldValueMetaData(TType.STRING)));
+    tmpMap.put(_Fields.UPTIME_SECS, new FieldMetaData("uptime_secs", TFieldRequirementType.REQUIRED, 
+        new FieldValueMetaData(TType.I32)));
+    tmpMap.put(_Fields.NUM_WORKERS, new FieldMetaData("num_workers", TFieldRequirementType.REQUIRED, 
+        new FieldValueMetaData(TType.I32)));
+    tmpMap.put(_Fields.NUM_USED_WORKERS, new FieldMetaData("num_used_workers", TFieldRequirementType.REQUIRED, 
+        new FieldValueMetaData(TType.I32)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    FieldMetaData.addStructMetaDataMap(SupervisorSummary.class, metaDataMap);
+  }
+
+  public SupervisorSummary() {
+  }
+
+  public SupervisorSummary(
+    String host,
+    int uptime_secs,
+    int num_workers,
+    int num_used_workers)
+  {
+    this();
+    this.host = host;
+    this.uptime_secs = uptime_secs;
+    set_uptime_secs_isSet(true);
+    this.num_workers = num_workers;
+    set_num_workers_isSet(true);
+    this.num_used_workers = num_used_workers;
+    set_num_used_workers_isSet(true);
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public SupervisorSummary(SupervisorSummary other) {
+    __isset_bit_vector.clear();
+    __isset_bit_vector.or(other.__isset_bit_vector);
+    if (other.is_set_host()) {
+      this.host = other.host;
+    }
+    this.uptime_secs = other.uptime_secs;
+    this.num_workers = other.num_workers;
+    this.num_used_workers = other.num_used_workers;
+  }
+
+  public SupervisorSummary deepCopy() {
+    return new SupervisorSummary(this);
+  }
+
+  @Deprecated
+  public SupervisorSummary clone() {
+    return new SupervisorSummary(this);
+  }
+
+  public String get_host() {
+    return this.host;
+  }
+
+  public void set_host(String host) {
+    this.host = host;
+  }
+
+  public void unset_host() {
+    this.host = null;
+  }
+
+  /** Returns true if field host is set (has been asigned a value) and false otherwise */
+  public boolean is_set_host() {
+    return this.host != null;
+  }
+
+  public void set_host_isSet(boolean value) {
+    if (!value) {
+      this.host = null;
+    }
+  }
+
+  public int get_uptime_secs() {
+    return this.uptime_secs;
+  }
+
+  public void set_uptime_secs(int uptime_secs) {
+    this.uptime_secs = uptime_secs;
+    set_uptime_secs_isSet(true);
+  }
+
+  public void unset_uptime_secs() {
+    __isset_bit_vector.clear(__UPTIME_SECS_ISSET_ID);
+  }
+
+  /** Returns true if field uptime_secs is set (has been asigned a value) and false otherwise */
+  public boolean is_set_uptime_secs() {
+    return __isset_bit_vector.get(__UPTIME_SECS_ISSET_ID);
+  }
+
+  public void set_uptime_secs_isSet(boolean value) {
+    __isset_bit_vector.set(__UPTIME_SECS_ISSET_ID, value);
+  }
+
+  public int get_num_workers() {
+    return this.num_workers;
+  }
+
+  public void set_num_workers(int num_workers) {
+    this.num_workers = num_workers;
+    set_num_workers_isSet(true);
+  }
+
+  public void unset_num_workers() {
+    __isset_bit_vector.clear(__NUM_WORKERS_ISSET_ID);
+  }
+
+  /** Returns true if field num_workers is set (has been asigned a value) and false otherwise */
+  public boolean is_set_num_workers() {
+    return __isset_bit_vector.get(__NUM_WORKERS_ISSET_ID);
+  }
+
+  public void set_num_workers_isSet(boolean value) {
+    __isset_bit_vector.set(__NUM_WORKERS_ISSET_ID, value);
+  }
+
+  public int get_num_used_workers() {
+    return this.num_used_workers;
+  }
+
+  public void set_num_used_workers(int num_used_workers) {
+    this.num_used_workers = num_used_workers;
+    set_num_used_workers_isSet(true);
+  }
+
+  public void unset_num_used_workers() {
+    __isset_bit_vector.clear(__NUM_USED_WORKERS_ISSET_ID);
+  }
+
+  /** Returns true if field num_used_workers is set (has been asigned a value) and false otherwise */
+  public boolean is_set_num_used_workers() {
+    return __isset_bit_vector.get(__NUM_USED_WORKERS_ISSET_ID);
+  }
+
+  public void set_num_used_workers_isSet(boolean value) {
+    __isset_bit_vector.set(__NUM_USED_WORKERS_ISSET_ID, value);
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case HOST:
+      if (value == null) {
+        unset_host();
+      } else {
+        set_host((String)value);
+      }
+      break;
+
+    case UPTIME_SECS:
+      if (value == null) {
+        unset_uptime_secs();
+      } else {
+        set_uptime_secs((Integer)value);
+      }
+      break;
+
+    case NUM_WORKERS:
+      if (value == null) {
+        unset_num_workers();
+      } else {
+        set_num_workers((Integer)value);
+      }
+      break;
+
+    case NUM_USED_WORKERS:
+      if (value == null) {
+        unset_num_used_workers();
+      } else {
+        set_num_used_workers((Integer)value);
+      }
+      break;
+
+    }
+  }
+
+  public void setFieldValue(int fieldID, Object value) {
+    setFieldValue(_Fields.findByThriftIdOrThrow(fieldID), value);
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case HOST:
+      return get_host();
+
+    case UPTIME_SECS:
+      return new Integer(get_uptime_secs());
+
+    case NUM_WORKERS:
+      return new Integer(get_num_workers());
+
+    case NUM_USED_WORKERS:
+      return new Integer(get_num_used_workers());
+
+    }
+    throw new IllegalStateException();
+  }
+
+  public Object getFieldValue(int fieldId) {
+    return getFieldValue(_Fields.findByThriftIdOrThrow(fieldId));
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been asigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    switch (field) {
+    case HOST:
+      return is_set_host();
+    case UPTIME_SECS:
+      return is_set_uptime_secs();
+    case NUM_WORKERS:
+      return is_set_num_workers();
+    case NUM_USED_WORKERS:
+      return is_set_num_used_workers();
+    }
+    throw new IllegalStateException();
+  }
+
+  public boolean isSet(int fieldID) {
+    return isSet(_Fields.findByThriftIdOrThrow(fieldID));
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof SupervisorSummary)
+      return this.equals((SupervisorSummary)that);
+    return false;
+  }
+
+  public boolean equals(SupervisorSummary that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_host = true && this.is_set_host();
+    boolean that_present_host = true && that.is_set_host();
+    if (this_present_host || that_present_host) {
+      if (!(this_present_host && that_present_host))
+        return false;
+      if (!this.host.equals(that.host))
+        return false;
+    }
+
+    boolean this_present_uptime_secs = true;
+    boolean that_present_uptime_secs = true;
+    if (this_present_uptime_secs || that_present_uptime_secs) {
+      if (!(this_present_uptime_secs && that_present_uptime_secs))
+        return false;
+      if (this.uptime_secs != that.uptime_secs)
+        return false;
+    }
+
+    boolean this_present_num_workers = true;
+    boolean that_present_num_workers = true;
+    if (this_present_num_workers || that_present_num_workers) {
+      if (!(this_present_num_workers && that_present_num_workers))
+        return false;
+      if (this.num_workers != that.num_workers)
+        return false;
+    }
+
+    boolean this_present_num_used_workers = true;
+    boolean that_present_num_used_workers = true;
+    if (this_present_num_used_workers || that_present_num_used_workers) {
+      if (!(this_present_num_used_workers && that_present_num_used_workers))
+        return false;
+      if (this.num_used_workers != that.num_used_workers)
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    HashCodeBuilder builder = new HashCodeBuilder();
+
+    boolean present_host = true && (is_set_host());
+    builder.append(present_host);
+    if (present_host)
+      builder.append(host);
+
+    boolean present_uptime_secs = true;
+    builder.append(present_uptime_secs);
+    if (present_uptime_secs)
+      builder.append(uptime_secs);
+
+    boolean present_num_workers = true;
+    builder.append(present_num_workers);
+    if (present_num_workers)
+      builder.append(num_workers);
+
+    boolean present_num_used_workers = true;
+    builder.append(present_num_used_workers);
+    if (present_num_used_workers)
+      builder.append(num_used_workers);
+
+    return builder.toHashCode();
+  }
+
+  public int compareTo(SupervisorSummary other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+    SupervisorSummary typedOther = (SupervisorSummary)other;
+
+    lastComparison = Boolean.valueOf(is_set_host()).compareTo(typedOther.is_set_host());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_host()) {      lastComparison = TBaseHelper.compareTo(this.host, typedOther.host);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_uptime_secs()).compareTo(typedOther.is_set_uptime_secs());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_uptime_secs()) {      lastComparison = TBaseHelper.compareTo(this.uptime_secs, typedOther.uptime_secs);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_num_workers()).compareTo(typedOther.is_set_num_workers());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_num_workers()) {      lastComparison = TBaseHelper.compareTo(this.num_workers, typedOther.num_workers);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_num_used_workers()).compareTo(typedOther.is_set_num_used_workers());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_num_used_workers()) {      lastComparison = TBaseHelper.compareTo(this.num_used_workers, typedOther.num_used_workers);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public void read(TProtocol iprot) throws TException {
+    TField field;
+    iprot.readStructBegin();
+    while (true)
+    {
+      field = iprot.readFieldBegin();
+      if (field.type == TType.STOP) { 
+        break;
+      }
+      switch (field.id) {
+        case 1: // HOST
+          if (field.type == TType.STRING) {
+            this.host = iprot.readString();
+          } else { 
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 2: // UPTIME_SECS
+          if (field.type == TType.I32) {
+            this.uptime_secs = iprot.readI32();
+            set_uptime_secs_isSet(true);
+          } else { 
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 3: // NUM_WORKERS
+          if (field.type == TType.I32) {
+            this.num_workers = iprot.readI32();
+            set_num_workers_isSet(true);
+          } else { 
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 4: // NUM_USED_WORKERS
+          if (field.type == TType.I32) {
+            this.num_used_workers = iprot.readI32();
+            set_num_used_workers_isSet(true);
+          } else { 
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        default:
+          TProtocolUtil.skip(iprot, field.type);
+      }
+      iprot.readFieldEnd();
+    }
+    iprot.readStructEnd();
+    validate();
+  }
+
+  public void write(TProtocol oprot) throws TException {
+    validate();
+
+    oprot.writeStructBegin(STRUCT_DESC);
+    if (this.host != null) {
+      oprot.writeFieldBegin(HOST_FIELD_DESC);
+      oprot.writeString(this.host);
+      oprot.writeFieldEnd();
+    }
+    oprot.writeFieldBegin(UPTIME_SECS_FIELD_DESC);
+    oprot.writeI32(this.uptime_secs);
+    oprot.writeFieldEnd();
+    oprot.writeFieldBegin(NUM_WORKERS_FIELD_DESC);
+    oprot.writeI32(this.num_workers);
+    oprot.writeFieldEnd();
+    oprot.writeFieldBegin(NUM_USED_WORKERS_FIELD_DESC);
+    oprot.writeI32(this.num_used_workers);
+    oprot.writeFieldEnd();
+    oprot.writeFieldStop();
+    oprot.writeStructEnd();
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("SupervisorSummary(");
+    boolean first = true;
+
+    sb.append("host:");
+    if (this.host == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.host);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("uptime_secs:");
+    sb.append(this.uptime_secs);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("num_workers:");
+    sb.append(this.num_workers);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("num_used_workers:");
+    sb.append(this.num_used_workers);
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws TException {
+    // check for required fields
+    if (!is_set_host()) {
+      throw new TProtocolException("Required field 'host' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_uptime_secs()) {
+      throw new TProtocolException("Required field 'uptime_secs' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_num_workers()) {
+      throw new TProtocolException("Required field 'num_workers' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_num_used_workers()) {
+      throw new TProtocolException("Required field 'num_used_workers' is unset! Struct:" + toString());
+    }
+
+  }
+
+}
+
diff --git a/src/jvm/backtype/storm/generated/TaskSpecificStats.java b/src/jvm/backtype/storm/generated/TaskSpecificStats.java
new file mode 100644
index 0000000..2506f31
--- /dev/null
+++ b/src/jvm/backtype/storm/generated/TaskSpecificStats.java
@@ -0,0 +1,302 @@
+/**
+ * Autogenerated by Thrift
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ */
+package backtype.storm.generated;
+
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.util.Arrays;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.thrift.*;
+import org.apache.thrift.async.*;
+import org.apache.thrift.meta_data.*;
+import org.apache.thrift.transport.*;
+import org.apache.thrift.protocol.*;
+
+public class TaskSpecificStats extends TUnion<TaskSpecificStats, TaskSpecificStats._Fields> {
+  private static final TStruct STRUCT_DESC = new TStruct("TaskSpecificStats");
+  private static final TField BOLT_FIELD_DESC = new TField("bolt", TType.STRUCT, (short)1);
+  private static final TField SPOUT_FIELD_DESC = new TField("spout", TType.STRUCT, (short)2);
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements TFieldIdEnum {
+    BOLT((short)1, "bolt"),
+    SPOUT((short)2, "spout");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // BOLT
+          return BOLT;
+        case 2: // SPOUT
+          return SPOUT;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  public static final Map<_Fields, FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, FieldMetaData> tmpMap = new EnumMap<_Fields, FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.BOLT, new FieldMetaData("bolt", TFieldRequirementType.DEFAULT, 
+        new StructMetaData(TType.STRUCT, BoltStats.class)));
+    tmpMap.put(_Fields.SPOUT, new FieldMetaData("spout", TFieldRequirementType.DEFAULT, 
+        new StructMetaData(TType.STRUCT, SpoutStats.class)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    FieldMetaData.addStructMetaDataMap(TaskSpecificStats.class, metaDataMap);
+  }
+
+  public TaskSpecificStats() {
+    super();
+  }
+
+  public TaskSpecificStats(_Fields setField, Object value) {
+    super(setField, value);
+  }
+
+  public TaskSpecificStats(TaskSpecificStats other) {
+    super(other);
+  }
+  public TaskSpecificStats deepCopy() {
+    return new TaskSpecificStats(this);
+  }
+
+  public static TaskSpecificStats bolt(BoltStats value) {
+    TaskSpecificStats x = new TaskSpecificStats();
+    x.set_bolt(value);
+    return x;
+  }
+
+  public static TaskSpecificStats spout(SpoutStats value) {
+    TaskSpecificStats x = new TaskSpecificStats();
+    x.set_spout(value);
+    return x;
+  }
+
+
+  @Override
+  protected void checkType(_Fields setField, Object value) throws ClassCastException {
+    switch (setField) {
+      case BOLT:
+        if (value instanceof BoltStats) {
+          break;
+        }
+        throw new ClassCastException("Was expecting value of type BoltStats for field 'bolt', but got " + value.getClass().getSimpleName());
+      case SPOUT:
+        if (value instanceof SpoutStats) {
+          break;
+        }
+        throw new ClassCastException("Was expecting value of type SpoutStats for field 'spout', but got " + value.getClass().getSimpleName());
+      default:
+        throw new IllegalArgumentException("Unknown field id " + setField);
+    }
+  }
+
+  @Override
+  protected Object readValue(TProtocol iprot, TField field) throws TException {
+    _Fields setField = _Fields.findByThriftId(field.id);
+    if (setField != null) {
+      switch (setField) {
+        case BOLT:
+          if (field.type == BOLT_FIELD_DESC.type) {
+            BoltStats bolt;
+            bolt = new BoltStats();
+            bolt.read(iprot);
+            return bolt;
+          } else {
+            TProtocolUtil.skip(iprot, field.type);
+            return null;
+          }
+        case SPOUT:
+          if (field.type == SPOUT_FIELD_DESC.type) {
+            SpoutStats spout;
+            spout = new SpoutStats();
+            spout.read(iprot);
+            return spout;
+          } else {
+            TProtocolUtil.skip(iprot, field.type);
+            return null;
+          }
+        default:
+          throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!");
+      }
+    } else {
+      TProtocolUtil.skip(iprot, field.type);
+      return null;
+    }
+  }
+
+  @Override
+  protected void writeValue(TProtocol oprot) throws TException {
+    switch (setField_) {
+      case BOLT:
+        BoltStats bolt = (BoltStats)value_;
+        bolt.write(oprot);
+        return;
+      case SPOUT:
+        SpoutStats spout = (SpoutStats)value_;
+        spout.write(oprot);
+        return;
+      default:
+        throw new IllegalStateException("Cannot write union with unknown field " + setField_);
+    }
+  }
+
+  @Override
+  protected TField getFieldDesc(_Fields setField) {
+    switch (setField) {
+      case BOLT:
+        return BOLT_FIELD_DESC;
+      case SPOUT:
+        return SPOUT_FIELD_DESC;
+      default:
+        throw new IllegalArgumentException("Unknown field id " + setField);
+    }
+  }
+
+  @Override
+  protected TStruct getStructDesc() {
+    return STRUCT_DESC;
+  }
+
+  @Override
+  protected _Fields enumForId(short id) {
+    return _Fields.findByThriftIdOrThrow(id);
+  }
+
+  public BoltStats get_bolt() {
+    if (getSetField() == _Fields.BOLT) {
+      return (BoltStats)getFieldValue();
+    } else {
+      throw new RuntimeException("Cannot get field 'bolt' because union is currently set to " + getFieldDesc(getSetField()).name);
+    }
+  }
+
+  public void set_bolt(BoltStats value) {
+    if (value == null) throw new NullPointerException();
+    setField_ = _Fields.BOLT;
+    value_ = value;
+  }
+
+  public SpoutStats get_spout() {
+    if (getSetField() == _Fields.SPOUT) {
+      return (SpoutStats)getFieldValue();
+    } else {
+      throw new RuntimeException("Cannot get field 'spout' because union is currently set to " + getFieldDesc(getSetField()).name);
+    }
+  }
+
+  public void set_spout(SpoutStats value) {
+    if (value == null) throw new NullPointerException();
+    setField_ = _Fields.SPOUT;
+    value_ = value;
+  }
+
+  public boolean equals(Object other) {
+    if (other instanceof TaskSpecificStats) {
+      return equals((TaskSpecificStats)other);
+    } else {
+      return false;
+    }
+  }
+
+  public boolean equals(TaskSpecificStats other) {
+    return other != null && getSetField() == other.getSetField() && ((value_ instanceof byte[]) ? 
+      Arrays.equals((byte[])getFieldValue(), (byte[])other.getFieldValue()) : getFieldValue().equals(other.getFieldValue()));
+  }
+
+  @Override
+  public int compareTo(TaskSpecificStats other) {
+    int lastComparison = TBaseHelper.compareTo(getSetField(), other.getSetField());
+    if (lastComparison == 0) {
+      Object myValue = getFieldValue();
+      if (myValue instanceof byte[]) {
+        return TBaseHelper.compareTo((byte[])myValue, (byte[])other.getFieldValue());
+      } else if (myValue instanceof List) {
+        return TBaseHelper.compareTo((List)myValue, (List)other.getFieldValue());
+      } else if (myValue instanceof Set) {
+        return TBaseHelper.compareTo((Set)myValue, (Set)other.getFieldValue());
+      } else if (myValue instanceof Map){
+        return TBaseHelper.compareTo((Map)myValue, (Map)other.getFieldValue());
+      } else {
+        return TBaseHelper.compareTo((Comparable)myValue, (Comparable)other.getFieldValue());
+      }
+    }
+    return lastComparison;
+  }
+
+
+  @Override
+  public int hashCode() {
+    HashCodeBuilder hcb = new HashCodeBuilder();
+    hcb.append(this.getClass().getName());
+    TFieldIdEnum setField = getSetField();
+    if (setField != null) {
+      hcb.append(setField.getThriftFieldId());
+      Object value = getFieldValue();
+      if (value instanceof TEnum) {
+        hcb.append(((TEnum)getFieldValue()).getValue());
+      } else {
+        hcb.append(value);
+      }
+    }
+    return hcb.toHashCode();
+  }
+}
diff --git a/src/jvm/backtype/storm/generated/TaskStats.java b/src/jvm/backtype/storm/generated/TaskStats.java
new file mode 100644
index 0000000..b73b467
--- /dev/null
+++ b/src/jvm/backtype/storm/generated/TaskStats.java
@@ -0,0 +1,667 @@
+/**
+ * Autogenerated by Thrift
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ */
+package backtype.storm.generated;
+
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.util.Arrays;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.thrift.*;
+import org.apache.thrift.async.*;
+import org.apache.thrift.meta_data.*;
+import org.apache.thrift.transport.*;
+import org.apache.thrift.protocol.*;
+
+public class TaskStats implements TBase<TaskStats, TaskStats._Fields>, java.io.Serializable, Cloneable {
+  private static final TStruct STRUCT_DESC = new TStruct("TaskStats");
+
+  private static final TField EMITTED_FIELD_DESC = new TField("emitted", TType.MAP, (short)1);
+  private static final TField TRANSFERRED_FIELD_DESC = new TField("transferred", TType.MAP, (short)2);
+  private static final TField SPECIFIC_FIELD_DESC = new TField("specific", TType.STRUCT, (short)3);
+
+  private Map<String,Map<Integer,Long>> emitted;
+  private Map<String,Map<Integer,Long>> transferred;
+  private TaskSpecificStats specific;
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements TFieldIdEnum {
+    EMITTED((short)1, "emitted"),
+    TRANSFERRED((short)2, "transferred"),
+    SPECIFIC((short)3, "specific");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // EMITTED
+          return EMITTED;
+        case 2: // TRANSFERRED
+          return TRANSFERRED;
+        case 3: // SPECIFIC
+          return SPECIFIC;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+
+  public static final Map<_Fields, FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, FieldMetaData> tmpMap = new EnumMap<_Fields, FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.EMITTED, new FieldMetaData("emitted", TFieldRequirementType.REQUIRED, 
+        new MapMetaData(TType.MAP, 
+            new FieldValueMetaData(TType.STRING), 
+            new MapMetaData(TType.MAP, 
+                new FieldValueMetaData(TType.I32), 
+                new FieldValueMetaData(TType.I64)))));
+    tmpMap.put(_Fields.TRANSFERRED, new FieldMetaData("transferred", TFieldRequirementType.REQUIRED, 
+        new MapMetaData(TType.MAP, 
+            new FieldValueMetaData(TType.STRING), 
+            new MapMetaData(TType.MAP, 
+                new FieldValueMetaData(TType.I32), 
+                new FieldValueMetaData(TType.I64)))));
+    tmpMap.put(_Fields.SPECIFIC, new FieldMetaData("specific", TFieldRequirementType.REQUIRED, 
+        new StructMetaData(TType.STRUCT, TaskSpecificStats.class)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    FieldMetaData.addStructMetaDataMap(TaskStats.class, metaDataMap);
+  }
+
+  public TaskStats() {
+  }
+
+  public TaskStats(
+    Map<String,Map<Integer,Long>> emitted,
+    Map<String,Map<Integer,Long>> transferred,
+    TaskSpecificStats specific)
+  {
+    this();
+    this.emitted = emitted;
+    this.transferred = transferred;
+    this.specific = specific;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public TaskStats(TaskStats other) {
+    if (other.is_set_emitted()) {
+      Map<String,Map<Integer,Long>> __this__emitted = new HashMap<String,Map<Integer,Long>>();
+      for (Map.Entry<String, Map<Integer,Long>> other_element : other.emitted.entrySet()) {
+
+        String other_element_key = other_element.getKey();
+        Map<Integer,Long> other_element_value = other_element.getValue();
+
+        String __this__emitted_copy_key = other_element_key;
+
+        Map<Integer,Long> __this__emitted_copy_value = new HashMap<Integer,Long>();
+        for (Map.Entry<Integer, Long> other_element_value_element : other_element_value.entrySet()) {
+
+          Integer other_element_value_element_key = other_element_value_element.getKey();
+          Long other_element_value_element_value = other_element_value_element.getValue();
+
+          Integer __this__emitted_copy_value_copy_key = other_element_value_element_key;
+
+          Long __this__emitted_copy_value_copy_value = other_element_value_element_value;
+
+          __this__emitted_copy_value.put(__this__emitted_copy_value_copy_key, __this__emitted_copy_value_copy_value);
+        }
+
+        __this__emitted.put(__this__emitted_copy_key, __this__emitted_copy_value);
+      }
+      this.emitted = __this__emitted;
+    }
+    if (other.is_set_transferred()) {
+      Map<String,Map<Integer,Long>> __this__transferred = new HashMap<String,Map<Integer,Long>>();
+      for (Map.Entry<String, Map<Integer,Long>> other_element : other.transferred.entrySet()) {
+
+        String other_element_key = other_element.getKey();
+        Map<Integer,Long> other_element_value = other_element.getValue();
+
+        String __this__transferred_copy_key = other_element_key;
+
+        Map<Integer,Long> __this__transferred_copy_value = new HashMap<Integer,Long>();
+        for (Map.Entry<Integer, Long> other_element_value_element : other_element_value.entrySet()) {
+
+          Integer other_element_value_element_key = other_element_value_element.getKey();
+          Long other_element_value_element_value = other_element_value_element.getValue();
+
+          Integer __this__transferred_copy_value_copy_key = other_element_value_element_key;
+
+          Long __this__transferred_copy_value_copy_value = other_element_value_element_value;
+
+          __this__transferred_copy_value.put(__this__transferred_copy_value_copy_key, __this__transferred_copy_value_copy_value);
+        }
+
+        __this__transferred.put(__this__transferred_copy_key, __this__transferred_copy_value);
+      }
+      this.transferred = __this__transferred;
+    }
+    if (other.is_set_specific()) {
+      this.specific = new TaskSpecificStats(other.specific);
+    }
+  }
+
+  public TaskStats deepCopy() {
+    return new TaskStats(this);
+  }
+
+  @Deprecated
+  public TaskStats clone() {
+    return new TaskStats(this);
+  }
+
+  public int get_emitted_size() {
+    return (this.emitted == null) ? 0 : this.emitted.size();
+  }
+
+  public void put_to_emitted(String key, Map<Integer,Long> val) {
+    if (this.emitted == null) {
+      this.emitted = new HashMap<String,Map<Integer,Long>>();
+    }
+    this.emitted.put(key, val);
+  }
+
+  public Map<String,Map<Integer,Long>> get_emitted() {
+    return this.emitted;
+  }
+
+  public void set_emitted(Map<String,Map<Integer,Long>> emitted) {
+    this.emitted = emitted;
+  }
+
+  public void unset_emitted() {
+    this.emitted = null;
+  }
+
+  /** Returns true if field emitted is set (has been asigned a value) and false otherwise */
+  public boolean is_set_emitted() {
+    return this.emitted != null;
+  }
+
+  public void set_emitted_isSet(boolean value) {
+    if (!value) {
+      this.emitted = null;
+    }
+  }
+
+  public int get_transferred_size() {
+    return (this.transferred == null) ? 0 : this.transferred.size();
+  }
+
+  public void put_to_transferred(String key, Map<Integer,Long> val) {
+    if (this.transferred == null) {
+      this.transferred = new HashMap<String,Map<Integer,Long>>();
+    }
+    this.transferred.put(key, val);
+  }
+
+  public Map<String,Map<Integer,Long>> get_transferred() {
+    return this.transferred;
+  }
+
+  public void set_transferred(Map<String,Map<Integer,Long>> transferred) {
+    this.transferred = transferred;
+  }
+
+  public void unset_transferred() {
+    this.transferred = null;
+  }
+
+  /** Returns true if field transferred is set (has been asigned a value) and false otherwise */
+  public boolean is_set_transferred() {
+    return this.transferred != null;
+  }
+
+  public void set_transferred_isSet(boolean value) {
+    if (!value) {
+      this.transferred = null;
+    }
+  }
+
+  public TaskSpecificStats get_specific() {
+    return this.specific;
+  }
+
+  public void set_specific(TaskSpecificStats specific) {
+    this.specific = specific;
+  }
+
+  public void unset_specific() {
+    this.specific = null;
+  }
+
+  /** Returns true if field specific is set (has been asigned a value) and false otherwise */
+  public boolean is_set_specific() {
+    return this.specific != null;
+  }
+
+  public void set_specific_isSet(boolean value) {
+    if (!value) {
+      this.specific = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case EMITTED:
+      if (value == null) {
+        unset_emitted();
+      } else {
+        set_emitted((Map<String,Map<Integer,Long>>)value);
+      }
+      break;
+
+    case TRANSFERRED:
+      if (value == null) {
+        unset_transferred();
+      } else {
+        set_transferred((Map<String,Map<Integer,Long>>)value);
+      }
+      break;
+
+    case SPECIFIC:
+      if (value == null) {
+        unset_specific();
+      } else {
+        set_specific((TaskSpecificStats)value);
+      }
+      break;
+
+    }
+  }
+
+  public void setFieldValue(int fieldID, Object value) {
+    setFieldValue(_Fields.findByThriftIdOrThrow(fieldID), value);
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case EMITTED:
+      return get_emitted();
+
+    case TRANSFERRED:
+      return get_transferred();
+
+    case SPECIFIC:
+      return get_specific();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  public Object getFieldValue(int fieldId) {
+    return getFieldValue(_Fields.findByThriftIdOrThrow(fieldId));
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been asigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    switch (field) {
+    case EMITTED:
+      return is_set_emitted();
+    case TRANSFERRED:
+      return is_set_transferred();
+    case SPECIFIC:
+      return is_set_specific();
+    }
+    throw new IllegalStateException();
+  }
+
+  public boolean isSet(int fieldID) {
+    return isSet(_Fields.findByThriftIdOrThrow(fieldID));
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof TaskStats)
+      return this.equals((TaskStats)that);
+    return false;
+  }
+
+  public boolean equals(TaskStats that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_emitted = true && this.is_set_emitted();
+    boolean that_present_emitted = true && that.is_set_emitted();
+    if (this_present_emitted || that_present_emitted) {
+      if (!(this_present_emitted && that_present_emitted))
+        return false;
+      if (!this.emitted.equals(that.emitted))
+        return false;
+    }
+
+    boolean this_present_transferred = true && this.is_set_transferred();
+    boolean that_present_transferred = true && that.is_set_transferred();
+    if (this_present_transferred || that_present_transferred) {
+      if (!(this_present_transferred && that_present_transferred))
+        return false;
+      if (!this.transferred.equals(that.transferred))
+        return false;
+    }
+
+    boolean this_present_specific = true && this.is_set_specific();
+    boolean that_present_specific = true && that.is_set_specific();
+    if (this_present_specific || that_present_specific) {
+      if (!(this_present_specific && that_present_specific))
+        return false;
+      if (!this.specific.equals(that.specific))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    HashCodeBuilder builder = new HashCodeBuilder();
+
+    boolean present_emitted = true && (is_set_emitted());
+    builder.append(present_emitted);
+    if (present_emitted)
+      builder.append(emitted);
+
+    boolean present_transferred = true && (is_set_transferred());
+    builder.append(present_transferred);
+    if (present_transferred)
+      builder.append(transferred);
+
+    boolean present_specific = true && (is_set_specific());
+    builder.append(present_specific);
+    if (present_specific)
+      builder.append(specific);
+
+    return builder.toHashCode();
+  }
+
+  public int compareTo(TaskStats other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+    TaskStats typedOther = (TaskStats)other;
+
+    lastComparison = Boolean.valueOf(is_set_emitted()).compareTo(typedOther.is_set_emitted());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_emitted()) {      lastComparison = TBaseHelper.compareTo(this.emitted, typedOther.emitted);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_transferred()).compareTo(typedOther.is_set_transferred());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_transferred()) {      lastComparison = TBaseHelper.compareTo(this.transferred, typedOther.transferred);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_specific()).compareTo(typedOther.is_set_specific());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_specific()) {      lastComparison = TBaseHelper.compareTo(this.specific, typedOther.specific);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public void read(TProtocol iprot) throws TException {
+    TField field;
+    iprot.readStructBegin();
+    while (true)
+    {
+      field = iprot.readFieldBegin();
+      if (field.type == TType.STOP) { 
+        break;
+      }
+      switch (field.id) {
+        case 1: // EMITTED
+          if (field.type == TType.MAP) {
+            {
+              TMap _map101 = iprot.readMapBegin();
+              this.emitted = new HashMap<String,Map<Integer,Long>>(2*_map101.size);
+              for (int _i102 = 0; _i102 < _map101.size; ++_i102)
+              {
+                String _key103;
+                Map<Integer,Long> _val104;
+                _key103 = iprot.readString();
+                {
+                  TMap _map105 = iprot.readMapBegin();
+                  _val104 = new HashMap<Integer,Long>(2*_map105.size);
+                  for (int _i106 = 0; _i106 < _map105.size; ++_i106)
+                  {
+                    int _key107;
+                    long _val108;
+                    _key107 = iprot.readI32();
+                    _val108 = iprot.readI64();
+                    _val104.put(_key107, _val108);
+                  }
+                  iprot.readMapEnd();
+                }
+                this.emitted.put(_key103, _val104);
+              }
+              iprot.readMapEnd();
+            }
+          } else { 
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 2: // TRANSFERRED
+          if (field.type == TType.MAP) {
+            {
+              TMap _map109 = iprot.readMapBegin();
+              this.transferred = new HashMap<String,Map<Integer,Long>>(2*_map109.size);
+              for (int _i110 = 0; _i110 < _map109.size; ++_i110)
+              {
+                String _key111;
+                Map<Integer,Long> _val112;
+                _key111 = iprot.readString();
+                {
+                  TMap _map113 = iprot.readMapBegin();
+                  _val112 = new HashMap<Integer,Long>(2*_map113.size);
+                  for (int _i114 = 0; _i114 < _map113.size; ++_i114)
+                  {
+                    int _key115;
+                    long _val116;
+                    _key115 = iprot.readI32();
+                    _val116 = iprot.readI64();
+                    _val112.put(_key115, _val116);
+                  }
+                  iprot.readMapEnd();
+                }
+                this.transferred.put(_key111, _val112);
+              }
+              iprot.readMapEnd();
+            }
+          } else { 
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 3: // SPECIFIC
+          if (field.type == TType.STRUCT) {
+            this.specific = new TaskSpecificStats();
+            this.specific.read(iprot);
+          } else { 
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        default:
+          TProtocolUtil.skip(iprot, field.type);
+      }
+      iprot.readFieldEnd();
+    }
+    iprot.readStructEnd();
+    validate();
+  }
+
+  public void write(TProtocol oprot) throws TException {
+    validate();
+
+    oprot.writeStructBegin(STRUCT_DESC);
+    if (this.emitted != null) {
+      oprot.writeFieldBegin(EMITTED_FIELD_DESC);
+      {
+        oprot.writeMapBegin(new TMap(TType.STRING, TType.MAP, this.emitted.size()));
+        for (Map.Entry<String, Map<Integer,Long>> _iter117 : this.emitted.entrySet())
+        {
+          oprot.writeString(_iter117.getKey());
+          {
+            oprot.writeMapBegin(new TMap(TType.I32, TType.I64, _iter117.getValue().size()));
+            for (Map.Entry<Integer, Long> _iter118 : _iter117.getValue().entrySet())
+            {
+              oprot.writeI32(_iter118.getKey());
+              oprot.writeI64(_iter118.getValue());
+            }
+            oprot.writeMapEnd();
+          }
+        }
+        oprot.writeMapEnd();
+      }
+      oprot.writeFieldEnd();
+    }
+    if (this.transferred != null) {
+      oprot.writeFieldBegin(TRANSFERRED_FIELD_DESC);
+      {
+        oprot.writeMapBegin(new TMap(TType.STRING, TType.MAP, this.transferred.size()));
+        for (Map.Entry<String, Map<Integer,Long>> _iter119 : this.transferred.entrySet())
+        {
+          oprot.writeString(_iter119.getKey());
+          {
+            oprot.writeMapBegin(new TMap(TType.I32, TType.I64, _iter119.getValue().size()));
+            for (Map.Entry<Integer, Long> _iter120 : _iter119.getValue().entrySet())
+            {
+              oprot.writeI32(_iter120.getKey());
+              oprot.writeI64(_iter120.getValue());
+            }
+            oprot.writeMapEnd();
+          }
+        }
+        oprot.writeMapEnd();
+      }
+      oprot.writeFieldEnd();
+    }
+    if (this.specific != null) {
+      oprot.writeFieldBegin(SPECIFIC_FIELD_DESC);
+      this.specific.write(oprot);
+      oprot.writeFieldEnd();
+    }
+    oprot.writeFieldStop();
+    oprot.writeStructEnd();
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("TaskStats(");
+    boolean first = true;
+
+    sb.append("emitted:");
+    if (this.emitted == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.emitted);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("transferred:");
+    if (this.transferred == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.transferred);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("specific:");
+    if (this.specific == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.specific);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws TException {
+    // check for required fields
+    if (!is_set_emitted()) {
+      throw new TProtocolException("Required field 'emitted' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_transferred()) {
+      throw new TProtocolException("Required field 'transferred' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_specific()) {
+      throw new TProtocolException("Required field 'specific' is unset! Struct:" + toString());
+    }
+
+  }
+
+}
+
diff --git a/src/jvm/backtype/storm/generated/TaskSummary.java b/src/jvm/backtype/storm/generated/TaskSummary.java
new file mode 100644
index 0000000..ff66b12
--- /dev/null
+++ b/src/jvm/backtype/storm/generated/TaskSummary.java
@@ -0,0 +1,906 @@
+/**
+ * Autogenerated by Thrift
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ */
+package backtype.storm.generated;
+
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.util.Arrays;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.thrift.*;
+import org.apache.thrift.async.*;
+import org.apache.thrift.meta_data.*;
+import org.apache.thrift.transport.*;
+import org.apache.thrift.protocol.*;
+
+public class TaskSummary implements TBase<TaskSummary, TaskSummary._Fields>, java.io.Serializable, Cloneable {
+  private static final TStruct STRUCT_DESC = new TStruct("TaskSummary");
+
+  private static final TField TASK_ID_FIELD_DESC = new TField("task_id", TType.I32, (short)1);
+  private static final TField COMPONENT_ID_FIELD_DESC = new TField("component_id", TType.I32, (short)2);
+  private static final TField HOST_FIELD_DESC = new TField("host", TType.STRING, (short)3);
+  private static final TField PORT_FIELD_DESC = new TField("port", TType.I32, (short)4);
+  private static final TField UPTIME_SECS_FIELD_DESC = new TField("uptime_secs", TType.I32, (short)5);
+  private static final TField ERRORS_FIELD_DESC = new TField("errors", TType.LIST, (short)6);
+  private static final TField STATS_FIELD_DESC = new TField("stats", TType.STRUCT, (short)7);
+
+  private int task_id;
+  private int component_id;
+  private String host;
+  private int port;
+  private int uptime_secs;
+  private List<ErrorInfo> errors;
+  private TaskStats stats;
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements TFieldIdEnum {
+    TASK_ID((short)1, "task_id"),
+    COMPONENT_ID((short)2, "component_id"),
+    HOST((short)3, "host"),
+    PORT((short)4, "port"),
+    UPTIME_SECS((short)5, "uptime_secs"),
+    ERRORS((short)6, "errors"),
+    STATS((short)7, "stats");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // TASK_ID
+          return TASK_ID;
+        case 2: // COMPONENT_ID
+          return COMPONENT_ID;
+        case 3: // HOST
+          return HOST;
+        case 4: // PORT
+          return PORT;
+        case 5: // UPTIME_SECS
+          return UPTIME_SECS;
+        case 6: // ERRORS
+          return ERRORS;
+        case 7: // STATS
+          return STATS;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __TASK_ID_ISSET_ID = 0;
+  private static final int __COMPONENT_ID_ISSET_ID = 1;
+  private static final int __PORT_ISSET_ID = 2;
+  private static final int __UPTIME_SECS_ISSET_ID = 3;
+  private BitSet __isset_bit_vector = new BitSet(4);
+
+  public static final Map<_Fields, FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, FieldMetaData> tmpMap = new EnumMap<_Fields, FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.TASK_ID, new FieldMetaData("task_id", TFieldRequirementType.REQUIRED, 
+        new FieldValueMetaData(TType.I32)));
+    tmpMap.put(_Fields.COMPONENT_ID, new FieldMetaData("component_id", TFieldRequirementType.REQUIRED, 
+        new FieldValueMetaData(TType.I32)));
+    tmpMap.put(_Fields.HOST, new FieldMetaData("host", TFieldRequirementType.REQUIRED, 
+        new FieldValueMetaData(TType.STRING)));
+    tmpMap.put(_Fields.PORT, new FieldMetaData("port", TFieldRequirementType.REQUIRED, 
+        new FieldValueMetaData(TType.I32)));
+    tmpMap.put(_Fields.UPTIME_SECS, new FieldMetaData("uptime_secs", TFieldRequirementType.REQUIRED, 
+        new FieldValueMetaData(TType.I32)));
+    tmpMap.put(_Fields.ERRORS, new FieldMetaData("errors", TFieldRequirementType.REQUIRED, 
+        new ListMetaData(TType.LIST, 
+            new StructMetaData(TType.STRUCT, ErrorInfo.class))));
+    tmpMap.put(_Fields.STATS, new FieldMetaData("stats", TFieldRequirementType.OPTIONAL, 
+        new StructMetaData(TType.STRUCT, TaskStats.class)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    FieldMetaData.addStructMetaDataMap(TaskSummary.class, metaDataMap);
+  }
+
+  public TaskSummary() {
+  }
+
+  public TaskSummary(
+    int task_id,
+    int component_id,
+    String host,
+    int port,
+    int uptime_secs,
+    List<ErrorInfo> errors)
+  {
+    this();
+    this.task_id = task_id;
+    set_task_id_isSet(true);
+    this.component_id = component_id;
+    set_component_id_isSet(true);
+    this.host = host;
+    this.port = port;
+    set_port_isSet(true);
+    this.uptime_secs = uptime_secs;
+    set_uptime_secs_isSet(true);
+    this.errors = errors;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public TaskSummary(TaskSummary other) {
+    __isset_bit_vector.clear();
+    __isset_bit_vector.or(other.__isset_bit_vector);
+    this.task_id = other.task_id;
+    this.component_id = other.component_id;
+    if (other.is_set_host()) {
+      this.host = other.host;
+    }
+    this.port = other.port;
+    this.uptime_secs = other.uptime_secs;
+    if (other.is_set_errors()) {
+      List<ErrorInfo> __this__errors = new ArrayList<ErrorInfo>();
+      for (ErrorInfo other_element : other.errors) {
+        __this__errors.add(new ErrorInfo(other_element));
+      }
+      this.errors = __this__errors;
+    }
+    if (other.is_set_stats()) {
+      this.stats = new TaskStats(other.stats);
+    }
+  }
+
+  public TaskSummary deepCopy() {
+    return new TaskSummary(this);
+  }
+
+  @Deprecated
+  public TaskSummary clone() {
+    return new TaskSummary(this);
+  }
+
+  public int get_task_id() {
+    return this.task_id;
+  }
+
+  public void set_task_id(int task_id) {
+    this.task_id = task_id;
+    set_task_id_isSet(true);
+  }
+
+  public void unset_task_id() {
+    __isset_bit_vector.clear(__TASK_ID_ISSET_ID);
+  }
+
+  /** Returns true if field task_id is set (has been asigned a value) and false otherwise */
+  public boolean is_set_task_id() {
+    return __isset_bit_vector.get(__TASK_ID_ISSET_ID);
+  }
+
+  public void set_task_id_isSet(boolean value) {
+    __isset_bit_vector.set(__TASK_ID_ISSET_ID, value);
+  }
+
+  public int get_component_id() {
+    return this.component_id;
+  }
+
+  public void set_component_id(int component_id) {
+    this.component_id = component_id;
+    set_component_id_isSet(true);
+  }
+
+  public void unset_component_id() {
+    __isset_bit_vector.clear(__COMPONENT_ID_ISSET_ID);
+  }
+
+  /** Returns true if field component_id is set (has been asigned a value) and false otherwise */
+  public boolean is_set_component_id() {
+    return __isset_bit_vector.get(__COMPONENT_ID_ISSET_ID);
+  }
+
+  public void set_component_id_isSet(boolean value) {
+    __isset_bit_vector.set(__COMPONENT_ID_ISSET_ID, value);
+  }
+
+  public String get_host() {
+    return this.host;
+  }
+
+  public void set_host(String host) {
+    this.host = host;
+  }
+
+  public void unset_host() {
+    this.host = null;
+  }
+
+  /** Returns true if field host is set (has been asigned a value) and false otherwise */
+  public boolean is_set_host() {
+    return this.host != null;
+  }
+
+  public void set_host_isSet(boolean value) {
+    if (!value) {
+      this.host = null;
+    }
+  }
+
+  public int get_port() {
+    return this.port;
+  }
+
+  public void set_port(int port) {
+    this.port = port;
+    set_port_isSet(true);
+  }
+
+  public void unset_port() {
+    __isset_bit_vector.clear(__PORT_ISSET_ID);
+  }
+
+  /** Returns true if field port is set (has been asigned a value) and false otherwise */
+  public boolean is_set_port() {
+    return __isset_bit_vector.get(__PORT_ISSET_ID);
+  }
+
+  public void set_port_isSet(boolean value) {
+    __isset_bit_vector.set(__PORT_ISSET_ID, value);
+  }
+
+  public int get_uptime_secs() {
+    return this.uptime_secs;
+  }
+
+  public void set_uptime_secs(int uptime_secs) {
+    this.uptime_secs = uptime_secs;
+    set_uptime_secs_isSet(true);
+  }
+
+  public void unset_uptime_secs() {
+    __isset_bit_vector.clear(__UPTIME_SECS_ISSET_ID);
+  }
+
+  /** Returns true if field uptime_secs is set (has been asigned a value) and false otherwise */
+  public boolean is_set_uptime_secs() {
+    return __isset_bit_vector.get(__UPTIME_SECS_ISSET_ID);
+  }
+
+  public void set_uptime_secs_isSet(boolean value) {
+    __isset_bit_vector.set(__UPTIME_SECS_ISSET_ID, value);
+  }
+
+  public int get_errors_size() {
+    return (this.errors == null) ? 0 : this.errors.size();
+  }
+
+  public java.util.Iterator<ErrorInfo> get_errors_iterator() {
+    return (this.errors == null) ? null : this.errors.iterator();
+  }
+
+  public void add_to_errors(ErrorInfo elem) {
+    if (this.errors == null) {
+      this.errors = new ArrayList<ErrorInfo>();
+    }
+    this.errors.add(elem);
+  }
+
+  public List<ErrorInfo> get_errors() {
+    return this.errors;
+  }
+
+  public void set_errors(List<ErrorInfo> errors) {
+    this.errors = errors;
+  }
+
+  public void unset_errors() {
+    this.errors = null;
+  }
+
+  /** Returns true if field errors is set (has been asigned a value) and false otherwise */
+  public boolean is_set_errors() {
+    return this.errors != null;
+  }
+
+  public void set_errors_isSet(boolean value) {
+    if (!value) {
+      this.errors = null;
+    }
+  }
+
+  public TaskStats get_stats() {
+    return this.stats;
+  }
+
+  public void set_stats(TaskStats stats) {
+    this.stats = stats;
+  }
+
+  public void unset_stats() {
+    this.stats = null;
+  }
+
+  /** Returns true if field stats is set (has been asigned a value) and false otherwise */
+  public boolean is_set_stats() {
+    return this.stats != null;
+  }
+
+  public void set_stats_isSet(boolean value) {
+    if (!value) {
+      this.stats = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case TASK_ID:
+      if (value == null) {
+        unset_task_id();
+      } else {
+        set_task_id((Integer)value);
+      }
+      break;
+
+    case COMPONENT_ID:
+      if (value == null) {
+        unset_component_id();
+      } else {
+        set_component_id((Integer)value);
+      }
+      break;
+
+    case HOST:
+      if (value == null) {
+        unset_host();
+      } else {
+        set_host((String)value);
+      }
+      break;
+
+    case PORT:
+      if (value == null) {
+        unset_port();
+      } else {
+        set_port((Integer)value);
+      }
+      break;
+
+    case UPTIME_SECS:
+      if (value == null) {
+        unset_uptime_secs();
+      } else {
+        set_uptime_secs((Integer)value);
+      }
+      break;
+
+    case ERRORS:
+      if (value == null) {
+        unset_errors();
+      } else {
+        set_errors((List<ErrorInfo>)value);
+      }
+      break;
+
+    case STATS:
+      if (value == null) {
+        unset_stats();
+      } else {
+        set_stats((TaskStats)value);
+      }
+      break;
+
+    }
+  }
+
+  public void setFieldValue(int fieldID, Object value) {
+    setFieldValue(_Fields.findByThriftIdOrThrow(fieldID), value);
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case TASK_ID:
+      return new Integer(get_task_id());
+
+    case COMPONENT_ID:
+      return new Integer(get_component_id());
+
+    case HOST:
+      return get_host();
+
+    case PORT:
+      return new Integer(get_port());
+
+    case UPTIME_SECS:
+      return new Integer(get_uptime_secs());
+
+    case ERRORS:
+      return get_errors();
+
+    case STATS:
+      return get_stats();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  public Object getFieldValue(int fieldId) {
+    return getFieldValue(_Fields.findByThriftIdOrThrow(fieldId));
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been asigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    switch (field) {
+    case TASK_ID:
+      return is_set_task_id();
+    case COMPONENT_ID:
+      return is_set_component_id();
+    case HOST:
+      return is_set_host();
+    case PORT:
+      return is_set_port();
+    case UPTIME_SECS:
+      return is_set_uptime_secs();
+    case ERRORS:
+      return is_set_errors();
+    case STATS:
+      return is_set_stats();
+    }
+    throw new IllegalStateException();
+  }
+
+  public boolean isSet(int fieldID) {
+    return isSet(_Fields.findByThriftIdOrThrow(fieldID));
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof TaskSummary)
+      return this.equals((TaskSummary)that);
+    return false;
+  }
+
+  public boolean equals(TaskSummary that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_task_id = true;
+    boolean that_present_task_id = true;
+    if (this_present_task_id || that_present_task_id) {
+      if (!(this_present_task_id && that_present_task_id))
+        return false;
+      if (this.task_id != that.task_id)
+        return false;
+    }
+
+    boolean this_present_component_id = true;
+    boolean that_present_component_id = true;
+    if (this_present_component_id || that_present_component_id) {
+      if (!(this_present_component_id && that_present_component_id))
+        return false;
+      if (this.component_id != that.component_id)
+        return false;
+    }
+
+    boolean this_present_host = true && this.is_set_host();
+    boolean that_present_host = true && that.is_set_host();
+    if (this_present_host || that_present_host) {
+      if (!(this_present_host && that_present_host))
+        return false;
+      if (!this.host.equals(that.host))
+        return false;
+    }
+
+    boolean this_present_port = true;
+    boolean that_present_port = true;
+    if (this_present_port || that_present_port) {
+      if (!(this_present_port && that_present_port))
+        return false;
+      if (this.port != that.port)
+        return false;
+    }
+
+    boolean this_present_uptime_secs = true;
+    boolean that_present_uptime_secs = true;
+    if (this_present_uptime_secs || that_present_uptime_secs) {
+      if (!(this_present_uptime_secs && that_present_uptime_secs))
+        return false;
+      if (this.uptime_secs != that.uptime_secs)
+        return false;
+    }
+
+    boolean this_present_errors = true && this.is_set_errors();
+    boolean that_present_errors = true && that.is_set_errors();
+    if (this_present_errors || that_present_errors) {
+      if (!(this_present_errors && that_present_errors))
+        return false;
+      if (!this.errors.equals(that.errors))
+        return false;
+    }
+
+    boolean this_present_stats = true && this.is_set_stats();
+    boolean that_present_stats = true && that.is_set_stats();
+    if (this_present_stats || that_present_stats) {
+      if (!(this_present_stats && that_present_stats))
+        return false;
+      if (!this.stats.equals(that.stats))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    HashCodeBuilder builder = new HashCodeBuilder();
+
+    boolean present_task_id = true;
+    builder.append(present_task_id);
+    if (present_task_id)
+      builder.append(task_id);
+
+    boolean present_component_id = true;
+    builder.append(present_component_id);
+    if (present_component_id)
+      builder.append(component_id);
+
+    boolean present_host = true && (is_set_host());
+    builder.append(present_host);
+    if (present_host)
+      builder.append(host);
+
+    boolean present_port = true;
+    builder.append(present_port);
+    if (present_port)
+      builder.append(port);
+
+    boolean present_uptime_secs = true;
+    builder.append(present_uptime_secs);
+    if (present_uptime_secs)
+      builder.append(uptime_secs);
+
+    boolean present_errors = true && (is_set_errors());
+    builder.append(present_errors);
+    if (present_errors)
+      builder.append(errors);
+
+    boolean present_stats = true && (is_set_stats());
+    builder.append(present_stats);
+    if (present_stats)
+      builder.append(stats);
+
+    return builder.toHashCode();
+  }
+
+  public int compareTo(TaskSummary other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+    TaskSummary typedOther = (TaskSummary)other;
+
+    lastComparison = Boolean.valueOf(is_set_task_id()).compareTo(typedOther.is_set_task_id());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_task_id()) {      lastComparison = TBaseHelper.compareTo(this.task_id, typedOther.task_id);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_component_id()).compareTo(typedOther.is_set_component_id());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_component_id()) {      lastComparison = TBaseHelper.compareTo(this.component_id, typedOther.component_id);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_host()).compareTo(typedOther.is_set_host());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_host()) {      lastComparison = TBaseHelper.compareTo(this.host, typedOther.host);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_port()).compareTo(typedOther.is_set_port());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_port()) {      lastComparison = TBaseHelper.compareTo(this.port, typedOther.port);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_uptime_secs()).compareTo(typedOther.is_set_uptime_secs());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_uptime_secs()) {      lastComparison = TBaseHelper.compareTo(this.uptime_secs, typedOther.uptime_secs);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_errors()).compareTo(typedOther.is_set_errors());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_errors()) {      lastComparison = TBaseHelper.compareTo(this.errors, typedOther.errors);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_stats()).compareTo(typedOther.is_set_stats());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_stats()) {      lastComparison = TBaseHelper.compareTo(this.stats, typedOther.stats);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public void read(TProtocol iprot) throws TException {
+    TField field;
+    iprot.readStructBegin();
+    while (true)
+    {
+      field = iprot.readFieldBegin();
+      if (field.type == TType.STOP) { 
+        break;
+      }
+      switch (field.id) {
+        case 1: // TASK_ID
+          if (field.type == TType.I32) {
+            this.task_id = iprot.readI32();
+            set_task_id_isSet(true);
+          } else { 
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 2: // COMPONENT_ID
+          if (field.type == TType.I32) {
+            this.component_id = iprot.readI32();
+            set_component_id_isSet(true);
+          } else { 
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 3: // HOST
+          if (field.type == TType.STRING) {
+            this.host = iprot.readString();
+          } else { 
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 4: // PORT
+          if (field.type == TType.I32) {
+            this.port = iprot.readI32();
+            set_port_isSet(true);
+          } else { 
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 5: // UPTIME_SECS
+          if (field.type == TType.I32) {
+            this.uptime_secs = iprot.readI32();
+            set_uptime_secs_isSet(true);
+          } else { 
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 6: // ERRORS
+          if (field.type == TType.LIST) {
+            {
+              TList _list121 = iprot.readListBegin();
+              this.errors = new ArrayList<ErrorInfo>(_list121.size);
+              for (int _i122 = 0; _i122 < _list121.size; ++_i122)
+              {
+                ErrorInfo _elem123;
+                _elem123 = new ErrorInfo();
+                _elem123.read(iprot);
+                this.errors.add(_elem123);
+              }
+              iprot.readListEnd();
+            }
+          } else { 
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 7: // STATS
+          if (field.type == TType.STRUCT) {
+            this.stats = new TaskStats();
+            this.stats.read(iprot);
+          } else { 
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        default:
+          TProtocolUtil.skip(iprot, field.type);
+      }
+      iprot.readFieldEnd();
+    }
+    iprot.readStructEnd();
+    validate();
+  }
+
+  public void write(TProtocol oprot) throws TException {
+    validate();
+
+    oprot.writeStructBegin(STRUCT_DESC);
+    oprot.writeFieldBegin(TASK_ID_FIELD_DESC);
+    oprot.writeI32(this.task_id);
+    oprot.writeFieldEnd();
+    oprot.writeFieldBegin(COMPONENT_ID_FIELD_DESC);
+    oprot.writeI32(this.component_id);
+    oprot.writeFieldEnd();
+    if (this.host != null) {
+      oprot.writeFieldBegin(HOST_FIELD_DESC);
+      oprot.writeString(this.host);
+      oprot.writeFieldEnd();
+    }
+    oprot.writeFieldBegin(PORT_FIELD_DESC);
+    oprot.writeI32(this.port);
+    oprot.writeFieldEnd();
+    oprot.writeFieldBegin(UPTIME_SECS_FIELD_DESC);
+    oprot.writeI32(this.uptime_secs);
+    oprot.writeFieldEnd();
+    if (this.errors != null) {
+      oprot.writeFieldBegin(ERRORS_FIELD_DESC);
+      {
+        oprot.writeListBegin(new TList(TType.STRUCT, this.errors.size()));
+        for (ErrorInfo _iter124 : this.errors)
+        {
+          _iter124.write(oprot);
+        }
+        oprot.writeListEnd();
+      }
+      oprot.writeFieldEnd();
+    }
+    if (this.stats != null) {
+      if (is_set_stats()) {
+        oprot.writeFieldBegin(STATS_FIELD_DESC);
+        this.stats.write(oprot);
+        oprot.writeFieldEnd();
+      }
+    }
+    oprot.writeFieldStop();
+    oprot.writeStructEnd();
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("TaskSummary(");
+    boolean first = true;
+
+    sb.append("task_id:");
+    sb.append(this.task_id);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("component_id:");
+    sb.append(this.component_id);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("host:");
+    if (this.host == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.host);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("port:");
+    sb.append(this.port);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("uptime_secs:");
+    sb.append(this.uptime_secs);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("errors:");
+    if (this.errors == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.errors);
+    }
+    first = false;
+    if (is_set_stats()) {
+      if (!first) sb.append(", ");
+      sb.append("stats:");
+      if (this.stats == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.stats);
+      }
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws TException {
+    // check for required fields
+    if (!is_set_task_id()) {
+      throw new TProtocolException("Required field 'task_id' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_component_id()) {
+      throw new TProtocolException("Required field 'component_id' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_host()) {
+      throw new TProtocolException("Required field 'host' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_port()) {
+      throw new TProtocolException("Required field 'port' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_uptime_secs()) {
+      throw new TProtocolException("Required field 'uptime_secs' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_errors()) {
+      throw new TProtocolException("Required field 'errors' is unset! Struct:" + toString());
+    }
+
+  }
+
+}
+
diff --git a/src/jvm/backtype/storm/generated/TopologyInfo.java b/src/jvm/backtype/storm/generated/TopologyInfo.java
new file mode 100644
index 0000000..b1ea881
--- /dev/null
+++ b/src/jvm/backtype/storm/generated/TopologyInfo.java
@@ -0,0 +1,640 @@
+/**
+ * Autogenerated by Thrift
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ */
+package backtype.storm.generated;
+
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.util.Arrays;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.thrift.*;
+import org.apache.thrift.async.*;
+import org.apache.thrift.meta_data.*;
+import org.apache.thrift.transport.*;
+import org.apache.thrift.protocol.*;
+
+public class TopologyInfo implements TBase<TopologyInfo, TopologyInfo._Fields>, java.io.Serializable, Cloneable {
+  private static final TStruct STRUCT_DESC = new TStruct("TopologyInfo");
+
+  private static final TField ID_FIELD_DESC = new TField("id", TType.STRING, (short)1);
+  private static final TField NAME_FIELD_DESC = new TField("name", TType.STRING, (short)2);
+  private static final TField UPTIME_SECS_FIELD_DESC = new TField("uptime_secs", TType.I32, (short)3);
+  private static final TField TASKS_FIELD_DESC = new TField("tasks", TType.LIST, (short)4);
+
+  private String id;
+  private String name;
+  private int uptime_secs;
+  private List<TaskSummary> tasks;
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements TFieldIdEnum {
+    ID((short)1, "id"),
+    NAME((short)2, "name"),
+    UPTIME_SECS((short)3, "uptime_secs"),
+    TASKS((short)4, "tasks");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // ID
+          return ID;
+        case 2: // NAME
+          return NAME;
+        case 3: // UPTIME_SECS
+          return UPTIME_SECS;
+        case 4: // TASKS
+          return TASKS;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __UPTIME_SECS_ISSET_ID = 0;
+  private BitSet __isset_bit_vector = new BitSet(1);
+
+  public static final Map<_Fields, FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, FieldMetaData> tmpMap = new EnumMap<_Fields, FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.ID, new FieldMetaData("id", TFieldRequirementType.REQUIRED, 
+        new FieldValueMetaData(TType.STRING)));
+    tmpMap.put(_Fields.NAME, new FieldMetaData("name", TFieldRequirementType.REQUIRED, 
+        new FieldValueMetaData(TType.STRING)));
+    tmpMap.put(_Fields.UPTIME_SECS, new FieldMetaData("uptime_secs", TFieldRequirementType.REQUIRED, 
+        new FieldValueMetaData(TType.I32)));
+    tmpMap.put(_Fields.TASKS, new FieldMetaData("tasks", TFieldRequirementType.REQUIRED, 
+        new ListMetaData(TType.LIST, 
+            new StructMetaData(TType.STRUCT, TaskSummary.class))));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    FieldMetaData.addStructMetaDataMap(TopologyInfo.class, metaDataMap);
+  }
+
+  public TopologyInfo() {
+  }
+
+  public TopologyInfo(
+    String id,
+    String name,
+    int uptime_secs,
+    List<TaskSummary> tasks)
+  {
+    this();
+    this.id = id;
+    this.name = name;
+    this.uptime_secs = uptime_secs;
+    set_uptime_secs_isSet(true);
+    this.tasks = tasks;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public TopologyInfo(TopologyInfo other) {
+    __isset_bit_vector.clear();
+    __isset_bit_vector.or(other.__isset_bit_vector);
+    if (other.is_set_id()) {
+      this.id = other.id;
+    }
+    if (other.is_set_name()) {
+      this.name = other.name;
+    }
+    this.uptime_secs = other.uptime_secs;
+    if (other.is_set_tasks()) {
+      List<TaskSummary> __this__tasks = new ArrayList<TaskSummary>();
+      for (TaskSummary other_element : other.tasks) {
+        __this__tasks.add(new TaskSummary(other_element));
+      }
+      this.tasks = __this__tasks;
+    }
+  }
+
+  public TopologyInfo deepCopy() {
+    return new TopologyInfo(this);
+  }
+
+  @Deprecated
+  public TopologyInfo clone() {
+    return new TopologyInfo(this);
+  }
+
+  public String get_id() {
+    return this.id;
+  }
+
+  public void set_id(String id) {
+    this.id = id;
+  }
+
+  public void unset_id() {
+    this.id = null;
+  }
+
+  /** Returns true if field id is set (has been asigned a value) and false otherwise */
+  public boolean is_set_id() {
+    return this.id != null;
+  }
+
+  public void set_id_isSet(boolean value) {
+    if (!value) {
+      this.id = null;
+    }
+  }
+
+  public String get_name() {
+    return this.name;
+  }
+
+  public void set_name(String name) {
+    this.name = name;
+  }
+
+  public void unset_name() {
+    this.name = null;
+  }
+
+  /** Returns true if field name is set (has been asigned a value) and false otherwise */
+  public boolean is_set_name() {
+    return this.name != null;
+  }
+
+  public void set_name_isSet(boolean value) {
+    if (!value) {
+      this.name = null;
+    }
+  }
+
+  public int get_uptime_secs() {
+    return this.uptime_secs;
+  }
+
+  public void set_uptime_secs(int uptime_secs) {
+    this.uptime_secs = uptime_secs;
+    set_uptime_secs_isSet(true);
+  }
+
+  public void unset_uptime_secs() {
+    __isset_bit_vector.clear(__UPTIME_SECS_ISSET_ID);
+  }
+
+  /** Returns true if field uptime_secs is set (has been asigned a value) and false otherwise */
+  public boolean is_set_uptime_secs() {
+    return __isset_bit_vector.get(__UPTIME_SECS_ISSET_ID);
+  }
+
+  public void set_uptime_secs_isSet(boolean value) {
+    __isset_bit_vector.set(__UPTIME_SECS_ISSET_ID, value);
+  }
+
+  public int get_tasks_size() {
+    return (this.tasks == null) ? 0 : this.tasks.size();
+  }
+
+  public java.util.Iterator<TaskSummary> get_tasks_iterator() {
+    return (this.tasks == null) ? null : this.tasks.iterator();
+  }
+
+  public void add_to_tasks(TaskSummary elem) {
+    if (this.tasks == null) {
+      this.tasks = new ArrayList<TaskSummary>();
+    }
+    this.tasks.add(elem);
+  }
+
+  public List<TaskSummary> get_tasks() {
+    return this.tasks;
+  }
+
+  public void set_tasks(List<TaskSummary> tasks) {
+    this.tasks = tasks;
+  }
+
+  public void unset_tasks() {
+    this.tasks = null;
+  }
+
+  /** Returns true if field tasks is set (has been asigned a value) and false otherwise */
+  public boolean is_set_tasks() {
+    return this.tasks != null;
+  }
+
+  public void set_tasks_isSet(boolean value) {
+    if (!value) {
+      this.tasks = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case ID:
+      if (value == null) {
+        unset_id();
+      } else {
+        set_id((String)value);
+      }
+      break;
+
+    case NAME:
+      if (value == null) {
+        unset_name();
+      } else {
+        set_name((String)value);
+      }
+      break;
+
+    case UPTIME_SECS:
+      if (value == null) {
+        unset_uptime_secs();
+      } else {
+        set_uptime_secs((Integer)value);
+      }
+      break;
+
+    case TASKS:
+      if (value == null) {
+        unset_tasks();
+      } else {
+        set_tasks((List<TaskSummary>)value);
+      }
+      break;
+
+    }
+  }
+
+  public void setFieldValue(int fieldID, Object value) {
+    setFieldValue(_Fields.findByThriftIdOrThrow(fieldID), value);
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case ID:
+      return get_id();
+
+    case NAME:
+      return get_name();
+
+    case UPTIME_SECS:
+      return new Integer(get_uptime_secs());
+
+    case TASKS:
+      return get_tasks();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  public Object getFieldValue(int fieldId) {
+    return getFieldValue(_Fields.findByThriftIdOrThrow(fieldId));
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been asigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    switch (field) {
+    case ID:
+      return is_set_id();
+    case NAME:
+      return is_set_name();
+    case UPTIME_SECS:
+      return is_set_uptime_secs();
+    case TASKS:
+      return is_set_tasks();
+    }
+    throw new IllegalStateException();
+  }
+
+  public boolean isSet(int fieldID) {
+    return isSet(_Fields.findByThriftIdOrThrow(fieldID));
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof TopologyInfo)
+      return this.equals((TopologyInfo)that);
+    return false;
+  }
+
+  public boolean equals(TopologyInfo that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_id = true && this.is_set_id();
+    boolean that_present_id = true && that.is_set_id();
+    if (this_present_id || that_present_id) {
+      if (!(this_present_id && that_present_id))
+        return false;
+      if (!this.id.equals(that.id))
+        return false;
+    }
+
+    boolean this_present_name = true && this.is_set_name();
+    boolean that_present_name = true && that.is_set_name();
+    if (this_present_name || that_present_name) {
+      if (!(this_present_name && that_present_name))
+        return false;
+      if (!this.name.equals(that.name))
+        return false;
+    }
+
+    boolean this_present_uptime_secs = true;
+    boolean that_present_uptime_secs = true;
+    if (this_present_uptime_secs || that_present_uptime_secs) {
+      if (!(this_present_uptime_secs && that_present_uptime_secs))
+        return false;
+      if (this.uptime_secs != that.uptime_secs)
+        return false;
+    }
+
+    boolean this_present_tasks = true && this.is_set_tasks();
+    boolean that_present_tasks = true && that.is_set_tasks();
+    if (this_present_tasks || that_present_tasks) {
+      if (!(this_present_tasks && that_present_tasks))
+        return false;
+      if (!this.tasks.equals(that.tasks))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    HashCodeBuilder builder = new HashCodeBuilder();
+
+    boolean present_id = true && (is_set_id());
+    builder.append(present_id);
+    if (present_id)
+      builder.append(id);
+
+    boolean present_name = true && (is_set_name());
+    builder.append(present_name);
+    if (present_name)
+      builder.append(name);
+
+    boolean present_uptime_secs = true;
+    builder.append(present_uptime_secs);
+    if (present_uptime_secs)
+      builder.append(uptime_secs);
+
+    boolean present_tasks = true && (is_set_tasks());
+    builder.append(present_tasks);
+    if (present_tasks)
+      builder.append(tasks);
+
+    return builder.toHashCode();
+  }
+
+  public int compareTo(TopologyInfo other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+    TopologyInfo typedOther = (TopologyInfo)other;
+
+    lastComparison = Boolean.valueOf(is_set_id()).compareTo(typedOther.is_set_id());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_id()) {      lastComparison = TBaseHelper.compareTo(this.id, typedOther.id);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_name()).compareTo(typedOther.is_set_name());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_name()) {      lastComparison = TBaseHelper.compareTo(this.name, typedOther.name);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_uptime_secs()).compareTo(typedOther.is_set_uptime_secs());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_uptime_secs()) {      lastComparison = TBaseHelper.compareTo(this.uptime_secs, typedOther.uptime_secs);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_tasks()).compareTo(typedOther.is_set_tasks());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_tasks()) {      lastComparison = TBaseHelper.compareTo(this.tasks, typedOther.tasks);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public void read(TProtocol iprot) throws TException {
+    TField field;
+    iprot.readStructBegin();
+    while (true)
+    {
+      field = iprot.readFieldBegin();
+      if (field.type == TType.STOP) { 
+        break;
+      }
+      switch (field.id) {
+        case 1: // ID
+          if (field.type == TType.STRING) {
+            this.id = iprot.readString();
+          } else { 
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 2: // NAME
+          if (field.type == TType.STRING) {
+            this.name = iprot.readString();
+          } else { 
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 3: // UPTIME_SECS
+          if (field.type == TType.I32) {
+            this.uptime_secs = iprot.readI32();
+            set_uptime_secs_isSet(true);
+          } else { 
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 4: // TASKS
+          if (field.type == TType.LIST) {
+            {
+              TList _list125 = iprot.readListBegin();
+              this.tasks = new ArrayList<TaskSummary>(_list125.size);
+              for (int _i126 = 0; _i126 < _list125.size; ++_i126)
+              {
+                TaskSummary _elem127;
+                _elem127 = new TaskSummary();
+                _elem127.read(iprot);
+                this.tasks.add(_elem127);
+              }
+              iprot.readListEnd();
+            }
+          } else { 
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        default:
+          TProtocolUtil.skip(iprot, field.type);
+      }
+      iprot.readFieldEnd();
+    }
+    iprot.readStructEnd();
+    validate();
+  }
+
+  public void write(TProtocol oprot) throws TException {
+    validate();
+
+    oprot.writeStructBegin(STRUCT_DESC);
+    if (this.id != null) {
+      oprot.writeFieldBegin(ID_FIELD_DESC);
+      oprot.writeString(this.id);
+      oprot.writeFieldEnd();
+    }
+    if (this.name != null) {
+      oprot.writeFieldBegin(NAME_FIELD_DESC);
+      oprot.writeString(this.name);
+      oprot.writeFieldEnd();
+    }
+    oprot.writeFieldBegin(UPTIME_SECS_FIELD_DESC);
+    oprot.writeI32(this.uptime_secs);
+    oprot.writeFieldEnd();
+    if (this.tasks != null) {
+      oprot.writeFieldBegin(TASKS_FIELD_DESC);
+      {
+        oprot.writeListBegin(new TList(TType.STRUCT, this.tasks.size()));
+        for (TaskSummary _iter128 : this.tasks)
+        {
+          _iter128.write(oprot);
+        }
+        oprot.writeListEnd();
+      }
+      oprot.writeFieldEnd();
+    }
+    oprot.writeFieldStop();
+    oprot.writeStructEnd();
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("TopologyInfo(");
+    boolean first = true;
+
+    sb.append("id:");
+    if (this.id == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.id);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("name:");
+    if (this.name == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.name);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("uptime_secs:");
+    sb.append(this.uptime_secs);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("tasks:");
+    if (this.tasks == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.tasks);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws TException {
+    // check for required fields
+    if (!is_set_id()) {
+      throw new TProtocolException("Required field 'id' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_name()) {
+      throw new TProtocolException("Required field 'name' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_uptime_secs()) {
+      throw new TProtocolException("Required field 'uptime_secs' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_tasks()) {
+      throw new TProtocolException("Required field 'tasks' is unset! Struct:" + toString());
+    }
+
+  }
+
+}
+
diff --git a/src/jvm/backtype/storm/generated/TopologySummary.java b/src/jvm/backtype/storm/generated/TopologySummary.java
new file mode 100644
index 0000000..0c00f16
--- /dev/null
+++ b/src/jvm/backtype/storm/generated/TopologySummary.java
@@ -0,0 +1,685 @@
+/**
+ * Autogenerated by Thrift
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ */
+package backtype.storm.generated;
+
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.util.Arrays;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.thrift.*;
+import org.apache.thrift.async.*;
+import org.apache.thrift.meta_data.*;
+import org.apache.thrift.transport.*;
+import org.apache.thrift.protocol.*;
+
+public class TopologySummary implements TBase<TopologySummary, TopologySummary._Fields>, java.io.Serializable, Cloneable {
+  private static final TStruct STRUCT_DESC = new TStruct("TopologySummary");
+
+  private static final TField ID_FIELD_DESC = new TField("id", TType.STRING, (short)1);
+  private static final TField NAME_FIELD_DESC = new TField("name", TType.STRING, (short)2);
+  private static final TField NUM_TASKS_FIELD_DESC = new TField("num_tasks", TType.I32, (short)3);
+  private static final TField NUM_WORKERS_FIELD_DESC = new TField("num_workers", TType.I32, (short)4);
+  private static final TField UPTIME_SECS_FIELD_DESC = new TField("uptime_secs", TType.I32, (short)5);
+
+  private String id;
+  private String name;
+  private int num_tasks;
+  private int num_workers;
+  private int uptime_secs;
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements TFieldIdEnum {
+    ID((short)1, "id"),
+    NAME((short)2, "name"),
+    NUM_TASKS((short)3, "num_tasks"),
+    NUM_WORKERS((short)4, "num_workers"),
+    UPTIME_SECS((short)5, "uptime_secs");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // ID
+          return ID;
+        case 2: // NAME
+          return NAME;
+        case 3: // NUM_TASKS
+          return NUM_TASKS;
+        case 4: // NUM_WORKERS
+          return NUM_WORKERS;
+        case 5: // UPTIME_SECS
+          return UPTIME_SECS;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __NUM_TASKS_ISSET_ID = 0;
+  private static final int __NUM_WORKERS_ISSET_ID = 1;
+  private static final int __UPTIME_SECS_ISSET_ID = 2;
+  private BitSet __isset_bit_vector = new BitSet(3);
+
+  public static final Map<_Fields, FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, FieldMetaData> tmpMap = new EnumMap<_Fields, FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.ID, new FieldMetaData("id", TFieldRequirementType.REQUIRED, 
+        new FieldValueMetaData(TType.STRING)));
+    tmpMap.put(_Fields.NAME, new FieldMetaData("name", TFieldRequirementType.REQUIRED, 
+        new FieldValueMetaData(TType.STRING)));
+    tmpMap.put(_Fields.NUM_TASKS, new FieldMetaData("num_tasks", TFieldRequirementType.REQUIRED, 
+        new FieldValueMetaData(TType.I32)));
+    tmpMap.put(_Fields.NUM_WORKERS, new FieldMetaData("num_workers", TFieldRequirementType.REQUIRED, 
+        new FieldValueMetaData(TType.I32)));
+    tmpMap.put(_Fields.UPTIME_SECS, new FieldMetaData("uptime_secs", TFieldRequirementType.REQUIRED, 
+        new FieldValueMetaData(TType.I32)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    FieldMetaData.addStructMetaDataMap(TopologySummary.class, metaDataMap);
+  }
+
+  public TopologySummary() {
+  }
+
+  public TopologySummary(
+    String id,
+    String name,
+    int num_tasks,
+    int num_workers,
+    int uptime_secs)
+  {
+    this();
+    this.id = id;
+    this.name = name;
+    this.num_tasks = num_tasks;
+    set_num_tasks_isSet(true);
+    this.num_workers = num_workers;
+    set_num_workers_isSet(true);
+    this.uptime_secs = uptime_secs;
+    set_uptime_secs_isSet(true);
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public TopologySummary(TopologySummary other) {
+    __isset_bit_vector.clear();
+    __isset_bit_vector.or(other.__isset_bit_vector);
+    if (other.is_set_id()) {
+      this.id = other.id;
+    }
+    if (other.is_set_name()) {
+      this.name = other.name;
+    }
+    this.num_tasks = other.num_tasks;
+    this.num_workers = other.num_workers;
+    this.uptime_secs = other.uptime_secs;
+  }
+
+  public TopologySummary deepCopy() {
+    return new TopologySummary(this);
+  }
+
+  @Deprecated
+  public TopologySummary clone() {
+    return new TopologySummary(this);
+  }
+
+  public String get_id() {
+    return this.id;
+  }
+
+  public void set_id(String id) {
+    this.id = id;
+  }
+
+  public void unset_id() {
+    this.id = null;
+  }
+
+  /** Returns true if field id is set (has been asigned a value) and false otherwise */
+  public boolean is_set_id() {
+    return this.id != null;
+  }
+
+  public void set_id_isSet(boolean value) {
+    if (!value) {
+      this.id = null;
+    }
+  }
+
+  public String get_name() {
+    return this.name;
+  }
+
+  public void set_name(String name) {
+    this.name = name;
+  }
+
+  public void unset_name() {
+    this.name = null;
+  }
+
+  /** Returns true if field name is set (has been asigned a value) and false otherwise */
+  public boolean is_set_name() {
+    return this.name != null;
+  }
+
+  public void set_name_isSet(boolean value) {
+    if (!value) {
+      this.name = null;
+    }
+  }
+
+  public int get_num_tasks() {
+    return this.num_tasks;
+  }
+
+  public void set_num_tasks(int num_tasks) {
+    this.num_tasks = num_tasks;
+    set_num_tasks_isSet(true);
+  }
+
+  public void unset_num_tasks() {
+    __isset_bit_vector.clear(__NUM_TASKS_ISSET_ID);
+  }
+
+  /** Returns true if field num_tasks is set (has been asigned a value) and false otherwise */
+  public boolean is_set_num_tasks() {
+    return __isset_bit_vector.get(__NUM_TASKS_ISSET_ID);
+  }
+
+  public void set_num_tasks_isSet(boolean value) {
+    __isset_bit_vector.set(__NUM_TASKS_ISSET_ID, value);
+  }
+
+  public int get_num_workers() {
+    return this.num_workers;
+  }
+
+  public void set_num_workers(int num_workers) {
+    this.num_workers = num_workers;
+    set_num_workers_isSet(true);
+  }
+
+  public void unset_num_workers() {
+    __isset_bit_vector.clear(__NUM_WORKERS_ISSET_ID);
+  }
+
+  /** Returns true if field num_workers is set (has been asigned a value) and false otherwise */
+  public boolean is_set_num_workers() {
+    return __isset_bit_vector.get(__NUM_WORKERS_ISSET_ID);
+  }
+
+  public void set_num_workers_isSet(boolean value) {
+    __isset_bit_vector.set(__NUM_WORKERS_ISSET_ID, value);
+  }
+
+  public int get_uptime_secs() {
+    return this.uptime_secs;
+  }
+
+  public void set_uptime_secs(int uptime_secs) {
+    this.uptime_secs = uptime_secs;
+    set_uptime_secs_isSet(true);
+  }
+
+  public void unset_uptime_secs() {
+    __isset_bit_vector.clear(__UPTIME_SECS_ISSET_ID);
+  }
+
+  /** Returns true if field uptime_secs is set (has been asigned a value) and false otherwise */
+  public boolean is_set_uptime_secs() {
+    return __isset_bit_vector.get(__UPTIME_SECS_ISSET_ID);
+  }
+
+  public void set_uptime_secs_isSet(boolean value) {
+    __isset_bit_vector.set(__UPTIME_SECS_ISSET_ID, value);
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case ID:
+      if (value == null) {
+        unset_id();
+      } else {
+        set_id((String)value);
+      }
+      break;
+
+    case NAME:
+      if (value == null) {
+        unset_name();
+      } else {
+        set_name((String)value);
+      }
+      break;
+
+    case NUM_TASKS:
+      if (value == null) {
+        unset_num_tasks();
+      } else {
+        set_num_tasks((Integer)value);
+      }
+      break;
+
+    case NUM_WORKERS:
+      if (value == null) {
+        unset_num_workers();
+      } else {
+        set_num_workers((Integer)value);
+      }
+      break;
+
+    case UPTIME_SECS:
+      if (value == null) {
+        unset_uptime_secs();
+      } else {
+        set_uptime_secs((Integer)value);
+      }
+      break;
+
+    }
+  }
+
+  public void setFieldValue(int fieldID, Object value) {
+    setFieldValue(_Fields.findByThriftIdOrThrow(fieldID), value);
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case ID:
+      return get_id();
+
+    case NAME:
+      return get_name();
+
+    case NUM_TASKS:
+      return new Integer(get_num_tasks());
+
+    case NUM_WORKERS:
+      return new Integer(get_num_workers());
+
+    case UPTIME_SECS:
+      return new Integer(get_uptime_secs());
+
+    }
+    throw new IllegalStateException();
+  }
+
+  public Object getFieldValue(int fieldId) {
+    return getFieldValue(_Fields.findByThriftIdOrThrow(fieldId));
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been asigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    switch (field) {
+    case ID:
+      return is_set_id();
+    case NAME:
+      return is_set_name();
+    case NUM_TASKS:
+      return is_set_num_tasks();
+    case NUM_WORKERS:
+      return is_set_num_workers();
+    case UPTIME_SECS:
+      return is_set_uptime_secs();
+    }
+    throw new IllegalStateException();
+  }
+
+  public boolean isSet(int fieldID) {
+    return isSet(_Fields.findByThriftIdOrThrow(fieldID));
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof TopologySummary)
+      return this.equals((TopologySummary)that);
+    return false;
+  }
+
+  public boolean equals(TopologySummary that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_id = true && this.is_set_id();
+    boolean that_present_id = true && that.is_set_id();
+    if (this_present_id || that_present_id) {
+      if (!(this_present_id && that_present_id))
+        return false;
+      if (!this.id.equals(that.id))
+        return false;
+    }
+
+    boolean this_present_name = true && this.is_set_name();
+    boolean that_present_name = true && that.is_set_name();
+    if (this_present_name || that_present_name) {
+      if (!(this_present_name && that_present_name))
+        return false;
+      if (!this.name.equals(that.name))
+        return false;
+    }
+
+    boolean this_present_num_tasks = true;
+    boolean that_present_num_tasks = true;
+    if (this_present_num_tasks || that_present_num_tasks) {
+      if (!(this_present_num_tasks && that_present_num_tasks))
+        return false;
+      if (this.num_tasks != that.num_tasks)
+        return false;
+    }
+
+    boolean this_present_num_workers = true;
+    boolean that_present_num_workers = true;
+    if (this_present_num_workers || that_present_num_workers) {
+      if (!(this_present_num_workers && that_present_num_workers))
+        return false;
+      if (this.num_workers != that.num_workers)
+        return false;
+    }
+
+    boolean this_present_uptime_secs = true;
+    boolean that_present_uptime_secs = true;
+    if (this_present_uptime_secs || that_present_uptime_secs) {
+      if (!(this_present_uptime_secs && that_present_uptime_secs))
+        return false;
+      if (this.uptime_secs != that.uptime_secs)
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    HashCodeBuilder builder = new HashCodeBuilder();
+
+    boolean present_id = true && (is_set_id());
+    builder.append(present_id);
+    if (present_id)
+      builder.append(id);
+
+    boolean present_name = true && (is_set_name());
+    builder.append(present_name);
+    if (present_name)
+      builder.append(name);
+
+    boolean present_num_tasks = true;
+    builder.append(present_num_tasks);
+    if (present_num_tasks)
+      builder.append(num_tasks);
+
+    boolean present_num_workers = true;
+    builder.append(present_num_workers);
+    if (present_num_workers)
+      builder.append(num_workers);
+
+    boolean present_uptime_secs = true;
+    builder.append(present_uptime_secs);
+    if (present_uptime_secs)
+      builder.append(uptime_secs);
+
+    return builder.toHashCode();
+  }
+
+  public int compareTo(TopologySummary other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+    TopologySummary typedOther = (TopologySummary)other;
+
+    lastComparison = Boolean.valueOf(is_set_id()).compareTo(typedOther.is_set_id());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_id()) {      lastComparison = TBaseHelper.compareTo(this.id, typedOther.id);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_name()).compareTo(typedOther.is_set_name());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_name()) {      lastComparison = TBaseHelper.compareTo(this.name, typedOther.name);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_num_tasks()).compareTo(typedOther.is_set_num_tasks());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_num_tasks()) {      lastComparison = TBaseHelper.compareTo(this.num_tasks, typedOther.num_tasks);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_num_workers()).compareTo(typedOther.is_set_num_workers());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_num_workers()) {      lastComparison = TBaseHelper.compareTo(this.num_workers, typedOther.num_workers);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(is_set_uptime_secs()).compareTo(typedOther.is_set_uptime_secs());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (is_set_uptime_secs()) {      lastComparison = TBaseHelper.compareTo(this.uptime_secs, typedOther.uptime_secs);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public void read(TProtocol iprot) throws TException {
+    TField field;
+    iprot.readStructBegin();
+    while (true)
+    {
+      field = iprot.readFieldBegin();
+      if (field.type == TType.STOP) { 
+        break;
+      }
+      switch (field.id) {
+        case 1: // ID
+          if (field.type == TType.STRING) {
+            this.id = iprot.readString();
+          } else { 
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 2: // NAME
+          if (field.type == TType.STRING) {
+            this.name = iprot.readString();
+          } else { 
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 3: // NUM_TASKS
+          if (field.type == TType.I32) {
+            this.num_tasks = iprot.readI32();
+            set_num_tasks_isSet(true);
+          } else { 
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 4: // NUM_WORKERS
+          if (field.type == TType.I32) {
+            this.num_workers = iprot.readI32();
+            set_num_workers_isSet(true);
+          } else { 
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 5: // UPTIME_SECS
+          if (field.type == TType.I32) {
+            this.uptime_secs = iprot.readI32();
+            set_uptime_secs_isSet(true);
+          } else { 
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        default:
+          TProtocolUtil.skip(iprot, field.type);
+      }
+      iprot.readFieldEnd();
+    }
+    iprot.readStructEnd();
+    validate();
+  }
+
+  public void write(TProtocol oprot) throws TException {
+    validate();
+
+    oprot.writeStructBegin(STRUCT_DESC);
+    if (this.id != null) {
+      oprot.writeFieldBegin(ID_FIELD_DESC);
+      oprot.writeString(this.id);
+      oprot.writeFieldEnd();
+    }
+    if (this.name != null) {
+      oprot.writeFieldBegin(NAME_FIELD_DESC);
+      oprot.writeString(this.name);
+      oprot.writeFieldEnd();
+    }
+    oprot.writeFieldBegin(NUM_TASKS_FIELD_DESC);
+    oprot.writeI32(this.num_tasks);
+    oprot.writeFieldEnd();
+    oprot.writeFieldBegin(NUM_WORKERS_FIELD_DESC);
+    oprot.writeI32(this.num_workers);
+    oprot.writeFieldEnd();
+    oprot.writeFieldBegin(UPTIME_SECS_FIELD_DESC);
+    oprot.writeI32(this.uptime_secs);
+    oprot.writeFieldEnd();
+    oprot.writeFieldStop();
+    oprot.writeStructEnd();
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("TopologySummary(");
+    boolean first = true;
+
+    sb.append("id:");
+    if (this.id == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.id);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("name:");
+    if (this.name == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.name);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("num_tasks:");
+    sb.append(this.num_tasks);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("num_workers:");
+    sb.append(this.num_workers);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("uptime_secs:");
+    sb.append(this.uptime_secs);
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws TException {
+    // check for required fields
+    if (!is_set_id()) {
+      throw new TProtocolException("Required field 'id' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_name()) {
+      throw new TProtocolException("Required field 'name' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_num_tasks()) {
+      throw new TProtocolException("Required field 'num_tasks' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_num_workers()) {
+      throw new TProtocolException("Required field 'num_workers' is unset! Struct:" + toString());
+    }
+
+    if (!is_set_uptime_secs()) {
+      throw new TProtocolException("Required field 'uptime_secs' is unset! Struct:" + toString());
+    }
+
+  }
+
+}
+
diff --git a/src/jvm/backtype/storm/planner/CompoundSpout.java b/src/jvm/backtype/storm/planner/CompoundSpout.java
new file mode 100644
index 0000000..f6ee3b2
--- /dev/null
+++ b/src/jvm/backtype/storm/planner/CompoundSpout.java
@@ -0,0 +1,8 @@
+package backtype.storm.planner;
+
+
+public class CompoundSpout
+        //implements ISpout
+{
+
+}
\ No newline at end of file
diff --git a/src/jvm/backtype/storm/planner/CompoundTask.java b/src/jvm/backtype/storm/planner/CompoundTask.java
new file mode 100644
index 0000000..7275835
--- /dev/null
+++ b/src/jvm/backtype/storm/planner/CompoundTask.java
@@ -0,0 +1,8 @@
+package backtype.storm.planner;
+
+
+public class CompoundTask
+//        implements IBolt
+{
+
+}
\ No newline at end of file
diff --git a/src/jvm/backtype/storm/planner/TaskBundle.java b/src/jvm/backtype/storm/planner/TaskBundle.java
new file mode 100644
index 0000000..d587ff5
--- /dev/null
+++ b/src/jvm/backtype/storm/planner/TaskBundle.java
@@ -0,0 +1,16 @@
+package backtype.storm.planner;
+
+import backtype.storm.task.IBolt;
+import java.io.Serializable;
+
+
+public class TaskBundle implements Serializable {
+    public IBolt task;
+    public int componentId;
+    
+    public TaskBundle(IBolt task, int componentId) {
+        this.task = task;
+        this.componentId = componentId;
+    }
+    
+}
\ No newline at end of file
diff --git a/src/jvm/backtype/storm/serialization/FieldSerialization.java b/src/jvm/backtype/storm/serialization/FieldSerialization.java
new file mode 100644
index 0000000..4330054
--- /dev/null
+++ b/src/jvm/backtype/storm/serialization/FieldSerialization.java
@@ -0,0 +1,29 @@
+package backtype.storm.serialization;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+
+public class FieldSerialization {
+    private int _token;
+    private ISerialization _serialization;
+
+    public FieldSerialization(int token, ISerialization serialization) {
+        _serialization = serialization;
+        _token = token;
+    }
+
+    public int getToken() {
+        return _token;
+    }
+
+    public void serialize(Object obj, DataOutputStream out) throws IOException {
+        _serialization.serialize(obj, out);
+    }
+
+    public Object deserialize(DataInputStream in) throws IOException {
+        return _serialization.deserialize(in);
+    }
+
+}
diff --git a/src/jvm/backtype/storm/serialization/ISerialization.java b/src/jvm/backtype/storm/serialization/ISerialization.java
new file mode 100644
index 0000000..b51e6a0
--- /dev/null
+++ b/src/jvm/backtype/storm/serialization/ISerialization.java
@@ -0,0 +1,31 @@
+package backtype.storm.serialization;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+/**
+ * Implement this interface to provide Storm with the ability to serialize and
+ * deserialize types besides the primitives that you want to put into tuples.
+ *
+ * <p>You must declare these new serializers in the configuration of either the cluster
+ * or the topology that will use these seralizations. Once you declare a serialization,
+ * Storm will make use of it automatically. See "topology.serializations" in
+ * {@link backtype.storm.Config} for more details.</p>
+ */
+public interface ISerialization<T> {
+    /**
+     * Returns whether this serialization can handle the given type.
+     */
+    public boolean accept(Class<T> c);
+
+    /**
+     * Serializes the provided object into the stream.
+     */
+    public void serialize(T object, DataOutputStream stream) throws IOException;
+
+    /**
+     * Reads an object of the proper type off of the stream.
+     */
+    public T deserialize(DataInputStream stream) throws IOException;
+}
\ No newline at end of file
diff --git a/src/jvm/backtype/storm/serialization/SerializationFactory.java b/src/jvm/backtype/storm/serialization/SerializationFactory.java
new file mode 100644
index 0000000..b7a64e1
--- /dev/null
+++ b/src/jvm/backtype/storm/serialization/SerializationFactory.java
@@ -0,0 +1,207 @@
+package backtype.storm.serialization;
+
+import backtype.storm.Config;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.log4j.Logger;
+
+
+public class SerializationFactory {
+    public static Logger LOG = Logger.getLogger(SerializationFactory.class);
+    private static byte[] EMPTY_BYTE_ARRAY = new byte[0];
+
+    private Map<Integer, ISerialization> _serializations = new HashMap<Integer, ISerialization>() {{
+        put(1, new ISerialization<Integer>() {
+                public boolean accept(Class c) {
+                    return Integer.class.equals(c);
+                }
+
+                public void serialize(Integer object, DataOutputStream stream) throws IOException {
+                    stream.writeInt(object);
+//                    WritableUtils.writeVInt(stream, object);
+                }
+
+                public Integer deserialize(DataInputStream stream) throws IOException {
+//                    return WritableUtils.readVInt(stream);
+                    return stream.readInt();
+                }
+        });
+        put(2, new ISerialization<Long>() {
+                public boolean accept(Class c) {
+                    return Long.class.equals(c);
+                }
+
+                public void serialize(Long object, DataOutputStream stream) throws IOException {
+                    stream.writeLong(object);
+//                    WritableUtils.writeVLong(stream, object);
+                }
+
+                public Long deserialize(DataInputStream stream) throws IOException {
+//                    return WritableUtils.readVLong(stream);
+                    return stream.readLong();
+                }
+        });
+        put(3, new ISerialization<Float>() {
+                public boolean accept(Class c) {
+                    return Float.class.equals(c);
+                }
+
+                public void serialize(Float object, DataOutputStream stream) throws IOException {
+                    stream.writeFloat(object);
+                }
+
+                public Float deserialize(DataInputStream stream) throws IOException {
+                    return stream.readFloat();
+                }
+        });
+        put(4, new ISerialization<Double>() {
+                public boolean accept(Class c) {
+                    return Double.class.equals(c);
+                }
+
+                public void serialize(Double object, DataOutputStream stream) throws IOException {
+                    stream.writeDouble(object);
+                }
+
+                public Double deserialize(DataInputStream stream) throws IOException {
+                    return stream.readDouble();
+                }
+        });
+        put(5, new ISerialization<Byte>() {
+                public boolean accept(Class c) {
+                    return Byte.class.equals(c);
+                }
+
+                public void serialize(Byte object, DataOutputStream stream) throws IOException {
+                    stream.writeByte(object);
+                }
+
+                public Byte deserialize(DataInputStream stream) throws IOException {
+                    return stream.readByte();
+                }
+        });
+        put(6, new ISerialization<Short>() {
+                public boolean accept(Class c) {
+                    return Short.class.equals(c);
+                }
+
+                public void serialize(Short object, DataOutputStream stream) throws IOException {
+                    stream.writeShort(object);
+                }
+
+                public Short deserialize(DataInputStream stream) throws IOException {
+                    return stream.readShort();
+                }
+        });
+        put(7, new ISerialization<String>() {
+                public boolean accept(Class c) {
+                    return String.class.equals(c);
+                }
+
+                public void serialize(String object, DataOutputStream stream) throws IOException {
+                    stream.writeUTF(object);
+                }
+
+                public String deserialize(DataInputStream stream) throws IOException {
+                    return stream.readUTF();
+                }
+        });
+        put(8, new ISerialization<Boolean>() {
+                public boolean accept(Class c) {
+                    return Boolean.class.equals(c);
+                }
+
+                public void serialize(Boolean object, DataOutputStream stream) throws IOException {
+                    stream.writeBoolean(object);
+                }
+
+                public Boolean deserialize(DataInputStream stream) throws IOException {
+                    return stream.readBoolean();
+                }
+        });
+        put(9, new ISerialization<byte[]>() {
+            
+                public boolean accept(Class c) {
+                    return EMPTY_BYTE_ARRAY.getClass().equals(c);
+                }
+
+                public void serialize(byte[] object, DataOutputStream stream) throws IOException {
+//                    WritableUtils.writeVInt(stream, object.length);
+                    stream.writeInt(object.length);
+                    stream.write(object, 0, object.length);
+                }
+
+                public byte[] deserialize(DataInputStream stream) throws IOException {
+//                    int size = WritableUtils.readVInt(stream);
+                    int size = stream.readInt();
+                    byte[] ret = new byte[size];
+                    stream.readFully(ret);
+                    return ret;
+                }
+        });
+    }};
+
+    public SerializationFactory(Map conf) {
+        boolean skipMissing = (Boolean) conf.get(Config.TOPOLOGY_SKIP_MISSING_SERIALIZATIONS);
+        Map<Object, String> customSerializations = (Map<Object, String>) conf.get(Config.TOPOLOGY_SERIALIZATIONS);
+        if(customSerializations==null) customSerializations = new HashMap<Object, String>();
+        for(Object tokenObj: customSerializations.keySet()) {
+            String serializationClassName = customSerializations.get(tokenObj);
+            int token = toToken(tokenObj);
+            if(token<=32) {
+                throw new RuntimeException("Illegal token " + token + " for " + serializationClassName);
+            }
+            try {
+                LOG.info("Loading custom serialization " + serializationClassName + " for token " + token);
+                Class serClass = Class.forName(serializationClassName);
+                _serializations.put(token, (ISerialization) serClass.newInstance());
+            } catch(ClassNotFoundException e) {
+                if(skipMissing) {
+                    LOG.info("Could not find serialization for " + serializationClassName + ". Skipping...");
+                } else {
+                    throw new RuntimeException(e);
+                }
+            } catch(InstantiationException e) {
+                throw new RuntimeException(e);
+            } catch(IllegalAccessException e) {
+                throw new RuntimeException(e);
+            }
+        }
+    }
+
+    private int toToken(Object tokenObj) {
+        if(tokenObj instanceof Long) {
+            return ((Long) tokenObj).intValue();
+        } else if(tokenObj instanceof Integer) {
+            return (Integer) tokenObj;
+        } else if (tokenObj instanceof String) {
+            // this special case exists because user's storm conf gets serialized to json, which will convert numbered keys into strings
+            // TODO: replace use of json in nimbus storm submit with yaml
+            return Integer.parseInt((String) tokenObj);
+        } else {
+            throw new RuntimeException("Unexpected token class " + tokenObj + " " + tokenObj.getClass().toString());
+        }
+    }
+
+
+    public FieldSerialization getSerializationForToken(int token) {
+        ISerialization ser = _serializations.get(token);
+        if(ser==null) {
+            throw new RuntimeException("Could not find serialization for token " + token);
+        }
+        return new FieldSerialization(token, ser);
+    }
+
+    public FieldSerialization getSerializationForClass(Class klass) {
+        for(int token: _serializations.keySet()) {
+            ISerialization ser = _serializations.get(token);
+            if(ser.accept(klass)) {
+                return getSerializationForToken(token);
+            }
+        }
+        throw new RuntimeException("Could not find serialization for class " + klass.toString());
+    }
+}
diff --git a/src/jvm/backtype/storm/serialization/TupleDeserializer.java b/src/jvm/backtype/storm/serialization/TupleDeserializer.java
new file mode 100644
index 0000000..b21ebbb
--- /dev/null
+++ b/src/jvm/backtype/storm/serialization/TupleDeserializer.java
@@ -0,0 +1,49 @@
+package backtype.storm.serialization;
+
+import backtype.storm.task.TopologyContext;
+import backtype.storm.tuple.MessageId;
+import backtype.storm.tuple.Tuple;
+import backtype.storm.utils.WritableUtils;
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class TupleDeserializer {
+    Map<Integer, Map<Integer, ValuesDeserializer>> _fieldSerializers = new HashMap<Integer, Map<Integer, ValuesDeserializer>>();
+    Map _conf;
+    TopologyContext _context;
+
+    public TupleDeserializer(Map conf, TopologyContext context) {
+        _conf = conf;
+        _context = context;
+    }
+
+    public Tuple deserialize(byte[] ser) throws IOException {
+        ByteArrayInputStream bin = new ByteArrayInputStream(ser);
+        DataInputStream in = new DataInputStream(bin);
+        int taskId = WritableUtils.readVInt(in);
+        int streamId = WritableUtils.readVInt(in);
+        MessageId id = MessageId.deserialize(in);
+        int componentId = _context.getComponentId(taskId);
+        ValuesDeserializer streamSerializers = getValuesDeserializer(_fieldSerializers, componentId, streamId);
+        List<Object> values = streamSerializers.deserializeFrom(in);
+        return new Tuple(_context, values, taskId, streamId, id);
+    }
+    
+    private ValuesDeserializer getValuesDeserializer(Map<Integer, Map<Integer, ValuesDeserializer>> deserializers, int componentId, int streamId) {
+        Map<Integer, ValuesDeserializer> streamToSerializers = deserializers.get(componentId);
+        if(streamToSerializers==null) {
+            streamToSerializers = new HashMap<Integer, ValuesDeserializer>();
+            deserializers.put(componentId, streamToSerializers);
+        }
+        ValuesDeserializer streamSerializers = streamToSerializers.get(streamId);
+        if(streamSerializers==null) {
+            streamSerializers = new ValuesDeserializer(_conf);
+            streamToSerializers.put(streamId, streamSerializers);
+        }
+        return streamSerializers;        
+    }
+}
\ No newline at end of file
diff --git a/src/jvm/backtype/storm/serialization/TupleSerializer.java b/src/jvm/backtype/storm/serialization/TupleSerializer.java
new file mode 100644
index 0000000..24de8f3
--- /dev/null
+++ b/src/jvm/backtype/storm/serialization/TupleSerializer.java
@@ -0,0 +1,58 @@
+package backtype.storm.serialization;
+
+import backtype.storm.tuple.Tuple;
+import backtype.storm.utils.CRC32OutputStream;
+import backtype.storm.utils.WritableUtils;
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+
+public class TupleSerializer {
+    ByteArrayOutputStream _outputter;
+    DataOutputStream _dataOutputter;
+    Map<Integer, Map<Integer, ValuesSerializer>> _fieldSerializers = new HashMap<Integer, Map<Integer, ValuesSerializer>>();
+    Map _conf;
+    
+    public TupleSerializer(Map conf) {
+        _outputter = new ByteArrayOutputStream();
+        _dataOutputter = new DataOutputStream(_outputter);
+        _conf = conf;
+    }    
+    
+    public byte[] serialize(Tuple tuple) throws IOException {
+        _outputter.reset();
+        WritableUtils.writeVInt(_dataOutputter, tuple.getSourceTask());
+        WritableUtils.writeVInt(_dataOutputter, tuple.getSourceStreamId());
+        tuple.getMessageId().serialize(_dataOutputter);
+        ValuesSerializer streamSerializers = getValuesSerializer(_fieldSerializers, tuple);
+        streamSerializers.serializeInto(tuple.getValues(), _dataOutputter);
+        return _outputter.toByteArray();
+    }
+
+    public long crc32(Tuple tuple) {
+        CRC32OutputStream hasher = new CRC32OutputStream();
+        try {
+            getValuesSerializer(_fieldSerializers, tuple).serializeInto(tuple.getValues(), new DataOutputStream(hasher));
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+        return hasher.getValue();
+    }    
+    
+    private ValuesSerializer getValuesSerializer(Map<Integer, Map<Integer, ValuesSerializer>> serializers, Tuple tuple) {
+        Map<Integer, ValuesSerializer> streamToSerializers = serializers.get(tuple.getSourceComponent());
+        if(streamToSerializers==null) {
+            streamToSerializers = new HashMap<Integer, ValuesSerializer>();
+            serializers.put(tuple.getSourceComponent(), streamToSerializers);
+        }
+        ValuesSerializer streamSerializers = streamToSerializers.get(tuple.getSourceStreamId());
+        if(streamSerializers==null) {
+            streamSerializers = new ValuesSerializer(_conf);
+            streamToSerializers.put(tuple.getSourceStreamId(), streamSerializers);
+        }
+        return streamSerializers;
+    }        
+}
diff --git a/src/jvm/backtype/storm/serialization/ValuesDeserializer.java b/src/jvm/backtype/storm/serialization/ValuesDeserializer.java
new file mode 100644
index 0000000..dcdc042
--- /dev/null
+++ b/src/jvm/backtype/storm/serialization/ValuesDeserializer.java
@@ -0,0 +1,49 @@
+package backtype.storm.serialization;
+
+import backtype.storm.utils.WritableUtils;
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+
+public class ValuesDeserializer {
+    SerializationFactory _factory;
+    List<FieldSerialization> _deserializers;
+    
+    public ValuesDeserializer(Map conf) {
+        _factory = new SerializationFactory(conf);        
+    }
+    
+    public List<Object> deserializeFrom(DataInputStream in) throws IOException {
+        int numValues = WritableUtils.readVInt(in);
+        if(_deserializers==null) {
+            _deserializers = new ArrayList<FieldSerialization>();
+            for(int i=0; i<numValues; i++) {
+                _deserializers.add(null);
+            }                    
+        }
+        if(numValues!=_deserializers.size()) {
+            throw new RuntimeException("Received a tuple with an unexpected number of fields");
+        }
+        if(numValues==0) return null;
+        List<Object> values = new ArrayList<Object>(numValues);
+        for(int i=0; i<_deserializers.size(); i++) {
+            int token = WritableUtils.readVInt(in);
+            if(token==0) {
+                values.add(null);
+            } else {
+                if(_deserializers.get(i)==null) {
+                    _deserializers.set(i, _factory.getSerializationForToken(token));
+                }
+                FieldSerialization fser = _deserializers.get(i);
+                if(token!=fser.getToken()) {
+                    throw new RuntimeException("Received field of different types " + token + " " + fser.getToken());
+                }
+                values.add(fser.deserialize(in));
+            }
+        }        
+        return values;        
+    }
+}
diff --git a/src/jvm/backtype/storm/serialization/ValuesSerializer.java b/src/jvm/backtype/storm/serialization/ValuesSerializer.java
new file mode 100644
index 0000000..a943ab8
--- /dev/null
+++ b/src/jvm/backtype/storm/serialization/ValuesSerializer.java
@@ -0,0 +1,54 @@
+package backtype.storm.serialization;
+
+import backtype.storm.utils.WritableUtils;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+
+/**
+ * Expects to see tuples of same size and field types
+ */
+public class ValuesSerializer {
+    List<FieldSerialization> _serializers;
+    SerializationFactory _factory;
+
+    public ValuesSerializer(Map conf) {
+        _factory = new SerializationFactory(conf);
+    }
+    
+    public void serializeInto(List values, DataOutputStream out) throws IOException {
+        int numValues = 0;
+        if(values!=null) {
+            numValues = values.size();
+        }
+        if(_serializers==null) {
+            _serializers = new ArrayList<FieldSerialization>();
+            for(int i=0; i<numValues; i++) {
+                _serializers.add(null);
+            }
+        }
+
+
+        if(_serializers.size()!=numValues) {
+            throw new RuntimeException("Received unexpected tuple with different size than a prior tuple on this stream " + values.toString());
+        }
+        
+        WritableUtils.writeVInt(out, numValues);
+        for(int i=0; i<_serializers.size(); i++) {
+            Object val = values.get(i);
+            if(val==null) {
+                WritableUtils.writeVInt(out, 0);
+            } else {
+                if(_serializers.get(i)==null) {
+                    _serializers.set(i, _factory.getSerializationForClass(val.getClass()));
+                }
+                FieldSerialization fs = _serializers.get(i);
+                WritableUtils.writeVInt(out, fs.getToken());
+                fs.serialize(values.get(i), out);
+            }
+        }
+    }
+}
diff --git a/src/jvm/backtype/storm/spout/ISpout.java b/src/jvm/backtype/storm/spout/ISpout.java
new file mode 100644
index 0000000..20a593b
--- /dev/null
+++ b/src/jvm/backtype/storm/spout/ISpout.java
@@ -0,0 +1,71 @@
+package backtype.storm.spout;
+
+import backtype.storm.task.TopologyContext;
+import java.util.Map;
+import java.io.Serializable;
+
+/**
+ * ISpout is the core interface for implementing spouts. A Spout is responsible
+ * for feeding messages into the topology for processing. For every tuple emitted by
+ * a spout, Storm will track the (potentially very large) DAG of tuples generated
+ * based on a tuple emitted by the spout. When Storm detects that every tuple in
+ * that DAG has been successfuly processed, it will send an ack message to the Spout.
+ *
+ * <p>If a tuple fails to be fully process within the configured timeout for the
+ * topology (see {@link backtype.storm.Config}), Storm will send a fail message to the spout for the message.</p>
+ *
+ * <p> When a Spout emits a tuple, it tags the tuple with a message id. The message id
+ * can be of any type. When Storm acks or fails a message, it will pass back to the
+ * spout the same message id to identify which tuple it's referring to. If the spout leaves out
+ * the message id, or sets it to null, then Storm will not track the message and the spout
+ * will not receive any ack or fail callbacks for the message.</p>
+ *
+ * <p>Storm executes ack, fail, and nextTuple all on the same thread. This means that an implementor
+ * of an ISpout does not need to worry about concurrency issues between those methods. However, it 
+ * also means that an implementor must ensure that nextTuple is non-blocking: otherwise 
+ * the method could block acks and fails that are pending to be processed.</p>
+ */
+public interface ISpout extends Serializable {
+    /**
+     * Called when a task for this component is initialized within a worker on the cluster.
+     * It provides the spout with the environment in which the spout executes.
+     *
+     * <p>This includes the:</p>
+     *
+     * @param conf The Storm configuration for this spout. This is the configuration provided to the topology merged in with cluster configuration on this machine.
+     * @param context This object can be used to get information about this task's place within the topology, including the task id and component id of this task, input and output information, etc.
+     * @param collector The collector is used to emit tuples from this spout. Tuples can be emitted at any time, including the open and close methods. The collector is thread-safe and should be saved as an instance variable of this spout object.
+     */
+    void open(Map conf, TopologyContext context, SpoutOutputCollector collector);
+
+    /**
+     * Called when an ISpout is going to be shutdown. There is no guarentee that close
+     * will be called, because the supervisor kill -9's worker processes on the cluster.
+     *
+     * <p>The one context where close is guaranteed to be called is a topology is
+     * killed when running Storm in local mode.</p>
+     */
+    void close();
+
+    /**
+     * When this method is called, Storm is requesting that the Spout emit tuples to the 
+     * output collector. This method should be non-blocking, so if the Spout has no tuples
+     * to emit, this method should return. It is recommended that the Spout call Thread.sleep
+     * for a short amount of time in this case.
+     */
+    void nextTuple();
+
+    /**
+     * Storm has determined that the tuple emitted by this spout with the msgId identifier
+     * has been fully processed. Typically, an implementation of this method will take that
+     * message off the queue and prevent it from being replayed.
+     */
+    void ack(Object msgId);
+
+    /**
+     * The tuple emitted by this spout with the msgId identifier has failed to be
+     * fully processed. Typically, an implementation of this method will put that
+     * message back on the queue to be replayed at a later time.
+     */
+    void fail(Object msgId);
+}
\ No newline at end of file
diff --git a/src/jvm/backtype/storm/spout/ISpoutOutputCollector.java b/src/jvm/backtype/storm/spout/ISpoutOutputCollector.java
new file mode 100644
index 0000000..dc29fbb
--- /dev/null
+++ b/src/jvm/backtype/storm/spout/ISpoutOutputCollector.java
@@ -0,0 +1,13 @@
+package backtype.storm.spout;
+
+import java.util.List;
+import backtype.storm.tuple.Tuple;
+
+public interface ISpoutOutputCollector {
+    /**
+        Returns the task ids that received the tuples.
+    */
+    List<Integer> emit(int streamId, List<Object> tuple, Object messageId);
+    void emitDirect(int taskId, int streamId, List<Object> tuple, Object messageId);
+}
+
diff --git a/src/jvm/backtype/storm/spout/RawScheme.java b/src/jvm/backtype/storm/spout/RawScheme.java
new file mode 100644
index 0000000..3d8dab3
--- /dev/null
+++ b/src/jvm/backtype/storm/spout/RawScheme.java
@@ -0,0 +1,15 @@
+package backtype.storm.spout;
+
+import backtype.storm.tuple.Fields;
+import java.util.List;
+import static backtype.storm.utils.Utils.tuple;
+
+public class RawScheme implements Scheme {
+    public List<Object> deserialize(byte[] ser) {
+        return tuple(ser);
+    }
+
+    public Fields getOutputFields() {
+        return new Fields("bytes");
+    }
+}
diff --git a/src/jvm/backtype/storm/spout/Scheme.java b/src/jvm/backtype/storm/spout/Scheme.java
new file mode 100644
index 0000000..e5a1cb9
--- /dev/null
+++ b/src/jvm/backtype/storm/spout/Scheme.java
@@ -0,0 +1,11 @@
+package backtype.storm.spout;
+
+import backtype.storm.tuple.Fields;
+import java.io.Serializable;
+import java.util.List;
+
+
+public interface Scheme extends Serializable {
+    public List<Object> deserialize(byte[] ser);
+    public Fields getOutputFields();
+}
diff --git a/src/jvm/backtype/storm/spout/ShellSpout.java b/src/jvm/backtype/storm/spout/ShellSpout.java
new file mode 100644
index 0000000..54902cc
--- /dev/null
+++ b/src/jvm/backtype/storm/spout/ShellSpout.java
@@ -0,0 +1,33 @@
+package backtype.storm.spout;
+
+import backtype.storm.generated.ShellComponent;
+import backtype.storm.task.TopologyContext;
+import java.util.Map;
+
+
+public class ShellSpout implements ISpout {
+    public ShellSpout(ShellComponent component) {
+
+    }
+    
+    public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) {
+        throw new UnsupportedOperationException("Not supported yet.");
+    }
+
+    public void close() {
+        throw new UnsupportedOperationException("Not supported yet.");
+    }
+
+    public void nextTuple() {
+        throw new UnsupportedOperationException("Not supported yet.");
+    }
+
+    public void ack(Object msgId) {
+        throw new UnsupportedOperationException("Not supported yet.");
+    }
+
+    public void fail(Object msgId) {
+        throw new UnsupportedOperationException("Not supported yet.");
+    }
+
+}
diff --git a/src/jvm/backtype/storm/spout/SpoutOutputCollector.java b/src/jvm/backtype/storm/spout/SpoutOutputCollector.java
new file mode 100644
index 0000000..23c022c
--- /dev/null
+++ b/src/jvm/backtype/storm/spout/SpoutOutputCollector.java
@@ -0,0 +1,104 @@
+package backtype.storm.spout;
+
+import backtype.storm.task.OutputCollector;
+import backtype.storm.utils.Utils;
+import java.util.List;
+
+/**
+ * This output collector exposes the API for emitting tuples from an {@link backtype.storm.topology.IRichSpout}.
+ * The main difference between this output collector and {@link OutputCollector}
+ * for {@link backtype.storm.topology.IRichBolt} is that spouts can tag messages with ids so that they can be
+ * acked or failed later on. This is the Spout portion of Storm's API to
+ * guarantee that each message is fully processed at least once.
+ */
+public class SpoutOutputCollector implements ISpoutOutputCollector {
+    ISpoutOutputCollector _delegate;
+
+    public SpoutOutputCollector(ISpoutOutputCollector delegate) {
+        _delegate = delegate;
+    }
+
+    /**
+     * Emits a new tuple to the specified output stream with the given message ID.
+     * When Storm detects that this tuple has been fully processed, or has failed
+     * to be fully processed, the spout will receive an ack or fail callback respectively
+     * with the messageId as long as the messageId was not null. If the messageId was null,
+     * Storm will not track the tuple and no callback will be received.
+     *
+     * @return the list of task ids that this tuple was sent to
+     */
+    public List<Integer> emit(int streamId, List<Object> tuple, Object messageId) {
+        return _delegate.emit(streamId, tuple, messageId);
+    }
+
+    /**
+     * Emits a new tuple to the default output stream with the given message ID.
+     * When Storm detects that this tuple has been fully processed, or has failed
+     * to be fully processed, the spout will receive an ack or fail callback respectively
+     * with the messageId as long as the messageId was not null. If the messageId was null,
+     * Storm will not track the tuple and no callback will be received.
+     *
+     * @return the list of task ids that this tuple was sent to
+     */
+    public List<Integer> emit(List<Object> tuple, Object messageId) {
+        return emit(Utils.DEFAULT_STREAM_ID, tuple, messageId);
+    }
+
+    /**
+     * Emits a tuple to the default output stream with a null message id. Storm will
+     * not track this message so ack and fail will never be called for this tuple.
+     */
+    public List<Integer> emit(List<Object> tuple) {
+        return emit(tuple, null);
+    }
+
+    /**
+     * Emits a tuple to the specified output stream with a null message id. Storm will
+     * not track this message so ack and fail will never be called for this tuple.
+     */
+    public List<Integer> emit(int streamId, List<Object> tuple) {
+        return emit(streamId, tuple, null);
+    }
+
+    /**
+     * Emits a tuple to the specified task on the specified output stream. This output
+     * stream must have been declared as a direct stream, and the specified task must
+     * use a direct grouping on this stream to receive the message. 
+     */
+    public void emitDirect(int taskId, int streamId, List<Object> tuple, Object messageId) {
+        _delegate.emitDirect(taskId, streamId, tuple, messageId);
+    }
+
+    /**
+     * Emits a tuple to the specified task on the default output stream. This output
+     * stream must have been declared as a direct stream, and the specified task must
+     * use a direct grouping on this stream to receive the message.
+     */
+    public void emitDirect(int taskId, List<Object> tuple, Object messageId) {
+        emitDirect(taskId, Utils.DEFAULT_STREAM_ID, tuple, messageId);
+    }
+    
+    /**
+     * Emits a tuple to the specified task on the specified output stream. This output
+     * stream must have been declared as a direct stream, and the specified task must
+     * use a direct grouping on this stream to receive the message.
+     *
+     * <p> Because no message id is specified, Storm will not track this message
+     * so ack and fail will never be called for this tuple.</p>
+     */
+    public void emitDirect(int taskId, int streamId, List<Object> tuple) {
+        emitDirect(taskId, streamId, tuple, null);
+    }
+
+    /**
+     * Emits a tuple to the specified task on the default output stream. This output
+     * stream must have been declared as a direct stream, and the specified task must
+     * use a direct grouping on this stream to receive the message.
+     *
+     * <p> Because no message id is specified, Storm will not track this message
+     * so ack and fail will never be called for this tuple.</p>
+     */
+    public void emitDirect(int taskId, List<Object> tuple) {
+        emitDirect(taskId, tuple, null);
+    }
+}
diff --git a/src/jvm/backtype/storm/state/IStateSpout.java b/src/jvm/backtype/storm/state/IStateSpout.java
new file mode 100644
index 0000000..f39fb56
--- /dev/null
+++ b/src/jvm/backtype/storm/state/IStateSpout.java
@@ -0,0 +1,12 @@
+package backtype.storm.state;
+
+import backtype.storm.task.TopologyContext;
+import java.io.Serializable;
+import java.util.Map;
+
+public interface IStateSpout extends Serializable {
+    void open(Map conf, TopologyContext context);
+    void close();
+    void nextTuple(StateSpoutOutputCollector collector);
+    void synchronize(SynchronizeOutputCollector collector);
+}
diff --git a/src/jvm/backtype/storm/state/IStateSpoutOutputCollector.java b/src/jvm/backtype/storm/state/IStateSpoutOutputCollector.java
new file mode 100644
index 0000000..221c197
--- /dev/null
+++ b/src/jvm/backtype/storm/state/IStateSpoutOutputCollector.java
@@ -0,0 +1,5 @@
+package backtype.storm.state;
+
+public interface IStateSpoutOutputCollector extends ISynchronizeOutputCollector {
+    void remove(int streamId, Object id);
+}
diff --git a/src/jvm/backtype/storm/state/ISubscribedState.java b/src/jvm/backtype/storm/state/ISubscribedState.java
new file mode 100644
index 0000000..8ba7925
--- /dev/null
+++ b/src/jvm/backtype/storm/state/ISubscribedState.java
@@ -0,0 +1,8 @@
+package backtype.storm.state;
+
+import backtype.storm.tuple.Tuple;
+
+public interface ISubscribedState {
+    void set(Object id, Tuple tuple);
+    void remove(Object id);
+}
diff --git a/src/jvm/backtype/storm/state/ISynchronizeOutputCollector.java b/src/jvm/backtype/storm/state/ISynchronizeOutputCollector.java
new file mode 100644
index 0000000..31a2a7f
--- /dev/null
+++ b/src/jvm/backtype/storm/state/ISynchronizeOutputCollector.java
@@ -0,0 +1,7 @@
+package backtype.storm.state;
+
+import java.util.List;
+
+public interface ISynchronizeOutputCollector {
+    void add(int streamId, Object id, List<Object> tuple);    
+}
diff --git a/src/jvm/backtype/storm/state/StateSpoutOutputCollector.java b/src/jvm/backtype/storm/state/StateSpoutOutputCollector.java
new file mode 100644
index 0000000..53a74ca
--- /dev/null
+++ b/src/jvm/backtype/storm/state/StateSpoutOutputCollector.java
@@ -0,0 +1,11 @@
+package backtype.storm.state;
+
+
+public class StateSpoutOutputCollector extends SynchronizeOutputCollector implements IStateSpoutOutputCollector {
+
+    @Override
+    public void remove(int streamId, Object id) {
+        throw new UnsupportedOperationException("Not supported yet.");
+    }
+
+}
diff --git a/src/jvm/backtype/storm/state/SynchronizeOutputCollector.java b/src/jvm/backtype/storm/state/SynchronizeOutputCollector.java
new file mode 100644
index 0000000..8bf56c6
--- /dev/null
+++ b/src/jvm/backtype/storm/state/SynchronizeOutputCollector.java
@@ -0,0 +1,13 @@
+package backtype.storm.state;
+
+import java.util.List;
+
+
+public class SynchronizeOutputCollector implements ISynchronizeOutputCollector {
+
+    @Override
+    public void add(int streamId, Object id, List<Object> tuple) {
+        throw new UnsupportedOperationException("Not supported yet.");
+    }
+
+}
diff --git a/src/jvm/backtype/storm/task/CoordinatedBolt.java b/src/jvm/backtype/storm/task/CoordinatedBolt.java
new file mode 100644
index 0000000..c76555b
--- /dev/null
+++ b/src/jvm/backtype/storm/task/CoordinatedBolt.java
@@ -0,0 +1,219 @@
+package backtype.storm.task;
+
+import backtype.storm.Constants;
+import backtype.storm.generated.Grouping;
+import backtype.storm.topology.IRichBolt;
+import backtype.storm.topology.OutputFieldsDeclarer;
+import backtype.storm.tuple.Fields;
+import backtype.storm.tuple.Tuple;
+import backtype.storm.utils.Utils;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import org.apache.log4j.Logger;
+import static backtype.storm.utils.Utils.get;
+import static backtype.storm.utils.Utils.tuple;
+
+
+public class CoordinatedBolt implements IRichBolt {
+    public static Logger LOG = Logger.getLogger(CoordinatedBolt.class);
+
+    public static interface FinishedCallback {
+        void finishedId(Object id);
+    }
+
+    public static class SourceArgs implements Serializable {
+        public boolean singleCount;
+
+        protected SourceArgs(boolean singleCount) {
+            this.singleCount = singleCount;
+        }
+
+        public static SourceArgs single() {
+            return new SourceArgs(true);
+        }
+
+        public static SourceArgs all() {
+            return new SourceArgs(false);
+        }
+    }
+
+    public class CoordinatedOutputCollector extends OutputCollector {
+        IOutputCollector _delegate;
+
+        public CoordinatedOutputCollector(IOutputCollector delegate) {
+            _delegate = delegate;
+        }
+
+        public List<Integer> emit(int stream, List<Tuple> anchors, List<Object> tuple) {
+            List<Integer> tasks = _delegate.emit(stream, anchors, tuple);
+            updateTaskCounts(tuple.get(0), tasks);
+            return tasks;
+        }
+
+        public void emitDirect(int task, int stream, List<Tuple> anchors, List<Object> tuple) {
+            updateTaskCounts(tuple.get(0), Arrays.asList(task));
+            _delegate.emitDirect(task, stream, anchors, tuple);
+        }
+
+        public void ack(Tuple tuple) {
+            _delegate.ack(tuple);
+            Object id = tuple.getValue(0);
+            synchronized(_tracked) {
+                _tracked.get(id).receivedTuples++;
+            }
+            checkFinishId(id);
+        }
+
+        public void fail(Tuple tuple) {
+            _delegate.fail(tuple);
+            Object id = tuple.getValue(0);
+            synchronized(_tracked) {
+                _tracked.get(id).receivedTuples++;
+            }
+            checkFinishId(id);
+        }
+        
+        public void reportError(Throwable error) {
+            _delegate.reportError(error);
+        }
+
+
+        private void updateTaskCounts(Object id, List<Integer> tasks) {
+            Map<Integer, Integer> taskEmittedTuples = _tracked.get(id).taskEmittedTuples;
+            for(Integer task: tasks) {
+                int newCount = get(taskEmittedTuples, task, 0) + 1;
+                taskEmittedTuples.put(task, newCount);
+            }
+        }
+    }
+
+    private SourceArgs _sourceArgs;
+    private IRichBolt _delegate;
+    private Integer _numSourceReports;
+    private List<Integer> _countOutTasks = new ArrayList<Integer>();;
+    private OutputCollector _collector;
+    private Map<Object, TrackingInfo> _tracked = new HashMap<Object, TrackingInfo>();
+    private boolean _allOut;
+
+    public static class TrackingInfo {
+        int reportCount = 0;
+        int expectedTupleCount = 0;
+        int receivedTuples = 0;
+        Map<Integer, Integer> taskEmittedTuples = new HashMap<Integer, Integer>();
+
+        @Override
+        public String toString() {
+            return "reportCount: " + reportCount + "\n" +
+                   "expectedTupleCount: " + expectedTupleCount + "\n" +
+                   "receivedTuples: " + receivedTuples + "\n" +
+                   taskEmittedTuples.toString();
+        }
+    }
+
+    
+    public CoordinatedBolt(IRichBolt delegate, SourceArgs sourceArgs) {
+        this(delegate, sourceArgs, false);
+    }
+
+    public CoordinatedBolt(IRichBolt delegate) {
+        this(delegate, null);
+    }
+
+    /**
+     * allOut indicates whether counts should be sent to all out tasks or just to those it sent tuples to
+     */
+    public CoordinatedBolt(IRichBolt delegate, SourceArgs sourceArgs, boolean allOut) {
+        _sourceArgs = sourceArgs;
+        _delegate = delegate;
+        _allOut = allOut;
+    }
+
+    public CoordinatedBolt(IRichBolt delegate, boolean allOut) {
+        this(delegate, null, allOut);
+    }
+
+
+    public void prepare(Map config, TopologyContext context, OutputCollector collector) {
+        _collector = collector;
+        _delegate.prepare(config, context, new CoordinatedOutputCollector(collector));
+        for(Integer component: Utils.get(context.getThisTargets(),
+                                        Constants.COORDINATED_STREAM_ID,
+                                        new HashMap<Integer, Grouping>())
+                                        .keySet()) {
+            for(Integer task: context.getComponentTasks(component)) {
+                _countOutTasks.add(task);
+            }
+        }
+        if(_sourceArgs!=null) {
+            if(_sourceArgs.singleCount) {
+                _numSourceReports = 1;
+            } else {
+                int sourceComponent = context.getThisSources().keySet().iterator().next().get_componentId();
+                _numSourceReports = context.getComponentTasks(sourceComponent).size();
+            }
+        }
+    }
+
+    private void checkFinishId(Object id) {
+        synchronized(_tracked) {
+            TrackingInfo track = _tracked.get(id);
+            if(track!=null &&
+               (_sourceArgs==null
+                    ||
+               track.reportCount==_numSourceReports &&
+               track.expectedTupleCount == track.receivedTuples)) {
+                if(_delegate instanceof FinishedCallback) {
+                    ((FinishedCallback)_delegate).finishedId(id);
+                }
+                Iterator<Integer> outTasks;
+                if(_allOut) outTasks = _countOutTasks.iterator();
+                else outTasks = track.taskEmittedTuples.keySet().iterator();
+                while(outTasks.hasNext()) {
+                    int task = outTasks.next();
+                    int numTuples = get(track.taskEmittedTuples, task, 0);
+                    _collector.emitDirect(task, Constants.COORDINATED_STREAM_ID, tuple(id, numTuples));
+                }
+                //TODO: need a thread that clears this out occassionally (or wait until have a map type that does this automatically)
+                _tracked.remove(id);
+            }
+        }
+    }
+
+    public void execute(Tuple tuple) {
+        Object id = tuple.getValue(0);
+        TrackingInfo track;
+        synchronized(_tracked) {
+            track = _tracked.get(id);
+            if(track==null) {
+                track = new TrackingInfo();
+                _tracked.put(id, track);
+            }
+        }
+
+        if(_sourceArgs!=null && tuple.getSourceStreamId()==Constants.COORDINATED_STREAM_ID) {
+            int count = (Integer) tuple.getValue(1);
+            synchronized(_tracked) {
+                track.reportCount++;
+                track.expectedTupleCount+=count;
+            }
+            checkFinishId(id);
+        } else {            
+            _delegate.execute(tuple);
+        }
+    }
+
+    public void cleanup() {
+        _delegate.cleanup();
+    }
+
+    public void declareOutputFields(OutputFieldsDeclarer declarer) {
+        _delegate.declareOutputFields(declarer);
+        declarer.declareStream(Constants.COORDINATED_STREAM_ID, true, new Fields("id", "count"));
+    }
+
+}
diff --git a/src/jvm/backtype/storm/task/IBolt.java b/src/jvm/backtype/storm/task/IBolt.java
new file mode 100644
index 0000000..8097f5d
--- /dev/null
+++ b/src/jvm/backtype/storm/task/IBolt.java
@@ -0,0 +1,67 @@
+package backtype.storm.task;
+
+import backtype.storm.tuple.Tuple;
+import java.util.Map;
+import java.io.Serializable;
+
+/**
+ * An IBolt represents a component that takes tuples as input and produces tuples
+ * as output. An IBolt can do everything from filtering to joining to functions
+ * to aggregations. It does not have to process a tuple immediately and may
+ * hold onto tuples to process later.
+ *
+ * <p>A bolt's lifecycle is as follows:</p>
+ *
+ * <p>IBolt object created on client machine. The IBolt is serialized into the topology
+ * (using Java serialization) and submitted to the master machine of the cluster (Nimbus).
+ * Nimbus then launches workers which deserialize the object, call prepare on it, and then
+ * start processing tuples.</p>
+ *
+ * <p>If you want to parameterize an IBolt, you should set the parameter's through its
+ * constructor and save the parameterization state as instance variables (which will
+ * then get serialized and shipped to every task executing this bolt across the cluster).</p>
+ *
+ * <p>When defining bolts in Java, you should use the IRichBolt interface which adds
+ * necessary methods for using the Java TopologyBuilder API.</p>
+ */
+public interface IBolt extends Serializable {
+    /**
+     * Called when a task for this component is initialized within a worker on the cluster.
+     * It provides the bolt with the environment in which the bolt executes.
+     *
+     * <p>This includes the:</p>
+     * 
+     * @param stormConf The Storm configuration for this bolt. This is the configuration provided to the topology merged in with cluster configuration on this machine.
+     * @param context This object can be used to get information about this task's place within the topology, including the task id and component id of this task, input and output information, etc.
+     * @param collector The collector is used to emit tuples from this bolt. Tuples can be emitted at any time, including the prepare and cleanup methods. The collector is not thread-safe and should be saved as an instance variable of this bolt object.
+     */
+    void prepare(Map stormConf, TopologyContext context, OutputCollector collector);
+
+    /**
+     * Process a single tuple of input. The Tuple object contains metadata on it
+     * about which component/stream/task it came from. The values of the Tuple can
+     * be accessed using Tuple#getValue. The IBolt does not have to process the Tuple
+     * immediately. It is perfectly fine to hang onto a tuple and process it later
+     * (for instance, to do an aggregation or join).
+     *
+     * <p>Tuples should be emitted using the OutputCollector provided through the prepare method.
+     * It is required that all input tuples are acked or failed at some point using the OutputCollector.
+     * Otherwise, Storm will be unable to determine when tuples coming off the spouts
+     * have been completed.</p>
+     *
+     * <p>For the common case of acking an input tuple at the end of the execute method,
+     * see IBasicBolt which automates this.</p>
+     * 
+     * @param input The input tuple to be processed.
+     */
+    void execute(Tuple input);
+
+    /**
+     * Called when an IBolt is going to be shutdown. There is no guarentee that cleanup
+     * will be called, because the supervisor kill -9's worker processes on the cluster.
+     *
+     * <p>The one context where cleanup is guaranteed to be called is when a topology
+     * is killed when running Storm in local mode.</p>
+     */
+    void cleanup();
+}
\ No newline at end of file
diff --git a/src/jvm/backtype/storm/task/IInternalOutputCollector.java b/src/jvm/backtype/storm/task/IInternalOutputCollector.java
new file mode 100644
index 0000000..418f6ef
--- /dev/null
+++ b/src/jvm/backtype/storm/task/IInternalOutputCollector.java
@@ -0,0 +1,12 @@
+package backtype.storm.task;
+
+import backtype.storm.tuple.Tuple;
+import java.util.List;
+
+public interface IInternalOutputCollector {
+    public List<Integer> emit(Tuple output);
+    public void emitDirect(int taskId, Tuple output);
+    public void ack(Tuple inputTuple, List<Long> generatedTupleIds);
+    public void fail(Tuple inputTuple);
+    public void reportError(Throwable error);
+}
\ No newline at end of file
diff --git a/src/jvm/backtype/storm/task/IOutputCollector.java b/src/jvm/backtype/storm/task/IOutputCollector.java
new file mode 100644
index 0000000..67ca343
--- /dev/null
+++ b/src/jvm/backtype/storm/task/IOutputCollector.java
@@ -0,0 +1,15 @@
+package backtype.storm.task;
+
+import java.util.List;
+import backtype.storm.tuple.Tuple;
+
+public interface IOutputCollector {
+    /**
+        Returns the task ids that received the tuples.
+    */
+    List<Integer> emit(int streamId, List<Tuple> anchors, List<Object> tuple);
+    void emitDirect(int taskId, int streamId, List<Tuple> anchors, List<Object> tuple);
+    void ack(Tuple input);
+    void fail(Tuple input);
+    void reportError(Throwable error);
+}
diff --git a/src/jvm/backtype/storm/task/KeyedFairBolt.java b/src/jvm/backtype/storm/task/KeyedFairBolt.java
new file mode 100644
index 0000000..308f92e
--- /dev/null
+++ b/src/jvm/backtype/storm/task/KeyedFairBolt.java
@@ -0,0 +1,61 @@
+package backtype.storm.task;
+
+import backtype.storm.task.CoordinatedBolt.FinishedCallback;
+import backtype.storm.topology.IRichBolt;
+import backtype.storm.topology.OutputFieldsDeclarer;
+import backtype.storm.tuple.Tuple;
+import backtype.storm.utils.KeyedRoundRobinQueue;
+import java.util.Map;
+
+
+public class KeyedFairBolt implements IRichBolt, FinishedCallback {
+    IRichBolt _delegate;
+    KeyedRoundRobinQueue<Tuple> _rrQueue;
+    Thread _executor;
+    FinishedCallback _callback;
+
+    public KeyedFairBolt(IRichBolt delegate) {
+        _delegate = delegate;
+    }
+
+    public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
+        if(_delegate instanceof FinishedCallback) {
+            _callback = (FinishedCallback) _delegate;
+        }
+        _delegate.prepare(stormConf, context, collector);
+        _rrQueue = new KeyedRoundRobinQueue<Tuple>();
+        _executor = new Thread(new Runnable() {
+            public void run() {
+                try {
+                    while(true) {
+                        _delegate.execute(_rrQueue.take());
+                    }
+                } catch (InterruptedException e) {
+
+                }
+            }
+        });
+        _executor.setDaemon(true);
+        _executor.start();
+    }
+
+    public void execute(Tuple input) {
+        Object key = input.getValue(0);
+        _rrQueue.add(key, input);
+    }
+
+    public void cleanup() {
+        _executor.interrupt();
+        _delegate.cleanup();
+    }
+
+    public void declareOutputFields(OutputFieldsDeclarer declarer) {
+        _delegate.declareOutputFields(declarer);
+    }
+
+    public void finishedId(Object id) {
+        if(_callback!=null) {
+            _callback.finishedId(id);
+        }
+    }
+}
diff --git a/src/jvm/backtype/storm/task/OutputCollector.java b/src/jvm/backtype/storm/task/OutputCollector.java
new file mode 100644
index 0000000..68f28c2
--- /dev/null
+++ b/src/jvm/backtype/storm/task/OutputCollector.java
@@ -0,0 +1,166 @@
+package backtype.storm.task;
+
+import backtype.storm.tuple.Tuple;
+import backtype.storm.utils.Utils;
+import java.util.Arrays;
+import java.util.List;
+
+/**
+ * This output collector exposes the API for emitting tuples from an IRichBolt.
+ * This is the core API for emitting tuples. For a simpler API, and a more restricted
+ * form of stream processing, see IBasicBolt and BasicOutputCollector.
+ */
+public abstract class OutputCollector implements IOutputCollector {
+
+    /**
+     * Emits a new tuple to a specific stream with a single anchor.
+     *
+     * @param streamId the stream to emit to
+     * @param anchor the tuple to anchor to
+     * @param tuple the new output tuple from this bolt
+     * @return the list of task ids that this new tuple was sent to
+     */
+    public List<Integer> emit(int streamId, Tuple anchor, List<Object> tuple) {
+        return emit(streamId, Arrays.asList(anchor), tuple);
+    }
+
+    /**
+     * Emits a new unanchored tuple to the specified stream. Beacuse it's unanchored,
+     * if a failure happens downstream, this new tuple won't affect whether any
+     * spout tuples are considered failed or not.
+     * 
+     * @param streamId the stream to emit to
+     * @param tuple the new output tuple from this bolt
+     * @return the list of task ids that this new tuple was sent to
+     */
+    public List<Integer> emit(int streamId, List<Object> tuple) {
+        return emit(streamId, (List) null, tuple);
+    }
+
+    /**
+     * Emits a new tuple to the default stream anchored on a group of input tuples.
+     * 
+     * @param anchors the tuples to anchor to
+     * @param tuple the new output tuple from this bolt
+     * @return the list of task ids that this new tuple was sent to
+     */
+    public List<Integer> emit(List<Tuple> anchors, List<Object> tuple) {
+        return emit(Utils.DEFAULT_STREAM_ID, anchors, tuple);
+    }
+
+
+    /**
+     * Emits a new tuple to the default stream anchored on a single tuple.
+     * 
+     * @param anchor the tuple to anchor to
+     * @param tuple the new output tuple from this bolt
+     * @return the list of task ids that this new tuple was sent to
+     */
+    public List<Integer> emit(Tuple anchor, List<Object> tuple) {
+        return emit(Utils.DEFAULT_STREAM_ID, anchor, tuple);
+    }
+
+    /**
+     * Emits a new unanchored tuple to the default stream. Beacuse it's unanchored,
+     * if a failure happens downstream, this new tuple won't affect whether any
+     * spout tuples are considered failed or not.
+     *
+     * @param tuple the new output tuple from this bolt
+     * @return the list of task ids that this new tuple was sent to
+     */
+    public List<Integer> emit(List<Object> tuple) {
+        return emit(Utils.DEFAULT_STREAM_ID, tuple);
+    }
+
+    /**
+     * Emits a tuple directly to the specified task id on the specified stream.
+     * If the target bolt does not subscribe to this bolt using a direct grouping,
+     * the tuple will not be sent. If the specified output stream is not declared
+     * as direct, or the target bolt subscribes with a non-direct grouping,
+     * an error will occur at runtime.
+     *
+     * @param taskId the taskId to send the new tuple to
+     * @param streamId the stream to send the tuple on. It must be declared as a direct stream in the topology definition.
+     * @param anchor the tuple to anchor to
+     * @param tuple the new output tuple from this bolt
+     */
+    public void emitDirect(int taskId, int streamId, Tuple anchor, List<Object> tuple) {
+        emitDirect(taskId, streamId, Arrays.asList(anchor), tuple);
+    }
+
+    /**
+     * Emits a tuple directly to the specified task id on the specified stream.
+     * If the target bolt does not subscribe to this bolt using a direct grouping,
+     * the tuple will not be sent. If the specified output stream is not declared
+     * as direct, or the target bolt subscribes with a non-direct grouping,
+     * an error will occur at runtime. Note that this method does not use anchors,
+     * so downstream failures won't affect the failure status of any spout tuples.
+     *
+     * @param taskId the taskId to send the new tuple to
+     * @param streamId the stream to send the tuple on. It must be declared as a direct stream in the topology definition.
+     * @param tuple the new output tuple from this bolt
+     */
+    public void emitDirect(int taskId, int streamId, List<Object> tuple) {
+        emitDirect(taskId, streamId, (List) null, tuple);
+    }
+
+    /**
+     * Emits a tuple directly to the specified task id on the default stream.
+     * If the target bolt does not subscribe to this bolt using a direct grouping,
+     * the tuple will not be sent. If the specified output stream is not declared
+     * as direct, or the target bolt subscribes with a non-direct grouping,
+     * an error will occur at runtime.
+     *
+     * <p>The default stream must be declared as direct in the topology definition.
+     * See OutputDeclarer#declare for how this is done when defining topologies
+     * in Java.</p>
+     *
+     * @param taskId the taskId to send the new tuple to
+     * @param anchosr the tuples to anchor to
+     * @param tuple the new output tuple from this bolt
+     */
+    public void emitDirect(int taskId, List<Tuple> anchors, List<Object> tuple) {
+        emitDirect(taskId, Utils.DEFAULT_STREAM_ID, anchors, tuple);
+    }
+
+    /**
+     * Emits a tuple directly to the specified task id on the default stream.
+     * If the target bolt does not subscribe to this bolt using a direct grouping,
+     * the tuple will not be sent. If the specified output stream is not declared
+     * as direct, or the target bolt subscribes with a non-direct grouping,
+     * an error will occur at runtime.
+     *
+     * <p>The default stream must be declared as direct in the topology definition.
+     * See OutputDeclarer#declare for how this is done when defining topologies
+     * in Java.</p>
+     *
+     * @param taskId the taskId to send the new tuple to
+     * @param anchor the tuple to anchor to
+     * @param tuple the new output tuple from this bolt
+     */
+    public void emitDirect(int taskId, Tuple anchor, List<Object> tuple) {
+        emitDirect(taskId, Utils.DEFAULT_STREAM_ID, anchor, tuple);
+    }
+
+
+    /**
+     * Emits a tuple directly to the specified task id on the default stream.
+     * If the target bolt does not subscribe to this bolt using a direct grouping,
+     * the tuple will not be sent. If the specified output stream is not declared
+     * as direct, or the target bolt subscribes with a non-direct grouping,
+     * an error will occur at runtime.
+     *
+     * <p>The default stream must be declared as direct in the topology definition.
+     * See OutputDeclarer#declare for how this is done when defining topologies
+     * in Java.</p>
+     *
+     * <p>Note that this method does not use anchors, so downstream failures won't
+     * affect the failure status of any spout tuples.</p>
+     *
+     * @param taskId the taskId to send the new tuple to
+     * @param tuple the new output tuple from this bolt
+     */
+    public void emitDirect(int taskId, List<Object> tuple) {
+        emitDirect(taskId, Utils.DEFAULT_STREAM_ID, tuple);
+    }
+}
diff --git a/src/jvm/backtype/storm/task/OutputCollectorImpl.java b/src/jvm/backtype/storm/task/OutputCollectorImpl.java
new file mode 100644
index 0000000..87740bf
--- /dev/null
+++ b/src/jvm/backtype/storm/task/OutputCollectorImpl.java
@@ -0,0 +1,82 @@
+package backtype.storm.task;
+
+import backtype.storm.tuple.MessageId;
+import backtype.storm.tuple.Tuple;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.List;
+import java.util.Map;
+
+
+//On tuple emit:
+//
+//we have: Map<Long, Long> for each anchor
+//
+//create map from root to id
+//
+//for each anchor, add a new unique id for its output (I)
+//udpate root -> id with xor of I for each of anchor's roots
+//
+//this map is the message id map for the new tuple
+
+public class OutputCollectorImpl extends OutputCollector {
+    private TopologyContext _context;
+    private IInternalOutputCollector _collector;
+    private Map<Tuple, List<Long>> _pendingAcks = new ConcurrentHashMap<Tuple, List<Long>>();
+    
+    public OutputCollectorImpl(TopologyContext context, IInternalOutputCollector collector) {
+        _context = context;
+        _collector = collector;
+    }
+    
+    public List<Integer> emit(int streamId, List<Tuple> anchors, List<Object> tuple) {
+        return _collector.emit(anchorTuple(anchors, streamId, tuple));
+    }
+    
+    public void emitDirect(int taskId, int streamId, List<Tuple> anchors, List<Object> tuple) {
+        _collector.emitDirect(taskId, anchorTuple(anchors, streamId, tuple));
+    }
+
+    private Tuple anchorTuple(List<Tuple> anchors, int streamId, List<Object> tuple) {
+        Map<Long, Long> anchorsToIds = new HashMap<Long, Long>();
+        if(anchors!=null) {
+            for(Tuple anchor: anchors) {
+                long newId = MessageId.generateId();
+                getExistingOutput(anchor).add(newId);
+                for(long root: anchor.getMessageId().getAnchorsToIds().keySet()) {
+                    Long curr = anchorsToIds.get(root);
+                    if(curr == null) curr = 0L;
+                    anchorsToIds.put(root, curr ^ newId);                
+                }
+            }
+        }
+        return new Tuple(_context, tuple, _context.getThisTaskId(), streamId, MessageId.makeId(anchorsToIds));
+    }
+
+    public void ack(Tuple input) {
+        List<Long> generated = getExistingOutput(input);
+        _pendingAcks.remove(input); //don't just do this directly in case there was no output
+        _collector.ack(input, generated);
+    }
+
+    public void fail(Tuple input) {
+        _pendingAcks.remove(input);
+        _collector.fail(input);
+    }
+    
+    public void reportError(Throwable error) {
+        _collector.reportError(error);
+    }
+
+    
+    private List<Long> getExistingOutput(Tuple anchor) {
+        if(_pendingAcks.containsKey(anchor)) {
+            return _pendingAcks.get(anchor);
+        } else {
+            List<Long> ret = new ArrayList<Long>();
+            _pendingAcks.put(anchor, ret);
+            return ret;
+        }
+    }
+}
diff --git a/src/jvm/backtype/storm/task/ShellBolt.java b/src/jvm/backtype/storm/task/ShellBolt.java
new file mode 100644
index 0000000..72dfd66
--- /dev/null
+++ b/src/jvm/backtype/storm/task/ShellBolt.java
@@ -0,0 +1,194 @@
+package backtype.storm.task;
+
+import backtype.storm.generated.ShellComponent;
+import backtype.storm.tuple.MessageId;
+import backtype.storm.tuple.Tuple;
+import backtype.storm.utils.Utils;
+import java.io.BufferedReader;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.log4j.Logger;
+import org.json.simple.JSONObject;
+import org.json.simple.JSONValue;
+
+/**
+ * A bolt that shells out to another process to process tuples. ShellBolt
+ * communicates with that process over stdio using a special protocol. An ~100
+ * line library is required to implement that protocol, and adapter libraries
+ * currently exist for Ruby and Python.
+ *
+ * <p>To run a ShellBolt on a cluster, the scripts that are shelled out to must be
+ * in the resources directory within the jar submitted to the master.
+ * During development/testing on a local machine, that resources directory just
+ * needs to be on the classpath.</p>
+ *
+ * <p>When creating topologies using the Java API, subclass this bolt and implement
+ * the IRichBolt interface to create components for the topology that use other languages. For example:
+ * </p>
+ *
+ * <pre>
+ * public class MyBolt extends ShellBolt implements IRichBolt {
+ *      public MyBolt() {
+ *          super("python", "mybolt.py");
+ *      }
+ *
+ *      public void declareOutputFields(OutputFieldsDeclarer declarer) {
+ *          declarer.declare(new Fields("field1", "field2"));
+ *      }
+ * }
+ * </pre>
+ */
+public class ShellBolt implements IBolt {
+    public static Logger LOG = Logger.getLogger(ShellBolt.class);
+
+    String _shellCommand;
+    String _codeResource;
+    Process _subprocess;
+    DataOutputStream _processin;
+    BufferedReader _processout;
+    OutputCollector _collector;
+    Map<Long, Tuple> _inputs = new HashMap<Long, Tuple>();
+
+    public ShellBolt(ShellComponent component) {
+        this(component.get_execution_command(), component.get_script());
+    }
+
+    public ShellBolt(String shellCommand, String codeResource) {
+        _shellCommand = shellCommand;
+        _codeResource = codeResource;
+    }
+
+    private String initializeSubprocess(TopologyContext context) {
+        //can change this to launchSubprocess and have it return the pid (that the subprcess returns)
+        ProcessBuilder builder = new ProcessBuilder(_shellCommand, _codeResource);
+        builder.directory(new File(context.getCodeDir()));
+        try {
+            _subprocess = builder.start();
+            _processin = new DataOutputStream(_subprocess.getOutputStream());
+            _processout = new BufferedReader(new InputStreamReader(_subprocess.getInputStream()));
+            sendToSubprocess(context.getPIDDir());
+            //subprocesses must send their pid first thing
+            String subpid = _processout.readLine();
+            LOG.info("Launched subprocess with pid " + subpid);
+            return subpid;
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
+        try {
+            initializeSubprocess(context);
+            _collector = collector;
+
+            sendToSubprocess(JSONValue.toJSONString(stormConf));
+            sendToSubprocess(context.toJSONString());
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    public void execute(Tuple input) {
+        //just need an id
+        long genId = MessageId.generateId();
+        _inputs.put(genId, input);
+        try {
+            JSONObject obj = new JSONObject();
+            obj.put("id", genId);
+            obj.put("comp", input.getSourceComponent());
+            obj.put("stream", input.getSourceStreamId());
+            obj.put("task", input.getSourceTask());
+            obj.put("tuple", input.getValues());
+            sendToSubprocess(obj.toString());
+            while(true) {
+              String line = "";
+              while(true) {
+                  String subline = _processout.readLine();
+                  if(subline==null)
+                      throw new RuntimeException("Pipe to subprocess seems to be broken!");
+                  if(subline.equals("sync")) {
+                      line = subline;
+                      break;
+                  }
+                  if(subline.equals("end")) {
+                      break;
+                  }
+                  if(line.length()!=0) {
+                      line+="\n";
+                  }
+                  line+=subline;
+              }
+              if(line.equals("sync")) {
+                  break;
+              } else {
+                  Map action = (Map) JSONValue.parse(line);
+                  String command = (String) action.get("command");
+                  if(command.equals("ack")) {
+                    Long id = (Long) action.get("id");
+                    Tuple acked = _inputs.remove(id);
+                    if(acked==null) {
+                        throw new RuntimeException("Acked a non-existent or already acked/failed id: " + id);
+                    }
+                    _collector.ack(acked);
+                  } else if (command.equals("fail")) {
+                    Long id = (Long) action.get("id");
+                    Tuple failed = _inputs.remove(id);
+                    if(failed==null) {
+                        throw new RuntimeException("Failed a non-existent or already acked/failed id: " + id);
+                    }
+                    _collector.fail(failed);
+                  } else if (command.equals("log")) {
+                    String msg = (String) action.get("msg");
+                    LOG.info("Shell msg: " + msg);
+                  } else if(command.equals("emit")) {
+                    Long stream = (Long) action.get("stream");
+                    if(stream==null) stream = (long)Utils.DEFAULT_STREAM_ID;
+                    Long task = (Long) action.get("task");
+                    List<Object> tuple = (List) action.get("tuple");
+                    List<Tuple> anchors = new ArrayList<Tuple>();
+                    Object anchorObj = action.get("anchors");
+                    if(anchorObj!=null) {
+                        if(anchorObj instanceof Long) {
+                            anchorObj = Arrays.asList(anchorObj);
+                        }
+                        for(Object o: (List) anchorObj) {
+                            anchors.add(_inputs.get((Long) o));
+                        }
+                    }
+                    if(task==null) {
+                       List<Integer> outtasks = _collector.emit((int)stream.longValue(), anchors, tuple);
+                       sendToSubprocess(JSONValue.toJSONString(outtasks));
+                    } else {
+                        _collector.emitDirect((int)task.longValue(), (int)stream.longValue(), anchors, tuple);
+                    }
+                  }
+              }
+            }
+        } catch(IOException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    public void cleanup() {
+        _subprocess.destroy();
+        _inputs.clear();
+        _processin = null;
+        _processout = null;
+        _collector = null;
+    }
+
+    private void sendToSubprocess(String str) throws IOException {
+        _processin.writeBytes(str + "\n");
+        _processin.writeBytes("end\n");
+        _processin.flush();
+    }
+
+}
diff --git a/src/jvm/backtype/storm/task/TopologyContext.java b/src/jvm/backtype/storm/task/TopologyContext.java
new file mode 100644
index 0000000..e90940a
--- /dev/null
+++ b/src/jvm/backtype/storm/task/TopologyContext.java
@@ -0,0 +1,313 @@
+package backtype.storm.task;
+
+import backtype.storm.generated.Bolt;
+import backtype.storm.generated.ComponentCommon;
+import backtype.storm.generated.GlobalStreamId;
+import backtype.storm.generated.Grouping;
+import backtype.storm.generated.SpoutSpec;
+import backtype.storm.generated.StateSpoutSpec;
+import backtype.storm.generated.StormTopology;
+import backtype.storm.state.ISubscribedState;
+import backtype.storm.tuple.Fields;
+import backtype.storm.utils.Utils;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.commons.lang.NotImplementedException;
+import org.json.simple.JSONValue;
+
+/**
+ * A TopologyContext is given to bolts and spouts in their "prepare" and "open"
+ * methods, respectively. This object provides information about the component's
+ * place within the topology, such as task ids, inputs and outputs, etc.
+ *
+ * <p>The TopologyContext is also used to declare ISubscribedState objects to
+ * synchronize state with StateSpouts this object is subscribed to.</p>
+ */
+public class TopologyContext {
+    private StormTopology _topology;
+    private Map<Integer, Integer> _taskToComponent;
+    private int _taskId;
+    private Map<Integer, List<Integer>> _componentToTasks;
+    private String _codeDir;
+    private String _pidDir;
+    private String _stormId;
+
+    public TopologyContext(StormTopology topology, Map<Integer, Integer> taskToComponent, String stormId, String codeDir, String pidDir, int taskId) {
+        _topology = topology;
+        _taskToComponent = taskToComponent;
+        _stormId = stormId;
+        _taskId = taskId;
+        _componentToTasks = new HashMap<Integer, List<Integer>>();
+        _pidDir = pidDir;
+        _codeDir = codeDir;
+        for(Integer task: taskToComponent.keySet()) {
+            int component = taskToComponent.get(task);
+            List<Integer> curr = _componentToTasks.get(component);
+            if(curr==null) curr = new ArrayList<Integer>();
+            curr.add(task);
+            _componentToTasks.put(component, curr);
+        }
+    }
+
+    /**
+     * All state from all subscribed state spouts streams will be synced with
+     * the provided object.
+     * 
+     * <p>It is recommended that your ISubscribedState object is kept as an instance
+     * variable of this object. The recommended usage of this method is as follows:</p>
+     *
+     * <p>
+     * _myState = context.setAllSubscribedState(new MyState());
+     * </p>
+     * @param obj Provided ISubscribedState implementation
+     * @return Returns the ISubscribedState object provided
+     */
+    public <T extends ISubscribedState> T setAllSubscribedState(T obj) {
+        //check that only subscribed to one component/stream for statespout
+        //setsubscribedstate appropriately
+        throw new NotImplementedException();
+    }
+
+
+    /**
+     * Synchronizes the default stream from the specified state spout component
+     * id with the provided ISubscribedState object.
+     *
+     * <p>The recommended usage of this method is as follows:</p>
+     * <p>
+     * _myState = context.setSubscribedState(componentId, new MyState());
+     * </p>
+     *
+     * @param componentId the id of the StateSpout component to subscribe to
+     * @param obj Provided ISubscribedState implementation
+     * @return Returns the ISubscribedState object provided
+     */
+    public <T extends ISubscribedState> T setSubscribedState(int componentId, T obj) {
+        return setSubscribedState(componentId, Utils.DEFAULT_STREAM_ID, obj);
+    }
+
+    /**
+     * Synchronizes the specified stream from the specified state spout component
+     * id with the provided ISubscribedState object.
+     *
+     * <p>The recommended usage of this method is as follows:</p>
+     * <p>
+     * _myState = context.setSubscribedState(componentId, streamId, new MyState());
+     * </p>
+     *
+     * @param componentId the id of the StateSpout component to subscribe to
+     * @param streamId the stream to subscribe to
+     * @param obj Provided ISubscribedState implementation
+     * @return Returns the ISubscribedState object provided
+     */
+    public <T extends ISubscribedState> T setSubscribedState(int componentId, int streamId, T obj) {
+        throw new NotImplementedException();
+    }
+
+    /**
+     * Gets the unique id assigned to this topology. The id is the storm name with a
+     * unique nonce appended to it.
+     * @return the storm id
+     */
+    public String getStormId() {
+        return _stormId;
+    }
+
+    /**
+     * Gets the task id of this task.
+     * 
+     * @return the task id
+     */
+    public int getThisTaskId() {
+        return _taskId;
+    }
+
+    /**
+     * Gets the Thrift object representing the topology.
+     * 
+     * @return the Thrift definition representing the topology
+     */
+    public StormTopology getRawTopology() {
+        return _topology;
+    }
+
+    /**
+     * Gets the component id for the specified task id. The component id maps
+     * to a component id specified for a Spout or Bolt in the topology definition.
+     *
+     * @param taskId the task id
+     * @return the component id for the input task id
+     */
+    public int getComponentId(int taskId) {
+        return _taskToComponent.get(taskId);
+    }
+
+    /**
+     * Gets the component id for this task. The component id maps
+     * to a component id specified for a Spout or Bolt in the topology definition.
+     * @return
+     */
+    public int getThisComponentId() {
+        return getComponentId(_taskId);
+    }
+
+    /**
+     * Gets the declared output fields for the specified stream id for the component
+     * this task is a part of.
+     */
+    public Fields getThisOutputFields(int streamId) {
+        return getComponentOutputFields(getThisComponentId(), streamId);
+    }
+
+    /**
+     * Gets the set of streams declared for the component of this task.
+     */
+    public Set<Integer> getThisStreams() {
+        return getComponentStreams(getThisComponentId());
+    }
+
+    /**
+     * Gets the set of streams declared for the specified component.
+     */
+    public Set<Integer> getComponentStreams(int componentId) {
+        return getComponentCommon(componentId).get_streams().keySet();
+    }
+
+    /**
+     * Gets the task ids allocated for the given component id. The task ids are
+     * always returned in ascending order.
+     */
+    public List<Integer> getComponentTasks(int componentId) {
+        List<Integer> ret = _componentToTasks.get(componentId);
+        if(ret==null) return new ArrayList<Integer>();
+        else return new ArrayList<Integer>(ret);
+    }
+
+    /**
+     * Gets the index of this task id in getComponentTasks(getThisComponentId()).
+     * An example use case for this method is determining which task
+     * accesses which resource in a distributed resource to ensure an even distribution.
+     */
+    public int getThisTaskIndex() {
+        List<Integer> tasks = new ArrayList<Integer>(getComponentTasks(getThisComponentId()));
+        Collections.sort(tasks);
+        for(int i=0; i<tasks.size(); i++) {
+            if(tasks.get(i) == getThisTaskId()) {
+                return i;
+            }
+        }
+        throw new RuntimeException("Fatal: could not find this task id in this component");
+    }
+
+    /**
+     * Gets the declared output fields for the specified component/stream.
+     */
+    public Fields getComponentOutputFields(int componentId, int streamId) {
+        return new Fields(getComponentCommon(componentId).get_streams().get(streamId).get_output_fields());
+    }
+
+    /**
+     * Gets the declared inputs to this component.
+     * 
+     * @return A map from subscribed component/stream to the grouping subscribed with.
+     */
+    public Map<GlobalStreamId, Grouping> getThisSources() {
+        return getSources(getThisComponentId());
+    }
+    
+    /**
+     * Gets the declared inputs to the specified component.
+     *
+     * @return A map from subscribed component/stream to the grouping subscribed with.
+     */
+    public Map<GlobalStreamId, Grouping> getSources(int componentId) {
+        Bolt bolt = _topology.get_bolts().get(componentId);
+        if(bolt==null) return null;
+        return bolt.get_inputs();
+    }
+
+    /**
+     * Gets information about who is consuming the outputs of this component, and how.
+     *
+     * @return Map from stream id to component id to the Grouping used.
+     */
+    public Map<Integer, Map<Integer, Grouping>> getThisTargets() {
+        return getTargets(getThisComponentId());
+    }
+
+    /**
+     * Gets information about who is consuming the outputs of the specified component,
+     * and how.
+     *
+     * @return Map from stream id to component id to the Grouping used.
+     */
+    public Map<Integer, Map<Integer, Grouping>> getTargets(int componentId) {
+        Map<Integer, Map<Integer, Grouping>> ret = new HashMap<Integer, Map<Integer, Grouping>>();
+        for(int otherComponentId: _topology.get_bolts().keySet()) {
+            Bolt bolt = _topology.get_bolts().get(otherComponentId);
+            for(GlobalStreamId id: bolt.get_inputs().keySet()) {
+                if(id.get_componentId()==componentId) {
+                    Map<Integer, Grouping> curr = ret.get(id.get_streamId());
+                    if(curr==null) curr = new HashMap<Integer, Grouping>();
+                    curr.put(otherComponentId, bolt.get_inputs().get(id));
+                    ret.put(id.get_streamId(), curr);
+                }
+            }
+        }
+        return ret;
+    }
+
+    public String toJSONString() {
+        Map obj = new HashMap();
+        obj.put("taskid", _taskId);
+        obj.put("task->component", _taskToComponent);
+        // TODO: jsonify StormTopology
+        // at the minimum should send source info
+        return JSONValue.toJSONString(obj);
+    }
+
+    private ComponentCommon getComponentCommon(int componentId) {
+       Bolt bolt =  _topology.get_bolts().get(componentId);
+       if(bolt!=null) {
+           return bolt.get_common();
+       }
+       SpoutSpec spoutSpec = _topology.get_spouts().get(componentId);
+       if(spoutSpec!=null) {
+           return spoutSpec.get_common();
+       }
+       StateSpoutSpec stateSpoutSpec = _topology.get_state_spouts().get(componentId);
+       if(stateSpoutSpec!=null) {
+           return stateSpoutSpec.get_common();
+       }
+       throw new IllegalArgumentException("Could not find component common for " + componentId);
+    }
+
+    /**
+     * Gets the location of the external resources for this worker on the
+     * local filesystem. These external resources typically include bolts implemented
+     * in other languages, such as Ruby or Python.
+     */
+    public String getCodeDir() {
+        return _codeDir;
+    }
+
+    /**
+     * If this task spawns any subprocesses, those subprocesses must immediately
+     * write their PID to this directory on the local filesystem to ensure that
+     * Storm properly destroys that process when the worker is shutdown.
+     */
+    public String getPIDDir() {
+        return _pidDir;
+    }
+
+    /**
+     * Gets a map from task id to component id.
+     */
+    public Map<Integer, Integer> getTaskToComponent() {
+        return _taskToComponent;
+    }
+}
\ No newline at end of file
diff --git a/src/jvm/backtype/storm/testing/AckFailDelegate.java b/src/jvm/backtype/storm/testing/AckFailDelegate.java
new file mode 100644
index 0000000..e7587d45
--- /dev/null
+++ b/src/jvm/backtype/storm/testing/AckFailDelegate.java
@@ -0,0 +1,8 @@
+package backtype.storm.testing;
+
+import java.io.Serializable;
+
+public interface AckFailDelegate extends Serializable {
+    public void ack(Object id);
+    public void fail(Object id);
+}
diff --git a/src/jvm/backtype/storm/testing/AckTracker.java b/src/jvm/backtype/storm/testing/AckTracker.java
new file mode 100644
index 0000000..0187f0a
--- /dev/null
+++ b/src/jvm/backtype/storm/testing/AckTracker.java
@@ -0,0 +1,35 @@
+package backtype.storm.testing;
+
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicInteger;
+
+public class AckTracker implements AckFailDelegate {
+    private static Map<String, AtomicInteger> acks = new ConcurrentHashMap<String, AtomicInteger>();
+    
+    private String _id;
+    
+    public AckTracker() {
+        _id = UUID.randomUUID().toString();
+        acks.put(_id, new AtomicInteger(0));
+    }
+    
+    @Override
+    public void ack(Object id) {
+        acks.get(_id).incrementAndGet();
+    }
+
+    @Override
+    public void fail(Object id) {
+    }
+    
+    public int getNumAcks() {
+        return acks.get(_id).intValue();
+    }
+    
+    public void resetNumAcks() {
+        acks.get(_id).set(0);
+    }
+    
+}
diff --git a/src/jvm/backtype/storm/testing/BoltTracker.java b/src/jvm/backtype/storm/testing/BoltTracker.java
new file mode 100644
index 0000000..c288703
--- /dev/null
+++ b/src/jvm/backtype/storm/testing/BoltTracker.java
@@ -0,0 +1,97 @@
+package backtype.storm.testing;
+
+import backtype.storm.task.IBolt;
+import backtype.storm.task.OutputCollector;
+import backtype.storm.task.TopologyContext;
+import backtype.storm.topology.IRichBolt;
+import backtype.storm.topology.OutputFieldsDeclarer;
+import backtype.storm.tuple.Fields;
+import backtype.storm.tuple.Tuple;
+import backtype.storm.tuple.Values;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+
+public class BoltTracker implements IRichBolt {
+    IBolt _delegate;
+
+    BoltTrackerOutputCollector _trackCollector;
+
+    private static class BoltTrackerOutputCollector extends OutputCollector {
+        public OutputCollector _collector;
+
+        public int transferred = 0;
+
+        public BoltTrackerOutputCollector(OutputCollector collector) {
+            _collector = collector;
+        }
+
+
+        public List<Integer> emit(int streamId, List<Tuple> anchors, List<Object> tuple) {
+            List<Integer> ret = _collector.emit(streamId, anchors, tuple);
+            transferred += ret.size();
+
+            //for the extra ack on branching or finishing
+            if(ret.size()!=1) {
+                Set<Long> anchorIds = new HashSet<Long>();
+                for(Tuple t: anchors) {
+                    anchorIds.addAll(t.getMessageId().getAnchors());
+                }
+                transferred += anchorIds.size();
+            }
+            return ret;
+        }
+
+        public void emitDirect(int taskId, int streamId, List<Tuple> anchors, List<Object> tuple) {
+            _collector.emitDirect(taskId, streamId, anchors, tuple);
+            transferred++;
+        }
+
+        public void ack(Tuple input) {
+            _collector.ack(input);
+            transferred+=input.getMessageId().getAnchors().size();
+        }
+
+        public void fail(Tuple input) {
+            _collector.fail(input);
+            transferred+=input.getMessageId().getAnchors().size();
+        }
+
+        public void reportError(Throwable error) {
+            _collector.reportError(error);
+        }
+
+    }
+
+    /*
+     * IBolt so it can be used to wrap acker bolts (which don't declare streams because they're implicit)
+     */
+    public BoltTracker(IBolt delegate) {
+        _delegate = delegate;
+    }
+
+    public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
+        _trackCollector = new BoltTrackerOutputCollector(collector);
+        _delegate.prepare(stormConf, context, _trackCollector);
+    }
+
+    public void execute(Tuple input) {
+        int currTransferred = _trackCollector.transferred;
+        _delegate.execute(input);
+        _trackCollector._collector.emit(TrackerAggregator.TRACK_STREAM,
+                new Values(1, _trackCollector.transferred - currTransferred, 0));
+    }
+
+    public void cleanup() {
+        _delegate.cleanup();
+    }
+
+    public void declareOutputFields(OutputFieldsDeclarer declarer) {
+        if(_delegate instanceof IRichBolt) {
+            ((IRichBolt)_delegate).declareOutputFields(declarer);
+        }
+        declarer.declareStream(TrackerAggregator.TRACK_STREAM, new Fields("processed", "transferred", "spoutEmitted"));
+    }
+}
diff --git a/src/jvm/backtype/storm/testing/FeederSpout.java b/src/jvm/backtype/storm/testing/FeederSpout.java
new file mode 100644
index 0000000..795ba62
--- /dev/null
+++ b/src/jvm/backtype/storm/testing/FeederSpout.java
@@ -0,0 +1,75 @@
+package backtype.storm.testing;
+
+import backtype.storm.topology.OutputFieldsDeclarer;
+import java.util.Map;
+import backtype.storm.spout.SpoutOutputCollector;
+import backtype.storm.task.TopologyContext;
+import backtype.storm.topology.IRichSpout;
+import backtype.storm.tuple.Fields;
+import backtype.storm.utils.Utils;
+import java.util.List;
+import java.util.HashMap;
+import java.util.UUID;
+import java.util.concurrent.LinkedBlockingQueue;
+
+
+public class FeederSpout implements IRichSpout {
+
+    public transient static Map<String, LinkedBlockingQueue<List<Object>>> _feeds = new HashMap<String, LinkedBlockingQueue<List<Object>>>();
+
+    private String _id;
+    private Fields _outFields;
+    private SpoutOutputCollector _collector;
+    private AckFailDelegate _ackFailDelegate;
+
+    public FeederSpout(Fields outFields) {
+        _id = UUID.randomUUID().toString();
+        _feeds.put(_id, new LinkedBlockingQueue<List<Object>>());
+        _outFields = outFields;
+    }
+
+    public void setAckFailDelegate(AckFailDelegate d) {
+        _ackFailDelegate = d;
+    }
+    
+    public void feed(List<Object> tuple) {
+        _feeds.get(_id).add(tuple);
+    }
+
+    public boolean isDistributed() {
+        return true;
+    }
+
+    public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) {
+        _collector = collector;
+    }
+
+    public void close() {
+
+    }
+
+    public void nextTuple() {
+        List<Object> tuple = _feeds.get(_id).poll();
+        if(tuple!=null) {
+            _collector.emit(tuple, UUID.randomUUID().toString());
+        } else {
+            Utils.sleep(100);                
+        }
+    }
+
+    public void ack(Object msgId){
+        if(_ackFailDelegate!=null) {
+            _ackFailDelegate.ack(msgId);
+        }
+    }
+
+    public void fail(Object msgId){
+        if(_ackFailDelegate!=null) {
+            _ackFailDelegate.fail(msgId);
+        }
+    }
+
+    public void declareOutputFields(OutputFieldsDeclarer declarer) {
+        declarer.declare(_outFields);
+    }
+}
\ No newline at end of file
diff --git a/src/jvm/backtype/storm/testing/FixedTuple.java b/src/jvm/backtype/storm/testing/FixedTuple.java
new file mode 100644
index 0000000..e98965d
--- /dev/null
+++ b/src/jvm/backtype/storm/testing/FixedTuple.java
@@ -0,0 +1,21 @@
+package backtype.storm.testing;
+
+import backtype.storm.utils.Utils;
+import java.io.Serializable;
+import java.util.List;
+
+public class FixedTuple implements Serializable {
+    public int stream;
+    public List<Object> values;
+
+    public FixedTuple(List<Object> values) {
+        this.stream = Utils.DEFAULT_STREAM_ID;
+        this.values = values;
+    }
+
+    public FixedTuple(int stream, List<Object> values) {
+        this.stream = stream;
+        this.values = values;
+    }
+
+}
\ No newline at end of file
diff --git a/src/jvm/backtype/storm/testing/FixedTupleSpout.java b/src/jvm/backtype/storm/testing/FixedTupleSpout.java
new file mode 100644
index 0000000..8fc8c6e
--- /dev/null
+++ b/src/jvm/backtype/storm/testing/FixedTupleSpout.java
@@ -0,0 +1,103 @@
+package backtype.storm.testing;
+
+import backtype.storm.spout.ISpout;
+import backtype.storm.spout.SpoutOutputCollector;
+import backtype.storm.task.TopologyContext;
+import backtype.storm.utils.Utils;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import static backtype.storm.utils.Utils.get;
+
+public class FixedTupleSpout implements ISpout {
+    private static final Map<String, Integer> acked = new HashMap<String, Integer>();
+    private static final Map<String, Integer> failed = new HashMap<String, Integer>();
+
+    public static int getNumAcked(String stormId) {
+        synchronized(acked) {
+            return get(acked, stormId, 0);
+        }
+    }
+
+    public static int getNumFailed(String stormId) {
+        synchronized(failed) {
+            return get(failed, stormId, 0);
+        }
+    }
+    
+    public static void clear(String stormId) {
+        acked.remove(stormId);
+        failed.remove(stormId);
+    }
+
+    private List<FixedTuple> _tuples;
+    private SpoutOutputCollector _collector;
+
+    private TopologyContext _context;
+    private List<FixedTuple> _serveTuples;
+    private Map<String, FixedTuple> _pending;
+
+    public FixedTupleSpout(List tuples) {
+        _tuples = new ArrayList<FixedTuple>();
+        for(Object o: tuples) {
+            FixedTuple ft;
+            if(o instanceof FixedTuple) {
+                ft = (FixedTuple) o;
+            } else {
+                ft = new FixedTuple((List) o);
+            }
+            _tuples.add(ft);
+        }
+    }
+
+    public List<FixedTuple> getSourceTuples() {
+        return _tuples;
+    }
+
+    public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) {
+        _context = context;
+        List<Integer> tasks = context.getComponentTasks(context.getThisComponentId());
+        int startIndex;
+        for(startIndex=0; startIndex<tasks.size(); startIndex++) {
+            if(tasks.get(startIndex)==context.getThisTaskId()) {
+                break;
+            }
+        }
+        _collector = collector;
+        _pending = new HashMap<String, FixedTuple>();
+        _serveTuples = new ArrayList<FixedTuple>();
+        for(int i=startIndex; i<_tuples.size(); i+=tasks.size()) {
+            _serveTuples.add(_tuples.get(i));
+        }
+    }
+
+    public void close() {
+    }
+
+    public void nextTuple() {
+        if(_serveTuples.size()>0) {
+            FixedTuple ft = _serveTuples.remove(0);
+            String id = UUID.randomUUID().toString();
+            _pending.put(id, ft);
+            _collector.emit(ft.stream, ft.values, id);
+        } else {
+            Utils.sleep(100);
+        }
+    }
+
+    public void ack(Object msgId) {
+        synchronized(acked) {
+            int curr = get(acked, _context.getStormId(), 0);
+            acked.put(_context.getStormId(), curr+1);
+        }
+    }
+
+    public void fail(Object msgId) {
+        synchronized(failed) {
+            int curr = get(failed, _context.getStormId(), 0);
+            failed.put(_context.getStormId(), curr+1);
+        }
+    }
+}
diff --git a/src/jvm/backtype/storm/testing/SpoutTracker.java b/src/jvm/backtype/storm/testing/SpoutTracker.java
new file mode 100644
index 0000000..272099f
--- /dev/null
+++ b/src/jvm/backtype/storm/testing/SpoutTracker.java
@@ -0,0 +1,94 @@
+package backtype.storm.testing;
+
+import backtype.storm.spout.ISpoutOutputCollector;
+import backtype.storm.spout.SpoutOutputCollector;
+import backtype.storm.task.TopologyContext;
+import backtype.storm.topology.IRichSpout;
+import backtype.storm.topology.OutputFieldsDeclarer;
+import backtype.storm.tuple.Fields;
+import backtype.storm.tuple.Values;
+import java.util.List;
+import java.util.Map;
+
+
+public class SpoutTracker implements IRichSpout {
+    IRichSpout _delegate;
+    SpoutTrackOutputCollector _tracker;
+
+
+    private static class SpoutTrackOutputCollector implements ISpoutOutputCollector {
+        public int transferred = 0;
+        public int emitted = 0;
+        public SpoutOutputCollector _collector;
+
+        public SpoutTrackOutputCollector(SpoutOutputCollector collector) {
+            _collector = collector;
+        }
+
+        public List<Integer> emit(int streamId, List<Object> tuple, Object messageId) {
+            List<Integer> ret = _collector.emit(streamId, tuple, messageId);
+            emitted++;
+            transferred += ret.size();
+            if(messageId!=null) transferred++;
+            //for the extra ack on branching or finishing
+            if(ret.size()!=1) {
+                transferred += 1;
+            }
+
+            return ret;
+        }
+
+        public void emitDirect(int taskId, int streamId, List<Object> tuple, Object messageId) {
+            _collector.emitDirect(taskId, streamId, tuple, messageId);
+            emitted++;
+            transferred++;
+            if(messageId!=null) transferred++;
+        }
+
+    }
+
+
+    public SpoutTracker(IRichSpout delegate) {
+        _delegate = delegate;
+    }
+
+    public boolean isDistributed() {
+        return _delegate.isDistributed();
+    }
+
+    public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) {
+        _tracker = new SpoutTrackOutputCollector(collector);
+        _delegate.open(conf, context, new SpoutOutputCollector(_tracker));
+    }
+
+    public void close() {
+        _delegate.close();
+    }
+
+    public void nextTuple() {
+        int curr = _tracker.transferred;
+        int currEmitted = _tracker.emitted;
+        _delegate.nextTuple();
+        int transferred = _tracker.transferred - curr;
+        if(transferred>0) {
+            _tracker._collector.emit(TrackerAggregator.TRACK_STREAM, new Values(0, transferred, _tracker.emitted - currEmitted));
+        }
+    }
+
+    public void ack(Object msgId) {
+        _delegate.ack(msgId);
+        _tracker._collector.emit(TrackerAggregator.TRACK_STREAM, new Values(1, 0, 0));
+    }
+
+    public void fail(Object msgId) {
+        _delegate.fail(msgId);
+        //TODO: this is not strictly correct, since the message could have just timed out
+        _tracker._collector.emit(TrackerAggregator.TRACK_STREAM, new Values(1, 0, 0));
+    }
+
+    public void declareOutputFields(OutputFieldsDeclarer declarer) {
+        _delegate.declareOutputFields(declarer);
+        declarer.declareStream(TrackerAggregator.TRACK_STREAM, new Fields("processed", "transferred", "spoutEmitted"));
+    }
+
+}
diff --git a/src/jvm/backtype/storm/testing/TestAggregatesCounter.java b/src/jvm/backtype/storm/testing/TestAggregatesCounter.java
new file mode 100644
index 0000000..0b14ec7
--- /dev/null
+++ b/src/jvm/backtype/storm/testing/TestAggregatesCounter.java
@@ -0,0 +1,45 @@
+package backtype.storm.testing;
+
+import backtype.storm.task.OutputCollector;
+import backtype.storm.topology.OutputFieldsDeclarer;
+import backtype.storm.tuple.Tuple;
+import backtype.storm.tuple.Fields;
+import java.util.Map;
+import backtype.storm.task.TopologyContext;
+import backtype.storm.topology.IRichBolt;
+import java.util.HashMap;
+import org.apache.log4j.Logger;
+import static backtype.storm.utils.Utils.tuple;
+
+
+public class TestAggregatesCounter implements IRichBolt {
+    public static Logger LOG = Logger.getLogger(TestWordCounter.class);
+
+    Map<String, Integer> _counts;
+    OutputCollector _collector;
+
+    public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
+        _collector = collector;
+        _counts = new HashMap<String, Integer>();
+    }
+
+    public void execute(Tuple input) {
+        String word = (String) input.getValues().get(0);
+        int count = (Integer) input.getValues().get(1);
+        _counts.put(word, count);
+        int globalCount = 0;
+        for(String w: _counts.keySet()) {
+            globalCount+=_counts.get(w);
+        }
+        _collector.emit(tuple(globalCount));
+        _collector.ack(input);
+    }
+
+    public void cleanup() {
+
+    }
+
+    public void declareOutputFields(OutputFieldsDeclarer declarer) {
+        declarer.declare(new Fields("agg-global"));
+    }
+}
\ No newline at end of file
diff --git a/src/jvm/backtype/storm/testing/TestGlobalCount.java b/src/jvm/backtype/storm/testing/TestGlobalCount.java
new file mode 100644
index 0000000..1093824
--- /dev/null
+++ b/src/jvm/backtype/storm/testing/TestGlobalCount.java
@@ -0,0 +1,42 @@
+package backtype.storm.testing;
+
+import backtype.storm.task.OutputCollector;
+import backtype.storm.topology.OutputFieldsDeclarer;
+import backtype.storm.tuple.Tuple;
+import backtype.storm.tuple.Fields;
+import java.util.Map;
+import backtype.storm.task.TopologyContext;
+import backtype.storm.topology.IRichBolt;
+import org.apache.log4j.Logger;
+import static backtype.storm.utils.Utils.tuple;
+
+
+public class TestGlobalCount implements IRichBolt {
+    public static Logger LOG = Logger.getLogger(TestWordCounter.class);
+
+    private int _count;
+    OutputCollector _collector;
+
+    public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
+        _collector = collector;
+        _count = 0;
+    }
+
+    public void execute(Tuple input) {
+        _count++;
+        _collector.emit(tuple(_count));
+        _collector.ack(input);
+    }
+
+    public void cleanup() {
+
+    }
+
+    public Fields getOutputFields() {
+        return new Fields("global-count");
+    }
+
+    public void declareOutputFields(OutputFieldsDeclarer declarer) {
+        declarer.declare(new Fields("global-count"));
+    }
+}
diff --git a/src/jvm/backtype/storm/testing/TestPlannerBolt.java b/src/jvm/backtype/storm/testing/TestPlannerBolt.java
new file mode 100644
index 0000000..5fc9229
--- /dev/null
+++ b/src/jvm/backtype/storm/testing/TestPlannerBolt.java
@@ -0,0 +1,32 @@
+package backtype.storm.testing;
+
+import backtype.storm.task.OutputCollector;
+import backtype.storm.topology.OutputFieldsDeclarer;
+import backtype.storm.tuple.Tuple;
+import backtype.storm.tuple.Fields;
+import java.util.Map;
+import backtype.storm.task.TopologyContext;
+import backtype.storm.topology.IRichBolt;
+
+
+public class TestPlannerBolt implements IRichBolt {
+    public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
+
+    }
+    
+    public void execute(Tuple input) {
+
+    }
+    
+    public void cleanup() {
+        
+    }
+    
+    public Fields getOutputFields() {
+        return new Fields("field1", "field2");
+    }
+
+    public void declareOutputFields(OutputFieldsDeclarer declarer) {
+        declarer.declare(getOutputFields());
+    }
+}
\ No newline at end of file
diff --git a/src/jvm/backtype/storm/testing/TestPlannerSpout.java b/src/jvm/backtype/storm/testing/TestPlannerSpout.java
new file mode 100644
index 0000000..1bad216
--- /dev/null
+++ b/src/jvm/backtype/storm/testing/TestPlannerSpout.java
@@ -0,0 +1,52 @@
+package backtype.storm.testing;
+
+import backtype.storm.topology.OutputFieldsDeclarer;
+import java.util.Map;
+import backtype.storm.spout.SpoutOutputCollector;
+import backtype.storm.task.TopologyContext;
+import backtype.storm.topology.IRichSpout;
+import backtype.storm.tuple.Fields;
+import backtype.storm.utils.Utils;
+
+
+public class TestPlannerSpout implements IRichSpout {
+    boolean _isDistributed;
+    
+    public TestPlannerSpout(boolean isDistributed) {
+        _isDistributed = isDistributed;
+    }
+    
+    public boolean isDistributed() {
+        return _isDistributed;
+    }
+    
+    public Fields getOutputFields() {
+        return new Fields("field1", "field2");
+    }
+
+    
+    public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) {
+        
+    }
+    
+    public void close() {
+        
+    }
+    
+    public void nextTuple() {
+        Utils.sleep(100);
+    }
+    
+    public void ack(Object msgId){
+        
+    }
+
+    public void fail(Object msgId){
+        
+    }
+
+    public void declareOutputFields(OutputFieldsDeclarer declarer) {
+        declarer.declare(getOutputFields());
+    }
+    
+}
\ No newline at end of file
diff --git a/src/jvm/backtype/storm/testing/TestWordCounter.java b/src/jvm/backtype/storm/testing/TestWordCounter.java
new file mode 100644
index 0000000..52aa234
--- /dev/null
+++ b/src/jvm/backtype/storm/testing/TestWordCounter.java
@@ -0,0 +1,43 @@
+package backtype.storm.testing;
+
+import backtype.storm.topology.OutputFieldsDeclarer;
+import backtype.storm.tuple.Tuple;
+import backtype.storm.tuple.Fields;
+import java.util.Map;
+import backtype.storm.task.TopologyContext;
+import backtype.storm.topology.BasicOutputCollector;
+import backtype.storm.topology.IBasicBolt;
+import java.util.HashMap;
+import org.apache.log4j.Logger;
+import static backtype.storm.utils.Utils.tuple;
+
+
+public class TestWordCounter implements IBasicBolt {
+    public static Logger LOG = Logger.getLogger(TestWordCounter.class);
+
+    Map<String, Integer> _counts;
+    
+    public void prepare(Map stormConf, TopologyContext context) {
+        _counts = new HashMap<String, Integer>();
+    }
+    
+    public void execute(Tuple input, BasicOutputCollector collector) {
+        String word = (String) input.getValues().get(0);
+        int count = 0;
+        if(_counts.containsKey(word)) {
+            count = _counts.get(word);
+        }
+        count++;
+        _counts.put(word, count);
+        collector.emit(tuple(word, count));
+    }
+    
+    public void cleanup() {
+        
+    }
+
+    public void declareOutputFields(OutputFieldsDeclarer declarer) {
+        declarer.declare(new Fields("word", "count"));
+    }
+
+}
\ No newline at end of file
diff --git a/src/jvm/backtype/storm/testing/TestWordSpout.java b/src/jvm/backtype/storm/testing/TestWordSpout.java
new file mode 100644
index 0000000..2acb0f1
--- /dev/null
+++ b/src/jvm/backtype/storm/testing/TestWordSpout.java
@@ -0,0 +1,59 @@
+package backtype.storm.testing;
+
+import backtype.storm.topology.OutputFieldsDeclarer;
+import java.util.Map;
+import backtype.storm.spout.SpoutOutputCollector;
+import backtype.storm.task.TopologyContext;
+import backtype.storm.topology.IRichSpout;
+import backtype.storm.tuple.Fields;
+import backtype.storm.tuple.Values;
+import backtype.storm.utils.Utils;
+import java.util.Random;
+import org.apache.log4j.Logger;
+
+
+public class TestWordSpout implements IRichSpout {
+    public static Logger LOG = Logger.getLogger(TestWordSpout.class);
+    boolean _isDistributed;
+    SpoutOutputCollector _collector;
+
+    public TestWordSpout() {
+        this(true);
+    }
+
+    public TestWordSpout(boolean isDistributed) {
+        _isDistributed = isDistributed;
+    }
+    
+    public boolean isDistributed() {
+        return _isDistributed;
+    }
+    
+    public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) {
+        _collector = collector;
+    }
+    
+    public void close() {
+        
+    }
+        
+    public void nextTuple() {
+        Utils.sleep(100);
+        final String[] words = new String[] {"nathan", "mike", "jackson", "golda", "bertels"};
+        final Random rand = new Random();
+        final String word = words[rand.nextInt(words.length)];
+        _collector.emit(new Values(word));
+    }
+    
+    public void ack(Object msgId) {
+
+    }
+
+    public void fail(Object msgId) {
+        
+    }
+    
+    public void declareOutputFields(OutputFieldsDeclarer declarer) {
+        declarer.declare(new Fields("word"));
+    }
+}
\ No newline at end of file
diff --git a/src/jvm/backtype/storm/testing/TrackerAggregator.java b/src/jvm/backtype/storm/testing/TrackerAggregator.java
new file mode 100644
index 0000000..8e5453b
--- /dev/null
+++ b/src/jvm/backtype/storm/testing/TrackerAggregator.java
@@ -0,0 +1,72 @@
+package backtype.storm.testing;
+
+import backtype.storm.task.OutputCollector;
+import backtype.storm.task.TopologyContext;
+import backtype.storm.topology.IRichBolt;
+import backtype.storm.topology.OutputFieldsDeclarer;
+import backtype.storm.tuple.Tuple;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.UUID;
+
+
+public class TrackerAggregator implements IRichBolt {
+    public static final Map<String, TrackStats> _stats = new HashMap<String, TrackStats>();
+
+    public static int TRACK_STREAM = 999;
+
+    TopologyContext _context;
+
+    public static class TrackStats {
+        int spoutEmitted = 0;
+        int transferred = 0;
+        int processed = 0;
+    }
+
+    String _id;
+
+    public TrackerAggregator() {
+        _id = UUID.randomUUID().toString();
+        _stats.put(_id, new TrackStats());
+    }
+
+    public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
+        _context = context;
+    }
+
+    public int getSpoutEmitted() {
+        synchronized(_stats) {
+            return _stats.get(_id).spoutEmitted;
+        }
+    }
+
+    public int getTransferred() {
+        synchronized(_stats) {
+            return _stats.get(_id).transferred;
+        }
+    }
+
+    public int getProcessed() {
+        synchronized(_stats) {
+            return _stats.get(_id).processed;
+        }
+    }
+
+    public void execute(Tuple input) {
+        int processed = input.getInteger(0);
+        int transferred = input.getInteger(1);
+        int spoutEmitted = input.getInteger(2);
+        synchronized(_stats) {
+            TrackStats stats = _stats.get(_id);
+            stats.processed+=processed;
+            stats.spoutEmitted+=spoutEmitted;
+            stats.transferred+=transferred;
+        }
+    }
+
+    public void cleanup() {        
+    }
+
+    public void declareOutputFields(OutputFieldsDeclarer declarer) {
+    }
+}
diff --git a/src/jvm/backtype/storm/testing/TupleCaptureBolt.java b/src/jvm/backtype/storm/testing/TupleCaptureBolt.java
new file mode 100644
index 0000000..885b58b
--- /dev/null
+++ b/src/jvm/backtype/storm/testing/TupleCaptureBolt.java
@@ -0,0 +1,49 @@
+package backtype.storm.testing;
+
+import backtype.storm.task.IBolt;
+import backtype.storm.task.OutputCollector;
+import backtype.storm.task.TopologyContext;
+import backtype.storm.tuple.Tuple;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+
+public class TupleCaptureBolt implements IBolt {
+    public static transient Map<String, Map<Integer, List<FixedTuple>>> emitted_tuples = new HashMap<String, Map<Integer, List<FixedTuple>>>();
+
+    private String _name;
+    private Map<Integer, List<FixedTuple>> _results = null;
+    private OutputCollector _collector;
+
+    public TupleCaptureBolt(String name) {
+        _name = name;
+        emitted_tuples.put(name, new HashMap<Integer, List<FixedTuple>>());
+    }
+
+    public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
+        _collector = collector;
+    }
+
+    public void execute(Tuple input) {
+        int component = input.getSourceComponent();
+        Map<Integer, List<FixedTuple>> captured = emitted_tuples.get(_name);
+        if(!captured.containsKey(component)) {
+           captured.put(component, new ArrayList<FixedTuple>());
+        }
+        captured.get(component).add(new FixedTuple(input.getSourceStreamId(), input.getValues()));
+        _collector.ack(input);
+    }
+
+    public Map<Integer, List<FixedTuple>> getResults() {
+        if(_results==null) {
+            _results = emitted_tuples.remove(_name);
+        }
+        return _results;
+    }
+
+    public void cleanup() {
+    }
+
+}
diff --git a/src/jvm/backtype/storm/topology/BasicBoltExecutor.java b/src/jvm/backtype/storm/topology/BasicBoltExecutor.java
new file mode 100644
index 0000000..f6b920d
--- /dev/null
+++ b/src/jvm/backtype/storm/topology/BasicBoltExecutor.java
@@ -0,0 +1,36 @@
+package backtype.storm.topology;
+
+import backtype.storm.task.OutputCollector;
+import backtype.storm.task.TopologyContext;
+import backtype.storm.tuple.Tuple;
+import java.util.Map;
+
+public class BasicBoltExecutor implements IRichBolt {
+    
+    private IBasicBolt _bolt;
+    private transient BasicOutputCollector _collector;
+    
+    public BasicBoltExecutor(IBasicBolt bolt) {
+        _bolt = bolt;
+    }
+
+    public void declareOutputFields(OutputFieldsDeclarer declarer) {
+        _bolt.declareOutputFields(declarer);
+    }
+
+    
+    public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
+        _bolt.prepare(stormConf, context);
+        _collector = new BasicOutputCollector(collector);
+    }
+
+    public void execute(Tuple input) {
+        _collector.setContext(input);
+        _bolt.execute(input, _collector);
+        _collector.getOutputter().ack(input);
+    }
+
+    public void cleanup() {
+        _bolt.cleanup();
+    }
+}
\ No newline at end of file
diff --git a/src/jvm/backtype/storm/topology/BasicOutputCollector.java b/src/jvm/backtype/storm/topology/BasicOutputCollector.java
new file mode 100644
index 0000000..ea15b58
--- /dev/null
+++ b/src/jvm/backtype/storm/topology/BasicOutputCollector.java
@@ -0,0 +1,42 @@
+package backtype.storm.topology;
+
+import backtype.storm.task.IOutputCollector;
+import backtype.storm.task.OutputCollector;
+import backtype.storm.tuple.Tuple;
+import backtype.storm.utils.Utils;
+import java.util.List;
+
+
+public class BasicOutputCollector implements IBasicOutputCollector {
+    private OutputCollector out;
+    private Tuple inputTuple;
+
+    public BasicOutputCollector(OutputCollector out) {
+        this.out = out;
+    }
+
+    public List<Integer> emit(int streamId, List<Object> tuple) {
+        return out.emit(streamId, inputTuple, tuple);
+    }
+
+    public List<Integer> emit(List<Object> tuple) {
+        return emit(Utils.DEFAULT_STREAM_ID, tuple);
+    }
+
+    public void setContext(Tuple inputTuple) {
+        this.inputTuple = inputTuple;
+    }
+
+    public IOutputCollector getOutputter() {
+        return out;
+    }
+
+    public void emitDirect(int taskId, int streamId, List<Object> tuple) {
+        out.emitDirect(taskId, streamId, inputTuple, tuple);
+    }
+
+    public void emitDirect(int taskId, List<Object> tuple) {
+        emitDirect(taskId, Utils.DEFAULT_STREAM_ID, tuple);
+    }
+
+}
diff --git a/src/jvm/backtype/storm/topology/IBasicBolt.java b/src/jvm/backtype/storm/topology/IBasicBolt.java
new file mode 100644
index 0000000..108a1a3
--- /dev/null
+++ b/src/jvm/backtype/storm/topology/IBasicBolt.java
@@ -0,0 +1,11 @@
+package backtype.storm.topology;
+
+import backtype.storm.task.TopologyContext;
+import backtype.storm.tuple.Tuple;
+import java.util.Map;
+
+public interface IBasicBolt extends IComponent {
+    void prepare(Map stormConf, TopologyContext context);
+    void execute(Tuple input, BasicOutputCollector collector);
+    void cleanup();
+}
\ No newline at end of file
diff --git a/src/jvm/backtype/storm/topology/IBasicOutputCollector.java b/src/jvm/backtype/storm/topology/IBasicOutputCollector.java
new file mode 100644
index 0000000..c773da1
--- /dev/null
+++ b/src/jvm/backtype/storm/topology/IBasicOutputCollector.java
@@ -0,0 +1,8 @@
+package backtype.storm.topology;
+
+import java.util.List;
+
+public interface IBasicOutputCollector {
+    List<Integer> emit(int streamId, List<Object> tuple);
+    void emitDirect(int taskId, int streamId, List<Object> tuple);
+}
diff --git a/src/jvm/backtype/storm/topology/IComponent.java b/src/jvm/backtype/storm/topology/IComponent.java
new file mode 100644
index 0000000..cb82cc9
--- /dev/null
+++ b/src/jvm/backtype/storm/topology/IComponent.java
@@ -0,0 +1,17 @@
+package backtype.storm.topology;
+
+import java.io.Serializable;
+
+/**
+ * Common methods for all possible components in a topology. This interface is used
+ * when defining topologies using the Java API. 
+ */
+public interface IComponent extends Serializable {
+
+    /**
+     * Declare the output schema for all the streams of this topology.
+     *
+     * @param declarer this is used to declare output stream ids, output fields, and whether or not each output stream is a direct stream
+     */
+    public void declareOutputFields(OutputFieldsDeclarer declarer);
+}
diff --git a/src/jvm/backtype/storm/topology/IRichBolt.java b/src/jvm/backtype/storm/topology/IRichBolt.java
new file mode 100644
index 0000000..634e075
--- /dev/null
+++ b/src/jvm/backtype/storm/topology/IRichBolt.java
@@ -0,0 +1,12 @@
+package backtype.storm.topology;
+
+import backtype.storm.task.IBolt;
+
+/**
+ * When writing topologies using Java, {@link IRichBolt} and {@link IRichSpout} are the main interfaces
+ * to use to implement components of the topology.
+ *
+ */
+public interface IRichBolt extends IBolt, IComponent {
+
+}
diff --git a/src/jvm/backtype/storm/topology/IRichSpout.java b/src/jvm/backtype/storm/topology/IRichSpout.java
new file mode 100644
index 0000000..f2f624a
--- /dev/null
+++ b/src/jvm/backtype/storm/topology/IRichSpout.java
@@ -0,0 +1,19 @@
+package backtype.storm.topology;
+
+import backtype.storm.spout.ISpout;
+
+/**
+ * When writing topologies using Java, {@link IRichBolt} and {@link IRichSpout} are the main interfaces
+ * to use to implement components of the topology.
+ *
+ */
+public interface IRichSpout extends ISpout, IComponent {
+    /**
+     * Returns if this spout is allowed to execute as multiple tasks. If
+     * this method returns false, the planner will ignore the parallelism
+     * assigned to this component and only use one task to execute this spout.
+     *
+     * @return Whether this spout is allowed to execute as multiple tasks
+     */
+    public boolean isDistributed();
+}
diff --git a/src/jvm/backtype/storm/topology/IRichStateSpout.java b/src/jvm/backtype/storm/topology/IRichStateSpout.java
new file mode 100644
index 0000000..1281aaa
--- /dev/null
+++ b/src/jvm/backtype/storm/topology/IRichStateSpout.java
@@ -0,0 +1,8 @@
+package backtype.storm.topology;
+
+import backtype.storm.state.IStateSpout;
+
+
+public interface IRichStateSpout extends IStateSpout, IComponent {
+
+}
diff --git a/src/jvm/backtype/storm/topology/InputDeclarer.java b/src/jvm/backtype/storm/topology/InputDeclarer.java
new file mode 100644
index 0000000..362639f
--- /dev/null
+++ b/src/jvm/backtype/storm/topology/InputDeclarer.java
@@ -0,0 +1,24 @@
+package backtype.storm.topology;
+
+import backtype.storm.tuple.Fields;
+
+
+public interface InputDeclarer {
+    public InputDeclarer fieldsGrouping(int componentId, Fields fields);
+    public InputDeclarer fieldsGrouping(int componentId, int streamId, Fields fields);
+
+    public InputDeclarer globalGrouping(int componentId);
+    public InputDeclarer globalGrouping(int componentId, int streamId);
+
+    public InputDeclarer shuffleGrouping(int componentId);
+    public InputDeclarer shuffleGrouping(int componentId, int streamId);
+
+    public InputDeclarer noneGrouping(int componentId);
+    public InputDeclarer noneGrouping(int componentId, int streamId);
+
+    public InputDeclarer allGrouping(int componentId);
+    public InputDeclarer allGrouping(int componentId, int streamId);
+
+    public InputDeclarer directGrouping(int componentId);
+    public InputDeclarer directGrouping(int componentId, int streamId);
+}
diff --git a/src/jvm/backtype/storm/topology/OutputFieldsDeclarer.java b/src/jvm/backtype/storm/topology/OutputFieldsDeclarer.java
new file mode 100644
index 0000000..43ef8c5
--- /dev/null
+++ b/src/jvm/backtype/storm/topology/OutputFieldsDeclarer.java
@@ -0,0 +1,15 @@
+package backtype.storm.topology;
+
+import backtype.storm.tuple.Fields;
+
+
+public interface OutputFieldsDeclarer {
+    /**
+     * Uses default stream id.
+     */
+    public void declare(Fields fields);
+    public void declare(boolean direct, Fields fields);
+    
+    public void declareStream(int streamId, Fields fields);
+    public void declareStream(int streamId, boolean direct, Fields fields);
+}
diff --git a/src/jvm/backtype/storm/topology/OutputFieldsGetter.java b/src/jvm/backtype/storm/topology/OutputFieldsGetter.java
new file mode 100644
index 0000000..7e9b139
--- /dev/null
+++ b/src/jvm/backtype/storm/topology/OutputFieldsGetter.java
@@ -0,0 +1,36 @@
+package backtype.storm.topology;
+
+import backtype.storm.generated.StreamInfo;
+import backtype.storm.tuple.Fields;
+import backtype.storm.utils.Utils;
+import java.util.HashMap;
+import java.util.Map;
+
+public class OutputFieldsGetter implements OutputFieldsDeclarer {
+    private Map<Integer, StreamInfo> _fields = new HashMap<Integer, StreamInfo>();
+
+    public void declare(Fields fields) {
+        declare(false, fields);
+    }
+
+    public void declare(boolean direct, Fields fields) {
+        declareStream(Utils.DEFAULT_STREAM_ID, direct, fields);
+    }
+
+    public void declareStream(int streamId, Fields fields) {
+        declareStream(streamId, false, fields);
+    }
+
+    public void declareStream(int streamId, boolean direct, Fields fields) {
+        if(_fields.containsKey(streamId)) {
+            throw new IllegalArgumentException("Fields for " + streamId + " already set");
+        }
+        _fields.put(streamId, new StreamInfo(fields.toList(), direct));
+    }
+
+
+    public Map<Integer, StreamInfo> getFieldsDeclaration() {
+        return _fields;
+    }
+
+}
\ No newline at end of file
diff --git a/src/jvm/backtype/storm/topology/TopologyBuilder.java b/src/jvm/backtype/storm/topology/TopologyBuilder.java
new file mode 100644
index 0000000..e4b5201
--- /dev/null
+++ b/src/jvm/backtype/storm/topology/TopologyBuilder.java
@@ -0,0 +1,271 @@
+package backtype.storm.topology;
+
+import backtype.storm.generated.Bolt;
+import backtype.storm.generated.ComponentCommon;
+import backtype.storm.generated.ComponentObject;
+import backtype.storm.generated.GlobalStreamId;
+import backtype.storm.generated.Grouping;
+import backtype.storm.generated.NullStruct;
+import backtype.storm.generated.SpoutSpec;
+import backtype.storm.generated.StateSpoutSpec;
+import backtype.storm.generated.StormTopology;
+import backtype.storm.tuple.Fields;
+import backtype.storm.utils.Utils;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * TopologyBuilder exposes the Java API for specifying a topology for Storm
+ * to execute. Topologies are Thrift structures in the end, but since the Thrift API
+ * is so verbose, TopologyBuilder greatly eases the process of creating topologies.
+ * The template for creating and submitting a topology looks something like:
+ *
+ * <pre>
+ * TopologyBuilder builder = new TopologyBuilder();
+ *
+ * builder.setSpout(1, new TestWordSpout(true), 5);
+ * builder.setSpout(2, new TestWordSpout(true), 3);
+ * builder.setBolt(3, new TestWordCounter(), 3)
+ *          .fieldsGrouping(1, new Fields("word"))
+ *          .fieldsGrouping(2, new Fields("word"));
+ * builder.setBolt(4, new TestGlobalCount())
+ *          .globalGrouping(1);
+ *
+ * Map conf = new HashMap();
+ * conf.put(Config.TOPOLOGY_WORKERS, 4);
+ * 
+ * StormSubmitter.submitTopology("mytopology", conf, builder.createTopology());
+ * </pre>
+ *
+ * Running the exact same topology in local mode (in process), and configuring it to log all tuples
+ * emitted, looks like the following. Note that it lets the topology run for 10 seconds
+ * before shutting down the local cluster.
+ *
+ * <pre>
+ * TopologyBuilder builder = new TopologyBuilder();
+ *
+ * builder.setSpout(1, new TestWordSpout(true), 5);
+ * builder.setSpout(2, new TestWordSpout(true), 3);
+ * builder.setBolt(3, new TestWordCounter(), 3)
+ *          .fieldsGrouping(1, new Fields("word"))
+ *          .fieldsGrouping(2, new Fields("word"));
+ * builder.setBolt(4, new TestGlobalCount())
+ *          .globalGrouping(1);
+ *
+ * Map conf = new HashMap();
+ * conf.put(Config.TOPOLOGY_WORKERS, 4);
+ * conf.put(Config.TOPOLOGY_DEBUG, true);
+ *
+ * LocalCluster cluster = new LocalCluster();
+ * cluster.submitTopology("mytopology", conf, builder.createTopology());
+ * Utils.sleep(10000);
+ * cluster.shutdown();
+ * </pre>
+ *
+ * <p>The pattern for TopologyBuilder is to map component ids to components using the setSpout
+ * and setBolt methods. Those methods return objects that are then used to declare
+ * the inputs for that component.</p>
+ */
+public class TopologyBuilder {
+    private Map<Integer, IRichBolt> _bolts = new HashMap<Integer, IRichBolt>();
+    private Map<Integer, Map<GlobalStreamId, Grouping>> _inputs = new HashMap<Integer, Map<GlobalStreamId, Grouping>>();
+    private Map<Integer, SpoutSpec> _spouts = new HashMap<Integer, SpoutSpec>();
+    private Map<Integer, StateSpoutSpec> _stateSpouts = new HashMap<Integer, StateSpoutSpec>();
+    private Map<Integer, Integer> _boltParallelismHints = new HashMap<Integer, Integer>();
+
+    public StormTopology createTopology() {
+        Map<Integer, Bolt> boltSpecs = new HashMap<Integer, Bolt>();
+        for(Integer boltId: _bolts.keySet()) {
+            IRichBolt bolt = _bolts.get(boltId);
+            Integer parallelism_hint = _boltParallelismHints.get(boltId);
+            Map<GlobalStreamId, Grouping> inputs = _inputs.get(boltId);
+            ComponentCommon common = getComponentCommon(bolt, parallelism_hint);
+            if(parallelism_hint!=null) {
+                common.set_parallelism_hint(parallelism_hint);
+            }
+            boltSpecs.put(boltId, new Bolt(inputs, ComponentObject.serialized_java(Utils.serialize(bolt)), common));
+        }
+        return new StormTopology(new HashMap<Integer, SpoutSpec>(_spouts),
+                                 boltSpecs,
+                                 new HashMap<Integer, StateSpoutSpec>(_stateSpouts));
+    }
+
+    /**
+     * Define a new bolt in this topology with parallelism of just one thread.
+     *
+     * @param id the id of this component. This id is referenced by other components that want to consume this bolt's outputs.
+     * @param bolt the bolt
+     * @return use the returned object to declare the inputs to this component
+     */
+    public InputDeclarer setBolt(int id, IRichBolt bolt) {
+        return setBolt(id, bolt, null);
+    }
+
+    /**
+     * Define a new bolt in this topology with the specified amount of parallelism.
+     *
+     * @param id the id of this component. This id is referenced by other components that want to consume this bolt's outputs.
+     * @param bolt the bolt
+     * @param parallelism_hint the number of tasks that should be assigned to execute this bolt. Each task will run on a thread in a process somewhere around the cluster.
+     * @return use the returned object to declare the inputs to this component
+     */
+    public InputDeclarer setBolt(int id, IRichBolt bolt, Integer parallelism_hint) {
+        validateUnusedId(id);
+        _bolts.put(id, bolt);
+        _boltParallelismHints.put(id, parallelism_hint);
+        _inputs.put(id, new HashMap<GlobalStreamId, Grouping>());
+        return new InputGetter(id);
+    }
+
+    /**
+     * Define a new bolt in this topology. This defines a basic bolt, which is a
+     * simpler to use but more restricted kind of bolt. Basic bolts are intended
+     * for non-aggregation processing and automate the anchoring/acking process to
+     * achieve proper reliability in the topology.
+     *
+     * @param id the id of this component. This id is referenced by other components that want to consume this bolt's outputs.
+     * @param bolt the basic bolt
+     * @return use the returned object to declare the inputs to this component
+     */
+    public InputDeclarer setBolt(int id, IBasicBolt bolt) {
+        return setBolt(id, bolt, null);
+    }
+
+    /**
+     * Define a new bolt in this topology. This defines a basic bolt, which is a
+     * simpler to use but more restricted kind of bolt. Basic bolts are intended
+     * for non-aggregation processing and automate the anchoring/acking process to
+     * achieve proper reliability in the topology.
+     *
+     * @param id the id of this component. This id is referenced by other components that want to consume this bolt's outputs.
+     * @param bolt the basic bolt
+     * @param parallelism_hint the number of tasks that should be assigned to execute this bolt. Each task will run on a thread in a process somwehere around the cluster.
+     * @return use the returned object to declare the inputs to this component
+     */
+    public InputDeclarer setBolt(int id, IBasicBolt bolt, Integer parallelism_hint) {
+        return setBolt(id, new BasicBoltExecutor(bolt), parallelism_hint);
+    }
+
+    /**
+     * Define a new spout in this topology.
+     *
+     * @param id the id of this component. This id is referenced by other components that want to consume this spout's outputs.
+     * @param spout the spout
+     */
+    public void setSpout(int id, IRichSpout spout) {
+        setSpout(id, spout, null);
+    }
+
+    /**
+     * Define a new spout in this topology with the specified parallelism. If the spout declares
+     * itself as non-distributed, the parallelism_hint will be ignored and only one task
+     * will be allocated to this component.
+     *
+     * @param id the id of this component. This id is referenced by other components that want to consume this spout's outputs.
+     * @param parallelism_hint the number of tasks that should be assigned to execute this spout. Each task will run on a thread in a process somwehere around the cluster.
+     * @param spout the spout
+     */
+    public void setSpout(int id, IRichSpout spout, Integer parallelism_hint) {
+        validateUnusedId(id);
+        _spouts.put(id, new SpoutSpec(ComponentObject.serialized_java(Utils.serialize(spout)), getComponentCommon(spout, parallelism_hint), spout.isDistributed()));
+    }
+
+    public void setStateSpout(int id, IRichStateSpout stateSpout) {
+        setStateSpout(id, stateSpout, null);
+    }
+
+    public void setStateSpout(int id, IRichStateSpout stateSpout, Integer parallelism_hint) {
+        validateUnusedId(id);
+        _stateSpouts.put(id,
+                         new StateSpoutSpec(
+                             ComponentObject.serialized_java(Utils.serialize(stateSpout)),
+                             getComponentCommon(stateSpout, parallelism_hint)));
+    }
+
+
+    private void validateUnusedId(int id) {
+        if(_bolts.containsKey(id)) {
+            throw new IllegalArgumentException("Bolt has already been declared for id " + id);
+        }
+        if(_spouts.containsKey(id)) {
+            throw new IllegalArgumentException("Spout has already been declared for id " + id);
+        }
+        if(_stateSpouts.containsKey(id)) {
+            throw new IllegalArgumentException("State spout has already been declared for id " + id);
+        }
+    }
+
+    private ComponentCommon getComponentCommon(IComponent component, Integer parallelism_hint) {
+        OutputFieldsGetter getter = new OutputFieldsGetter();
+        component.declareOutputFields(getter);
+        ComponentCommon common = new ComponentCommon(getter.getFieldsDeclaration());
+        if(parallelism_hint!=null) {
+            common.set_parallelism_hint(parallelism_hint);
+        }
+        return common;
+        
+    }
+
+    protected class InputGetter implements InputDeclarer {
+        private int _boltId;
+
+        public InputGetter(int boltId) {
+            _boltId = boltId;
+        }
+
+        public InputDeclarer fieldsGrouping(int componentId, Fields fields) {
+            return fieldsGrouping(componentId, Utils.DEFAULT_STREAM_ID, fields);
+        }
+
+        public InputDeclarer fieldsGrouping(int componentId, int streamId, Fields fields) {
+            return grouping(componentId, streamId, Grouping.fields(fields.toList()));
+        }
+
+        public InputDeclarer globalGrouping(int componentId) {
+            return globalGrouping(componentId, Utils.DEFAULT_STREAM_ID);
+        }
+
+        public InputDeclarer globalGrouping(int componentId, int streamId) {
+            return grouping(componentId, streamId, Grouping.fields(new ArrayList<String>()));
+        }
+
+        public InputDeclarer shuffleGrouping(int componentId) {
+            return shuffleGrouping(componentId, Utils.DEFAULT_STREAM_ID);
+        }
+
+        public InputDeclarer shuffleGrouping(int componentId, int streamId) {
+            return grouping(componentId, streamId, Grouping.shuffle(new NullStruct()));
+        }
+
+        public InputDeclarer noneGrouping(int componentId) {
+            return noneGrouping(componentId, Utils.DEFAULT_STREAM_ID);
+        }
+
+        public InputDeclarer noneGrouping(int componentId, int streamId) {
+            return grouping(componentId, streamId, Grouping.none(new NullStruct()));
+        }
+
+        public InputDeclarer allGrouping(int componentId) {
+            return allGrouping(componentId, Utils.DEFAULT_STREAM_ID);
+        }
+
+        public InputDeclarer allGrouping(int componentId, int streamId) {
+            return grouping(componentId, streamId, Grouping.all(new NullStruct()));
+        }
+
+        public InputDeclarer directGrouping(int componentId) {
+            return directGrouping(componentId, Utils.DEFAULT_STREAM_ID);
+        }
+
+        public InputDeclarer directGrouping(int componentId, int streamId) {
+            return grouping(componentId, streamId, Grouping.direct(new NullStruct()));
+        }
+
+        private InputDeclarer grouping(int componentId, int streamId, Grouping grouping) {
+            _inputs.get(_boltId).put(new GlobalStreamId(componentId, streamId), grouping);
+            return this;
+        }
+        
+    }
+}
diff --git a/src/jvm/backtype/storm/tuple/Fields.java b/src/jvm/backtype/storm/tuple/Fields.java
new file mode 100644
index 0000000..b7732db
--- /dev/null
+++ b/src/jvm/backtype/storm/tuple/Fields.java
@@ -0,0 +1,53 @@
+package backtype.storm.tuple;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.io.Serializable;
+
+public class Fields implements Iterable<String>, Serializable {
+    private List<String> _fields;
+    private Map<String, Integer> _index = new HashMap<String, Integer>();
+    
+    public Fields(String... fields) {
+        this(Arrays.asList(fields));
+    }
+    
+    public Fields(List<String> fields) {
+        _fields = new ArrayList<String>(fields);
+        index();
+    }
+    
+    public List<Object> select(Fields selector, List<Object> tuple) {
+        List<Object> ret = new ArrayList<Object>(selector.size());
+        for(String s: selector) {
+            ret.add(tuple.get(_index.get(s)));
+        }
+        return ret;
+    }
+
+    public List<String> toList() {
+        return new ArrayList<String>(_fields);
+    }
+    
+    public int size() {
+        return _fields.size();
+    }
+
+    public String get(int index) {
+        return _fields.get(index);
+    }
+
+    public Iterator<String> iterator() {
+        return _fields.iterator();
+    }
+    
+    private void index() {
+        for(int i=0; i<_fields.size(); i++) {
+            _index.put(_fields.get(i), i);
+        }
+    }
+}
\ No newline at end of file
diff --git a/src/jvm/backtype/storm/tuple/MessageId.java b/src/jvm/backtype/storm/tuple/MessageId.java
new file mode 100644
index 0000000..0f7370f
--- /dev/null
+++ b/src/jvm/backtype/storm/tuple/MessageId.java
@@ -0,0 +1,81 @@
+package backtype.storm.tuple;
+
+import backtype.storm.utils.WritableUtils;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.UUID;
+
+public class MessageId {
+    private Map<Long, Long> _anchorsToIds;
+    
+    public static long generateId() {
+        return UUID.randomUUID().getLeastSignificantBits();
+    }
+
+    public static MessageId makeUnanchored() {
+        return makeId(new HashMap<Long, Long>());
+    }
+        
+    public static MessageId makeId(Map<Long, Long> anchorsToIds) {
+        return new MessageId(anchorsToIds);
+    }
+        
+    public static MessageId makeRootId(long id) {
+        Map<Long, Long> anchorsToIds = new HashMap<Long, Long>();
+        anchorsToIds.put(id, id);
+        return new MessageId(anchorsToIds);
+    }
+    
+    protected MessageId(Map<Long, Long> anchorsToIds) {
+        _anchorsToIds = anchorsToIds;
+    }
+
+    public Map<Long, Long> getAnchorsToIds() {
+        return _anchorsToIds;
+    }
+
+    public Set<Long> getAnchors() {
+        return _anchorsToIds.keySet();
+    }    
+    
+    @Override
+    public int hashCode() {
+        return _anchorsToIds.hashCode();
+    }
+
+    @Override
+    public boolean equals(Object other) {
+        if(other instanceof MessageId) {
+            return _anchorsToIds == ((MessageId) other)._anchorsToIds;
+        } else {
+            return false;
+        }
+    }
+
+    @Override
+    public String toString() {
+        return _anchorsToIds.toString();
+    }
+
+    public void serialize(DataOutputStream out) throws IOException {
+        WritableUtils.writeVInt(out, _anchorsToIds.size());
+        for(Entry<Long, Long> anchorToId: _anchorsToIds.entrySet()) {
+            out.writeLong(anchorToId.getKey());
+            out.writeLong(anchorToId.getValue());
+        }
+    }
+
+    public static MessageId deserialize(DataInputStream in) throws IOException {
+        int numAnchors = WritableUtils.readVInt(in);
+        Map<Long, Long> anchorsToIds = new HashMap<Long, Long>();
+        for(int i=0; i<numAnchors; i++) {
+            anchorsToIds.put(in.readLong(), in.readLong());
+        }
+        return new MessageId(anchorsToIds);
+    }
+}
\ No newline at end of file
diff --git a/src/jvm/backtype/storm/tuple/Tuple.java b/src/jvm/backtype/storm/tuple/Tuple.java
new file mode 100644
index 0000000..75786f2
--- /dev/null
+++ b/src/jvm/backtype/storm/tuple/Tuple.java
@@ -0,0 +1,144 @@
+package backtype.storm.tuple;
+
+import backtype.storm.task.TopologyContext;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class Tuple {
+    private List<Object> values;
+    private int taskId;
+    private int streamId;
+    private TopologyContext context;
+    private MessageId id;
+
+    //needs to get taskId explicitly b/c could be in a different task than where it was created
+    public Tuple(TopologyContext context, List<Object> values, int taskId, int streamId, MessageId id) {
+        this.values = values;
+        this.taskId = taskId;
+        this.streamId = streamId;
+        this.id = id;
+        this.context = context;
+        //TODO: should find a way to include this information here
+        //TODO: should only leave out the connection info?
+        //TODO: have separate methods for "user" and "system" topology?
+        if(streamId>=0) {
+            int componentId = context.getComponentId(taskId);
+            if(componentId>=0) {
+                Fields schema = context.getComponentOutputFields(componentId, streamId);
+                if(values.size()!=schema.size()) {
+                    throw new IllegalArgumentException(
+                            "Tuple created with wrong number of fields. " +
+                            "Expected " + schema.size() + " fields but got " +
+                            values.size() + " fields");
+                }
+            }
+        }
+    }
+
+    public Tuple(TopologyContext context, List<Object> values, int taskId, int streamId) {
+        this(context, values, taskId, streamId, MessageId.makeUnanchored());
+    }
+
+    public Tuple copyWithNewId(long id) {
+        Map<Long, Long> newIds = new HashMap<Long, Long>();
+        for(Long anchor: this.id.getAnchorsToIds().keySet()) {
+            newIds.put(anchor, id);
+        }
+        return new Tuple(this.context, this.values, this.taskId, this.streamId, MessageId.makeId(newIds));
+    }
+
+    public int size() {
+        return values.size();
+    }
+
+    public Object getValue(int i) {
+        return values.get(i);
+    }
+
+    public String getString(int i) {
+        return (String) values.get(i);
+    }
+
+    public Integer getInteger(int i) {
+        return (Integer) values.get(i);
+    }
+
+    public Long getLong(int i) {
+        return (Long) values.get(i);
+    }
+
+    public Boolean getBoolean(int i) {
+        return (Boolean) values.get(i);
+    }
+
+    public Short getShort(int i) {
+        return (Short) values.get(i);
+    }
+
+    public Byte getByte(int i) {
+        return (Byte) values.get(i);
+    }
+
+    public Double getDouble(int i) {
+        return (Double) values.get(i);
+    }
+
+    public Float getFloat(int i) {
+        return (Float) values.get(i);
+    }
+
+    public byte[] getBinary(int i) {
+        return (byte[]) values.get(i);
+    }
+    
+    @Deprecated
+    public List<Object> getTuple() {
+        return values;
+    }
+
+    public List<Object> getValues() {
+        return values;
+    }
+    
+    public Fields getFields() {
+        return context.getComponentOutputFields(getSourceComponent(), getSourceStreamId());
+    }
+
+    public List<Object> select(Fields selector) {
+        return getFields().select(selector, values);
+    }
+    
+    public int getSourceComponent() {
+        return context.getComponentId(taskId);
+    }
+    
+    public int getSourceTask() {
+        return taskId;
+    }
+    
+    public int getSourceStreamId() {
+        return streamId;
+    }
+    
+    public MessageId getMessageId() {
+        return id;
+    }
+    
+    @Override
+    public String toString() {
+        return "source: " + getSourceComponent() + ":" + taskId + ", stream: " + streamId + ", id: "+ id.toString() + ", " + values.toString();
+    }
+    
+    @Override
+    public boolean equals(Object other) {
+        // for OutputCollector
+        return this == other;
+    }
+    
+    @Override
+    public int hashCode() {
+        // for OutputCollector
+        return System.identityHashCode(this);
+    }
+}
diff --git a/src/jvm/backtype/storm/tuple/Values.java b/src/jvm/backtype/storm/tuple/Values.java
new file mode 100644
index 0000000..ce94f7f
--- /dev/null
+++ b/src/jvm/backtype/storm/tuple/Values.java
@@ -0,0 +1,16 @@
+package backtype.storm.tuple;
+
+import java.util.ArrayList;
+
+/**
+ * A convenience class for making tuple values using new Values("field1", 2, 3)
+ * syntax.
+ */
+public class Values extends ArrayList<Object>{
+    public Values(Object... vals) {
+        super(vals.length);
+        for(Object o: vals) {
+            add(o);
+        }
+    }
+}
diff --git a/src/jvm/backtype/storm/utils/BufferFileInputStream.java b/src/jvm/backtype/storm/utils/BufferFileInputStream.java
new file mode 100644
index 0000000..f6f0ca0
--- /dev/null
+++ b/src/jvm/backtype/storm/utils/BufferFileInputStream.java
@@ -0,0 +1,37 @@
+package backtype.storm.utils;
+
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.Arrays;
+
+
+public class BufferFileInputStream {
+    byte[] buffer;
+    FileInputStream stream;
+
+    public BufferFileInputStream(String file, int bufferSize) throws FileNotFoundException {
+        stream = new FileInputStream(file);
+        buffer = new byte[bufferSize];
+    }
+
+    public BufferFileInputStream(String file) throws FileNotFoundException {
+        this(file, 15*1024);
+    }
+
+    public byte[] read() throws IOException {
+        int length = stream.read(buffer);
+        if(length==-1) {
+            close();
+            return new byte[0];
+        } else if(length==buffer.length) {
+            return buffer;
+        } else {
+            return Arrays.copyOf(buffer, length);
+        }
+    }
+
+    public void close() throws IOException {
+        stream.close();
+    }
+}
diff --git a/src/jvm/backtype/storm/utils/CRC32OutputStream.java b/src/jvm/backtype/storm/utils/CRC32OutputStream.java
new file mode 100644
index 0000000..71bec55
--- /dev/null
+++ b/src/jvm/backtype/storm/utils/CRC32OutputStream.java
@@ -0,0 +1,27 @@
+package backtype.storm.utils;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.zip.CRC32;
+
+public class CRC32OutputStream extends OutputStream {
+    private CRC32 hasher;
+    
+    public CRC32OutputStream() {
+        hasher = new CRC32();
+    }
+    
+    public long getValue() {
+        return hasher.getValue();
+    }
+
+    @Override
+    public void write(int i) throws IOException {
+        hasher.update(i);
+    }
+
+    @Override
+    public void write(byte[] bytes, int start, int end) throws IOException {
+        hasher.update(bytes, start, end);
+    }    
+}
diff --git a/src/jvm/backtype/storm/utils/DRPCClient.java b/src/jvm/backtype/storm/utils/DRPCClient.java
new file mode 100644
index 0000000..4132efa
--- /dev/null
+++ b/src/jvm/backtype/storm/utils/DRPCClient.java
@@ -0,0 +1,40 @@
+package backtype.storm.utils;
+
+import backtype.storm.generated.DistributedRPC;
+import org.apache.thrift.TException;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.apache.thrift.transport.TFramedTransport;
+import org.apache.thrift.transport.TSocket;
+import org.apache.thrift.transport.TTransport;
+
+
+public class DRPCClient {
+    private TTransport conn;
+    private DistributedRPC.Client client;
+
+    public DRPCClient(String host, int port) {
+        try {
+            conn = new TFramedTransport(new TSocket(host, port));
+            client = new DistributedRPC.Client(new TBinaryProtocol(conn));
+            conn.open();
+        } catch(TException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    public String execute(String func, String args) throws TException {
+        return client.execute(func, args);
+    }
+
+    public void result(String id, String result) throws TException {
+        client.result(id, result);
+    }
+
+    public DistributedRPC.Client getClient() {
+        return client;
+    }
+
+    public void close() {
+        conn.close();
+    }
+}
diff --git a/src/jvm/backtype/storm/utils/KeyedRoundRobinQueue.java b/src/jvm/backtype/storm/utils/KeyedRoundRobinQueue.java
new file mode 100644
index 0000000..920f79c
--- /dev/null
+++ b/src/jvm/backtype/storm/utils/KeyedRoundRobinQueue.java
@@ -0,0 +1,51 @@
+package backtype.storm.utils;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.concurrent.Semaphore;
+
+public class KeyedRoundRobinQueue<V> {
+    private final Object _lock = new Object();
+    private Semaphore _size = new Semaphore(0);
+    private Map<Object, Queue<V>> _queues = new HashMap<Object, Queue<V>>();
+    private List<Object> _keyOrder = new ArrayList<Object>();
+    private int _currIndex = 0;
+
+    public void add(Object key, V val) {
+        synchronized(_lock) {
+            Queue<V> queue = _queues.get(key);
+            if(queue==null) {
+                queue = new LinkedList<V>();
+                _queues.put(key, queue);
+                _keyOrder.add(key);
+            }
+            queue.add(val);
+        }
+        _size.release();
+    }
+
+    public V take() throws InterruptedException {
+        _size.acquire();
+        synchronized(_lock) {
+            Object key = _keyOrder.get(_currIndex);
+            Queue<V> queue = _queues.get(key);
+            V ret = queue.remove();
+            if(queue.isEmpty()) {
+                _keyOrder.remove(_currIndex);
+                _queues.remove(key);
+                if(_keyOrder.size()==0) {
+                    _currIndex = 0;
+                } else {
+                    _currIndex = _currIndex % _keyOrder.size();
+                }
+            } else {
+                _currIndex = (_currIndex + 1) % _keyOrder.size();
+            }
+            return ret;
+        }
+    }
+}
diff --git a/src/jvm/backtype/storm/utils/LocalState.java b/src/jvm/backtype/storm/utils/LocalState.java
new file mode 100644
index 0000000..00d0f96
--- /dev/null
+++ b/src/jvm/backtype/storm/utils/LocalState.java
@@ -0,0 +1,44 @@
+package backtype.storm.utils;
+
+import org.apache.commons.io.FileUtils;
+import java.io.File;
+import java.util.Map;
+import java.util.HashMap;
+import java.io.IOException;
+
+
+/**
+ * A simple, durable, atomic K/V database. *Very inefficient*, should only be used for occasional reads/writes.
+ * Every read/write hits disk.
+ */
+public class LocalState {
+    private VersionedStore _vs;
+    
+    public LocalState(String backingDir) throws IOException {
+        _vs = new VersionedStore(backingDir);
+    }
+    
+    public synchronized Map<Object, Object> snapshot() throws IOException {
+        String latestPath = _vs.mostRecentVersionPath();
+        if(latestPath==null) return new HashMap<Object, Object>();
+        return (Map<Object, Object>) Utils.deserialize(FileUtils.readFileToByteArray(new File(latestPath)));
+    }
+    
+    public Object get(Object key) throws IOException {
+        return snapshot().get(key);
+    }
+    
+    public synchronized void put(Object key, Object val) throws IOException {
+        Map<Object, Object> curr = snapshot();
+        curr.put(key, val);
+        persist(curr);
+    }
+    
+    private void persist(Map<Object, Object> val) throws IOException {
+        byte[] toWrite = Utils.serialize(val);
+        String newPath = _vs.createVersion();
+        FileUtils.writeByteArrayToFile(new File(newPath), toWrite);
+        _vs.succeedVersion(newPath);
+        _vs.cleanup(4);
+    }
+}
\ No newline at end of file
diff --git a/src/jvm/backtype/storm/utils/NimbusClient.java b/src/jvm/backtype/storm/utils/NimbusClient.java
new file mode 100644
index 0000000..5fb6fb5
--- /dev/null
+++ b/src/jvm/backtype/storm/utils/NimbusClient.java
@@ -0,0 +1,44 @@
+package backtype.storm.utils;
+
+import backtype.storm.Config;
+import backtype.storm.generated.Nimbus;
+import java.util.Map;
+import org.apache.thrift.TException;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.apache.thrift.transport.TFramedTransport;
+import org.apache.thrift.transport.TSocket;
+import org.apache.thrift.transport.TTransport;
+
+
+public class NimbusClient {
+    public static NimbusClient getConfiguredClient(Map conf) {
+        String nimbusHost = (String) conf.get(Config.NIMBUS_HOST);
+        int nimbusPort = ((Long) conf.get(Config.NIMBUS_THRIFT_PORT)).intValue();
+        return new NimbusClient(nimbusHost, nimbusPort);
+    }
+
+    private TTransport conn;
+    private Nimbus.Client client;
+
+    public NimbusClient(String host) {
+        this(host, 6627);
+    }
+
+    public NimbusClient(String host, int port) {
+        try {
+            conn = new TFramedTransport(new TSocket(host, port));
+            client = new Nimbus.Client(new TBinaryProtocol(conn));
+            conn.open();
+        } catch(TException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    public Nimbus.Client getClient() {
+        return client;
+    }
+
+    public void close() {
+        conn.close();
+    }
+}
diff --git a/src/jvm/backtype/storm/utils/Time.java b/src/jvm/backtype/storm/utils/Time.java
new file mode 100644
index 0000000..b3abe0e
--- /dev/null
+++ b/src/jvm/backtype/storm/utils/Time.java
@@ -0,0 +1,78 @@
+package backtype.storm.utils;
+
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.log4j.Logger;
+
+
+public class Time {
+    public static Logger LOG = Logger.getLogger(Time.class);    
+    
+    private static AtomicBoolean simulating = new AtomicBoolean(false);
+    //TODO: should probably use weak references here or something
+    private static Map<Thread, AtomicLong> threadSleepTimes;
+    private static final Object sleepTimesLock = new Object();
+    
+    private static AtomicLong simulatedCurrTimeMs; //should this be a thread local that's allowed to keep advancing?
+    
+    public static void startSimulating() {
+        simulating.set(true);
+        simulatedCurrTimeMs = new AtomicLong(0);
+        threadSleepTimes = new ConcurrentHashMap<Thread, AtomicLong>();
+    }
+    
+    public static void stopSimulating() {
+        simulating.set(false);             
+        threadSleepTimes = null;  
+    }
+    
+    public static boolean isSimulating() {
+        return simulating.get();
+    }
+    
+    public static void sleepUntil(long targetTimeMs) throws InterruptedException {
+        if(simulating.get()) {
+            synchronized(sleepTimesLock) {
+                threadSleepTimes.put(Thread.currentThread(), new AtomicLong(targetTimeMs));
+            }
+            while(simulatedCurrTimeMs.get() < targetTimeMs) {
+                Thread.sleep(10);
+            }
+            synchronized(sleepTimesLock) {
+                threadSleepTimes.remove(Thread.currentThread());
+            }
+        } else {
+            long sleepTime = targetTimeMs-currentTimeMillis();
+            if(sleepTime>0) 
+                Thread.sleep(sleepTime);
+        }
+    }
+    
+    public static void sleep(long ms) throws InterruptedException {
+        sleepUntil(currentTimeMillis()+ms);
+    }
+    
+    public static long currentTimeMillis() {
+        if(simulating.get()) {
+            return simulatedCurrTimeMs.get();
+        } else {
+            return System.currentTimeMillis();
+        }
+    }
+    
+    public static void advanceTime(long ms) {
+        if(!simulating.get()) throw new IllegalStateException("Cannot simulate time unless in simulation mode");
+        simulatedCurrTimeMs.set(simulatedCurrTimeMs.get() + ms);
+    }
+    
+    public static boolean isThreadWaiting(Thread t) {
+        if(!simulating.get()) throw new IllegalStateException("Must be in simulation mode");
+        AtomicLong time;
+        synchronized(sleepTimesLock) {
+            time = threadSleepTimes.get(t);
+        }
+        return time!=null && currentTimeMillis() < time.longValue();
+    }    
+}
diff --git a/src/jvm/backtype/storm/utils/TimeCacheMap.java b/src/jvm/backtype/storm/utils/TimeCacheMap.java
new file mode 100644
index 0000000..dad2efe
--- /dev/null
+++ b/src/jvm/backtype/storm/utils/TimeCacheMap.java
@@ -0,0 +1,149 @@
+package backtype.storm.utils;
+
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.Map;
+import java.util.Map.Entry;
+
+/**
+ * Expires keys that have not been updated in the configured number of seconds.
+ * The algorithm used will take between expirationSecs and
+ * expirationSecs * (1 + 1 / (numBuckets-1)) to actually expire the message.
+ *
+ * get, put, remove, containsKey, and size take O(numBuckets) time to run.
+ *
+ * The advantage of this design is that the expiration thread only locks the object
+ * for O(1) time, meaning the object is essentially always available for gets/puts.
+ */
+public class TimeCacheMap<K, V> {
+    //this default ensures things expire at most 50% past the expiration time
+    private static final int DEFAULT_NUM_BUCKETS = 3;
+
+    public static interface ExpiredCallback<K, V> {
+        public void expire(K key, V val);
+    }
+
+    private LinkedList<HashMap<K, V>> _buckets;
+
+    private final Object _lock = new Object();
+    private Thread _cleaner;
+    private ExpiredCallback _callback;
+    
+    public TimeCacheMap(int expirationSecs, int numBuckets, ExpiredCallback<K, V> callback) {
+        if(numBuckets<2) {
+            throw new IllegalArgumentException("numBuckets must be >= 2");
+        }
+        _buckets = new LinkedList<HashMap<K, V>>();
+        for(int i=0; i<numBuckets; i++) {
+            _buckets.add(new HashMap<K, V>());
+        }
+
+
+        _callback = callback;
+        final long expirationMillis = expirationSecs * 1000L;
+        final long sleepTime = expirationMillis / (numBuckets-1);
+        _cleaner = new Thread(new Runnable() {
+            public void run() {
+                try {
+                    while(true) {
+                        Map<K, V> dead = null;
+                        Time.sleep(sleepTime);
+                        synchronized(_lock) {
+                            dead = _buckets.removeLast();
+                            _buckets.addFirst(new HashMap<K, V>());
+                        }
+                        if(_callback!=null) {
+                            for(Entry<K, V> entry: dead.entrySet()) {
+                                _callback.expire(entry.getKey(), entry.getValue());
+                            }
+                        }
+                    }
+                } catch (InterruptedException ex) {
+
+                }
+            }
+        });
+        _cleaner.setDaemon(true);
+        _cleaner.start();
+    }
+
+    public TimeCacheMap(int expirationSecs, ExpiredCallback<K, V> callback) {
+        this(expirationSecs, DEFAULT_NUM_BUCKETS, callback);
+    }
+
+    public TimeCacheMap(int expirationSecs) {
+        this(expirationSecs, DEFAULT_NUM_BUCKETS);
+    }
+
+    public TimeCacheMap(int expirationSecs, int numBuckets) {
+        this(expirationSecs, numBuckets, null);
+    }
+
+
+    public boolean containsKey(K key) {
+        synchronized(_lock) {
+            for(HashMap<K, V> bucket: _buckets) {
+                if(bucket.containsKey(key)) {
+                    return true;
+                }
+            }
+            return false;
+        }
+    }
+
+    public V get(K key) {
+        synchronized(_lock) {
+            for(HashMap<K, V> bucket: _buckets) {
+                if(bucket.containsKey(key)) {
+                    return bucket.get(key);
+                }
+            }
+            return null;
+        }
+    }
+
+    public void put(K key, V value) {
+        synchronized(_lock) {
+            Iterator<HashMap<K, V>> it = _buckets.iterator();
+            HashMap<K, V> bucket = it.next();
+            bucket.put(key, value);
+            while(it.hasNext()) {
+                bucket = it.next();
+                bucket.remove(key);
+            }
+        }
+    }
+    
+    public Object remove(K key) {
+        synchronized(_lock) {
+            for(HashMap<K, V> bucket: _buckets) {
+                if(bucket.containsKey(key)) {
+                    return bucket.remove(key);
+                }
+            }
+            return null;
+        }
+    }
+
+    public int size() {
+        synchronized(_lock) {
+            int size = 0;
+            for(HashMap<K, V> bucket: _buckets) {
+                size+=bucket.size();
+            }
+            return size;
+        }
+    }
+
+    @Override
+    protected void finalize() throws Throwable {
+        try {
+            _cleaner.interrupt();
+        } finally {
+            super.finalize();
+        }
+    }
+
+    
+}
diff --git a/src/jvm/backtype/storm/utils/Utils.java b/src/jvm/backtype/storm/utils/Utils.java
new file mode 100644
index 0000000..dbc1d49
--- /dev/null
+++ b/src/jvm/backtype/storm/utils/Utils.java
@@ -0,0 +1,144 @@
+package backtype.storm.utils;
+
+import backtype.storm.generated.ComponentObject;
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.FileReader;
+import java.io.InputStreamReader;
+import java.io.InputStream;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import org.apache.thrift.TException;
+import org.jvyaml.YAML;
+
+public class Utils {
+    public static final int DEFAULT_STREAM_ID = 1;
+
+    public static byte[] serialize(Object obj) {
+        try {
+            ByteArrayOutputStream bos = new ByteArrayOutputStream();
+            ObjectOutputStream oos = new ObjectOutputStream(bos);
+            oos.writeObject(obj);
+            oos.close();
+            return bos.toByteArray();
+        } catch(IOException ioe) {
+            throw new RuntimeException(ioe);
+        }
+    }
+
+    public static Object deserialize(byte[] serialized) {
+        try {
+            ByteArrayInputStream bis = new ByteArrayInputStream(serialized);
+            ObjectInputStream ois = new ObjectInputStream(bis);
+            Object ret = ois.readObject();
+            ois.close();
+            return ret;
+        } catch(IOException ioe) {
+            throw new RuntimeException(ioe);
+        } catch(ClassNotFoundException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    public static <T> String join(Iterable<T> coll, String sep) {
+        Iterator<T> it = coll.iterator();
+        String ret = "";
+        while(it.hasNext()) {
+            ret = ret + it.next();
+            if(it.hasNext()) {
+                ret = ret + sep;
+            }
+        }
+        return ret;
+    }
+
+    public static void sleep(long millis) {
+        try {
+            Time.sleep(millis);
+        } catch(InterruptedException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    public static Map readYamlConfig(String path) {
+        try {
+            Map ret = (Map) YAML.load(new FileReader(path));
+            if(ret==null) ret = new HashMap();
+            return new HashMap(ret);
+        } catch (FileNotFoundException ex) {
+            throw new RuntimeException(ex);
+        }
+    }
+
+    public static Map findAndReadConfigFile(String name, boolean mustExist) {
+        InputStream is = Object.class.getResourceAsStream("/" + name);
+        if(is==null) {
+            if(mustExist) throw new RuntimeException("Could not find config file on classpath " + name);
+            else return new HashMap();
+        }
+        Map ret = (Map) YAML.load(new InputStreamReader(is));
+        if(ret==null) ret = new HashMap();
+        return new HashMap(ret);
+    }
+
+    public static Map findAndReadConfigFile(String name) {
+       return findAndReadConfigFile(name, true);
+    }
+
+    public static Map readDefaultConfig() {
+        return findAndReadConfigFile("defaults.yaml", true);
+    }
+
+    public static Map readStormConfig() {
+        Map ret = readDefaultConfig();
+        Map storm = findAndReadConfigFile("storm.yaml", false);
+        ret.putAll(storm);
+        return ret;
+    }
+
+    public static Object getSetComponentObject(ComponentObject obj) {
+        if(obj.getSetField()==ComponentObject._Fields.SERIALIZED_JAVA) {
+            return Utils.deserialize(obj.get_serialized_java());
+        } else {
+            return obj.get_shell();
+        }
+    }
+
+    public static <S, T> T get(Map<S, T> m, S key, T def) {
+        T ret = m.get(key);
+        if(ret==null) {
+            ret = def;
+        }
+        return ret;
+    }
+    
+    public static List<Object> tuple(Object... values) {
+        List<Object> ret = new ArrayList<Object>();
+        for(Object v: values) {
+            ret.add(v);
+        }
+        return ret;
+    }
+
+    public static void downloadFromMaster(Map conf, String file, String localFile) throws IOException, TException {
+        NimbusClient client = NimbusClient.getConfiguredClient(conf);
+        String id = client.getClient().beginFileDownload(file);
+        FileOutputStream out = new FileOutputStream(localFile);
+        while(true) {
+            byte[] chunk = client.getClient().downloadChunk(id);
+            if(chunk.length==0) {
+                break;
+            }
+            out.write(chunk);
+        }
+        out.close();
+    }
+}
diff --git a/src/jvm/backtype/storm/utils/VersionedStore.java b/src/jvm/backtype/storm/utils/VersionedStore.java
new file mode 100644
index 0000000..d355712
--- /dev/null
+++ b/src/jvm/backtype/storm/utils/VersionedStore.java
@@ -0,0 +1,166 @@
+package backtype.storm.utils;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.io.File;
+
+import org.apache.commons.io.FileUtils;
+
+public class VersionedStore {
+    private static final String FINISHED_VERSION_SUFFIX = ".version";
+
+    private String _root;
+    
+    public VersionedStore(String path) throws IOException {
+      _root = path;
+      mkdirs(_root);
+    }
+
+    public String getRoot() {
+        return _root;
+    }
+
+    public String versionPath(long version) {
+        return new File(_root, "" + version).getAbsolutePath();
+    }
+
+    public String mostRecentVersionPath() throws IOException {
+        Long v = mostRecentVersion();
+        if(v==null) return null;
+        return versionPath(v);
+    }
+
+    public String mostRecentVersionPath(long maxVersion) throws IOException {
+        Long v = mostRecentVersion(maxVersion);
+        if(v==null) return null;
+        return versionPath(v);
+    }
+
+    public Long mostRecentVersion() throws IOException {
+        List<Long> all = getAllVersions();
+        if(all.size()==0) return null;
+        return all.get(0);
+    }
+
+    public Long mostRecentVersion(long maxVersion) throws IOException {
+        List<Long> all = getAllVersions();
+        for(Long v: all) {
+            if(v <= maxVersion) return v;
+        }
+        return null;
+    }
+
+    public String createVersion() throws IOException {
+        Long mostRecent = mostRecentVersion();
+        long version = Time.currentTimeMillis();
+        if(mostRecent!=null && version <= mostRecent) {
+            version = mostRecent + 1;
+        }
+        return createVersion(version);
+    }
+
+    public String createVersion(long version) throws IOException {
+        String ret = versionPath(version);
+        if(getAllVersions().contains(version))
+            throw new RuntimeException("Version already exists or data already exists");
+        else
+            return ret;
+    }
+
+    public void failVersion(String path) throws IOException {
+        deleteVersion(validateAndGetVersion(path));
+    }
+
+    public void deleteVersion(long version) throws IOException {
+        File versionFile = new File(versionPath(version));
+        File tokenFile = new File(tokenPath(version));
+        
+        if(versionFile.exists()) {
+            FileUtils.forceDelete(versionFile);
+        }
+        if(tokenFile.exists()) {
+            FileUtils.forceDelete(tokenFile);
+        }        
+    }
+
+    public void succeedVersion(String path) throws IOException {
+        long version = validateAndGetVersion(path);
+        // should rewrite this to do a file move
+        createNewFile(tokenPath(version));
+    }
+
+    public void cleanup() throws IOException {
+        cleanup(-1);
+    }
+
+    public void cleanup(int versionsToKeep) throws IOException {
+        List<Long> versions = getAllVersions();
+        if(versionsToKeep >= 0) {
+            versions = versions.subList(0, Math.min(versions.size(), versionsToKeep));
+        }
+        HashSet<Long> keepers = new HashSet<Long>(versions);
+
+        for(String p: listDir(_root)) {
+            Long v = parseVersion(p);
+            if(v!=null && !keepers.contains(v)) {
+                deleteVersion(v);
+            }
+        }
+    }
+
+    /**
+     * Sorted from most recent to oldest
+     */
+    public List<Long> getAllVersions() throws IOException {
+        List<Long> ret = new ArrayList<Long>();
+        for(String s: listDir(_root)) {
+            if(s.endsWith(FINISHED_VERSION_SUFFIX)) {
+                ret.add(validateAndGetVersion(s));
+            }
+        }
+        Collections.sort(ret);
+        Collections.reverse(ret);
+        return ret;
+    }
+
+    private String tokenPath(long version) {
+        return new File(_root, "" + version + FINISHED_VERSION_SUFFIX).getAbsolutePath();
+    }
+
+    private long validateAndGetVersion(String path) {
+        Long v = parseVersion(path);
+        if(v==null) throw new RuntimeException(path + " is not a valid version");
+        return v;
+    }
+
+    private Long parseVersion(String path) {
+        String name = new File(path).getName();
+        if(name.endsWith(FINISHED_VERSION_SUFFIX)) {
+            name = name.substring(0, name.length()-FINISHED_VERSION_SUFFIX.length());
+        }
+        try {
+            return Long.parseLong(name);
+        } catch(NumberFormatException e) {
+            return null;
+        }
+    }
+
+    private void createNewFile(String path) throws IOException {
+        new File(path).createNewFile();
+    }
+
+    private void mkdirs(String path) throws IOException {
+        new File(path).mkdirs();
+    }
+    
+    private List<String> listDir(String dir) throws IOException {
+        List<String> ret = new ArrayList<String>();
+        for(File f: new File(dir).listFiles()) {
+            ret.add(f.getAbsolutePath());
+        }
+        return ret;
+    }
+}
diff --git a/src/jvm/backtype/storm/utils/WritableUtils.java b/src/jvm/backtype/storm/utils/WritableUtils.java
new file mode 100644
index 0000000..98ff0c1
--- /dev/null
+++ b/src/jvm/backtype/storm/utils/WritableUtils.java
@@ -0,0 +1,358 @@
+/**
+ * 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.
+ */
+
+/**
+ * This file originally comes from the Apache Hadoop project. Changes have been made to the file.
+ *
+ */
+
+package backtype.storm.utils;
+
+import java.io.*;
+
+
+import java.util.zip.GZIPInputStream;
+import java.util.zip.GZIPOutputStream;
+
+public final class WritableUtils  {
+
+  public static byte[] readCompressedByteArray(DataInput in) throws IOException {
+    int length = in.readInt();
+    if (length == -1) return null;
+    byte[] buffer = new byte[length];
+    in.readFully(buffer);      // could/should use readFully(buffer,0,length)?
+    GZIPInputStream gzi = new GZIPInputStream(new ByteArrayInputStream(buffer, 0, buffer.length));
+    byte[] outbuf = new byte[length];
+    ByteArrayOutputStream bos =  new ByteArrayOutputStream();
+    int len;
+    while((len=gzi.read(outbuf, 0, outbuf.length)) != -1){
+      bos.write(outbuf, 0, len);
+    }
+    byte[] decompressed =  bos.toByteArray();
+    bos.close();
+    gzi.close();
+    return decompressed;
+  }
+
+  public static void skipCompressedByteArray(DataInput in) throws IOException {
+    int length = in.readInt();
+    if (length != -1) {
+      skipFully(in, length);
+    }
+  }
+
+  public static int  writeCompressedByteArray(DataOutput out, byte[] bytes) throws IOException {
+    if (bytes != null) {
+      ByteArrayOutputStream bos =  new ByteArrayOutputStream();
+      GZIPOutputStream gzout = new GZIPOutputStream(bos);
+      gzout.write(bytes, 0, bytes.length);
+      gzout.close();
+      byte[] buffer = bos.toByteArray();
+      int len = buffer.length;
+      out.writeInt(len);
+      out.write(buffer, 0, len);
+      /* debug only! Once we have confidence, can lose this. */
+      return ((bytes.length != 0) ? (100*buffer.length)/bytes.length : 0);
+    } else {
+      out.writeInt(-1);
+      return -1;
+    }
+  }
+
+
+  /* Ugly utility, maybe someone else can do this better  */
+  public static String readCompressedString(DataInput in) throws IOException {
+    byte[] bytes = readCompressedByteArray(in);
+    if (bytes == null) return null;
+    return new String(bytes, "UTF-8");
+  }
+
+
+  public static int  writeCompressedString(DataOutput out, String s) throws IOException {
+    return writeCompressedByteArray(out, (s != null) ? s.getBytes("UTF-8") : null);
+  }
+
+  /*
+   *
+   * Write a String as a Network Int n, followed by n Bytes
+   * Alternative to 16 bit read/writeUTF.
+   * Encoding standard is... ?
+   *
+   */
+  public static void writeString(DataOutput out, String s) throws IOException {
+    if (s != null) {
+      byte[] buffer = s.getBytes("UTF-8");
+      int len = buffer.length;
+      out.writeInt(len);
+      out.write(buffer, 0, len);
+    } else {
+      out.writeInt(-1);
+    }
+  }
+
+  /*
+   * Read a String as a Network Int n, followed by n Bytes
+   * Alternative to 16 bit read/writeUTF.
+   * Encoding standard is... ?
+   *
+   */
+  public static String readString(DataInput in) throws IOException{
+    int length = in.readInt();
+    if (length == -1) return null;
+    byte[] buffer = new byte[length];
+    in.readFully(buffer);      // could/should use readFully(buffer,0,length)?
+    return new String(buffer,"UTF-8");
+  }
+
+
+  /*
+   * Write a String array as a Nework Int N, followed by Int N Byte Array Strings.
+   * Could be generalised using introspection.
+   *
+   */
+  public static void writeStringArray(DataOutput out, String[] s) throws IOException{
+    out.writeInt(s.length);
+    for(int i = 0; i < s.length; i++) {
+      writeString(out, s[i]);
+    }
+  }
+
+  /*
+   * Write a String array as a Nework Int N, followed by Int N Byte Array of
+   * compressed Strings. Handles also null arrays and null values.
+   * Could be generalised using introspection.
+   *
+   */
+  public static void writeCompressedStringArray(DataOutput out, String[] s) throws IOException{
+    if (s == null) {
+      out.writeInt(-1);
+      return;
+    }
+    out.writeInt(s.length);
+    for(int i = 0; i < s.length; i++) {
+      writeCompressedString(out, s[i]);
+    }
+  }
+
+  /*
+   * Write a String array as a Nework Int N, followed by Int N Byte Array Strings.
+   * Could be generalised using introspection. Actually this bit couldn't...
+   *
+   */
+  public static String[] readStringArray(DataInput in) throws IOException {
+    int len = in.readInt();
+    if (len == -1) return null;
+    String[] s = new String[len];
+    for(int i = 0; i < len; i++) {
+      s[i] = readString(in);
+    }
+    return s;
+  }
+
+
+  /*
+   * Write a String array as a Nework Int N, followed by Int N Byte Array Strings.
+   * Could be generalised using introspection. Handles null arrays and null values.
+   *
+   */
+  public static  String[] readCompressedStringArray(DataInput in) throws IOException {
+    int len = in.readInt();
+    if (len == -1) return null;
+    String[] s = new String[len];
+    for(int i = 0; i < len; i++) {
+      s[i] = readCompressedString(in);
+    }
+    return s;
+  }
+
+
+  /*
+   *
+   * Test Utility Method Display Byte Array.
+   *
+   */
+  public static void displayByteArray(byte[] record){
+    int i;
+    for(i=0;i < record.length -1; i++){
+      if (i % 16 == 0) { System.out.println(); }
+      System.out.print(Integer.toHexString(record[i]  >> 4 & 0x0F));
+      System.out.print(Integer.toHexString(record[i] & 0x0F));
+      System.out.print(",");
+    }
+    System.out.print(Integer.toHexString(record[i]  >> 4 & 0x0F));
+    System.out.print(Integer.toHexString(record[i] & 0x0F));
+    System.out.println();
+  }
+
+
+  /**
+   * Serializes an integer to a binary stream with zero-compressed encoding.
+   * For -120 <= i <= 127, only one byte is used with the actual value.
+   * For other values of i, the first byte value indicates whether the
+   * integer is positive or negative, and the number of bytes that follow.
+   * If the first byte value v is between -121 and -124, the following integer
+   * is positive, with number of bytes that follow are -(v+120).
+   * If the first byte value v is between -125 and -128, the following integer
+   * is negative, with number of bytes that follow are -(v+124). Bytes are
+   * stored in the high-non-zero-byte-first order.
+   *
+   * @param stream Binary output stream
+   * @param i Integer to be serialized
+   * @throws java.io.IOException
+   */
+  public static void writeVInt(DataOutput stream, int i) throws IOException {
+    writeVLong(stream, i);
+  }
+
+  /**
+   * Serializes a long to a binary stream with zero-compressed encoding.
+   * For -112 <= i <= 127, only one byte is used with the actual value.
+   * For other values of i, the first byte value indicates whether the
+   * long is positive or negative, and the number of bytes that follow.
+   * If the first byte value v is between -113 and -120, the following long
+   * is positive, with number of bytes that follow are -(v+112).
+   * If the first byte value v is between -121 and -128, the following long
+   * is negative, with number of bytes that follow are -(v+120). Bytes are
+   * stored in the high-non-zero-byte-first order.
+   *
+   * @param stream Binary output stream
+   * @param i Long to be serialized
+   * @throws java.io.IOException
+   */
+  public static void writeVLong(DataOutput stream, long i) throws IOException {
+    if (i >= -112 && i <= 127) {
+      stream.writeByte((byte)i);
+      return;
+    }
+
+    int len = -112;
+    if (i < 0) {
+      i ^= -1L; // take one's complement'
+      len = -120;
+    }
+
+    long tmp = i;
+    while (tmp != 0) {
+      tmp = tmp >> 8;
+      len--;
+    }
+
+    stream.writeByte((byte)len);
+
+    len = (len < -120) ? -(len + 120) : -(len + 112);
+
+    for (int idx = len; idx != 0; idx--) {
+      int shiftbits = (idx - 1) * 8;
+      long mask = 0xFFL << shiftbits;
+      stream.writeByte((byte)((i & mask) >> shiftbits));
+    }
+  }
+
+
+  /**
+   * Reads a zero-compressed encoded long from input stream and returns it.
+   * @param stream Binary input stream
+   * @throws java.io.IOException
+   * @return deserialized long from stream.
+   */
+  public static long readVLong(DataInput stream) throws IOException {
+    byte firstByte = stream.readByte();
+    int len = decodeVIntSize(firstByte);
+    if (len == 1) {
+      return firstByte;
+    }
+    long i = 0;
+    for (int idx = 0; idx < len-1; idx++) {
+      byte b = stream.readByte();
+      i = i << 8;
+      i = i | (b & 0xFF);
+    }
+    return (isNegativeVInt(firstByte) ? (i ^ -1L) : i);
+  }
+
+  /**
+   * Reads a zero-compressed encoded integer from input stream and returns it.
+   * @param stream Binary input stream
+   * @throws java.io.IOException
+   * @return deserialized integer from stream.
+   */
+  public static int readVInt(DataInput stream) throws IOException {
+    return (int) readVLong(stream);
+  }
+
+  /**
+   * Given the first byte of a vint/vlong, determine the sign
+   * @param value the first byte
+   * @return is the value negative
+   */
+  public static boolean isNegativeVInt(byte value) {
+    return value < -120 || (value >= -112 && value < 0);
+  }
+
+  /**
+   * Parse the first byte of a vint/vlong to determine the number of bytes
+   * @param value the first byte of the vint/vlong
+   * @return the total number of bytes (1 to 9)
+   */
+  public static int decodeVIntSize(byte value) {
+    if (value >= -112) {
+      return 1;
+    } else if (value < -120) {
+      return -119 - value;
+    }
+    return -111 - value;
+  }
+
+  /**
+   * Get the encoded length if an integer is stored in a variable-length format
+   * @return the encoded length
+   */
+  public static int getVIntSize(long i) {
+    if (i >= -112 && i <= 127) {
+      return 1;
+    }
+
+    if (i < 0) {
+      i ^= -1L; // take one's complement'
+    }
+    // find the number of bytes with non-leading zeros
+    int dataBits = Long.SIZE - Long.numberOfLeadingZeros(i);
+    // find the number of data bytes + length byte
+    return (dataBits + 7) / 8 + 1;
+  }
+
+  /**
+   * Skip <i>len</i> number of bytes in input stream<i>in</i>
+   * @param in input stream
+   * @param len number of bytes to skip
+   * @throws IOException when skipped less number of bytes
+   */
+  public static void skipFully(DataInput in, int len) throws IOException {
+    int total = 0;
+    int cur = 0;
+
+    while ((total<len) && ((cur = in.skipBytes(len-total)) > 0)) {
+        total += cur;
+    }
+
+    if (total<len) {
+      throw new IOException("Not able to skip " + len + " bytes, possibly " +
+                            "due to end of input.");
+    }
+  }
+}
diff --git a/src/multilang/fy/storm.fancypack b/src/multilang/fy/storm.fancypack
new file mode 100644
index 0000000..ba091f5
--- /dev/null
+++ b/src/multilang/fy/storm.fancypack
@@ -0,0 +1,9 @@
+Fancy Package Specification new: "storm" with: {
+  author: "Christopher Bertels"
+  email: "chris@fancy-lang.org"
+  include_files: ["storm.fy"]
+  description: """Fancy library for Storm, a computation system for distributed, scalable, and fault-tolerant stream processing."""
+  homepage: "http://www.backtype.com"
+  version: "0.1.0"
+  ruby_dependencies: [["json"]]
+}
diff --git a/src/multilang/fy/storm.fy b/src/multilang/fy/storm.fy
new file mode 100644
index 0000000..4a1b55a
--- /dev/null
+++ b/src/multilang/fy/storm.fy
@@ -0,0 +1,163 @@
+require("rubygems")
+require("json")
+
+class Storm {
+  class Protocol {
+    """
+    Storm Protocol class.
+    Contains all methods implementing the Storm multilang protocol using stdio.
+    """
+
+    Input = STDIN
+    Output = STDOUT
+
+    def read_string_message {
+      """
+      @return @String@ message send by the parent Storm process.
+      """
+
+      msg = ""
+      loop: {
+        line = Input readline chomp
+        { break } if: (line == "end")
+        msg << line
+        msg << "\n"
+      }
+      msg chomp
+    }
+
+    def read_message {
+      """
+      @return @Hash@ that is a JSON parsed message from the parent Storm process.
+      """
+
+      JSON parse(read_string_message)
+    }
+
+    def send: message {
+      """
+      @message Message to be sent to the parent Storm process converted to JSON.
+
+      Sends a message as JSON to the parent Storm process.
+      """
+
+      Output println: $ message to_json()
+      Output println: "end"
+      Output flush
+    }
+
+    def sync {
+      Output println: "sync"
+      Output flush
+    }
+
+    def send_pid: heartbeat_dir {
+      pid = Process pid()
+      Output println: pid
+      Output flush
+      File open(heartbeat_dir ++ "/" ++ pid, "w") close
+    }
+
+    def emit_tuple: tup stream: stream (nil) anchors: anchors ([]) direct: direct (nil) {
+      m = <['command => 'emit, 'anchors => anchors map: 'id, 'tuple => tup]>
+      { m['stream]: stream } if: stream
+      { m['task]: direct } if: direct
+      send: m
+    }
+
+    def emit: tup stream: stream (nil) anchors: anchors ([]) direct: direct (nil) {
+      emit_tuple: tup stream: stream anchors: anchors direct: direct
+      read_message
+    }
+
+    def ack: tuple {
+      """
+      @tuple @Storm Tuple@ to be acked by Storm.
+      """
+
+      send: <['command => 'ack, 'id => tuple id]>
+    }
+
+    def fail: tuple {
+      """
+      @tuple @Storm Tuple@ to be failed by Storm.
+      """
+
+      send: <['command => 'fail, 'id => tuple id]>
+    }
+
+    def log: message {
+      """
+      @message Message to be logged by Storm.
+      """
+
+      send: <['command => 'log, 'msg => message to_s]>
+    }
+
+    def read_env {
+      """
+      @return @Tuple@ of Storm (config, context).
+      """
+
+      (read_message, read_message)
+    }
+  }
+
+  class Tuple {
+    """
+    Tuples are used by storm as principal data component sent between bolts and emitted by spouts.
+    Contains a unique id, the component, stream and task it came from and the values associated with it.
+    """
+
+    read_write_slots: [ 'id, 'component, 'stream, 'task, 'values ]
+
+    def initialize: @id component: @component stream: @stream task: @task values: @values {}
+
+    def Tuple from_hash: hash {
+      """
+      @hash @Hash@ of values to be used for a new @Storm Tuple@.
+      @return A new @Storm Tuple@ based on the values in @hash.
+
+      Helper method to create a new tuple from a @Hash@.
+      """
+
+      id, component, stream, task, values = hash values_at: ("id", "comp", "stream", "task", "tuple")
+      Tuple new: id component: component stream: stream task: task values: values
+    }
+  }
+
+  class Bolt {
+    """
+    Bolts represent the actual work processes that receive tuples and
+    emit new @Storm Tuple@s on their output stream (possible consumed by other Bolts).
+    """
+
+    include: Storm Protocol
+
+    def initialize: @conf (nil) context: @context (nil) {}
+
+    def process: tuple {}
+
+    def run {
+      """
+      Runs the bolt, causing it to receive messages, perform work defined in @Bolt#run
+      and possibly emit new messages (@Storm Tuple@s).
+      """
+
+      heartbeat_dir = read_string_message
+      send_pid: heartbeat_dir
+      @conf, @context = read_env
+      try {
+        loop: {
+          process: $ Tuple from_hash: read_message
+          sync
+        }
+      } catch Exception => e {
+        log: e
+      }
+    }
+  }
+
+  class Spout {
+  }
+}
\ No newline at end of file
diff --git a/src/multilang/py/storm.py b/src/multilang/py/storm.py
new file mode 100755
index 0000000..6bb6d04
--- /dev/null
+++ b/src/multilang/py/storm.py
@@ -0,0 +1,143 @@
+import sys
+import os
+import traceback
+
+try:
+    import cjson
+    json_encode = cjson.encode
+    json_decode = lambda x: cjson.decode(x, all_unicode=True)
+except ImportError:
+    import json
+    json_encode = lambda x: json.dumps(x, ensure_ascii=False)
+    json_decode = lambda x: json.loads(unicode(x))
+
+def readStringMsg():
+    msg = ""
+    while True:
+        line = sys.stdin.readline()[0:-1]
+        if line == "end":
+            break
+        msg = msg + line + "\n"
+    return msg[0:-1]
+
+ANCHOR_TUPLE = None
+
+#reads lines and reconstructs newlines appropriately
+def readMsg():
+    return json_decode(readStringMsg())
+
+def sendToParent(s):
+    print s
+    print "end"
+    sys.stdout.flush()
+    
+def sync():
+    print "sync"
+    sys.stdout.flush()
+
+def sendpid(heartbeatdir):
+    pid = os.getpid()
+    print pid
+    sys.stdout.flush()
+    open(heartbeatdir + "/" + str(pid), "w").close()    
+
+def sendMsgToParent(amap):
+    sendToParent(json_encode(amap))
+
+def emittuple(tup, stream=None, anchors = [], directTask=None):
+    global ANCHOR_TUPLE
+    if ANCHOR_TUPLE is not None:
+        anchors = [ANCHOR_TUPLE]
+    m = {"command": "emit"}
+    if stream is not None:
+        m["stream"] = stream
+    m["anchors"] = map(lambda a: a.id, anchors)
+    if directTask is not None:
+        m["task"] = directTask
+    m["tuple"] = tup
+    sendMsgToParent(m)
+    
+def emit(tup, stream=None, anchors = []):
+    emittuple(tup, stream=stream, anchors=anchors)
+    #read back task ids
+    return readMsg()
+    
+def emitDirect(task, tup, stream=None, anchors = []):
+    emittuple(tup, stream=stream, anchors=anchors, directTask=task)
+
+def ack(tup):
+    sendMsgToParent({"command": "ack", "id": tup.id})
+
+def fail(tup):
+    sendMsgToParent({"command": "fail", "id": tup.id})
+
+def log(msg):
+    sendMsgToParent({"command": "log", "msg": msg})
+
+# read the stormconf and context
+def readenv():
+    conf = readMsg()
+    context = readMsg()
+    return [conf, context]
+
+def readtuple():
+    tupmap = readMsg()
+    return Tuple(tupmap["id"], tupmap["comp"], tupmap["stream"], tupmap["task"], tupmap["tuple"])
+
+def initbolt():
+    heartbeatdir = readStringMsg()
+    sendpid(heartbeatdir)
+    return readenv()
+
+class Tuple:    
+    def __init__(self, id, component, stream, task, values):
+        self.id = id
+        self.component = component
+        self.stream = stream
+        self.task = task
+        self.values = values
+
+class Bolt:
+    def initialize(self, stormconf, context):
+        pass
+    
+    def process(self, tuple):
+        pass
+    
+    def run(self):
+        conf, context = initbolt()
+        self.initialize(conf, context)
+        try:
+            while True:
+                tup = readtuple()
+                self.process(tup)
+                sync()
+        except Exception, e:
+            log(traceback.format_exc(e))        
+
+class BasicBolt:
+    def initialize(self, stormconf, context):
+        pass
+    
+    def process(self, tuple):
+        pass
+    
+    def run(self):
+        global ANCHOR_TUPLE
+        conf, context = initbolt()
+        self.initialize(conf, context)
+        try:
+            while True:
+                tup = readtuple()
+                ANCHOR_TUPLE = tup
+                self.process(tup)
+                ack(tup)
+                sync()
+        except Exception, e:
+            log(traceback.format_exc(e))
+
+class Spout:
+    pass
+
+
+
diff --git a/src/multilang/rb/storm.rb b/src/multilang/rb/storm.rb
new file mode 100644
index 0000000..32c17b6
--- /dev/null
+++ b/src/multilang/rb/storm.rb
@@ -0,0 +1,116 @@
+require "rubygems"
+require "json"
+
+module Storm
+  module Protocol
+    def read_string_message
+      msg = ""
+      loop do
+        line = STDIN.readline.chomp
+        break if line == "end"
+        msg << line
+        msg << "\n"
+      end
+      msg.chomp
+    end
+
+    def read_message
+      JSON.parse read_string_message
+    end
+
+    def send_to_parent(s)
+      puts s
+      puts "end"
+      STDOUT.flush
+    end
+
+    def sync
+      puts "sync"
+      STDOUT.flush
+    end
+
+    def send_pid(heartbeat_dir)
+      pid = Process.pid
+      puts pid
+      STDOUT.flush
+      File.open("#{heartbeat_dir}/#{pid}", "w").close
+    end
+
+    def send_msg_to_parent(hash)
+      send_to_parent hash.to_json
+    end
+
+    def emit_tuple(tup, stream = nil, anchors = [], direct = nil)
+      m = {:command => :emit, :anchors => anchors.map(&:id), :tuple => tup}
+      m[:stream] = stream if stream
+      m[:task] = direct if direct
+      send_msg_to_parent m
+    end
+
+    def emit(tup, stream = nil, anchors = [], direct = nil)
+      emit_tuple tup, stream, anchors, direct
+      read_message
+    end
+
+    def ack(tup)
+      send_msg_to_parent :command => :ack, :id => tup.id
+    end
+
+    def fail(tup)
+      send_msg_to_parent :command => :fail, :id => tup.id
+    end
+
+    def log(msg)
+      send_msg_to_parent :command => :log, :msg => msg.to_s
+    end
+
+    def read_env
+      [read_message, read_message]
+    end
+  end
+
+  class Tuple
+    attr_accessor :id, :component, :stream, :task, :values
+
+    def initialize(id, component, stream, task, values)
+      @id = id
+      @component = component
+      @stream = stream
+      @task = task
+      @values = values
+    end
+
+    def self.from_hash(hash)
+      Tuple.new(*hash.values_at("id", "comp", "stream", "task", "tuple"))
+    end
+  end
+
+  class Bolt
+    include Storm::Protocol
+
+    def initialize(conf = nil, context = nil)
+      @conf = conf
+      @context = context
+    end
+
+    def process(tuple)
+    end
+
+    def run
+      heartbeat_dir = read_string_message
+      send_pid heartbeat_dir
+      @conf, @context = read_env
+      begin
+        while true
+          process Tuple.from_hash(read_message)
+          sync
+        end
+      rescue Exception => e
+        log e
+      end
+    end
+  end
+
+  class Spout
+  end
+end
diff --git a/src/py/__init__.py b/src/py/__init__.py
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/src/py/__init__.py
diff --git a/src/py/storm/DistributedRPC-remote b/src/py/storm/DistributedRPC-remote
new file mode 100755
index 0000000..74518ba
--- /dev/null
+++ b/src/py/storm/DistributedRPC-remote
@@ -0,0 +1,86 @@
+#!/usr/bin/env python
+#
+# Autogenerated by Thrift
+#
+# DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+#
+
+import sys
+import pprint
+from urlparse import urlparse
+from thrift.transport import TTransport
+from thrift.transport import TSocket
+from thrift.transport import THttpClient
+from thrift.protocol import TBinaryProtocol
+
+import DistributedRPC
+from ttypes import *
+
+if len(sys.argv) <= 1 or sys.argv[1] == '--help':
+  print ''
+  print 'Usage: ' + sys.argv[0] + ' [-h host:port] [-u url] [-f[ramed]] function [arg1 [arg2...]]'
+  print ''
+  print 'Functions:'
+  print '  string execute(string functionName, string funcArgs)'
+  print '  void result(string id, string result)'
+  print ''
+  sys.exit(0)
+
+pp = pprint.PrettyPrinter(indent = 2)
+host = 'localhost'
+port = 9090
+uri = ''
+framed = False
+http = False
+argi = 1
+
+if sys.argv[argi] == '-h':
+  parts = sys.argv[argi+1].split(':') 
+  host = parts[0]
+  port = int(parts[1])
+  argi += 2
+
+if sys.argv[argi] == '-u':
+  url = urlparse(sys.argv[argi+1])
+  parts = url[1].split(':') 
+  host = parts[0]
+  if len(parts) > 1:
+    port = int(parts[1])
+  else:
+    port = 80
+  uri = url[2]
+  http = True
+  argi += 2
+
+if sys.argv[argi] == '-f' or sys.argv[argi] == '-framed':
+  framed = True
+  argi += 1
+
+cmd = sys.argv[argi]
+args = sys.argv[argi+1:]
+
+if http:
+  transport = THttpClient.THttpClient(host, port, uri)
+else:
+  socket = TSocket.TSocket(host, port)
+  if framed:
+    transport = TTransport.TFramedTransport(socket)
+  else:
+    transport = TTransport.TBufferedTransport(socket)
+protocol = TBinaryProtocol.TBinaryProtocol(transport)
+client = DistributedRPC.Client(protocol)
+transport.open()
+
+if cmd == 'execute':
+  if len(args) != 2:
+    print 'execute requires 2 args'
+    sys.exit(1)
+  pp.pprint(client.execute(args[0],args[1],))
+
+elif cmd == 'result':
+  if len(args) != 2:
+    print 'result requires 2 args'
+    sys.exit(1)
+  pp.pprint(client.result(args[0],args[1],))
+
+transport.close()
diff --git a/src/py/storm/DistributedRPC.py b/src/py/storm/DistributedRPC.py
new file mode 100644
index 0000000..e6eea6d
--- /dev/null
+++ b/src/py/storm/DistributedRPC.py
@@ -0,0 +1,459 @@
+#
+# Autogenerated by Thrift
+#
+# DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+#
+
+from thrift.Thrift import *
+import sys
+from ttypes import *
+from thrift.Thrift import TProcessor
+from thrift.transport import TTransport
+from thrift.protocol import TBinaryProtocol
+try:
+  from thrift.protocol import fastbinary
+except:
+  fastbinary = None
+
+
+class Iface:
+  def execute(self, functionName, funcArgs):
+    """
+    Parameters:
+     - functionName
+     - funcArgs
+    """
+    pass
+
+  def result(self, id, result):
+    """
+    Parameters:
+     - id
+     - result
+    """
+    pass
+
+
+class Client(Iface):
+  def __init__(self, iprot, oprot=None):
+    self._iprot = self._oprot = iprot
+    if oprot != None:
+      self._oprot = oprot
+    self._seqid = 0
+
+  def execute(self, functionName, funcArgs):
+    """
+    Parameters:
+     - functionName
+     - funcArgs
+    """
+    self.send_execute(functionName, funcArgs)
+    return self.recv_execute()
+
+  def send_execute(self, functionName, funcArgs):
+    self._oprot.writeMessageBegin('execute', TMessageType.CALL, self._seqid)
+    args = execute_args()
+    args.functionName = functionName
+    args.funcArgs = funcArgs
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_execute(self, ):
+    (fname, mtype, rseqid) = self._iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(self._iprot)
+      self._iprot.readMessageEnd()
+      raise x
+    result = execute_result()
+    result.read(self._iprot)
+    self._iprot.readMessageEnd()
+    if result.success != None:
+      return result.success
+    raise TApplicationException(TApplicationException.MISSING_RESULT, "execute failed: unknown result");
+
+  def result(self, id, result):
+    """
+    Parameters:
+     - id
+     - result
+    """
+    self.send_result(id, result)
+    self.recv_result()
+
+  def send_result(self, id, result):
+    self._oprot.writeMessageBegin('result', TMessageType.CALL, self._seqid)
+    args = result_args()
+    args.id = id
+    args.result = result
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_result(self, ):
+    (fname, mtype, rseqid) = self._iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(self._iprot)
+      self._iprot.readMessageEnd()
+      raise x
+    result = result_result()
+    result.read(self._iprot)
+    self._iprot.readMessageEnd()
+    return
+
+
+class Processor(Iface, TProcessor):
+  def __init__(self, handler):
+    self._handler = handler
+    self._processMap = {}
+    self._processMap["execute"] = Processor.process_execute
+    self._processMap["result"] = Processor.process_result
+
+  def process(self, iprot, oprot):
+    (name, type, seqid) = iprot.readMessageBegin()
+    if name not in self._processMap:
+      iprot.skip(TType.STRUCT)
+      iprot.readMessageEnd()
+      x = TApplicationException(TApplicationException.UNKNOWN_METHOD, 'Unknown function %s' % (name))
+      oprot.writeMessageBegin(name, TMessageType.EXCEPTION, seqid)
+      x.write(oprot)
+      oprot.writeMessageEnd()
+      oprot.trans.flush()
+      return
+    else:
+      self._processMap[name](self, seqid, iprot, oprot)
+    return True
+
+  def process_execute(self, seqid, iprot, oprot):
+    args = execute_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = execute_result()
+    result.success = self._handler.execute(args.functionName, args.funcArgs)
+    oprot.writeMessageBegin("execute", TMessageType.REPLY, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
+  def process_result(self, seqid, iprot, oprot):
+    args = result_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = result_result()
+    self._handler.result(args.id, args.result)
+    oprot.writeMessageBegin("result", TMessageType.REPLY, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
+
+# HELPER FUNCTIONS AND STRUCTURES
+
+class execute_args:
+  """
+  Attributes:
+   - functionName
+   - funcArgs
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'functionName', None, None, ), # 1
+    (2, TType.STRING, 'funcArgs', None, None, ), # 2
+  )
+
+  def __init__(self, functionName=None, funcArgs=None,):
+    self.functionName = functionName
+    self.funcArgs = funcArgs
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRING:
+          self.functionName = iprot.readString().decode('utf-8');
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRING:
+          self.funcArgs = iprot.readString().decode('utf-8');
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('execute_args')
+    if self.functionName != None:
+      oprot.writeFieldBegin('functionName', TType.STRING, 1)
+      oprot.writeString(self.functionName.encode('utf-8'));
+      oprot.writeFieldEnd()
+    if self.funcArgs != None:
+      oprot.writeFieldBegin('funcArgs', TType.STRING, 2)
+      oprot.writeString(self.funcArgs.encode('utf-8'));
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+  def union_value(self):
+    d = self.__dict__
+    for key in d:
+      val = d[key]
+      if not val is None:
+        return val
+
+  def get_set_field(self):
+    for attr in self.__dict__:
+      if not self.__dict__[attr] is None:
+        return attr
+
+  def get_set_field_id(self):
+    for idx, tup in enumerate(self.__class__.thrift_spec):
+      if tup:
+        key = tup[2]
+        if not self.__dict__[key] is None:
+          return idx
+
+class execute_result:
+  """
+  Attributes:
+   - success
+  """
+
+  thrift_spec = (
+    (0, TType.STRING, 'success', None, None, ), # 0
+  )
+
+  def __init__(self, success=None,):
+    self.success = success
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 0:
+        if ftype == TType.STRING:
+          self.success = iprot.readString().decode('utf-8');
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('execute_result')
+    if self.success != None:
+      oprot.writeFieldBegin('success', TType.STRING, 0)
+      oprot.writeString(self.success.encode('utf-8'));
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+  def union_value(self):
+    d = self.__dict__
+    for key in d:
+      val = d[key]
+      if not val is None:
+        return val
+
+  def get_set_field(self):
+    for attr in self.__dict__:
+      if not self.__dict__[attr] is None:
+        return attr
+
+  def get_set_field_id(self):
+    for idx, tup in enumerate(self.__class__.thrift_spec):
+      if tup:
+        key = tup[2]
+        if not self.__dict__[key] is None:
+          return idx
+
+class result_args:
+  """
+  Attributes:
+   - id
+   - result
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'id', None, None, ), # 1
+    (2, TType.STRING, 'result', None, None, ), # 2
+  )
+
+  def __init__(self, id=None, result=None,):
+    self.id = id
+    self.result = result
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRING:
+          self.id = iprot.readString().decode('utf-8');
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRING:
+          self.result = iprot.readString().decode('utf-8');
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('result_args')
+    if self.id != None:
+      oprot.writeFieldBegin('id', TType.STRING, 1)
+      oprot.writeString(self.id.encode('utf-8'));
+      oprot.writeFieldEnd()
+    if self.result != None:
+      oprot.writeFieldBegin('result', TType.STRING, 2)
+      oprot.writeString(self.result.encode('utf-8'));
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+  def union_value(self):
+    d = self.__dict__
+    for key in d:
+      val = d[key]
+      if not val is None:
+        return val
+
+  def get_set_field(self):
+    for attr in self.__dict__:
+      if not self.__dict__[attr] is None:
+        return attr
+
+  def get_set_field_id(self):
+    for idx, tup in enumerate(self.__class__.thrift_spec):
+      if tup:
+        key = tup[2]
+        if not self.__dict__[key] is None:
+          return idx
+
+class result_result:
+
+  thrift_spec = (
+  )
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('result_result')
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+  def union_value(self):
+    d = self.__dict__
+    for key in d:
+      val = d[key]
+      if not val is None:
+        return val
+
+  def get_set_field(self):
+    for attr in self.__dict__:
+      if not self.__dict__[attr] is None:
+        return attr
+
+  def get_set_field_id(self):
+    for idx, tup in enumerate(self.__class__.thrift_spec):
+      if tup:
+        key = tup[2]
+        if not self.__dict__[key] is None:
+          return idx
+
+
diff --git a/src/py/storm/Nimbus-remote b/src/py/storm/Nimbus-remote
new file mode 100755
index 0000000..95712ea
--- /dev/null
+++ b/src/py/storm/Nimbus-remote
@@ -0,0 +1,149 @@
+#!/usr/bin/env python
+#
+# Autogenerated by Thrift
+#
+# DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+#
+
+import sys
+import pprint
+from urlparse import urlparse
+from thrift.transport import TTransport
+from thrift.transport import TSocket
+from thrift.transport import THttpClient
+from thrift.protocol import TBinaryProtocol
+
+import Nimbus
+from ttypes import *
+
+if len(sys.argv) <= 1 or sys.argv[1] == '--help':
+  print ''
+  print 'Usage: ' + sys.argv[0] + ' [-h host:port] [-u url] [-f[ramed]] function [arg1 [arg2...]]'
+  print ''
+  print 'Functions:'
+  print '  void submitTopology(string name, string uploadedJarLocation, string jsonConf, StormTopology topology)'
+  print '  void killTopology(string name)'
+  print '  string beginFileUpload()'
+  print '  void uploadChunk(string location, string chunk)'
+  print '  void finishFileUpload(string location)'
+  print '  string beginFileDownload(string file)'
+  print '  string downloadChunk(string id)'
+  print '  ClusterSummary getClusterInfo()'
+  print '  TopologyInfo getTopologyInfo(string id)'
+  print '  string getTopologyConf(string id)'
+  print '  StormTopology getTopology(string id)'
+  print ''
+  sys.exit(0)
+
+pp = pprint.PrettyPrinter(indent = 2)
+host = 'localhost'
+port = 9090
+uri = ''
+framed = False
+http = False
+argi = 1
+
+if sys.argv[argi] == '-h':
+  parts = sys.argv[argi+1].split(':') 
+  host = parts[0]
+  port = int(parts[1])
+  argi += 2
+
+if sys.argv[argi] == '-u':
+  url = urlparse(sys.argv[argi+1])
+  parts = url[1].split(':') 
+  host = parts[0]
+  if len(parts) > 1:
+    port = int(parts[1])
+  else:
+    port = 80
+  uri = url[2]
+  http = True
+  argi += 2
+
+if sys.argv[argi] == '-f' or sys.argv[argi] == '-framed':
+  framed = True
+  argi += 1
+
+cmd = sys.argv[argi]
+args = sys.argv[argi+1:]
+
+if http:
+  transport = THttpClient.THttpClient(host, port, uri)
+else:
+  socket = TSocket.TSocket(host, port)
+  if framed:
+    transport = TTransport.TFramedTransport(socket)
+  else:
+    transport = TTransport.TBufferedTransport(socket)
+protocol = TBinaryProtocol.TBinaryProtocol(transport)
+client = Nimbus.Client(protocol)
+transport.open()
+
+if cmd == 'submitTopology':
+  if len(args) != 4:
+    print 'submitTopology requires 4 args'
+    sys.exit(1)
+  pp.pprint(client.submitTopology(args[0],args[1],args[2],eval(args[3]),))
+
+elif cmd == 'killTopology':
+  if len(args) != 1:
+    print 'killTopology requires 1 args'
+    sys.exit(1)
+  pp.pprint(client.killTopology(args[0],))
+
+elif cmd == 'beginFileUpload':
+  if len(args) != 0:
+    print 'beginFileUpload requires 0 args'
+    sys.exit(1)
+  pp.pprint(client.beginFileUpload())
+
+elif cmd == 'uploadChunk':
+  if len(args) != 2:
+    print 'uploadChunk requires 2 args'
+    sys.exit(1)
+  pp.pprint(client.uploadChunk(args[0],args[1],))
+
+elif cmd == 'finishFileUpload':
+  if len(args) != 1:
+    print 'finishFileUpload requires 1 args'
+    sys.exit(1)
+  pp.pprint(client.finishFileUpload(args[0],))
+
+elif cmd == 'beginFileDownload':
+  if len(args) != 1:
+    print 'beginFileDownload requires 1 args'
+    sys.exit(1)
+  pp.pprint(client.beginFileDownload(args[0],))
+
+elif cmd == 'downloadChunk':
+  if len(args) != 1:
+    print 'downloadChunk requires 1 args'
+    sys.exit(1)
+  pp.pprint(client.downloadChunk(args[0],))
+
+elif cmd == 'getClusterInfo':
+  if len(args) != 0:
+    print 'getClusterInfo requires 0 args'
+    sys.exit(1)
+  pp.pprint(client.getClusterInfo())
+
+elif cmd == 'getTopologyInfo':
+  if len(args) != 1:
+    print 'getTopologyInfo requires 1 args'
+    sys.exit(1)
+  pp.pprint(client.getTopologyInfo(args[0],))
+
+elif cmd == 'getTopologyConf':
+  if len(args) != 1:
+    print 'getTopologyConf requires 1 args'
+    sys.exit(1)
+  pp.pprint(client.getTopologyConf(args[0],))
+
+elif cmd == 'getTopology':
+  if len(args) != 1:
+    print 'getTopology requires 1 args'
+    sys.exit(1)
+  pp.pprint(client.getTopology(args[0],))
+
+transport.close()
diff --git a/src/py/storm/Nimbus.py b/src/py/storm/Nimbus.py
new file mode 100644
index 0000000..3629f9d
--- /dev/null
+++ b/src/py/storm/Nimbus.py
@@ -0,0 +1,2283 @@
+#
+# Autogenerated by Thrift
+#
+# DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+#
+
+from thrift.Thrift import *
+import sys
+from ttypes import *
+from thrift.Thrift import TProcessor
+from thrift.transport import TTransport
+from thrift.protocol import TBinaryProtocol
+try:
+  from thrift.protocol import fastbinary
+except:
+  fastbinary = None
+
+
+class Iface:
+  def submitTopology(self, name, uploadedJarLocation, jsonConf, topology):
+    """
+    Parameters:
+     - name
+     - uploadedJarLocation
+     - jsonConf
+     - topology
+    """
+    pass
+
+  def killTopology(self, name):
+    """
+    Parameters:
+     - name
+    """
+    pass
+
+  def beginFileUpload(self, ):
+    pass
+
+  def uploadChunk(self, location, chunk):
+    """
+    Parameters:
+     - location
+     - chunk
+    """
+    pass
+
+  def finishFileUpload(self, location):
+    """
+    Parameters:
+     - location
+    """
+    pass
+
+  def beginFileDownload(self, file):
+    """
+    Parameters:
+     - file
+    """
+    pass
+
+  def downloadChunk(self, id):
+    """
+    Parameters:
+     - id
+    """
+    pass
+
+  def getClusterInfo(self, ):
+    pass
+
+  def getTopologyInfo(self, id):
+    """
+    Parameters:
+     - id
+    """
+    pass
+
+  def getTopologyConf(self, id):
+    """
+    Parameters:
+     - id
+    """
+    pass
+
+  def getTopology(self, id):
+    """
+    Parameters:
+     - id
+    """
+    pass
+
+
+class Client(Iface):
+  def __init__(self, iprot, oprot=None):
+    self._iprot = self._oprot = iprot
+    if oprot != None:
+      self._oprot = oprot
+    self._seqid = 0
+
+  def submitTopology(self, name, uploadedJarLocation, jsonConf, topology):
+    """
+    Parameters:
+     - name
+     - uploadedJarLocation
+     - jsonConf
+     - topology
+    """
+    self.send_submitTopology(name, uploadedJarLocation, jsonConf, topology)
+    self.recv_submitTopology()
+
+  def send_submitTopology(self, name, uploadedJarLocation, jsonConf, topology):
+    self._oprot.writeMessageBegin('submitTopology', TMessageType.CALL, self._seqid)
+    args = submitTopology_args()
+    args.name = name
+    args.uploadedJarLocation = uploadedJarLocation
+    args.jsonConf = jsonConf
+    args.topology = topology
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_submitTopology(self, ):
+    (fname, mtype, rseqid) = self._iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(self._iprot)
+      self._iprot.readMessageEnd()
+      raise x
+    result = submitTopology_result()
+    result.read(self._iprot)
+    self._iprot.readMessageEnd()
+    if result.e != None:
+      raise result.e
+    if result.ite != None:
+      raise result.ite
+    return
+
+  def killTopology(self, name):
+    """
+    Parameters:
+     - name
+    """
+    self.send_killTopology(name)
+    self.recv_killTopology()
+
+  def send_killTopology(self, name):
+    self._oprot.writeMessageBegin('killTopology', TMessageType.CALL, self._seqid)
+    args = killTopology_args()
+    args.name = name
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_killTopology(self, ):
+    (fname, mtype, rseqid) = self._iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(self._iprot)
+      self._iprot.readMessageEnd()
+      raise x
+    result = killTopology_result()
+    result.read(self._iprot)
+    self._iprot.readMessageEnd()
+    if result.e != None:
+      raise result.e
+    return
+
+  def beginFileUpload(self, ):
+    self.send_beginFileUpload()
+    return self.recv_beginFileUpload()
+
+  def send_beginFileUpload(self, ):
+    self._oprot.writeMessageBegin('beginFileUpload', TMessageType.CALL, self._seqid)
+    args = beginFileUpload_args()
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_beginFileUpload(self, ):
+    (fname, mtype, rseqid) = self._iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(self._iprot)
+      self._iprot.readMessageEnd()
+      raise x
+    result = beginFileUpload_result()
+    result.read(self._iprot)
+    self._iprot.readMessageEnd()
+    if result.success != None:
+      return result.success
+    raise TApplicationException(TApplicationException.MISSING_RESULT, "beginFileUpload failed: unknown result");
+
+  def uploadChunk(self, location, chunk):
+    """
+    Parameters:
+     - location
+     - chunk
+    """
+    self.send_uploadChunk(location, chunk)
+    self.recv_uploadChunk()
+
+  def send_uploadChunk(self, location, chunk):
+    self._oprot.writeMessageBegin('uploadChunk', TMessageType.CALL, self._seqid)
+    args = uploadChunk_args()
+    args.location = location
+    args.chunk = chunk
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_uploadChunk(self, ):
+    (fname, mtype, rseqid) = self._iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(self._iprot)
+      self._iprot.readMessageEnd()
+      raise x
+    result = uploadChunk_result()
+    result.read(self._iprot)
+    self._iprot.readMessageEnd()
+    return
+
+  def finishFileUpload(self, location):
+    """
+    Parameters:
+     - location
+    """
+    self.send_finishFileUpload(location)
+    self.recv_finishFileUpload()
+
+  def send_finishFileUpload(self, location):
+    self._oprot.writeMessageBegin('finishFileUpload', TMessageType.CALL, self._seqid)
+    args = finishFileUpload_args()
+    args.location = location
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_finishFileUpload(self, ):
+    (fname, mtype, rseqid) = self._iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(self._iprot)
+      self._iprot.readMessageEnd()
+      raise x
+    result = finishFileUpload_result()
+    result.read(self._iprot)
+    self._iprot.readMessageEnd()
+    return
+
+  def beginFileDownload(self, file):
+    """
+    Parameters:
+     - file
+    """
+    self.send_beginFileDownload(file)
+    return self.recv_beginFileDownload()
+
+  def send_beginFileDownload(self, file):
+    self._oprot.writeMessageBegin('beginFileDownload', TMessageType.CALL, self._seqid)
+    args = beginFileDownload_args()
+    args.file = file
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_beginFileDownload(self, ):
+    (fname, mtype, rseqid) = self._iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(self._iprot)
+      self._iprot.readMessageEnd()
+      raise x
+    result = beginFileDownload_result()
+    result.read(self._iprot)
+    self._iprot.readMessageEnd()
+    if result.success != None:
+      return result.success
+    raise TApplicationException(TApplicationException.MISSING_RESULT, "beginFileDownload failed: unknown result");
+
+  def downloadChunk(self, id):
+    """
+    Parameters:
+     - id
+    """
+    self.send_downloadChunk(id)
+    return self.recv_downloadChunk()
+
+  def send_downloadChunk(self, id):
+    self._oprot.writeMessageBegin('downloadChunk', TMessageType.CALL, self._seqid)
+    args = downloadChunk_args()
+    args.id = id
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_downloadChunk(self, ):
+    (fname, mtype, rseqid) = self._iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(self._iprot)
+      self._iprot.readMessageEnd()
+      raise x
+    result = downloadChunk_result()
+    result.read(self._iprot)
+    self._iprot.readMessageEnd()
+    if result.success != None:
+      return result.success
+    raise TApplicationException(TApplicationException.MISSING_RESULT, "downloadChunk failed: unknown result");
+
+  def getClusterInfo(self, ):
+    self.send_getClusterInfo()
+    return self.recv_getClusterInfo()
+
+  def send_getClusterInfo(self, ):
+    self._oprot.writeMessageBegin('getClusterInfo', TMessageType.CALL, self._seqid)
+    args = getClusterInfo_args()
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_getClusterInfo(self, ):
+    (fname, mtype, rseqid) = self._iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(self._iprot)
+      self._iprot.readMessageEnd()
+      raise x
+    result = getClusterInfo_result()
+    result.read(self._iprot)
+    self._iprot.readMessageEnd()
+    if result.success != None:
+      return result.success
+    raise TApplicationException(TApplicationException.MISSING_RESULT, "getClusterInfo failed: unknown result");
+
+  def getTopologyInfo(self, id):
+    """
+    Parameters:
+     - id
+    """
+    self.send_getTopologyInfo(id)
+    return self.recv_getTopologyInfo()
+
+  def send_getTopologyInfo(self, id):
+    self._oprot.writeMessageBegin('getTopologyInfo', TMessageType.CALL, self._seqid)
+    args = getTopologyInfo_args()
+    args.id = id
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_getTopologyInfo(self, ):
+    (fname, mtype, rseqid) = self._iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(self._iprot)
+      self._iprot.readMessageEnd()
+      raise x
+    result = getTopologyInfo_result()
+    result.read(self._iprot)
+    self._iprot.readMessageEnd()
+    if result.success != None:
+      return result.success
+    if result.e != None:
+      raise result.e
+    raise TApplicationException(TApplicationException.MISSING_RESULT, "getTopologyInfo failed: unknown result");
+
+  def getTopologyConf(self, id):
+    """
+    Parameters:
+     - id
+    """
+    self.send_getTopologyConf(id)
+    return self.recv_getTopologyConf()
+
+  def send_getTopologyConf(self, id):
+    self._oprot.writeMessageBegin('getTopologyConf', TMessageType.CALL, self._seqid)
+    args = getTopologyConf_args()
+    args.id = id
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_getTopologyConf(self, ):
+    (fname, mtype, rseqid) = self._iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(self._iprot)
+      self._iprot.readMessageEnd()
+      raise x
+    result = getTopologyConf_result()
+    result.read(self._iprot)
+    self._iprot.readMessageEnd()
+    if result.success != None:
+      return result.success
+    if result.e != None:
+      raise result.e
+    raise TApplicationException(TApplicationException.MISSING_RESULT, "getTopologyConf failed: unknown result");
+
+  def getTopology(self, id):
+    """
+    Parameters:
+     - id
+    """
+    self.send_getTopology(id)
+    return self.recv_getTopology()
+
+  def send_getTopology(self, id):
+    self._oprot.writeMessageBegin('getTopology', TMessageType.CALL, self._seqid)
+    args = getTopology_args()
+    args.id = id
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_getTopology(self, ):
+    (fname, mtype, rseqid) = self._iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(self._iprot)
+      self._iprot.readMessageEnd()
+      raise x
+    result = getTopology_result()
+    result.read(self._iprot)
+    self._iprot.readMessageEnd()
+    if result.success != None:
+      return result.success
+    if result.e != None:
+      raise result.e
+    raise TApplicationException(TApplicationException.MISSING_RESULT, "getTopology failed: unknown result");
+
+
+class Processor(Iface, TProcessor):
+  def __init__(self, handler):
+    self._handler = handler
+    self._processMap = {}
+    self._processMap["submitTopology"] = Processor.process_submitTopology
+    self._processMap["killTopology"] = Processor.process_killTopology
+    self._processMap["beginFileUpload"] = Processor.process_beginFileUpload
+    self._processMap["uploadChunk"] = Processor.process_uploadChunk
+    self._processMap["finishFileUpload"] = Processor.process_finishFileUpload
+    self._processMap["beginFileDownload"] = Processor.process_beginFileDownload
+    self._processMap["downloadChunk"] = Processor.process_downloadChunk
+    self._processMap["getClusterInfo"] = Processor.process_getClusterInfo
+    self._processMap["getTopologyInfo"] = Processor.process_getTopologyInfo
+    self._processMap["getTopologyConf"] = Processor.process_getTopologyConf
+    self._processMap["getTopology"] = Processor.process_getTopology
+
+  def process(self, iprot, oprot):
+    (name, type, seqid) = iprot.readMessageBegin()
+    if name not in self._processMap:
+      iprot.skip(TType.STRUCT)
+      iprot.readMessageEnd()
+      x = TApplicationException(TApplicationException.UNKNOWN_METHOD, 'Unknown function %s' % (name))
+      oprot.writeMessageBegin(name, TMessageType.EXCEPTION, seqid)
+      x.write(oprot)
+      oprot.writeMessageEnd()
+      oprot.trans.flush()
+      return
+    else:
+      self._processMap[name](self, seqid, iprot, oprot)
+    return True
+
+  def process_submitTopology(self, seqid, iprot, oprot):
+    args = submitTopology_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = submitTopology_result()
+    try:
+      self._handler.submitTopology(args.name, args.uploadedJarLocation, args.jsonConf, args.topology)
+    except AlreadyAliveException, e:
+      result.e = e
+    except InvalidTopologyException, ite:
+      result.ite = ite
+    oprot.writeMessageBegin("submitTopology", TMessageType.REPLY, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
+  def process_killTopology(self, seqid, iprot, oprot):
+    args = killTopology_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = killTopology_result()
+    try:
+      self._handler.killTopology(args.name)
+    except NotAliveException, e:
+      result.e = e
+    oprot.writeMessageBegin("killTopology", TMessageType.REPLY, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
+  def process_beginFileUpload(self, seqid, iprot, oprot):
+    args = beginFileUpload_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = beginFileUpload_result()
+    result.success = self._handler.beginFileUpload()
+    oprot.writeMessageBegin("beginFileUpload", TMessageType.REPLY, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
+  def process_uploadChunk(self, seqid, iprot, oprot):
+    args = uploadChunk_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = uploadChunk_result()
+    self._handler.uploadChunk(args.location, args.chunk)
+    oprot.writeMessageBegin("uploadChunk", TMessageType.REPLY, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
+  def process_finishFileUpload(self, seqid, iprot, oprot):
+    args = finishFileUpload_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = finishFileUpload_result()
+    self._handler.finishFileUpload(args.location)
+    oprot.writeMessageBegin("finishFileUpload", TMessageType.REPLY, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
+  def process_beginFileDownload(self, seqid, iprot, oprot):
+    args = beginFileDownload_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = beginFileDownload_result()
+    result.success = self._handler.beginFileDownload(args.file)
+    oprot.writeMessageBegin("beginFileDownload", TMessageType.REPLY, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
+  def process_downloadChunk(self, seqid, iprot, oprot):
+    args = downloadChunk_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = downloadChunk_result()
+    result.success = self._handler.downloadChunk(args.id)
+    oprot.writeMessageBegin("downloadChunk", TMessageType.REPLY, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
+  def process_getClusterInfo(self, seqid, iprot, oprot):
+    args = getClusterInfo_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = getClusterInfo_result()
+    result.success = self._handler.getClusterInfo()
+    oprot.writeMessageBegin("getClusterInfo", TMessageType.REPLY, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
+  def process_getTopologyInfo(self, seqid, iprot, oprot):
+    args = getTopologyInfo_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = getTopologyInfo_result()
+    try:
+      result.success = self._handler.getTopologyInfo(args.id)
+    except NotAliveException, e:
+      result.e = e
+    oprot.writeMessageBegin("getTopologyInfo", TMessageType.REPLY, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
+  def process_getTopologyConf(self, seqid, iprot, oprot):
+    args = getTopologyConf_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = getTopologyConf_result()
+    try:
+      result.success = self._handler.getTopologyConf(args.id)
+    except NotAliveException, e:
+      result.e = e
+    oprot.writeMessageBegin("getTopologyConf", TMessageType.REPLY, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
+  def process_getTopology(self, seqid, iprot, oprot):
+    args = getTopology_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = getTopology_result()
+    try:
+      result.success = self._handler.getTopology(args.id)
+    except NotAliveException, e:
+      result.e = e
+    oprot.writeMessageBegin("getTopology", TMessageType.REPLY, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
+
+# HELPER FUNCTIONS AND STRUCTURES
+
+class submitTopology_args:
+  """
+  Attributes:
+   - name
+   - uploadedJarLocation
+   - jsonConf
+   - topology
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'name', None, None, ), # 1
+    (2, TType.STRING, 'uploadedJarLocation', None, None, ), # 2
+    (3, TType.STRING, 'jsonConf', None, None, ), # 3
+    (4, TType.STRUCT, 'topology', (StormTopology, StormTopology.thrift_spec), None, ), # 4
+  )
+
+  def __init__(self, name=None, uploadedJarLocation=None, jsonConf=None, topology=None,):
+    self.name = name
+    self.uploadedJarLocation = uploadedJarLocation
+    self.jsonConf = jsonConf
+    self.topology = topology
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRING:
+          self.name = iprot.readString().decode('utf-8');
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRING:
+          self.uploadedJarLocation = iprot.readString().decode('utf-8');
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.STRING:
+          self.jsonConf = iprot.readString().decode('utf-8');
+        else:
+          iprot.skip(ftype)
+      elif fid == 4:
+        if ftype == TType.STRUCT:
+          self.topology = StormTopology()
+          self.topology.read(iprot)
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('submitTopology_args')
+    if self.name != None:
+      oprot.writeFieldBegin('name', TType.STRING, 1)
+      oprot.writeString(self.name.encode('utf-8'));
+      oprot.writeFieldEnd()
+    if self.uploadedJarLocation != None:
+      oprot.writeFieldBegin('uploadedJarLocation', TType.STRING, 2)
+      oprot.writeString(self.uploadedJarLocation.encode('utf-8'));
+      oprot.writeFieldEnd()
+    if self.jsonConf != None:
+      oprot.writeFieldBegin('jsonConf', TType.STRING, 3)
+      oprot.writeString(self.jsonConf.encode('utf-8'));
+      oprot.writeFieldEnd()
+    if self.topology != None:
+      oprot.writeFieldBegin('topology', TType.STRUCT, 4)
+      self.topology.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+  def union_value(self):
+    d = self.__dict__
+    for key in d:
+      val = d[key]
+      if not val is None:
+        return val
+
+  def get_set_field(self):
+    for attr in self.__dict__:
+      if not self.__dict__[attr] is None:
+        return attr
+
+  def get_set_field_id(self):
+    for idx, tup in enumerate(self.__class__.thrift_spec):
+      if tup:
+        key = tup[2]
+        if not self.__dict__[key] is None:
+          return idx
+
+class submitTopology_result:
+  """
+  Attributes:
+   - e
+   - ite
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRUCT, 'e', (AlreadyAliveException, AlreadyAliveException.thrift_spec), None, ), # 1
+    (2, TType.STRUCT, 'ite', (InvalidTopologyException, InvalidTopologyException.thrift_spec), None, ), # 2
+  )
+
+  def __init__(self, e=None, ite=None,):
+    self.e = e
+    self.ite = ite
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRUCT:
+          self.e = AlreadyAliveException()
+          self.e.read(iprot)
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRUCT:
+          self.ite = InvalidTopologyException()
+          self.ite.read(iprot)
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('submitTopology_result')
+    if self.e != None:
+      oprot.writeFieldBegin('e', TType.STRUCT, 1)
+      self.e.write(oprot)
+      oprot.writeFieldEnd()
+    if self.ite != None:
+      oprot.writeFieldBegin('ite', TType.STRUCT, 2)
+      self.ite.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+  def union_value(self):
+    d = self.__dict__
+    for key in d:
+      val = d[key]
+      if not val is None:
+        return val
+
+  def get_set_field(self):
+    for attr in self.__dict__:
+      if not self.__dict__[attr] is None:
+        return attr
+
+  def get_set_field_id(self):
+    for idx, tup in enumerate(self.__class__.thrift_spec):
+      if tup:
+        key = tup[2]
+        if not self.__dict__[key] is None:
+          return idx
+
+class killTopology_args:
+  """
+  Attributes:
+   - name
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'name', None, None, ), # 1
+  )
+
+  def __init__(self, name=None,):
+    self.name = name
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRING:
+          self.name = iprot.readString().decode('utf-8');
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('killTopology_args')
+    if self.name != None:
+      oprot.writeFieldBegin('name', TType.STRING, 1)
+      oprot.writeString(self.name.encode('utf-8'));
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+  def union_value(self):
+    d = self.__dict__
+    for key in d:
+      val = d[key]
+      if not val is None:
+        return val
+
+  def get_set_field(self):
+    for attr in self.__dict__:
+      if not self.__dict__[attr] is None:
+        return attr
+
+  def get_set_field_id(self):
+    for idx, tup in enumerate(self.__class__.thrift_spec):
+      if tup:
+        key = tup[2]
+        if not self.__dict__[key] is None:
+          return idx
+
+class killTopology_result:
+  """
+  Attributes:
+   - e
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRUCT, 'e', (NotAliveException, NotAliveException.thrift_spec), None, ), # 1
+  )
+
+  def __init__(self, e=None,):
+    self.e = e
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRUCT:
+          self.e = NotAliveException()
+          self.e.read(iprot)
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('killTopology_result')
+    if self.e != None:
+      oprot.writeFieldBegin('e', TType.STRUCT, 1)
+      self.e.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+  def union_value(self):
+    d = self.__dict__
+    for key in d:
+      val = d[key]
+      if not val is None:
+        return val
+
+  def get_set_field(self):
+    for attr in self.__dict__:
+      if not self.__dict__[attr] is None:
+        return attr
+
+  def get_set_field_id(self):
+    for idx, tup in enumerate(self.__class__.thrift_spec):
+      if tup:
+        key = tup[2]
+        if not self.__dict__[key] is None:
+          return idx
+
+class beginFileUpload_args:
+
+  thrift_spec = (
+  )
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('beginFileUpload_args')
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+  def union_value(self):
+    d = self.__dict__
+    for key in d:
+      val = d[key]
+      if not val is None:
+        return val
+
+  def get_set_field(self):
+    for attr in self.__dict__:
+      if not self.__dict__[attr] is None:
+        return attr
+
+  def get_set_field_id(self):
+    for idx, tup in enumerate(self.__class__.thrift_spec):
+      if tup:
+        key = tup[2]
+        if not self.__dict__[key] is None:
+          return idx
+
+class beginFileUpload_result:
+  """
+  Attributes:
+   - success
+  """
+
+  thrift_spec = (
+    (0, TType.STRING, 'success', None, None, ), # 0
+  )
+
+  def __init__(self, success=None,):
+    self.success = success
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 0:
+        if ftype == TType.STRING:
+          self.success = iprot.readString().decode('utf-8');
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('beginFileUpload_result')
+    if self.success != None:
+      oprot.writeFieldBegin('success', TType.STRING, 0)
+      oprot.writeString(self.success.encode('utf-8'));
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+  def union_value(self):
+    d = self.__dict__
+    for key in d:
+      val = d[key]
+      if not val is None:
+        return val
+
+  def get_set_field(self):
+    for attr in self.__dict__:
+      if not self.__dict__[attr] is None:
+        return attr
+
+  def get_set_field_id(self):
+    for idx, tup in enumerate(self.__class__.thrift_spec):
+      if tup:
+        key = tup[2]
+        if not self.__dict__[key] is None:
+          return idx
+
+class uploadChunk_args:
+  """
+  Attributes:
+   - location
+   - chunk
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'location', None, None, ), # 1
+    (2, TType.STRING, 'chunk', None, None, ), # 2
+  )
+
+  def __init__(self, location=None, chunk=None,):
+    self.location = location
+    self.chunk = chunk
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRING:
+          self.location = iprot.readString().decode('utf-8');
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRING:
+          self.chunk = iprot.readString();
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('uploadChunk_args')
+    if self.location != None:
+      oprot.writeFieldBegin('location', TType.STRING, 1)
+      oprot.writeString(self.location.encode('utf-8'));
+      oprot.writeFieldEnd()
+    if self.chunk != None:
+      oprot.writeFieldBegin('chunk', TType.STRING, 2)
+      oprot.writeString(self.chunk);
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+  def union_value(self):
+    d = self.__dict__
+    for key in d:
+      val = d[key]
+      if not val is None:
+        return val
+
+  def get_set_field(self):
+    for attr in self.__dict__:
+      if not self.__dict__[attr] is None:
+        return attr
+
+  def get_set_field_id(self):
+    for idx, tup in enumerate(self.__class__.thrift_spec):
+      if tup:
+        key = tup[2]
+        if not self.__dict__[key] is None:
+          return idx
+
+class uploadChunk_result:
+
+  thrift_spec = (
+  )
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('uploadChunk_result')
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+  def union_value(self):
+    d = self.__dict__
+    for key in d:
+      val = d[key]
+      if not val is None:
+        return val
+
+  def get_set_field(self):
+    for attr in self.__dict__:
+      if not self.__dict__[attr] is None:
+        return attr
+
+  def get_set_field_id(self):
+    for idx, tup in enumerate(self.__class__.thrift_spec):
+      if tup:
+        key = tup[2]
+        if not self.__dict__[key] is None:
+          return idx
+
+class finishFileUpload_args:
+  """
+  Attributes:
+   - location
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'location', None, None, ), # 1
+  )
+
+  def __init__(self, location=None,):
+    self.location = location
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRING:
+          self.location = iprot.readString().decode('utf-8');
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('finishFileUpload_args')
+    if self.location != None:
+      oprot.writeFieldBegin('location', TType.STRING, 1)
+      oprot.writeString(self.location.encode('utf-8'));
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+  def union_value(self):
+    d = self.__dict__
+    for key in d:
+      val = d[key]
+      if not val is None:
+        return val
+
+  def get_set_field(self):
+    for attr in self.__dict__:
+      if not self.__dict__[attr] is None:
+        return attr
+
+  def get_set_field_id(self):
+    for idx, tup in enumerate(self.__class__.thrift_spec):
+      if tup:
+        key = tup[2]
+        if not self.__dict__[key] is None:
+          return idx
+
+class finishFileUpload_result:
+
+  thrift_spec = (
+  )
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('finishFileUpload_result')
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+  def union_value(self):
+    d = self.__dict__
+    for key in d:
+      val = d[key]
+      if not val is None:
+        return val
+
+  def get_set_field(self):
+    for attr in self.__dict__:
+      if not self.__dict__[attr] is None:
+        return attr
+
+  def get_set_field_id(self):
+    for idx, tup in enumerate(self.__class__.thrift_spec):
+      if tup:
+        key = tup[2]
+        if not self.__dict__[key] is None:
+          return idx
+
+class beginFileDownload_args:
+  """
+  Attributes:
+   - file
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'file', None, None, ), # 1
+  )
+
+  def __init__(self, file=None,):
+    self.file = file
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRING:
+          self.file = iprot.readString().decode('utf-8');
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('beginFileDownload_args')
+    if self.file != None:
+      oprot.writeFieldBegin('file', TType.STRING, 1)
+      oprot.writeString(self.file.encode('utf-8'));
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+  def union_value(self):
+    d = self.__dict__
+    for key in d:
+      val = d[key]
+      if not val is None:
+        return val
+
+  def get_set_field(self):
+    for attr in self.__dict__:
+      if not self.__dict__[attr] is None:
+        return attr
+
+  def get_set_field_id(self):
+    for idx, tup in enumerate(self.__class__.thrift_spec):
+      if tup:
+        key = tup[2]
+        if not self.__dict__[key] is None:
+          return idx
+
+class beginFileDownload_result:
+  """
+  Attributes:
+   - success
+  """
+
+  thrift_spec = (
+    (0, TType.STRING, 'success', None, None, ), # 0
+  )
+
+  def __init__(self, success=None,):
+    self.success = success
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 0:
+        if ftype == TType.STRING:
+          self.success = iprot.readString().decode('utf-8');
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('beginFileDownload_result')
+    if self.success != None:
+      oprot.writeFieldBegin('success', TType.STRING, 0)
+      oprot.writeString(self.success.encode('utf-8'));
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+  def union_value(self):
+    d = self.__dict__
+    for key in d:
+      val = d[key]
+      if not val is None:
+        return val
+
+  def get_set_field(self):
+    for attr in self.__dict__:
+      if not self.__dict__[attr] is None:
+        return attr
+
+  def get_set_field_id(self):
+    for idx, tup in enumerate(self.__class__.thrift_spec):
+      if tup:
+        key = tup[2]
+        if not self.__dict__[key] is None:
+          return idx
+
+class downloadChunk_args:
+  """
+  Attributes:
+   - id
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'id', None, None, ), # 1
+  )
+
+  def __init__(self, id=None,):
+    self.id = id
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRING:
+          self.id = iprot.readString().decode('utf-8');
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('downloadChunk_args')
+    if self.id != None:
+      oprot.writeFieldBegin('id', TType.STRING, 1)
+      oprot.writeString(self.id.encode('utf-8'));
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+  def union_value(self):
+    d = self.__dict__
+    for key in d:
+      val = d[key]
+      if not val is None:
+        return val
+
+  def get_set_field(self):
+    for attr in self.__dict__:
+      if not self.__dict__[attr] is None:
+        return attr
+
+  def get_set_field_id(self):
+    for idx, tup in enumerate(self.__class__.thrift_spec):
+      if tup:
+        key = tup[2]
+        if not self.__dict__[key] is None:
+          return idx
+
+class downloadChunk_result:
+  """
+  Attributes:
+   - success
+  """
+
+  thrift_spec = (
+    (0, TType.STRING, 'success', None, None, ), # 0
+  )
+
+  def __init__(self, success=None,):
+    self.success = success
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 0:
+        if ftype == TType.STRING:
+          self.success = iprot.readString();
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('downloadChunk_result')
+    if self.success != None:
+      oprot.writeFieldBegin('success', TType.STRING, 0)
+      oprot.writeString(self.success);
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+  def union_value(self):
+    d = self.__dict__
+    for key in d:
+      val = d[key]
+      if not val is None:
+        return val
+
+  def get_set_field(self):
+    for attr in self.__dict__:
+      if not self.__dict__[attr] is None:
+        return attr
+
+  def get_set_field_id(self):
+    for idx, tup in enumerate(self.__class__.thrift_spec):
+      if tup:
+        key = tup[2]
+        if not self.__dict__[key] is None:
+          return idx
+
+class getClusterInfo_args:
+
+  thrift_spec = (
+  )
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('getClusterInfo_args')
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+  def union_value(self):
+    d = self.__dict__
+    for key in d:
+      val = d[key]
+      if not val is None:
+        return val
+
+  def get_set_field(self):
+    for attr in self.__dict__:
+      if not self.__dict__[attr] is None:
+        return attr
+
+  def get_set_field_id(self):
+    for idx, tup in enumerate(self.__class__.thrift_spec):
+      if tup:
+        key = tup[2]
+        if not self.__dict__[key] is None:
+          return idx
+
+class getClusterInfo_result:
+  """
+  Attributes:
+   - success
+  """
+
+  thrift_spec = (
+    (0, TType.STRUCT, 'success', (ClusterSummary, ClusterSummary.thrift_spec), None, ), # 0
+  )
+
+  def __init__(self, success=None,):
+    self.success = success
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 0:
+        if ftype == TType.STRUCT:
+          self.success = ClusterSummary()
+          self.success.read(iprot)
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('getClusterInfo_result')
+    if self.success != None:
+      oprot.writeFieldBegin('success', TType.STRUCT, 0)
+      self.success.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+  def union_value(self):
+    d = self.__dict__
+    for key in d:
+      val = d[key]
+      if not val is None:
+        return val
+
+  def get_set_field(self):
+    for attr in self.__dict__:
+      if not self.__dict__[attr] is None:
+        return attr
+
+  def get_set_field_id(self):
+    for idx, tup in enumerate(self.__class__.thrift_spec):
+      if tup:
+        key = tup[2]
+        if not self.__dict__[key] is None:
+          return idx
+
+class getTopologyInfo_args:
+  """
+  Attributes:
+   - id
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'id', None, None, ), # 1
+  )
+
+  def __init__(self, id=None,):
+    self.id = id
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRING:
+          self.id = iprot.readString().decode('utf-8');
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('getTopologyInfo_args')
+    if self.id != None:
+      oprot.writeFieldBegin('id', TType.STRING, 1)
+      oprot.writeString(self.id.encode('utf-8'));
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+  def union_value(self):
+    d = self.__dict__
+    for key in d:
+      val = d[key]
+      if not val is None:
+        return val
+
+  def get_set_field(self):
+    for attr in self.__dict__:
+      if not self.__dict__[attr] is None:
+        return attr
+
+  def get_set_field_id(self):
+    for idx, tup in enumerate(self.__class__.thrift_spec):
+      if tup:
+        key = tup[2]
+        if not self.__dict__[key] is None:
+          return idx
+
+class getTopologyInfo_result:
+  """
+  Attributes:
+   - success
+   - e
+  """
+
+  thrift_spec = (
+    (0, TType.STRUCT, 'success', (TopologyInfo, TopologyInfo.thrift_spec), None, ), # 0
+    (1, TType.STRUCT, 'e', (NotAliveException, NotAliveException.thrift_spec), None, ), # 1
+  )
+
+  def __init__(self, success=None, e=None,):
+    self.success = success
+    self.e = e
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 0:
+        if ftype == TType.STRUCT:
+          self.success = TopologyInfo()
+          self.success.read(iprot)
+        else:
+          iprot.skip(ftype)
+      elif fid == 1:
+        if ftype == TType.STRUCT:
+          self.e = NotAliveException()
+          self.e.read(iprot)
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('getTopologyInfo_result')
+    if self.success != None:
+      oprot.writeFieldBegin('success', TType.STRUCT, 0)
+      self.success.write(oprot)
+      oprot.writeFieldEnd()
+    if self.e != None:
+      oprot.writeFieldBegin('e', TType.STRUCT, 1)
+      self.e.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+  def union_value(self):
+    d = self.__dict__
+    for key in d:
+      val = d[key]
+      if not val is None:
+        return val
+
+  def get_set_field(self):
+    for attr in self.__dict__:
+      if not self.__dict__[attr] is None:
+        return attr
+
+  def get_set_field_id(self):
+    for idx, tup in enumerate(self.__class__.thrift_spec):
+      if tup:
+        key = tup[2]
+        if not self.__dict__[key] is None:
+          return idx
+
+class getTopologyConf_args:
+  """
+  Attributes:
+   - id
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'id', None, None, ), # 1
+  )
+
+  def __init__(self, id=None,):
+    self.id = id
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRING:
+          self.id = iprot.readString().decode('utf-8');
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('getTopologyConf_args')
+    if self.id != None:
+      oprot.writeFieldBegin('id', TType.STRING, 1)
+      oprot.writeString(self.id.encode('utf-8'));
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+  def union_value(self):
+    d = self.__dict__
+    for key in d:
+      val = d[key]
+      if not val is None:
+        return val
+
+  def get_set_field(self):
+    for attr in self.__dict__:
+      if not self.__dict__[attr] is None:
+        return attr
+
+  def get_set_field_id(self):
+    for idx, tup in enumerate(self.__class__.thrift_spec):
+      if tup:
+        key = tup[2]
+        if not self.__dict__[key] is None:
+          return idx
+
+class getTopologyConf_result:
+  """
+  Attributes:
+   - success
+   - e
+  """
+
+  thrift_spec = (
+    (0, TType.STRING, 'success', None, None, ), # 0
+    (1, TType.STRUCT, 'e', (NotAliveException, NotAliveException.thrift_spec), None, ), # 1
+  )
+
+  def __init__(self, success=None, e=None,):
+    self.success = success
+    self.e = e
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 0:
+        if ftype == TType.STRING:
+          self.success = iprot.readString().decode('utf-8');
+        else:
+          iprot.skip(ftype)
+      elif fid == 1:
+        if ftype == TType.STRUCT:
+          self.e = NotAliveException()
+          self.e.read(iprot)
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('getTopologyConf_result')
+    if self.success != None:
+      oprot.writeFieldBegin('success', TType.STRING, 0)
+      oprot.writeString(self.success.encode('utf-8'));
+      oprot.writeFieldEnd()
+    if self.e != None:
+      oprot.writeFieldBegin('e', TType.STRUCT, 1)
+      self.e.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+  def union_value(self):
+    d = self.__dict__
+    for key in d:
+      val = d[key]
+      if not val is None:
+        return val
+
+  def get_set_field(self):
+    for attr in self.__dict__:
+      if not self.__dict__[attr] is None:
+        return attr
+
+  def get_set_field_id(self):
+    for idx, tup in enumerate(self.__class__.thrift_spec):
+      if tup:
+        key = tup[2]
+        if not self.__dict__[key] is None:
+          return idx
+
+class getTopology_args:
+  """
+  Attributes:
+   - id
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'id', None, None, ), # 1
+  )
+
+  def __init__(self, id=None,):
+    self.id = id
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRING:
+          self.id = iprot.readString().decode('utf-8');
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('getTopology_args')
+    if self.id != None:
+      oprot.writeFieldBegin('id', TType.STRING, 1)
+      oprot.writeString(self.id.encode('utf-8'));
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+  def union_value(self):
+    d = self.__dict__
+    for key in d:
+      val = d[key]
+      if not val is None:
+        return val
+
+  def get_set_field(self):
+    for attr in self.__dict__:
+      if not self.__dict__[attr] is None:
+        return attr
+
+  def get_set_field_id(self):
+    for idx, tup in enumerate(self.__class__.thrift_spec):
+      if tup:
+        key = tup[2]
+        if not self.__dict__[key] is None:
+          return idx
+
+class getTopology_result:
+  """
+  Attributes:
+   - success
+   - e
+  """
+
+  thrift_spec = (
+    (0, TType.STRUCT, 'success', (StormTopology, StormTopology.thrift_spec), None, ), # 0
+    (1, TType.STRUCT, 'e', (NotAliveException, NotAliveException.thrift_spec), None, ), # 1
+  )
+
+  def __init__(self, success=None, e=None,):
+    self.success = success
+    self.e = e
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 0:
+        if ftype == TType.STRUCT:
+          self.success = StormTopology()
+          self.success.read(iprot)
+        else:
+          iprot.skip(ftype)
+      elif fid == 1:
+        if ftype == TType.STRUCT:
+          self.e = NotAliveException()
+          self.e.read(iprot)
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('getTopology_result')
+    if self.success != None:
+      oprot.writeFieldBegin('success', TType.STRUCT, 0)
+      self.success.write(oprot)
+      oprot.writeFieldEnd()
+    if self.e != None:
+      oprot.writeFieldBegin('e', TType.STRUCT, 1)
+      self.e.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+  def union_value(self):
+    d = self.__dict__
+    for key in d:
+      val = d[key]
+      if not val is None:
+        return val
+
+  def get_set_field(self):
+    for attr in self.__dict__:
+      if not self.__dict__[attr] is None:
+        return attr
+
+  def get_set_field_id(self):
+    for idx, tup in enumerate(self.__class__.thrift_spec):
+      if tup:
+        key = tup[2]
+        if not self.__dict__[key] is None:
+          return idx
+
+
diff --git a/src/py/storm/__init__.py b/src/py/storm/__init__.py
new file mode 100644
index 0000000..e2ec2b6
--- /dev/null
+++ b/src/py/storm/__init__.py
@@ -0,0 +1 @@
+__all__ = ['ttypes', 'constants', 'Nimbus', 'DistributedRPC']
diff --git a/src/py/storm/constants.py b/src/py/storm/constants.py
new file mode 100644
index 0000000..4f6fe5c
--- /dev/null
+++ b/src/py/storm/constants.py
@@ -0,0 +1,10 @@
+#
+# Autogenerated by Thrift
+#
+# DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+#
+
+from thrift.Thrift import *
+import sys
+from ttypes import *
+
diff --git a/src/py/storm/ttypes.py b/src/py/storm/ttypes.py
new file mode 100644
index 0000000..4db93f0
--- /dev/null
+++ b/src/py/storm/ttypes.py
@@ -0,0 +1,2599 @@
+#
+# Autogenerated by Thrift
+#
+# DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+#
+
+from thrift.Thrift import *
+import sys
+sys.path = ["/".join(__file__.split("/")[:-1]) + "/.."] + sys.path
+
+from thrift.transport import TTransport
+from thrift.protocol import TBinaryProtocol
+try:
+  from thrift.protocol import fastbinary
+except:
+  fastbinary = None
+
+
+class NullStruct:
+
+  thrift_spec = (
+  )
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('NullStruct')
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+  def union_value(self):
+    d = self.__dict__
+    for key in d:
+      val = d[key]
+      if not val is None:
+        return val
+
+  def get_set_field(self):
+    for attr in self.__dict__:
+      if not self.__dict__[attr] is None:
+        return attr
+
+  def get_set_field_id(self):
+    for idx, tup in enumerate(self.__class__.thrift_spec):
+      if tup:
+        key = tup[2]
+        if not self.__dict__[key] is None:
+          return idx
+
+class Grouping:
+  """
+  Attributes:
+   - fields
+   - shuffle
+   - all
+   - none
+   - direct
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.LIST, 'fields', (TType.STRING,None), None, ), # 1
+    (2, TType.STRUCT, 'shuffle', (NullStruct, NullStruct.thrift_spec), None, ), # 2
+    (3, TType.STRUCT, 'all', (NullStruct, NullStruct.thrift_spec), None, ), # 3
+    (4, TType.STRUCT, 'none', (NullStruct, NullStruct.thrift_spec), None, ), # 4
+    (5, TType.STRUCT, 'direct', (NullStruct, NullStruct.thrift_spec), None, ), # 5
+  )
+
+  def __init__(self, fields=None, shuffle=None, all=None, none=None, direct=None,):
+    self.fields = fields
+    self.shuffle = shuffle
+    self.all = all
+    self.none = none
+    self.direct = direct
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.LIST:
+          self.fields = []
+          (_etype3, _size0) = iprot.readListBegin()
+          for _i4 in xrange(_size0):
+            _elem5 = iprot.readString().decode('utf-8');
+            self.fields.append(_elem5)
+          iprot.readListEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRUCT:
+          self.shuffle = NullStruct()
+          self.shuffle.read(iprot)
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.STRUCT:
+          self.all = NullStruct()
+          self.all.read(iprot)
+        else:
+          iprot.skip(ftype)
+      elif fid == 4:
+        if ftype == TType.STRUCT:
+          self.none = NullStruct()
+          self.none.read(iprot)
+        else:
+          iprot.skip(ftype)
+      elif fid == 5:
+        if ftype == TType.STRUCT:
+          self.direct = NullStruct()
+          self.direct.read(iprot)
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('Grouping')
+    if self.fields != None:
+      oprot.writeFieldBegin('fields', TType.LIST, 1)
+      oprot.writeListBegin(TType.STRING, len(self.fields))
+      for iter6 in self.fields:
+        oprot.writeString(iter6.encode('utf-8'));
+      oprot.writeListEnd()
+      oprot.writeFieldEnd()
+    if self.shuffle != None:
+      oprot.writeFieldBegin('shuffle', TType.STRUCT, 2)
+      self.shuffle.write(oprot)
+      oprot.writeFieldEnd()
+    if self.all != None:
+      oprot.writeFieldBegin('all', TType.STRUCT, 3)
+      self.all.write(oprot)
+      oprot.writeFieldEnd()
+    if self.none != None:
+      oprot.writeFieldBegin('none', TType.STRUCT, 4)
+      self.none.write(oprot)
+      oprot.writeFieldEnd()
+    if self.direct != None:
+      oprot.writeFieldBegin('direct', TType.STRUCT, 5)
+      self.direct.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+  def union_value(self):
+    d = self.__dict__
+    for key in d:
+      val = d[key]
+      if not val is None:
+        return val
+
+  def get_set_field(self):
+    for attr in self.__dict__:
+      if not self.__dict__[attr] is None:
+        return attr
+
+  def get_set_field_id(self):
+    for idx, tup in enumerate(self.__class__.thrift_spec):
+      if tup:
+        key = tup[2]
+        if not self.__dict__[key] is None:
+          return idx
+
+class StreamInfo:
+  """
+  Attributes:
+   - output_fields
+   - direct
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.LIST, 'output_fields', (TType.STRING,None), None, ), # 1
+    (2, TType.BOOL, 'direct', None, None, ), # 2
+  )
+
+  def __init__(self, output_fields=None, direct=None,):
+    self.output_fields = output_fields
+    self.direct = direct
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.LIST:
+          self.output_fields = []
+          (_etype10, _size7) = iprot.readListBegin()
+          for _i11 in xrange(_size7):
+            _elem12 = iprot.readString().decode('utf-8');
+            self.output_fields.append(_elem12)
+          iprot.readListEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.BOOL:
+          self.direct = iprot.readBool();
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('StreamInfo')
+    if self.output_fields != None:
+      oprot.writeFieldBegin('output_fields', TType.LIST, 1)
+      oprot.writeListBegin(TType.STRING, len(self.output_fields))
+      for iter13 in self.output_fields:
+        oprot.writeString(iter13.encode('utf-8'));
+      oprot.writeListEnd()
+      oprot.writeFieldEnd()
+    if self.direct != None:
+      oprot.writeFieldBegin('direct', TType.BOOL, 2)
+      oprot.writeBool(self.direct)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+  def union_value(self):
+    d = self.__dict__
+    for key in d:
+      val = d[key]
+      if not val is None:
+        return val
+
+  def get_set_field(self):
+    for attr in self.__dict__:
+      if not self.__dict__[attr] is None:
+        return attr
+
+  def get_set_field_id(self):
+    for idx, tup in enumerate(self.__class__.thrift_spec):
+      if tup:
+        key = tup[2]
+        if not self.__dict__[key] is None:
+          return idx
+
+class ShellComponent:
+  """
+  Attributes:
+   - execution_command
+   - script
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'execution_command', None, None, ), # 1
+    (2, TType.STRING, 'script', None, None, ), # 2
+  )
+
+  def __init__(self, execution_command=None, script=None,):
+    self.execution_command = execution_command
+    self.script = script
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRING:
+          self.execution_command = iprot.readString().decode('utf-8');
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRING:
+          self.script = iprot.readString().decode('utf-8');
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('ShellComponent')
+    if self.execution_command != None:
+      oprot.writeFieldBegin('execution_command', TType.STRING, 1)
+      oprot.writeString(self.execution_command.encode('utf-8'));
+      oprot.writeFieldEnd()
+    if self.script != None:
+      oprot.writeFieldBegin('script', TType.STRING, 2)
+      oprot.writeString(self.script.encode('utf-8'));
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+  def union_value(self):
+    d = self.__dict__
+    for key in d:
+      val = d[key]
+      if not val is None:
+        return val
+
+  def get_set_field(self):
+    for attr in self.__dict__:
+      if not self.__dict__[attr] is None:
+        return attr
+
+  def get_set_field_id(self):
+    for idx, tup in enumerate(self.__class__.thrift_spec):
+      if tup:
+        key = tup[2]
+        if not self.__dict__[key] is None:
+          return idx
+
+class ComponentObject:
+  """
+  Attributes:
+   - serialized_java
+   - shell
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'serialized_java', None, None, ), # 1
+    (2, TType.STRUCT, 'shell', (ShellComponent, ShellComponent.thrift_spec), None, ), # 2
+  )
+
+  def __init__(self, serialized_java=None, shell=None,):
+    self.serialized_java = serialized_java
+    self.shell = shell
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRING:
+          self.serialized_java = iprot.readString();
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRUCT:
+          self.shell = ShellComponent()
+          self.shell.read(iprot)
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('ComponentObject')
+    if self.serialized_java != None:
+      oprot.writeFieldBegin('serialized_java', TType.STRING, 1)
+      oprot.writeString(self.serialized_java);
+      oprot.writeFieldEnd()
+    if self.shell != None:
+      oprot.writeFieldBegin('shell', TType.STRUCT, 2)
+      self.shell.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+  def union_value(self):
+    d = self.__dict__
+    for key in d:
+      val = d[key]
+      if not val is None:
+        return val
+
+  def get_set_field(self):
+    for attr in self.__dict__:
+      if not self.__dict__[attr] is None:
+        return attr
+
+  def get_set_field_id(self):
+    for idx, tup in enumerate(self.__class__.thrift_spec):
+      if tup:
+        key = tup[2]
+        if not self.__dict__[key] is None:
+          return idx
+
+class ComponentCommon:
+  """
+  Attributes:
+   - streams
+   - parallelism_hint
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.MAP, 'streams', (TType.I32,None,TType.STRUCT,(StreamInfo, StreamInfo.thrift_spec)), None, ), # 1
+    (2, TType.I32, 'parallelism_hint', None, None, ), # 2
+  )
+
+  def __init__(self, streams=None, parallelism_hint=None,):
+    self.streams = streams
+    self.parallelism_hint = parallelism_hint
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.MAP:
+          self.streams = {}
+          (_ktype15, _vtype16, _size14 ) = iprot.readMapBegin() 
+          for _i18 in xrange(_size14):
+            _key19 = iprot.readI32();
+            _val20 = StreamInfo()
+            _val20.read(iprot)
+            self.streams[_key19] = _val20
+          iprot.readMapEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.I32:
+          self.parallelism_hint = iprot.readI32();
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('ComponentCommon')
+    if self.streams != None:
+      oprot.writeFieldBegin('streams', TType.MAP, 1)
+      oprot.writeMapBegin(TType.I32, TType.STRUCT, len(self.streams))
+      for kiter21,viter22 in self.streams.items():
+        oprot.writeI32(kiter21)
+        viter22.write(oprot)
+      oprot.writeMapEnd()
+      oprot.writeFieldEnd()
+    if self.parallelism_hint != None:
+      oprot.writeFieldBegin('parallelism_hint', TType.I32, 2)
+      oprot.writeI32(self.parallelism_hint)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+  def union_value(self):
+    d = self.__dict__
+    for key in d:
+      val = d[key]
+      if not val is None:
+        return val
+
+  def get_set_field(self):
+    for attr in self.__dict__:
+      if not self.__dict__[attr] is None:
+        return attr
+
+  def get_set_field_id(self):
+    for idx, tup in enumerate(self.__class__.thrift_spec):
+      if tup:
+        key = tup[2]
+        if not self.__dict__[key] is None:
+          return idx
+
+class SpoutSpec:
+  """
+  Attributes:
+   - spout_object
+   - common
+   - distributed
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRUCT, 'spout_object', (ComponentObject, ComponentObject.thrift_spec), None, ), # 1
+    (2, TType.STRUCT, 'common', (ComponentCommon, ComponentCommon.thrift_spec), None, ), # 2
+    (3, TType.BOOL, 'distributed', None, None, ), # 3
+  )
+
+  def __init__(self, spout_object=None, common=None, distributed=None,):
+    self.spout_object = spout_object
+    self.common = common
+    self.distributed = distributed
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRUCT:
+          self.spout_object = ComponentObject()
+          self.spout_object.read(iprot)
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRUCT:
+          self.common = ComponentCommon()
+          self.common.read(iprot)
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.BOOL:
+          self.distributed = iprot.readBool();
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('SpoutSpec')
+    if self.spout_object != None:
+      oprot.writeFieldBegin('spout_object', TType.STRUCT, 1)
+      self.spout_object.write(oprot)
+      oprot.writeFieldEnd()
+    if self.common != None:
+      oprot.writeFieldBegin('common', TType.STRUCT, 2)
+      self.common.write(oprot)
+      oprot.writeFieldEnd()
+    if self.distributed != None:
+      oprot.writeFieldBegin('distributed', TType.BOOL, 3)
+      oprot.writeBool(self.distributed)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+  def union_value(self):
+    d = self.__dict__
+    for key in d:
+      val = d[key]
+      if not val is None:
+        return val
+
+  def get_set_field(self):
+    for attr in self.__dict__:
+      if not self.__dict__[attr] is None:
+        return attr
+
+  def get_set_field_id(self):
+    for idx, tup in enumerate(self.__class__.thrift_spec):
+      if tup:
+        key = tup[2]
+        if not self.__dict__[key] is None:
+          return idx
+
+class GlobalStreamId:
+  """
+  Attributes:
+   - componentId
+   - streamId
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.I32, 'componentId', None, None, ), # 1
+    (2, TType.I32, 'streamId', None, None, ), # 2
+  )
+
+  def __init__(self, componentId=None, streamId=None,):
+    self.componentId = componentId
+    self.streamId = streamId
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.I32:
+          self.componentId = iprot.readI32();
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.I32:
+          self.streamId = iprot.readI32();
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('GlobalStreamId')
+    if self.componentId != None:
+      oprot.writeFieldBegin('componentId', TType.I32, 1)
+      oprot.writeI32(self.componentId)
+      oprot.writeFieldEnd()
+    if self.streamId != None:
+      oprot.writeFieldBegin('streamId', TType.I32, 2)
+      oprot.writeI32(self.streamId)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+  def union_value(self):
+    d = self.__dict__
+    for key in d:
+      val = d[key]
+      if not val is None:
+        return val
+
+  def get_set_field(self):
+    for attr in self.__dict__:
+      if not self.__dict__[attr] is None:
+        return attr
+
+  def get_set_field_id(self):
+    for idx, tup in enumerate(self.__class__.thrift_spec):
+      if tup:
+        key = tup[2]
+        if not self.__dict__[key] is None:
+          return idx
+
+class Bolt:
+  """
+  Attributes:
+   - inputs
+   - bolt_object
+   - common
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.MAP, 'inputs', (TType.STRUCT,(GlobalStreamId, GlobalStreamId.thrift_spec),TType.STRUCT,(Grouping, Grouping.thrift_spec)), None, ), # 1
+    (2, TType.STRUCT, 'bolt_object', (ComponentObject, ComponentObject.thrift_spec), None, ), # 2
+    (3, TType.STRUCT, 'common', (ComponentCommon, ComponentCommon.thrift_spec), None, ), # 3
+  )
+
+  def __init__(self, inputs=None, bolt_object=None, common=None,):
+    self.inputs = inputs
+    self.bolt_object = bolt_object
+    self.common = common
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.MAP:
+          self.inputs = {}
+          (_ktype24, _vtype25, _size23 ) = iprot.readMapBegin() 
+          for _i27 in xrange(_size23):
+            _key28 = GlobalStreamId()
+            _key28.read(iprot)
+            _val29 = Grouping()
+            _val29.read(iprot)
+            self.inputs[_key28] = _val29
+          iprot.readMapEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRUCT:
+          self.bolt_object = ComponentObject()
+          self.bolt_object.read(iprot)
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.STRUCT:
+          self.common = ComponentCommon()
+          self.common.read(iprot)
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('Bolt')
+    if self.inputs != None:
+      oprot.writeFieldBegin('inputs', TType.MAP, 1)
+      oprot.writeMapBegin(TType.STRUCT, TType.STRUCT, len(self.inputs))
+      for kiter30,viter31 in self.inputs.items():
+        kiter30.write(oprot)
+        viter31.write(oprot)
+      oprot.writeMapEnd()
+      oprot.writeFieldEnd()
+    if self.bolt_object != None:
+      oprot.writeFieldBegin('bolt_object', TType.STRUCT, 2)
+      self.bolt_object.write(oprot)
+      oprot.writeFieldEnd()
+    if self.common != None:
+      oprot.writeFieldBegin('common', TType.STRUCT, 3)
+      self.common.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+  def union_value(self):
+    d = self.__dict__
+    for key in d:
+      val = d[key]
+      if not val is None:
+        return val
+
+  def get_set_field(self):
+    for attr in self.__dict__:
+      if not self.__dict__[attr] is None:
+        return attr
+
+  def get_set_field_id(self):
+    for idx, tup in enumerate(self.__class__.thrift_spec):
+      if tup:
+        key = tup[2]
+        if not self.__dict__[key] is None:
+          return idx
+
+class StateSpoutSpec:
+  """
+  Attributes:
+   - state_spout_object
+   - common
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRUCT, 'state_spout_object', (ComponentObject, ComponentObject.thrift_spec), None, ), # 1
+    (2, TType.STRUCT, 'common', (ComponentCommon, ComponentCommon.thrift_spec), None, ), # 2
+  )
+
+  def __init__(self, state_spout_object=None, common=None,):
+    self.state_spout_object = state_spout_object
+    self.common = common
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRUCT:
+          self.state_spout_object = ComponentObject()
+          self.state_spout_object.read(iprot)
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRUCT:
+          self.common = ComponentCommon()
+          self.common.read(iprot)
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('StateSpoutSpec')
+    if self.state_spout_object != None:
+      oprot.writeFieldBegin('state_spout_object', TType.STRUCT, 1)
+      self.state_spout_object.write(oprot)
+      oprot.writeFieldEnd()
+    if self.common != None:
+      oprot.writeFieldBegin('common', TType.STRUCT, 2)
+      self.common.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+  def union_value(self):
+    d = self.__dict__
+    for key in d:
+      val = d[key]
+      if not val is None:
+        return val
+
+  def get_set_field(self):
+    for attr in self.__dict__:
+      if not self.__dict__[attr] is None:
+        return attr
+
+  def get_set_field_id(self):
+    for idx, tup in enumerate(self.__class__.thrift_spec):
+      if tup:
+        key = tup[2]
+        if not self.__dict__[key] is None:
+          return idx
+
+class StormTopology:
+  """
+  Attributes:
+   - spouts
+   - bolts
+   - state_spouts
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.MAP, 'spouts', (TType.I32,None,TType.STRUCT,(SpoutSpec, SpoutSpec.thrift_spec)), None, ), # 1
+    (2, TType.MAP, 'bolts', (TType.I32,None,TType.STRUCT,(Bolt, Bolt.thrift_spec)), None, ), # 2
+    (3, TType.MAP, 'state_spouts', (TType.I32,None,TType.STRUCT,(StateSpoutSpec, StateSpoutSpec.thrift_spec)), None, ), # 3
+  )
+
+  def __init__(self, spouts=None, bolts=None, state_spouts=None,):
+    self.spouts = spouts
+    self.bolts = bolts
+    self.state_spouts = state_spouts
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.MAP:
+          self.spouts = {}
+          (_ktype33, _vtype34, _size32 ) = iprot.readMapBegin() 
+          for _i36 in xrange(_size32):
+            _key37 = iprot.readI32();
+            _val38 = SpoutSpec()
+            _val38.read(iprot)
+            self.spouts[_key37] = _val38
+          iprot.readMapEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.MAP:
+          self.bolts = {}
+          (_ktype40, _vtype41, _size39 ) = iprot.readMapBegin() 
+          for _i43 in xrange(_size39):
+            _key44 = iprot.readI32();
+            _val45 = Bolt()
+            _val45.read(iprot)
+            self.bolts[_key44] = _val45
+          iprot.readMapEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.MAP:
+          self.state_spouts = {}
+          (_ktype47, _vtype48, _size46 ) = iprot.readMapBegin() 
+          for _i50 in xrange(_size46):
+            _key51 = iprot.readI32();
+            _val52 = StateSpoutSpec()
+            _val52.read(iprot)
+            self.state_spouts[_key51] = _val52
+          iprot.readMapEnd()
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('StormTopology')
+    if self.spouts != None:
+      oprot.writeFieldBegin('spouts', TType.MAP, 1)
+      oprot.writeMapBegin(TType.I32, TType.STRUCT, len(self.spouts))
+      for kiter53,viter54 in self.spouts.items():
+        oprot.writeI32(kiter53)
+        viter54.write(oprot)
+      oprot.writeMapEnd()
+      oprot.writeFieldEnd()
+    if self.bolts != None:
+      oprot.writeFieldBegin('bolts', TType.MAP, 2)
+      oprot.writeMapBegin(TType.I32, TType.STRUCT, len(self.bolts))
+      for kiter55,viter56 in self.bolts.items():
+        oprot.writeI32(kiter55)
+        viter56.write(oprot)
+      oprot.writeMapEnd()
+      oprot.writeFieldEnd()
+    if self.state_spouts != None:
+      oprot.writeFieldBegin('state_spouts', TType.MAP, 3)
+      oprot.writeMapBegin(TType.I32, TType.STRUCT, len(self.state_spouts))
+      for kiter57,viter58 in self.state_spouts.items():
+        oprot.writeI32(kiter57)
+        viter58.write(oprot)
+      oprot.writeMapEnd()
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+  def union_value(self):
+    d = self.__dict__
+    for key in d:
+      val = d[key]
+      if not val is None:
+        return val
+
+  def get_set_field(self):
+    for attr in self.__dict__:
+      if not self.__dict__[attr] is None:
+        return attr
+
+  def get_set_field_id(self):
+    for idx, tup in enumerate(self.__class__.thrift_spec):
+      if tup:
+        key = tup[2]
+        if not self.__dict__[key] is None:
+          return idx
+
+class AlreadyAliveException(Exception):
+  """
+  Attributes:
+   - msg
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'msg', None, None, ), # 1
+  )
+
+  def __init__(self, msg=None,):
+    self.msg = msg
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRING:
+          self.msg = iprot.readString().decode('utf-8');
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('AlreadyAliveException')
+    if self.msg != None:
+      oprot.writeFieldBegin('msg', TType.STRING, 1)
+      oprot.writeString(self.msg.encode('utf-8'));
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def __str__(self):
+    return repr(self)
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+  def union_value(self):
+    d = self.__dict__
+    for key in d:
+      val = d[key]
+      if not val is None:
+        return val
+
+  def get_set_field(self):
+    for attr in self.__dict__:
+      if not self.__dict__[attr] is None:
+        return attr
+
+  def get_set_field_id(self):
+    for idx, tup in enumerate(self.__class__.thrift_spec):
+      if tup:
+        key = tup[2]
+        if not self.__dict__[key] is None:
+          return idx
+
+class NotAliveException(Exception):
+  """
+  Attributes:
+   - msg
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'msg', None, None, ), # 1
+  )
+
+  def __init__(self, msg=None,):
+    self.msg = msg
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRING:
+          self.msg = iprot.readString().decode('utf-8');
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('NotAliveException')
+    if self.msg != None:
+      oprot.writeFieldBegin('msg', TType.STRING, 1)
+      oprot.writeString(self.msg.encode('utf-8'));
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def __str__(self):
+    return repr(self)
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+  def union_value(self):
+    d = self.__dict__
+    for key in d:
+      val = d[key]
+      if not val is None:
+        return val
+
+  def get_set_field(self):
+    for attr in self.__dict__:
+      if not self.__dict__[attr] is None:
+        return attr
+
+  def get_set_field_id(self):
+    for idx, tup in enumerate(self.__class__.thrift_spec):
+      if tup:
+        key = tup[2]
+        if not self.__dict__[key] is None:
+          return idx
+
+class InvalidTopologyException(Exception):
+  """
+  Attributes:
+   - msg
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'msg', None, None, ), # 1
+  )
+
+  def __init__(self, msg=None,):
+    self.msg = msg
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRING:
+          self.msg = iprot.readString().decode('utf-8');
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('InvalidTopologyException')
+    if self.msg != None:
+      oprot.writeFieldBegin('msg', TType.STRING, 1)
+      oprot.writeString(self.msg.encode('utf-8'));
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def __str__(self):
+    return repr(self)
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+  def union_value(self):
+    d = self.__dict__
+    for key in d:
+      val = d[key]
+      if not val is None:
+        return val
+
+  def get_set_field(self):
+    for attr in self.__dict__:
+      if not self.__dict__[attr] is None:
+        return attr
+
+  def get_set_field_id(self):
+    for idx, tup in enumerate(self.__class__.thrift_spec):
+      if tup:
+        key = tup[2]
+        if not self.__dict__[key] is None:
+          return idx
+
+class TopologySummary:
+  """
+  Attributes:
+   - id
+   - name
+   - num_tasks
+   - num_workers
+   - uptime_secs
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'id', None, None, ), # 1
+    (2, TType.STRING, 'name', None, None, ), # 2
+    (3, TType.I32, 'num_tasks', None, None, ), # 3
+    (4, TType.I32, 'num_workers', None, None, ), # 4
+    (5, TType.I32, 'uptime_secs', None, None, ), # 5
+  )
+
+  def __init__(self, id=None, name=None, num_tasks=None, num_workers=None, uptime_secs=None,):
+    self.id = id
+    self.name = name
+    self.num_tasks = num_tasks
+    self.num_workers = num_workers
+    self.uptime_secs = uptime_secs
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRING:
+          self.id = iprot.readString().decode('utf-8');
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRING:
+          self.name = iprot.readString().decode('utf-8');
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.I32:
+          self.num_tasks = iprot.readI32();
+        else:
+          iprot.skip(ftype)
+      elif fid == 4:
+        if ftype == TType.I32:
+          self.num_workers = iprot.readI32();
+        else:
+          iprot.skip(ftype)
+      elif fid == 5:
+        if ftype == TType.I32:
+          self.uptime_secs = iprot.readI32();
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('TopologySummary')
+    if self.id != None:
+      oprot.writeFieldBegin('id', TType.STRING, 1)
+      oprot.writeString(self.id.encode('utf-8'));
+      oprot.writeFieldEnd()
+    if self.name != None:
+      oprot.writeFieldBegin('name', TType.STRING, 2)
+      oprot.writeString(self.name.encode('utf-8'));
+      oprot.writeFieldEnd()
+    if self.num_tasks != None:
+      oprot.writeFieldBegin('num_tasks', TType.I32, 3)
+      oprot.writeI32(self.num_tasks)
+      oprot.writeFieldEnd()
+    if self.num_workers != None:
+      oprot.writeFieldBegin('num_workers', TType.I32, 4)
+      oprot.writeI32(self.num_workers)
+      oprot.writeFieldEnd()
+    if self.uptime_secs != None:
+      oprot.writeFieldBegin('uptime_secs', TType.I32, 5)
+      oprot.writeI32(self.uptime_secs)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+  def union_value(self):
+    d = self.__dict__
+    for key in d:
+      val = d[key]
+      if not val is None:
+        return val
+
+  def get_set_field(self):
+    for attr in self.__dict__:
+      if not self.__dict__[attr] is None:
+        return attr
+
+  def get_set_field_id(self):
+    for idx, tup in enumerate(self.__class__.thrift_spec):
+      if tup:
+        key = tup[2]
+        if not self.__dict__[key] is None:
+          return idx
+
+class SupervisorSummary:
+  """
+  Attributes:
+   - host
+   - uptime_secs
+   - num_workers
+   - num_used_workers
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'host', None, None, ), # 1
+    (2, TType.I32, 'uptime_secs', None, None, ), # 2
+    (3, TType.I32, 'num_workers', None, None, ), # 3
+    (4, TType.I32, 'num_used_workers', None, None, ), # 4
+  )
+
+  def __init__(self, host=None, uptime_secs=None, num_workers=None, num_used_workers=None,):
+    self.host = host
+    self.uptime_secs = uptime_secs
+    self.num_workers = num_workers
+    self.num_used_workers = num_used_workers
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRING:
+          self.host = iprot.readString().decode('utf-8');
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.I32:
+          self.uptime_secs = iprot.readI32();
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.I32:
+          self.num_workers = iprot.readI32();
+        else:
+          iprot.skip(ftype)
+      elif fid == 4:
+        if ftype == TType.I32:
+          self.num_used_workers = iprot.readI32();
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('SupervisorSummary')
+    if self.host != None:
+      oprot.writeFieldBegin('host', TType.STRING, 1)
+      oprot.writeString(self.host.encode('utf-8'));
+      oprot.writeFieldEnd()
+    if self.uptime_secs != None:
+      oprot.writeFieldBegin('uptime_secs', TType.I32, 2)
+      oprot.writeI32(self.uptime_secs)
+      oprot.writeFieldEnd()
+    if self.num_workers != None:
+      oprot.writeFieldBegin('num_workers', TType.I32, 3)
+      oprot.writeI32(self.num_workers)
+      oprot.writeFieldEnd()
+    if self.num_used_workers != None:
+      oprot.writeFieldBegin('num_used_workers', TType.I32, 4)
+      oprot.writeI32(self.num_used_workers)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+  def union_value(self):
+    d = self.__dict__
+    for key in d:
+      val = d[key]
+      if not val is None:
+        return val
+
+  def get_set_field(self):
+    for attr in self.__dict__:
+      if not self.__dict__[attr] is None:
+        return attr
+
+  def get_set_field_id(self):
+    for idx, tup in enumerate(self.__class__.thrift_spec):
+      if tup:
+        key = tup[2]
+        if not self.__dict__[key] is None:
+          return idx
+
+class ClusterSummary:
+  """
+  Attributes:
+   - supervisors
+   - nimbus_uptime_secs
+   - topologies
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.LIST, 'supervisors', (TType.STRUCT,(SupervisorSummary, SupervisorSummary.thrift_spec)), None, ), # 1
+    (2, TType.I32, 'nimbus_uptime_secs', None, None, ), # 2
+    (3, TType.LIST, 'topologies', (TType.STRUCT,(TopologySummary, TopologySummary.thrift_spec)), None, ), # 3
+  )
+
+  def __init__(self, supervisors=None, nimbus_uptime_secs=None, topologies=None,):
+    self.supervisors = supervisors
+    self.nimbus_uptime_secs = nimbus_uptime_secs
+    self.topologies = topologies
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.LIST:
+          self.supervisors = []
+          (_etype62, _size59) = iprot.readListBegin()
+          for _i63 in xrange(_size59):
+            _elem64 = SupervisorSummary()
+            _elem64.read(iprot)
+            self.supervisors.append(_elem64)
+          iprot.readListEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.I32:
+          self.nimbus_uptime_secs = iprot.readI32();
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.LIST:
+          self.topologies = []
+          (_etype68, _size65) = iprot.readListBegin()
+          for _i69 in xrange(_size65):
+            _elem70 = TopologySummary()
+            _elem70.read(iprot)
+            self.topologies.append(_elem70)
+          iprot.readListEnd()
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('ClusterSummary')
+    if self.supervisors != None:
+      oprot.writeFieldBegin('supervisors', TType.LIST, 1)
+      oprot.writeListBegin(TType.STRUCT, len(self.supervisors))
+      for iter71 in self.supervisors:
+        iter71.write(oprot)
+      oprot.writeListEnd()
+      oprot.writeFieldEnd()
+    if self.nimbus_uptime_secs != None:
+      oprot.writeFieldBegin('nimbus_uptime_secs', TType.I32, 2)
+      oprot.writeI32(self.nimbus_uptime_secs)
+      oprot.writeFieldEnd()
+    if self.topologies != None:
+      oprot.writeFieldBegin('topologies', TType.LIST, 3)
+      oprot.writeListBegin(TType.STRUCT, len(self.topologies))
+      for iter72 in self.topologies:
+        iter72.write(oprot)
+      oprot.writeListEnd()
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+  def union_value(self):
+    d = self.__dict__
+    for key in d:
+      val = d[key]
+      if not val is None:
+        return val
+
+  def get_set_field(self):
+    for attr in self.__dict__:
+      if not self.__dict__[attr] is None:
+        return attr
+
+  def get_set_field_id(self):
+    for idx, tup in enumerate(self.__class__.thrift_spec):
+      if tup:
+        key = tup[2]
+        if not self.__dict__[key] is None:
+          return idx
+
+class ErrorInfo:
+  """
+  Attributes:
+   - error
+   - error_time_secs
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'error', None, None, ), # 1
+    (2, TType.I32, 'error_time_secs', None, None, ), # 2
+  )
+
+  def __init__(self, error=None, error_time_secs=None,):
+    self.error = error
+    self.error_time_secs = error_time_secs
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRING:
+          self.error = iprot.readString().decode('utf-8');
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.I32:
+          self.error_time_secs = iprot.readI32();
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('ErrorInfo')
+    if self.error != None:
+      oprot.writeFieldBegin('error', TType.STRING, 1)
+      oprot.writeString(self.error.encode('utf-8'));
+      oprot.writeFieldEnd()
+    if self.error_time_secs != None:
+      oprot.writeFieldBegin('error_time_secs', TType.I32, 2)
+      oprot.writeI32(self.error_time_secs)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+  def union_value(self):
+    d = self.__dict__
+    for key in d:
+      val = d[key]
+      if not val is None:
+        return val
+
+  def get_set_field(self):
+    for attr in self.__dict__:
+      if not self.__dict__[attr] is None:
+        return attr
+
+  def get_set_field_id(self):
+    for idx, tup in enumerate(self.__class__.thrift_spec):
+      if tup:
+        key = tup[2]
+        if not self.__dict__[key] is None:
+          return idx
+
+class BoltStats:
+  """
+  Attributes:
+   - acked
+   - failed
+   - process_ms_avg
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.MAP, 'acked', (TType.STRING,None,TType.MAP,(TType.STRUCT,(GlobalStreamId, GlobalStreamId.thrift_spec),TType.I64,None)), None, ), # 1
+    (2, TType.MAP, 'failed', (TType.STRING,None,TType.MAP,(TType.STRUCT,(GlobalStreamId, GlobalStreamId.thrift_spec),TType.I64,None)), None, ), # 2
+    (3, TType.MAP, 'process_ms_avg', (TType.STRING,None,TType.MAP,(TType.STRUCT,(GlobalStreamId, GlobalStreamId.thrift_spec),TType.DOUBLE,None)), None, ), # 3
+  )
+
+  def __init__(self, acked=None, failed=None, process_ms_avg=None,):
+    self.acked = acked
+    self.failed = failed
+    self.process_ms_avg = process_ms_avg
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.MAP:
+          self.acked = {}
+          (_ktype74, _vtype75, _size73 ) = iprot.readMapBegin() 
+          for _i77 in xrange(_size73):
+            _key78 = iprot.readString().decode('utf-8');
+            _val79 = {}
+            (_ktype81, _vtype82, _size80 ) = iprot.readMapBegin() 
+            for _i84 in xrange(_size80):
+              _key85 = GlobalStreamId()
+              _key85.read(iprot)
+              _val86 = iprot.readI64();
+              _val79[_key85] = _val86
+            iprot.readMapEnd()
+            self.acked[_key78] = _val79
+          iprot.readMapEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.MAP:
+          self.failed = {}
+          (_ktype88, _vtype89, _size87 ) = iprot.readMapBegin() 
+          for _i91 in xrange(_size87):
+            _key92 = iprot.readString().decode('utf-8');
+            _val93 = {}
+            (_ktype95, _vtype96, _size94 ) = iprot.readMapBegin() 
+            for _i98 in xrange(_size94):
+              _key99 = GlobalStreamId()
+              _key99.read(iprot)
+              _val100 = iprot.readI64();
+              _val93[_key99] = _val100
+            iprot.readMapEnd()
+            self.failed[_key92] = _val93
+          iprot.readMapEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.MAP:
+          self.process_ms_avg = {}
+          (_ktype102, _vtype103, _size101 ) = iprot.readMapBegin() 
+          for _i105 in xrange(_size101):
+            _key106 = iprot.readString().decode('utf-8');
+            _val107 = {}
+            (_ktype109, _vtype110, _size108 ) = iprot.readMapBegin() 
+            for _i112 in xrange(_size108):
+              _key113 = GlobalStreamId()
+              _key113.read(iprot)
+              _val114 = iprot.readDouble();
+              _val107[_key113] = _val114
+            iprot.readMapEnd()
+            self.process_ms_avg[_key106] = _val107
+          iprot.readMapEnd()
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('BoltStats')
+    if self.acked != None:
+      oprot.writeFieldBegin('acked', TType.MAP, 1)
+      oprot.writeMapBegin(TType.STRING, TType.MAP, len(self.acked))
+      for kiter115,viter116 in self.acked.items():
+        oprot.writeString(kiter115.encode('utf-8'));
+        oprot.writeMapBegin(TType.STRUCT, TType.I64, len(viter116))
+        for kiter117,viter118 in viter116.items():
+          kiter117.write(oprot)
+          oprot.writeI64(viter118)
+        oprot.writeMapEnd()
+      oprot.writeMapEnd()
+      oprot.writeFieldEnd()
+    if self.failed != None:
+      oprot.writeFieldBegin('failed', TType.MAP, 2)
+      oprot.writeMapBegin(TType.STRING, TType.MAP, len(self.failed))
+      for kiter119,viter120 in self.failed.items():
+        oprot.writeString(kiter119.encode('utf-8'));
+        oprot.writeMapBegin(TType.STRUCT, TType.I64, len(viter120))
+        for kiter121,viter122 in viter120.items():
+          kiter121.write(oprot)
+          oprot.writeI64(viter122)
+        oprot.writeMapEnd()
+      oprot.writeMapEnd()
+      oprot.writeFieldEnd()
+    if self.process_ms_avg != None:
+      oprot.writeFieldBegin('process_ms_avg', TType.MAP, 3)
+      oprot.writeMapBegin(TType.STRING, TType.MAP, len(self.process_ms_avg))
+      for kiter123,viter124 in self.process_ms_avg.items():
+        oprot.writeString(kiter123.encode('utf-8'));
+        oprot.writeMapBegin(TType.STRUCT, TType.DOUBLE, len(viter124))
+        for kiter125,viter126 in viter124.items():
+          kiter125.write(oprot)
+          oprot.writeDouble(viter126)
+        oprot.writeMapEnd()
+      oprot.writeMapEnd()
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+  def union_value(self):
+    d = self.__dict__
+    for key in d:
+      val = d[key]
+      if not val is None:
+        return val
+
+  def get_set_field(self):
+    for attr in self.__dict__:
+      if not self.__dict__[attr] is None:
+        return attr
+
+  def get_set_field_id(self):
+    for idx, tup in enumerate(self.__class__.thrift_spec):
+      if tup:
+        key = tup[2]
+        if not self.__dict__[key] is None:
+          return idx
+
+class SpoutStats:
+  """
+  Attributes:
+   - acked
+   - failed
+   - complete_ms_avg
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.MAP, 'acked', (TType.STRING,None,TType.MAP,(TType.I32,None,TType.I64,None)), None, ), # 1
+    (2, TType.MAP, 'failed', (TType.STRING,None,TType.MAP,(TType.I32,None,TType.I64,None)), None, ), # 2
+    (3, TType.MAP, 'complete_ms_avg', (TType.STRING,None,TType.MAP,(TType.I32,None,TType.DOUBLE,None)), None, ), # 3
+  )
+
+  def __init__(self, acked=None, failed=None, complete_ms_avg=None,):
+    self.acked = acked
+    self.failed = failed
+    self.complete_ms_avg = complete_ms_avg
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.MAP:
+          self.acked = {}
+          (_ktype128, _vtype129, _size127 ) = iprot.readMapBegin() 
+          for _i131 in xrange(_size127):
+            _key132 = iprot.readString().decode('utf-8');
+            _val133 = {}
+            (_ktype135, _vtype136, _size134 ) = iprot.readMapBegin() 
+            for _i138 in xrange(_size134):
+              _key139 = iprot.readI32();
+              _val140 = iprot.readI64();
+              _val133[_key139] = _val140
+            iprot.readMapEnd()
+            self.acked[_key132] = _val133
+          iprot.readMapEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.MAP:
+          self.failed = {}
+          (_ktype142, _vtype143, _size141 ) = iprot.readMapBegin() 
+          for _i145 in xrange(_size141):
+            _key146 = iprot.readString().decode('utf-8');
+            _val147 = {}
+            (_ktype149, _vtype150, _size148 ) = iprot.readMapBegin() 
+            for _i152 in xrange(_size148):
+              _key153 = iprot.readI32();
+              _val154 = iprot.readI64();
+              _val147[_key153] = _val154
+            iprot.readMapEnd()
+            self.failed[_key146] = _val147
+          iprot.readMapEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.MAP:
+          self.complete_ms_avg = {}
+          (_ktype156, _vtype157, _size155 ) = iprot.readMapBegin() 
+          for _i159 in xrange(_size155):
+            _key160 = iprot.readString().decode('utf-8');
+            _val161 = {}
+            (_ktype163, _vtype164, _size162 ) = iprot.readMapBegin() 
+            for _i166 in xrange(_size162):
+              _key167 = iprot.readI32();
+              _val168 = iprot.readDouble();
+              _val161[_key167] = _val168
+            iprot.readMapEnd()
+            self.complete_ms_avg[_key160] = _val161
+          iprot.readMapEnd()
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('SpoutStats')
+    if self.acked != None:
+      oprot.writeFieldBegin('acked', TType.MAP, 1)
+      oprot.writeMapBegin(TType.STRING, TType.MAP, len(self.acked))
+      for kiter169,viter170 in self.acked.items():
+        oprot.writeString(kiter169.encode('utf-8'));
+        oprot.writeMapBegin(TType.I32, TType.I64, len(viter170))
+        for kiter171,viter172 in viter170.items():
+          oprot.writeI32(kiter171)
+          oprot.writeI64(viter172)
+        oprot.writeMapEnd()
+      oprot.writeMapEnd()
+      oprot.writeFieldEnd()
+    if self.failed != None:
+      oprot.writeFieldBegin('failed', TType.MAP, 2)
+      oprot.writeMapBegin(TType.STRING, TType.MAP, len(self.failed))
+      for kiter173,viter174 in self.failed.items():
+        oprot.writeString(kiter173.encode('utf-8'));
+        oprot.writeMapBegin(TType.I32, TType.I64, len(viter174))
+        for kiter175,viter176 in viter174.items():
+          oprot.writeI32(kiter175)
+          oprot.writeI64(viter176)
+        oprot.writeMapEnd()
+      oprot.writeMapEnd()
+      oprot.writeFieldEnd()
+    if self.complete_ms_avg != None:
+      oprot.writeFieldBegin('complete_ms_avg', TType.MAP, 3)
+      oprot.writeMapBegin(TType.STRING, TType.MAP, len(self.complete_ms_avg))
+      for kiter177,viter178 in self.complete_ms_avg.items():
+        oprot.writeString(kiter177.encode('utf-8'));
+        oprot.writeMapBegin(TType.I32, TType.DOUBLE, len(viter178))
+        for kiter179,viter180 in viter178.items():
+          oprot.writeI32(kiter179)
+          oprot.writeDouble(viter180)
+        oprot.writeMapEnd()
+      oprot.writeMapEnd()
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+  def union_value(self):
+    d = self.__dict__
+    for key in d:
+      val = d[key]
+      if not val is None:
+        return val
+
+  def get_set_field(self):
+    for attr in self.__dict__:
+      if not self.__dict__[attr] is None:
+        return attr
+
+  def get_set_field_id(self):
+    for idx, tup in enumerate(self.__class__.thrift_spec):
+      if tup:
+        key = tup[2]
+        if not self.__dict__[key] is None:
+          return idx
+
+class TaskSpecificStats:
+  """
+  Attributes:
+   - bolt
+   - spout
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRUCT, 'bolt', (BoltStats, BoltStats.thrift_spec), None, ), # 1
+    (2, TType.STRUCT, 'spout', (SpoutStats, SpoutStats.thrift_spec), None, ), # 2
+  )
+
+  def __init__(self, bolt=None, spout=None,):
+    self.bolt = bolt
+    self.spout = spout
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRUCT:
+          self.bolt = BoltStats()
+          self.bolt.read(iprot)
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRUCT:
+          self.spout = SpoutStats()
+          self.spout.read(iprot)
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('TaskSpecificStats')
+    if self.bolt != None:
+      oprot.writeFieldBegin('bolt', TType.STRUCT, 1)
+      self.bolt.write(oprot)
+      oprot.writeFieldEnd()
+    if self.spout != None:
+      oprot.writeFieldBegin('spout', TType.STRUCT, 2)
+      self.spout.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+  def union_value(self):
+    d = self.__dict__
+    for key in d:
+      val = d[key]
+      if not val is None:
+        return val
+
+  def get_set_field(self):
+    for attr in self.__dict__:
+      if not self.__dict__[attr] is None:
+        return attr
+
+  def get_set_field_id(self):
+    for idx, tup in enumerate(self.__class__.thrift_spec):
+      if tup:
+        key = tup[2]
+        if not self.__dict__[key] is None:
+          return idx
+
+class TaskStats:
+  """
+  Attributes:
+   - emitted
+   - transferred
+   - specific
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.MAP, 'emitted', (TType.STRING,None,TType.MAP,(TType.I32,None,TType.I64,None)), None, ), # 1
+    (2, TType.MAP, 'transferred', (TType.STRING,None,TType.MAP,(TType.I32,None,TType.I64,None)), None, ), # 2
+    (3, TType.STRUCT, 'specific', (TaskSpecificStats, TaskSpecificStats.thrift_spec), None, ), # 3
+  )
+
+  def __init__(self, emitted=None, transferred=None, specific=None,):
+    self.emitted = emitted
+    self.transferred = transferred
+    self.specific = specific
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.MAP:
+          self.emitted = {}
+          (_ktype182, _vtype183, _size181 ) = iprot.readMapBegin() 
+          for _i185 in xrange(_size181):
+            _key186 = iprot.readString().decode('utf-8');
+            _val187 = {}
+            (_ktype189, _vtype190, _size188 ) = iprot.readMapBegin() 
+            for _i192 in xrange(_size188):
+              _key193 = iprot.readI32();
+              _val194 = iprot.readI64();
+              _val187[_key193] = _val194
+            iprot.readMapEnd()
+            self.emitted[_key186] = _val187
+          iprot.readMapEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.MAP:
+          self.transferred = {}
+          (_ktype196, _vtype197, _size195 ) = iprot.readMapBegin() 
+          for _i199 in xrange(_size195):
+            _key200 = iprot.readString().decode('utf-8');
+            _val201 = {}
+            (_ktype203, _vtype204, _size202 ) = iprot.readMapBegin() 
+            for _i206 in xrange(_size202):
+              _key207 = iprot.readI32();
+              _val208 = iprot.readI64();
+              _val201[_key207] = _val208
+            iprot.readMapEnd()
+            self.transferred[_key200] = _val201
+          iprot.readMapEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.STRUCT:
+          self.specific = TaskSpecificStats()
+          self.specific.read(iprot)
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('TaskStats')
+    if self.emitted != None:
+      oprot.writeFieldBegin('emitted', TType.MAP, 1)
+      oprot.writeMapBegin(TType.STRING, TType.MAP, len(self.emitted))
+      for kiter209,viter210 in self.emitted.items():
+        oprot.writeString(kiter209.encode('utf-8'));
+        oprot.writeMapBegin(TType.I32, TType.I64, len(viter210))
+        for kiter211,viter212 in viter210.items():
+          oprot.writeI32(kiter211)
+          oprot.writeI64(viter212)
+        oprot.writeMapEnd()
+      oprot.writeMapEnd()
+      oprot.writeFieldEnd()
+    if self.transferred != None:
+      oprot.writeFieldBegin('transferred', TType.MAP, 2)
+      oprot.writeMapBegin(TType.STRING, TType.MAP, len(self.transferred))
+      for kiter213,viter214 in self.transferred.items():
+        oprot.writeString(kiter213.encode('utf-8'));
+        oprot.writeMapBegin(TType.I32, TType.I64, len(viter214))
+        for kiter215,viter216 in viter214.items():
+          oprot.writeI32(kiter215)
+          oprot.writeI64(viter216)
+        oprot.writeMapEnd()
+      oprot.writeMapEnd()
+      oprot.writeFieldEnd()
+    if self.specific != None:
+      oprot.writeFieldBegin('specific', TType.STRUCT, 3)
+      self.specific.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+  def union_value(self):
+    d = self.__dict__
+    for key in d:
+      val = d[key]
+      if not val is None:
+        return val
+
+  def get_set_field(self):
+    for attr in self.__dict__:
+      if not self.__dict__[attr] is None:
+        return attr
+
+  def get_set_field_id(self):
+    for idx, tup in enumerate(self.__class__.thrift_spec):
+      if tup:
+        key = tup[2]
+        if not self.__dict__[key] is None:
+          return idx
+
+class TaskSummary:
+  """
+  Attributes:
+   - task_id
+   - component_id
+   - host
+   - port
+   - uptime_secs
+   - errors
+   - stats
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.I32, 'task_id', None, None, ), # 1
+    (2, TType.I32, 'component_id', None, None, ), # 2
+    (3, TType.STRING, 'host', None, None, ), # 3
+    (4, TType.I32, 'port', None, None, ), # 4
+    (5, TType.I32, 'uptime_secs', None, None, ), # 5
+    (6, TType.LIST, 'errors', (TType.STRUCT,(ErrorInfo, ErrorInfo.thrift_spec)), None, ), # 6
+    (7, TType.STRUCT, 'stats', (TaskStats, TaskStats.thrift_spec), None, ), # 7
+  )
+
+  def __init__(self, task_id=None, component_id=None, host=None, port=None, uptime_secs=None, errors=None, stats=None,):
+    self.task_id = task_id
+    self.component_id = component_id
+    self.host = host
+    self.port = port
+    self.uptime_secs = uptime_secs
+    self.errors = errors
+    self.stats = stats
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.I32:
+          self.task_id = iprot.readI32();
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.I32:
+          self.component_id = iprot.readI32();
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.STRING:
+          self.host = iprot.readString().decode('utf-8');
+        else:
+          iprot.skip(ftype)
+      elif fid == 4:
+        if ftype == TType.I32:
+          self.port = iprot.readI32();
+        else:
+          iprot.skip(ftype)
+      elif fid == 5:
+        if ftype == TType.I32:
+          self.uptime_secs = iprot.readI32();
+        else:
+          iprot.skip(ftype)
+      elif fid == 6:
+        if ftype == TType.LIST:
+          self.errors = []
+          (_etype220, _size217) = iprot.readListBegin()
+          for _i221 in xrange(_size217):
+            _elem222 = ErrorInfo()
+            _elem222.read(iprot)
+            self.errors.append(_elem222)
+          iprot.readListEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 7:
+        if ftype == TType.STRUCT:
+          self.stats = TaskStats()
+          self.stats.read(iprot)
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('TaskSummary')
+    if self.task_id != None:
+      oprot.writeFieldBegin('task_id', TType.I32, 1)
+      oprot.writeI32(self.task_id)
+      oprot.writeFieldEnd()
+    if self.component_id != None:
+      oprot.writeFieldBegin('component_id', TType.I32, 2)
+      oprot.writeI32(self.component_id)
+      oprot.writeFieldEnd()
+    if self.host != None:
+      oprot.writeFieldBegin('host', TType.STRING, 3)
+      oprot.writeString(self.host.encode('utf-8'));
+      oprot.writeFieldEnd()
+    if self.port != None:
+      oprot.writeFieldBegin('port', TType.I32, 4)
+      oprot.writeI32(self.port)
+      oprot.writeFieldEnd()
+    if self.uptime_secs != None:
+      oprot.writeFieldBegin('uptime_secs', TType.I32, 5)
+      oprot.writeI32(self.uptime_secs)
+      oprot.writeFieldEnd()
+    if self.errors != None:
+      oprot.writeFieldBegin('errors', TType.LIST, 6)
+      oprot.writeListBegin(TType.STRUCT, len(self.errors))
+      for iter223 in self.errors:
+        iter223.write(oprot)
+      oprot.writeListEnd()
+      oprot.writeFieldEnd()
+    if self.stats != None:
+      oprot.writeFieldBegin('stats', TType.STRUCT, 7)
+      self.stats.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+  def union_value(self):
+    d = self.__dict__
+    for key in d:
+      val = d[key]
+      if not val is None:
+        return val
+
+  def get_set_field(self):
+    for attr in self.__dict__:
+      if not self.__dict__[attr] is None:
+        return attr
+
+  def get_set_field_id(self):
+    for idx, tup in enumerate(self.__class__.thrift_spec):
+      if tup:
+        key = tup[2]
+        if not self.__dict__[key] is None:
+          return idx
+
+class TopologyInfo:
+  """
+  Attributes:
+   - id
+   - name
+   - uptime_secs
+   - tasks
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'id', None, None, ), # 1
+    (2, TType.STRING, 'name', None, None, ), # 2
+    (3, TType.I32, 'uptime_secs', None, None, ), # 3
+    (4, TType.LIST, 'tasks', (TType.STRUCT,(TaskSummary, TaskSummary.thrift_spec)), None, ), # 4
+  )
+
+  def __init__(self, id=None, name=None, uptime_secs=None, tasks=None,):
+    self.id = id
+    self.name = name
+    self.uptime_secs = uptime_secs
+    self.tasks = tasks
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRING:
+          self.id = iprot.readString().decode('utf-8');
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRING:
+          self.name = iprot.readString().decode('utf-8');
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.I32:
+          self.uptime_secs = iprot.readI32();
+        else:
+          iprot.skip(ftype)
+      elif fid == 4:
+        if ftype == TType.LIST:
+          self.tasks = []
+          (_etype227, _size224) = iprot.readListBegin()
+          for _i228 in xrange(_size224):
+            _elem229 = TaskSummary()
+            _elem229.read(iprot)
+            self.tasks.append(_elem229)
+          iprot.readListEnd()
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('TopologyInfo')
+    if self.id != None:
+      oprot.writeFieldBegin('id', TType.STRING, 1)
+      oprot.writeString(self.id.encode('utf-8'));
+      oprot.writeFieldEnd()
+    if self.name != None:
+      oprot.writeFieldBegin('name', TType.STRING, 2)
+      oprot.writeString(self.name.encode('utf-8'));
+      oprot.writeFieldEnd()
+    if self.uptime_secs != None:
+      oprot.writeFieldBegin('uptime_secs', TType.I32, 3)
+      oprot.writeI32(self.uptime_secs)
+      oprot.writeFieldEnd()
+    if self.tasks != None:
+      oprot.writeFieldBegin('tasks', TType.LIST, 4)
+      oprot.writeListBegin(TType.STRUCT, len(self.tasks))
+      for iter230 in self.tasks:
+        iter230.write(oprot)
+      oprot.writeListEnd()
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+  def union_value(self):
+    d = self.__dict__
+    for key in d:
+      val = d[key]
+      if not val is None:
+        return val
+
+  def get_set_field(self):
+    for attr in self.__dict__:
+      if not self.__dict__[attr] is None:
+        return attr
+
+  def get_set_field_id(self):
+    for idx, tup in enumerate(self.__class__.thrift_spec):
+      if tup:
+        key = tup[2]
+        if not self.__dict__[key] is None:
+          return idx
+
diff --git a/src/storm.thrift b/src/storm.thrift
new file mode 100644
index 0000000..2e541dd
--- /dev/null
+++ b/src/storm.thrift
@@ -0,0 +1,174 @@
+#!/usr/local/bin/thrift --gen java:beans,nocamel,hashcode
+
+namespace java backtype.storm.generated
+
+struct NullStruct {
+  
+}
+
+union Grouping {
+  1: list<string> fields; //empty list means global grouping
+  2: NullStruct shuffle; // tuple is sent to random task
+  3: NullStruct all; // tuple is sent to every task
+  4: NullStruct none; // tuple is sent to a single task (storm's choice) -> allows storm to optimize the topology by bundling tasks into a single process
+  5: NullStruct direct; // this bolt expects the source bolt to send tuples directly to it
+}
+
+struct StreamInfo {
+  1: required list<string> output_fields;
+  2: required bool direct;
+}
+
+struct ShellComponent {
+  1: string execution_command;
+  2: string script;
+}
+
+union ComponentObject {
+  1: binary serialized_java;
+  2: ShellComponent shell;
+}
+
+struct ComponentCommon {
+  1: required map<i32, StreamInfo> streams; //key is stream id
+  2: optional i32 parallelism_hint; //how many threads across the cluster should be dedicated to this component
+}
+
+struct SpoutSpec {
+  1: required ComponentObject spout_object;
+  2: required ComponentCommon common;
+  3: required bool distributed;
+}
+
+struct GlobalStreamId {
+  1: required i32 componentId;
+  2: required i32 streamId;
+  #Going to need to add an enum for the stream type (NORMAL or FAILURE)
+}
+
+struct Bolt {
+  1: required map<GlobalStreamId, Grouping> inputs; //a join would have multiple inputs
+  2: required ComponentObject bolt_object;
+  3: required ComponentCommon common;
+}
+
+// not implemented yet
+// this will eventually be the basis for subscription implementation in storm
+struct StateSpoutSpec {
+  1: required ComponentObject state_spout_object;
+  2: required ComponentCommon common;
+}
+
+struct StormTopology {
+  //ids must be unique across maps
+  1: required map<i32, SpoutSpec> spouts;
+  2: required map<i32, Bolt> bolts;
+  3: required map<i32, StateSpoutSpec> state_spouts;
+  // #workers to use is in conf
+}
+
+exception AlreadyAliveException {
+  1: required string msg;
+}
+
+exception NotAliveException {
+  1: required string msg;
+}
+
+exception InvalidTopologyException {
+  1: required string msg;
+}
+
+struct TopologySummary {
+  1: required string id;
+  2: required string name;
+  3: required i32 num_tasks;
+  4: required i32 num_workers;
+  5: required i32 uptime_secs;
+}
+
+struct SupervisorSummary {
+  1: required string host;
+  2: required i32 uptime_secs;
+  3: required i32 num_workers;
+  4: required i32 num_used_workers;  
+}
+
+struct ClusterSummary {
+  1: required list<SupervisorSummary> supervisors;
+  2: required i32 nimbus_uptime_secs;
+  3: required list<TopologySummary> topologies;
+}
+
+struct ErrorInfo {
+  1: required string error;
+  2: required i32 error_time_secs;
+}
+
+struct BoltStats {
+  1: required map<string, map<GlobalStreamId, i64>> acked;  
+  2: required map<string, map<GlobalStreamId, i64>> failed;  
+  3: required map<string, map<GlobalStreamId, double>> process_ms_avg;
+}
+
+struct SpoutStats {
+  1: required map<string, map<i32, i64>> acked;
+  2: required map<string, map<i32, i64>> failed;
+  3: required map<string, map<i32, double>> complete_ms_avg;
+}
+
+union TaskSpecificStats {
+  1: BoltStats bolt;
+  2: SpoutStats spout;
+}
+
+// Stats are a map from the time window (all time or a number indicating number of seconds in the window)
+//    to the stats. Usually stats are a stream id to a count or average.
+struct TaskStats {
+  1: required map<string, map<i32, i64>> emitted;
+  2: required map<string, map<i32, i64>> transferred;
+  3: required TaskSpecificStats specific;
+}
+
+struct TaskSummary {
+  1: required i32 task_id;
+  2: required i32 component_id;
+  3: required string host;
+  4: required i32 port;
+  5: required i32 uptime_secs;
+  6: required list<ErrorInfo> errors;
+  7: optional TaskStats stats;
+}
+
+struct TopologyInfo {
+  1: required string id;
+  2: required string name;
+  3: required i32 uptime_secs;
+  4: required list<TaskSummary> tasks;
+}
+
+service Nimbus {
+  void submitTopology(1: string name, 2: string uploadedJarLocation, 3: string jsonConf, 4: StormTopology topology) throws (1: AlreadyAliveException e, 2: InvalidTopologyException ite);
+  void killTopology(1: string name) throws (1: NotAliveException e);
+  // need to add functions for asking about status of storms, what nodes they're running on, looking at task logs
+
+  string beginFileUpload();
+  void uploadChunk(1: string location, 2: binary chunk);
+  void finishFileUpload(1: string location);
+  
+  string beginFileDownload(1: string file);
+  //can stop downloading chunks when receive 0-length byte array back
+  binary downloadChunk(1: string id);
+  
+  // stats functions
+  ClusterSummary getClusterInfo();
+  TopologyInfo getTopologyInfo(1: string id) throws (1: NotAliveException e);
+  //returns json
+  string getTopologyConf(1: string id) throws (1: NotAliveException e);
+  StormTopology getTopology(1: string id) throws (1: NotAliveException e);
+}
+
+service DistributedRPC {
+  string execute(1: string functionName, 2: string funcArgs);
+  void result(1: string id, 2: string result);
+}
diff --git a/src/ui/public/css/bootstrap-1.1.0.css b/src/ui/public/css/bootstrap-1.1.0.css
new file mode 100644
index 0000000..948d5fc
--- /dev/null
+++ b/src/ui/public/css/bootstrap-1.1.0.css
@@ -0,0 +1,1894 @@
+/*!
+ * Bootstrap v1.1.0
+ *
+ * Copyright 2011 Twitter, Inc
+ * Licensed under the Apache License v2.0
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Designed and built with all the love in the world @twitter by @mdo and @fat.
+ * Date: Mon Aug 22 23:50:35 PDT 2011
+ */
+/* Reset.less
+ * Props to Eric Meyer (meyerweb.com) for his CSS reset file. We're using an adapted version here	that cuts out some of the reset HTML elements we will never need here (i.e., dfn, samp, etc).
+ * ------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- */
+html, body {
+  margin: 0;
+  padding: 0;
+}
+h1,
+h2,
+h3,
+h4,
+h5,
+h6,
+p,
+blockquote,
+pre,
+a,
+abbr,
+acronym,
+address,
+cite,
+code,
+del,
+dfn,
+em,
+img,
+q,
+s,
+samp,
+small,
+strike,
+strong,
+sub,
+sup,
+tt,
+var,
+dd,
+dl,
+dt,
+li,
+ol,
+ul,
+fieldset,
+form,
+label,
+legend,
+button,
+table,
+caption,
+tbody,
+tfoot,
+thead,
+tr,
+th,
+td {
+  margin: 0;
+  padding: 0;
+  border: 0;
+  font-weight: normal;
+  font-style: normal;
+  font-size: 100%;
+  line-height: 1;
+  font-family: inherit;
+}
+table {
+  border-collapse: collapse;
+  border-spacing: 0;
+}
+ol, ul {
+  list-style: none;
+}
+q:before,
+q:after,
+blockquote:before,
+blockquote:after {
+  content: "";
+}
+header,
+section,
+footer,
+article,
+aside {
+  display: block;
+}
+/* Preboot.less
+ * Variables and mixins to pre-ignite any new web development project
+ * ------------------------------------------------------------------ */
+.clearfix {
+  zoom: 1;
+}
+.clearfix:before, .clearfix:after {
+  display: table;
+  content: "";
+}
+.clearfix:after {
+  clear: both;
+}
+.center-block {
+  display: block;
+  margin: 0 auto;
+}
+.container {
+  width: 940px;
+  margin: 0 auto;
+  zoom: 1;
+}
+.container:before, .container:after {
+  display: table;
+  content: "";
+}
+.container:after {
+  clear: both;
+}
+/*
+ * Scaffolding
+ * Basic and global styles for generating a grid system, structural layout, and page templates
+ * ------------------------------------------------------------------------------------------- */
+.row {
+  zoom: 1;
+}
+.row:before, .row:after {
+  display: table;
+  content: "";
+}
+.row:after {
+  clear: both;
+}
+.row .span1 {
+  float: left;
+  width: 40px;
+  margin-left: 20px;
+}
+.row .span1:first-child {
+  margin-left: 0;
+}
+.row .span2 {
+  float: left;
+  width: 100px;
+  margin-left: 20px;
+}
+.row .span2:first-child {
+  margin-left: 0;
+}
+.row .span3 {
+  float: left;
+  width: 160px;
+  margin-left: 20px;
+}
+.row .span3:first-child {
+  margin-left: 0;
+}
+.row .span4 {
+  float: left;
+  width: 220px;
+  margin-left: 20px;
+}
+.row .span4:first-child {
+  margin-left: 0;
+}
+.row .span5 {
+  float: left;
+  width: 280px;
+  margin-left: 20px;
+}
+.row .span5:first-child {
+  margin-left: 0;
+}
+.row .span6 {
+  float: left;
+  width: 340px;
+  margin-left: 20px;
+}
+.row .span6:first-child {
+  margin-left: 0;
+}
+.row .span7 {
+  float: left;
+  width: 400px;
+  margin-left: 20px;
+}
+.row .span7:first-child {
+  margin-left: 0;
+}
+.row .span8 {
+  float: left;
+  width: 460px;
+  margin-left: 20px;
+}
+.row .span8:first-child {
+  margin-left: 0;
+}
+.row .span9 {
+  float: left;
+  width: 520px;
+  margin-left: 20px;
+}
+.row .span9:first-child {
+  margin-left: 0;
+}
+.row .span10 {
+  float: left;
+  width: 580px;
+  margin-left: 20px;
+}
+.row .span10:first-child {
+  margin-left: 0;
+}
+.row .span11 {
+  float: left;
+  width: 640px;
+  margin-left: 20px;
+}
+.row .span11:first-child {
+  margin-left: 0;
+}
+.row .span12 {
+  float: left;
+  width: 700px;
+  margin-left: 20px;
+}
+.row .span12:first-child {
+  margin-left: 0;
+}
+.row .span13 {
+  float: left;
+  width: 760px;
+  margin-left: 20px;
+}
+.row .span13:first-child {
+  margin-left: 0;
+}
+.row .span14 {
+  float: left;
+  width: 820px;
+  margin-left: 20px;
+}
+.row .span14:first-child {
+  margin-left: 0;
+}
+.row .span15 {
+  float: left;
+  width: 880px;
+  margin-left: 20px;
+}
+.row .span15:first-child {
+  margin-left: 0;
+}
+.row .span16 {
+  float: left;
+  width: 940px;
+  margin-left: 20px;
+}
+.row .span16:first-child {
+  margin-left: 0;
+}
+.row .offset1 {
+  margin-left: 80px !important;
+}
+.row .offset1:first-child {
+  margin-left: 60px !important;
+}
+.row .offset2 {
+  margin-left: 140px !important;
+}
+.row .offset2:first-child {
+  margin-left: 120px !important;
+}
+.row .offset3 {
+  margin-left: 200px !important;
+}
+.row .offset3:first-child {
+  margin-left: 180px !important;
+}
+.row .offset4 {
+  margin-left: 260px !important;
+}
+.row .offset4:first-child {
+  margin-left: 240px !important;
+}
+.row .offset5 {
+  margin-left: 320px !important;
+}
+.row .offset5:first-child {
+  margin-left: 300px !important;
+}
+.row .offset6 {
+  margin-left: 380px !important;
+}
+.row .offset6:first-child {
+  margin-left: 360px !important;
+}
+.row .offset7 {
+  margin-left: 440px !important;
+}
+.row .offset7:first-child {
+  margin-left: 420px !important;
+}
+.row .offset8 {
+  margin-left: 500px !important;
+}
+.row .offset8:first-child {
+  margin-left: 480px !important;
+}
+.row .offset9 {
+  margin-left: 500px !important;
+}
+.row .offset9:first-child {
+  margin-left: 480px !important;
+}
+.row .offset10 {
+  margin-left: 620px !important;
+}
+.row .offset10:first-child {
+  margin-left: 600px !important;
+}
+.row .offset11 {
+  margin-left: 680px !important;
+}
+.row .offset11:first-child {
+  margin-left: 660px !important;
+}
+.row .offset12 {
+  margin-left: 740px !important;
+}
+.row .offset12:first-child {
+  margin-left: 720px !important;
+}
+html, body {
+  background-color: #fff;
+}
+body {
+  margin: 0;
+  font-family: "Helvetica Neue", Helvetica, Arial, sans-serif;
+  font-size: 13px;
+  font-weight: normal;
+  line-height: 18px;
+  color: #808080;
+  text-rendering: optimizeLegibility;
+}
+div.container {
+  width: 940px;
+  margin: 0 auto;
+}
+div.container-fluid {
+  padding: 0 20px;
+  zoom: 1;
+}
+div.container-fluid:before, div.container-fluid:after {
+  display: table;
+  content: "";
+}
+div.container-fluid:after {
+  clear: both;
+}
+div.container-fluid div.sidebar {
+  float: left;
+  width: 220px;
+}
+div.container-fluid div.content {
+  min-width: 700px;
+  max-width: 1180px;
+  margin-left: 240px;
+}
+a {
+  color: #0069d6;
+  text-decoration: none;
+  line-height: inherit;
+  font-weight: inherit;
+}
+a:hover {
+  color: #0050a3;
+  text-decoration: underline;
+}
+.btn {
+  cursor: pointer;
+  display: inline-block;
+  background-color: #e6e6e6;
+  background-repeat: no-repeat;
+  background-image: -webkit-gradient(linear, 0 0, 0 100%, from(#ffffff), color-stop(0.25, #ffffff), to(#e6e6e6));
+  background-image: -webkit-linear-gradient(#ffffff, #ffffff 0.25, #e6e6e6);
+  background-image: -moz-linear-gradient(#ffffff, #ffffff 0.25, #e6e6e6);
+  background-image: -ms-linear-gradient(#ffffff, #ffffff 0.25, #e6e6e6);
+  background-image: -o-linear-gradient(#ffffff, #ffffff 0.25, #e6e6e6);
+  background-image: linear-gradient(#ffffff, #ffffff 0.25, #e6e6e6);
+  padding: 4px 14px;
+  text-shadow: 0 1px 1px rgba(255, 255, 255, 0.75);
+  color: #333;
+  font-size: 13px;
+  line-height: 18px;
+  border: 1px solid #ccc;
+  border-bottom-color: #bbb;
+  -webkit-border-radius: 4px;
+  -moz-border-radius: 4px;
+  border-radius: 4px;
+  -webkit-box-shadow: inset 0 1px 0 rgba(255, 255, 255, 0.2), 0 1px 2px rgba(0, 0, 0, 0.05);
+  -moz-box-shadow: inset 0 1px 0 rgba(255, 255, 255, 0.2), 0 1px 2px rgba(0, 0, 0, 0.05);
+  box-shadow: inset 0 1px 0 rgba(255, 255, 255, 0.2), 0 1px 2px rgba(0, 0, 0, 0.05);
+  -webkit-transition: 0.1s linear all;
+  -moz-transition: 0.1s linear all;
+  transition: 0.1s linear all;
+}
+.btn:hover {
+  background-position: 0 -15px;
+  color: #333;
+  text-decoration: none;
+}
+.btn.primary, .btn.danger {
+  color: #fff;
+}
+.btn.primary:hover, .btn.danger:hover {
+  color: #fff;
+}
+.btn.primary {
+  background-color: #0064cd;
+  background-repeat: repeat-x;
+  background-image: -khtml-gradient(linear, left top, left bottom, from(#049cdb), to(#0064cd));
+  background-image: -moz-linear-gradient(#049cdb, #0064cd);
+  background-image: -ms-linear-gradient(#049cdb, #0064cd);
+  background-image: -webkit-gradient(linear, left top, left bottom, color-stop(0%, #049cdb), color-stop(100%, #0064cd));
+  background-image: -webkit-linear-gradient(#049cdb, #0064cd);
+  background-image: -o-linear-gradient(#049cdb, #0064cd);
+  background-image: linear-gradient(#049cdb, #0064cd);
+  text-shadow: 0 -1px 0 rgba(0, 0, 0, 0.25);
+  border-color: #0064cd #0064cd #003f81;
+  border-color: rgba(0, 0, 0, 0.1) rgba(0, 0, 0, 0.1) rgba(0, 0, 0, 0.25);
+}
+.btn.danger {
+  background-color: #9d261d;
+  background-repeat: repeat-x;
+  background-image: -khtml-gradient(linear, left top, left bottom, from(#d83a2e), to(#9d261d));
+  background-image: -moz-linear-gradient(#d83a2e, #9d261d);
+  background-image: -ms-linear-gradient(#d83a2e, #9d261d);
+  background-image: -webkit-gradient(linear, left top, left bottom, color-stop(0%, #d83a2e), color-stop(100%, #9d261d));
+  background-image: -webkit-linear-gradient(#d83a2e, #9d261d);
+  background-image: -o-linear-gradient(#d83a2e, #9d261d);
+  background-image: linear-gradient(#d83a2e, #9d261d);
+  text-shadow: 0 -1px 0 rgba(0, 0, 0, 0.25);
+  border-color: #9d261d #9d261d #5c1611;
+  border-color: rgba(0, 0, 0, 0.1) rgba(0, 0, 0, 0.1) rgba(0, 0, 0, 0.25);
+}
+.btn.large {
+  font-size: 16px;
+  line-height: 28px;
+  -webkit-border-radius: 6px;
+  -moz-border-radius: 6px;
+  border-radius: 6px;
+}
+.btn.small {
+  padding-right: 9px;
+  padding-left: 9px;
+  font-size: 11px;
+}
+.btn.disabled {
+  background-image: none;
+  filter: alpha(opacity=65);
+  -khtml-opacity: 0.65;
+  -moz-opacity: 0.65;
+  opacity: 0.65;
+  cursor: default;
+}
+.btn:disabled {
+  background-image: none;
+  filter: alpha(opacity=65);
+  -khtml-opacity: 0.65;
+  -moz-opacity: 0.65;
+  opacity: 0.65;
+  cursor: default;
+}
+.btn:active {
+  -webkit-box-shadow: inset 0 3px 7px rgba(0, 0, 0, 0.1), 0 1px 2px rgba(0, 0, 0, 0.05);
+  -moz-box-shadow: inset 0 3px 7px rgba(0, 0, 0, 0.1), 0 1px 2px rgba(0, 0, 0, 0.05);
+  box-shadow: inset 0 3px 7px rgba(0, 0, 0, 0.1), 0 1px 2px rgba(0, 0, 0, 0.05);
+}
+button.btn::-moz-focus-inner, input[type=submit].btn::-moz-focus-inner {
+  padding: 0;
+  border: 0;
+}
+/* Typography.less
+ * Headings, body text, lists, code, and more for a versatile and durable typography system
+ * ---------------------------------------------------------------------------------------- */
+p {
+  font-size: 13px;
+  font-weight: normal;
+  line-height: 18px;
+  margin-bottom: 9px;
+}
+p small {
+  font-size: 11px;
+  color: #bfbfbf;
+}
+h1,
+h2,
+h3,
+h4,
+h5,
+h6 {
+  font-weight: bold;
+  color: #404040;
+}
+h1 small,
+h2 small,
+h3 small,
+h4 small,
+h5 small,
+h6 small {
+  color: #bfbfbf;
+}
+h1 {
+  margin-bottom: 18px;
+  font-size: 30px;
+  line-height: 36px;
+}
+h1 small {
+  font-size: 18px;
+}
+h2 {
+  font-size: 24px;
+  line-height: 36px;
+}
+h2 small {
+  font-size: 14px;
+}
+h3,
+h4,
+h5,
+h6 {
+  line-height: 36px;
+}
+h3 {
+  font-size: 18px;
+}
+h3 small {
+  font-size: 14px;
+}
+h4 {
+  font-size: 16px;
+}
+h4 small {
+  font-size: 12px;
+}
+h5 {
+  font-size: 14px;
+}
+h6 {
+  font-size: 13px;
+  color: #bfbfbf;
+  text-transform: uppercase;
+}
+ul, ol {
+  margin: 0 0 18px 25px;
+}
+ul ul,
+ul ol,
+ol ol,
+ol ul {
+  margin-bottom: 0;
+}
+ul {
+  list-style: disc;
+}
+ol {
+  list-style: decimal;
+}
+li {
+  line-height: 18px;
+  color: #808080;
+}
+ul.unstyled {
+  list-style: none;
+  margin-left: 0;
+}
+dl {
+  margin-bottom: 18px;
+}
+dl dt, dl dd {
+  line-height: 18px;
+}
+dl dt {
+  font-weight: bold;
+}
+dl dd {
+  margin-left: 9px;
+}
+hr {
+  margin: 0 0 19px;
+  border: 0;
+  border-bottom: 1px solid #eee;
+}
+strong {
+  font-style: inherit;
+  font-weight: bold;
+  line-height: inherit;
+}
+em {
+  font-style: italic;
+  font-weight: inherit;
+  line-height: inherit;
+}
+.muted {
+  color: #e6e6e6;
+}
+blockquote {
+  margin-bottom: 18px;
+  border-left: 5px solid #eee;
+  padding-left: 15px;
+}
+blockquote p {
+  font-size: 14px;
+  font-weight: 300;
+  line-height: 18px;
+  margin-bottom: 0;
+}
+blockquote small {
+  display: block;
+  font-size: 12px;
+  font-weight: 300;
+  line-height: 18px;
+  color: #bfbfbf;
+}
+blockquote small:before {
+  content: '\2014 \00A0';
+}
+address {
+  display: block;
+  line-height: 18px;
+  margin-bottom: 18px;
+}
+code, pre {
+  padding: 0 3px 2px;
+  font-family: Monaco, Andale Mono, Courier New, monospace;
+  font-size: 12px;
+  -webkit-border-radius: 3px;
+  -moz-border-radius: 3px;
+  border-radius: 3px;
+}
+code {
+  background-color: #fee9cc;
+  color: rgba(0, 0, 0, 0.75);
+  padding: 1px 3px;
+}
+pre {
+  background-color: #f5f5f5;
+  display: block;
+  padding: 17px;
+  margin: 0 0 18px;
+  line-height: 18px;
+  font-size: 12px;
+  border: 1px solid #ccc;
+  border: 1px solid rgba(0, 0, 0, 0.15);
+  -webkit-border-radius: 3px;
+  -moz-border-radius: 3px;
+  border-radius: 3px;
+  white-space: pre-wrap;
+}
+/* Forms.less
+ * Base styles for various input types, form layouts, and states
+ * ------------------------------------------------------------- */
+form {
+  margin-bottom: 18px;
+}
+form fieldset {
+  margin-bottom: 18px;
+  padding-top: 18px;
+}
+form fieldset legend {
+  display: block;
+  margin-left: 150px;
+  font-size: 20px;
+  line-height: 1;
+  *margin: 0 0 5px 145px;
+  /* IE6-7 */
+
+  *line-height: 1.5;
+  /* IE6-7 */
+
+  color: #404040;
+}
+form .clearfix {
+  margin-bottom: 18px;
+}
+form label,
+form input,
+form select,
+form textarea {
+  font-family: "Helvetica Neue", Helvetica, Arial, sans-serif;
+  font-size: 13px;
+  font-weight: normal;
+  line-height: normal;
+}
+form label {
+  padding-top: 6px;
+  font-size: 13px;
+  line-height: 18px;
+  float: left;
+  width: 130px;
+  text-align: right;
+  color: #404040;
+}
+form div.input {
+  margin-left: 150px;
+}
+form input[type=checkbox], form input[type=radio] {
+  cursor: pointer;
+}
+form input[type=text],
+form input[type=password],
+form textarea,
+form select,
+form .uneditable-input {
+  display: inline-block;
+  width: 210px;
+  margin: 0;
+  padding: 4px;
+  font-size: 13px;
+  line-height: 18px;
+  height: 18px;
+  color: #808080;
+  border: 1px solid #ccc;
+  -webkit-border-radius: 3px;
+  -moz-border-radius: 3px;
+  border-radius: 3px;
+}
+form select, form input[type=file] {
+  height: 27px;
+  line-height: 27px;
+}
+form textarea {
+  height: auto;
+}
+form .uneditable-input {
+  background-color: #eee;
+  display: block;
+  border-color: #ccc;
+  -webkit-box-shadow: inset 0 1px 2px rgba(0, 0, 0, 0.075);
+  -moz-box-shadow: inset 0 1px 2px rgba(0, 0, 0, 0.075);
+  box-shadow: inset 0 1px 2px rgba(0, 0, 0, 0.075);
+}
+form :-moz-placeholder {
+  color: #bfbfbf;
+}
+form ::-webkit-input-placeholder {
+  color: #bfbfbf;
+}
+form input[type=text],
+form input[type=password],
+form select,
+form textarea {
+  -webkit-transition: border linear 0.2s, box-shadow linear 0.2s;
+  -moz-transition: border linear 0.2s, box-shadow linear 0.2s;
+  transition: border linear 0.2s, box-shadow linear 0.2s;
+  -webkit-box-shadow: inset 0 1px 3px rgba(0, 0, 0, 0.1);
+  -moz-box-shadow: inset 0 1px 3px rgba(0, 0, 0, 0.1);
+  box-shadow: inset 0 1px 3px rgba(0, 0, 0, 0.1);
+}
+form input[type=text]:focus, form input[type=password]:focus, form textarea:focus {
+  outline: none;
+  border-color: rgba(82, 168, 236, 0.8);
+  -webkit-box-shadow: inset 0 1px 3px rgba(0, 0, 0, 0.1), 0 0 8px rgba(82, 168, 236, 0.6);
+  -moz-box-shadow: inset 0 1px 3px rgba(0, 0, 0, 0.1), 0 0 8px rgba(82, 168, 236, 0.6);
+  box-shadow: inset 0 1px 3px rgba(0, 0, 0, 0.1), 0 0 8px rgba(82, 168, 236, 0.6);
+}
+form div.error {
+  background: #fae5e3;
+  padding: 10px 0;
+  margin: -10px 0 10px;
+  -webkit-border-radius: 4px;
+  -moz-border-radius: 4px;
+  border-radius: 4px;
+}
+form div.error > label, form div.error span.help-inline, form div.error span.help-block {
+  color: #9d261d;
+}
+form div.error input[type=text], form div.error input[type=password], form div.error textarea {
+  border-color: #c87872;
+  -webkit-box-shadow: 0 0 3px rgba(171, 41, 32, 0.25);
+  -moz-box-shadow: 0 0 3px rgba(171, 41, 32, 0.25);
+  box-shadow: 0 0 3px rgba(171, 41, 32, 0.25);
+}
+form div.error input[type=text]:focus, form div.error input[type=password]:focus, form div.error textarea:focus {
+  border-color: #b9554d;
+  -webkit-box-shadow: 0 0 6px rgba(171, 41, 32, 0.5);
+  -moz-box-shadow: 0 0 6px rgba(171, 41, 32, 0.5);
+  box-shadow: 0 0 6px rgba(171, 41, 32, 0.5);
+}
+form div.error .input-prepend span.add-on, form div.error .input-append span.add-on {
+  background: #f4c8c5;
+  border-color: #c87872;
+  color: #b9554d;
+}
+form .input-mini,
+form input.mini,
+form textarea.mini,
+form select.mini {
+  width: 60px;
+}
+form .input-small,
+form input.small,
+form textarea.small,
+form select.small {
+  width: 90px;
+}
+form .input-medium,
+form input.medium,
+form textarea.medium,
+form select.medium {
+  width: 150px;
+}
+form .input-large,
+form input.large,
+form textarea.large,
+form select.large {
+  width: 210px;
+}
+form .input-xlarge,
+form input.xlarge,
+form textarea.xlarge,
+form select.xlarge {
+  width: 270px;
+}
+form .input-xxlarge,
+form input.xxlarge,
+form textarea.xxlarge,
+form select.xxlarge {
+  width: 530px;
+}
+form textarea.xxlarge {
+  overflow-y: scroll;
+}
+form input[readonly]:focus, form textarea[readonly]:focus, form input.disabled {
+  background: #f5f5f5;
+  border-color: #ddd;
+  -webkit-box-shadow: none;
+  -moz-box-shadow: none;
+  box-shadow: none;
+}
+.actions {
+  background: #f5f5f5;
+  margin-top: 18px;
+  margin-bottom: 18px;
+  padding: 17px 20px 18px 150px;
+  border-top: 1px solid #ddd;
+  -webkit-border-radius: 0 0 3px 3px;
+  -moz-border-radius: 0 0 3px 3px;
+  border-radius: 0 0 3px 3px;
+}
+.actions .secondary-action {
+  float: right;
+}
+.actions .secondary-action a {
+  line-height: 30px;
+}
+.actions .secondary-action a:hover {
+  text-decoration: underline;
+}
+.help-inline, .help-block {
+  font-size: 12px;
+  line-height: 18px;
+  color: #bfbfbf;
+}
+.help-inline {
+  padding-left: 5px;
+  *position: relative;
+  /* IE6-7 */
+
+  *top: -5px;
+  /* IE6-7 */
+
+}
+.help-block {
+  display: block;
+  max-width: 600px;
+}
+.inline-inputs {
+  color: #808080;
+}
+.inline-inputs span, .inline-inputs input[type=text] {
+  display: inline-block;
+}
+.inline-inputs input.mini {
+  width: 60px;
+}
+.inline-inputs input.small {
+  width: 90px;
+}
+.inline-inputs span {
+  padding: 0 2px 0 1px;
+}
+.input-prepend input[type=text],
+.input-append input[type=text],
+.input-prepend input[type=password],
+.input-append input[type=password] {
+  -webkit-border-radius: 0 3px 3px 0;
+  -moz-border-radius: 0 3px 3px 0;
+  border-radius: 0 3px 3px 0;
+}
+.input-prepend .add-on, .input-append .add-on {
+  background: #f5f5f5;
+  float: left;
+  display: block;
+  width: auto;
+  min-width: 16px;
+  padding: 4px 4px 4px 5px;
+  color: #bfbfbf;
+  font-weight: normal;
+  line-height: 18px;
+  height: 18px;
+  text-align: center;
+  text-shadow: 0 1px 0 #fff;
+  border: 1px solid #ccc;
+  border-right-width: 0;
+  -webkit-border-radius: 3px 0 0 3px;
+  -moz-border-radius: 3px 0 0 3px;
+  border-radius: 3px 0 0 3px;
+}
+.input-prepend .active, .input-append .active {
+  background: #a9dba9;
+  border-color: #46a546;
+}
+.input-prepend .add-on {
+  *margin-top: 1px;
+  /* IE6-7 */
+
+}
+.input-append input[type=text], .input-append input[type=password] {
+  float: left;
+  -webkit-border-radius: 3px 0 0 3px;
+  -moz-border-radius: 3px 0 0 3px;
+  border-radius: 3px 0 0 3px;
+}
+.input-append .add-on {
+  -webkit-border-radius: 0 3px 3px 0;
+  -moz-border-radius: 0 3px 3px 0;
+  border-radius: 0 3px 3px 0;
+  border-right-width: 1px;
+  border-left-width: 0;
+}
+.inputs-list {
+  margin: 0 0 5px;
+  width: 100%;
+}
+.inputs-list li {
+  display: block;
+  padding: 0;
+  width: 100%;
+}
+.inputs-list li label {
+  display: block;
+  float: none;
+  width: auto;
+  padding: 0;
+  line-height: 18px;
+  text-align: left;
+  white-space: normal;
+}
+.inputs-list li label strong {
+  color: #808080;
+}
+.inputs-list li label small {
+  font-size: 12px;
+  font-weight: normal;
+}
+.inputs-list li ul.inputs-list {
+  margin-left: 25px;
+  margin-bottom: 10px;
+  padding-top: 0;
+}
+.inputs-list li:first-child {
+  padding-top: 5px;
+}
+.inputs-list input[type=radio], .inputs-list input[type=checkbox] {
+  margin-bottom: 0;
+}
+form.form-stacked {
+  padding-left: 20px;
+}
+form.form-stacked fieldset {
+  padding-top: 9px;
+}
+form.form-stacked legend {
+  margin-left: 0;
+}
+form.form-stacked label {
+  display: block;
+  float: none;
+  width: auto;
+  font-weight: bold;
+  text-align: left;
+  line-height: 20px;
+  padding-top: 0;
+}
+form.form-stacked .clearfix {
+  margin-bottom: 9px;
+}
+form.form-stacked .clearfix div.input {
+  margin-left: 0;
+}
+form.form-stacked .inputs-list {
+  margin-bottom: 0;
+}
+form.form-stacked .inputs-list li {
+  padding-top: 0;
+}
+form.form-stacked .inputs-list li label {
+  font-weight: normal;
+  padding-top: 0;
+}
+form.form-stacked div.error {
+  padding-top: 10px;
+  padding-bottom: 10px;
+  padding-left: 10px;
+  margin-top: 0;
+  margin-left: -10px;
+}
+form.form-stacked .actions {
+  margin-left: -20px;
+  padding-left: 20px;
+}
+/*
+ * Tables.less
+ * Tables for, you guessed it, tabular data
+ * ---------------------------------------- */
+table {
+  width: 100%;
+  margin-bottom: 18px;
+  padding: 0;
+  border-collapse: separate;
+  font-size: 13px;
+}
+table th, table td {
+  padding: 10px 10px 9px;
+  line-height: 13.5px;
+  text-align: left;
+  vertical-align: middle;
+  border-bottom: 1px solid #ddd;
+}
+table th {
+  padding-top: 9px;
+  font-weight: bold;
+  border-bottom-width: 2px;
+}
+.zebra-striped tbody tr:nth-child(odd) td {
+  background-color: #f9f9f9;
+}
+.zebra-striped tbody tr:hover td {
+  background-color: #f5f5f5;
+}
+.zebra-striped .header {
+  cursor: pointer;
+}
+.zebra-striped .header:after {
+  content: "";
+  float: right;
+  margin-top: 7px;
+  border-width: 0 4px 4px;
+  border-style: solid;
+  border-color: #000 transparent;
+  visibility: hidden;
+}
+.zebra-striped .headerSortUp, .zebra-striped .headerSortDown {
+  background-color: rgba(141, 192, 219, 0.25);
+  text-shadow: 0 1px 1px rgba(255, 255, 255, 0.75);
+  -webkit-border-radius: 3px 3px 0 0;
+  -moz-border-radius: 3px 3px 0 0;
+  border-radius: 3px 3px 0 0;
+}
+.zebra-striped .header:hover:after {
+  visibility: visible;
+}
+.zebra-striped .headerSortDown:after, .zebra-striped .headerSortDown:hover:after {
+  visibility: visible;
+  filter: alpha(opacity=60);
+  -khtml-opacity: 0.6;
+  -moz-opacity: 0.6;
+  opacity: 0.6;
+}
+.zebra-striped .headerSortUp:after {
+  border-bottom: none;
+  border-left: 4px solid transparent;
+  border-right: 4px solid transparent;
+  border-top: 4px solid #000;
+  visibility: visible;
+  -webkit-box-shadow: none;
+  -moz-box-shadow: none;
+  box-shadow: none;
+  filter: alpha(opacity=60);
+  -khtml-opacity: 0.6;
+  -moz-opacity: 0.6;
+  opacity: 0.6;
+}
+table .blue {
+  color: #049cdb;
+  border-bottom-color: #049cdb;
+}
+table .headerSortUp.blue, table .headerSortDown.blue {
+  background-color: #ade6fe;
+}
+table .green {
+  color: #46a546;
+  border-bottom-color: #46a546;
+}
+table .headerSortUp.green, table .headerSortDown.green {
+  background-color: #cdeacd;
+}
+table .red {
+  color: #9d261d;
+  border-bottom-color: #9d261d;
+}
+table .headerSortUp.red, table .headerSortDown.red {
+  background-color: #f4c8c5;
+}
+table .yellow {
+  color: #ffc40d;
+  border-bottom-color: #ffc40d;
+}
+table .headerSortUp.yellow, table .headerSortDown.yellow {
+  background-color: #fff6d9;
+}
+table .orange {
+  color: #f89406;
+  border-bottom-color: #f89406;
+}
+table .headerSortUp.orange, table .headerSortDown.orange {
+  background-color: #fee9cc;
+}
+table .purple {
+  color: #7a43b6;
+  border-bottom-color: #7a43b6;
+}
+table .headerSortUp.purple, table .headerSortDown.purple {
+  background-color: #e2d5f0;
+}
+/* Patterns.less
+ * Repeatable UI elements outside the base styles provided from the scaffolding
+ * ---------------------------------------------------------------------------- */
+.topbar {
+  height: 40px;
+  position: fixed;
+  top: 0;
+  left: 0;
+  right: 0;
+  z-index: 10000;
+  overflow: visible;
+}
+.topbar .fill {
+  background: #222;
+  background-color: #222222;
+  background-repeat: repeat-x;
+  background-image: -khtml-gradient(linear, left top, left bottom, from(#333333), to(#222222));
+  background-image: -moz-linear-gradient(#333333, #222222);
+  background-image: -ms-linear-gradient(#333333, #222222);
+  background-image: -webkit-gradient(linear, left top, left bottom, color-stop(0%, #333333), color-stop(100%, #222222));
+  background-image: -webkit-linear-gradient(#333333, #222222);
+  background-image: -o-linear-gradient(#333333, #222222);
+  background-image: linear-gradient(#333333, #222222);
+  -webkit-box-shadow: 0 1px 3px rgba(0, 0, 0, 0.25), inset 0 -1px 0 rgba(0, 0, 0, 0.1);
+  -moz-box-shadow: 0 1px 3px rgba(0, 0, 0, 0.25), inset 0 -1px 0 rgba(0, 0, 0, 0.1);
+  box-shadow: 0 1px 3px rgba(0, 0, 0, 0.25), inset 0 -1px 0 rgba(0, 0, 0, 0.1);
+}
+.topbar a {
+  color: #bfbfbf;
+  text-shadow: 0 -1px 0 rgba(0, 0, 0, 0.25);
+}
+.topbar a:hover, .topbar ul li.active a {
+  background-color: #333;
+  background-color: rgba(255, 255, 255, 0.05);
+  color: #ffffff;
+  text-decoration: none;
+}
+.topbar h3 {
+  position: relative;
+}
+.topbar h3 a {
+  float: left;
+  display: block;
+  padding: 8px 20px 12px;
+  margin-left: -20px;
+  color: #ffffff;
+  font-size: 20px;
+  font-weight: 200;
+  line-height: 1;
+}
+.topbar form {
+  float: left;
+  margin: 5px 0 0 0;
+  position: relative;
+  filter: alpha(opacity=100);
+  -khtml-opacity: 1;
+  -moz-opacity: 1;
+  opacity: 1;
+}
+.topbar form input {
+  background-color: #444;
+  background-color: rgba(255, 255, 255, 0.3);
+  font-family: "Helvetica Neue", Helvetica, Arial, sans-serif;
+  font-size: normal;
+  font-weight: 13px;
+  line-height: 1;
+  width: 220px;
+  padding: 4px 9px;
+  color: #fff;
+  color: rgba(255, 255, 255, 0.75);
+  border: 1px solid #111;
+  -webkit-border-radius: 4px;
+  -moz-border-radius: 4px;
+  border-radius: 4px;
+  -webkit-box-shadow: inset 0 1px 2px rgba(0, 0, 0, 0.1), 0 1px 0px rgba(255, 255, 255, 0.25);
+  -moz-box-shadow: inset 0 1px 2px rgba(0, 0, 0, 0.1), 0 1px 0px rgba(255, 255, 255, 0.25);
+  box-shadow: inset 0 1px 2px rgba(0, 0, 0, 0.1), 0 1px 0px rgba(255, 255, 255, 0.25);
+  -webkit-transition: none;
+  -moz-transition: none;
+  transition: none;
+}
+.topbar form input:-moz-placeholder {
+  color: #e6e6e6;
+}
+.topbar form input::-webkit-input-placeholder {
+  color: #e6e6e6;
+}
+.topbar form input:hover {
+  background-color: #bfbfbf;
+  background-color: rgba(255, 255, 255, 0.5);
+  color: #fff;
+}
+.topbar form input:focus, .topbar form input.focused {
+  outline: none;
+  background-color: #fff;
+  color: #404040;
+  text-shadow: 0 1px 0 #fff;
+  border: 0;
+  padding: 5px 10px;
+  -webkit-box-shadow: 0 0 3px rgba(0, 0, 0, 0.15);
+  -moz-box-shadow: 0 0 3px rgba(0, 0, 0, 0.15);
+  box-shadow: 0 0 3px rgba(0, 0, 0, 0.15);
+}
+.topbar ul {
+  display: block;
+  float: left;
+  margin: 0 10px 0 0;
+  position: relative;
+}
+.topbar ul.secondary-nav {
+  float: right;
+  margin-left: 10px;
+  margin-right: 0;
+}
+.topbar ul li {
+  display: block;
+  float: left;
+  font-size: 13px;
+}
+.topbar ul li a {
+  display: block;
+  float: none;
+  padding: 10px 10px 11px;
+  line-height: 19px;
+  text-decoration: none;
+}
+.topbar ul li a:hover {
+  color: #fff;
+  text-decoration: none;
+}
+.topbar ul li.active a {
+  background-color: #222;
+  background-color: rgba(0, 0, 0, 0.5);
+}
+.topbar ul.primary-nav li ul {
+  left: 0;
+}
+.topbar ul.secondary-nav li ul {
+  right: 0;
+}
+.topbar ul li.menu {
+  position: relative;
+}
+.topbar ul li.menu a.menu:after {
+  width: 0px;
+  height: 0px;
+  display: inline-block;
+  content: "&darr;";
+  text-indent: -99999px;
+  vertical-align: top;
+  margin-top: 8px;
+  margin-left: 4px;
+  border-left: 4px solid transparent;
+  border-right: 4px solid transparent;
+  border-top: 4px solid #fff;
+  filter: alpha(opacity=50);
+  -khtml-opacity: 0.5;
+  -moz-opacity: 0.5;
+  opacity: 0.5;
+}
+.topbar ul li.menu.open a.menu, .topbar ul li.menu.open a:hover {
+  background-color: #444;
+  background-color: rgba(255, 255, 255, 0.1);
+  *background-color: #444;
+  /* IE6-7 */
+
+  color: #fff;
+}
+.topbar ul li.menu.open ul {
+  display: block;
+}
+.topbar ul li.menu.open ul li a {
+  background-color: transparent;
+  font-weight: normal;
+}
+.topbar ul li.menu.open ul li a:hover {
+  background-color: rgba(255, 255, 255, 0.1);
+  *background-color: #444;
+  /* IE6-7 */
+
+  color: #fff;
+}
+.topbar ul li.menu.open ul li.active a {
+  background-color: rgba(255, 255, 255, 0.1);
+  font-weight: bold;
+}
+.topbar ul li ul {
+  background-color: #333;
+  float: left;
+  display: none;
+  position: absolute;
+  top: 40px;
+  min-width: 160px;
+  max-width: 220px;
+  _width: 160px;
+  margin-left: 0;
+  margin-right: 0;
+  padding: 0;
+  text-align: left;
+  border: 0;
+  zoom: 1;
+  -webkit-border-radius: 0 0 5px 5px;
+  -moz-border-radius: 0 0 5px 5px;
+  border-radius: 0 0 5px 5px;
+  -webkit-box-shadow: 0 1px 2px rgba(0, 0, 0, 0.6);
+  -moz-box-shadow: 0 1px 2px rgba(0, 0, 0, 0.6);
+  box-shadow: 0 1px 2px rgba(0, 0, 0, 0.6);
+}
+.topbar ul li ul li {
+  float: none;
+  clear: both;
+  display: block;
+  background: none;
+  font-size: 12px;
+}
+.topbar ul li ul li a {
+  display: block;
+  padding: 6px 15px;
+  clear: both;
+  font-weight: normal;
+  line-height: 19px;
+  color: #bbb;
+}
+.topbar ul li ul li a:hover {
+  background-color: #333;
+  background-color: rgba(255, 255, 255, 0.25);
+  color: #fff;
+}
+.topbar ul li ul li.divider {
+  height: 1px;
+  overflow: hidden;
+  background: #222;
+  background: rgba(0, 0, 0, 0.2);
+  border-bottom: 1px solid rgba(255, 255, 255, 0.1);
+  margin: 5px 0;
+}
+.topbar ul li ul li span {
+  clear: both;
+  display: block;
+  background: rgba(0, 0, 0, 0.2);
+  padding: 6px 15px;
+  cursor: default;
+  color: #808080;
+  border-top: 1px solid rgba(0, 0, 0, 0.2);
+}
+.hero-unit {
+  background-color: #f5f5f5;
+  margin-top: 60px;
+  margin-bottom: 30px;
+  padding: 60px;
+  -webkit-border-radius: 6px;
+  -moz-border-radius: 6px;
+  border-radius: 6px;
+}
+.hero-unit h1 {
+  margin-bottom: 0;
+  font-size: 60px;
+  line-height: 1;
+  letter-spacing: -1px;
+}
+.hero-unit p {
+  font-size: 18px;
+  font-weight: 200;
+  line-height: 27px;
+}
+footer {
+  margin-top: 17px;
+  padding-top: 17px;
+  border-top: 1px solid #eee;
+}
+.page-header {
+  margin-bottom: 17px;
+  border-bottom: 1px solid #ddd;
+  -webkit-box-shadow: 0 1px 0 rgba(255, 255, 255, 0.5);
+  -moz-box-shadow: 0 1px 0 rgba(255, 255, 255, 0.5);
+  box-shadow: 0 1px 0 rgba(255, 255, 255, 0.5);
+}
+.page-header h1 {
+  margin-bottom: 8px;
+}
+.alert-message {
+  background-color: rgba(0, 0, 0, 0.15);
+  background-repeat: repeat-x;
+  background-image: -khtml-gradient(linear, left top, left bottom, from(transparent), to(rgba(0, 0, 0, 0.15)));
+  background-image: -moz-linear-gradient(transparent, rgba(0, 0, 0, 0.15));
+  background-image: -ms-linear-gradient(transparent, rgba(0, 0, 0, 0.15));
+  background-image: -webkit-gradient(linear, left top, left bottom, color-stop(0%, transparent), color-stop(100%, rgba(0, 0, 0, 0.15)));
+  background-image: -webkit-linear-gradient(transparent, rgba(0, 0, 0, 0.15));
+  background-image: -o-linear-gradient(transparent, rgba(0, 0, 0, 0.15));
+  background-image: linear-gradient(transparent, rgba(0, 0, 0, 0.15));
+  filter: "progid:DXImageTransform.Microsoft.gradient(startColorstr='#00000000', endColorstr='#15000000')";
+  background-color: #e6e6e6;
+  margin-bottom: 18px;
+  padding: 8px 15px;
+  color: #fff;
+  text-shadow: 0 -1px 0 rgba(0, 0, 0, 0.3);
+  border-bottom: 1px solid rgba(0, 0, 0, 0.3);
+  -webkit-border-radius: 4px;
+  -moz-border-radius: 4px;
+  border-radius: 4px;
+}
+.alert-message p {
+  color: #fff;
+  margin-bottom: 0;
+}
+.alert-message p + p {
+  margin-top: 5px;
+}
+.alert-message.error {
+  background-color: #d83a2e;
+  background-repeat: repeat-x;
+  background-image: -khtml-gradient(linear, left top, left bottom, from(#e4776f), to(#d83a2e));
+  background-image: -moz-linear-gradient(#e4776f, #d83a2e);
+  background-image: -ms-linear-gradient(#e4776f, #d83a2e);
+  background-image: -webkit-gradient(linear, left top, left bottom, color-stop(0%, #e4776f), color-stop(100%, #d83a2e));
+  background-image: -webkit-linear-gradient(#e4776f, #d83a2e);
+  background-image: -o-linear-gradient(#e4776f, #d83a2e);
+  background-image: linear-gradient(#e4776f, #d83a2e);
+  border-bottom-color: #b32b21;
+}
+.alert-message.warning {
+  background-color: #ffd040;
+  background-repeat: repeat-x;
+  background-image: -khtml-gradient(linear, left top, left bottom, from(#ffe38d), to(#ffd040));
+  background-image: -moz-linear-gradient(#ffe38d, #ffd040);
+  background-image: -ms-linear-gradient(#ffe38d, #ffd040);
+  background-image: -webkit-gradient(linear, left top, left bottom, color-stop(0%, #ffe38d), color-stop(100%, #ffd040));
+  background-image: -webkit-linear-gradient(#ffe38d, #ffd040);
+  background-image: -o-linear-gradient(#ffe38d, #ffd040);
+  background-image: linear-gradient(#ffe38d, #ffd040);
+  border-bottom-color: #ffc40d;
+}
+.alert-message.success {
+  background-color: #62bc62;
+  background-repeat: repeat-x;
+  background-image: -khtml-gradient(linear, left top, left bottom, from(#97d397), to(#62bc62));
+  background-image: -moz-linear-gradient(#97d397, #62bc62);
+  background-image: -ms-linear-gradient(#97d397, #62bc62);
+  background-image: -webkit-gradient(linear, left top, left bottom, color-stop(0%, #97d397), color-stop(100%, #62bc62));
+  background-image: -webkit-linear-gradient(#97d397, #62bc62);
+  background-image: -o-linear-gradient(#97d397, #62bc62);
+  background-image: linear-gradient(#97d397, #62bc62);
+  border-bottom-color: #46a546;
+}
+.alert-message.info {
+  background-color: #04aef4;
+  background-repeat: repeat-x;
+  background-image: -khtml-gradient(linear, left top, left bottom, from(#62cffc), to(#04aef4));
+  background-image: -moz-linear-gradient(#62cffc, #04aef4);
+  background-image: -ms-linear-gradient(#62cffc, #04aef4);
+  background-image: -webkit-gradient(linear, left top, left bottom, color-stop(0%, #62cffc), color-stop(100%, #04aef4));
+  background-image: -webkit-linear-gradient(#62cffc, #04aef4);
+  background-image: -o-linear-gradient(#62cffc, #04aef4);
+  background-image: linear-gradient(#62cffc, #04aef4);
+  border-bottom-color: #049cdb;
+}
+.alert-message .close {
+  float: right;
+  margin-top: -2px;
+  color: #000;
+  font-size: 20px;
+  font-weight: bold;
+  text-shadow: 0 1px 0 #ffffff;
+  filter: alpha(opacity=20);
+  -khtml-opacity: 0.2;
+  -moz-opacity: 0.2;
+  opacity: 0.2;
+}
+.alert-message .close:hover {
+  text-decoration: none;
+  filter: alpha(opacity=40);
+  -khtml-opacity: 0.4;
+  -moz-opacity: 0.4;
+  opacity: 0.4;
+}
+.block-message {
+  margin-bottom: 18px;
+  padding: 14px;
+  color: #404040;
+  color: rgba(0, 0, 0, 0.8);
+  *color: #404040;
+  /* IE 6-7 */
+
+  text-shadow: 0 1px 0 rgba(255, 255, 255, 0.25);
+  -webkit-border-radius: 6px;
+  -moz-border-radius: 6px;
+  border-radius: 6px;
+}
+.block-message p {
+  color: #404040;
+  color: rgba(0, 0, 0, 0.8);
+  *color: #404040;
+  /* IE 6-7 */
+
+  margin-right: 30px;
+  margin-bottom: 0;
+}
+.block-message ul {
+  margin-bottom: 0;
+}
+.block-message strong {
+  display: block;
+}
+.block-message.error {
+  background: #f8dcda;
+  border: 1px solid #f4c8c5;
+}
+.block-message.warning {
+  background: #fff0c0;
+  border: 1px solid #ffe38d;
+}
+.block-message.success {
+  background: #dff1df;
+  border: 1px solid #bbe2bb;
+}
+.block-message.info {
+  background: #c7eefe;
+  border: 1px solid #ade6fe;
+}
+.tabs, .pills {
+  margin: 0 0 20px;
+  padding: 0;
+  zoom: 1;
+}
+.tabs:before,
+.pills:before,
+.tabs:after,
+.pills:after {
+  display: table;
+  content: "";
+}
+.tabs:after, .pills:after {
+  clear: both;
+}
+.tabs li, .pills li {
+  display: inline;
+}
+.tabs li a, .pills li a {
+  float: left;
+  width: auto;
+}
+.tabs {
+  width: 100%;
+  border-bottom: 1px solid #bfbfbf;
+}
+.tabs li a {
+  margin-bottom: -1px;
+  margin-right: 2px;
+  padding: 0 15px;
+  line-height: 35px;
+  -webkit-border-radius: 3px 3px 0 0;
+  -moz-border-radius: 3px 3px 0 0;
+  border-radius: 3px 3px 0 0;
+}
+.tabs li a:hover {
+  background-color: #e6e6e6;
+  border-bottom: 1px solid #bfbfbf;
+}
+.tabs li.active a {
+  background-color: #fff;
+  padding: 0 14px;
+  border: 1px solid #ccc;
+  border-bottom: 0;
+  color: #808080;
+}
+.pills li a {
+  margin: 5px 3px 5px 0;
+  padding: 0 15px;
+  text-shadow: 0 1px 1px #fff;
+  line-height: 30px;
+  -webkit-border-radius: 15px;
+  -moz-border-radius: 15px;
+  border-radius: 15px;
+}
+.pills li a:hover {
+  background: #0050a3;
+  color: #fff;
+  text-decoration: none;
+  text-shadow: 0 1px 1px rgba(0, 0, 0, 0.25);
+}
+.pills li.active a {
+  background: #0069d6;
+  color: #fff;
+  text-shadow: 0 1px 1px rgba(0, 0, 0, 0.25);
+}
+.pagination {
+  height: 36px;
+  margin: 18px 0;
+}
+.pagination ul {
+  float: left;
+  margin: 0;
+  border: 1px solid #ddd;
+  border: 1px solid rgba(0, 0, 0, 0.15);
+  -webkit-border-radius: 3px;
+  -moz-border-radius: 3px;
+  border-radius: 3px;
+  -webkit-box-shadow: 0 1px 2px rgba(0, 0, 0, 0.05);
+  -moz-box-shadow: 0 1px 2px rgba(0, 0, 0, 0.05);
+  box-shadow: 0 1px 2px rgba(0, 0, 0, 0.05);
+}
+.pagination ul li {
+  display: inline;
+}
+.pagination ul li a {
+  float: left;
+  padding: 0 14px;
+  line-height: 34px;
+  border-right: 1px solid;
+  border-right-color: #ddd;
+  border-right-color: rgba(0, 0, 0, 0.15);
+  *border-right-color: #ddd;
+  /* IE6-7 */
+
+  text-decoration: none;
+}
+.pagination ul li a:hover, .pagination ul li.active a {
+  background-color: #c7eefe;
+}
+.pagination ul li.disabled a, .pagination ul li.disabled a:hover {
+  background-color: none;
+  color: #bfbfbf;
+}
+.pagination ul li.next a {
+  border: 0;
+}
+.well {
+  background-color: #f5f5f5;
+  margin-bottom: 20px;
+  padding: 19px;
+  min-height: 20px;
+  border: 1px solid #eee;
+  border: 1px solid rgba(0, 0, 0, 0.05);
+  -webkit-border-radius: 4px;
+  -moz-border-radius: 4px;
+  border-radius: 4px;
+  -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.05);
+  -moz-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.05);
+  box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.05);
+}
+.modal-backdrop {
+  background-color: rgba(0, 0, 0, 0.5);
+  position: fixed;
+  top: 0;
+  left: 0;
+  right: 0;
+  bottom: 0;
+  z-index: 1000;
+}
+.modal {
+  position: fixed;
+  top: 50%;
+  left: 50%;
+  z-index: 2000;
+  width: 560px;
+  margin: -280px 0 0 -250px;
+  background-color: #ffffff;
+  border: 1px solid #999;
+  border: 1px solid rgba(0, 0, 0, 0.3);
+  *border: 1px solid #999;
+  /* IE6-7 */
+
+  -webkit-border-radius: 6px;
+  -moz-border-radius: 6px;
+  border-radius: 6px;
+  -webkit-box-shadow: 0 3px 7px rgba(0, 0, 0, 0.3);
+  -moz-box-shadow: 0 3px 7px rgba(0, 0, 0, 0.3);
+  box-shadow: 0 3px 7px rgba(0, 0, 0, 0.3);
+  -webkit-background-clip: padding-box;
+  -moz-background-clip: padding-box;
+  background-clip: padding-box;
+}
+.modal .modal-header {
+  border-bottom: 1px solid #eee;
+  padding: 5px 20px;
+}
+.modal .modal-header .close {
+  position: absolute;
+  right: 10px;
+  top: 10px;
+  color: #999;
+  line-height: 10px;
+  font-size: 18px;
+}
+.modal .modal-body {
+  padding: 20px;
+}
+.modal .modal-footer {
+  background-color: #f5f5f5;
+  padding: 14px 20px 15px;
+  border-top: 1px solid #ddd;
+  -webkit-border-radius: 0 0 6px 6px;
+  -moz-border-radius: 0 0 6px 6px;
+  border-radius: 0 0 6px 6px;
+  -webkit-box-shadow: inset 0 1px 0 #ffffff;
+  -moz-box-shadow: inset 0 1px 0 #ffffff;
+  box-shadow: inset 0 1px 0 #ffffff;
+  zoom: 1;
+}
+.modal .modal-footer:before, .modal .modal-footer:after {
+  display: table;
+  content: "";
+}
+.modal .modal-footer:after {
+  clear: both;
+}
+.modal .modal-footer .btn {
+  float: right;
+  margin-left: 10px;
+}
+.twipsy {
+  display: block;
+  position: absolute;
+  visibility: visible;
+  padding: 5px;
+  font-size: 11px;
+  z-index: 1000;
+  filter: alpha(opacity=80);
+  -khtml-opacity: 0.8;
+  -moz-opacity: 0.8;
+  opacity: 0.8;
+}
+.twipsy.above .twipsy-arrow {
+  bottom: 0;
+  left: 50%;
+  margin-left: -5px;
+  border-left: 5px solid transparent;
+  border-right: 5px solid transparent;
+  border-top: 5px solid #000000;
+}
+.twipsy.left .twipsy-arrow {
+  top: 50%;
+  right: 0;
+  margin-top: -5px;
+  border-top: 5px solid transparent;
+  border-bottom: 5px solid transparent;
+  border-left: 5px solid #000000;
+}
+.twipsy.below .twipsy-arrow {
+  top: 0;
+  left: 50%;
+  margin-left: -5px;
+  border-left: 5px solid transparent;
+  border-right: 5px solid transparent;
+  border-bottom: 5px solid #000000;
+}
+.twipsy.right .twipsy-arrow {
+  top: 50%;
+  left: 0;
+  margin-top: -5px;
+  border-top: 5px solid transparent;
+  border-bottom: 5px solid transparent;
+  border-right: 5px solid #000000;
+}
+.twipsy .twipsy-inner {
+  padding: 3px 8px;
+  background-color: #000;
+  color: white;
+  text-align: center;
+  max-width: 200px;
+  text-decoration: none;
+  -webkit-border-radius: 4px;
+  -moz-border-radius: 4px;
+  border-radius: 4px;
+}
+.twipsy .twipsy-arrow {
+  position: absolute;
+  width: 0;
+  height: 0;
+}
+.popover {
+  position: absolute;
+  top: 0;
+  left: 0;
+  z-index: 1000;
+  padding: 5px;
+  display: none;
+}
+.popover.above .arrow {
+  bottom: 0;
+  left: 50%;
+  margin-left: -5px;
+  border-left: 5px solid transparent;
+  border-right: 5px solid transparent;
+  border-top: 5px solid #000000;
+}
+.popover.right .arrow {
+  top: 50%;
+  left: 0;
+  margin-top: -5px;
+  border-top: 5px solid transparent;
+  border-bottom: 5px solid transparent;
+  border-right: 5px solid #000000;
+}
+.popover.below .arrow {
+  top: 0;
+  left: 50%;
+  margin-left: -5px;
+  border-left: 5px solid transparent;
+  border-right: 5px solid transparent;
+  border-bottom: 5px solid #000000;
+}
+.popover.left .arrow {
+  top: 50%;
+  right: 0;
+  margin-top: -5px;
+  border-top: 5px solid transparent;
+  border-bottom: 5px solid transparent;
+  border-left: 5px solid #000000;
+}
+.popover .arrow {
+  position: absolute;
+  width: 0;
+  height: 0;
+}
+.popover .inner {
+  background-color: #333;
+  background-color: rgba(0, 0, 0, 0.8);
+  *background-color: #333;
+  /* IE 6-7 */
+
+  padding: 3px;
+  overflow: hidden;
+  width: 280px;
+  -webkit-border-radius: 6px;
+  -moz-border-radius: 6px;
+  border-radius: 6px;
+  -webkit-box-shadow: 0 3px 7px rgba(0, 0, 0, 0.3);
+  -moz-box-shadow: 0 3px 7px rgba(0, 0, 0, 0.3);
+  box-shadow: 0 3px 7px rgba(0, 0, 0, 0.3);
+}
+.popover .title {
+  background-color: #f5f5f5;
+  padding: 9px 15px;
+  line-height: 1;
+  -webkit-border-radius: 3px 3px 0 0;
+  -moz-border-radius: 3px 3px 0 0;
+  border-radius: 3px 3px 0 0;
+  border-bottom: 1px solid #eee;
+}
+.popover .content {
+  background-color: #ffffff;
+  padding: 14px;
+  -webkit-border-radius: 0 0 3px 3px;
+  -moz-border-radius: 0 0 3px 3px;
+  border-radius: 0 0 3px 3px;
+  -webkit-background-clip: padding-box;
+  -moz-background-clip: padding-box;
+  background-clip: padding-box;
+}
+.popover .content p, .popover .content ul, .popover .content ol {
+  margin-bottom: 0;
+}
\ No newline at end of file
diff --git a/src/ui/public/js/jquery-1.6.2.min.js b/src/ui/public/js/jquery-1.6.2.min.js
new file mode 100644
index 0000000..8cdc80e
--- /dev/null
+++ b/src/ui/public/js/jquery-1.6.2.min.js
@@ -0,0 +1,18 @@
+/*!
+ * jQuery JavaScript Library v1.6.2
+ * http://jquery.com/
+ *
+ * Copyright 2011, John Resig
+ * Dual licensed under the MIT or GPL Version 2 licenses.
+ * http://jquery.org/license
+ *
+ * Includes Sizzle.js
+ * http://sizzlejs.com/
+ * Copyright 2011, The Dojo Foundation
+ * Released under the MIT, BSD, and GPL Licenses.
+ *
+ * Date: Thu Jun 30 14:16:56 2011 -0400
+ */
+(function(a,b){function cv(a){return f.isWindow(a)?a:a.nodeType===9?a.defaultView||a.parentWindow:!1}function cs(a){if(!cg[a]){var b=c.body,d=f("<"+a+">").appendTo(b),e=d.css("display");d.remove();if(e==="none"||e===""){ch||(ch=c.createElement("iframe"),ch.frameBorder=ch.width=ch.height=0),b.appendChild(ch);if(!ci||!ch.createElement)ci=(ch.contentWindow||ch.contentDocument).document,ci.write((c.compatMode==="CSS1Compat"?"<!doctype html>":"")+"<html><body>"),ci.close();d=ci.createElement(a),ci.body.appendChild(d),e=f.css(d,"display"),b.removeChild(ch)}cg[a]=e}return cg[a]}function cr(a,b){var c={};f.each(cm.concat.apply([],cm.slice(0,b)),function(){c[this]=a});return c}function cq(){cn=b}function cp(){setTimeout(cq,0);return cn=f.now()}function cf(){try{return new a.ActiveXObject("Microsoft.XMLHTTP")}catch(b){}}function ce(){try{return new a.XMLHttpRequest}catch(b){}}function b$(a,c){a.dataFilter&&(c=a.dataFilter(c,a.dataType));var d=a.dataTypes,e={},g,h,i=d.length,j,k=d[0],l,m,n,o,p;for(g=1;g<i;g++){if(g===1)for(h in a.converters)typeof h=="string"&&(e[h.toLowerCase()]=a.converters[h]);l=k,k=d[g];if(k==="*")k=l;else if(l!=="*"&&l!==k){m=l+" "+k,n=e[m]||e["* "+k];if(!n){p=b;for(o in e){j=o.split(" ");if(j[0]===l||j[0]==="*"){p=e[j[1]+" "+k];if(p){o=e[o],o===!0?n=p:p===!0&&(n=o);break}}}}!n&&!p&&f.error("No conversion from "+m.replace(" "," to ")),n!==!0&&(c=n?n(c):p(o(c)))}}return c}function bZ(a,c,d){var e=a.contents,f=a.dataTypes,g=a.responseFields,h,i,j,k;for(i in g)i in d&&(c[g[i]]=d[i]);while(f[0]==="*")f.shift(),h===b&&(h=a.mimeType||c.getResponseHeader("content-type"));if(h)for(i in e)if(e[i]&&e[i].test(h)){f.unshift(i);break}if(f[0]in d)j=f[0];else{for(i in d){if(!f[0]||a.converters[i+" "+f[0]]){j=i;break}k||(k=i)}j=j||k}if(j){j!==f[0]&&f.unshift(j);return d[j]}}function bY(a,b,c,d){if(f.isArray(b))f.each(b,function(b,e){c||bC.test(a)?d(a,e):bY(a+"["+(typeof e=="object"||f.isArray(e)?b:"")+"]",e,c,d)});else if(!c&&b!=null&&typeof b=="object")for(var e in b)bY(a+"["+e+"]",b[e],c,d);else d(a,b)}function bX(a,c,d,e,f,g){f=f||c.dataTypes[0],g=g||{},g[f]=!0;var h=a[f],i=0,j=h?h.length:0,k=a===bR,l;for(;i<j&&(k||!l);i++)l=h[i](c,d,e),typeof l=="string"&&(!k||g[l]?l=b:(c.dataTypes.unshift(l),l=bX(a,c,d,e,l,g)));(k||!l)&&!g["*"]&&(l=bX(a,c,d,e,"*",g));return l}function bW(a){return function(b,c){typeof b!="string"&&(c=b,b="*");if(f.isFunction(c)){var d=b.toLowerCase().split(bN),e=0,g=d.length,h,i,j;for(;e<g;e++)h=d[e],j=/^\+/.test(h),j&&(h=h.substr(1)||"*"),i=a[h]=a[h]||[],i[j?"unshift":"push"](c)}}}function bA(a,b,c){var d=b==="width"?a.offsetWidth:a.offsetHeight,e=b==="width"?bv:bw;if(d>0){c!=="border"&&f.each(e,function(){c||(d-=parseFloat(f.css(a,"padding"+this))||0),c==="margin"?d+=parseFloat(f.css(a,c+this))||0:d-=parseFloat(f.css(a,"border"+this+"Width"))||0});return d+"px"}d=bx(a,b,b);if(d<0||d==null)d=a.style[b]||0;d=parseFloat(d)||0,c&&f.each(e,function(){d+=parseFloat(f.css(a,"padding"+this))||0,c!=="padding"&&(d+=parseFloat(f.css(a,"border"+this+"Width"))||0),c==="margin"&&(d+=parseFloat(f.css(a,c+this))||0)});return d+"px"}function bm(a,b){b.src?f.ajax({url:b.src,async:!1,dataType:"script"}):f.globalEval((b.text||b.textContent||b.innerHTML||"").replace(be,"/*$0*/")),b.parentNode&&b.parentNode.removeChild(b)}function bl(a){f.nodeName(a,"input")?bk(a):"getElementsByTagName"in a&&f.grep(a.getElementsByTagName("input"),bk)}function bk(a){if(a.type==="checkbox"||a.type==="radio")a.defaultChecked=a.checked}function bj(a){return"getElementsByTagName"in a?a.getElementsByTagName("*"):"querySelectorAll"in a?a.querySelectorAll("*"):[]}function bi(a,b){var c;if(b.nodeType===1){b.clearAttributes&&b.clearAttributes(),b.mergeAttributes&&b.mergeAttributes(a),c=b.nodeName.toLowerCase();if(c==="object")b.outerHTML=a.outerHTML;else if(c!=="input"||a.type!=="checkbox"&&a.type!=="radio"){if(c==="option")b.selected=a.defaultSelected;else if(c==="input"||c==="textarea")b.defaultValue=a.defaultValue}else a.checked&&(b.defaultChecked=b.checked=a.checked),b.value!==a.value&&(b.value=a.value);b.removeAttribute(f.expando)}}function bh(a,b){if(b.nodeType===1&&!!f.hasData(a)){var c=f.expando,d=f.data(a),e=f.data(b,d);if(d=d[c]){var g=d.events;e=e[c]=f.extend({},d);if(g){delete e.handle,e.events={};for(var h in g)for(var i=0,j=g[h].length;i<j;i++)f.event.add(b,h+(g[h][i].namespace?".":"")+g[h][i].namespace,g[h][i],g[h][i].data)}}}}function bg(a,b){return f.nodeName(a,"table")?a.getElementsByTagName("tbody")[0]||a.appendChild(a.ownerDocument.createElement("tbody")):a}function W(a,b,c){b=b||0;if(f.isFunction(b))return f.grep(a,function(a,d){var e=!!b.call(a,d,a);return e===c});if(b.nodeType)return f.grep(a,function(a,d){return a===b===c});if(typeof b=="string"){var d=f.grep(a,function(a){return a.nodeType===1});if(R.test(b))return f.filter(b,d,!c);b=f.filter(b,d)}return f.grep(a,function(a,d){return f.inArray(a,b)>=0===c})}function V(a){return!a||!a.parentNode||a.parentNode.nodeType===11}function N(a,b){return(a&&a!=="*"?a+".":"")+b.replace(z,"`").replace(A,"&")}function M(a){var b,c,d,e,g,h,i,j,k,l,m,n,o,p=[],q=[],r=f._data(this,"events");if(!(a.liveFired===this||!r||!r.live||a.target.disabled||a.button&&a.type==="click")){a.namespace&&(n=new RegExp("(^|\\.)"+a.namespace.split(".").join("\\.(?:.*\\.)?")+"(\\.|$)")),a.liveFired=this;var s=r.live.slice(0);for(i=0;i<s.length;i++)g=s[i],g.origType.replace(x,"")===a.type?q.push(g.selector):s.splice(i--,1);e=f(a.target).closest(q,a.currentTarget);for(j=0,k=e.length;j<k;j++){m=e[j];for(i=0;i<s.length;i++){g=s[i];if(m.selector===g.selector&&(!n||n.test(g.namespace))&&!m.elem.disabled){h=m.elem,d=null;if(g.preType==="mouseenter"||g.preType==="mouseleave")a.type=g.preType,d=f(a.relatedTarget).closest(g.selector)[0],d&&f.contains(h,d)&&(d=h);(!d||d!==h)&&p.push({elem:h,handleObj:g,level:m.level})}}}for(j=0,k=p.length;j<k;j++){e=p[j];if(c&&e.level>c)break;a.currentTarget=e.elem,a.data=e.handleObj.data,a.handleObj=e.handleObj,o=e.handleObj.origHandler.apply(e.elem,arguments);if(o===!1||a.isPropagationStopped()){c=e.level,o===!1&&(b=!1);if(a.isImmediatePropagationStopped())break}}return b}}function K(a,c,d){var e=f.extend({},d[0]);e.type=a,e.originalEvent={},e.liveFired=b,f.event.handle.call(c,e),e.isDefaultPrevented()&&d[0].preventDefault()}function E(){return!0}function D(){return!1}function m(a,c,d){var e=c+"defer",g=c+"queue",h=c+"mark",i=f.data(a,e,b,!0);i&&(d==="queue"||!f.data(a,g,b,!0))&&(d==="mark"||!f.data(a,h,b,!0))&&setTimeout(function(){!f.data(a,g,b,!0)&&!f.data(a,h,b,!0)&&(f.removeData(a,e,!0),i.resolve())},0)}function l(a){for(var b in a)if(b!=="toJSON")return!1;return!0}function k(a,c,d){if(d===b&&a.nodeType===1){var e="data-"+c.replace(j,"$1-$2").toLowerCase();d=a.getAttribute(e);if(typeof d=="string"){try{d=d==="true"?!0:d==="false"?!1:d==="null"?null:f.isNaN(d)?i.test(d)?f.parseJSON(d):d:parseFloat(d)}catch(g){}f.data(a,c,d)}else d=b}return d}var c=a.document,d=a.navigator,e=a.location,f=function(){function J(){if(!e.isReady){try{c.documentElement.doScroll("left")}catch(a){setTimeout(J,1);return}e.ready()}}var e=function(a,b){return new e.fn.init(a,b,h)},f=a.jQuery,g=a.$,h,i=/^(?:[^<]*(<[\w\W]+>)[^>]*$|#([\w\-]*)$)/,j=/\S/,k=/^\s+/,l=/\s+$/,m=/\d/,n=/^<(\w+)\s*\/?>(?:<\/\1>)?$/,o=/^[\],:{}\s]*$/,p=/\\(?:["\\\/bfnrt]|u[0-9a-fA-F]{4})/g,q=/"[^"\\\n\r]*"|true|false|null|-?\d+(?:\.\d*)?(?:[eE][+\-]?\d+)?/g,r=/(?:^|:|,)(?:\s*\[)+/g,s=/(webkit)[ \/]([\w.]+)/,t=/(opera)(?:.*version)?[ \/]([\w.]+)/,u=/(msie) ([\w.]+)/,v=/(mozilla)(?:.*? rv:([\w.]+))?/,w=/-([a-z])/ig,x=function(a,b){return b.toUpperCase()},y=d.userAgent,z,A,B,C=Object.prototype.toString,D=Object.prototype.hasOwnProperty,E=Array.prototype.push,F=Array.prototype.slice,G=String.prototype.trim,H=Array.prototype.indexOf,I={};e.fn=e.prototype={constructor:e,init:function(a,d,f){var g,h,j,k;if(!a)return this;if(a.nodeType){this.context=this[0]=a,this.length=1;return this}if(a==="body"&&!d&&c.body){this.context=c,this[0]=c.body,this.selector=a,this.length=1;return this}if(typeof a=="string"){a.charAt(0)!=="<"||a.charAt(a.length-1)!==">"||a.length<3?g=i.exec(a):g=[null,a,null];if(g&&(g[1]||!d)){if(g[1]){d=d instanceof e?d[0]:d,k=d?d.ownerDocument||d:c,j=n.exec(a),j?e.isPlainObject(d)?(a=[c.createElement(j[1])],e.fn.attr.call(a,d,!0)):a=[k.createElement(j[1])]:(j=e.buildFragment([g[1]],[k]),a=(j.cacheable?e.clone(j.fragment):j.fragment).childNodes);return e.merge(this,a)}h=c.getElementById(g[2]);if(h&&h.parentNode){if(h.id!==g[2])return f.find(a);this.length=1,this[0]=h}this.context=c,this.selector=a;return this}return!d||d.jquery?(d||f).find(a):this.constructor(d).find(a)}if(e.isFunction(a))return f.ready(a);a.selector!==b&&(this.selector=a.selector,this.context=a.context);return e.makeArray(a,this)},selector:"",jquery:"1.6.2",length:0,size:function(){return this.length},toArray:function(){return F.call(this,0)},get:function(a){return a==null?this.toArray():a<0?this[this.length+a]:this[a]},pushStack:function(a,b,c){var d=this.constructor();e.isArray(a)?E.apply(d,a):e.merge(d,a),d.prevObject=this,d.context=this.context,b==="find"?d.selector=this.selector+(this.selector?" ":"")+c:b&&(d.selector=this.selector+"."+b+"("+c+")");return d},each:function(a,b){return e.each(this,a,b)},ready:function(a){e.bindReady(),A.done(a);return this},eq:function(a){return a===-1?this.slice(a):this.slice(a,+a+1)},first:function(){return this.eq(0)},last:function(){return this.eq(-1)},slice:function(){return this.pushStack(F.apply(this,arguments),"slice",F.call(arguments).join(","))},map:function(a){return this.pushStack(e.map(this,function(b,c){return a.call(b,c,b)}))},end:function(){return this.prevObject||this.constructor(null)},push:E,sort:[].sort,splice:[].splice},e.fn.init.prototype=e.fn,e.extend=e.fn.extend=function(){var a,c,d,f,g,h,i=arguments[0]||{},j=1,k=arguments.length,l=!1;typeof i=="boolean"&&(l=i,i=arguments[1]||{},j=2),typeof i!="object"&&!e.isFunction(i)&&(i={}),k===j&&(i=this,--j);for(;j<k;j++)if((a=arguments[j])!=null)for(c in a){d=i[c],f=a[c];if(i===f)continue;l&&f&&(e.isPlainObject(f)||(g=e.isArray(f)))?(g?(g=!1,h=d&&e.isArray(d)?d:[]):h=d&&e.isPlainObject(d)?d:{},i[c]=e.extend(l,h,f)):f!==b&&(i[c]=f)}return i},e.extend({noConflict:function(b){a.$===e&&(a.$=g),b&&a.jQuery===e&&(a.jQuery=f);return e},isReady:!1,readyWait:1,holdReady:function(a){a?e.readyWait++:e.ready(!0)},ready:function(a){if(a===!0&&!--e.readyWait||a!==!0&&!e.isReady){if(!c.body)return setTimeout(e.ready,1);e.isReady=!0;if(a!==!0&&--e.readyWait>0)return;A.resolveWith(c,[e]),e.fn.trigger&&e(c).trigger("ready").unbind("ready")}},bindReady:function(){if(!A){A=e._Deferred();if(c.readyState==="complete")return setTimeout(e.ready,1);if(c.addEventListener)c.addEventListener("DOMContentLoaded",B,!1),a.addEventListener("load",e.ready,!1);else if(c.attachEvent){c.attachEvent("onreadystatechange",B),a.attachEvent("onload",e.ready);var b=!1;try{b=a.frameElement==null}catch(d){}c.documentElement.doScroll&&b&&J()}}},isFunction:function(a){return e.type(a)==="function"},isArray:Array.isArray||function(a){return e.type(a)==="array"},isWindow:function(a){return a&&typeof a=="object"&&"setInterval"in a},isNaN:function(a){return a==null||!m.test(a)||isNaN(a)},type:function(a){return a==null?String(a):I[C.call(a)]||"object"},isPlainObject:function(a){if(!a||e.type(a)!=="object"||a.nodeType||e.isWindow(a))return!1;if(a.constructor&&!D.call(a,"constructor")&&!D.call(a.constructor.prototype,"isPrototypeOf"))return!1;var c;for(c in a);return c===b||D.call(a,c)},isEmptyObject:function(a){for(var b in a)return!1;return!0},error:function(a){throw a},parseJSON:function(b){if(typeof b!="string"||!b)return null;b=e.trim(b);if(a.JSON&&a.JSON.parse)return a.JSON.parse(b);if(o.test(b.replace(p,"@").replace(q,"]").replace(r,"")))return(new Function("return "+b))();e.error("Invalid JSON: "+b)},parseXML:function(b,c,d){a.DOMParser?(d=new DOMParser,c=d.parseFromString(b,"text/xml")):(c=new ActiveXObject("Microsoft.XMLDOM"),c.async="false",c.loadXML(b)),d=c.documentElement,(!d||!d.nodeName||d.nodeName==="parsererror")&&e.error("Invalid XML: "+b);return c},noop:function(){},globalEval:function(b){b&&j.test(b)&&(a.execScript||function(b){a.eval.call(a,b)})(b)},camelCase:function(a){return a.replace(w,x)},nodeName:function(a,b){return a.nodeName&&a.nodeName.toUpperCase()===b.toUpperCase()},each:function(a,c,d){var f,g=0,h=a.length,i=h===b||e.isFunction(a);if(d){if(i){for(f in a)if(c.apply(a[f],d)===!1)break}else for(;g<h;)if(c.apply(a[g++],d)===!1)break}else if(i){for(f in a)if(c.call(a[f],f,a[f])===!1)break}else for(;g<h;)if(c.call(a[g],g,a[g++])===!1)break;return a},trim:G?function(a){return a==null?"":G.call(a)}:function(a){return a==null?"":(a+"").replace(k,"").replace(l,"")},makeArray:function(a,b){var c=b||[];if(a!=null){var d=e.type(a);a.length==null||d==="string"||d==="function"||d==="regexp"||e.isWindow(a)?E.call(c,a):e.merge(c,a)}return c},inArray:function(a,b){if(H)return H.call(b,a);for(var c=0,d=b.length;c<d;c++)if(b[c]===a)return c;return-1},merge:function(a,c){var d=a.length,e=0;if(typeof c.length=="number")for(var f=c.length;e<f;e++)a[d++]=c[e];else while(c[e]!==b)a[d++]=c[e++];a.length=d;return a},grep:function(a,b,c){var d=[],e;c=!!c;for(var f=0,g=a.length;f<g;f++)e=!!b(a[f],f),c!==e&&d.push(a[f]);return d},map:function(a,c,d){var f,g,h=[],i=0,j=a.length,k=a instanceof e||j!==b&&typeof j=="number"&&(j>0&&a[0]&&a[j-1]||j===0||e.isArray(a));if(k)for(;i<j;i++)f=c(a[i],i,d),f!=null&&(h[h.length]=f);else for(g in a)f=c(a[g],g,d),f!=null&&(h[h.length]=f);return h.concat.apply([],h)},guid:1,proxy:function(a,c){if(typeof c=="string"){var d=a[c];c=a,a=d}if(!e.isFunction(a))return b;var f=F.call(arguments,2),g=function(){return a.apply(c,f.concat(F.call(arguments)))};g.guid=a.guid=a.guid||g.guid||e.guid++;return g},access:function(a,c,d,f,g,h){var i=a.length;if(typeof c=="object"){for(var j in c)e.access(a,j,c[j],f,g,d);return a}if(d!==b){f=!h&&f&&e.isFunction(d);for(var k=0;k<i;k++)g(a[k],c,f?d.call(a[k],k,g(a[k],c)):d,h);return a}return i?g(a[0],c):b},now:function(){return(new Date).getTime()},uaMatch:function(a){a=a.toLowerCase();var b=s.exec(a)||t.exec(a)||u.exec(a)||a.indexOf("compatible")<0&&v.exec(a)||[];return{browser:b[1]||"",version:b[2]||"0"}},sub:function(){function a(b,c){return new a.fn.init(b,c)}e.extend(!0,a,this),a.superclass=this,a.fn=a.prototype=this(),a.fn.constructor=a,a.sub=this.sub,a.fn.init=function(d,f){f&&f instanceof e&&!(f instanceof a)&&(f=a(f));return e.fn.init.call(this,d,f,b)},a.fn.init.prototype=a.fn;var b=a(c);return a},browser:{}}),e.each("Boolean Number String Function Array Date RegExp Object".split(" "),function(a,b){I["[object "+b+"]"]=b.toLowerCase()}),z=e.uaMatch(y),z.browser&&(e.browser[z.browser]=!0,e.browser.version=z.version),e.browser.webkit&&(e.browser.safari=!0),j.test(" ")&&(k=/^[\s\xA0]+/,l=/[\s\xA0]+$/),h=e(c),c.addEventListener?B=function(){c.removeEventListener("DOMContentLoaded",B,!1),e.ready()}:c.attachEvent&&(B=function(){c.readyState==="complete"&&(c.detachEvent("onreadystatechange",B),e.ready())});return e}(),g="done fail isResolved isRejected promise then always pipe".split(" "),h=[].slice;f.extend({_Deferred:function(){var a=[],b,c,d,e={done:function(){if(!d){var c=arguments,g,h,i,j,k;b&&(k=b,b=0);for(g=0,h=c.length;g<h;g++)i=c[g],j=f.type(i),j==="array"?e.done.apply(e,i):j==="function"&&a.push(i);k&&e.resolveWith(k[0],k[1])}return this},resolveWith:function(e,f){if(!d&&!b&&!c){f=f||[],c=1;try{while(a[0])a.shift().apply(e,f)}finally{b=[e,f],c=0}}return this},resolve:function(){e.resolveWith(this,arguments);return this},isResolved:function(){return!!c||!!b},cancel:function(){d=1,a=[];return this}};return e},Deferred:function(a){var b=f._Deferred(),c=f._Deferred(),d;f.extend(b,{then:function(a,c){b.done(a).fail(c);return this},always:function(){return b.done.apply(b,arguments).fail.apply(this,arguments)},fail:c.done,rejectWith:c.resolveWith,reject:c.resolve,isRejected:c.isResolved,pipe:function(a,c){return f.Deferred(function(d){f.each({done:[a,"resolve"],fail:[c,"reject"]},function(a,c){var e=c[0],g=c[1],h;f.isFunction(e)?b[a](function(){h=e.apply(this,arguments),h&&f.isFunction(h.promise)?h.promise().then(d.resolve,d.reject):d[g](h)}):b[a](d[g])})}).promise()},promise:function(a){if(a==null){if(d)return d;d=a={}}var c=g.length;while(c--)a[g[c]]=b[g[c]];return a}}),b.done(c.cancel).fail(b.cancel),delete b.cancel,a&&a.call(b,b);return b},when:function(a){function i(a){return function(c){b[a]=arguments.length>1?h.call(arguments,0):c,--e||g.resolveWith(g,h.call(b,0))}}var b=arguments,c=0,d=b.length,e=d,g=d<=1&&a&&f.isFunction(a.promise)?a:f.Deferred();if(d>1){for(;c<d;c++)b[c]&&f.isFunction(b[c].promise)?b[c].promise().then(i(c),g.reject):--e;e||g.resolveWith(g,b)}else g!==a&&g.resolveWith(g,d?[a]:[]);return g.promise()}}),f.support=function(){var a=c.createElement("div"),b=c.documentElement,d,e,g,h,i,j,k,l,m,n,o,p,q,r,s,t,u;a.setAttribute("className","t"),a.innerHTML="   <link/><table></table><a href='/a' style='top:1px;float:left;opacity:.55;'>a</a><input type='checkbox'/>",d=a.getElementsByTagName("*"),e=a.getElementsByTagName("a")[0];if(!d||!d.length||!e)return{};g=c.createElement("select"),h=g.appendChild(c.createElement("option")),i=a.getElementsByTagName("input")[0],k={leadingWhitespace:a.firstChild.nodeType===3,tbody:!a.getElementsByTagName("tbody").length,htmlSerialize:!!a.getElementsByTagName("link").length,style:/top/.test(e.getAttribute("style")),hrefNormalized:e.getAttribute("href")==="/a",opacity:/^0.55$/.test(e.style.opacity),cssFloat:!!e.style.cssFloat,checkOn:i.value==="on",optSelected:h.selected,getSetAttribute:a.className!=="t",submitBubbles:!0,changeBubbles:!0,focusinBubbles:!1,deleteExpando:!0,noCloneEvent:!0,inlineBlockNeedsLayout:!1,shrinkWrapBlocks:!1,reliableMarginRight:!0},i.checked=!0,k.noCloneChecked=i.cloneNode(!0).checked,g.disabled=!0,k.optDisabled=!h.disabled;try{delete a.test}catch(v){k.deleteExpando=!1}!a.addEventListener&&a.attachEvent&&a.fireEvent&&(a.attachEvent("onclick",function(){k.noCloneEvent=!1}),a.cloneNode(!0).fireEvent("onclick")),i=c.createElement("input"),i.value="t",i.setAttribute("type","radio"),k.radioValue=i.value==="t",i.setAttribute("checked","checked"),a.appendChild(i),l=c.createDocumentFragment(),l.appendChild(a.firstChild),k.checkClone=l.cloneNode(!0).cloneNode(!0).lastChild.checked,a.innerHTML="",a.style.width=a.style.paddingLeft="1px",m=c.getElementsByTagName("body")[0],o=c.createElement(m?"div":"body"),p={visibility:"hidden",width:0,height:0,border:0,margin:0},m&&f.extend(p,{position:"absolute",left:-1e3,top:-1e3});for(t in p)o.style[t]=p[t];o.appendChild(a),n=m||b,n.insertBefore(o,n.firstChild),k.appendChecked=i.checked,k.boxModel=a.offsetWidth===2,"zoom"in a.style&&(a.style.display="inline",a.style.zoom=1,k.inlineBlockNeedsLayout=a.offsetWidth===2,a.style.display="",a.innerHTML="<div style='width:4px;'></div>",k.shrinkWrapBlocks=a.offsetWidth!==2),a.innerHTML="<table><tr><td style='padding:0;border:0;display:none'></td><td>t</td></tr></table>",q=a.getElementsByTagName("td"),u=q[0].offsetHeight===0,q[0].style.display="",q[1].style.display="none",k.reliableHiddenOffsets=u&&q[0].offsetHeight===0,a.innerHTML="",c.defaultView&&c.defaultView.getComputedStyle&&(j=c.createElement("div"),j.style.width="0",j.style.marginRight="0",a.appendChild(j),k.reliableMarginRight=(parseInt((c.defaultView.getComputedStyle(j,null)||{marginRight:0}).marginRight,10)||0)===0),o.innerHTML="",n.removeChild(o);if(a.attachEvent)for(t in{submit:1,change:1,focusin:1})s="on"+t,u=s in a,u||(a.setAttribute(s,"return;"),u=typeof a[s]=="function"),k[t+"Bubbles"]=u;o=l=g=h=m=j=a=i=null;return k}(),f.boxModel=f.support.boxModel;var i=/^(?:\{.*\}|\[.*\])$/,j=/([a-z])([A-Z])/g;f.extend({cache:{},uuid:0,expando:"jQuery"+(f.fn.jquery+Math.random()).replace(/\D/g,""),noData:{embed:!0,object:"clsid:D27CDB6E-AE6D-11cf-96B8-444553540000",applet:!0},hasData:function(a){a=a.nodeType?f.cache[a[f.expando]]:a[f.expando];return!!a&&!l(a)},data:function(a,c,d,e){if(!!f.acceptData(a)){var g=f.expando,h=typeof c=="string",i,j=a.nodeType,k=j?f.cache:a,l=j?a[f.expando]:a[f.expando]&&f.expando;if((!l||e&&l&&!k[l][g])&&h&&d===b)return;l||(j?a[f.expando]=l=++f.uuid:l=f.expando),k[l]||(k[l]={},j||(k[l].toJSON=f.noop));if(typeof c=="object"||typeof c=="function")e?k[l][g]=f.extend(k[l][g],c):k[l]=f.extend(k[l],c);i=k[l],e&&(i[g]||(i[g]={}),i=i[g]),d!==b&&(i[f.camelCase(c)]=d);if(c==="events"&&!i[c])return i[g]&&i[g].events;return h?i[f.camelCase(c)]||i[c]:i}},removeData:function(b,c,d){if(!!f.acceptData(b)){var e=f.expando,g=b.nodeType,h=g?f.cache:b,i=g?b[f.expando]:f.expando;if(!h[i])return;if(c){var j=d?h[i][e]:h[i];if(j){delete j[c];if(!l(j))return}}if(d){delete h[i][e];if(!l(h[i]))return}var k=h[i][e];f.support.deleteExpando||h!=a?delete h[i]:h[i]=null,k?(h[i]={},g||(h[i].toJSON=f.noop),h[i][e]=k):g&&(f.support.deleteExpando?delete b[f.expando]:b.removeAttribute?b.removeAttribute(f.expando):b[f.expando]=null)}},_data:function(a,b,c){return f.data(a,b,c,!0)},acceptData:function(a){if(a.nodeName){var b=f.noData[a.nodeName.toLowerCase()];if(b)return b!==!0&&a.getAttribute("classid")===b}return!0}}),f.fn.extend({data:function(a,c){var d=null;if(typeof a=="undefined"){if(this.length){d=f.data(this[0]);if(this[0].nodeType===1){var e=this[0].attributes,g;for(var h=0,i=e.length;h<i;h++)g=e[h].name,g.indexOf("data-")===0&&(g=f.camelCase(g.substring(5)),k(this[0],g,d[g]))}}return d}if(typeof a=="object")return this.each(function(){f.data(this,a)});var j=a.split(".");j[1]=j[1]?"."+j[1]:"";if(c===b){d=this.triggerHandler("getData"+j[1]+"!",[j[0]]),d===b&&this.length&&(d=f.data(this[0],a),d=k(this[0],a,d));return d===b&&j[1]?this.data(j[0]):d}return this.each(function(){var b=f(this),d=[j[0],c];b.triggerHandler("setData"+j[1]+"!",d),f.data(this,a,c),b.triggerHandler("changeData"+j[1]+"!",d)})},removeData:function(a){return this.each(function(){f.removeData(this,a)})}}),f.extend({_mark:function(a,c){a&&(c=(c||"fx")+"mark",f.data(a,c,(f.data(a,c,b,!0)||0)+1,!0))},_unmark:function(a,c,d){a!==!0&&(d=c,c=a,a=!1);if(c){d=d||"fx";var e=d+"mark",g=a?0:(f.data(c,e,b,!0)||1)-1;g?f.data(c,e,g,!0):(f.removeData(c,e,!0),m(c,d,"mark"))}},queue:function(a,c,d){if(a){c=(c||"fx")+"queue";var e=f.data(a,c,b,!0);d&&(!e||f.isArray(d)?e=f.data(a,c,f.makeArray(d),!0):e.push(d));return e||[]}},dequeue:function(a,b){b=b||"fx";var c=f.queue(a,b),d=c.shift(),e;d==="inprogress"&&(d=c.shift()),d&&(b==="fx"&&c.unshift("inprogress"),d.call(a,function(){f.dequeue(a,b)})),c.length||(f.removeData(a,b+"queue",!0),m(a,b,"queue"))}}),f.fn.extend({queue:function(a,c){typeof a!="string"&&(c=a,a="fx");if(c===b)return f.queue(this[0],a);return this.each(function(){var b=f.queue(this,a,c);a==="fx"&&b[0]!=="inprogress"&&f.dequeue(this,a)})},dequeue:function(a){return this.each(function(){f.dequeue(this,a)})},delay:function(a,b){a=f.fx?f.fx.speeds[a]||a:a,b=b||"fx";return this.queue(b,function(){var c=this;setTimeout(function(){f.dequeue(c,b)},a)})},clearQueue:function(a){return this.queue(a||"fx",[])},promise:function(a,c){function m(){--h||d.resolveWith(e,[e])}typeof a!="string"&&(c=a,a=b),a=a||"fx";var d=f.Deferred(),e=this,g=e.length,h=1,i=a+"defer",j=a+"queue",k=a+"mark",l;while(g--)if(l=f.data(e[g],i,b,!0)||(f.data(e[g],j,b,!0)||f.data(e[g],k,b,!0))&&f.data(e[g],i,f._Deferred(),!0))h++,l.done(m);m();return d.promise()}});var n=/[\n\t\r]/g,o=/\s+/,p=/\r/g,q=/^(?:button|input)$/i,r=/^(?:button|input|object|select|textarea)$/i,s=/^a(?:rea)?$/i,t=/^(?:autofocus|autoplay|async|checked|controls|defer|disabled|hidden|loop|multiple|open|readonly|required|scoped|selected)$/i,u=/\:|^on/,v,w;f.fn.extend({attr:function(a,b){return f.access(this,a,b,!0,f.attr)},removeAttr:function(a){return this.each(function(){f.removeAttr(this,a)})},prop:function(a,b){return f.access(this,a,b,!0,f.prop)},removeProp:function(a){a=f.propFix[a]||a;return this.each(function(){try{this[a]=b,delete this[a]}catch(c){}})},addClass:function(a){var b,c,d,e,g,h,i;if(f.isFunction(a))return this.each(function(b){f(this).addClass(a.call(this,b,this.className))});if(a&&typeof a=="string"){b=a.split(o);for(c=0,d=this.length;c<d;c++){e=this[c];if(e.nodeType===1)if(!e.className&&b.length===1)e.className=a;else{g=" "+e.className+" ";for(h=0,i=b.length;h<i;h++)~g.indexOf(" "+b[h]+" ")||(g+=b[h]+" ");e.className=f.trim(g)}}}return this},removeClass:function(a){var c,d,e,g,h,i,j;if(f.isFunction(a))return this.each(function(b){f(this).removeClass(a.call(this,b,this.className))});if(a&&typeof a=="string"||a===b){c=(a||"").split(o);for(d=0,e=this.length;d<e;d++){g=this[d];if(g.nodeType===1&&g.className)if(a){h=(" "+g.className+" ").replace(n," ");for(i=0,j=c.length;i<j;i++)h=h.replace(" "+c[i]+" "," ");g.className=f.trim(h)}else g.className=""}}return this},toggleClass:function(a,b){var c=typeof a,d=typeof b=="boolean";if(f.isFunction(a))return this.each(function(c){f(this).toggleClass(a.call(this,c,this.className,b),b)});return this.each(function(){if(c==="string"){var e,g=0,h=f(this),i=b,j=a.split(o);while(e=j[g++])i=d?i:!h.hasClass(e),h[i?"addClass":"removeClass"](e)}else if(c==="undefined"||c==="boolean")this.className&&f._data(this,"__className__",this.className),this.className=this.className||a===!1?"":f._data(this,"__className__")||""})},hasClass:function(a){var b=" "+a+" ";for(var c=0,d=this.length;c<d;c++)if((" "+this[c].className+" ").replace(n," ").indexOf(b)>-1)return!0;return!1},val:function(a){var c,d,e=this[0];if(!arguments.length){if(e){c=f.valHooks[e.nodeName.toLowerCase()]||f.valHooks[e.type];if(c&&"get"in c&&(d=c.get(e,"value"))!==b)return d;d=e.value;return typeof d=="string"?d.replace(p,""):d==null?"":d}return b}var g=f.isFunction(a);return this.each(function(d){var e=f(this),h;if(this.nodeType===1){g?h=a.call(this,d,e.val()):h=a,h==null?h="":typeof h=="number"?h+="":f.isArray(h)&&(h=f.map(h,function(a){return a==null?"":a+""})),c=f.valHooks[this.nodeName.toLowerCase()]||f.valHooks[this.type];if(!c||!("set"in c)||c.set(this,h,"value")===b)this.value=h}})}}),f.extend({valHooks:{option:{get:function(a){var b=a.attributes.value;return!b||b.specified?a.value:a.text}},select:{get:function(a){var b,c=a.selectedIndex,d=[],e=a.options,g=a.type==="select-one";if(c<0)return null;for(var h=g?c:0,i=g?c+1:e.length;h<i;h++){var j=e[h];if(j.selected&&(f.support.optDisabled?!j.disabled:j.getAttribute("disabled")===null)&&(!j.parentNode.disabled||!f.nodeName(j.parentNode,"optgroup"))){b=f(j).val();if(g)return b;d.push(b)}}if(g&&!d.length&&e.length)return f(e[c]).val();return d},set:function(a,b){var c=f.makeArray(b);f(a).find("option").each(function(){this.selected=f.inArray(f(this).val(),c)>=0}),c.length||(a.selectedIndex=-1);return c}}},attrFn:{val:!0,css:!0,html:!0,text:!0,data:!0,width:!0,height:!0,offset:!0},attrFix:{tabindex:"tabIndex"},attr:function(a,c,d,e){var g=a.nodeType;if(!a||g===3||g===8||g===2)return b;if(e&&c in f.attrFn)return f(a)[c](d);if(!("getAttribute"in a))return f.prop(a,c,d);var h,i,j=g!==1||!f.isXMLDoc(a);j&&(c=f.attrFix[c]||c,i=f.attrHooks[c],i||(t.test(c)?i=w:v&&c!=="className"&&(f.nodeName(a,"form")||u.test(c))&&(i=v)));if(d!==b){if(d===null){f.removeAttr(a,c);return b}if(i&&"set"in i&&j&&(h=i.set(a,d,c))!==b)return h;a.setAttribute(c,""+d);return d}if(i&&"get"in i&&j&&(h=i.get(a,c))!==null)return h;h=a.getAttribute(c);return h===null?b:h},removeAttr:function(a,b){var c;a.nodeType===1&&(b=f.attrFix[b]||b,f.support.getSetAttribute?a.removeAttribute(b):(f.attr(a,b,""),a.removeAttributeNode(a.getAttributeNode(b))),t.test(b)&&(c=f.propFix[b]||b)in a&&(a[c]=!1))},attrHooks:{type:{set:function(a,b){if(q.test(a.nodeName)&&a.parentNode)f.error("type property can't be changed");else if(!f.support.radioValue&&b==="radio"&&f.nodeName(a,"input")){var c=a.value;a.setAttribute("type",b),c&&(a.value=c);return b}}},tabIndex:{get:function(a){var c=a.getAttributeNode("tabIndex");return c&&c.specified?parseInt(c.value,10):r.test(a.nodeName)||s.test(a.nodeName)&&a.href?0:b}},value:{get:function(a,b){if(v&&f.nodeName(a,"button"))return v.get(a,b);return b in a?a.value:null},set:function(a,b,c){if(v&&f.nodeName(a,"button"))return v.set(a,b,c);a.value=b}}},propFix:{tabindex:"tabIndex",readonly:"readOnly","for":"htmlFor","class":"className",maxlength:"maxLength",cellspacing:"cellSpacing",cellpadding:"cellPadding",rowspan:"rowSpan",colspan:"colSpan",usemap:"useMap",frameborder:"frameBorder",contenteditable:"contentEditable"},prop:function(a,c,d){var e=a.nodeType;if(!a||e===3||e===8||e===2)return b;var g,h,i=e!==1||!f.isXMLDoc(a);i&&(c=f.propFix[c]||c,h=f.propHooks[c]);return d!==b?h&&"set"in h&&(g=h.set(a,d,c))!==b?g:a[c]=d:h&&"get"in h&&(g=h.get(a,c))!==b?g:a[c]},propHooks:{}}),w={get:function(a,c){return f.prop(a,c)?c.toLowerCase():b},set:function(a,b,c){var d;b===!1?f.removeAttr(a,c):(d=f.propFix[c]||c,d in a&&(a[d]=!0),a.setAttribute(c,c.toLowerCase()));return c}},f.support.getSetAttribute||(f.attrFix=f.propFix,v=f.attrHooks.name=f.attrHooks.title=f.valHooks.button={get:function(a,c){var d;d=a.getAttributeNode(c);return d&&d.nodeValue!==""?d.nodeValue:b},set:function(a,b,c){var d=a.getAttributeNode(c);if(d){d.nodeValue=b;return b}}},f.each(["width","height"],function(a,b){f.attrHooks[b]=f.extend(f.attrHooks[b],{set:function(a,c){if(c===""){a.setAttribute(b,"auto");return c}}})})),f.support.hrefNormalized||f.each(["href","src","width","height"],function(a,c){f.attrHooks[c]=f.extend(f.attrHooks[c],{get:function(a){var d=a.getAttribute(c,2);return d===null?b:d}})}),f.support.style||(f.attrHooks.style={get:function(a){return a.style.cssText.toLowerCase()||b},set:function(a,b){return a.style.cssText=""+b}}),f.support.optSelected||(f.propHooks.selected=f.extend(f.propHooks.selected,{get:function(a){var b=a.parentNode;b&&(b.selectedIndex,b.parentNode&&b.parentNode.selectedIndex)}})),f.support.checkOn||f.each(["radio","checkbox"],function(){f.valHooks[this]={get:function(a){return a.getAttribute("value")===null?"on":a.value}}}),f.each(["radio","checkbox"],function(){f.valHooks[this]=f.extend(f.valHooks[this],{set:function(a,b){if(f.isArray(b))return a.checked=f.inArray(f(a).val(),b)>=0}})});var x=/\.(.*)$/,y=/^(?:textarea|input|select)$/i,z=/\./g,A=/ /g,B=/[^\w\s.|`]/g,C=function(a){return a.replace(B,"\\$&")};f.event={add:function(a,c,d,e){if(a.nodeType!==3&&a.nodeType!==8){if(d===!1)d=D;else if(!d)return;var g,h;d.handler&&(g=d,d=g.handler),d.guid||(d.guid=f.guid++);var i=f._data(a);if(!i)return;var j=i.events,k=i.handle;j||(i.events=j={}),k||(i.handle=k=function(a){return typeof f!="undefined"&&(!a||f.event.triggered!==a.type)?f.event.handle.apply(k.elem,arguments):b}),k.elem=a,c=c.split(" ");var l,m=0,n;while(l=c[m++]){h=g?f.extend({},g):{handler:d,data:e},l.indexOf(".")>-1?(n=l.split("."),l=n.shift(),h.namespace=n.slice(0).sort().join(".")):(n=[],h.namespace=""),h.type=l,h.guid||(h.guid=d.guid);var o=j[l],p=f.event.special[l]||{};if(!o){o=j[l]=[];if(!p.setup||p.setup.call(a,e,n,k)===!1)a.addEventListener?a.addEventListener(l,k,!1):a.attachEvent&&a.attachEvent("on"+l,k)}p.add&&(p.add.call(a,h),h.handler.guid||(h.handler.guid=d.guid)),o.push(h),f.event.global[l]=!0}a=null}},global:{},remove:function(a,c,d,e){if(a.nodeType!==3&&a.nodeType!==8){d===!1&&(d=D);var g,h,i,j,k=0,l,m,n,o,p,q,r,s=f.hasData(a)&&f._data(a),t=s&&s.events;if(!s||!t)return;c&&c.type&&(d=c.handler,c=c.type);if(!c||typeof c=="string"&&c.charAt(0)==="."){c=c||"";for(h in t)f.event.remove(a,h+c);return}c=c.split(" ");while(h=c[k++]){r=h,q=null,l=h.indexOf(".")<0,m=[],l||(m=h.split("."),h=m.shift(),n=new RegExp("(^|\\.)"+f.map(m.slice(0).sort(),C).join("\\.(?:.*\\.)?")+"(\\.|$)")),p=t[h];if(!p)continue;if(!d){for(j=0;j<p.length;j++){q=p[j];if(l||n.test(q.namespace))f.event.remove(a,r,q.handler,j),p.splice(j--,1)}continue}o=f.event.special[h]||{};for(j=e||0;j<p.length;j++){q=p[j];if(d.guid===q.guid){if(l||n.test(q.namespace))e==null&&p.splice(j--,1),o.remove&&o.remove.call(a,q);if(e!=null)break}}if(p.length===0||e!=null&&p.length===1)(!o.teardown||o.teardown.call(a,m)===!1)&&f.removeEvent(a,h,s.handle),g=null,delete t[h]}if(f.isEmptyObject(t)){var u=s.handle;u&&(u.elem=null),delete s.events,delete s.handle,f.isEmptyObject(s)&&f.removeData(a,b,!0)}}},customEvent:{getData:!0,setData:!0,changeData:!0},trigger:function(c,d,e,g){var h=c.type||c,i=[],j;h.indexOf("!")>=0&&(h=h.slice(0,-1),j=!0),h.indexOf(".")>=0&&(i=h.split("."),h=i.
+shift(),i.sort());if(!!e&&!f.event.customEvent[h]||!!f.event.global[h]){c=typeof c=="object"?c[f.expando]?c:new f.Event(h,c):new f.Event(h),c.type=h,c.exclusive=j,c.namespace=i.join("."),c.namespace_re=new RegExp("(^|\\.)"+i.join("\\.(?:.*\\.)?")+"(\\.|$)");if(g||!e)c.preventDefault(),c.stopPropagation();if(!e){f.each(f.cache,function(){var a=f.expando,b=this[a];b&&b.events&&b.events[h]&&f.event.trigger(c,d,b.handle.elem)});return}if(e.nodeType===3||e.nodeType===8)return;c.result=b,c.target=e,d=d!=null?f.makeArray(d):[],d.unshift(c);var k=e,l=h.indexOf(":")<0?"on"+h:"";do{var m=f._data(k,"handle");c.currentTarget=k,m&&m.apply(k,d),l&&f.acceptData(k)&&k[l]&&k[l].apply(k,d)===!1&&(c.result=!1,c.preventDefault()),k=k.parentNode||k.ownerDocument||k===c.target.ownerDocument&&a}while(k&&!c.isPropagationStopped());if(!c.isDefaultPrevented()){var n,o=f.event.special[h]||{};if((!o._default||o._default.call(e.ownerDocument,c)===!1)&&(h!=="click"||!f.nodeName(e,"a"))&&f.acceptData(e)){try{l&&e[h]&&(n=e[l],n&&(e[l]=null),f.event.triggered=h,e[h]())}catch(p){}n&&(e[l]=n),f.event.triggered=b}}return c.result}},handle:function(c){c=f.event.fix(c||a.event);var d=((f._data(this,"events")||{})[c.type]||[]).slice(0),e=!c.exclusive&&!c.namespace,g=Array.prototype.slice.call(arguments,0);g[0]=c,c.currentTarget=this;for(var h=0,i=d.length;h<i;h++){var j=d[h];if(e||c.namespace_re.test(j.namespace)){c.handler=j.handler,c.data=j.data,c.handleObj=j;var k=j.handler.apply(this,g);k!==b&&(c.result=k,k===!1&&(c.preventDefault(),c.stopPropagation()));if(c.isImmediatePropagationStopped())break}}return c.result},props:"altKey attrChange attrName bubbles button cancelable charCode clientX clientY ctrlKey currentTarget data detail eventPhase fromElement handler keyCode layerX layerY metaKey newValue offsetX offsetY pageX pageY prevValue relatedNode relatedTarget screenX screenY shiftKey srcElement target toElement view wheelDelta which".split(" "),fix:function(a){if(a[f.expando])return a;var d=a;a=f.Event(d);for(var e=this.props.length,g;e;)g=this.props[--e],a[g]=d[g];a.target||(a.target=a.srcElement||c),a.target.nodeType===3&&(a.target=a.target.parentNode),!a.relatedTarget&&a.fromElement&&(a.relatedTarget=a.fromElement===a.target?a.toElement:a.fromElement);if(a.pageX==null&&a.clientX!=null){var h=a.target.ownerDocument||c,i=h.documentElement,j=h.body;a.pageX=a.clientX+(i&&i.scrollLeft||j&&j.scrollLeft||0)-(i&&i.clientLeft||j&&j.clientLeft||0),a.pageY=a.clientY+(i&&i.scrollTop||j&&j.scrollTop||0)-(i&&i.clientTop||j&&j.clientTop||0)}a.which==null&&(a.charCode!=null||a.keyCode!=null)&&(a.which=a.charCode!=null?a.charCode:a.keyCode),!a.metaKey&&a.ctrlKey&&(a.metaKey=a.ctrlKey),!a.which&&a.button!==b&&(a.which=a.button&1?1:a.button&2?3:a.button&4?2:0);return a},guid:1e8,proxy:f.proxy,special:{ready:{setup:f.bindReady,teardown:f.noop},live:{add:function(a){f.event.add(this,N(a.origType,a.selector),f.extend({},a,{handler:M,guid:a.handler.guid}))},remove:function(a){f.event.remove(this,N(a.origType,a.selector),a)}},beforeunload:{setup:function(a,b,c){f.isWindow(this)&&(this.onbeforeunload=c)},teardown:function(a,b){this.onbeforeunload===b&&(this.onbeforeunload=null)}}}},f.removeEvent=c.removeEventListener?function(a,b,c){a.removeEventListener&&a.removeEventListener(b,c,!1)}:function(a,b,c){a.detachEvent&&a.detachEvent("on"+b,c)},f.Event=function(a,b){if(!this.preventDefault)return new f.Event(a,b);a&&a.type?(this.originalEvent=a,this.type=a.type,this.isDefaultPrevented=a.defaultPrevented||a.returnValue===!1||a.getPreventDefault&&a.getPreventDefault()?E:D):this.type=a,b&&f.extend(this,b),this.timeStamp=f.now(),this[f.expando]=!0},f.Event.prototype={preventDefault:function(){this.isDefaultPrevented=E;var a=this.originalEvent;!a||(a.preventDefault?a.preventDefault():a.returnValue=!1)},stopPropagation:function(){this.isPropagationStopped=E;var a=this.originalEvent;!a||(a.stopPropagation&&a.stopPropagation(),a.cancelBubble=!0)},stopImmediatePropagation:function(){this.isImmediatePropagationStopped=E,this.stopPropagation()},isDefaultPrevented:D,isPropagationStopped:D,isImmediatePropagationStopped:D};var F=function(a){var b=a.relatedTarget,c=!1,d=a.type;a.type=a.data,b!==this&&(b&&(c=f.contains(this,b)),c||(f.event.handle.apply(this,arguments),a.type=d))},G=function(a){a.type=a.data,f.event.handle.apply(this,arguments)};f.each({mouseenter:"mouseover",mouseleave:"mouseout"},function(a,b){f.event.special[a]={setup:function(c){f.event.add(this,b,c&&c.selector?G:F,a)},teardown:function(a){f.event.remove(this,b,a&&a.selector?G:F)}}}),f.support.submitBubbles||(f.event.special.submit={setup:function(a,b){if(!f.nodeName(this,"form"))f.event.add(this,"click.specialSubmit",function(a){var b=a.target,c=b.type;(c==="submit"||c==="image")&&f(b).closest("form").length&&K("submit",this,arguments)}),f.event.add(this,"keypress.specialSubmit",function(a){var b=a.target,c=b.type;(c==="text"||c==="password")&&f(b).closest("form").length&&a.keyCode===13&&K("submit",this,arguments)});else return!1},teardown:function(a){f.event.remove(this,".specialSubmit")}});if(!f.support.changeBubbles){var H,I=function(a){var b=a.type,c=a.value;b==="radio"||b==="checkbox"?c=a.checked:b==="select-multiple"?c=a.selectedIndex>-1?f.map(a.options,function(a){return a.selected}).join("-"):"":f.nodeName(a,"select")&&(c=a.selectedIndex);return c},J=function(c){var d=c.target,e,g;if(!!y.test(d.nodeName)&&!d.readOnly){e=f._data(d,"_change_data"),g=I(d),(c.type!=="focusout"||d.type!=="radio")&&f._data(d,"_change_data",g);if(e===b||g===e)return;if(e!=null||g)c.type="change",c.liveFired=b,f.event.trigger(c,arguments[1],d)}};f.event.special.change={filters:{focusout:J,beforedeactivate:J,click:function(a){var b=a.target,c=f.nodeName(b,"input")?b.type:"";(c==="radio"||c==="checkbox"||f.nodeName(b,"select"))&&J.call(this,a)},keydown:function(a){var b=a.target,c=f.nodeName(b,"input")?b.type:"";(a.keyCode===13&&!f.nodeName(b,"textarea")||a.keyCode===32&&(c==="checkbox"||c==="radio")||c==="select-multiple")&&J.call(this,a)},beforeactivate:function(a){var b=a.target;f._data(b,"_change_data",I(b))}},setup:function(a,b){if(this.type==="file")return!1;for(var c in H)f.event.add(this,c+".specialChange",H[c]);return y.test(this.nodeName)},teardown:function(a){f.event.remove(this,".specialChange");return y.test(this.nodeName)}},H=f.event.special.change.filters,H.focus=H.beforeactivate}f.support.focusinBubbles||f.each({focus:"focusin",blur:"focusout"},function(a,b){function e(a){var c=f.event.fix(a);c.type=b,c.originalEvent={},f.event.trigger(c,null,c.target),c.isDefaultPrevented()&&a.preventDefault()}var d=0;f.event.special[b]={setup:function(){d++===0&&c.addEventListener(a,e,!0)},teardown:function(){--d===0&&c.removeEventListener(a,e,!0)}}}),f.each(["bind","one"],function(a,c){f.fn[c]=function(a,d,e){var g;if(typeof a=="object"){for(var h in a)this[c](h,d,a[h],e);return this}if(arguments.length===2||d===!1)e=d,d=b;c==="one"?(g=function(a){f(this).unbind(a,g);return e.apply(this,arguments)},g.guid=e.guid||f.guid++):g=e;if(a==="unload"&&c!=="one")this.one(a,d,e);else for(var i=0,j=this.length;i<j;i++)f.event.add(this[i],a,g,d);return this}}),f.fn.extend({unbind:function(a,b){if(typeof a=="object"&&!a.preventDefault)for(var c in a)this.unbind(c,a[c]);else for(var d=0,e=this.length;d<e;d++)f.event.remove(this[d],a,b);return this},delegate:function(a,b,c,d){return this.live(b,c,d,a)},undelegate:function(a,b,c){return arguments.length===0?this.unbind("live"):this.die(b,null,c,a)},trigger:function(a,b){return this.each(function(){f.event.trigger(a,b,this)})},triggerHandler:function(a,b){if(this[0])return f.event.trigger(a,b,this[0],!0)},toggle:function(a){var b=arguments,c=a.guid||f.guid++,d=0,e=function(c){var e=(f.data(this,"lastToggle"+a.guid)||0)%d;f.data(this,"lastToggle"+a.guid,e+1),c.preventDefault();return b[e].apply(this,arguments)||!1};e.guid=c;while(d<b.length)b[d++].guid=c;return this.click(e)},hover:function(a,b){return this.mouseenter(a).mouseleave(b||a)}});var L={focus:"focusin",blur:"focusout",mouseenter:"mouseover",mouseleave:"mouseout"};f.each(["live","die"],function(a,c){f.fn[c]=function(a,d,e,g){var h,i=0,j,k,l,m=g||this.selector,n=g?this:f(this.context);if(typeof a=="object"&&!a.preventDefault){for(var o in a)n[c](o,d,a[o],m);return this}if(c==="die"&&!a&&g&&g.charAt(0)==="."){n.unbind(g);return this}if(d===!1||f.isFunction(d))e=d||D,d=b;a=(a||"").split(" ");while((h=a[i++])!=null){j=x.exec(h),k="",j&&(k=j[0],h=h.replace(x,""));if(h==="hover"){a.push("mouseenter"+k,"mouseleave"+k);continue}l=h,L[h]?(a.push(L[h]+k),h=h+k):h=(L[h]||h)+k;if(c==="live")for(var p=0,q=n.length;p<q;p++)f.event.add(n[p],"live."+N(h,m),{data:d,selector:m,handler:e,origType:h,origHandler:e,preType:l});else n.unbind("live."+N(h,m),e)}return this}}),f.each("blur focus focusin focusout load resize scroll unload click dblclick mousedown mouseup mousemove mouseover mouseout mouseenter mouseleave change select submit keydown keypress keyup error".split(" "),function(a,b){f.fn[b]=function(a,c){c==null&&(c=a,a=null);return arguments.length>0?this.bind(b,a,c):this.trigger(b)},f.attrFn&&(f.attrFn[b]=!0)}),function(){function u(a,b,c,d,e,f){for(var g=0,h=d.length;g<h;g++){var i=d[g];if(i){var j=!1;i=i[a];while(i){if(i.sizcache===c){j=d[i.sizset];break}if(i.nodeType===1){f||(i.sizcache=c,i.sizset=g);if(typeof b!="string"){if(i===b){j=!0;break}}else if(k.filter(b,[i]).length>0){j=i;break}}i=i[a]}d[g]=j}}}function t(a,b,c,d,e,f){for(var g=0,h=d.length;g<h;g++){var i=d[g];if(i){var j=!1;i=i[a];while(i){if(i.sizcache===c){j=d[i.sizset];break}i.nodeType===1&&!f&&(i.sizcache=c,i.sizset=g);if(i.nodeName.toLowerCase()===b){j=i;break}i=i[a]}d[g]=j}}}var a=/((?:\((?:\([^()]+\)|[^()]+)+\)|\[(?:\[[^\[\]]*\]|['"][^'"]*['"]|[^\[\]'"]+)+\]|\\.|[^ >+~,(\[\\]+)+|[>+~])(\s*,\s*)?((?:.|\r|\n)*)/g,d=0,e=Object.prototype.toString,g=!1,h=!0,i=/\\/g,j=/\W/;[0,0].sort(function(){h=!1;return 0});var k=function(b,d,f,g){f=f||[],d=d||c;var h=d;if(d.nodeType!==1&&d.nodeType!==9)return[];if(!b||typeof b!="string")return f;var i,j,n,o,q,r,s,t,u=!0,w=k.isXML(d),x=[],y=b;do{a.exec(""),i=a.exec(y);if(i){y=i[3],x.push(i[1]);if(i[2]){o=i[3];break}}}while(i);if(x.length>1&&m.exec(b))if(x.length===2&&l.relative[x[0]])j=v(x[0]+x[1],d);else{j=l.relative[x[0]]?[d]:k(x.shift(),d);while(x.length)b=x.shift(),l.relative[b]&&(b+=x.shift()),j=v(b,j)}else{!g&&x.length>1&&d.nodeType===9&&!w&&l.match.ID.test(x[0])&&!l.match.ID.test(x[x.length-1])&&(q=k.find(x.shift(),d,w),d=q.expr?k.filter(q.expr,q.set)[0]:q.set[0]);if(d){q=g?{expr:x.pop(),set:p(g)}:k.find(x.pop(),x.length===1&&(x[0]==="~"||x[0]==="+")&&d.parentNode?d.parentNode:d,w),j=q.expr?k.filter(q.expr,q.set):q.set,x.length>0?n=p(j):u=!1;while(x.length)r=x.pop(),s=r,l.relative[r]?s=x.pop():r="",s==null&&(s=d),l.relative[r](n,s,w)}else n=x=[]}n||(n=j),n||k.error(r||b);if(e.call(n)==="[object Array]")if(!u)f.push.apply(f,n);else if(d&&d.nodeType===1)for(t=0;n[t]!=null;t++)n[t]&&(n[t]===!0||n[t].nodeType===1&&k.contains(d,n[t]))&&f.push(j[t]);else for(t=0;n[t]!=null;t++)n[t]&&n[t].nodeType===1&&f.push(j[t]);else p(n,f);o&&(k(o,h,f,g),k.uniqueSort(f));return f};k.uniqueSort=function(a){if(r){g=h,a.sort(r);if(g)for(var b=1;b<a.length;b++)a[b]===a[b-1]&&a.splice(b--,1)}return a},k.matches=function(a,b){return k(a,null,null,b)},k.matchesSelector=function(a,b){return k(b,null,null,[a]).length>0},k.find=function(a,b,c){var d;if(!a)return[];for(var e=0,f=l.order.length;e<f;e++){var g,h=l.order[e];if(g=l.leftMatch[h].exec(a)){var j=g[1];g.splice(1,1);if(j.substr(j.length-1)!=="\\"){g[1]=(g[1]||"").replace(i,""),d=l.find[h](g,b,c);if(d!=null){a=a.replace(l.match[h],"");break}}}}d||(d=typeof b.getElementsByTagName!="undefined"?b.getElementsByTagName("*"):[]);return{set:d,expr:a}},k.filter=function(a,c,d,e){var f,g,h=a,i=[],j=c,m=c&&c[0]&&k.isXML(c[0]);while(a&&c.length){for(var n in l.filter)if((f=l.leftMatch[n].exec(a))!=null&&f[2]){var o,p,q=l.filter[n],r=f[1];g=!1,f.splice(1,1);if(r.substr(r.length-1)==="\\")continue;j===i&&(i=[]);if(l.preFilter[n]){f=l.preFilter[n](f,j,d,i,e,m);if(!f)g=o=!0;else if(f===!0)continue}if(f)for(var s=0;(p=j[s])!=null;s++)if(p){o=q(p,f,s,j);var t=e^!!o;d&&o!=null?t?g=!0:j[s]=!1:t&&(i.push(p),g=!0)}if(o!==b){d||(j=i),a=a.replace(l.match[n],"");if(!g)return[];break}}if(a===h)if(g==null)k.error(a);else break;h=a}return j},k.error=function(a){throw"Syntax error, unrecognized expression: "+a};var l=k.selectors={order:["ID","NAME","TAG"],match:{ID:/#((?:[\w\u00c0-\uFFFF\-]|\\.)+)/,CLASS:/\.((?:[\w\u00c0-\uFFFF\-]|\\.)+)/,NAME:/\[name=['"]*((?:[\w\u00c0-\uFFFF\-]|\\.)+)['"]*\]/,ATTR:/\[\s*((?:[\w\u00c0-\uFFFF\-]|\\.)+)\s*(?:(\S?=)\s*(?:(['"])(.*?)\3|(#?(?:[\w\u00c0-\uFFFF\-]|\\.)*)|)|)\s*\]/,TAG:/^((?:[\w\u00c0-\uFFFF\*\-]|\\.)+)/,CHILD:/:(only|nth|last|first)-child(?:\(\s*(even|odd|(?:[+\-]?\d+|(?:[+\-]?\d*)?n\s*(?:[+\-]\s*\d+)?))\s*\))?/,POS:/:(nth|eq|gt|lt|first|last|even|odd)(?:\((\d*)\))?(?=[^\-]|$)/,PSEUDO:/:((?:[\w\u00c0-\uFFFF\-]|\\.)+)(?:\((['"]?)((?:\([^\)]+\)|[^\(\)]*)+)\2\))?/},leftMatch:{},attrMap:{"class":"className","for":"htmlFor"},attrHandle:{href:function(a){return a.getAttribute("href")},type:function(a){return a.getAttribute("type")}},relative:{"+":function(a,b){var c=typeof b=="string",d=c&&!j.test(b),e=c&&!d;d&&(b=b.toLowerCase());for(var f=0,g=a.length,h;f<g;f++)if(h=a[f]){while((h=h.previousSibling)&&h.nodeType!==1);a[f]=e||h&&h.nodeName.toLowerCase()===b?h||!1:h===b}e&&k.filter(b,a,!0)},">":function(a,b){var c,d=typeof b=="string",e=0,f=a.length;if(d&&!j.test(b)){b=b.toLowerCase();for(;e<f;e++){c=a[e];if(c){var g=c.parentNode;a[e]=g.nodeName.toLowerCase()===b?g:!1}}}else{for(;e<f;e++)c=a[e],c&&(a[e]=d?c.parentNode:c.parentNode===b);d&&k.filter(b,a,!0)}},"":function(a,b,c){var e,f=d++,g=u;typeof b=="string"&&!j.test(b)&&(b=b.toLowerCase(),e=b,g=t),g("parentNode",b,f,a,e,c)},"~":function(a,b,c){var e,f=d++,g=u;typeof b=="string"&&!j.test(b)&&(b=b.toLowerCase(),e=b,g=t),g("previousSibling",b,f,a,e,c)}},find:{ID:function(a,b,c){if(typeof b.getElementById!="undefined"&&!c){var d=b.getElementById(a[1]);return d&&d.parentNode?[d]:[]}},NAME:function(a,b){if(typeof b.getElementsByName!="undefined"){var c=[],d=b.getElementsByName(a[1]);for(var e=0,f=d.length;e<f;e++)d[e].getAttribute("name")===a[1]&&c.push(d[e]);return c.length===0?null:c}},TAG:function(a,b){if(typeof b.getElementsByTagName!="undefined")return b.getElementsByTagName(a[1])}},preFilter:{CLASS:function(a,b,c,d,e,f){a=" "+a[1].replace(i,"")+" ";if(f)return a;for(var g=0,h;(h=b[g])!=null;g++)h&&(e^(h.className&&(" "+h.className+" ").replace(/[\t\n\r]/g," ").indexOf(a)>=0)?c||d.push(h):c&&(b[g]=!1));return!1},ID:function(a){return a[1].replace(i,"")},TAG:function(a,b){return a[1].replace(i,"").toLowerCase()},CHILD:function(a){if(a[1]==="nth"){a[2]||k.error(a[0]),a[2]=a[2].replace(/^\+|\s*/g,"");var b=/(-?)(\d*)(?:n([+\-]?\d*))?/.exec(a[2]==="even"&&"2n"||a[2]==="odd"&&"2n+1"||!/\D/.test(a[2])&&"0n+"+a[2]||a[2]);a[2]=b[1]+(b[2]||1)-0,a[3]=b[3]-0}else a[2]&&k.error(a[0]);a[0]=d++;return a},ATTR:function(a,b,c,d,e,f){var g=a[1]=a[1].replace(i,"");!f&&l.attrMap[g]&&(a[1]=l.attrMap[g]),a[4]=(a[4]||a[5]||"").replace(i,""),a[2]==="~="&&(a[4]=" "+a[4]+" ");return a},PSEUDO:function(b,c,d,e,f){if(b[1]==="not")if((a.exec(b[3])||"").length>1||/^\w/.test(b[3]))b[3]=k(b[3],null,null,c);else{var g=k.filter(b[3],c,d,!0^f);d||e.push.apply(e,g);return!1}else if(l.match.POS.test(b[0])||l.match.CHILD.test(b[0]))return!0;return b},POS:function(a){a.unshift(!0);return a}},filters:{enabled:function(a){return a.disabled===!1&&a.type!=="hidden"},disabled:function(a){return a.disabled===!0},checked:function(a){return a.checked===!0},selected:function(a){a.parentNode&&a.parentNode.selectedIndex;return a.selected===!0},parent:function(a){return!!a.firstChild},empty:function(a){return!a.firstChild},has:function(a,b,c){return!!k(c[3],a).length},header:function(a){return/h\d/i.test(a.nodeName)},text:function(a){var b=a.getAttribute("type"),c=a.type;return a.nodeName.toLowerCase()==="input"&&"text"===c&&(b===c||b===null)},radio:function(a){return a.nodeName.toLowerCase()==="input"&&"radio"===a.type},checkbox:function(a){return a.nodeName.toLowerCase()==="input"&&"checkbox"===a.type},file:function(a){return a.nodeName.toLowerCase()==="input"&&"file"===a.type},password:function(a){return a.nodeName.toLowerCase()==="input"&&"password"===a.type},submit:function(a){var b=a.nodeName.toLowerCase();return(b==="input"||b==="button")&&"submit"===a.type},image:function(a){return a.nodeName.toLowerCase()==="input"&&"image"===a.type},reset:function(a){var b=a.nodeName.toLowerCase();return(b==="input"||b==="button")&&"reset"===a.type},button:function(a){var b=a.nodeName.toLowerCase();return b==="input"&&"button"===a.type||b==="button"},input:function(a){return/input|select|textarea|button/i.test(a.nodeName)},focus:function(a){return a===a.ownerDocument.activeElement}},setFilters:{first:function(a,b){return b===0},last:function(a,b,c,d){return b===d.length-1},even:function(a,b){return b%2===0},odd:function(a,b){return b%2===1},lt:function(a,b,c){return b<c[3]-0},gt:function(a,b,c){return b>c[3]-0},nth:function(a,b,c){return c[3]-0===b},eq:function(a,b,c){return c[3]-0===b}},filter:{PSEUDO:function(a,b,c,d){var e=b[1],f=l.filters[e];if(f)return f(a,c,b,d);if(e==="contains")return(a.textContent||a.innerText||k.getText([a])||"").indexOf(b[3])>=0;if(e==="not"){var g=b[3];for(var h=0,i=g.length;h<i;h++)if(g[h]===a)return!1;return!0}k.error(e)},CHILD:function(a,b){var c=b[1],d=a;switch(c){case"only":case"first":while(d=d.previousSibling)if(d.nodeType===1)return!1;if(c==="first")return!0;d=a;case"last":while(d=d.nextSibling)if(d.nodeType===1)return!1;return!0;case"nth":var e=b[2],f=b[3];if(e===1&&f===0)return!0;var g=b[0],h=a.parentNode;if(h&&(h.sizcache!==g||!a.nodeIndex)){var i=0;for(d=h.firstChild;d;d=d.nextSibling)d.nodeType===1&&(d.nodeIndex=++i);h.sizcache=g}var j=a.nodeIndex-f;return e===0?j===0:j%e===0&&j/e>=0}},ID:function(a,b){return a.nodeType===1&&a.getAttribute("id")===b},TAG:function(a,b){return b==="*"&&a.nodeType===1||a.nodeName.toLowerCase()===b},CLASS:function(a,b){return(" "+(a.className||a.getAttribute("class"))+" ").indexOf(b)>-1},ATTR:function(a,b){var c=b[1],d=l.attrHandle[c]?l.attrHandle[c](a):a[c]!=null?a[c]:a.getAttribute(c),e=d+"",f=b[2],g=b[4];return d==null?f==="!=":f==="="?e===g:f==="*="?e.indexOf(g)>=0:f==="~="?(" "+e+" ").indexOf(g)>=0:g?f==="!="?e!==g:f==="^="?e.indexOf(g)===0:f==="$="?e.substr(e.length-g.length)===g:f==="|="?e===g||e.substr(0,g.length+1)===g+"-":!1:e&&d!==!1},POS:function(a,b,c,d){var e=b[2],f=l.setFilters[e];if(f)return f(a,c,b,d)}}},m=l.match.POS,n=function(a,b){return"\\"+(b-0+1)};for(var o in l.match)l.match[o]=new RegExp(l.match[o].source+/(?![^\[]*\])(?![^\(]*\))/.source),l.leftMatch[o]=new RegExp(/(^(?:.|\r|\n)*?)/.source+l.match[o].source.replace(/\\(\d+)/g,n));var p=function(a,b){a=Array.prototype.slice.call(a,0);if(b){b.push.apply(b,a);return b}return a};try{Array.prototype.slice.call(c.documentElement.childNodes,0)[0].nodeType}catch(q){p=function(a,b){var c=0,d=b||[];if(e.call(a)==="[object Array]")Array.prototype.push.apply(d,a);else if(typeof a.length=="number")for(var f=a.length;c<f;c++)d.push(a[c]);else for(;a[c];c++)d.push(a[c]);return d}}var r,s;c.documentElement.compareDocumentPosition?r=function(a,b){if(a===b){g=!0;return 0}if(!a.compareDocumentPosition||!b.compareDocumentPosition)return a.compareDocumentPosition?-1:1;return a.compareDocumentPosition(b)&4?-1:1}:(r=function(a,b){if(a===b){g=!0;return 0}if(a.sourceIndex&&b.sourceIndex)return a.sourceIndex-b.sourceIndex;var c,d,e=[],f=[],h=a.parentNode,i=b.parentNode,j=h;if(h===i)return s(a,b);if(!h)return-1;if(!i)return 1;while(j)e.unshift(j),j=j.parentNode;j=i;while(j)f.unshift(j),j=j.parentNode;c=e.length,d=f.length;for(var k=0;k<c&&k<d;k++)if(e[k]!==f[k])return s(e[k],f[k]);return k===c?s(a,f[k],-1):s(e[k],b,1)},s=function(a,b,c){if(a===b)return c;var d=a.nextSibling;while(d){if(d===b)return-1;d=d.nextSibling}return 1}),k.getText=function(a){var b="",c;for(var d=0;a[d];d++)c=a[d],c.nodeType===3||c.nodeType===4?b+=c.nodeValue:c.nodeType!==8&&(b+=k.getText(c.childNodes));return b},function(){var a=c.createElement("div"),d="script"+(new Date).getTime(),e=c.documentElement;a.innerHTML="<a name='"+d+"'/>",e.insertBefore(a,e.firstChild),c.getElementById(d)&&(l.find.ID=function(a,c,d){if(typeof c.getElementById!="undefined"&&!d){var e=c.getElementById(a[1]);return e?e.id===a[1]||typeof e.getAttributeNode!="undefined"&&e.getAttributeNode("id").nodeValue===a[1]?[e]:b:[]}},l.filter.ID=function(a,b){var c=typeof a.getAttributeNode!="undefined"&&a.getAttributeNode("id");return a.nodeType===1&&c&&c.nodeValue===b}),e.removeChild(a),e=a=null}(),function(){var a=c.createElement("div");a.appendChild(c.createComment("")),a.getElementsByTagName("*").length>0&&(l.find.TAG=function(a,b){var c=b.getElementsByTagName(a[1]);if(a[1]==="*"){var d=[];for(var e=0;c[e];e++)c[e].nodeType===1&&d.push(c[e]);c=d}return c}),a.innerHTML="<a href='#'></a>",a.firstChild&&typeof a.firstChild.getAttribute!="undefined"&&a.firstChild.getAttribute("href")!=="#"&&(l.attrHandle.href=function(a){return a.getAttribute("href",2)}),a=null}(),c.querySelectorAll&&function(){var a=k,b=c.createElement("div"),d="__sizzle__";b.innerHTML="<p class='TEST'></p>";if(!b.querySelectorAll||b.querySelectorAll(".TEST").length!==0){k=function(b,e,f,g){e=e||c;if(!g&&!k.isXML(e)){var h=/^(\w+$)|^\.([\w\-]+$)|^#([\w\-]+$)/.exec(b);if(h&&(e.nodeType===1||e.nodeType===9)){if(h[1])return p(e.getElementsByTagName(b),f);if(h[2]&&l.find.CLASS&&e.getElementsByClassName)return p(e.getElementsByClassName(h[2]),f)}if(e.nodeType===9){if(b==="body"&&e.body)return p([e.body],f);if(h&&h[3]){var i=e.getElementById(h[3]);if(!i||!i.parentNode)return p([],f);if(i.id===h[3])return p([i],f)}try{return p(e.querySelectorAll(b),f)}catch(j){}}else if(e.nodeType===1&&e.nodeName.toLowerCase()!=="object"){var m=e,n=e.getAttribute("id"),o=n||d,q=e.parentNode,r=/^\s*[+~]/.test(b);n?o=o.replace(/'/g,"\\$&"):e.setAttribute("id",o),r&&q&&(e=e.parentNode);try{if(!r||q)return p(e.querySelectorAll("[id='"+o+"'] "+b),f)}catch(s){}finally{n||m.removeAttribute("id")}}}return a(b,e,f,g)};for(var e in a)k[e]=a[e];b=null}}(),function(){var a=c.documentElement,b=a.matchesSelector||a.mozMatchesSelector||a.webkitMatchesSelector||a.msMatchesSelector;if(b){var d=!b.call(c.createElement("div"),"div"),e=!1;try{b.call(c.documentElement,"[test!='']:sizzle")}catch(f){e=!0}k.matchesSelector=function(a,c){c=c.replace(/\=\s*([^'"\]]*)\s*\]/g,"='$1']");if(!k.isXML(a))try{if(e||!l.match.PSEUDO.test(c)&&!/!=/.test(c)){var f=b.call(a,c);if(f||!d||a.document&&a.document.nodeType!==11)return f}}catch(g){}return k(c,null,null,[a]).length>0}}}(),function(){var a=c.createElement("div");a.innerHTML="<div class='test e'></div><div class='test'></div>";if(!!a.getElementsByClassName&&a.getElementsByClassName("e").length!==0){a.lastChild.className="e";if(a.getElementsByClassName("e").length===1)return;l.order.splice(1,0,"CLASS"),l.find.CLASS=function(a,b,c){if(typeof b.getElementsByClassName!="undefined"&&!c)return b.getElementsByClassName(a[1])},a=null}}(),c.documentElement.contains?k.contains=function(a,b){return a!==b&&(a.contains?a.contains(b):!0)}:c.documentElement.compareDocumentPosition?k.contains=function(a,b){return!!(a.compareDocumentPosition(b)&16)}:k.contains=function(){return!1},k.isXML=function(a){var b=(a?a.ownerDocument||a:0).documentElement;return b?b.nodeName!=="HTML":!1};var v=function(a,b){var c,d=[],e="",f=b.nodeType?[b]:b;while(c=l.match.PSEUDO.exec(a))e+=c[0],a=a.replace(l.match.PSEUDO,"");a=l.relative[a]?a+"*":a;for(var g=0,h=f.length;g<h;g++)k(a,f[g],d);return k.filter(e,d)};f.find=k,f.expr=k.selectors,f.expr[":"]=f.expr.filters,f.unique=k.uniqueSort,f.text=k.getText,f.isXMLDoc=k.isXML,f.contains=k.contains}();var O=/Until$/,P=/^(?:parents|prevUntil|prevAll)/,Q=/,/,R=/^.[^:#\[\.,]*$/,S=Array.prototype.slice,T=f.expr.match.POS,U={children:!0,contents:!0,next:!0,prev:!0};f.fn.extend({find:function(a){var b=this,c,d;if(typeof a!="string")return f(a).filter(function(){for(c=0,d=b.length;c<d;c++)if(f.contains(b[c],this))return!0});var e=this.pushStack("","find",a),g,h,i;for(c=0,d=this.length;c<d;c++){g=e.length,f.find(a,this[c],e);if(c>0)for(h=g;h<e.length;h++)for(i=0;i<g;i++)if(e[i]===e[h]){e.splice(h--,1);break}}return e},has:function(a){var b=f(a);return this.filter(function(){for(var a=0,c=b.length;a<c;a++)if(f.contains(this,b[a]))return!0})},not:function(a){return this.pushStack(W(this,a,!1),"not",a)},filter:function(a){return this.pushStack(W(this,a,!0),"filter",a)},is:function(a){return!!a&&(typeof a=="string"?f.filter(a,this).length>0:this.filter(a).length>0)},closest:function(a,b){var c=[],d,e,g=this[0];if(f.isArray(a)){var h,i,j={},k=1;if(g&&a.length){for(d=0,e=a.length;d<e;d++)i=a[d],j[i]||(j[i]=T.test(i)?f(i,b||this.context):i);while(g&&g.ownerDocument&&g!==b){for(i in j)h=j[i],(h.jquery?h.index(g)>-1:f(g).is(h))&&c.push({selector:i,elem:g,level:k});g=g.parentNode,k++}}return c}var l=T.test(a)||typeof a!="string"?f(a,b||this.context):0;for(d=0,e=this.length;d<e;d++){g=this[d];while(g){if(l?l.index(g)>-1:f.find.matchesSelector(g,a)){c.push(g);break}g=g.parentNode;if(!g||!g.ownerDocument||g===b||g.nodeType===11)break}}c=c.length>1?f.unique(c):c;return this.pushStack(c,"closest",a)},index:function(a){if(!a||typeof a=="string")return f.inArray(this[0],a?f(a):this.parent().children());return f.inArray(a.jquery?a[0]:a,this)},add:function(a,b){var c=typeof a=="string"?f(a,b):f.makeArray(a&&a.nodeType?[a]:a),d=f.merge(this.get(),c);return this.pushStack(V(c[0])||V(d[0])?d:f.unique(d))},andSelf:function(){return this.add(this.prevObject)}}),f.each({parent:function(a){var b=a.parentNode;return b&&b.nodeType!==11?b:null},parents:function(a){return f.dir(a,"parentNode")},parentsUntil:function(a,b,c){return f.dir(a,"parentNode",c)},next:function(a){return f.nth(a,2,"nextSibling")},prev:function(a){return f.nth(a,2,"previousSibling")},nextAll:function(a){return f.dir(a,"nextSibling")},prevAll:function(a){return f.dir(a,"previousSibling")},nextUntil:function(a,b,c){return f.dir(a,"nextSibling",c)},prevUntil:function(a,b,c){return f.dir(a,"previousSibling",c)},siblings:function(a){return f.sibling(a.parentNode.firstChild,a)},children:function(a){return f.sibling(a.firstChild)},contents:function(a){return f.nodeName(a,"iframe")?a.contentDocument||a.contentWindow.document:f.makeArray(a.childNodes)}},function(a,b){f.fn[a]=function(c,d){var e=f.map(this,b,c),g=S.call(arguments);O.test(a)||(d=c),d&&typeof d=="string"&&(e=f.filter(d,e)),e=this.length>1&&!U[a]?f.unique(e):e,(this.length>1||Q.test(d))&&P.test(a)&&(e=e.reverse());return this.pushStack(e,a,g.join(","))}}),f.extend({filter:function(a,b,c){c&&(a=":not("+a+")");return b.length===1?f.find.matchesSelector(b[0],a)?[b[0]]:[]:f.find.matches(a,b)},dir:function(a,c,d){var e=[],g=a[c];while(g&&g.nodeType!==9&&(d===b||g.nodeType!==1||!f(g).is(d)))g.nodeType===1&&e.push(g),g=g[c];return e},nth:function(a,b,c,d){b=b||1;var e=0;for(;a;a=a[c])if(a.nodeType===1&&++e===b)break;return a},sibling:function(a,b){var c=[];for(;a;a=a.nextSibling)a.nodeType===1&&a!==b&&c.push(a);return c}});var X=/ jQuery\d+="(?:\d+|null)"/g,Y=/^\s+/,Z=/<(?!area|br|col|embed|hr|img|input|link|meta|param)(([\w:]+)[^>]*)\/>/ig,$=/<([\w:]+)/,_=/<tbody/i,ba=/<|&#?\w+;/,bb=/<(?:script|object|embed|option|style)/i,bc=/checked\s*(?:[^=]|=\s*.checked.)/i,bd=/\/(java|ecma)script/i,be=/^\s*<!(?:\[CDATA\[|\-\-)/,bf={option:[1,"<select multiple='multiple'>","</select>"],legend:[1,"<fieldset>","</fieldset>"],thead:[1,"<table>","</table>"],tr:[2,"<table><tbody>","</tbody></table>"],td:[3,"<table><tbody><tr>","</tr></tbody></table>"],col:[2,"<table><tbody></tbody><colgroup>","</colgroup></table>"],area:[1,"<map>","</map>"],_default:[0,"",""]};bf.optgroup=bf.option,bf.tbody=bf.tfoot=bf.colgroup=bf.caption=bf.thead,bf.th=bf.td,f.support.htmlSerialize||(bf._default=[1,"div<div>","</div>"]),f.fn.extend({text:function(a){if(f.isFunction(a))return this.each(function(b){var c=f(this);c.text(a.call(this,b,c.text()))});if(typeof a!="object"&&a!==b)return this.empty().append((this[0]&&this[0].ownerDocument||c).createTextNode(a));return f.text(this)},wrapAll:function(a){if(f.isFunction(a))return this.each(function(b){f(this).wrapAll(a.call(this,b))});if(this[0]){var b=f(a,this[0].ownerDocument).eq(0).clone(!0);this[0].parentNode&&b.insertBefore(this[0]),b.map(function(){var a=this;while(a.firstChild&&a.firstChild.nodeType===1)a=a.firstChild;return a}).append(this)}return this},wrapInner:function(a){if(f.isFunction(a))return this.each(function(b){f(this).wrapInner(a.call(this,b))});return this.each(function(){var b=f(this),c=b.contents();c.length?c.wrapAll(a):b.append(a)})},wrap:function(a){return this.each(function(){f(this).wrapAll(a)})},unwrap:function(){return this.parent().each(function(){f.nodeName(this,"body")||f(this).replaceWith(this.childNodes)}).end()},append:function(){return this.domManip(arguments,!0,function(a){this.nodeType===1&&this.appendChild(a)})},prepend:function(){return this.domManip(arguments,!0,function(a){this.nodeType===1&&this.insertBefore(a,this.firstChild)})},before:function(){if(this[0]&&this[0].parentNode)return this.domManip(arguments,!1,function(a){this.parentNode.insertBefore(a,this)});if(arguments.length){var a=f(arguments[0]);a.push.apply(a,this.toArray());return this.pushStack(a,"before",arguments)}},after:function(){if(this[0]&&this[0].parentNode)return this.domManip(arguments,!1,function(a){this.parentNode.insertBefore(a,this.nextSibling)});if(arguments.length){var a=this.pushStack(this,"after",arguments);a.push.apply(a,f(arguments[0]).toArray());return a}},remove:function(a,b){for(var c=0,d;(d=this[c])!=null;c++)if(!a||f.filter(a,[d]).length)!b&&d.nodeType===1&&(f.cleanData(d.getElementsByTagName("*")),f.cleanData([d])),d.parentNode&&d.parentNode.removeChild(d);return this},empty:function(){for(var a=0,b;(b=this[a])!=null;a++){b.nodeType===1&&f.cleanData(b.getElementsByTagName("*"));while(b.firstChild)b.removeChild(b.firstChild)}return this},clone:function(a,b){a=a==null?!1:a,b=b==null?a:b;return this.map(function(){return f.clone(this,a,b)})},html:function(a){if(a===b)return this[0]&&this[0].nodeType===1?this[0].innerHTML.replace(X,""):null;if(typeof a=="string"&&!bb.test(a)&&(f.support.leadingWhitespace||!Y.test(a))&&!bf[($.exec(a)||["",""])[1].toLowerCase()]){a=a.replace(Z,"<$1></$2>");try{for(var c=0,d=this.length;c<d;c++)this[c].nodeType===1&&(f.cleanData(this[c].getElementsByTagName("*")),this[c].innerHTML=a)}catch(e){this.empty().append(a)}}else f.isFunction(a)?this.each(function(b){var c=f(this);c.html(a.call(this,b,c.html()))}):this.empty().append(a);return this},replaceWith:function(a){if(this[0]&&this[0].parentNode){if(f.isFunction(a))return this.each(function(b){var c=f(this),d=c.html();c.replaceWith(a.call(this,b,d))});typeof a!="string"&&(a=f(a).detach());return this.each(function(){var b=this.nextSibling,c=this.parentNode;f(this).remove(),b?f(b).before(a):f(c).append(a)})}return this.length?this.pushStack(f(f.isFunction(a)?a():a),"replaceWith",a):this},detach:function(a){return this.remove(a,!0)},domManip:function(a,c,d){var e,g,h,i,j=a[0],k=[];if(!f.support.checkClone&&arguments.length===3&&typeof j=="string"&&bc.test(j))return this.each(function(){f(this).domManip(a,c,d,!0)});if(f.isFunction(j))return this.each(function(e){var g=f(this);a[0]=j.call(this,e,c?g.html():b),g.domManip(a,c,d)});if(this[0]){i=j&&j.parentNode,f.support.parentNode&&i&&i.nodeType===11&&i.childNodes.length===this.length?e={fragment:i}:e=f.buildFragment(a,this,k),h=e.fragment,h.childNodes.length===1?g=h=h.firstChild:g=h.firstChild;if(g){c=c&&f.nodeName(g,"tr");for(var l=0,m=this.length,n=m-1;l<m;l++)d.call(c?bg(this[l],g):this[l],e.cacheable||m>1&&l<n?f.clone(h,!0,!0):h)}k.length&&f.each(k,bm)}return this}}),f.buildFragment=function(a,b,d){var e,g,h,i;b&&b[0]&&(i=b[0].ownerDocument||b[0]),i.createDocumentFragment||(i=c),a.length===1&&typeof a[0]=="string"&&a[0].length<512&&i===c&&a[0].charAt(0)==="<"&&!bb.test(a[0])&&(f.support.checkClone||!bc.test(a[0]))&&(g=!0,h=f.fragments[a[0]],h&&h!==1&&(e=h)),e||(e=i.createDocumentFragment(),f.clean(a,i,e,d)),g&&(f.fragments[a[0]]=h?e:1);return{fragment:e,cacheable:g}},f.fragments={},f.each({appendTo:"append",prependTo:"prepend",insertBefore:"before",insertAfter:"after",replaceAll:"replaceWith"},function(a,b){f.fn[a]=function(c){var d=[],e=f(c),g=this.length===1&&this[0].parentNode;if(g&&g.nodeType===11&&g.childNodes.length===1&&e.length===1){e[b](this[0]);return this}for(var h=0,i=e.length;h<i;h++){var j=(h>0?this.clone(!0):this).get();f(e[h])[b](j),d=d.concat(j
+)}return this.pushStack(d,a,e.selector)}}),f.extend({clone:function(a,b,c){var d=a.cloneNode(!0),e,g,h;if((!f.support.noCloneEvent||!f.support.noCloneChecked)&&(a.nodeType===1||a.nodeType===11)&&!f.isXMLDoc(a)){bi(a,d),e=bj(a),g=bj(d);for(h=0;e[h];++h)bi(e[h],g[h])}if(b){bh(a,d);if(c){e=bj(a),g=bj(d);for(h=0;e[h];++h)bh(e[h],g[h])}}e=g=null;return d},clean:function(a,b,d,e){var g;b=b||c,typeof b.createElement=="undefined"&&(b=b.ownerDocument||b[0]&&b[0].ownerDocument||c);var h=[],i;for(var j=0,k;(k=a[j])!=null;j++){typeof k=="number"&&(k+="");if(!k)continue;if(typeof k=="string")if(!ba.test(k))k=b.createTextNode(k);else{k=k.replace(Z,"<$1></$2>");var l=($.exec(k)||["",""])[1].toLowerCase(),m=bf[l]||bf._default,n=m[0],o=b.createElement("div");o.innerHTML=m[1]+k+m[2];while(n--)o=o.lastChild;if(!f.support.tbody){var p=_.test(k),q=l==="table"&&!p?o.firstChild&&o.firstChild.childNodes:m[1]==="<table>"&&!p?o.childNodes:[];for(i=q.length-1;i>=0;--i)f.nodeName(q[i],"tbody")&&!q[i].childNodes.length&&q[i].parentNode.removeChild(q[i])}!f.support.leadingWhitespace&&Y.test(k)&&o.insertBefore(b.createTextNode(Y.exec(k)[0]),o.firstChild),k=o.childNodes}var r;if(!f.support.appendChecked)if(k[0]&&typeof (r=k.length)=="number")for(i=0;i<r;i++)bl(k[i]);else bl(k);k.nodeType?h.push(k):h=f.merge(h,k)}if(d){g=function(a){return!a.type||bd.test(a.type)};for(j=0;h[j];j++)if(e&&f.nodeName(h[j],"script")&&(!h[j].type||h[j].type.toLowerCase()==="text/javascript"))e.push(h[j].parentNode?h[j].parentNode.removeChild(h[j]):h[j]);else{if(h[j].nodeType===1){var s=f.grep(h[j].getElementsByTagName("script"),g);h.splice.apply(h,[j+1,0].concat(s))}d.appendChild(h[j])}}return h},cleanData:function(a){var b,c,d=f.cache,e=f.expando,g=f.event.special,h=f.support.deleteExpando;for(var i=0,j;(j=a[i])!=null;i++){if(j.nodeName&&f.noData[j.nodeName.toLowerCase()])continue;c=j[f.expando];if(c){b=d[c]&&d[c][e];if(b&&b.events){for(var k in b.events)g[k]?f.event.remove(j,k):f.removeEvent(j,k,b.handle);b.handle&&(b.handle.elem=null)}h?delete j[f.expando]:j.removeAttribute&&j.removeAttribute(f.expando),delete d[c]}}}});var bn=/alpha\([^)]*\)/i,bo=/opacity=([^)]*)/,bp=/([A-Z]|^ms)/g,bq=/^-?\d+(?:px)?$/i,br=/^-?\d/,bs=/^[+\-]=/,bt=/[^+\-\.\de]+/g,bu={position:"absolute",visibility:"hidden",display:"block"},bv=["Left","Right"],bw=["Top","Bottom"],bx,by,bz;f.fn.css=function(a,c){if(arguments.length===2&&c===b)return this;return f.access(this,a,c,!0,function(a,c,d){return d!==b?f.style(a,c,d):f.css(a,c)})},f.extend({cssHooks:{opacity:{get:function(a,b){if(b){var c=bx(a,"opacity","opacity");return c===""?"1":c}return a.style.opacity}}},cssNumber:{fillOpacity:!0,fontWeight:!0,lineHeight:!0,opacity:!0,orphans:!0,widows:!0,zIndex:!0,zoom:!0},cssProps:{"float":f.support.cssFloat?"cssFloat":"styleFloat"},style:function(a,c,d,e){if(!!a&&a.nodeType!==3&&a.nodeType!==8&&!!a.style){var g,h,i=f.camelCase(c),j=a.style,k=f.cssHooks[i];c=f.cssProps[i]||i;if(d===b){if(k&&"get"in k&&(g=k.get(a,!1,e))!==b)return g;return j[c]}h=typeof d;if(h==="number"&&isNaN(d)||d==null)return;h==="string"&&bs.test(d)&&(d=+d.replace(bt,"")+parseFloat(f.css(a,c)),h="number"),h==="number"&&!f.cssNumber[i]&&(d+="px");if(!k||!("set"in k)||(d=k.set(a,d))!==b)try{j[c]=d}catch(l){}}},css:function(a,c,d){var e,g;c=f.camelCase(c),g=f.cssHooks[c],c=f.cssProps[c]||c,c==="cssFloat"&&(c="float");if(g&&"get"in g&&(e=g.get(a,!0,d))!==b)return e;if(bx)return bx(a,c)},swap:function(a,b,c){var d={};for(var e in b)d[e]=a.style[e],a.style[e]=b[e];c.call(a);for(e in b)a.style[e]=d[e]}}),f.curCSS=f.css,f.each(["height","width"],function(a,b){f.cssHooks[b]={get:function(a,c,d){var e;if(c){if(a.offsetWidth!==0)return bA(a,b,d);f.swap(a,bu,function(){e=bA(a,b,d)});return e}},set:function(a,b){if(!bq.test(b))return b;b=parseFloat(b);if(b>=0)return b+"px"}}}),f.support.opacity||(f.cssHooks.opacity={get:function(a,b){return bo.test((b&&a.currentStyle?a.currentStyle.filter:a.style.filter)||"")?parseFloat(RegExp.$1)/100+"":b?"1":""},set:function(a,b){var c=a.style,d=a.currentStyle;c.zoom=1;var e=f.isNaN(b)?"":"alpha(opacity="+b*100+")",g=d&&d.filter||c.filter||"";c.filter=bn.test(g)?g.replace(bn,e):g+" "+e}}),f(function(){f.support.reliableMarginRight||(f.cssHooks.marginRight={get:function(a,b){var c;f.swap(a,{display:"inline-block"},function(){b?c=bx(a,"margin-right","marginRight"):c=a.style.marginRight});return c}})}),c.defaultView&&c.defaultView.getComputedStyle&&(by=function(a,c){var d,e,g;c=c.replace(bp,"-$1").toLowerCase();if(!(e=a.ownerDocument.defaultView))return b;if(g=e.getComputedStyle(a,null))d=g.getPropertyValue(c),d===""&&!f.contains(a.ownerDocument.documentElement,a)&&(d=f.style(a,c));return d}),c.documentElement.currentStyle&&(bz=function(a,b){var c,d=a.currentStyle&&a.currentStyle[b],e=a.runtimeStyle&&a.runtimeStyle[b],f=a.style;!bq.test(d)&&br.test(d)&&(c=f.left,e&&(a.runtimeStyle.left=a.currentStyle.left),f.left=b==="fontSize"?"1em":d||0,d=f.pixelLeft+"px",f.left=c,e&&(a.runtimeStyle.left=e));return d===""?"auto":d}),bx=by||bz,f.expr&&f.expr.filters&&(f.expr.filters.hidden=function(a){var b=a.offsetWidth,c=a.offsetHeight;return b===0&&c===0||!f.support.reliableHiddenOffsets&&(a.style.display||f.css(a,"display"))==="none"},f.expr.filters.visible=function(a){return!f.expr.filters.hidden(a)});var bB=/%20/g,bC=/\[\]$/,bD=/\r?\n/g,bE=/#.*$/,bF=/^(.*?):[ \t]*([^\r\n]*)\r?$/mg,bG=/^(?:color|date|datetime|email|hidden|month|number|password|range|search|tel|text|time|url|week)$/i,bH=/^(?:about|app|app\-storage|.+\-extension|file|widget):$/,bI=/^(?:GET|HEAD)$/,bJ=/^\/\//,bK=/\?/,bL=/<script\b[^<]*(?:(?!<\/script>)<[^<]*)*<\/script>/gi,bM=/^(?:select|textarea)/i,bN=/\s+/,bO=/([?&])_=[^&]*/,bP=/^([\w\+\.\-]+:)(?:\/\/([^\/?#:]*)(?::(\d+))?)?/,bQ=f.fn.load,bR={},bS={},bT,bU;try{bT=e.href}catch(bV){bT=c.createElement("a"),bT.href="",bT=bT.href}bU=bP.exec(bT.toLowerCase())||[],f.fn.extend({load:function(a,c,d){if(typeof a!="string"&&bQ)return bQ.apply(this,arguments);if(!this.length)return this;var e=a.indexOf(" ");if(e>=0){var g=a.slice(e,a.length);a=a.slice(0,e)}var h="GET";c&&(f.isFunction(c)?(d=c,c=b):typeof c=="object"&&(c=f.param(c,f.ajaxSettings.traditional),h="POST"));var i=this;f.ajax({url:a,type:h,dataType:"html",data:c,complete:function(a,b,c){c=a.responseText,a.isResolved()&&(a.done(function(a){c=a}),i.html(g?f("<div>").append(c.replace(bL,"")).find(g):c)),d&&i.each(d,[c,b,a])}});return this},serialize:function(){return f.param(this.serializeArray())},serializeArray:function(){return this.map(function(){return this.elements?f.makeArray(this.elements):this}).filter(function(){return this.name&&!this.disabled&&(this.checked||bM.test(this.nodeName)||bG.test(this.type))}).map(function(a,b){var c=f(this).val();return c==null?null:f.isArray(c)?f.map(c,function(a,c){return{name:b.name,value:a.replace(bD,"\r\n")}}):{name:b.name,value:c.replace(bD,"\r\n")}}).get()}}),f.each("ajaxStart ajaxStop ajaxComplete ajaxError ajaxSuccess ajaxSend".split(" "),function(a,b){f.fn[b]=function(a){return this.bind(b,a)}}),f.each(["get","post"],function(a,c){f[c]=function(a,d,e,g){f.isFunction(d)&&(g=g||e,e=d,d=b);return f.ajax({type:c,url:a,data:d,success:e,dataType:g})}}),f.extend({getScript:function(a,c){return f.get(a,b,c,"script")},getJSON:function(a,b,c){return f.get(a,b,c,"json")},ajaxSetup:function(a,b){b?f.extend(!0,a,f.ajaxSettings,b):(b=a,a=f.extend(!0,f.ajaxSettings,b));for(var c in{context:1,url:1})c in b?a[c]=b[c]:c in f.ajaxSettings&&(a[c]=f.ajaxSettings[c]);return a},ajaxSettings:{url:bT,isLocal:bH.test(bU[1]),global:!0,type:"GET",contentType:"application/x-www-form-urlencoded",processData:!0,async:!0,accepts:{xml:"application/xml, text/xml",html:"text/html",text:"text/plain",json:"application/json, text/javascript","*":"*/*"},contents:{xml:/xml/,html:/html/,json:/json/},responseFields:{xml:"responseXML",text:"responseText"},converters:{"* text":a.String,"text html":!0,"text json":f.parseJSON,"text xml":f.parseXML}},ajaxPrefilter:bW(bR),ajaxTransport:bW(bS),ajax:function(a,c){function w(a,c,l,m){if(s!==2){s=2,q&&clearTimeout(q),p=b,n=m||"",v.readyState=a?4:0;var o,r,u,w=l?bZ(d,v,l):b,x,y;if(a>=200&&a<300||a===304){if(d.ifModified){if(x=v.getResponseHeader("Last-Modified"))f.lastModified[k]=x;if(y=v.getResponseHeader("Etag"))f.etag[k]=y}if(a===304)c="notmodified",o=!0;else try{r=b$(d,w),c="success",o=!0}catch(z){c="parsererror",u=z}}else{u=c;if(!c||a)c="error",a<0&&(a=0)}v.status=a,v.statusText=c,o?h.resolveWith(e,[r,c,v]):h.rejectWith(e,[v,c,u]),v.statusCode(j),j=b,t&&g.trigger("ajax"+(o?"Success":"Error"),[v,d,o?r:u]),i.resolveWith(e,[v,c]),t&&(g.trigger("ajaxComplete",[v,d]),--f.active||f.event.trigger("ajaxStop"))}}typeof a=="object"&&(c=a,a=b),c=c||{};var d=f.ajaxSetup({},c),e=d.context||d,g=e!==d&&(e.nodeType||e instanceof f)?f(e):f.event,h=f.Deferred(),i=f._Deferred(),j=d.statusCode||{},k,l={},m={},n,o,p,q,r,s=0,t,u,v={readyState:0,setRequestHeader:function(a,b){if(!s){var c=a.toLowerCase();a=m[c]=m[c]||a,l[a]=b}return this},getAllResponseHeaders:function(){return s===2?n:null},getResponseHeader:function(a){var c;if(s===2){if(!o){o={};while(c=bF.exec(n))o[c[1].toLowerCase()]=c[2]}c=o[a.toLowerCase()]}return c===b?null:c},overrideMimeType:function(a){s||(d.mimeType=a);return this},abort:function(a){a=a||"abort",p&&p.abort(a),w(0,a);return this}};h.promise(v),v.success=v.done,v.error=v.fail,v.complete=i.done,v.statusCode=function(a){if(a){var b;if(s<2)for(b in a)j[b]=[j[b],a[b]];else b=a[v.status],v.then(b,b)}return this},d.url=((a||d.url)+"").replace(bE,"").replace(bJ,bU[1]+"//"),d.dataTypes=f.trim(d.dataType||"*").toLowerCase().split(bN),d.crossDomain==null&&(r=bP.exec(d.url.toLowerCase()),d.crossDomain=!(!r||r[1]==bU[1]&&r[2]==bU[2]&&(r[3]||(r[1]==="http:"?80:443))==(bU[3]||(bU[1]==="http:"?80:443)))),d.data&&d.processData&&typeof d.data!="string"&&(d.data=f.param(d.data,d.traditional)),bX(bR,d,c,v);if(s===2)return!1;t=d.global,d.type=d.type.toUpperCase(),d.hasContent=!bI.test(d.type),t&&f.active++===0&&f.event.trigger("ajaxStart");if(!d.hasContent){d.data&&(d.url+=(bK.test(d.url)?"&":"?")+d.data),k=d.url;if(d.cache===!1){var x=f.now(),y=d.url.replace(bO,"$1_="+x);d.url=y+(y===d.url?(bK.test(d.url)?"&":"?")+"_="+x:"")}}(d.data&&d.hasContent&&d.contentType!==!1||c.contentType)&&v.setRequestHeader("Content-Type",d.contentType),d.ifModified&&(k=k||d.url,f.lastModified[k]&&v.setRequestHeader("If-Modified-Since",f.lastModified[k]),f.etag[k]&&v.setRequestHeader("If-None-Match",f.etag[k])),v.setRequestHeader("Accept",d.dataTypes[0]&&d.accepts[d.dataTypes[0]]?d.accepts[d.dataTypes[0]]+(d.dataTypes[0]!=="*"?", */*; q=0.01":""):d.accepts["*"]);for(u in d.headers)v.setRequestHeader(u,d.headers[u]);if(d.beforeSend&&(d.beforeSend.call(e,v,d)===!1||s===2)){v.abort();return!1}for(u in{success:1,error:1,complete:1})v[u](d[u]);p=bX(bS,d,c,v);if(!p)w(-1,"No Transport");else{v.readyState=1,t&&g.trigger("ajaxSend",[v,d]),d.async&&d.timeout>0&&(q=setTimeout(function(){v.abort("timeout")},d.timeout));try{s=1,p.send(l,w)}catch(z){status<2?w(-1,z):f.error(z)}}return v},param:function(a,c){var d=[],e=function(a,b){b=f.isFunction(b)?b():b,d[d.length]=encodeURIComponent(a)+"="+encodeURIComponent(b)};c===b&&(c=f.ajaxSettings.traditional);if(f.isArray(a)||a.jquery&&!f.isPlainObject(a))f.each(a,function(){e(this.name,this.value)});else for(var g in a)bY(g,a[g],c,e);return d.join("&").replace(bB,"+")}}),f.extend({active:0,lastModified:{},etag:{}});var b_=f.now(),ca=/(\=)\?(&|$)|\?\?/i;f.ajaxSetup({jsonp:"callback",jsonpCallback:function(){return f.expando+"_"+b_++}}),f.ajaxPrefilter("json jsonp",function(b,c,d){var e=b.contentType==="application/x-www-form-urlencoded"&&typeof b.data=="string";if(b.dataTypes[0]==="jsonp"||b.jsonp!==!1&&(ca.test(b.url)||e&&ca.test(b.data))){var g,h=b.jsonpCallback=f.isFunction(b.jsonpCallback)?b.jsonpCallback():b.jsonpCallback,i=a[h],j=b.url,k=b.data,l="$1"+h+"$2";b.jsonp!==!1&&(j=j.replace(ca,l),b.url===j&&(e&&(k=k.replace(ca,l)),b.data===k&&(j+=(/\?/.test(j)?"&":"?")+b.jsonp+"="+h))),b.url=j,b.data=k,a[h]=function(a){g=[a]},d.always(function(){a[h]=i,g&&f.isFunction(i)&&a[h](g[0])}),b.converters["script json"]=function(){g||f.error(h+" was not called");return g[0]},b.dataTypes[0]="json";return"script"}}),f.ajaxSetup({accepts:{script:"text/javascript, application/javascript, application/ecmascript, application/x-ecmascript"},contents:{script:/javascript|ecmascript/},converters:{"text script":function(a){f.globalEval(a);return a}}}),f.ajaxPrefilter("script",function(a){a.cache===b&&(a.cache=!1),a.crossDomain&&(a.type="GET",a.global=!1)}),f.ajaxTransport("script",function(a){if(a.crossDomain){var d,e=c.head||c.getElementsByTagName("head")[0]||c.documentElement;return{send:function(f,g){d=c.createElement("script"),d.async="async",a.scriptCharset&&(d.charset=a.scriptCharset),d.src=a.url,d.onload=d.onreadystatechange=function(a,c){if(c||!d.readyState||/loaded|complete/.test(d.readyState))d.onload=d.onreadystatechange=null,e&&d.parentNode&&e.removeChild(d),d=b,c||g(200,"success")},e.insertBefore(d,e.firstChild)},abort:function(){d&&d.onload(0,1)}}}});var cb=a.ActiveXObject?function(){for(var a in cd)cd[a](0,1)}:!1,cc=0,cd;f.ajaxSettings.xhr=a.ActiveXObject?function(){return!this.isLocal&&ce()||cf()}:ce,function(a){f.extend(f.support,{ajax:!!a,cors:!!a&&"withCredentials"in a})}(f.ajaxSettings.xhr()),f.support.ajax&&f.ajaxTransport(function(c){if(!c.crossDomain||f.support.cors){var d;return{send:function(e,g){var h=c.xhr(),i,j;c.username?h.open(c.type,c.url,c.async,c.username,c.password):h.open(c.type,c.url,c.async);if(c.xhrFields)for(j in c.xhrFields)h[j]=c.xhrFields[j];c.mimeType&&h.overrideMimeType&&h.overrideMimeType(c.mimeType),!c.crossDomain&&!e["X-Requested-With"]&&(e["X-Requested-With"]="XMLHttpRequest");try{for(j in e)h.setRequestHeader(j,e[j])}catch(k){}h.send(c.hasContent&&c.data||null),d=function(a,e){var j,k,l,m,n;try{if(d&&(e||h.readyState===4)){d=b,i&&(h.onreadystatechange=f.noop,cb&&delete cd[i]);if(e)h.readyState!==4&&h.abort();else{j=h.status,l=h.getAllResponseHeaders(),m={},n=h.responseXML,n&&n.documentElement&&(m.xml=n),m.text=h.responseText;try{k=h.statusText}catch(o){k=""}!j&&c.isLocal&&!c.crossDomain?j=m.text?200:404:j===1223&&(j=204)}}}catch(p){e||g(-1,p)}m&&g(j,k,m,l)},!c.async||h.readyState===4?d():(i=++cc,cb&&(cd||(cd={},f(a).unload(cb)),cd[i]=d),h.onreadystatechange=d)},abort:function(){d&&d(0,1)}}}});var cg={},ch,ci,cj=/^(?:toggle|show|hide)$/,ck=/^([+\-]=)?([\d+.\-]+)([a-z%]*)$/i,cl,cm=[["height","marginTop","marginBottom","paddingTop","paddingBottom"],["width","marginLeft","marginRight","paddingLeft","paddingRight"],["opacity"]],cn,co=a.webkitRequestAnimationFrame||a.mozRequestAnimationFrame||a.oRequestAnimationFrame;f.fn.extend({show:function(a,b,c){var d,e;if(a||a===0)return this.animate(cr("show",3),a,b,c);for(var g=0,h=this.length;g<h;g++)d=this[g],d.style&&(e=d.style.display,!f._data(d,"olddisplay")&&e==="none"&&(e=d.style.display=""),e===""&&f.css(d,"display")==="none"&&f._data(d,"olddisplay",cs(d.nodeName)));for(g=0;g<h;g++){d=this[g];if(d.style){e=d.style.display;if(e===""||e==="none")d.style.display=f._data(d,"olddisplay")||""}}return this},hide:function(a,b,c){if(a||a===0)return this.animate(cr("hide",3),a,b,c);for(var d=0,e=this.length;d<e;d++)if(this[d].style){var g=f.css(this[d],"display");g!=="none"&&!f._data(this[d],"olddisplay")&&f._data(this[d],"olddisplay",g)}for(d=0;d<e;d++)this[d].style&&(this[d].style.display="none");return this},_toggle:f.fn.toggle,toggle:function(a,b,c){var d=typeof a=="boolean";f.isFunction(a)&&f.isFunction(b)?this._toggle.apply(this,arguments):a==null||d?this.each(function(){var b=d?a:f(this).is(":hidden");f(this)[b?"show":"hide"]()}):this.animate(cr("toggle",3),a,b,c);return this},fadeTo:function(a,b,c,d){return this.filter(":hidden").css("opacity",0).show().end().animate({opacity:b},a,c,d)},animate:function(a,b,c,d){var e=f.speed(b,c,d);if(f.isEmptyObject(a))return this.each(e.complete,[!1]);a=f.extend({},a);return this[e.queue===!1?"each":"queue"](function(){e.queue===!1&&f._mark(this);var b=f.extend({},e),c=this.nodeType===1,d=c&&f(this).is(":hidden"),g,h,i,j,k,l,m,n,o;b.animatedProperties={};for(i in a){g=f.camelCase(i),i!==g&&(a[g]=a[i],delete a[i]),h=a[g],f.isArray(h)?(b.animatedProperties[g]=h[1],h=a[g]=h[0]):b.animatedProperties[g]=b.specialEasing&&b.specialEasing[g]||b.easing||"swing";if(h==="hide"&&d||h==="show"&&!d)return b.complete.call(this);c&&(g==="height"||g==="width")&&(b.overflow=[this.style.overflow,this.style.overflowX,this.style.overflowY],f.css(this,"display")==="inline"&&f.css(this,"float")==="none"&&(f.support.inlineBlockNeedsLayout?(j=cs(this.nodeName),j==="inline"?this.style.display="inline-block":(this.style.display="inline",this.style.zoom=1)):this.style.display="inline-block"))}b.overflow!=null&&(this.style.overflow="hidden");for(i in a)k=new f.fx(this,b,i),h=a[i],cj.test(h)?k[h==="toggle"?d?"show":"hide":h]():(l=ck.exec(h),m=k.cur(),l?(n=parseFloat(l[2]),o=l[3]||(f.cssNumber[i]?"":"px"),o!=="px"&&(f.style(this,i,(n||1)+o),m=(n||1)/k.cur()*m,f.style(this,i,m+o)),l[1]&&(n=(l[1]==="-="?-1:1)*n+m),k.custom(m,n,o)):k.custom(m,h,""));return!0})},stop:function(a,b){a&&this.queue([]),this.each(function(){var a=f.timers,c=a.length;b||f._unmark(!0,this);while(c--)a[c].elem===this&&(b&&a[c](!0),a.splice(c,1))}),b||this.dequeue();return this}}),f.each({slideDown:cr("show",1),slideUp:cr("hide",1),slideToggle:cr("toggle",1),fadeIn:{opacity:"show"},fadeOut:{opacity:"hide"},fadeToggle:{opacity:"toggle"}},function(a,b){f.fn[a]=function(a,c,d){return this.animate(b,a,c,d)}}),f.extend({speed:function(a,b,c){var d=a&&typeof a=="object"?f.extend({},a):{complete:c||!c&&b||f.isFunction(a)&&a,duration:a,easing:c&&b||b&&!f.isFunction(b)&&b};d.duration=f.fx.off?0:typeof d.duration=="number"?d.duration:d.duration in f.fx.speeds?f.fx.speeds[d.duration]:f.fx.speeds._default,d.old=d.complete,d.complete=function(a){f.isFunction(d.old)&&d.old.call(this),d.queue!==!1?f.dequeue(this):a!==!1&&f._unmark(this)};return d},easing:{linear:function(a,b,c,d){return c+d*a},swing:function(a,b,c,d){return(-Math.cos(a*Math.PI)/2+.5)*d+c}},timers:[],fx:function(a,b,c){this.options=b,this.elem=a,this.prop=c,b.orig=b.orig||{}}}),f.fx.prototype={update:function(){this.options.step&&this.options.step.call(this.elem,this.now,this),(f.fx.step[this.prop]||f.fx.step._default)(this)},cur:function(){if(this.elem[this.prop]!=null&&(!this.elem.style||this.elem.style[this.prop]==null))return this.elem[this.prop];var a,b=f.css(this.elem,this.prop);return isNaN(a=parseFloat(b))?!b||b==="auto"?0:b:a},custom:function(a,b,c){function h(a){return d.step(a)}var d=this,e=f.fx,g;this.startTime=cn||cp(),this.start=a,this.end=b,this.unit=c||this.unit||(f.cssNumber[this.prop]?"":"px"),this.now=this.start,this.pos=this.state=0,h.elem=this.elem,h()&&f.timers.push(h)&&!cl&&(co?(cl=!0,g=function(){cl&&(co(g),e.tick())},co(g)):cl=setInterval(e.tick,e.interval))},show:function(){this.options.orig[this.prop]=f.style(this.elem,this.prop),this.options.show=!0,this.custom(this.prop==="width"||this.prop==="height"?1:0,this.cur()),f(this.elem).show()},hide:function(){this.options.orig[this.prop]=f.style(this.elem,this.prop),this.options.hide=!0,this.custom(this.cur(),0)},step:function(a){var b=cn||cp(),c=!0,d=this.elem,e=this.options,g,h;if(a||b>=e.duration+this.startTime){this.now=this.end,this.pos=this.state=1,this.update(),e.animatedProperties[this.prop]=!0;for(g in e.animatedProperties)e.animatedProperties[g]!==!0&&(c=!1);if(c){e.overflow!=null&&!f.support.shrinkWrapBlocks&&f.each(["","X","Y"],function(a,b){d.style["overflow"+b]=e.overflow[a]}),e.hide&&f(d).hide();if(e.hide||e.show)for(var i in e.animatedProperties)f.style(d,i,e.orig[i]);e.complete.call(d)}return!1}e.duration==Infinity?this.now=b:(h=b-this.startTime,this.state=h/e.duration,this.pos=f.easing[e.animatedProperties[this.prop]](this.state,h,0,1,e.duration),this.now=this.start+(this.end-this.start)*this.pos),this.update();return!0}},f.extend(f.fx,{tick:function(){for(var a=f.timers,b=0;b<a.length;++b)a[b]()||a.splice(b--,1);a.length||f.fx.stop()},interval:13,stop:function(){clearInterval(cl),cl=null},speeds:{slow:600,fast:200,_default:400},step:{opacity:function(a){f.style(a.elem,"opacity",a.now)},_default:function(a){a.elem.style&&a.elem.style[a.prop]!=null?a.elem.style[a.prop]=(a.prop==="width"||a.prop==="height"?Math.max(0,a.now):a.now)+a.unit:a.elem[a.prop]=a.now}}}),f.expr&&f.expr.filters&&(f.expr.filters.animated=function(a){return f.grep(f.timers,function(b){return a===b.elem}).length});var ct=/^t(?:able|d|h)$/i,cu=/^(?:body|html)$/i;"getBoundingClientRect"in c.documentElement?f.fn.offset=function(a){var b=this[0],c;if(a)return this.each(function(b){f.offset.setOffset(this,a,b)});if(!b||!b.ownerDocument)return null;if(b===b.ownerDocument.body)return f.offset.bodyOffset(b);try{c=b.getBoundingClientRect()}catch(d){}var e=b.ownerDocument,g=e.documentElement;if(!c||!f.contains(g,b))return c?{top:c.top,left:c.left}:{top:0,left:0};var h=e.body,i=cv(e),j=g.clientTop||h.clientTop||0,k=g.clientLeft||h.clientLeft||0,l=i.pageYOffset||f.support.boxModel&&g.scrollTop||h.scrollTop,m=i.pageXOffset||f.support.boxModel&&g.scrollLeft||h.scrollLeft,n=c.top+l-j,o=c.left+m-k;return{top:n,left:o}}:f.fn.offset=function(a){var b=this[0];if(a)return this.each(function(b){f.offset.setOffset(this,a,b)});if(!b||!b.ownerDocument)return null;if(b===b.ownerDocument.body)return f.offset.bodyOffset(b);f.offset.initialize();var c,d=b.offsetParent,e=b,g=b.ownerDocument,h=g.documentElement,i=g.body,j=g.defaultView,k=j?j.getComputedStyle(b,null):b.currentStyle,l=b.offsetTop,m=b.offsetLeft;while((b=b.parentNode)&&b!==i&&b!==h){if(f.offset.supportsFixedPosition&&k.position==="fixed")break;c=j?j.getComputedStyle(b,null):b.currentStyle,l-=b.scrollTop,m-=b.scrollLeft,b===d&&(l+=b.offsetTop,m+=b.offsetLeft,f.offset.doesNotAddBorder&&(!f.offset.doesAddBorderForTableAndCells||!ct.test(b.nodeName))&&(l+=parseFloat(c.borderTopWidth)||0,m+=parseFloat(c.borderLeftWidth)||0),e=d,d=b.offsetParent),f.offset.subtractsBorderForOverflowNotVisible&&c.overflow!=="visible"&&(l+=parseFloat(c.borderTopWidth)||0,m+=parseFloat(c.borderLeftWidth)||0),k=c}if(k.position==="relative"||k.position==="static")l+=i.offsetTop,m+=i.offsetLeft;f.offset.supportsFixedPosition&&k.position==="fixed"&&(l+=Math.max(h.scrollTop,i.scrollTop),m+=Math.max(h.scrollLeft,i.scrollLeft));return{top:l,left:m}},f.offset={initialize:function(){var a=c.body,b=c.createElement("div"),d,e,g,h,i=parseFloat(f.css(a,"marginTop"))||0,j="<div style='position:absolute;top:0;left:0;margin:0;border:5px solid #000;padding:0;width:1px;height:1px;'><div></div></div><table style='position:absolute;top:0;left:0;margin:0;border:5px solid #000;padding:0;width:1px;height:1px;' cellpadding='0' cellspacing='0'><tr><td></td></tr></table>";f.extend(b.style,{position:"absolute",top:0,left:0,margin:0,border:0,width:"1px",height:"1px",visibility:"hidden"}),b.innerHTML=j,a.insertBefore(b,a.firstChild),d=b.firstChild,e=d.firstChild,h=d.nextSibling.firstChild.firstChild,this.doesNotAddBorder=e.offsetTop!==5,this.doesAddBorderForTableAndCells=h.offsetTop===5,e.style.position="fixed",e.style.top="20px",this.supportsFixedPosition=e.offsetTop===20||e.offsetTop===15,e.style.position=e.style.top="",d.style.overflow="hidden",d.style.position="relative",this.subtractsBorderForOverflowNotVisible=e.offsetTop===-5,this.doesNotIncludeMarginInBodyOffset=a.offsetTop!==i,a.removeChild(b),f.offset.initialize=f.noop},bodyOffset:function(a){var b=a.offsetTop,c=a.offsetLeft;f.offset.initialize(),f.offset.doesNotIncludeMarginInBodyOffset&&(b+=parseFloat(f.css(a,"marginTop"))||0,c+=parseFloat(f.css(a,"marginLeft"))||0);return{top:b,left:c}},setOffset:function(a,b,c){var d=f.css(a,"position");d==="static"&&(a.style.position="relative");var e=f(a),g=e.offset(),h=f.css(a,"top"),i=f.css(a,"left"),j=(d==="absolute"||d==="fixed")&&f.inArray("auto",[h,i])>-1,k={},l={},m,n;j?(l=e.position(),m=l.top,n=l.left):(m=parseFloat(h)||0,n=parseFloat(i)||0),f.isFunction(b)&&(b=b.call(a,c,g)),b.top!=null&&(k.top=b.top-g.top+m),b.left!=null&&(k.left=b.left-g.left+n),"using"in b?b.using.call(a,k):e.css(k)}},f.fn.extend({position:function(){if(!this[0])return null;var a=this[0],b=this.offsetParent(),c=this.offset(),d=cu.test(b[0].nodeName)?{top:0,left:0}:b.offset();c.top-=parseFloat(f.css(a,"marginTop"))||0,c.left-=parseFloat(f.css(a,"marginLeft"))||0,d.top+=parseFloat(f.css(b[0],"borderTopWidth"))||0,d.left+=parseFloat(f.css(b[0],"borderLeftWidth"))||0;return{top:c.top-d.top,left:c.left-d.left}},offsetParent:function(){return this.map(function(){var a=this.offsetParent||c.body;while(a&&!cu.test(a.nodeName)&&f.css(a,"position")==="static")a=a.offsetParent;return a})}}),f.each(["Left","Top"],function(a,c){var d="scroll"+c;f.fn[d]=function(c){var e,g;if(c===b){e=this[0];if(!e)return null;g=cv(e);return g?"pageXOffset"in g?g[a?"pageYOffset":"pageXOffset"]:f.support.boxModel&&g.document.documentElement[d]||g.document.body[d]:e[d]}return this.each(function(){g=cv(this),g?g.scrollTo(a?f(g).scrollLeft():c,a?c:f(g).scrollTop()):this[d]=c})}}),f.each(["Height","Width"],function(a,c){var d=c.toLowerCase();f.fn["inner"+c]=function(){var a=this[0];return a&&a.style?parseFloat(f.css(a,d,"padding")):null},f.fn["outer"+c]=function(a){var b=this[0];return b&&b.style?parseFloat(f.css(b,d,a?"margin":"border")):null},f.fn[d]=function(a){var e=this[0];if(!e)return a==null?null:this;if(f.isFunction(a))return this.each(function(b){var c=f(this);c[d](a.call(this,b,c[d]()))});if(f.isWindow(e)){var g=e.document.documentElement["client"+c];return e.document.compatMode==="CSS1Compat"&&g||e.document.body["client"+c]||g}if(e.nodeType===9)return Math.max(e.documentElement["client"+c],e.body["scroll"+c],e.documentElement["scroll"+c],e.body["offset"+c],e.documentElement["offset"+c]);if(a===b){var h=f.css(e,d),i=parseFloat(h);return f.isNaN(i)?h:i}return this.css(d,typeof a=="string"?a:a+"px")}}),a.jQuery=a.$=f})(window);
\ No newline at end of file
diff --git a/src/ui/public/js/jquery.tablesorter.min.js b/src/ui/public/js/jquery.tablesorter.min.js
new file mode 100644
index 0000000..4f8c77c
--- /dev/null
+++ b/src/ui/public/js/jquery.tablesorter.min.js
@@ -0,0 +1,4 @@
+
+(function($){$.extend({tablesorter:new
+function(){var parsers=[],widgets=[];this.defaults={cssHeader:"header",cssAsc:"headerSortUp",cssDesc:"headerSortDown",cssChildRow:"expand-child",sortInitialOrder:"asc",sortMultiSortKey:"shiftKey",sortForce:null,sortAppend:null,sortLocaleCompare:true,textExtraction:"simple",parsers:{},widgets:[],widgetZebra:{css:["even","odd"]},headers:{},widthFixed:false,cancelSelection:true,sortList:[],headerList:[],dateFormat:"us",decimal:'/\.|\,/g',onRenderHeader:null,selectorHeaders:'thead th',debug:false};function benchmark(s,d){log(s+","+(new Date().getTime()-d.getTime())+"ms");}this.benchmark=benchmark;function log(s){if(typeof console!="undefined"&&typeof console.debug!="undefined"){console.log(s);}else{alert(s);}}function buildParserCache(table,$headers){if(table.config.debug){var parsersDebug="";}if(table.tBodies.length==0)return;var rows=table.tBodies[0].rows;if(rows[0]){var list=[],cells=rows[0].cells,l=cells.length;for(var i=0;i<l;i++){var p=false;if($.metadata&&($($headers[i]).metadata()&&$($headers[i]).metadata().sorter)){p=getParserById($($headers[i]).metadata().sorter);}else if((table.config.headers[i]&&table.config.headers[i].sorter)){p=getParserById(table.config.headers[i].sorter);}if(!p){p=detectParserForColumn(table,rows,-1,i);}if(table.config.debug){parsersDebug+="column:"+i+" parser:"+p.id+"\n";}list.push(p);}}if(table.config.debug){log(parsersDebug);}return list;};function detectParserForColumn(table,rows,rowIndex,cellIndex){var l=parsers.length,node=false,nodeValue=false,keepLooking=true;while(nodeValue==''&&keepLooking){rowIndex++;if(rows[rowIndex]){node=getNodeFromRowAndCellIndex(rows,rowIndex,cellIndex);nodeValue=trimAndGetNodeText(table.config,node);if(table.config.debug){log('Checking if value was empty on row:'+rowIndex);}}else{keepLooking=false;}}for(var i=1;i<l;i++){if(parsers[i].is(nodeValue,table,node)){return parsers[i];}}return parsers[0];}function getNodeFromRowAndCellIndex(rows,rowIndex,cellIndex){return rows[rowIndex].cells[cellIndex];}function trimAndGetNodeText(config,node){return $.trim(getElementText(config,node));}function getParserById(name){var l=parsers.length;for(var i=0;i<l;i++){if(parsers[i].id.toLowerCase()==name.toLowerCase()){return parsers[i];}}return false;}function buildCache(table){if(table.config.debug){var cacheTime=new Date();}var totalRows=(table.tBodies[0]&&table.tBodies[0].rows.length)||0,totalCells=(table.tBodies[0].rows[0]&&table.tBodies[0].rows[0].cells.length)||0,parsers=table.config.parsers,cache={row:[],normalized:[]};for(var i=0;i<totalRows;++i){var c=$(table.tBodies[0].rows[i]),cols=[];if(c.hasClass(table.config.cssChildRow)){cache.row[cache.row.length-1]=cache.row[cache.row.length-1].add(c);continue;}cache.row.push(c);for(var j=0;j<totalCells;++j){cols.push(parsers[j].format(getElementText(table.config,c[0].cells[j]),table,c[0].cells[j]));}cols.push(cache.normalized.length);cache.normalized.push(cols);cols=null;};if(table.config.debug){benchmark("Building cache for "+totalRows+" rows:",cacheTime);}return cache;};function getElementText(config,node){var text="";if(!node)return"";if(!config.supportsTextContent)config.supportsTextContent=node.textContent||false;if(config.textExtraction=="simple"){if(config.supportsTextContent){text=node.textContent;}else{if(node.childNodes[0]&&node.childNodes[0].hasChildNodes()){text=node.childNodes[0].innerHTML;}else{text=node.innerHTML;}}}else{if(typeof(config.textExtraction)=="function"){text=config.textExtraction(node);}else{text=$(node).text();}}return text;}function appendToTable(table,cache){if(table.config.debug){var appendTime=new Date()}var c=cache,r=c.row,n=c.normalized,totalRows=n.length,checkCell=(n[0].length-1),tableBody=$(table.tBodies[0]),rows=[];for(var i=0;i<totalRows;i++){var pos=n[i][checkCell];rows.push(r[pos]);if(!table.config.appender){var l=r[pos].length;for(var j=0;j<l;j++){tableBody[0].appendChild(r[pos][j]);}}}if(table.config.appender){table.config.appender(table,rows);}rows=null;if(table.config.debug){benchmark("Rebuilt table:",appendTime);}applyWidget(table);setTimeout(function(){$(table).trigger("sortEnd");},0);};function buildHeaders(table){if(table.config.debug){var time=new Date();}var meta=($.metadata)?true:false;var header_index=computeTableHeaderCellIndexes(table);$tableHeaders=$(table.config.selectorHeaders,table).each(function(index){this.column=header_index[this.parentNode.rowIndex+"-"+this.cellIndex];this.order=formatSortingOrder(table.config.sortInitialOrder);this.count=this.order;if(checkHeaderMetadata(this)||checkHeaderOptions(table,index))this.sortDisabled=true;if(checkHeaderOptionsSortingLocked(table,index))this.order=this.lockedOrder=checkHeaderOptionsSortingLocked(table,index);if(!this.sortDisabled){var $th=$(this).addClass(table.config.cssHeader);if(table.config.onRenderHeader)table.config.onRenderHeader.apply($th);}table.config.headerList[index]=this;});if(table.config.debug){benchmark("Built headers:",time);log($tableHeaders);}return $tableHeaders;};function computeTableHeaderCellIndexes(t){var matrix=[];var lookup={};var thead=t.getElementsByTagName('THEAD')[0];var trs=thead.getElementsByTagName('TR');for(var i=0;i<trs.length;i++){var cells=trs[i].cells;for(var j=0;j<cells.length;j++){var c=cells[j];var rowIndex=c.parentNode.rowIndex;var cellId=rowIndex+"-"+c.cellIndex;var rowSpan=c.rowSpan||1;var colSpan=c.colSpan||1
+var firstAvailCol;if(typeof(matrix[rowIndex])=="undefined"){matrix[rowIndex]=[];}for(var k=0;k<matrix[rowIndex].length+1;k++){if(typeof(matrix[rowIndex][k])=="undefined"){firstAvailCol=k;break;}}lookup[cellId]=firstAvailCol;for(var k=rowIndex;k<rowIndex+rowSpan;k++){if(typeof(matrix[k])=="undefined"){matrix[k]=[];}var matrixrow=matrix[k];for(var l=firstAvailCol;l<firstAvailCol+colSpan;l++){matrixrow[l]="x";}}}}return lookup;}function checkCellColSpan(table,rows,row){var arr=[],r=table.tHead.rows,c=r[row].cells;for(var i=0;i<c.length;i++){var cell=c[i];if(cell.colSpan>1){arr=arr.concat(checkCellColSpan(table,headerArr,row++));}else{if(table.tHead.length==1||(cell.rowSpan>1||!r[row+1])){arr.push(cell);}}}return arr;};function checkHeaderMetadata(cell){if(($.metadata)&&($(cell).metadata().sorter===false)){return true;};return false;}function checkHeaderOptions(table,i){if((table.config.headers[i])&&(table.config.headers[i].sorter===false)){return true;};return false;}function checkHeaderOptionsSortingLocked(table,i){if((table.config.headers[i])&&(table.config.headers[i].lockedOrder))return table.config.headers[i].lockedOrder;return false;}function applyWidget(table){var c=table.config.widgets;var l=c.length;for(var i=0;i<l;i++){getWidgetById(c[i]).format(table);}}function getWidgetById(name){var l=widgets.length;for(var i=0;i<l;i++){if(widgets[i].id.toLowerCase()==name.toLowerCase()){return widgets[i];}}};function formatSortingOrder(v){if(typeof(v)!="Number"){return(v.toLowerCase()=="desc")?1:0;}else{return(v==1)?1:0;}}function isValueInArray(v,a){var l=a.length;for(var i=0;i<l;i++){if(a[i][0]==v){return true;}}return false;}function setHeadersCss(table,$headers,list,css){$headers.removeClass(css[0]).removeClass(css[1]);var h=[];$headers.each(function(offset){if(!this.sortDisabled){h[this.column]=$(this);}});var l=list.length;for(var i=0;i<l;i++){h[list[i][0]].addClass(css[list[i][1]]);}}function fixColumnWidth(table,$headers){var c=table.config;if(c.widthFixed){var colgroup=$('<colgroup>');$("tr:first td",table.tBodies[0]).each(function(){colgroup.append($('<col>').css('width',$(this).width()));});$(table).prepend(colgroup);};}function updateHeaderSortCount(table,sortList){var c=table.config,l=sortList.length;for(var i=0;i<l;i++){var s=sortList[i],o=c.headerList[s[0]];o.count=s[1];o.count++;}}function multisort(table,sortList,cache){if(table.config.debug){var sortTime=new Date();}var dynamicExp="var sortWrapper = function(a,b) {",l=sortList.length;for(var i=0;i<l;i++){var c=sortList[i][0];var order=sortList[i][1];var s=(table.config.parsers[c].type=="text")?((order==0)?makeSortFunction("text","asc",c):makeSortFunction("text","desc",c)):((order==0)?makeSortFunction("numeric","asc",c):makeSortFunction("numeric","desc",c));var e="e"+i;dynamicExp+="var "+e+" = "+s;dynamicExp+="if("+e+") { return "+e+"; } ";dynamicExp+="else { ";}var orgOrderCol=cache.normalized[0].length-1;dynamicExp+="return a["+orgOrderCol+"]-b["+orgOrderCol+"];";for(var i=0;i<l;i++){dynamicExp+="}; ";}dynamicExp+="return 0; ";dynamicExp+="}; ";if(table.config.debug){benchmark("Evaling expression:"+dynamicExp,new Date());}eval(dynamicExp);cache.normalized.sort(sortWrapper);if(table.config.debug){benchmark("Sorting on "+sortList.toString()+" and dir "+order+" time:",sortTime);}return cache;};function makeSortFunction(type,direction,index){var a="a["+index+"]",b="b["+index+"]";if(type=='text'&&direction=='asc'){return"("+a+" == "+b+" ? 0 : ("+a+" === null ? Number.POSITIVE_INFINITY : ("+b+" === null ? Number.NEGATIVE_INFINITY : ("+a+" < "+b+") ? -1 : 1 )));";}else if(type=='text'&&direction=='desc'){return"("+a+" == "+b+" ? 0 : ("+a+" === null ? Number.POSITIVE_INFINITY : ("+b+" === null ? Number.NEGATIVE_INFINITY : ("+b+" < "+a+") ? -1 : 1 )));";}else if(type=='numeric'&&direction=='asc'){return"("+a+" === null && "+b+" === null) ? 0 :("+a+" === null ? Number.POSITIVE_INFINITY : ("+b+" === null ? Number.NEGATIVE_INFINITY : "+a+" - "+b+"));";}else if(type=='numeric'&&direction=='desc'){return"("+a+" === null && "+b+" === null) ? 0 :("+a+" === null ? Number.POSITIVE_INFINITY : ("+b+" === null ? Number.NEGATIVE_INFINITY : "+b+" - "+a+"));";}};function makeSortText(i){return"((a["+i+"] < b["+i+"]) ? -1 : ((a["+i+"] > b["+i+"]) ? 1 : 0));";};function makeSortTextDesc(i){return"((b["+i+"] < a["+i+"]) ? -1 : ((b["+i+"] > a["+i+"]) ? 1 : 0));";};function makeSortNumeric(i){return"a["+i+"]-b["+i+"];";};function makeSortNumericDesc(i){return"b["+i+"]-a["+i+"];";};function sortText(a,b){if(table.config.sortLocaleCompare)return a.localeCompare(b);return((a<b)?-1:((a>b)?1:0));};function sortTextDesc(a,b){if(table.config.sortLocaleCompare)return b.localeCompare(a);return((b<a)?-1:((b>a)?1:0));};function sortNumeric(a,b){return a-b;};function sortNumericDesc(a,b){return b-a;};function getCachedSortType(parsers,i){return parsers[i].type;};this.construct=function(settings){return this.each(function(){if(!this.tHead||!this.tBodies)return;var $this,$document,$headers,cache,config,shiftDown=0,sortOrder;this.config={};config=$.extend(this.config,$.tablesorter.defaults,settings);$this=$(this);$.data(this,"tablesorter",config);$headers=buildHeaders(this);this.config.parsers=buildParserCache(this,$headers);cache=buildCache(this);var sortCSS=[config.cssDesc,config.cssAsc];fixColumnWidth(this);$headers.click(function(e){var totalRows=($this[0].tBodies[0]&&$this[0].tBodies[0].rows.length)||0;if(!this.sortDisabled&&totalRows>0){$this.trigger("sortStart");var $cell=$(this);var i=this.column;this.order=this.count++%2;if(this.lockedOrder)this.order=this.lockedOrder;if(!e[config.sortMultiSortKey]){config.sortList=[];if(config.sortForce!=null){var a=config.sortForce;for(var j=0;j<a.length;j++){if(a[j][0]!=i){config.sortList.push(a[j]);}}}config.sortList.push([i,this.order]);}else{if(isValueInArray(i,config.sortList)){for(var j=0;j<config.sortList.length;j++){var s=config.sortList[j],o=config.headerList[s[0]];if(s[0]==i){o.count=s[1];o.count++;s[1]=o.count%2;}}}else{config.sortList.push([i,this.order]);}};setTimeout(function(){setHeadersCss($this[0],$headers,config.sortList,sortCSS);appendToTable($this[0],multisort($this[0],config.sortList,cache));},1);return false;}}).mousedown(function(){if(config.cancelSelection){this.onselectstart=function(){return false};return false;}});$this.bind("update",function(){var me=this;setTimeout(function(){me.config.parsers=buildParserCache(me,$headers);cache=buildCache(me);},1);}).bind("updateCell",function(e,cell){var config=this.config;var pos=[(cell.parentNode.rowIndex-1),cell.cellIndex];cache.normalized[pos[0]][pos[1]]=config.parsers[pos[1]].format(getElementText(config,cell),cell);}).bind("sorton",function(e,list){$(this).trigger("sortStart");config.sortList=list;var sortList=config.sortList;updateHeaderSortCount(this,sortList);setHeadersCss(this,$headers,sortList,sortCSS);appendToTable(this,multisort(this,sortList,cache));}).bind("appendCache",function(){appendToTable(this,cache);}).bind("applyWidgetId",function(e,id){getWidgetById(id).format(this);}).bind("applyWidgets",function(){applyWidget(this);});if($.metadata&&($(this).metadata()&&$(this).metadata().sortlist)){config.sortList=$(this).metadata().sortlist;}if(config.sortList.length>0){$this.trigger("sorton",[config.sortList]);}applyWidget(this);});};this.addParser=function(parser){var l=parsers.length,a=true;for(var i=0;i<l;i++){if(parsers[i].id.toLowerCase()==parser.id.toLowerCase()){a=false;}}if(a){parsers.push(parser);};};this.addWidget=function(widget){widgets.push(widget);};this.formatFloat=function(s){var i=parseFloat(s);return(isNaN(i))?0:i;};this.formatInt=function(s){var i=parseInt(s);return(isNaN(i))?0:i;};this.isDigit=function(s,config){return/^[-+]?\d*$/.test($.trim(s.replace(/[,.']/g,'')));};this.clearTableBody=function(table){if($.browser.msie){function empty(){while(this.firstChild)this.removeChild(this.firstChild);}empty.apply(table.tBodies[0]);}else{table.tBodies[0].innerHTML="";}};}});$.fn.extend({tablesorter:$.tablesorter.construct});var ts=$.tablesorter;ts.addParser({id:"text",is:function(s){return true;},format:function(s){return $.trim(s.toLocaleLowerCase());},type:"text"});ts.addParser({id:"digit",is:function(s,table){var c=table.config;return $.tablesorter.isDigit(s,c);},format:function(s){return $.tablesorter.formatFloat(s);},type:"numeric"});ts.addParser({id:"currency",is:function(s){return/^[£$€?.]/.test(s);},format:function(s){return $.tablesorter.formatFloat(s.replace(new RegExp(/[£$€]/g),""));},type:"numeric"});ts.addParser({id:"ipAddress",is:function(s){return/^\d{2,3}[\.]\d{2,3}[\.]\d{2,3}[\.]\d{2,3}$/.test(s);},format:function(s){var a=s.split("."),r="",l=a.length;for(var i=0;i<l;i++){var item=a[i];if(item.length==2){r+="0"+item;}else{r+=item;}}return $.tablesorter.formatFloat(r);},type:"numeric"});ts.addParser({id:"url",is:function(s){return/^(https?|ftp|file):\/\/$/.test(s);},format:function(s){return jQuery.trim(s.replace(new RegExp(/(https?|ftp|file):\/\//),''));},type:"text"});ts.addParser({id:"isoDate",is:function(s){return/^\d{4}[\/-]\d{1,2}[\/-]\d{1,2}$/.test(s);},format:function(s){return $.tablesorter.formatFloat((s!="")?new Date(s.replace(new RegExp(/-/g),"/")).getTime():"0");},type:"numeric"});ts.addParser({id:"percent",is:function(s){return/\%$/.test($.trim(s));},format:function(s){return $.tablesorter.formatFloat(s.replace(new RegExp(/%/g),""));},type:"numeric"});ts.addParser({id:"usLongDate",is:function(s){return s.match(new RegExp(/^[A-Za-z]{3,10}\.? [0-9]{1,2}, ([0-9]{4}|'?[0-9]{2}) (([0-2]?[0-9]:[0-5][0-9])|([0-1]?[0-9]:[0-5][0-9]\s(AM|PM)))$/));},format:function(s){return $.tablesorter.formatFloat(new Date(s).getTime());},type:"numeric"});ts.addParser({id:"shortDate",is:function(s){return/\d{1,2}[\/\-]\d{1,2}[\/\-]\d{2,4}/.test(s);},format:function(s,table){var c=table.config;s=s.replace(/\-/g,"/");if(c.dateFormat=="us"){s=s.replace(/(\d{1,2})[\/\-](\d{1,2})[\/\-](\d{4})/,"$3/$1/$2");}else if(c.dateFormat=="uk"){s=s.replace(/(\d{1,2})[\/\-](\d{1,2})[\/\-](\d{4})/,"$3/$2/$1");}else if(c.dateFormat=="dd/mm/yy"||c.dateFormat=="dd-mm-yy"){s=s.replace(/(\d{1,2})[\/\-](\d{1,2})[\/\-](\d{2})/,"$1/$2/$3");}return $.tablesorter.formatFloat(new Date(s).getTime());},type:"numeric"});ts.addParser({id:"time",is:function(s){return/^(([0-2]?[0-9]:[0-5][0-9])|([0-1]?[0-9]:[0-5][0-9]\s(am|pm)))$/.test(s);},format:function(s){return $.tablesorter.formatFloat(new Date("2000/01/01 "+s).getTime());},type:"numeric"});ts.addParser({id:"metadata",is:function(s){return false;},format:function(s,table,cell){var c=table.config,p=(!c.parserMetadataName)?'sortValue':c.parserMetadataName;return $(cell).metadata()[p];},type:"numeric"});ts.addWidget({id:"zebra",format:function(table){if(table.config.debug){var time=new Date();}var $tr,row=-1,odd;$("tr:visible",table.tBodies[0]).each(function(i){$tr=$(this);if(!$tr.hasClass(table.config.cssChildRow))row++;odd=(row%2==0);$tr.removeClass(table.config.widgetZebra.css[odd?0:1]).addClass(table.config.widgetZebra.css[odd?1:0])});if(table.config.debug){$.tablesorter.benchmark("Applying Zebra widget",time);}}});})(jQuery);
\ No newline at end of file
diff --git a/test/clj/backtype/storm/cluster_test.clj b/test/clj/backtype/storm/cluster_test.clj
new file mode 100644
index 0000000..95d9d3d
--- /dev/null
+++ b/test/clj/backtype/storm/cluster_test.clj
@@ -0,0 +1,235 @@
+(ns backtype.storm.cluster-test
+  (:import [java.util Arrays])
+  (:import [backtype.storm.daemon.common Assignment StormBase SupervisorInfo])
+  (:use [clojure test])
+  (:use [backtype.storm cluster config util testing]))
+
+(def ZK-PORT 2181)
+
+(defn mk-config []
+  (merge (read-storm-config)
+         {STORM-ZOOKEEPER-PORT ZK-PORT
+          STORM-ZOOKEEPER-SERVERS ["localhost"]}))
+
+(defn mk-state
+  ([] (mk-distributed-cluster-state (mk-config)))
+  ([cb]
+     (let [ret (mk-state)]
+       (.register ret cb)
+       ret )))
+
+(defn mk-storm-state [] (mk-storm-cluster-state (mk-config)))
+
+(deftest test-basics
+  (with-inprocess-zookeeper ZK-PORT
+    (let [state (mk-state)]
+      (.set-data state "/root" (barr 1 2 3))
+      (is (Arrays/equals (barr 1 2 3) (.get-data state "/root" false)))
+      (is (= nil (.get-data state "/a" false)))
+      (.set-data state "/root/a" (barr 1 2))
+      (.set-data state "/root" (barr 1))
+      (is (Arrays/equals (barr 1) (.get-data state "/root" false)))
+      (is (Arrays/equals (barr 1 2) (.get-data state "/root/a" false)))
+      (.set-data state "/a/b/c/d" (barr 99))
+      (is (Arrays/equals (barr 99) (.get-data state "/a/b/c/d" false)))
+      (.mkdirs state "/lalala")
+      (is (= [] (.get-children state "/lalala" false)))
+      (is (= #{"root" "a" "lalala"} (set (.get-children state "/" false))))
+      (.delete-node state "/a")
+      (is (= #{"root" "lalala"} (set (.get-children state "/" false))))
+      (is (= nil (.get-data state "/a/b/c/d" false)))
+      (.close state)
+      )))
+
+(deftest test-multi-state
+  (with-inprocess-zookeeper ZK-PORT
+    (let [state1 (mk-state)
+          state2 (mk-state)]
+      (.set-data state1 "/root" (barr 1))
+      (is (Arrays/equals (barr 1) (.get-data state1 "/root" false)))
+      (is (Arrays/equals (barr 1) (.get-data state2 "/root" false)))
+      (.delete-node state2 "/root")
+      (is (= nil (.get-data state1 "/root" false)))
+      (is (= nil (.get-data state2 "/root" false)))
+      (.close state1)
+      (.close state2)
+      )))
+
+(deftest test-ephemeral
+  (with-inprocess-zookeeper ZK-PORT
+    (let [state1 (mk-state)
+          state2 (mk-state)
+          state3 (mk-state)]
+      (.set-ephemeral-node state1 "/a" (barr 1))
+      (is (Arrays/equals (barr 1) (.get-data state1 "/a" false)))
+      (is (Arrays/equals (barr 1) (.get-data state2 "/a" false)))
+      (.close state3)
+      (is (Arrays/equals (barr 1) (.get-data state1 "/a" false)))
+      (is (Arrays/equals (barr 1) (.get-data state2 "/a" false)))
+      (.close state1)
+      (is (= nil (.get-data state2 "/a" false)))
+      (.close state1)
+      (.close state2)
+      (.close state3)
+      )))
+
+(defn mk-callback-tester []
+  (let [last (atom nil)
+        cb (fn [type path]
+              (reset! last {:type type :path path}))]
+    [last cb]
+    ))
+
+(defn read-and-reset! [aatom]
+  (let [time (System/currentTimeMillis)]
+  (loop []
+    (if-let [val @aatom]
+      (do
+        (reset! aatom nil)
+        val)
+      (do
+        (when (> (- (System/currentTimeMillis) time) 30000)
+          (throw (RuntimeException. "Waited too long for atom to change state")))
+        (Thread/sleep 10)
+        (recur))
+      ))))
+
+(deftest test-callbacks
+  (with-inprocess-zookeeper ZK-PORT
+    (let [[state1-last-cb state1-cb] (mk-callback-tester)
+          state1 (mk-state state1-cb)
+          [state2-last-cb state2-cb] (mk-callback-tester)
+          state2 (mk-state state2-cb)]
+      (.set-data state1 "/root" (barr 1))
+      (.get-data state2 "/root" true)
+      (is (= nil @state1-last-cb))
+      (is (= nil @state2-last-cb))
+      (.set-data state2 "/root" (barr 2))
+      (is (= {:type :node-data-changed :path "/root"} (read-and-reset! state2-last-cb)))
+      (is (= nil @state1-last-cb))
+
+      (.set-data state2 "/root" (barr 3))
+      (is (= nil @state2-last-cb))
+      (.get-data state2 "/root" true)
+      (.get-data state2 "/root" false)
+      (.delete-node state1 "/root")
+      (is (= {:type :node-deleted :path "/root"} (read-and-reset! state2-last-cb)))
+      (.get-data state2 "/root" true)
+      (.set-ephemeral-node state1 "/root" (barr 1 2 3 4))
+      (is (= {:type :node-created :path "/root"} (read-and-reset! state2-last-cb)))
+
+      (.get-children state1 "/" true)
+      (.set-data state2 "/a" (barr 9))
+      (is (= nil @state2-last-cb))
+      (is (= {:type :node-children-changed :path "/"} (read-and-reset! state1-last-cb)))
+
+      (.get-data state2 "/root" true)
+      (.set-ephemeral-node state1 "/root" (barr 1 2))
+      (is (= {:type :node-data-changed :path "/root"} (read-and-reset! state2-last-cb)))
+
+      (.mkdirs state1 "/ccc")
+      (.get-children state1 "/ccc" true)
+      (.get-data state2 "/ccc/b" true)
+      (.set-data state2 "/ccc/b" (barr 8))
+      (is (= {:type :node-created :path "/ccc/b"} (read-and-reset! state2-last-cb)))
+      (is (= {:type :node-children-changed :path "/ccc"} (read-and-reset! state1-last-cb)))
+
+      (.get-data state2 "/root" true)
+      (.get-data state2 "/root2" true)
+      (.close state1)
+
+      (is (= {:type :node-deleted :path "/root"} (read-and-reset! state2-last-cb)))
+      (.set-data state2 "/root2" (barr 9))
+      (is (= {:type :node-created :path "/root2"} (read-and-reset! state2-last-cb)))
+      (.close state2)
+      )))
+
+
+(deftest test-storm-cluster-state-basics
+  (with-inprocess-zookeeper ZK-PORT
+    (let [state (mk-storm-state)
+          assignment1 (Assignment. "/aaa" {} {1 [2 2002 1]} {})
+          assignment2 (Assignment. "/aaa" {} {1 [2 2002]} {})
+          base1 (StormBase. "/tmp/storm1" 1)
+          base2 (StormBase. "/tmp/storm2" 2)]
+      (is (= [] (.assignments state nil)))
+      (.set-assignment! state "storm1" assignment1)
+      (is (= assignment1 (.assignment-info state "storm1" nil)))
+      (is (= nil (.assignment-info state "storm3" nil)))
+      (.set-assignment! state "storm1" assignment2)
+      (.set-assignment! state "storm3" assignment1)
+      (is (= #{"storm1" "storm3"} (set (.assignments state nil))))
+      (is (= assignment2 (.assignment-info state "storm1" nil)))
+      (is (= assignment1 (.assignment-info state "storm3" nil)))
+      
+      (is (= [] (.active-storms state)))
+      (.activate-storm! state "storm1" base1)
+      (is (= ["storm1"] (.active-storms state)))
+      (is (= base1 (.storm-base state "storm1" nil)))
+      (is (= nil (.storm-base state "storm2" nil)))
+      (.activate-storm! state "storm2" base2)
+      (is (= base1 (.storm-base state "storm1" nil)))
+      (is (= base2 (.storm-base state "storm2" nil)))
+      (is (= #{"storm1" "storm2"} (set (.active-storms state))))
+      (.deactivate-storm! state "storm1")
+      (is (= base2 (.storm-base state "storm2" nil)))
+      (is (= #{"storm2"} (set (.active-storms state))))
+
+
+      ;; TODO add tests for task info and task heartbeat setting and getting
+      (.disconnect state)
+      )))
+
+(defn- validate-errors! [state storm-id task errors-list]
+  (let [errors (.task-errors state storm-id task)]
+    (is (= (count errors) (count errors-list)))
+    (doseq [[error target] (map vector errors errors-list)]
+      (is (.contains (:error error) target))
+      )))
+
+(deftest test-storm-cluster-state-errors
+  (with-inprocess-zookeeper ZK-PORT
+    (with-simulated-time
+      (let [state (mk-storm-state)]
+        (.report-task-error state "a" 1 (RuntimeException.))
+        (validate-errors! state "a" 1 ["RuntimeException"])
+        (advance-time-secs! 2)
+        (.report-task-error state "a" 1 (IllegalArgumentException.))
+        (validate-errors! state "a" 1 ["RuntimeException" "IllegalArgumentException"])
+        (doseq [i (range 10)]
+          (.report-task-error state "a" 2 (RuntimeException.))
+          (advance-time-secs! 2))
+        (validate-errors! state "a" 2 (repeat 10 "RuntimeException"))
+        (doseq [i (range 5)]
+          (.report-task-error state "a" 2 (IllegalArgumentException.))
+          (advance-time-secs! 2))
+        (validate-errors! state "a" 2 (concat (repeat 5 "RuntimeException")
+                                              (repeat 5 "IllegalArgumentException")))
+        (.disconnect state)
+        ))
+    ))
+
+
+(deftest test-supervisor-state
+  (with-inprocess-zookeeper ZK-PORT
+    (let [state1 (mk-storm-state)
+          state2 (mk-storm-state)]
+      (is (= [] (.supervisors state1 nil)))
+      (.supervisor-heartbeat! state2 "2" {:a 1})
+      (.supervisor-heartbeat! state1 "1" {})
+      (is (= {:a 1} (.supervisor-info state1 "2")))
+      (is (= {} (.supervisor-info state1 "1")))
+      (is (= #{"1" "2"} (set (.supervisors state1 nil))))
+      (is (= #{"1" "2"} (set (.supervisors state2 nil))))
+      (.disconnect state2)
+      (is (= #{"1"} (set (.supervisors state1 nil))))
+      (.disconnect state1)
+      (.disconnect state2)
+      )))
+
+(deftest test-storm-state-callbacks
+  ;; TODO finish
+  )
+
+
+
diff --git a/test/clj/backtype/storm/integration_test.clj b/test/clj/backtype/storm/integration_test.clj
new file mode 100644
index 0000000..4e6e3d8
--- /dev/null
+++ b/test/clj/backtype/storm/integration_test.clj
@@ -0,0 +1,330 @@
+(ns backtype.storm.integration-test
+  (:use [clojure test])
+  (:import [backtype.storm.testing TestWordCounter TestWordSpout TestGlobalCount TestAggregatesCounter])
+  (:use [backtype.storm bootstrap testing])
+  (:use [backtype.storm.daemon common])
+  )
+
+(bootstrap)
+
+;; (deftest test-counter
+;;   (with-local-cluster [cluster :supervisors 4]
+;;     (let [state (:storm-cluster-state cluster)
+;;           nimbus (:nimbus cluster)
+;;           topology (thrift/mk-topology
+;;                     {1 (thrift/mk-spout-spec (TestWordSpout. true) :parallelism-hint 3)}
+;;                     {2 (thrift/mk-bolt-spec {1 ["word"]} (TestWordCounter.) :parallelism-hint 4)
+;;                      3 (thrift/mk-bolt-spec {1 :global} (TestGlobalCount.))
+;;                      4 (thrift/mk-bolt-spec {2 :global} (TestAggregatesCounter.))
+;;                      })]
+;;         (submit-local-topology nimbus
+;;                             "counter"
+;;                             {TOPOLOGY-OPTIMIZE false TOPOLOGY-WORKERS 20 TOPOLOGY-MESSAGE-TIMEOUT-SECS 3 TOPOLOGY-DEBUG true}
+;;                             topology)
+;;         (Thread/sleep 10000)
+;;         (.killTopology nimbus "counter")
+;;         (Thread/sleep 10000)
+;;         )))
+
+;; (deftest test-multilang-fy
+;;   (with-local-cluster [cluster :supervisors 4]
+;;     (let [nimbus (:nimbus cluster)
+;;           topology (thrift/mk-topology
+;;                       {1 (thrift/mk-spout-spec (TestWordSpout. false))}
+;;                       {2 (thrift/mk-shell-bolt-spec {1 :shuffle} "fancy" "tester.fy" ["word"] :parallelism-hint 1)}
+;;                       )]
+;;       (submit-local-topology nimbus
+;;                           "test"
+;;                           {TOPOLOGY-OPTIMIZE false TOPOLOGY-WORKERS 20 TOPOLOGY-MESSAGE-TIMEOUT-SECS 3 TOPOLOGY-DEBUG true}
+;;                           topology)
+;;       (Thread/sleep 10000)
+;;       (.killTopology nimbus "test")
+;;       (Thread/sleep 10000)
+;;       )))
+
+;; (deftest test-multilang-rb
+;;   (with-local-cluster [cluster :supervisors 4]
+;;     (let [nimbus (:nimbus cluster)
+;;           topology (thrift/mk-topology
+;;                       {1 (thrift/mk-spout-spec (TestWordSpout. false))}
+;;                       {2 (thrift/mk-shell-bolt-spec {1 :shuffle} "ruby" "tester.rb" ["word"] :parallelism-hint 1)}
+;;                       )]
+;;       (submit-local-topology nimbus
+;;                           "test"
+;;                           {TOPOLOGY-OPTIMIZE false TOPOLOGY-WORKERS 20 TOPOLOGY-MESSAGE-TIMEOUT-SECS 3 TOPOLOGY-DEBUG true}
+;;                           topology)
+;;       (Thread/sleep 10000)
+;;       (.killTopology nimbus "test")
+;;       (Thread/sleep 10000)
+;;       )))
+
+
+(deftest test-multilang-py
+  (with-local-cluster [cluster :supervisors 4]
+    (let [nimbus (:nimbus cluster)
+          topology (thrift/mk-topology
+                      {1 (thrift/mk-spout-spec (TestWordSpout. false))}
+                      {2 (thrift/mk-shell-bolt-spec {1 :shuffle} "python" "tester.py" ["word"] :parallelism-hint 1)}
+                      )]
+      (submit-local-topology nimbus
+                          "test"
+                          {TOPOLOGY-OPTIMIZE false TOPOLOGY-WORKERS 20 TOPOLOGY-MESSAGE-TIMEOUT-SECS 3 TOPOLOGY-DEBUG true}
+                          topology)
+      (Thread/sleep 10000)
+      (.killTopology nimbus "test")
+      (Thread/sleep 10000)
+      )))
+
+
+(deftest test-basic-topology
+  (with-simulated-time-local-cluster [cluster :supervisors 4]
+    (let [topology (thrift/mk-topology
+                    {1 (thrift/mk-spout-spec (TestWordSpout. true) :parallelism-hint 3)}
+                    {2 (thrift/mk-bolt-spec {1 ["word"]} (TestWordCounter.) :parallelism-hint 4)
+                     3 (thrift/mk-bolt-spec {1 :global} (TestGlobalCount.))
+                     4 (thrift/mk-bolt-spec {2 :global} (TestAggregatesCounter.))
+                     })
+          results (complete-topology cluster
+                                     topology
+                                     :mock-sources {1 [["nathan"] ["bob"] ["joey"] ["nathan"]]}
+                                     :storm-conf {TOPOLOGY-DEBUG true})]
+      (is (ms= [["nathan"] ["bob"] ["joey"] ["nathan"]]
+             (read-tuples results 1)))
+      (is (ms= [["nathan" 1] ["nathan" 2] ["bob" 1] ["joey" 1]]
+               (read-tuples results 2)))
+      (is (= [[1] [2] [3] [4]]
+               (read-tuples results 3)))
+      (is (= [[1] [2] [3] [4]]
+             (read-tuples results 4)))
+      )))
+
+(deftest test-shuffle
+  (with-simulated-time-local-cluster [cluster :supervisors 4]
+    (let [topology (thrift/mk-topology
+                    {1 (thrift/mk-spout-spec (TestWordSpout. true) :parallelism-hint 4)}
+                    {2 (thrift/mk-bolt-spec {1 :shuffle} (TestGlobalCount.)
+                                            :parallelism-hint 6)
+                     })
+          results (complete-topology cluster
+                                     topology
+                                     ;; important for test that
+                                     ;; #tuples = multiple of 4 and 6
+                                     :mock-sources {1 [["a"] ["b"]
+                                                       ["a"] ["b"]
+                                                       ["a"] ["b"]
+                                                       ["a"] ["b"]
+                                                       ["a"] ["b"]
+                                                       ["a"] ["b"]
+                                                       ["a"] ["b"]
+                                                       ["a"] ["b"]
+                                                       ["a"] ["b"]
+                                                       ["a"] ["b"]
+                                                       ["a"] ["b"]
+                                                       ["a"] ["b"]
+                                                       ]}
+                                     )]
+      (is (ms= (apply concat (repeat 6 [[1] [2] [3] [4]]))
+               (read-tuples results 2)))
+      )))
+
+(defbolt lalala-bolt1 ["word"] [tuple collector]
+  (let [ret (-> (.getValue tuple 0) (str "lalala"))]
+    (.emit collector tuple [ret])
+    (.ack collector tuple)
+    ))
+
+(defboltfull lalala-bolt2 ["word"]
+  :let [state (atom nil)]
+  :prepare ([conf context collector]
+              (println "prepare")
+              (reset! state "lalala")
+              )
+  :execute ([tuple collector]
+              (let [ret (-> (.getValue tuple 0) (str @state))]
+                (.emit collector tuple [ret])
+                (.ack collector tuple)
+                )))
+
+(defboltfull lalala-bolt3 ["word"]
+  :let [state (atom nil)]
+  :params [prefix]
+  :prepare ([conf context collector]
+              (reset! state (str prefix "lalala"))
+              )
+  :execute ([tuple collector]
+              (let [ret (-> (.getValue tuple 0) (str @state))]
+                (.emit collector tuple [ret])
+                (.ack collector tuple)
+                )))
+
+(deftest test-clojure-bolt
+  (with-simulated-time-local-cluster [cluster :supervisors 4]
+    (let [nimbus (:nimbus cluster)
+          topology (thrift/mk-topology
+                      {1 (thrift/mk-spout-spec (TestWordSpout. false))}
+                      {2 (thrift/mk-bolt-spec {1 :shuffle}
+                                              lalala-bolt1)
+                       3 (thrift/mk-bolt-spec {1 :shuffle}
+                                              lalala-bolt2)
+                       4 (thrift/mk-bolt-spec {1 :shuffle}
+                                              (lalala-bolt3 "_nathan_"))}
+                      )
+          results (complete-topology cluster
+                                     topology
+                                     :mock-sources {1 [["david"]
+                                                       ["adam"]
+                                                       ]}
+                                     )]
+      (is (ms= [["davidlalala"] ["adamlalala"]] (read-tuples results 2)))
+      (is (ms= [["davidlalala"] ["adamlalala"]] (read-tuples results 3)))
+      (is (ms= [["david_nathan_lalala"] ["adam_nathan_lalala"]] (read-tuples results 4)))
+      )))
+
+(defn ack-tracking-feeder [fields]
+  (let [tracker (AckTracker.)]
+    [(doto (feeder-spout fields)
+       (.setAckFailDelegate tracker))
+     (fn [val]
+       (is (= (.getNumAcks tracker) val))
+       (.resetNumAcks tracker)
+       )]
+    ))
+
+(defboltfull branching-bolt ["num"]
+  :params [amt]
+  :execute ([tuple collector]
+              (doseq [i (range amt)]
+                (.emit collector tuple [i]))
+              (.ack collector tuple)
+              ))
+
+(defboltfull agg-bolt ["num"]
+  :let [seen (atom [])]
+  :params [amt]
+  :execute ([tuple collector]
+              (swap! seen conj tuple)
+              (when (= (count @seen) amt)
+                (.emit collector @seen [1])
+                (doseq [s @seen]
+                  (.ack collector s))
+                (reset! seen [])
+                )))
+
+(defbolt ack-bolt {}
+  [tuple collector]
+  (.ack collector tuple))
+
+(defbolt identity-bolt ["num"]
+  [tuple collector]
+  (.emit collector tuple (.getValues tuple))
+  (.ack collector tuple))
+
+(deftest test-acking
+  (with-tracked-cluster [cluster]
+    (let [[feeder1 checker1] (ack-tracking-feeder ["num"])
+          [feeder2 checker2] (ack-tracking-feeder ["num"])
+          [feeder3 checker3] (ack-tracking-feeder ["num"])
+          tracked (mk-tracked-topology
+                   {1 [feeder1]
+                    2 [feeder2]
+                    3 [feeder3]}
+                   {4 [{1 :shuffle} (branching-bolt 2)]
+                    5 [{2 :shuffle} (branching-bolt 4)]
+                    6 [{3 :shuffle} (branching-bolt 1)]
+                    7 [{4 :shuffle
+                        5 :shuffle
+                        6 :shuffle} (agg-bolt 3)]
+                    8 [{7 :shuffle} (branching-bolt 2)]
+                    9 [{8 :shuffle} ack-bolt]}
+                   )]
+      (submit-local-topology (:nimbus cluster)
+                             "test"
+                             {}
+                             (:topology tracked))
+      (.feed feeder1 [1])
+      (tracked-wait tracked 1)
+      (checker1 0)
+      (.feed feeder2 [1])
+      (tracked-wait tracked 1)
+      (checker1 1)
+      (checker2 1)
+      (.feed feeder1 [1])
+      (tracked-wait tracked 1)
+      (checker1 0)
+      (.feed feeder1 [1])
+      (tracked-wait tracked 1)
+      (checker1 1)
+      (.feed feeder3 [1])
+      (tracked-wait tracked 1)
+      (checker1 0)
+      (checker3 0)
+      (.feed feeder2 [1])
+      (tracked-wait tracked 1)
+      (checker1 1)
+      (checker2 1)
+      (checker3 1)
+      
+      )))
+
+(deftest test-ack-branching
+  (with-tracked-cluster [cluster]
+    (let [[feeder checker] (ack-tracking-feeder ["num"])
+          tracked (mk-tracked-topology
+                   {1 [feeder]}
+                   {2 [{1 :shuffle} identity-bolt]
+                    3 [{1 :shuffle} identity-bolt]
+                    4 [{2 :shuffle
+                        3 :shuffle} (agg-bolt 4)]})]
+      (submit-local-topology (:nimbus cluster)
+                             "test"
+                             {}
+                             (:topology tracked))
+      (.feed feeder [1])
+      (tracked-wait tracked 1)
+      (checker 0)
+      (.feed feeder [1])
+      (tracked-wait tracked 1)
+      (checker 2)
+      )))
+
+(defbolt dup-anchor ["num"]
+  [tuple collector]
+  (.emit collector [tuple tuple] [1])
+  (.ack collector tuple))
+
+(deftest test-acking-self-anchor
+  (with-tracked-cluster [cluster]
+    (let [[feeder checker] (ack-tracking-feeder ["num"])
+          tracked (mk-tracked-topology
+                   {1 [feeder]}
+                   {2 [{1 :shuffle} dup-anchor]
+                    3 [{2 :shuffle} ack-bolt]})]
+      (submit-local-topology (:nimbus cluster)
+                             "test"
+                             {}
+                             (:topology tracked))
+      (.feed feeder [1])
+      (tracked-wait tracked 1)
+      (checker 1)
+      (.feed feeder [1])
+      (.feed feeder [1])
+      (.feed feeder [1])
+      (tracked-wait tracked 3)
+      (checker 3)
+      )))
+
+(deftest test-acking-branching-complex
+  ;; test acking with branching in the topology
+  )
+
+
+(deftest test-fields-grouping
+  ;; 1. put a shitload of random tuples through it and test that counts are right
+  ;; 2. test that different spouts with different phints group the same way
+  )
+
+(deftest test-all-grouping
+  )
+
+(deftest test-direct-grouping
+  )
diff --git a/test/clj/backtype/storm/local_state_test.clj b/test/clj/backtype/storm/local_state_test.clj
new file mode 100644
index 0000000..71e7363
--- /dev/null
+++ b/test/clj/backtype/storm/local_state_test.clj
@@ -0,0 +1,28 @@
+(ns backtype.storm.local-state-test
+  (:use [clojure test])
+  (:use [backtype.storm testing])
+  (:import [backtype.storm.utils LocalState]))
+
+(deftest test-local-state
+  (with-local-tmp [dir1 dir2]
+    (let [ls1 (LocalState. dir1)
+          ls2 (LocalState. dir2)]
+      (is (= {} (.snapshot ls1)))
+      (.put ls1 "a" 1)
+      (.put ls1 "b" 2)
+      (is (= {"a" 1 "b" 2} (.snapshot ls1)))
+      (is (= {} (.snapshot ls2)))
+      (is (= 1 (.get ls1 "a")))
+      (is (= nil (.get ls1 "c")))
+      (is (= 2 (.get ls1 "b")))
+      (is (= {"a" 1 "b" 2} (.snapshot (LocalState. dir1))))
+      (.put ls2 "b" 1)
+      (.put ls2 "b" 2)
+      (.put ls2 "b" 3)
+      (.put ls2 "b" 4)
+      (.put ls2 "b" 5)
+      (.put ls2 "b" 6)
+      (.put ls2 "b" 7)
+      (.put ls2 "b" 8)
+      (is (= 8 (.get ls2 "b")))
+      )))
diff --git a/test/clj/backtype/storm/nimbus_test.clj b/test/clj/backtype/storm/nimbus_test.clj
new file mode 100644
index 0000000..8ab844d
--- /dev/null
+++ b/test/clj/backtype/storm/nimbus_test.clj
@@ -0,0 +1,356 @@
+(ns backtype.storm.nimbus-test
+  (:use [clojure test])
+  (:use [clojure.contrib.def :only [defnk]])
+  (:require [backtype.storm.daemon [nimbus :as nimbus]])
+  (:import [backtype.storm.testing TestWordCounter TestWordSpout TestGlobalCount TestAggregatesCounter])
+  (:use [backtype.storm bootstrap testing])
+  (:use [backtype.storm.daemon common])
+  )
+
+(bootstrap)
+
+(defn storm-component-info [state storm-name]
+  (let [storm-id (get-storm-id state storm-name)]
+    (reverse-map (storm-task-info state storm-id))))
+
+(defn storm-num-workers [state storm-name]
+  (let [storm-id (get-storm-id state storm-name)
+        assignment (.assignment-info state storm-id nil)]
+    (count (reverse-map (:task->node+port assignment)))
+    ))
+
+(defn do-task-heartbeat [cluster storm-id task-id]
+  (let [state (:storm-cluster-state cluster)]
+    (.task-heartbeat! state storm-id task-id (TaskHeartbeat. (current-time-secs) 10 {}))
+    ))
+
+(defn task-assignment [cluster storm-id task-id]
+  (let [state (:storm-cluster-state cluster)
+        assignment (.assignment-info state storm-id nil)]
+    ((:task->node+port assignment) task-id)
+    ))
+
+(defn slot-assignments [cluster storm-id]
+  (let [state (:storm-cluster-state cluster)
+        assignment (.assignment-info state storm-id nil)]
+    (reverse-map (:task->node+port assignment))
+    ))
+
+(defn task-start-times [cluster storm-id]
+  (let [state (:storm-cluster-state cluster)
+        assignment (.assignment-info state storm-id nil)]
+    (:task->start-time-secs assignment)
+    ))
+
+(defnk check-consistency [cluster storm-name :assigned? true]
+  (let [state (:storm-cluster-state cluster)
+        storm-id (get-storm-id state storm-name)
+        task-ids (.task-ids state storm-id)
+        assignment (.assignment-info state storm-id nil)
+        task->node+port (:task->node+port assignment)
+        all-nodes (set (map first (vals task->node+port)))]
+    (when assigned?
+      (is (= (set task-ids) (set (keys task->node+port)))))
+    (doseq [[t s] task->node+port]
+      (is (not-nil? s)))
+    (is (= all-nodes (set (keys (:node->host assignment)))))
+    (doseq [[t s] task->node+port]
+      (is (not-nil? ((:task->start-time-secs assignment) t))))
+    ))
+
+(deftest test-assignment
+  (with-local-cluster [cluster :supervisors 4 :ports-per-supervisor 3 :daemon-conf {SUPERVISOR-ENABLE false TOPOLOGY-ACKERS 0}]
+    (let [state (:storm-cluster-state cluster)
+          nimbus (:nimbus cluster)
+          topology (thrift/mk-topology
+                    {1 (thrift/mk-spout-spec (TestPlannerSpout. false) :parallelism-hint 3)}
+                    {2 (thrift/mk-bolt-spec {1 :none} (TestPlannerBolt.) :parallelism-hint 4)
+                     3 (thrift/mk-bolt-spec {2 :none} (TestPlannerBolt.))})
+          topology2 (thrift/mk-topology
+                     {1 (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 12)}
+                     {2 (thrift/mk-bolt-spec {1 :none} (TestPlannerBolt.) :parallelism-hint 6)
+                      3 (thrift/mk-bolt-spec {1 :global} (TestPlannerBolt.) :parallelism-hint 8)
+                      4 (thrift/mk-bolt-spec {1 :global 2 :none} (TestPlannerBolt.) :parallelism-hint 4)}
+                     )
+          _ (submit-local-topology nimbus "mystorm" {TOPOLOGY-OPTIMIZE false TOPOLOGY-WORKERS 4} topology)
+          task-info (storm-component-info state "mystorm")]
+      (check-consistency cluster "mystorm")
+      ;; 3 should be assigned once (if it were optimized, we'd have
+      ;; different topology)
+      (is (= 1 (count (.assignments state nil))))
+      (is (= 1 (count (task-info 1))))
+      (is (= 4 (count (task-info 2))))
+      (is (= 1 (count (task-info 3))))
+      (is (= 4 (storm-num-workers state "mystorm")))
+      (submit-local-topology nimbus "storm2" {TOPOLOGY-OPTIMIZE false TOPOLOGY-WORKERS 20} topology2)
+      (check-consistency cluster "storm2")
+      (is (= 2 (count (.assignments state nil))))
+      (let [task-info (storm-component-info state "storm2")]
+        (is (= 12 (count (task-info 1))))
+        (is (= 6 (count (task-info 2))))
+        (is (= 1 (count (task-info 3))))
+        (is (= 4 (count (task-info 4))))
+        (is (= 8 (storm-num-workers state "storm2")))
+        )
+      )))
+
+(deftest test-over-parallelism-assignment
+  (with-local-cluster [cluster :supervisors 2 :ports-per-supervisor 5 :daemon-conf {SUPERVISOR-ENABLE false TOPOLOGY-ACKERS 0}]
+    (let [state (:storm-cluster-state cluster)
+          nimbus (:nimbus cluster)
+          topology (thrift/mk-topology
+                     {1 (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 21)}
+                     {2 (thrift/mk-bolt-spec {1 :none} (TestPlannerBolt.) :parallelism-hint 9)
+                      3 (thrift/mk-bolt-spec {1 :none} (TestPlannerBolt.) :parallelism-hint 2)
+                      4 (thrift/mk-bolt-spec {1 :none} (TestPlannerBolt.) :parallelism-hint 10)}
+                     )
+          _ (submit-local-topology nimbus "test" {TOPOLOGY-OPTIMIZE false TOPOLOGY-WORKERS 7} topology)
+          task-info (storm-component-info state "test")]
+      (check-consistency cluster "test")
+      (is (= 21 (count (task-info 1))))
+      (is (= 9 (count (task-info 2))))
+      (is (= 2 (count (task-info 3))))
+      (is (= 10 (count (task-info 4))))
+      (is (= 7 (storm-num-workers state "test")))
+    )))
+
+
+
+(deftest test-kill-storm
+  (with-simulated-time-local-cluster [cluster :supervisors 2 :ports-per-supervisor 5
+    :daemon-conf {SUPERVISOR-ENABLE false
+                  NIMBUS-TASK-TIMEOUT-SECS 30
+                  NIMBUS-MONITOR-FREQ-SECS 10
+                  TOPOLOGY-ACKERS 0}]
+    (letlocals
+      (bind conf (:daemon-conf cluster))
+      (bind topology (thrift/mk-topology
+                       {1 (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 14)}
+                       {}
+                       ))
+      (bind state (:storm-cluster-state cluster))
+      (submit-local-topology (:nimbus cluster) "test" {TOPOLOGY-MESSAGE-TIMEOUT-SECS 20} topology)
+      (bind storm-id (get-storm-id state "test"))
+      (advance-cluster-time cluster 5)
+      (is (not-nil? (.storm-base state storm-id nil)))
+      (is (not-nil? (.assignment-info state storm-id nil)))
+      (.killTopology (:nimbus cluster) "test")
+      ;; check that storm is deactivated but alive
+      (is (nil? (.storm-base state storm-id nil)))
+      (is (not-nil? (.assignment-info state storm-id nil)))
+      (advance-cluster-time cluster 18)
+      ;; check that storm is deactivated but alive
+      (is (= 1 (count (.task-storms state))))
+      (is (= 1 (count (.heartbeat-storms state))))
+      (advance-cluster-time cluster 3)
+      (is (nil? (.storm-base state storm-id nil)))
+      (is (nil? (.assignment-info state storm-id nil)))
+      (is (empty? (.task-storms state)))
+      (is (empty? (.heartbeat-storms state)))
+      ;; TODO: check that code on nimbus was cleaned up locally...
+
+      (is (thrown? NotAliveException (.killTopology (:nimbus cluster) "lalala")))
+      (submit-local-topology (:nimbus cluster) "2test" {TOPOLOGY-MESSAGE-TIMEOUT-SECS 10} topology)
+      (is (thrown? AlreadyAliveException (submit-local-topology (:nimbus cluster) "2test" {} topology)))
+      (bind storm-id (get-storm-id state "2test"))
+      (is (not-nil? (.storm-base state storm-id nil)))
+      (.killTopology (:nimbus cluster) "2test")
+      (submit-local-topology (:nimbus cluster) "2test" {} topology)
+      (bind storm-id2 (get-storm-id state "2test"))
+      (is (not= storm-id storm-id2))
+      (is (not-nil? (.storm-base state storm-id2 nil)))
+      (is (nil? (.storm-base state storm-id nil)))
+      (is (not-nil? (.assignment-info state storm-id nil)))
+      (is (= 2 (count (.task-storms state))))
+      (is (= 2 (count (.heartbeat-storms state))))
+      (advance-cluster-time cluster 11)
+      (is (nil? (.assignment-info state storm-id nil)))
+      (is (not-nil? (.storm-base state storm-id2 nil)))
+      (is (not-nil? (.assignment-info state storm-id2 nil)))
+      (is (= 1 (count (.task-storms state))))
+      (is (= 1 (count (.heartbeat-storms state))))
+
+      (submit-local-topology (:nimbus cluster) "test3" {TOPOLOGY-MESSAGE-TIMEOUT-SECS 5} topology)
+      (bind storm-id3 (get-storm-id state "test3"))
+      (advance-cluster-time cluster 1)
+      (.deactivate-storm! state storm-id3)
+      (is (nil? (.storm-base state storm-id3 nil)))
+      (is (not-nil? (.assignment-info state storm-id3 nil)))
+      (is (= 2 (count (.task-storms state))))
+      (is (= 2 (count (.heartbeat-storms state))))
+
+      (advance-cluster-time cluster (+ 5 1 (conf NIMBUS-MONITOR-FREQ-SECS)))
+      (is (nil? (.assignment-info state storm-id3 nil)))
+      (is (= 1 (count (.task-storms state))))
+      (is (= 1 (count (.heartbeat-storms state))))
+
+      ;; test that it doesn't clean up heartbeats until all tasks have timed out
+      (submit-local-topology (:nimbus cluster) "test3" {TOPOLOGY-MESSAGE-TIMEOUT-SECS 5} topology)
+      (bind storm-id3 (get-storm-id state "test3"))
+      (bind task-id (first (.task-ids state storm-id3)))
+      (do-task-heartbeat cluster storm-id task-id)
+      (.killTopology (:nimbus cluster) "test3")
+      (advance-cluster-time cluster 6)
+      (is (= 1 (count (.task-storms state))))
+      (is (= 2 (count (.heartbeat-storms state))))
+      (advance-cluster-time cluster 10)
+      (is (= 2 (count (.heartbeat-storms state))))
+      (advance-cluster-time cluster 30)
+      (is (= 1 (count (.heartbeat-storms state))))
+
+      )))
+
+(deftest test-reassignment
+  (with-simulated-time-local-cluster [cluster :supervisors 2 :ports-per-supervisor 5
+    :daemon-conf {SUPERVISOR-ENABLE false
+                  NIMBUS-TASK-LAUNCH-SECS 60
+                  NIMBUS-TASK-TIMEOUT-SECS 20
+                  NIMBUS-MONITOR-FREQ-SECS 10
+                  NIMBUS-SUPERVISOR-TIMEOUT-SECS 100
+                  TOPOLOGY-ACKERS 0}]
+    (letlocals
+      (bind conf (:daemon-conf cluster))
+      (bind topology (thrift/mk-topology
+                       {1 (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 2)}
+                       {}
+                       ))
+      (bind state (:storm-cluster-state cluster))
+      (submit-local-topology (:nimbus cluster) "test" {TOPOLOGY-WORKERS 2} topology)
+      (check-consistency cluster "test")
+      (bind storm-id (get-storm-id state "test"))
+      (bind [task-id1 task-id2]  (.task-ids state storm-id))
+      (bind ass1 (task-assignment cluster storm-id task-id1))
+      (bind ass2 (task-assignment cluster storm-id task-id2))
+
+      (advance-cluster-time cluster 59)
+      (do-task-heartbeat cluster storm-id task-id1)
+      (do-task-heartbeat cluster storm-id task-id2)
+
+      (advance-cluster-time cluster 13)
+      (is (= ass1 (task-assignment cluster storm-id task-id1)))
+      (is (= ass2 (task-assignment cluster storm-id task-id2)))
+      (do-task-heartbeat cluster storm-id task-id1)
+
+      (advance-cluster-time cluster 11)
+      (do-task-heartbeat cluster storm-id task-id1)
+      (is (= ass1 (task-assignment cluster storm-id task-id1)))
+      (check-consistency cluster "test")
+
+      ; have to wait an extra 10 seconds because nimbus may not
+      ; resynchronize its heartbeat time till monitor-time secs after
+      (advance-cluster-time cluster 11)
+      (do-task-heartbeat cluster storm-id task-id1)
+      (is (= ass1 (task-assignment cluster storm-id task-id1)))
+      (check-consistency cluster "test")
+      
+      (advance-cluster-time cluster 11)
+      (is (= ass1 (task-assignment cluster storm-id task-id1)))
+      (is (not= ass2 (task-assignment cluster storm-id task-id2)))
+      (bind ass2 (task-assignment cluster storm-id task-id2))
+      (check-consistency cluster "test")
+
+      (advance-cluster-time cluster 31)
+      (is (not= ass1 (task-assignment cluster storm-id task-id1)))
+      (is (= ass2 (task-assignment cluster storm-id task-id2)))  ; tests launch timeout
+      (check-consistency cluster "test")
+
+
+      (bind ass1 (task-assignment cluster storm-id task-id1))
+      (bind active-supervisor (first ass2))
+      (kill-supervisor cluster active-supervisor)
+
+      (doseq [i (range 12)]
+        (do-task-heartbeat cluster storm-id task-id1)
+        (do-task-heartbeat cluster storm-id task-id2)
+        (advance-cluster-time cluster 10)
+        )
+      ;; tests that it doesn't reassign tasks if they're heartbeating even if supervisor times out
+      (is (= ass1 (task-assignment cluster storm-id task-id1)))
+      (is (= ass2 (task-assignment cluster storm-id task-id2)))
+      (check-consistency cluster "test")
+
+      (advance-cluster-time cluster 30)
+
+      (bind ass1 (task-assignment cluster storm-id task-id1))
+      (bind ass2 (task-assignment cluster storm-id task-id2))
+      (is (not-nil? ass1))
+      (is (not-nil? ass2))
+      (is (not= active-supervisor (first (task-assignment cluster storm-id task-id2))))
+      (is (not= active-supervisor (first (task-assignment cluster storm-id task-id1))))
+      (check-consistency cluster "test")
+
+      (doseq [supervisor-id (.supervisors state nil)]
+        (kill-supervisor cluster supervisor-id))
+
+      (advance-cluster-time cluster 90)
+      (bind ass1 (task-assignment cluster storm-id task-id1))
+      (bind ass2 (task-assignment cluster storm-id task-id2))
+      (is (nil? ass1))
+      (is (nil? ass2))
+      (check-consistency cluster "test" :assigned? false)
+
+      (add-supervisor cluster)
+      (advance-cluster-time cluster 11)
+      (check-consistency cluster "test")
+      )))
+
+(defn check-distribution [slot-tasks distribution]
+  (let [dist (multi-set (map count (vals slot-tasks)))]
+    (is (= dist (multi-set distribution)))
+    ))
+
+(deftest test-reassign-squeezed-topology
+  (with-simulated-time-local-cluster [cluster :supervisors 1 :ports-per-supervisor 1
+    :daemon-conf {SUPERVISOR-ENABLE false
+                  NIMBUS-TASK-LAUNCH-SECS 60
+                  NIMBUS-TASK-TIMEOUT-SECS 20
+                  NIMBUS-MONITOR-FREQ-SECS 10
+                  TOPOLOGY-ACKERS 0}]
+    (letlocals
+      (bind topology (thrift/mk-topology
+                        {1 (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 9)}
+                        {}))
+      (bind state (:storm-cluster-state cluster))
+      (submit-local-topology (:nimbus cluster) "test" {TOPOLOGY-WORKERS 4} topology)  ; distribution should be 2, 2, 2, 3 ideally
+      (bind storm-id (get-storm-id state "test"))
+      (bind slot-tasks (slot-assignments cluster storm-id))
+      (check-distribution (slot-assignments cluster storm-id) [9])
+      (check-consistency cluster "test")
+
+      (add-supervisor cluster :ports 2)
+      (advance-cluster-time cluster 11)
+      (bind slot-tasks (slot-assignments cluster storm-id))
+      (bind task->start (task-start-times cluster storm-id))
+      (check-distribution slot-tasks [3 3 3])
+      (check-consistency cluster "test")
+
+      (add-supervisor cluster :ports 8)
+      ;; this actually works for any time > 0, since zookeeper fires an event causing immediate reassignment
+      ;; doesn't work for time = 0 because it's not waiting for cluster yet, so test might happen before reassignment finishes
+      (advance-cluster-time cluster 11)
+      (bind slot-tasks2 (slot-assignments cluster storm-id))
+      (bind task->start2 (task-start-times cluster storm-id))
+      (check-distribution slot-tasks2 [2 2 2 3])
+      (check-consistency cluster "test")
+
+      (bind common (first (find-first (fn [[k v]] (= 3 (count v))) slot-tasks2)))
+      (is (not-nil? common))
+      (is (= (slot-tasks2 common) (slot-tasks common)))
+      
+      ;; check that start times are changed for everything but the common one
+      (bind same-tasks (slot-tasks2 common))
+      (bind changed-tasks (apply concat (vals (dissoc slot-tasks2 common))))
+      (doseq [t same-tasks]
+        (is (= (task->start t) (task->start2 t))))
+      (doseq [t changed-tasks]
+        (is (not= (task->start t) (task->start2 t))))
+      )))
+
+(deftest test-no-overlapping-slots
+  ;; test that same node+port never appears across 2 assignments
+  )
+
+(deftest test-stateless
+  ;; test that nimbus can die and restart without any problems
+  )
+
diff --git a/test/clj/backtype/storm/supervisor_test.clj b/test/clj/backtype/storm/supervisor_test.clj
new file mode 100644
index 0000000..10a6a6d
--- /dev/null
+++ b/test/clj/backtype/storm/supervisor_test.clj
@@ -0,0 +1,239 @@
+(ns backtype.storm.supervisor-test
+  (:use [clojure test])
+  (:import [backtype.storm.testing TestWordCounter TestWordSpout TestGlobalCount TestAggregatesCounter])
+  (:use [backtype.storm bootstrap testing])
+  (:use [backtype.storm.daemon common])
+  (:require [backtype.storm.daemon [worker :as worker] [supervisor :as supervisor]])
+  )
+
+(bootstrap)
+
+
+(defn worker-assignment
+  "Return [storm-id taskids]"
+  [cluster supervisor-id port]
+  (let [state (:storm-cluster-state cluster)
+        slot-assigns (for [storm-id (.assignments state nil)]
+                        (let [tasks (-> (.assignment-info state storm-id nil)
+                                        :task->node+port
+                                        reverse-map
+                                        (get [supervisor-id port] ))]
+                          (when tasks [storm-id tasks])
+                          ))
+        ret (find-first not-nil? slot-assigns)]
+    (when-not ret
+      (throw (RuntimeException. "Could not find assignment for worker")))
+    ret
+    ))
+
+(defn heartbeat-worker [supervisor port storm-id task-ids]
+  (let [conf (.get-conf supervisor)]
+    (worker/do-heartbeat conf (find-worker-id conf port) port storm-id task-ids)))
+
+(defn heartbeat-workers [cluster supervisor-id ports]
+  (let [sup (get-supervisor cluster supervisor-id)]
+    (doseq [p ports]
+      (let [[storm-id task-ids] (worker-assignment cluster supervisor-id p)]
+        (heartbeat-worker sup p storm-id task-ids)
+        ))
+    ))
+
+(defn validate-launched-once [launched supervisor->ports storm-id]
+  (let [counts (map count (vals launched))
+        launched-supervisor->ports (apply merge-with concat
+                                     (for [[s p] (keys launched)]
+                                       {s [p]}
+                                       ))]
+    (is (every? (partial = 1) counts))
+    (is (= launched-supervisor->ports supervisor->ports))
+    ))
+
+(deftest launches-assignment
+  (with-simulated-time-local-cluster [cluster :supervisors 0
+    :daemon-conf {NIMBUS-REASSIGN false
+                  SUPERVISOR-WORKER-START-TIMEOUT-SECS 5
+                  SUPERVISOR-WORKER-TIMEOUT-SECS 15
+                  SUPERVISOR-MONITOR-FREQUENCY-SECS 3}]
+    (letlocals
+      (bind topology (thrift/mk-topology
+                       {1 (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 4)}
+                       {}))
+      (bind sup1 (add-supervisor cluster :id "sup1" :ports [1 2 3 4]))
+      (bind changed (capture-changed-workers
+                        (submit-mocked-assignment
+                          (:nimbus cluster)
+                          "test"
+                          {TOPOLOGY-WORKERS 3}
+                          topology
+                          {1 1
+                           2 1
+                           3 1
+                           4 1}
+                          {1 ["sup1" 1]
+                           2 ["sup1" 2]
+                           3 ["sup1" 3]
+                           4 ["sup1" 3]
+                           })
+                        (advance-cluster-time cluster 2)
+                        (heartbeat-workers cluster "sup1" [1 2 3])
+                        (advance-cluster-time cluster 10)))
+      (bind storm-id (get-storm-id (:storm-cluster-state cluster) "test"))
+      (is (empty? (:shutdown changed)))
+      (validate-launched-once (:launched changed) {"sup1" [1 2 3]} storm-id)
+      (bind changed (capture-changed-workers
+                        (doseq [i (range 10)]
+                          (heartbeat-workers cluster "sup1" [1 2 3])
+                          (advance-cluster-time cluster 10))
+                        ))
+      (is (empty? (:shutdown changed)))
+      (is (empty? (:launched changed)))
+      (bind changed (capture-changed-workers
+                      (heartbeat-workers cluster "sup1" [1 2])
+                      (advance-cluster-time cluster 10)
+                      ))
+      (validate-launched-once (:launched changed) {"sup1" [3]} storm-id)
+      (is (= {["sup1" 3] 1} (:shutdown changed)))
+      )))
+
+(deftest test-multiple-active-storms-multiple-supervisors
+  (with-simulated-time-local-cluster [cluster :supervisors 0
+    :daemon-conf {NIMBUS-REASSIGN false
+                  SUPERVISOR-WORKER-START-TIMEOUT-SECS 5
+                  SUPERVISOR-WORKER-TIMEOUT-SECS 15
+                  SUPERVISOR-MONITOR-FREQUENCY-SECS 3}]
+    (letlocals
+      (bind topology (thrift/mk-topology
+                       {1 (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 4)}
+                       {}))
+      (bind topology2 (thrift/mk-topology
+                       {1 (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 3)}
+                       {}))
+      (bind sup1 (add-supervisor cluster :id "sup1" :ports [1 2 3 4]))
+      (bind sup2 (add-supervisor cluster :id "sup2" :ports [1 2]))
+      (bind changed (capture-changed-workers
+                        (submit-mocked-assignment
+                          (:nimbus cluster)
+                          "test"
+                          {TOPOLOGY-WORKERS 3 TOPOLOGY-MESSAGE-TIMEOUT-SECS 40}
+                          topology
+                          {1 1
+                           2 1
+                           3 1
+                           4 1}
+                          {1 ["sup1" 1]
+                           2 ["sup1" 2]
+                           3 ["sup2" 1]
+                           4 ["sup2" 1]
+                           })
+                        (advance-cluster-time cluster 2)
+                        (heartbeat-workers cluster "sup1" [1 2])
+                        (heartbeat-workers cluster "sup2" [1])
+                        ))
+      (bind storm-id (get-storm-id (:storm-cluster-state cluster) "test"))
+      (is (empty? (:shutdown changed)))
+      (validate-launched-once (:launched changed) {"sup1" [1 2] "sup2" [1]} storm-id)
+      (bind changed (capture-changed-workers
+                        (submit-mocked-assignment
+                          (:nimbus cluster)
+                          "test2"
+                          {TOPOLOGY-WORKERS 2}
+                          topology2
+                          {1 1
+                           2 1
+                           3 1}
+                          {1 ["sup1" 3]
+                           2 ["sup1" 3]
+                           3 ["sup2" 2]
+                           })
+                        (advance-cluster-time cluster 2)
+                        (heartbeat-workers cluster "sup1" [3])
+                        (heartbeat-workers cluster "sup2" [2])
+                        ))
+      (bind storm-id2 (get-storm-id (:storm-cluster-state cluster) "test2"))
+      (is (empty? (:shutdown changed)))
+      (validate-launched-once (:launched changed) {"sup1" [3] "sup2" [2]} storm-id2)
+      (bind changed (capture-changed-workers
+        (.killTopology (:nimbus cluster) "test")
+        (doseq [i (range 4)]
+          (advance-cluster-time cluster 8)
+          (heartbeat-workers cluster "sup1" [1 2 3])
+          (heartbeat-workers cluster "sup2" [1 2])
+          )))
+      (is (empty? (:shutdown changed)))
+      (is (empty? (:launched changed)))
+      (bind changed (capture-changed-workers
+        (advance-cluster-time cluster 12)
+        ))
+      (is (empty? (:launched changed)))
+      (is (= {["sup1" 1] 1 ["sup1" 2] 1 ["sup2" 1] 1} (:shutdown changed)))
+      (bind changed (capture-changed-workers
+        (doseq [i (range 10)]
+          (heartbeat-workers cluster "sup1" [3])
+          (heartbeat-workers cluster "sup2" [2])
+          (advance-cluster-time cluster 10)
+          )))
+      (is (empty? (:shutdown changed)))
+      (is (empty? (:launched changed)))
+      ;; TODO check that downloaded code is cleaned up only for the one storm
+      )))
+
+(defn get-heartbeat [cluster supervisor-id]
+  (.supervisor-info (:storm-cluster-state cluster) supervisor-id))
+
+(defn check-heartbeat [cluster supervisor-id within-secs]
+  (let [hb (get-heartbeat cluster supervisor-id)
+        time-secs (:time-secs hb)
+        now (current-time-secs)
+        delta (- now time-secs)]
+    (is (>= delta 0))
+    (is (<= delta within-secs))
+    ))
+
+(deftest heartbeats-to-nimbus
+  (with-simulated-time-local-cluster [cluster :supervisors 0
+    :daemon-conf {SUPERVISOR-WORKER-START-TIMEOUT-SECS 15
+                  SUPERVISOR-HEARTBEAT-FREQUENCY-SECS 3}]
+    (letlocals
+      (bind sup1 (add-supervisor cluster :id "sup" :ports [5 6 7]))
+      (advance-cluster-time cluster 4)
+      (bind hb (get-heartbeat cluster "sup"))
+      (is (= #{5 6 7} (set (:worker-ports hb))))
+      (check-heartbeat cluster "sup" 3)
+      (advance-cluster-time cluster 3)
+      (check-heartbeat cluster "sup" 3)
+      (advance-cluster-time cluster 3)
+      (check-heartbeat cluster "sup" 3)
+      (advance-cluster-time cluster 15)
+      (check-heartbeat cluster "sup" 3)
+      (bind topology (thrift/mk-topology
+                       {1 (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 4)}
+                       {}))
+      ;; prevent them from launching by capturing them
+      (capture-changed-workers (submit-local-topology (:nimbus cluster) "test" {TOPOLOGY-WORKERS 2} topology))
+      (advance-cluster-time cluster 3)
+      (check-heartbeat cluster "sup" 3)
+      (advance-cluster-time cluster 3)
+      (check-heartbeat cluster "sup" 3)
+      (advance-cluster-time cluster 3)
+      (check-heartbeat cluster "sup" 3)
+      (advance-cluster-time cluster 20)
+      (check-heartbeat cluster "sup" 3)
+
+      )))
+
+(deftest test-workers-go-bananas
+  ;; test that multiple workers are started for a port, and test that
+  ;; supervisor shuts down propertly (doesn't shutdown the most
+  ;; recently launched one, checks heartbeats correctly, etc.)
+  )
+
+(deftest downloads-code
+  )
+
+(deftest test-stateless
+  )
+
+(deftest cleans-up-on-unassign
+  ;; TODO just do reassign, and check that cleans up worker states after killing but doesn't get rid of downloaded code
+  )
+
diff --git a/test/clj/backtype/storm/versioned_store_test.clj b/test/clj/backtype/storm/versioned_store_test.clj
new file mode 100644
index 0000000..ed66853c
--- /dev/null
+++ b/test/clj/backtype/storm/versioned_store_test.clj
@@ -0,0 +1,30 @@
+(ns backtype.storm.versioned-store-test
+  (:use [clojure test])
+  (:use [backtype.storm testing])
+  (:import [backtype.storm.utils VersionedStore]))
+
+(defmacro defvstest [name [vs-sym] & body]
+  `(deftest ~name
+    (with-local-tmp [dir#]
+      (let [~vs-sym (VersionedStore. dir#)]
+        ~@body
+        ))))
+
+(defvstest test-empty-version [vs]
+  (let [v (.createVersion vs)]
+    (.succeedVersion vs v)
+    (is (= 1 (count (.getAllVersions vs))))
+    (is (= v (.mostRecentVersionPath vs)))
+    ))
+
+(defvstest test-multiple-versions [vs]
+  (.succeedVersion vs (.createVersion vs))
+  (Thread/sleep 100)
+  (let [v (.createVersion vs)]
+    (.succeedVersion vs v)
+    (is (= 2 (count (.getAllVersions vs))))
+    (is (= v (.mostRecentVersionPath vs)))
+    
+    (.createVersion vs)
+    (is (= v (.mostRecentVersionPath vs)))
+    ))
diff --git a/test/clj/zilch/test/mq.clj b/test/clj/zilch/test/mq.clj
new file mode 100644
index 0000000..78baa80
--- /dev/null
+++ b/test/clj/zilch/test/mq.clj
@@ -0,0 +1,86 @@
+(ns zilch.test.mq
+  (:use clojure.test)
+  (:import [java.util Arrays UUID])
+  (:require [zilch.mq :as mq]))
+
+(defn uuid [] (str (UUID/randomUUID)))
+
+(defn random-msg []
+  (byte-array (map byte (for [i (range (int (rand 100)))]
+    (int (rand 100))
+    ))))
+
+(def url
+     (str "inproc://" (uuid))
+     ;; (str "ipc://" (uuid))
+     ;; (str "tcp://127.0.0.1:" (+ 4000 (Math/round (rand 1000)))))
+     )
+
+(deftest zilch
+  (testing "zilch"
+    (testing "should be able to"
+
+      (testing "push / pull"
+        (mq/with-context context 2
+          (with-open [s0 (-> context
+                             (mq/socket mq/pull)
+                             (mq/bind url))
+                      s1 (-> context
+                             (mq/socket mq/push)
+                             (mq/connect url))]
+            (let [msg (random-msg)
+                  push (future (mq/send s1 msg))
+                  pull (future (mq/recv s0))]
+              (is (Arrays/equals msg @pull))))))
+
+      (testing "pub / sub"
+        (mq/with-context context 2
+          (with-open [s0 (-> context
+                             (mq/socket mq/pub)
+                             (mq/bind url))
+                      s1 (-> context
+                             (mq/socket mq/sub)
+                             (mq/subscribe)
+                             (mq/connect url))]
+            (let [msg (random-msg)
+                  pub (future (mq/send s0 msg))
+                  sub (future (mq/recv s1))]
+              (is (Arrays/equals msg @sub))))))
+
+      (testing "pair / pair"
+        (mq/with-context context 2
+          (with-open [s0 (-> context
+                             (mq/socket mq/pair)
+                             (mq/bind url))
+                      s1 (-> context
+                             (mq/socket mq/pair)
+                             (mq/connect url))]
+            (let [msg0 (random-msg)
+                  pair0 (future (mq/send s0 msg0)
+                                (mq/recv s0))
+                  msg1 (random-msg)
+                  pair1 (future (mq/send s1 msg1)
+                                (mq/recv s1))]
+              (is (Arrays/equals msg1 @pair0))
+              (is (Arrays/equals msg0 @pair1))))))
+
+      (testing "req / rep"
+        (mq/with-context context 2
+          (with-open [s0 (-> context
+                             (mq/socket mq/rep)
+                             (mq/bind url))
+                      s1 (-> context
+                             (mq/socket mq/req)
+                             (mq/connect url))]
+            (let [msg (random-msg)
+                  req (future (mq/send s1 msg)
+                              (mq/recv s1))
+                  rep (future (mq/recv s0)
+                              (mq/send s0 msg))]
+              (is (Arrays/equals msg @req))))))
+
+      (testing "req / xrep")
+
+      (testing "xreq / rep")
+
+      (testing "xreq / xrep"))))
diff --git a/test/multilang/fy/bolt.fy b/test/multilang/fy/bolt.fy
new file mode 100644
index 0000000..cadf9af
--- /dev/null
+++ b/test/multilang/fy/bolt.fy
@@ -0,0 +1,45 @@
+require: "mocks"
+
+class TestBolt : Storm Bolt {
+  def process: tuple {
+    emit: $ [tuple values join: ", "]
+    ack: tuple
+  }
+}
+
+FancySpec describe: Storm Bolt with: {
+  before_each: {
+    Storm Protocol Input clear
+    Storm Protocol Output clear
+    @storm = Storm Protocol new
+    @in = Storm Protocol Input
+    @out = Storm Protocol Output
+  }
+
+  it: "runs as as expected" for: 'run when: {
+    conf = <['some_conf => false]>
+    context = <['some_context => true]>
+    tup1 = <['id => 1, 'comp => 2, 'stream => 3, 'task => 4, 'tuple => [1,2,3,4]]>
+    task_ids_1 = <['task_ids => [1,2,3,4]]> # part of the protocol, random values though
+    tup2 = <['id => 2, 'comp => 3, 'stream => 4, 'task => 5, 'tuple => ["hello", "world"]]>
+    task_ids_2 = <['task_ids => [2,3,4,5]]> # same here
+
+    @in input: [
+      "/tmp/", conf to_json() , context to_json(),
+      # tuples:
+      tup1 to_json(), task_ids_1 to_json(),
+      tup2 to_json(), task_ids_2 to_json()
+    ]
+
+    b = TestBolt new
+    b run
+
+    @out sent select: |m| {
+      m includes?: $ tup1['tuple] join: ", "
+    } size is == 1
+
+    @out sent select: |m| {
+      m includes?: $ tup2['tuple] join: ", "
+    } size is == 1
+  }
+}
\ No newline at end of file
diff --git a/test/multilang/fy/mocks.fy b/test/multilang/fy/mocks.fy
new file mode 100644
index 0000000..2aa8064
--- /dev/null
+++ b/test/multilang/fy/mocks.fy
@@ -0,0 +1,56 @@
+require: "../../../src/multilang/fy/storm"
+
+class MockedIO {
+  def initialize {
+    @out = []
+    @in = []
+  }
+
+  def print: string {
+    @out << (string to_s)
+  }
+
+  def println: string {
+    @out << (string ++ "\n")
+  }
+
+  def input: input {
+    input each: |i| {
+      @in << (i ++ "\n")
+      @in << "end\n"
+    }
+  }
+
+  def readline {
+    if: (@in empty?) then: {
+      "No input left" raise!
+    }
+    @in shift
+  }
+
+  def receive_msg: msg {
+    @in << (msg ++ "\n")
+    @in << "end\n"
+  }
+
+  def clear {
+    @in = []
+    @out = []
+  }
+
+  def flush {
+  }
+
+  def received {
+    @in
+  }
+
+  def sent {
+    @out
+  }
+}
+
+class Storm Protocol {
+  Input = MockedIO new
+  Output = MockedIO new
+}
\ No newline at end of file
diff --git a/test/multilang/fy/protocol.fy b/test/multilang/fy/protocol.fy
new file mode 100644
index 0000000..2d5b281
--- /dev/null
+++ b/test/multilang/fy/protocol.fy
@@ -0,0 +1,65 @@
+require: "mocks"
+
+FancySpec describe: Storm Protocol with: {
+  before_each: {
+    Storm Protocol Input clear
+    Storm Protocol Output clear
+    @storm = Storm Protocol new
+    @in = Storm Protocol Input
+    @out = Storm Protocol Output
+    @tuple = Storm Tuple new: 1 component: 2 stream: 3 task: 4 values: [1,2,3,4]
+  }
+
+  it: "reads a string message correctly" for: 'read_string_message when: {
+    @in receive_msg: "/tmp/"
+    @storm read_string_message is == "/tmp/"
+  }
+
+  it: "reads a json message correctly" for: 'read_message when: {
+    @in receive_msg: "{\"foo\":123, \"bar\":\"foobar\", \"tuple\":[1,2,\"cool\"]}"
+    msg = @storm read_message
+    msg is == <["foo" => 123, "bar" => "foobar", "tuple" => [1,2,"cool"]]>
+  }
+
+  it: "sends a message correctly" for: 'send: when: {
+    msg = <['hello => "world", 'testing => 42]>
+    @storm send: msg
+    @out sent is == ["#{msg to_json()}\n", "end\n"]
+  }
+
+  it: "sends the pid to the parent process" for: 'send_pid: when: {
+    @storm send_pid: "/tmp/"
+    pid = Process pid()
+    @out sent is == ["#{pid}\n"]
+  }
+
+  it: "syncs with the parent process" for: 'sync when: {
+    @storm sync
+    @out sent is == ["sync\n"]
+  }
+
+  it: "emits a tuple to storm" for: 'emit_tuple:stream:anchors:direct: when: {
+    tuple_values = ["hello", "world"]
+    @storm emit_tuple: tuple_values
+    emit_msg = JSON parse(@out sent[-2]) # last one is "end"
+    emit_msg is == <["command" => "emit", "anchors" => [], "tuple" => tuple_values]>
+  }
+
+  it: "acks a tuple" for: 'ack: when: {
+    @storm ack: @tuple
+    ack_msg = JSON parse(@out sent[-2])
+    ack_msg is == <["command" => "ack", "id" => @tuple id]>
+  }
+
+  it: "fails a tuple" for: 'fail: when: {
+    @storm fail: @tuple
+    fail_msg = JSON parse(@out sent[-2])
+    fail_msg is == <["command" => "fail", "id" => @tuple id]>
+  }
+
+  it: "logs a message" for: 'log: when: {
+    @storm log: "log test"
+    log_msg = JSON parse(@out sent[-2])
+    log_msg is == <["command" => "log", "msg" => "log test"]>
+  }
+}
\ No newline at end of file