+
+Eclipse Public License - Version 1.0
+
+
+
+
+
+
Copyright (c) Nathan Marz. All rights reserved.
+
Eclipse Public License - v 1.0
+
+
THE ACCOMPANYING PROGRAM IS PROVIDED UNDER THE TERMS OF THIS ECLIPSE
+PUBLIC LICENSE ("AGREEMENT"). ANY USE, REPRODUCTION OR
+DISTRIBUTION OF THE PROGRAM CONSTITUTES RECIPIENT'S ACCEPTANCE OF THIS
+AGREEMENT.
+
+
1. DEFINITIONS
+
+
"Contribution" means:
+
+
a) in the case of the initial Contributor, the initial
+code and documentation distributed under this Agreement, and
+
b) in the case of each subsequent Contributor:
+
i) changes to the Program, and
+
ii) additions to the Program;
+
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.
+
+
"Contributor" means any person or entity that distributes
+the Program.
+
+
"Licensed Patents" 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.
+
+
"Program" means the Contributions distributed in accordance
+with this Agreement.
+
+
"Recipient" means anyone who receives the Program under
+this Agreement, including all Contributors.
+
+
2. GRANT OF RIGHTS
+
+
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.
+
+
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.
+
+
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.
+
+
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.
+
+
3. REQUIREMENTS
+
+
A Contributor may choose to distribute the Program in object code
+form under its own license agreement, provided that:
+
+
a) it complies with the terms and conditions of this
+Agreement; and
+
+
b) its license agreement:
+
+
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;
+
+
ii) effectively excludes on behalf of all Contributors
+all liability for damages, including direct, indirect, special,
+incidental and consequential damages, such as lost profits;
+
+
iii) states that any provisions which differ from this
+Agreement are offered by that Contributor alone and not by any other
+party; and
+
+
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.
+
+
When the Program is made available in source code form:
+
+
a) it must be made available under this Agreement; and
+
+
b) a copy of this Agreement must be included with each
+copy of the Program.
+
+
Contributors may not remove or alter any copyright notices contained
+within the Program.
+
+
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.
+
+
4. COMMERCIAL DISTRIBUTION
+
+
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
+("Commercial Contributor") hereby agrees to defend and
+indemnify every other Contributor ("Indemnified Contributor")
+against any losses, damages and costs (collectively "Losses")
+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.
+
+
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.
+
+
5. NO WARRANTY
+
+
EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, THE PROGRAM IS
+PROVIDED ON AN "AS IS" 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.
+
+
6. DISCLAIMER OF LIABILITY
+
+
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.
+
+
7. GENERAL
+
+
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.
+
+
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.
+
+
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.
+
+
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.
+
+
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.
+
+
+
+
diff --git a/README.markdown b/README.markdown
new file mode 100644
index 000000000..372054a3e
--- /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 000000000..23c883766
--- /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 000000000..7b5dcb261
--- /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 000000000..b7a871b4a
--- /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 000000000..8f7905be2
--- /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 000000000..a9e5c2672
--- /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 000000000..d222a3852
--- /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 000000000..f1c14b46b
--- /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 000000000..2401e028d
--- /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 000000000..8cd69bd83
--- /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 000000000..5650086b3
--- /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 000000000..6acb075d1
--- /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 000000000..cac2b4597
--- /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 000000000..390d2aa40
--- /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 000000000..09482e706
--- /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 000000000..113a641d9
--- /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 000000000..a0d0a1105
--- /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 000000000..4a3a7594a
--- /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 000000000..ba43a14d8
--- /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 000000000..06de2b92a
--- /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 000000000..17dfded78
--- /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 000000000..5bd4392ba
--- /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 000000000..11decf35f
--- /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 000000000..f6cdc17d3
--- /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 000000000..6f15cf57f
--- /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 000000000..08fb12e5d
--- /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 000000000..3c8eb9448
--- /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 000000000..708e6b014
--- /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 000000000..c579c59e0
--- /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 000000000..42058f3fd
--- /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 000000000..14bbd76ff
--- /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 000000000..ce30ce933
--- /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 000000000..2c9248a22
--- /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 000000000..bf4998869
--- /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 000000000..cb582a972
--- /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 000000000..8a74131f3
--- /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 000000000..6c0240bf8
--- /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 000000000..e5678657c
--- /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 000000000..d354fb18c
--- /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 000000000..5e7311133
--- /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 000000000..96db018a4
--- /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 000000000..bad0429ad
--- /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 000000000..b38007c50
--- /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 000000000..a59a93c47
--- /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 000000000..593bf3a4e
--- /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 000000000..a53e7932e
--- /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.
+ *
+ *
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.
+ */
+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.
+ *
+ *
A separate timeout exists for launch because there can be quite a bit of overhead
+ * to launching new JVM's and configuring them.
+ */
+ 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.
+ *
+ *
If this is set to 0, then Storm will immediately ack tuples as soon
+ * as they come off the spout, effectively disabling reliability.
+ */
+ 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 000000000..acd6faba8
--- /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 000000000..3e305aad9
--- /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 000000000..f31d04abe
--- /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 000000000..5dd6cf583
--- /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 _fields;
+ String _namespace;
+ String _fnName;
+ List