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

[11/31] storm git commit: STORM-1617: Release Specific Documentation 0.9.x

STORM-1617: Release Specific Documentation 0.9.x

Conflicts:
	.gitignore
	docs/README.md
	docs/_config.yml
	docs/_includes/footer.html
	docs/_includes/head.html
	docs/_includes/header.html
	docs/_layouts/about.html
	docs/_layouts/default.html
	docs/_layouts/documentation.html
	docs/_layouts/page.html
	docs/_layouts/post.html
	docs/assets/css/bootstrap.css
	docs/assets/css/bootstrap.css.map
	docs/assets/js/bootstrap.min.js
	docs/images/logos/alibaba.jpg
	docs/images/logos/groupon.jpg
	docs/images/logos/parc.png
	docs/images/logos/webmd.jpg
	docs/images/topology.png

Conflicts:
	.gitignore
	docs/README.md
	docs/STORM-UI-REST-API.md
	docs/_config.yml
	docs/_includes/footer.html
	docs/_includes/head.html
	docs/_includes/header.html
	docs/_layouts/about.html
	docs/_layouts/default.html
	docs/_layouts/documentation.html
	docs/_layouts/page.html
	docs/_layouts/post.html
	docs/assets/css/bootstrap.css
	docs/assets/css/bootstrap.css.map
	docs/assets/js/bootstrap.min.js
	docs/images/logos/alibaba.jpg
	docs/images/logos/groupon.jpg
	docs/images/logos/parc.png
	docs/images/logos/webmd.jpg


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

Branch: refs/heads/1.x-branch
Commit: d491c3ff225f6fa7a027bc04b4ebb575f4f128e8
Parents: 885aaec
Author: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Authored: Sat Mar 19 12:15:21 2016 -0500
Committer: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Committed: Sat Mar 19 12:40:40 2016 -0500

----------------------------------------------------------------------
 .gitignore                                      |    1 +
 docs/Acking-framework-implementation.md         |   36 +
 docs/Clojure-DSL.md                             |  264 +
 docs/Command-line-client.md                     |  100 +
 docs/Common-patterns.md                         |   86 +
 docs/Concepts.md                                |  115 +
 docs/Configuration.md                           |   29 +
 docs/Contributing-to-Storm.md                   |   31 +
 docs/Creating-a-new-Storm-project.md            |   25 +
 docs/DSLs-and-multilang-adapters.md             |    9 +
 ...Defining-a-non-jvm-language-dsl-for-storm.md |   36 +
 docs/Distributed-RPC.md                         |  197 +
 docs/Documentation.md                           |   50 +
 docs/FAQ.md                                     |  121 +
 docs/Fault-tolerance.md                         |   28 +
 docs/Guaranteeing-message-processing.md         |  179 +
 docs/Hooks.md                                   |    7 +
 docs/Implementation-docs.md                     |   18 +
 docs/Installing-native-dependencies.md          |   38 +
 docs/Kestrel-and-Storm.md                       |  198 +
 docs/Lifecycle-of-a-topology.md                 |   80 +
 docs/Local-mode.md                              |   27 +
 docs/Maven.md                                   |   56 +
 docs/Message-passing-implementation.md          |   28 +
 docs/Metrics.md                                 |   34 +
 docs/Multilang-protocol.md                      |  221 +
 docs/Powered-By.md                              | 1028 +++
 docs/Project-ideas.md                           |    6 +
 docs/README.md                                  |   61 +
 docs/Rationale.md                               |   31 +
 ...unning-topologies-on-a-production-cluster.md |   75 +
 docs/SECURITY.md                                |   79 +
 docs/STORM-UI-REST-API.md                       |  678 ++
 docs/Serialization-(prior-to-0.6.0).md          |   50 +
 docs/Serialization.md                           |   60 +
 docs/Serializers.md                             |    4 +
 docs/Setting-up-a-Storm-cluster.md              |   83 +
 docs/Setting-up-a-Storm-project-in-Eclipse.md   |    1 +
 docs/Setting-up-development-environment.md      |   39 +
 docs/Spout-implementations.md                   |    8 +
 ...guage-protocol-(versions-0.7.0-and-below).md |  122 +
 docs/Structure-of-the-codebase.md               |  140 +
 docs/Support-for-non-java-languages.md          |    7 +
 docs/Transactional-topologies.md                |  359 +
 docs/Trident-API-Overview.md                    |  311 +
 docs/Trident-spouts.md                          |   42 +
 docs/Trident-state.md                           |  330 +
 docs/Trident-tutorial.md                        |  253 +
 docs/Troubleshooting.md                         |  144 +
 docs/Tutorial.md                                |  310 +
 ...nding-the-parallelism-of-a-Storm-topology.md |  121 +
 docs/Using-non-JVM-languages-with-Storm.md      |   52 +
 docs/_config.yml                                |   18 +
 docs/_includes/footer.html                      |   55 +
 docs/_includes/head.html                        |   34 +
 docs/_includes/header.html                      |   59 +
 docs/_layouts/about.html                        |   43 +
 docs/_layouts/default.html                      |   18 +
 docs/_layouts/documentation.html                |    9 +
 docs/_layouts/page.html                         |    5 +
 docs/_layouts/post.html                         |   61 +
 docs/_plugins/releases.rb                       |   84 +
 docs/assets/css/bootstrap.css                   | 6800 ++++++++++++++++++
 docs/assets/css/bootstrap.css.map               |    1 +
 docs/assets/css/font-awesome.min.css            |    4 +
 docs/assets/css/main.scss                       |   48 +
 docs/assets/css/owl.carousel.css                |   71 +
 docs/assets/css/owl.theme.css                   |   79 +
 docs/assets/css/style.css                       |  503 ++
 docs/assets/js/bootstrap.min.js                 |    7 +
 docs/assets/js/jquery.min.js                    |    6 +
 docs/assets/js/owl.carousel.min.js              |   47 +
 docs/assets/js/storm.js                         |   67 +
 docs/css/style.css                              |  553 ++
 docs/favicon.ico                                |  Bin 0 -> 1150 bytes
 docs/images/ack_tree.png                        |  Bin 0 -> 31463 bytes
 docs/images/batched-stream.png                  |  Bin 0 -> 66336 bytes
 docs/images/drpc-workflow.png                   |  Bin 0 -> 66199 bytes
 docs/images/eclipse-project-properties.png      |  Bin 0 -> 80810 bytes
 docs/images/example-of-a-running-topology.png   |  Bin 0 -> 81430 bytes
 docs/images/footer-bg.png                       |  Bin 0 -> 138 bytes
 docs/images/grouping.png                        |  Bin 0 -> 39701 bytes
 docs/images/header-bg.png                       |  Bin 0 -> 470 bytes
 docs/images/ld-library-path-eclipse-linux.png   |  Bin 0 -> 114597 bytes
 docs/images/loading.gif                         |  Bin 0 -> 12150 bytes
 docs/images/logo.png                            |  Bin 0 -> 26889 bytes
 docs/images/logos/aeris.jpg                     |  Bin 0 -> 7420 bytes
 docs/images/logos/alibaba.jpg                   |  Bin 0 -> 10317 bytes
 docs/images/logos/bai.jpg                       |  Bin 0 -> 10026 bytes
 docs/images/logos/cerner.jpg                    |  Bin 0 -> 7244 bytes
 docs/images/logos/flipboard.jpg                 |  Bin 0 -> 8318 bytes
 docs/images/logos/fullcontact.jpg               |  Bin 0 -> 6172 bytes
 docs/images/logos/groupon.jpg                   |  Bin 0 -> 9849 bytes
 docs/images/logos/health-market-science.jpg     |  Bin 0 -> 6509 bytes
 docs/images/logos/images.png                    |  Bin 0 -> 7339 bytes
 docs/images/logos/infochimp.jpg                 |  Bin 0 -> 5290 bytes
 docs/images/logos/klout.jpg                     |  Bin 0 -> 7251 bytes
 docs/images/logos/loggly.jpg                    |  Bin 0 -> 9258 bytes
 docs/images/logos/ooyala.jpg                    |  Bin 0 -> 5675 bytes
 docs/images/logos/parc.png                      |  Bin 0 -> 13720 bytes
 docs/images/logos/premise.jpg                   |  Bin 0 -> 5391 bytes
 docs/images/logos/qiy.jpg                       |  Bin 0 -> 7441 bytes
 docs/images/logos/quicklizard.jpg               |  Bin 0 -> 7382 bytes
 docs/images/logos/rocketfuel.jpg                |  Bin 0 -> 10007 bytes
 docs/images/logos/rubicon.jpg                   |  Bin 0 -> 7120 bytes
 docs/images/logos/spider.jpg                    |  Bin 0 -> 6265 bytes
 docs/images/logos/spotify.jpg                   |  Bin 0 -> 6445 bytes
 docs/images/logos/taobao.jpg                    |  Bin 0 -> 16814 bytes
 docs/images/logos/the-weather-channel.jpg       |  Bin 0 -> 13295 bytes
 docs/images/logos/twitter.jpg                   |  Bin 0 -> 7139 bytes
 docs/images/logos/verisign.jpg                  |  Bin 0 -> 5982 bytes
 docs/images/logos/webmd.jpg                     |  Bin 0 -> 8226 bytes
 docs/images/logos/wego.jpg                      |  Bin 0 -> 6836 bytes
 docs/images/logos/yahoo-japan.jpg               |  Bin 0 -> 10350 bytes
 docs/images/logos/yahoo.png                     |  Bin 0 -> 13067 bytes
 docs/images/logos/yelp.jpg                      |  Bin 0 -> 7220 bytes
 ...onships-worker-processes-executors-tasks.png |  Bin 0 -> 54804 bytes
 docs/images/spout-vs-state.png                  |  Bin 0 -> 24804 bytes
 docs/images/storm-cluster.png                   |  Bin 0 -> 34604 bytes
 docs/images/storm-flow.png                      |  Bin 0 -> 59688 bytes
 docs/images/topology-tasks.png                  |  Bin 0 -> 45960 bytes
 docs/images/transactional-batches.png           |  Bin 0 -> 23293 bytes
 docs/images/transactional-commit-flow.png       |  Bin 0 -> 17725 bytes
 docs/images/transactional-design-2.png          |  Bin 0 -> 13537 bytes
 docs/images/transactional-spout-structure.png   |  Bin 0 -> 25067 bytes
 docs/images/trident-to-storm1.png               |  Bin 0 -> 67173 bytes
 docs/images/trident-to-storm2.png               |  Bin 0 -> 68943 bytes
 docs/images/tuple-dag.png                       |  Bin 0 -> 18849 bytes
 docs/images/tuple_tree.png                      |  Bin 0 -> 58186 bytes
 docs/index.md                                   |   69 +
 130 files changed, 15049 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/d491c3ff/.gitignore
----------------------------------------------------------------------
diff --git a/.gitignore b/.gitignore
index 54bd289..6ec109b 100644
--- a/.gitignore
+++ b/.gitignore
@@ -40,3 +40,4 @@ metastore_db
 .classpath
 logs
 build
+/docs/javadocs

http://git-wip-us.apache.org/repos/asf/storm/blob/d491c3ff/docs/Acking-framework-implementation.md
----------------------------------------------------------------------
diff --git a/docs/Acking-framework-implementation.md b/docs/Acking-framework-implementation.md
new file mode 100644
index 0000000..5ca5d93
--- /dev/null
+++ b/docs/Acking-framework-implementation.md
@@ -0,0 +1,36 @@
+---
+layout: documentation
+---
+[Storm's acker](https://github.com/apache/incubator-storm/blob/46c3ba7/storm-core/src/clj/backtype/storm/daemon/acker.clj#L28) tracks completion of each tupletree with a checksum hash: each time a tuple is sent, its value is XORed into the checksum, and each time a tuple is acked its value is XORed in again. If all tuples have been successfully acked, the checksum will be zero (the odds that the checksum will be zero otherwise are vanishingly small).
+
+You can read a bit more about the [reliability mechanism](Guaranteeing-message-processing.html#what-is-storms-reliability-api) elsewhere on the wiki -- this explains the internal details.
+
+### acker `execute()`
+
+The acker is actually a regular bolt, with its  [execute method](https://github.com/apache/incubator-storm/blob/46c3ba7/storm-core/src/clj/backtype/storm/daemon/acker.clj#L36) defined withing `mk-acker-bolt`.  When a new tupletree is born, the spout sends the XORed edge-ids of each tuple recipient, which the acker records in its `pending` ledger. Every time an executor acks a tuple, the acker receives a partial checksum that is the XOR of the tuple's own edge-id (clearing it from the ledger) and the edge-id of each downstream tuple the executor emitted (thus entering them into the ledger).
+
+This is accomplished as follows.
+
+On a tick tuple, just advance pending tupletree checksums towards death and return. Otherwise, update or create the record for this tupletree:
+
+* on init: initialize with the given checksum value, and record the spout's id for later.
+* on ack:  xor the partial checksum into the existing checksum value
+* on fail: just mark it as failed
+
+Next, [put the record](https://github.com/apache/incubator-storm/blob/46c3ba7/storm-core/src/clj/backtype/storm/daemon/acker.clj#L50)),  into the RotatingMap (thus resetting is countdown to expiry) and take action:
+
+* if the total checksum is zero, the tupletree is complete: remove it from the pending collection and notify the spout of success
+* if the tupletree has failed, it is also complete:   remove it from the pending collection and notify the spout of failure
+
+Finally, pass on an ack of our own.
+
+### Pending tuples and the `RotatingMap`
+
+The acker stores pending tuples in a [`RotatingMap`](https://github.com/apache/incubator-storm/blob/master/storm-core/src/jvm/backtype/storm/utils/RotatingMap.java#L19), a simple device used in several places within Storm to efficiently time-expire a process.
+
+The RotatingMap behaves as a HashMap, and offers the same O(1) access guarantees.
+
+Internally, it holds several HashMaps ('buckets') of its own, each holding a cohort of records that will expire at the same time.  Let's call the longest-lived bucket death row, and the most recent the nursery. Whenever a value is `.put()` to the RotatingMap, it is relocated to the nursery -- and removed from any other bucket it might have been in (effectively resetting its death clock).
+
+Whenever its owner calls `.rotate()`, the RotatingMap advances each cohort one step further towards expiration. (Typically, Storm objects call rotate on every receipt of a system tick stream tuple.) If there are any key-value pairs in the former death row bucket, the RotatingMap invokes a callback (given in the constructor) for each key-value pair, letting its owner take appropriate action (eg, failing a tuple.
+

http://git-wip-us.apache.org/repos/asf/storm/blob/d491c3ff/docs/Clojure-DSL.md
----------------------------------------------------------------------
diff --git a/docs/Clojure-DSL.md b/docs/Clojure-DSL.md
new file mode 100644
index 0000000..b3109fa
--- /dev/null
+++ b/docs/Clojure-DSL.md
@@ -0,0 +1,264 @@
+---
+layout: documentation
+---
+Storm comes with a Clojure DSL for defining spouts, bolts, and topologies. The Clojure DSL has access to everything the Java API exposes, so if you're a Clojure user you can code Storm topologies without touching Java at all. The Clojure DSL is defined in the source in the [backtype.storm.clojure](https://github.com/apache/incubator-storm/blob/0.5.3/src/clj/backtype/storm/clojure.clj) namespace.
+
+This page outlines all the pieces of the Clojure DSL, including:
+
+1. Defining topologies
+2. `defbolt`
+3. `defspout`
+4. Running topologies in local mode or on a cluster
+5. Testing topologies
+
+### Defining topologies
+
+To define a topology, use the `topology` function. `topology` takes in two arguments: a map of "spout specs" and a map of "bolt specs". Each spout and bolt spec wires the code for the component into the topology by specifying things like inputs and parallelism.
+
+Let's take a look at an example topology definition [from the storm-starter project](https://github.com/nathanmarz/storm-starter/blob/master/src/clj/storm/starter/clj/word_count.clj):
+
+```clojure
+(topology
+ {"1" (spout-spec sentence-spout)
+  "2" (spout-spec (sentence-spout-parameterized
+                   ["the cat jumped over the door"
+                    "greetings from a faraway land"])
+                   :p 2)}
+ {"3" (bolt-spec {"1" :shuffle "2" :shuffle}
+                 split-sentence
+                 :p 5)
+  "4" (bolt-spec {"3" ["word"]}
+                 word-count
+                 :p 6)})
+```
+
+The maps of spout and bolt specs are maps from the component id to the corresponding spec. The component ids must be unique across the maps. Just like defining topologies in Java, component ids are used when declaring inputs for bolts in the topology.
+
+#### spout-spec
+
+`spout-spec` takes as arguments the spout implementation (an object that implements [IRichSpout](javadocs/backtype/storm/topology/IRichSpout.html)) and optional keyword arguments. The only option that exists currently is the `:p` option, which specifies the parallelism for the spout. If you omit `:p`, the spout will execute as a single task.
+
+#### bolt-spec
+
+`bolt-spec` takes as arguments the input declaration for the bolt, the bolt implementation (an object that implements [IRichBolt](javadocs/backtype/storm/topology/IRichBolt.html)), and optional keyword arguments.
+
+The input declaration is a map from stream ids to stream groupings. A stream id can have one of two forms:
+
+1. `[==component id== ==stream id==]`: Subscribes to a specific stream on a component
+2. `==component id==`: Subscribes to the default stream on a component
+
+A stream grouping can be one of the following:
+
+1. `:shuffle`: subscribes with a shuffle grouping
+2. Vector of field names, like `["id" "name"]`: subscribes with a fields grouping on the specified fields
+3. `:global`: subscribes with a global grouping
+4. `:all`: subscribes with an all grouping
+5. `:direct`: subscribes with a direct grouping
+
+See [Concepts](Concepts.html) for more info on stream groupings. Here's an example input declaration showcasing the various ways to declare inputs:
+
+```clojure
+{["2" "1"] :shuffle
+ "3" ["field1" "field2"]
+ ["4" "2"] :global}
+```
+
+This input declaration subscribes to three streams total. It subscribes to stream "1" on component "2" with a shuffle grouping, subscribes to the default stream on component "3" with a fields grouping on the fields "field1" and "field2", and subscribes to stream "2" on component "4" with a global grouping.
+
+Like `spout-spec`, the only current supported keyword argument for `bolt-spec` is `:p` which specifies the parallelism for the bolt.
+
+#### shell-bolt-spec
+
+`shell-bolt-spec` is used for defining bolts that are implemented in a non-JVM language. It takes as arguments the input declaration, the command line program to run, the name of the file implementing the bolt, an output specification, and then the same keyword arguments that `bolt-spec` accepts.
+
+Here's an example `shell-bolt-spec`:
+
+```clojure
+(shell-bolt-spec {"1" :shuffle "2" ["id"]}
+                 "python"
+                 "mybolt.py"
+                 ["outfield1" "outfield2"]
+                 :p 25)
+```
+
+The syntax of output declarations is described in more detail in the `defbolt` section below. See [Using non JVM languages with Storm](Using-non-JVM-languages-with-Storm.html) for more details on how multilang works within Storm.
+
+### defbolt
+
+`defbolt` is used for defining bolts in Clojure. Bolts have the constraint that they must be serializable, and this is why you can't just reify `IRichBolt` to implement a bolt (closures aren't serializable). `defbolt` works around this restriction and provides a nicer syntax for defining bolts than just implementing a Java interface.
+
+At its fullest expressiveness, `defbolt` supports parameterized bolts and maintaining state in a closure around the bolt implementation. It also provides shortcuts for defining bolts that don't need this extra functionality. The signature for `defbolt` looks like the following:
+
+(defbolt _name_ _output-declaration_ *_option-map_ & _impl_)
+
+Omitting the option map is equivalent to having an option map of `{:prepare false}`.
+
+#### Simple bolts
+
+Let's start with the simplest form of `defbolt`. Here's an example bolt that splits a tuple containing a sentence into a tuple for each word:
+
+```clojure
+(defbolt split-sentence ["word"] [tuple collector]
+  (let [words (.split (.getString tuple 0) " ")]
+    (doseq [w words]
+      (emit-bolt! collector [w] :anchor tuple))
+    (ack! collector tuple)
+    ))
+```
+
+Since the option map is omitted, this is a non-prepared bolt. The DSL simply expects an implementation for the `execute` method of `IRichBolt`. The implementation takes two parameters, the tuple and the `OutputCollector`, and is followed by the body of the `execute` function. The DSL automatically type-hints the parameters for you so you don't need to worry about reflection if you use Java interop.
+
+This implementation binds `split-sentence` to an actual `IRichBolt` object that you can use in topologies, like so:
+
+```clojure
+(bolt-spec {"1" :shuffle}
+           split-sentence
+           :p 5)
+```
+
+
+#### Parameterized bolts
+
+Many times you want to parameterize your bolts with other arguments. For example, let's say you wanted to have a bolt that appends a suffix to every input string it receives, and you want that suffix to be set at runtime. You do this with `defbolt` by including a `:params` option in the option map, like so:
+
+```clojure
+(defbolt suffix-appender ["word"] {:params [suffix]}
+  [tuple collector]
+  (emit-bolt! collector [(str (.getString tuple 0) suffix)] :anchor tuple)
+  )
+```
+
+Unlike the previous example, `suffix-appender` will be bound to a function that returns an `IRichBolt` rather than be an `IRichBolt` object directly. This is caused by specifying `:params` in its option map. So to use `suffix-appender` in a topology, you would do something like:
+
+```clojure
+(bolt-spec {"1" :shuffle}
+           (suffix-appender "-suffix")
+           :p 10)
+```
+
+#### Prepared bolts
+
+To do more complex bolts, such as ones that do joins and streaming aggregations, the bolt needs to store state. You can do this by creating a prepared bolt which is specified by including `{:prepare true}` in the option map. Consider, for example, this bolt that implements word counting:
+
+```clojure
+(defbolt word-count ["word" "count"] {:prepare true}
+  [conf context collector]
+  (let [counts (atom {})]
+    (bolt
+     (execute [tuple]
+       (let [word (.getString tuple 0)]
+         (swap! counts (partial merge-with +) {word 1})
+         (emit-bolt! collector [word (@counts word)] :anchor tuple)
+         (ack! collector tuple)
+         )))))
+```
+
+The implementation for a prepared bolt is a function that takes as input the topology config, `TopologyContext`, and `OutputCollector`, and returns an implementation of the `IBolt` interface. This design allows you to have a closure around the implementation of `execute` and `cleanup`. 
+
+In this example, the word counts are stored in the closure in a map called `counts`. The `bolt` macro is used to create the `IBolt` implementation. The `bolt` macro is a more concise way to implement the interface than reifying, and it automatically type-hints all of the method parameters. This bolt implements the execute method which updates the count in the map and emits the new word count.
+
+Note that the `execute` method in prepared bolts only takes as input the tuple since the `OutputCollector` is already in the closure of the function (for simple bolts the collector is a second parameter to the `execute` function).
+
+Prepared bolts can be parameterized just like simple bolts.
+
+#### Output declarations
+
+The Clojure DSL has a concise syntax for declaring the outputs of a bolt. The most general way to declare the outputs is as a map from stream id a stream spec. For example:
+
+```clojure
+{"1" ["field1" "field2"]
+ "2" (direct-stream ["f1" "f2" "f3"])
+ "3" ["f1"]}
+```
+
+The stream id is a string, while the stream spec is either a vector of fields or a vector of fields wrapped by `direct-stream`. `direct stream` marks the stream as a direct stream (See [Concepts](Concepts.html) and [Direct groupings]() for more details on direct streams).
+
+If the bolt only has one output stream, you can define the default stream of the bolt by using a vector instead of a map for the output declaration. For example:
+
+```clojure
+["word" "count"]
+```
+This declares the output of the bolt as the fields ["word" "count"] on the default stream id.
+
+#### Emitting, acking, and failing
+
+Rather than use the Java methods on `OutputCollector` directly, the DSL provides a nicer set of functions for using `OutputCollector`: `emit-bolt!`, `emit-direct-bolt!`, `ack!`, and `fail!`.
+
+1. `emit-bolt!`: takes as parameters the `OutputCollector`, the values to emit (a Clojure sequence), and keyword arguments for `:anchor` and `:stream`. `:anchor` can be a single tuple or a list of tuples, and `:stream` is the id of the stream to emit to. Omitting the keyword arguments emits an unanchored tuple to the default stream.
+2. `emit-direct-bolt!`: takes as parameters the `OutputCollector`, the task id to send the tuple to, the values to emit, and keyword arguments for `:anchor` and `:stream`. This function can only emit to streams declared as direct streams.
+2. `ack!`: takes as parameters the `OutputCollector` and the tuple to ack.
+3. `fail!`: takes as parameters the `OutputCollector` and the tuple to fail.
+
+See [Guaranteeing message processing](Guaranteeing-message-processing.html) for more info on acking and anchoring.
+
+### defspout
+
+`defspout` is used for defining spouts in Clojure. Like bolts, spouts must be serializable so you can't just reify `IRichSpout` to do spout implementations in Clojure. `defspout` works around this restriction and provides a nicer syntax for defining spouts than just implementing a Java interface.
+
+The signature for `defspout` looks like the following:
+
+(defspout _name_ _output-declaration_ *_option-map_ & _impl_)
+
+If you leave out the option map, it defaults to {:prepare true}. The output declaration for `defspout` has the same syntax as `defbolt`.
+
+Here's an example `defspout` implementation from [storm-starter](https://github.com/nathanmarz/storm-starter/blob/master/src/clj/storm/starter/clj/word_count.clj):
+
+```clojure
+(defspout sentence-spout ["sentence"]
+  [conf context collector]
+  (let [sentences ["a little brown dog"
+                   "the man petted the dog"
+                   "four score and seven years ago"
+                   "an apple a day keeps the doctor away"]]
+    (spout
+     (nextTuple []
+       (Thread/sleep 100)
+       (emit-spout! collector [(rand-nth sentences)])         
+       )
+     (ack [id]
+        ;; You only need to define this method for reliable spouts
+        ;; (such as one that reads off of a queue like Kestrel)
+        ;; This is an unreliable spout, so it does nothing here
+        ))))
+```
+
+The implementation takes in as input the topology config, `TopologyContext`, and `SpoutOutputCollector`. The implementation returns an `ISpout` object. Here, the `nextTuple` function emits a random sentence from `sentences`. 
+
+This spout isn't reliable, so the `ack` and `fail` methods will never be called. A reliable spout will add a message id when emitting tuples, and then `ack` or `fail` will be called when the tuple is completed or failed respectively. See [Guaranteeing message processing](Guaranteeing-message-processing.html) for more info on how reliability works within Storm.
+
+`emit-spout!` takes in as parameters the `SpoutOutputCollector` and the new tuple to be emitted, and accepts as keyword arguments `:stream` and `:id`. `:stream` specifies the stream to emit to, and `:id` specifies a message id for the tuple (used in the `ack` and `fail` callbacks). Omitting these arguments emits an unanchored tuple to the default output stream.
+
+There is also a `emit-direct-spout!` function that emits a tuple to a direct stream and takes an additional argument as the second parameter of the task id to send the tuple to.
+
+Spouts can be parameterized just like bolts, in which case the symbol is bound to a function returning `IRichSpout` instead of the `IRichSpout` itself. You can also declare an unprepared spout which only defines the `nextTuple` method. Here is an example of an unprepared spout that emits random sentences parameterized at runtime:
+
+```clojure
+(defspout sentence-spout-parameterized ["word"] {:params [sentences] :prepare false}
+  [collector]
+  (Thread/sleep 500)
+  (emit-spout! collector [(rand-nth sentences)]))
+```
+
+The following example illustrates how to use this spout in a `spout-spec`:
+
+```clojure
+(spout-spec (sentence-spout-parameterized
+                   ["the cat jumped over the door"
+                    "greetings from a faraway land"])
+            :p 2)
+```
+
+### Running topologies in local mode or on a cluster
+
+That's all there is to the Clojure DSL. To submit topologies in remote mode or local mode, just use the `StormSubmitter` or `LocalCluster` classes just like you would from Java.
+
+To create topology configs, it's easiest to use the `backtype.storm.config` namespace which defines constants for all of the possible configs. The constants are the same as the static constants in the `Config` class, except with dashes instead of underscores. For example, here's a topology config that sets the number of workers to 15 and configures the topology in debug mode:
+
+```clojure
+{TOPOLOGY-DEBUG true
+ TOPOLOGY-WORKERS 15}
+```
+
+### Testing topologies
+
+[This blog post](http://www.pixelmachine.org/2011/12/17/Testing-Storm-Topologies.html) and its [follow-up](http://www.pixelmachine.org/2011/12/21/Testing-Storm-Topologies-Part-2.html) give a good overview of Storm's powerful built-in facilities for testing topologies in Clojure.

http://git-wip-us.apache.org/repos/asf/storm/blob/d491c3ff/docs/Command-line-client.md
----------------------------------------------------------------------
diff --git a/docs/Command-line-client.md b/docs/Command-line-client.md
new file mode 100644
index 0000000..0e645d7
--- /dev/null
+++ b/docs/Command-line-client.md
@@ -0,0 +1,100 @@
+---
+layout: documentation
+---
+This page describes all the commands that are possible with the "storm" command line client. To learn how to set up your "storm" client to talk to a remote cluster, follow the instructions in [Setting up development environment](Setting-up-a-development-environment.html).
+
+These commands are:
+
+1. jar
+1. kill
+1. activate
+1. deactivate
+1. rebalance
+1. repl
+1. classpath
+1. localconfvalue
+1. remoteconfvalue
+1. nimbus
+1. supervisor
+1. ui
+1. drpc
+
+### jar
+
+Syntax: `storm jar topology-jar-path class ...`
+
+Runs the main method of `class` with the specified arguments. The storm jars and configs in `~/.storm` are put on the classpath. The process is configured so that [StormSubmitter](javadocs/backtype/storm/StormSubmitter.html) will upload the jar at `topology-jar-path` when the topology is submitted.
+
+### kill
+
+Syntax: `storm kill topology-name [-w wait-time-secs]`
+
+Kills the topology with the name `topology-name`. Storm will first deactivate the topology's spouts for the duration of the topology's message timeout to allow all messages currently being processed to finish processing. Storm will then shutdown the workers and clean up their state. You can override the length of time Storm waits between deactivation and shutdown with the -w flag.
+
+### activate
+
+Syntax: `storm activate topology-name`
+
+Activates the specified topology's spouts.
+
+### deactivate
+
+Syntax: `storm deactivate topology-name`
+
+Deactivates the specified topology's spouts.
+
+### rebalance
+
+Syntax: `storm rebalance topology-name [-w wait-time-secs]`
+
+Sometimes you may wish to spread out where the workers for a topology are running. For example, let's say you have a 10 node cluster running 4 workers per node, and then let's say you add another 10 nodes to the cluster. You may wish to have Storm spread out the workers for the running topology so that each node runs 2 workers. One way to do this is to kill the topology and resubmit it, but Storm provides a "rebalance" command that provides an easier way to do this. 
+
+Rebalance will first deactivate the topology for the duration of the message timeout (overridable with the -w flag) and then redistribute the workers evenly around the cluster. The topology will then return to its previous state of activation (so a deactivated topology will still be deactivated and an activated topology will go back to being activated).
+
+### repl
+
+Syntax: `storm repl`
+
+Opens up a Clojure REPL with the storm jars and configuration on the classpath. Useful for debugging.
+
+### classpath
+
+Syntax: `storm classpath`
+
+Prints the classpath used by the storm client when running commands.
+
+### localconfvalue
+
+Syntax: `storm localconfvalue conf-name`
+
+Prints out the value for `conf-name` in the local Storm configs. The local Storm configs are the ones in `~/.storm/storm.yaml` merged in with the configs in `defaults.yaml`.
+
+### remoteconfvalue
+
+Syntax: `storm remoteconfvalue conf-name`
+
+Prints out the value for `conf-name` in the cluster's Storm configs. The cluster's Storm configs are the ones in `$STORM-PATH/conf/storm.yaml` merged in with the configs in `defaults.yaml`. This command must be run on a cluster machine.
+
+### nimbus
+
+Syntax: `storm nimbus`
+
+Launches the nimbus daemon. This command should be run under supervision with a tool like [daemontools](http://cr.yp.to/daemontools.html) or [monit](http://mmonit.com/monit/). See [Setting up a Storm cluster](Setting-up-a-Storm-cluster.html) for more information.
+
+### supervisor
+
+Syntax: `storm supervisor`
+
+Launches the supervisor daemon. This command should be run under supervision with a tool like [daemontools](http://cr.yp.to/daemontools.html) or [monit](http://mmonit.com/monit/). See [Setting up a Storm cluster](Setting-up-a-Storm-cluster.html) for more information.
+
+### ui
+
+Syntax: `storm ui`
+
+Launches the UI daemon. The UI provides a web interface for a Storm cluster and shows detailed stats about running topologies. This command should be run under supervision with a tool like [daemontools](http://cr.yp.to/daemontools.html) or [monit](http://mmonit.com/monit/). See [Setting up a Storm cluster](Setting-up-a-Storm-cluster.html) for more information.
+
+### drpc
+
+Syntax: `storm drpc`
+
+Launches a DRPC daemon. This command should be run under supervision with a tool like [daemontools](http://cr.yp.to/daemontools.html) or [monit](http://mmonit.com/monit/). See [Distributed RPC](Distributed-RPC.html) for more information.

http://git-wip-us.apache.org/repos/asf/storm/blob/d491c3ff/docs/Common-patterns.md
----------------------------------------------------------------------
diff --git a/docs/Common-patterns.md b/docs/Common-patterns.md
new file mode 100644
index 0000000..3f8c979
--- /dev/null
+++ b/docs/Common-patterns.md
@@ -0,0 +1,86 @@
+---
+layout: documentation
+---
+
+This page lists a variety of common patterns in Storm topologies.
+
+1. Streaming joins
+2. Batching
+3. BasicBolt
+4. In-memory caching + fields grouping combo
+5. Streaming top N
+6. TimeCacheMap for efficiently keeping a cache of things that have been recently updated
+7. CoordinatedBolt and KeyedFairBolt for Distributed RPC
+
+### Joins
+
+A streaming join combines two or more data streams together based on some common field. Whereas a normal database join has finite input and clear semantics for a join, a streaming join has infinite input and unclear semantics for what a join should be.
+
+The join type you need will vary per application. Some applications join all tuples for two streams over a finite window of time, whereas other applications expect exactly one tuple for each side of the join for each join field. Other applications may do the join completely differently. The common pattern among all these join types is partitioning multiple input streams in the same way. This is easily accomplished in Storm by using a fields grouping on the same fields for many input streams to the joiner bolt. For example:
+
+```java
+builder.setBolt("join", new MyJoiner(), parallelism)
+  .fieldsGrouping("1", new Fields("joinfield1", "joinfield2"))
+  .fieldsGrouping("2", new Fields("joinfield1", "joinfield2"))
+  .fieldsGrouping("3", new Fields("joinfield1", "joinfield2"));
+```
+
+The different streams don't have to have the same field names, of course.
+
+
+### Batching
+
+Oftentimes for efficiency reasons or otherwise, you want to process a group of tuples in batch rather than individually. For example, you may want to batch updates to a database or do a streaming aggregation of some sort.
+
+If you want reliability in your data processing, the right way to do this is to hold on to tuples in an instance variable while the bolt waits to do the batching. Once you do the batch operation, you then ack all the tuples you were holding onto.
+
+If the bolt emits tuples, then you may want to use multi-anchoring to ensure reliability. It all depends on the specific application. See [Guaranteeing message processing](Guaranteeing-message-processing.html) for more details on how reliability works.
+
+### BasicBolt
+Many bolts follow a similar pattern of reading an input tuple, emitting zero or more tuples based on that input tuple, and then acking that input tuple immediately at the end of the execute method. Bolts that match this pattern are things like functions and filters. This is such a common pattern that Storm exposes an interface called [IBasicBolt](javadocs/backtype/storm/topology/IBasicBolt.html) that automates this pattern for you. See [Guaranteeing message processing](Guaranteeing-message-processing.html) for more information.
+
+### In-memory caching + fields grouping combo
+
+It's common to keep caches in-memory in Storm bolts. Caching becomes particularly powerful when you combine it with a fields grouping. For example, suppose you have a bolt that expands short URLs (like bit.ly, t.co, etc.) into long URLs. You can increase performance by keeping an LRU cache of short URL to long URL expansions to avoid doing the same HTTP requests over and over. Suppose component "urls" emits short URLS, and component "expand" expands short URLs into long URLs and keeps a cache internally. Consider the difference between the two following snippets of code:
+
+```java
+builder.setBolt("expand", new ExpandUrl(), parallelism)
+  .shuffleGrouping(1);
+```
+
+```java
+builder.setBolt("expand", new ExpandUrl(), parallelism)
+  .fieldsGrouping("urls", new Fields("url"));
+```
+
+The second approach will have vastly more effective caches, since the same URL will always go to the same task. This avoids having duplication across any of the caches in the tasks and makes it much more likely that a short URL will hit the cache.
+
+### Streaming top N
+
+A common continuous computation done on Storm is a "streaming top N" of some sort. Suppose you have a bolt that emits tuples of the form ["value", "count"] and you want a bolt that emits the top N tuples based on count. The simplest way to do this is to have a bolt that does a global grouping on the stream and maintains a list in memory of the top N items.
+
+This approach obviously doesn't scale to large streams since the entire stream has to go through one task. A better way to do the computation is to do many top N's in parallel across partitions of the stream, and then merge those top N's together to get the global top N. The pattern looks like this:
+
+```java
+builder.setBolt("rank", new RankObjects(), parallellism)
+  .fieldsGrouping("objects", new Fields("value"));
+builder.setBolt("merge", new MergeObjects())
+  .globalGrouping("rank");
+```
+
+This pattern works because of the fields grouping done by the first bolt which gives the partitioning you need for this to be semantically correct. You can see an example of this pattern in storm-starter [here](https://github.com/nathanmarz/storm-starter/blob/master/src/jvm/storm/starter/RollingTopWords.java).
+
+
+### TimeCacheMap for efficiently keeping a cache of things that have been recently updated
+
+You sometimes want to keep a cache in memory of items that have been recently "active" and have items that have been inactive for some time be automatically expires. [TimeCacheMap](javadocs/backtype/storm/utils/TimeCacheMap.html) is an efficient data structure for doing this and provides hooks so you can insert callbacks whenever an item is expired.
+
+### CoordinatedBolt and KeyedFairBolt for Distributed RPC
+
+When building distributed RPC applications on top of Storm, there are two common patterns that are usually needed. These are encapsulated by [CoordinatedBolt](javadocs/backtype/storm/task/CoordinatedBolt.html) and [KeyedFairBolt](javadocs/backtype/storm/task/KeyedFairBolt.html) which are part of the "standard library" that ships with the Storm codebase.
+
+`CoordinatedBolt` wraps the bolt containing your logic and figures out when your bolt has received all the tuples for any given request. It makes heavy use of direct streams to do this.
+
+`KeyedFairBolt` also wraps the bolt containing your logic and makes sure your topology processes multiple DRPC invocations at the same time, instead of doing them serially one at a time.
+
+See [Distributed RPC](Distributed-RPC.html) for more details.

http://git-wip-us.apache.org/repos/asf/storm/blob/d491c3ff/docs/Concepts.md
----------------------------------------------------------------------
diff --git a/docs/Concepts.md b/docs/Concepts.md
new file mode 100644
index 0000000..33779f2
--- /dev/null
+++ b/docs/Concepts.md
@@ -0,0 +1,115 @@
+---
+layout: documentation
+---
+
+This page lists the main concepts of Storm and links to resources where you can find more information. The concepts discussed are:
+
+1. Topologies
+2. Streams
+3. Spouts
+4. Bolts
+5. Stream groupings
+6. Reliability
+7. Tasks
+8. Workers
+
+### Topologies
+
+The logic for a realtime application is packaged into a Storm topology. A Storm topology is analogous to a MapReduce job. One key difference is that a MapReduce job eventually finishes, whereas a topology runs forever (or until you kill it, of course). A topology is a graph of spouts and bolts that are connected with stream groupings. These concepts are described below.
+
+**Resources:**
+
+* [TopologyBuilder](javadocs/backtype/storm/topology/TopologyBuilder.html): use this class to construct topologies in Java
+* [Running topologies on a production cluster](Running-topologies-on-a-production-cluster.html)
+* [Local mode](Local-mode.html): Read this to learn how to develop and test topologies in local mode.
+
+### Streams
+
+The stream is the core abstraction in Storm. A stream is an unbounded sequence of tuples that is processed and created in parallel in a distributed fashion. Streams are defined with a schema that names the fields in the stream's tuples. By default, tuples can contain integers, longs, shorts, bytes, strings, doubles, floats, booleans, and byte arrays. You can also define your own serializers so that custom types can be used natively within tuples.
+
+Every stream is given an id when declared. Since single-stream spouts and bolts are so common, [OutputFieldsDeclarer](javadocs/backtype/storm/topology/OutputFieldsDeclarer.html) has convenience methods for declaring a single stream without specifying an id. In this case, the stream is given the default id of "default".
+
+
+**Resources:**
+
+* [Tuple](javadocs/backtype/storm/tuple/Tuple.html): streams are composed of tuples
+* [OutputFieldsDeclarer](javadocs/backtype/storm/topology/OutputFieldsDeclarer.html): used to declare streams and their schemas
+* [Serialization](Serialization.html): Information about Storm's dynamic typing of tuples and declaring custom serializations
+* [ISerialization](javadocs/backtype/storm/serialization/ISerialization.html): custom serializers must implement this interface
+* [CONFIG.TOPOLOGY_SERIALIZATIONS](javadocs/backtype/storm/Config.html#TOPOLOGY_SERIALIZATIONS): custom serializers can be registered using this configuration
+
+### Spouts
+
+A spout is a source of streams in a topology. Generally spouts will read tuples from an external source and emit them into the topology (e.g. a Kestrel queue or the Twitter API). Spouts can either be __reliable__ or __unreliable__. A reliable spout is capable of replaying a tuple if it failed to be processed by Storm, whereas an unreliable spout forgets about the tuple as soon as it is emitted.
+
+Spouts can emit more than one stream. To do so, declare multiple streams using the `declareStream` method of [OutputFieldsDeclarer](javadocs/backtype/storm/topology/OutputFieldsDeclarer.html) and specify the stream to emit to when using the `emit` method on [SpoutOutputCollector](javadocs/backtype/storm/spout/SpoutOutputCollector.html). 
+
+The main method on spouts is `nextTuple`. `nextTuple` either emits a new tuple into the topology or simply returns if there are no new tuples to emit. It is imperative that `nextTuple` does not block for any spout implementation, because Storm calls all the spout methods on the same thread.
+
+The other main methods on spouts are `ack` and `fail`. These are called when Storm detects that a tuple emitted from the spout either successfully completed through the topology or failed to be completed. `ack` and `fail` are only called for reliable spouts. See [the Javadoc](javadocs/backtype/storm/spout/ISpout.html) for more information.
+
+**Resources:**
+
+* [IRichSpout](javadocs/backtype/storm/topology/IRichSpout.html): this is the interface that spouts must implement. 
+* [Guaranteeing message processing](Guaranteeing-message-processing.html)
+
+### Bolts
+
+All processing in topologies is done in bolts. Bolts can do anything from filtering, functions, aggregations, joins, talking to databases, and more. 
+
+Bolts can do simple stream transformations. Doing complex stream transformations often requires multiple steps and thus multiple bolts. For example, transforming a stream of tweets into a stream of trending images requires at least two steps: a bolt to do a rolling count of retweets for each image, and one or more bolts to stream out the top X images (you can do this particular stream transformation in a more scalable way with three bolts than with two). 
+
+Bolts can emit more than one stream. To do so, declare multiple streams using the `declareStream` method of [OutputFieldsDeclarer](javadocs/backtype/storm/topology/OutputFieldsDeclarer.html) and specify the stream to emit to when using the `emit` method on [OutputCollector](javadocs/backtype/storm/task/OutputCollector.html).
+
+When you declare a bolt's input streams, you always subscribe to specific streams of another component. If you want to subscribe to all the streams of another component, you have to subscribe to each one individually. [InputDeclarer](javadocs/backtype/storm/topology/InputDeclarer.html) has syntactic sugar for subscribing to streams declared on the default stream id. Saying `declarer.shuffleGrouping("1")` subscribes to the default stream on component "1" and is equivalent to `declarer.shuffleGrouping("1", DEFAULT_STREAM_ID)`. 
+
+The main method in bolts is the `execute` method which takes in as input a new tuple. Bolts emit new tuples using the [OutputCollector](javadocs/backtype/storm/task/OutputCollector.html) object. Bolts must call the `ack` method on the `OutputCollector` for every tuple they process so that Storm knows when tuples are completed (and can eventually determine that its safe to ack the original spout tuples). For the common case of processing an input tuple, emitting 0 or more tuples based on that tuple, and then acking the input tuple, Storm provides an [IBasicBolt](javadocs/backtype/storm/topology/IBasicBolt.html) interface which does the acking automatically.
+
+Its perfectly fine to launch new threads in bolts that do processing asynchronously. [OutputCollector](javadocs/backtype/storm/task/OutputCollector.html) is thread-safe and can be called at any time.
+
+**Resources:**
+
+* [IRichBolt](javadocs/backtype/storm/topology/IRichBolt.html): this is general interface for bolts.
+* [IBasicBolt](javadocs/backtype/storm/topology/IBasicBolt.html): this is a convenience interface for defining bolts that do filtering or simple functions.
+* [OutputCollector](javadocs/backtype/storm/task/OutputCollector.html): bolts emit tuples to their output streams using an instance of this class
+* [Guaranteeing message processing](Guaranteeing-message-processing.html)
+
+### Stream groupings
+
+Part of defining a topology is specifying for each bolt which streams it should receive as input. A stream grouping defines how that stream should be partitioned among the bolt's tasks.
+
+There are seven built-in stream groupings in Storm, and you can implement a custom stream grouping by implementing the [CustomStreamGrouping](javadocs/backtype/storm/grouping/CustomStreamGrouping.html) interface:
+
+1. **Shuffle grouping**: Tuples are randomly distributed across the bolt's tasks in a way such that each bolt is guaranteed to get an equal number of tuples.
+2. **Fields grouping**: The stream is partitioned by the fields specified in the grouping. For example, if the stream is grouped by the "user-id" field, tuples with the same "user-id" will always go to the same task, but tuples with different "user-id"'s may go to different tasks.
+3. **All grouping**: The stream is replicated across all the bolt's tasks. Use this grouping with care.
+4. **Global grouping**: The entire stream goes to a single one of the bolt's tasks. Specifically, it goes to the task with the lowest id.
+5. **None grouping**: This grouping specifies that you don't care how the stream is grouped. Currently, none groupings are equivalent to shuffle groupings. Eventually though, Storm will push down bolts with none groupings to execute in the same thread as the bolt or spout they subscribe from (when possible).
+6. **Direct grouping**: This is a special kind of grouping. A stream grouped this way means that the __producer__ of the tuple decides which task of the consumer will receive this tuple. Direct groupings can only be declared on streams that have been declared as direct streams. Tuples emitted to a direct stream must be emitted using one of the [emitDirect](javadocs/backtype/storm/task/OutputCollector.html#emitDirect(int, int, java.util.List) methods. A bolt can get the task ids of its consumers by either using the provided [TopologyContext](javadocs/backtype/storm/task/TopologyContext.html) or by keeping track of the output of the `emit` method in [OutputCollector](javadocs/backtype/storm/task/OutputCollector.html) (which returns the task ids that the tuple was sent to).  
+7. **Local or shuffle grouping**: If the target bolt has one or more tasks in the same worker process, tuples will be shuffled to just those in-process tasks. Otherwise, this acts like a normal shuffle grouping.
+
+**Resources:**
+
+* [TopologyBuilder](javadocs/backtype/storm/topology/TopologyBuilder.html): use this class to define topologies
+* [InputDeclarer](javadocs/backtype/storm/topology/InputDeclarer.html): this object is returned whenever `setBolt` is called on `TopologyBuilder` and is used for declaring a bolt's input streams and how those streams should be grouped
+* [CoordinatedBolt](javadocs/backtype/storm/task/CoordinatedBolt.html): this bolt is useful for distributed RPC topologies and makes heavy use of direct streams and direct groupings
+
+### Reliability
+
+Storm guarantees that every spout tuple will be fully processed by the topology. It does this by tracking the tree of tuples triggered by every spout tuple and determining when that tree of tuples has been successfully completed. Every topology has a "message timeout" associated with it. If Storm fails to detect that a spout tuple has been completed within that timeout, then it fails the tuple and replays it later. 
+
+To take advantage of Storm's reliability capabilities, you must tell Storm when new edges in a tuple tree are being created and tell Storm whenever you've finished processing an individual tuple. These are done using the [OutputCollector](javadocs/backtype/storm/task/OutputCollector.html) object that bolts use to emit tuples. Anchoring is done in the `emit` method, and you declare that you're finished with a tuple using the `ack` method.
+
+This is all explained in much more detail in [Guaranteeing message processing](Guaranteeing-message-processing.html). 
+
+### Tasks
+
+Each spout or bolt executes as many tasks across the cluster. Each task corresponds to one thread of execution, and stream groupings define how to send tuples from one set of tasks to another set of tasks. You set the parallelism for each spout or bolt in the `setSpout` and `setBolt` methods of [TopologyBuilder](javadocs/backtype/storm/topology/TopologyBuilder.html). 
+
+### Workers
+
+Topologies execute across one or more worker processes. Each worker process is a physical JVM and executes a subset of all the tasks for the topology. For example, if the combined parallelism of the topology is 300 and 50 workers are allocated, then each worker will execute 6 tasks (as threads within the worker). Storm tries to spread the tasks evenly across all the workers.
+
+**Resources:**
+
+* [Config.TOPOLOGY_WORKERS](javadocs/backtype/storm/Config.html#TOPOLOGY_WORKERS): this config sets the number of workers to allocate for executing the topology

http://git-wip-us.apache.org/repos/asf/storm/blob/d491c3ff/docs/Configuration.md
----------------------------------------------------------------------
diff --git a/docs/Configuration.md b/docs/Configuration.md
new file mode 100644
index 0000000..8e8ca77
--- /dev/null
+++ b/docs/Configuration.md
@@ -0,0 +1,29 @@
+---
+layout: documentation
+---
+Storm has a variety of configurations for tweaking the behavior of nimbus, supervisors, and running topologies. Some configurations are system configurations and cannot be modified on a topology by topology basis, whereas other configurations can be modified per topology. 
+
+Every configuration has a default value defined in [defaults.yaml](https://github.com/apache/incubator-storm/blob/master/conf/defaults.yaml) in the Storm codebase. You can override these configurations by defining a storm.yaml in the classpath of Nimbus and the supervisors. Finally, you can define a topology-specific configuration that you submit along with your topology when using [StormSubmitter](javadocs/backtype/storm/StormSubmitter.html). However, the topology-specific configuration can only override configs prefixed with "TOPOLOGY".
+
+Storm 0.7.0 and onwards lets you override configuration on a per-bolt/per-spout basis. The only configurations that can be overriden this way are:
+
+1. "topology.debug"
+2. "topology.max.spout.pending"
+3. "topology.max.task.parallelism"
+4. "topology.kryo.register": This works a little bit differently than the other ones, since the serializations will be available to all components in the topology. More details on [Serialization](Serialization.html). 
+
+The Java API lets you specify component specific configurations in two ways:
+
+1. *Internally:* Override `getComponentConfiguration` in any spout or bolt and return the component-specific configuration map.
+2. *Externally:* `setSpout` and `setBolt` in `TopologyBuilder` return an object with methods `addConfiguration` and `addConfigurations` that can be used to override the configurations for the component.
+
+The preference order for configuration values is defaults.yaml < storm.yaml < topology specific configuration < internal component specific configuration < external component specific configuration. 
+
+
+**Resources:**
+
+* [Config](javadocs/backtype/storm/Config.html): a listing of all configurations as well as a helper class for creating topology specific configurations
+* [defaults.yaml](https://github.com/apache/incubator-storm/blob/master/conf/defaults.yaml): the default values for all configurations
+* [Setting up a Storm cluster](Setting-up-a-Storm-cluster.html): explains how to create and configure a Storm cluster
+* [Running topologies on a production cluster](Running-topologies-on-a-production-cluster.html): lists useful configurations when running topologies on a cluster
+* [Local mode](Local-mode.html): lists useful configurations when using local mode

http://git-wip-us.apache.org/repos/asf/storm/blob/d491c3ff/docs/Contributing-to-Storm.md
----------------------------------------------------------------------
diff --git a/docs/Contributing-to-Storm.md b/docs/Contributing-to-Storm.md
new file mode 100644
index 0000000..dff23fb
--- /dev/null
+++ b/docs/Contributing-to-Storm.md
@@ -0,0 +1,31 @@
+---
+layout: documentation
+---
+
+### Getting started with contributing
+
+Some of the issues on the [issue tracker](https://issues.apache.org/jira/browse/STORM) are marked with the "Newbie" label. If you're interesting in contributing to Storm but don't know where to begin, these are good issues to start with. These issues are a great way to get your feet wet with learning the codebase because they require learning about only an isolated portion of the codebase and are a relatively small amount of work.
+
+### Learning the codebase
+
+The [Implementation docs](Implementation-docs.html) section of the wiki gives detailed walkthroughs of the codebase. Reading through these docs is highly recommended to understand the codebase.
+
+### Contribution process
+
+Contributions to the Storm codebase should be sent as GitHub pull requests. If there's any problems to the pull request we can iterate on it using GitHub's commenting features.
+
+For small patches, feel free to submit pull requests directly for them. For larger contributions, please use the following process. The idea behind this process is to prevent any wasted work and catch design issues early on:
+
+1. Open an issue on the [issue tracker](https://issues.apache.org/jira/browse/STORM) if one doesn't exist already
+2. Comment on the issue with your plan for implementing the issue. Explain what pieces of the codebase you're going to touch and how everything is going to fit together.
+3. Storm committers will iterate with you on the design to make sure you're on the right track
+4. Implement your issue, submit a pull request, and iterate from there.
+
+### Modules built on top of Storm
+
+Modules built on top of Storm (like spouts, bolts, etc) that aren't appropriate for Storm core can be done as your own project or as part of [@stormprocessor](https://github.com/stormprocessor). To be part of @stormprocessor put your project on your own Github and then send an email to the mailing list proposing to make it part of @stormprocessor. Then the community can discuss whether it's useful enough to be part of @stormprocessor. Then you'll be added to the @stormprocessor organization and can maintain your project there. The advantage of hosting your module in @stormprocessor is that it will be easier for potential users to find your project.
+
+### Contributing documentation
+
+Documentation contributions are very welcome! The best way to send contributions is as emails through the mailing list.
+

http://git-wip-us.apache.org/repos/asf/storm/blob/d491c3ff/docs/Creating-a-new-Storm-project.md
----------------------------------------------------------------------
diff --git a/docs/Creating-a-new-Storm-project.md b/docs/Creating-a-new-Storm-project.md
new file mode 100644
index 0000000..feb49b8
--- /dev/null
+++ b/docs/Creating-a-new-Storm-project.md
@@ -0,0 +1,25 @@
+---
+layout: documentation
+---
+This page outlines how to set up a Storm project for development. The steps are:
+
+1. Add Storm jars to classpath
+2. If using multilang, add multilang dir to classpath
+
+Follow along to see how to set up the [storm-starter](http://github.com/nathanmarz/storm-starter) project in Eclipse.
+
+### Add Storm jars to classpath
+
+You'll need the Storm jars on your classpath to develop Storm topologies. Using [Maven](Maven.html) is highly recommended. [Here's an example](https://github.com/nathanmarz/storm-starter/blob/master/m2-pom.xml) of how to setup your pom.xml for a Storm project. If you don't want to use Maven, you can include the jars from the Storm release on your classpath. 
+
+[storm-starter](http://github.com/nathanmarz/storm-starter) uses [Leiningen](http://github.com/technomancy/leiningen) for build and dependency resolution. You can install leiningen by downloading [this script](https://raw.github.com/technomancy/leiningen/stable/bin/lein), placing it on your path, and making it executable. To retrieve the dependencies for Storm, simply run `lein deps` in the project root.
+
+To set up the classpath in Eclipse, create a new Java project, include `src/jvm/` as a source path, and make sure all the jars in `lib/` and `lib/dev/` are in the `Referenced Libraries` section of the project.
+
+### If using multilang, add multilang dir to classpath
+
+If you implement spouts or bolts in languages other than Java, then those implementations should be under the `multilang/resources/` directory of the project. For Storm to find these files in local mode, the `resources/` dir needs to be on the classpath. You can do this in Eclipse by adding `multilang/` as a source folder. You may also need to add multilang/resources as a source directory.
+
+For more information on writing topologies in other languages, see [Using non-JVM languages with Storm](Using-non-JVM-languages-with-Storm.html).
+
+To test that everything is working in Eclipse, you should now be able to `Run` the `WordCountTopology.java` file. You will see messages being emitted at the console for 10 seconds.

http://git-wip-us.apache.org/repos/asf/storm/blob/d491c3ff/docs/DSLs-and-multilang-adapters.md
----------------------------------------------------------------------
diff --git a/docs/DSLs-and-multilang-adapters.md b/docs/DSLs-and-multilang-adapters.md
new file mode 100644
index 0000000..31bd453
--- /dev/null
+++ b/docs/DSLs-and-multilang-adapters.md
@@ -0,0 +1,9 @@
+---
+layout: documentation
+---
+* [Scala DSL](https://github.com/velvia/ScalaStorm)
+* [JRuby DSL](https://github.com/colinsurprenant/redstorm)
+* [Clojure DSL](Clojure-DSL.html)
+* [Storm/Esper integration](https://github.com/tomdz/storm-esper): Streaming SQL on top of Storm
+* [io-storm](https://github.com/gphat/io-storm): Perl multilang adapter
+* [storm-php](https://github.com/lazyshot/storm-php): PHP multilang adapter

http://git-wip-us.apache.org/repos/asf/storm/blob/d491c3ff/docs/Defining-a-non-jvm-language-dsl-for-storm.md
----------------------------------------------------------------------
diff --git a/docs/Defining-a-non-jvm-language-dsl-for-storm.md b/docs/Defining-a-non-jvm-language-dsl-for-storm.md
new file mode 100644
index 0000000..f52f4ab
--- /dev/null
+++ b/docs/Defining-a-non-jvm-language-dsl-for-storm.md
@@ -0,0 +1,36 @@
+---
+layout: documentation
+---
+The right place to start to learn how to make a non-JVM DSL for Storm is [storm-core/src/storm.thrift](https://github.com/apache/incubator-storm/blob/master/storm-core/src/storm.thrift). Since Storm topologies are just Thrift structures, and Nimbus is a Thrift daemon, you can create and submit topologies in any language.
+
+When you create the Thrift structs for spouts and bolts, the code for the spout or bolt is specified in the ComponentObject struct:
+
+```
+union ComponentObject {
+  1: binary serialized_java;
+  2: ShellComponent shell;
+  3: JavaObject java_object;
+}
+```
+
+For a Python DSL, you would want to make use of "2" and "3". ShellComponent lets you specify a script to run that component (e.g., your python code). And JavaObject lets you specify native java spouts and bolts for the component (and Storm will use reflection to create that spout or bolt).
+
+There's a "storm shell" command that will help with submitting a topology. Its usage is like this:
+
+```
+storm shell resources/ python topology.py arg1 arg2
+```
+
+storm shell will then package resources/ into a jar, upload the jar to Nimbus, and call your topology.py script like this:
+
+```
+python topology.py arg1 arg2 {nimbus-host} {nimbus-port} {uploaded-jar-location}
+```
+
+Then you can connect to Nimbus using the Thrift API and submit the topology, passing {uploaded-jar-location} into the submitTopology method. For reference, here's the submitTopology definition:
+
+```java
+void submitTopology(1: string name, 2: string uploadedJarLocation, 3: string jsonConf, 4: StormTopology topology) throws (1: AlreadyAliveException e, 2: InvalidTopologyException ite);
+```
+
+Finally, one of the key things to do in a non-JVM DSL is make it easy to define the entire topology in one file (the bolts, spouts, and the definition of the topology).

http://git-wip-us.apache.org/repos/asf/storm/blob/d491c3ff/docs/Distributed-RPC.md
----------------------------------------------------------------------
diff --git a/docs/Distributed-RPC.md b/docs/Distributed-RPC.md
new file mode 100644
index 0000000..fc75ee4
--- /dev/null
+++ b/docs/Distributed-RPC.md
@@ -0,0 +1,197 @@
+---
+layout: documentation
+---
+The idea behind distributed RPC (DRPC) is to parallelize the computation of really intense functions on the fly using Storm. The Storm topology takes in as input a stream of function arguments, and it emits an output stream of the results for each of those function calls. 
+
+DRPC is not so much a feature of Storm as it is a pattern expressed from Storm's primitives of streams, spouts, bolts, and topologies. DRPC could have been packaged as a separate library from Storm, but it's so useful that it's bundled with Storm.
+
+### High level overview
+
+Distributed RPC is coordinated by a "DRPC server" (Storm comes packaged with an implementation of this). The DRPC server coordinates receiving an RPC request, sending the request to the Storm topology, receiving the results from the Storm topology, and sending the results back to the waiting client. From a client's perspective, a distributed RPC call looks just like a regular RPC call. For example, here's how a client would compute the results for the "reach" function with the argument "http://twitter.com":
+
+```java
+DRPCClient client = new DRPCClient("drpc-host", 3772);
+String result = client.execute("reach", "http://twitter.com");
+```
+
+The distributed RPC workflow looks like this:
+
+![Tasks in a topology](images/drpc-workflow.png)
+
+A client sends the DRPC server the name of the function to execute and the arguments to that function. The topology implementing that function uses a `DRPCSpout` to receive a function invocation stream from the DRPC server. Each function invocation is tagged with a unique id by the DRPC server. The topology then computes the result and at the end of the topology a bolt called `ReturnResults` connects to the DRPC server and gives it the result for the function invocation id. The DRPC server then uses the id to match up that result with which client is waiting, unblocks the waiting client, and sends it the result.
+
+### LinearDRPCTopologyBuilder
+
+Storm comes with a topology builder called [LinearDRPCTopologyBuilder](javadocs/backtype/storm/drpc/LinearDRPCTopologyBuilder.html) that automates almost all the steps involved for doing DRPC. These include:
+
+1. Setting up the spout
+2. Returning the results to the DRPC server
+3. Providing functionality to bolts for doing finite aggregations over groups of tuples
+
+Let's look at a simple example. Here's the implementation of a DRPC topology that returns its input argument with a "!" appended:
+
+```java
+public static class ExclaimBolt extends BaseBasicBolt {
+    public void execute(Tuple tuple, BasicOutputCollector collector) {
+        String input = tuple.getString(1);
+        collector.emit(new Values(tuple.getValue(0), input + "!"));
+    }
+
+    public void declareOutputFields(OutputFieldsDeclarer declarer) {
+        declarer.declare(new Fields("id", "result"));
+    }
+}
+
+public static void main(String[] args) throws Exception {
+    LinearDRPCTopologyBuilder builder = new LinearDRPCTopologyBuilder("exclamation");
+    builder.addBolt(new ExclaimBolt(), 3);
+    // ...
+}
+```
+
+As you can see, there's very little to it. When creating the `LinearDRPCTopologyBuilder`, you tell it the name of the DRPC function for the topology. A single DRPC server can coordinate many functions, and the function name distinguishes the functions from one another. The first bolt you declare will take in as input 2-tuples, where the first field is the request id and the second field is the arguments for that request. `LinearDRPCTopologyBuilder` expects the last bolt to emit an output stream containing 2-tuples of the form [id, result]. Finally, all intermediate tuples must contain the request id as the first field.
+
+In this example, `ExclaimBolt` simply appends a "!" to the second field of the tuple. `LinearDRPCTopologyBuilder` handles the rest of the coordination of connecting to the DRPC server and sending results back.
+
+### Local mode DRPC
+
+DRPC can be run in local mode. Here's how to run the above example in local mode:
+
+```java
+LocalDRPC drpc = new LocalDRPC();
+LocalCluster cluster = new LocalCluster();
+
+cluster.submitTopology("drpc-demo", conf, builder.createLocalTopology(drpc));
+
+System.out.println("Results for 'hello':" + drpc.execute("exclamation", "hello"));
+
+cluster.shutdown();
+drpc.shutdown();
+```
+
+First you create a `LocalDRPC` object. This object simulates a DRPC server in process, just like how `LocalCluster` simulates a Storm cluster in process. Then you create the `LocalCluster` to run the topology in local mode. `LinearDRPCTopologyBuilder` has separate methods for creating local topologies and remote topologies. In local mode the `LocalDRPC` object does not bind to any ports so the topology needs to know about the object to communicate with it. This is why `createLocalTopology` takes in the `LocalDRPC` object as input.
+
+After launching the topology, you can do DRPC invocations using the `execute` method on `LocalDRPC`.
+
+### Remote mode DRPC
+
+Using DRPC on an actual cluster is also straightforward. There's three steps:
+
+1. Launch DRPC server(s)
+2. Configure the locations of the DRPC servers
+3. Submit DRPC topologies to Storm cluster
+
+Launching a DRPC server can be done with the `storm` script and is just like launching Nimbus or the UI:
+
+```
+bin/storm drpc
+```
+
+Next, you need to configure your Storm cluster to know the locations of the DRPC server(s). This is how `DRPCSpout` knows from where to read function invocations. This can be done through the `storm.yaml` file or the topology configurations. Configuring this through the `storm.yaml` looks something like this:
+
+```yaml
+drpc.servers:
+  - "drpc1.foo.com"
+  - "drpc2.foo.com"
+```
+
+Finally, you launch DRPC topologies using `StormSubmitter` just like you launch any other topology. To run the above example in remote mode, you do something like this:
+
+```java
+StormSubmitter.submitTopology("exclamation-drpc", conf, builder.createRemoteTopology());
+```
+
+`createRemoteTopology` is used to create topologies suitable for Storm clusters.
+
+### A more complex example
+
+The exclamation DRPC example was a toy example for illustrating the concepts of DRPC. Let's look at a more complex example which really needs the parallelism a Storm cluster provides for computing the DRPC function. The example we'll look at is computing the reach of a URL on Twitter.
+
+The reach of a URL is the number of unique people exposed to a URL on Twitter. To compute reach, you need to:
+
+1. Get all the people who tweeted the URL
+2. Get all the followers of all those people
+3. Unique the set of followers
+4. Count the unique set of followers
+
+A single reach computation can involve thousands of database calls and tens of millions of follower records during the computation. It's a really, really intense computation. As you're about to see, implementing this function on top of Storm is dead simple. On a single machine, reach can take minutes to compute; on a Storm cluster, you can compute reach for even the hardest URLs in a couple seconds.
+
+A sample reach topology is defined in storm-starter [here](https://github.com/nathanmarz/storm-starter/blob/master/src/jvm/storm/starter/ReachTopology.java). Here's how you define the reach topology:
+
+```java
+LinearDRPCTopologyBuilder builder = new LinearDRPCTopologyBuilder("reach");
+builder.addBolt(new GetTweeters(), 3);
+builder.addBolt(new GetFollowers(), 12)
+        .shuffleGrouping();
+builder.addBolt(new PartialUniquer(), 6)
+        .fieldsGrouping(new Fields("id", "follower"));
+builder.addBolt(new CountAggregator(), 2)
+        .fieldsGrouping(new Fields("id"));
+```
+
+The topology executes as four steps:
+
+1. `GetTweeters` gets the users who tweeted the URL. It transforms an input stream of `[id, url]` into an output stream of `[id, tweeter]`. Each `url` tuple will map to many `tweeter` tuples.
+2. `GetFollowers` gets the followers for the tweeters. It transforms an input stream of `[id, tweeter]` into an output stream of `[id, follower]`. Across all the tasks, there may of course be duplication of follower tuples when someone follows multiple people who tweeted the same URL.
+3. `PartialUniquer` groups the followers stream by the follower id. This has the effect of the same follower going to the same task. So each task of `PartialUniquer` will receive mutually independent sets of followers. Once `PartialUniquer` receives all the follower tuples directed at it for the request id, it emits the unique count of its subset of followers.
+4. Finally, `CountAggregator` receives the partial counts from each of the `PartialUniquer` tasks and sums them up to complete the reach computation.
+
+Let's take a look at the `PartialUniquer` bolt:
+
+```java
+public class PartialUniquer extends BaseBatchBolt {
+    BatchOutputCollector _collector;
+    Object _id;
+    Set<String> _followers = new HashSet<String>();
+    
+    @Override
+    public void prepare(Map conf, TopologyContext context, BatchOutputCollector collector, Object id) {
+        _collector = collector;
+        _id = id;
+    }
+
+    @Override
+    public void execute(Tuple tuple) {
+        _followers.add(tuple.getString(1));
+    }
+    
+    @Override
+    public void finishBatch() {
+        _collector.emit(new Values(_id, _followers.size()));
+    }
+
+    @Override
+    public void declareOutputFields(OutputFieldsDeclarer declarer) {
+        declarer.declare(new Fields("id", "partial-count"));
+    }
+}
+```
+
+`PartialUniquer` implements `IBatchBolt` by extending `BaseBatchBolt`. A batch bolt provides a first class API to processing a batch of tuples as a concrete unit. A new instance of the batch bolt is created for each request id, and Storm takes care of cleaning up the instances when appropriate. 
+
+When `PartialUniquer` receives a follower tuple in the `execute` method, it adds it to the set for the request id in an internal `HashSet`. 
+
+Batch bolts provide the `finishBatch` method which is called after all the tuples for this batch targeted at this task have been processed. In the callback, `PartialUniquer` emits a single tuple containing the unique count for its subset of follower ids.
+
+Under the hood, `CoordinatedBolt` is used to detect when a given bolt has received all of the tuples for any given request id. `CoordinatedBolt` makes use of direct streams to manage this coordination.
+
+The rest of the topology should be self-explanatory. As you can see, every single step of the reach computation is done in parallel, and defining the DRPC topology was extremely simple.
+
+### Non-linear DRPC topologies
+
+`LinearDRPCTopologyBuilder` only handles "linear" DRPC topologies, where the computation is expressed as a sequence of steps (like reach). It's not hard to imagine functions that would require a more complicated topology with branching and merging of the bolts. For now, to do this you'll need to drop down into using `CoordinatedBolt` directly. Be sure to talk about your use case for non-linear DRPC topologies on the mailing list to inform the construction of more general abstractions for DRPC topologies.
+
+### How LinearDRPCTopologyBuilder works
+
+* DRPCSpout emits [args, return-info]. return-info is the host and port of the DRPC server as well as the id generated by the DRPC server
+* constructs a topology comprising of:
+  * DRPCSpout
+  * PrepareRequest (generates a request id and creates a stream for the return info and a stream for the args)
+  * CoordinatedBolt wrappers and direct groupings
+  * JoinResult (joins the result with the return info)
+  * ReturnResult (connects to the DRPC server and returns the result)
+* LinearDRPCTopologyBuilder is a good example of a higher level abstraction built on top of Storm's primitives
+
+### Advanced
+* KeyedFairBolt for weaving the processing of multiple requests at the same time
+* How to use `CoordinatedBolt` directly

http://git-wip-us.apache.org/repos/asf/storm/blob/d491c3ff/docs/Documentation.md
----------------------------------------------------------------------
diff --git a/docs/Documentation.md b/docs/Documentation.md
new file mode 100644
index 0000000..8da874c
--- /dev/null
+++ b/docs/Documentation.md
@@ -0,0 +1,50 @@
+---
+layout: documentation
+---
+### Basics of Storm
+
+* [Javadoc](javadocs/index.html)
+* [Concepts](Concepts.html)
+* [Configuration](Configuration.html)
+* [Guaranteeing message processing](Guaranteeing-message-processing.html)
+* [Fault-tolerance](Fault-tolerance.html)
+* [Command line client](Command-line-client.html)
+* [Understanding the parallelism of a Storm topology](Understanding-the-parallelism-of-a-Storm-topology.html)
+* [FAQ](FAQ.html)
+
+### Trident
+
+Trident is an alternative interface to Storm. It provides exactly-once processing, "transactional" datastore persistence, and a set of common stream analytics operations.
+
+* [Trident Tutorial](Trident-tutorial.html)     -- basic concepts and walkthrough
+* [Trident API Overview](Trident-API-Overview.html) -- operations for transforming and orchestrating data
+* [Trident State](Trident-state.html)        -- exactly-once processing and fast, persistent aggregation
+* [Trident spouts](Trident-spouts.html)       -- transactional and non-transactional data intake
+
+### Setup and deploying
+
+* [Setting up a Storm cluster](Setting-up-a-Storm-cluster.html)
+* [Local mode](Local-mode.html)
+* [Troubleshooting](Troubleshooting.html)
+* [Running topologies on a production cluster](Running-topologies-on-a-production-cluster.html)
+* [Building Storm](Maven.html) with Maven
+
+### Intermediate
+
+* [Serialization](Serialization.html)
+* [Common patterns](Common-patterns.html)
+* [Clojure DSL](Clojure-DSL.html)
+* [Using non-JVM languages with Storm](Using-non-JVM-languages-with-Storm.html)
+* [Distributed RPC](Distributed-RPC.html)
+* [Transactional topologies](Transactional-topologies.html)
+* [Kestrel and Storm](Kestrel-and-Storm.html)
+* [Direct groupings](Direct-groupings.html)
+* [Hooks](Hooks.html)
+* [Metrics](Metrics.html)
+* [Lifecycle of a trident tuple]()
+
+### Advanced
+
+* [Defining a non-JVM language DSL for Storm](Defining-a-non-jvm-language-dsl-for-storm.html)
+* [Multilang protocol](Multilang-protocol.html) (how to provide support for another language)
+* [Implementation docs](Implementation-docs.html)