STORM-1617: 0.10.x release docs

Conflicts:
        docs/images/topology.png


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

Branch: refs/heads/master
Commit: d909db8fe2aee7633b6f3b95ae71c415efea495f
Parents: abbfb97
Author: Robert (Bobby) Evans <[email protected]>
Authored: Sat Mar 19 12:33:31 2016 -0500
Committer: Robert (Bobby) Evans <[email protected]>
Committed: Sat Mar 19 12:58:31 2016 -0500

----------------------------------------------------------------------
 docs/Clojure-DSL.md                             |   8 +-
 docs/Command-line-client.md                     |   8 +-
 docs/Common-patterns.md                         |  18 +-
 docs/Concepts.md                                |  26 +-
 docs/Configuration.md                           |   6 +-
 docs/Contributing-to-Storm.md                   |   4 +-
 docs/Creating-a-new-Storm-project.md            |   8 +-
 docs/DSLs-and-multilang-adapters.md             |   5 +-
 docs/Daemon-Fault-Tolerance.md                  |  30 +
 ...Defining-a-non-jvm-language-dsl-for-storm.md |   4 +-
 docs/Distributed-RPC.md                         |   4 +-
 docs/FAQ.md                                     |  14 +-
 docs/Guaranteeing-message-processing.md         |  10 +-
 docs/Hooks.md                                   |   4 +-
 docs/Implementation-docs.md                     |   3 +-
 docs/Kestrel-and-Storm.md                       |   4 +-
 docs/Lifecycle-of-a-topology.md                 |  72 +-
 docs/Local-mode.md                              |   2 +
 docs/Maven.md                                   |  52 +-
 docs/Message-passing-implementation.md          |  36 +-
 docs/Metrics.md                                 |  20 +-
 docs/Multilang-protocol.md                      |  76 +-
 docs/Rationale.md                               |   2 +
 ...unning-topologies-on-a-production-cluster.md |   4 +-
 docs/SECURITY.md                                | 459 +++++++++-
 docs/STORM-UI-REST-API.md                       |   2 +-
 docs/Serialization.md                           |   8 +-
 docs/Setting-up-a-Storm-cluster.md              |  46 +-
 docs/Setting-up-development-environment.md      |  16 +-
 docs/Spout-implementations.md                   |   2 +
 docs/Structure-of-the-codebase.md               |  92 +-
 docs/Support-for-non-java-languages.md          |   2 +
 docs/Transactional-topologies.md                |  14 +-
 docs/Trident-API-Overview.md                    | 228 ++++-
 docs/Trident-spouts.md                          |  10 +-
 docs/Trident-state.md                           |  15 +-
 docs/Trident-tutorial.md                        |   5 +-
 docs/Troubleshooting.md                         |  40 +-
 docs/Tutorial.md                                |  22 +-
 ...nding-the-parallelism-of-a-Storm-topology.md |  30 +-
 docs/Using-non-JVM-languages-with-Storm.md      |   1 +
 docs/flux.md                                    | 835 +++++++++++++++++++
 docs/images/topology.png                        | Bin 0 -> 23147 bytes
 docs/index.md                                   | 118 ++-
 docs/storm-eventhubs.md                         |  40 +
 docs/storm-hbase.md                             | 241 ++++++
 docs/storm-hdfs.md                              | 368 ++++++++
 docs/storm-hive.md                              | 111 +++
 docs/storm-jdbc.md                              | 285 +++++++
 docs/storm-kafka.md                             | 287 +++++++
 docs/storm-redis.md                             | 258 ++++++
 51 files changed, 3599 insertions(+), 356 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/d909db8f/docs/Clojure-DSL.md
----------------------------------------------------------------------
diff --git a/docs/Clojure-DSL.md b/docs/Clojure-DSL.md
index b3109fa..234febe 100644
--- a/docs/Clojure-DSL.md
+++ b/docs/Clojure-DSL.md
@@ -1,7 +1,9 @@
 ---
+title: Clojure DSL
 layout: documentation
+documentation: true
 ---
-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.
+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]({{page.git-blob-base}}/storm-core/src/clj/backtype/storm/clojure.clj)
 namespace.
 
 This page outlines all the pieces of the Clojure DSL, including:
 
@@ -15,7 +17,7 @@ This page outlines all the pieces of the Clojure DSL, 
including:
 
 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):
+Let's take a look at an example topology definition [from the storm-starter 
project]({{page.git-blob-base}}/examples/storm-starter/src/clj/storm/starter/clj/word_count.clj):
 
 ```clojure
 (topology
@@ -201,7 +203,7 @@ The signature for `defspout` looks like the following:
 
 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):
+Here's an example `defspout` implementation from 
[storm-starter]({{page.git-blob-base}}/examples/storm-starter/src/clj/storm/starter/clj/word_count.clj):
 
 ```clojure
 (defspout sentence-spout ["sentence"]

http://git-wip-us.apache.org/repos/asf/storm/blob/d909db8f/docs/Command-line-client.md
----------------------------------------------------------------------
diff --git a/docs/Command-line-client.md b/docs/Command-line-client.md
index 0e645d7..4634467 100644
--- a/docs/Command-line-client.md
+++ b/docs/Command-line-client.md
@@ -1,7 +1,9 @@
 ---
+title: Command Line Client
 layout: documentation
+documentation: true
 ---
-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).
+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-development-environment.html).
 
 These commands are:
 
@@ -45,12 +47,14 @@ Deactivates the specified topology's spouts.
 
 ### rebalance
 
-Syntax: `storm rebalance topology-name [-w wait-time-secs]`
+Syntax: `storm rebalance topology-name [-w wait-time-secs] [-n 
new-num-workers] [-e component=parallelism]*`
 
 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).
 
+The rebalance command can also be used to change the parallelism of a running 
topology. Use the -n and -e switches to change the number of workers or number 
of executors of a component respectively.
+
 ### repl
 
 Syntax: `storm repl`

http://git-wip-us.apache.org/repos/asf/storm/blob/d909db8f/docs/Common-patterns.md
----------------------------------------------------------------------
diff --git a/docs/Common-patterns.md b/docs/Common-patterns.md
index 3f8c979..1c97f6d 100644
--- a/docs/Common-patterns.md
+++ b/docs/Common-patterns.md
@@ -1,5 +1,7 @@
 ---
+title: Common Topology Patterns
 layout: documentation
+documentation: true
 ---
 
 This page lists a variety of common patterns in Storm topologies.
@@ -62,14 +64,26 @@ A common continuous computation done on Storm is a 
"streaming top N" of some sor
 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)
+builder.setBolt("rank", new RankObjects(), parallelism)
   .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).
+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]({{page.git-blob-base}}/examples/storm-starter/src/jvm/storm/starter/RollingTopWords.java).
 
+If however you have a known skew in the data being processed it can be 
advantageous to use partialKeyGrouping instead of fieldsGrouping.  This will 
distribute the load for each key between two downstream bolts instead of a 
single one.
+
+```java
+builder.setBolt("count", new CountObjects(), parallelism)
+  .partialKeyGrouping("objects", new Fields("value"));
+builder.setBolt("rank" new AggregateCountsAndRank(), parallelism)
+  .fieldsGrouping("count", new Fields("key"))
+builder.setBolt("merge", new MergeRanksObjects())
+  .globalGrouping("rank");
+``` 
+
+The topology needs an extra layer of processing to aggregate the partial 
counts from the upstream bolts but this only processes aggregated values now so 
the bolt it is not subject to the load caused by the skewed data. You can see 
an example of this pattern in storm-starter 
[here]({{page.git-blob-base}}/examples/storm-starter/src/jvm/storm/starter/SkewedRollingTopWords.java).
 
 ### TimeCacheMap for efficiently keeping a cache of things that have been 
recently updated
 

http://git-wip-us.apache.org/repos/asf/storm/blob/d909db8f/docs/Concepts.md
----------------------------------------------------------------------
diff --git a/docs/Concepts.md b/docs/Concepts.md
index 33779f2..01dbd11 100644
--- a/docs/Concepts.md
+++ b/docs/Concepts.md
@@ -1,5 +1,7 @@
 ---
+title: Concepts
 layout: documentation
+documentation: true
 ---
 
 This page lists the main concepts of Storm and links to resources where you 
can find more information. The concepts discussed are:
@@ -35,14 +37,12 @@ Every stream is given an id when declared. Since 
single-stream spouts and bolts
 * [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).
 
+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.
 
@@ -50,7 +50,7 @@ The other main methods on spouts are `ack` and `fail`. These 
are called when Sto
 
 **Resources:**
 
-* [IRichSpout](javadocs/backtype/storm/topology/IRichSpout.html): this is the 
interface that spouts must implement. 
+* [IRichSpout](javadocs/backtype/storm/topology/IRichSpout.html): this is the 
interface that spouts must implement.
 * [Guaranteeing message processing](Guaranteeing-message-processing.html)
 
 ### Bolts
@@ -61,7 +61,7 @@ Bolts can do simple stream transformations. Doing complex 
stream transformations
 
 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)`. 
+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.
 
@@ -78,21 +78,21 @@ Its perfectly fine to launch new threads in bolts that do 
processing asynchronou
 
 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:
+There are eight 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.
+3. **Partial Key grouping**: The stream is partitioned by the fields specified 
in the grouping, like the Fields grouping, but are load balanced between two 
downstream bolts, which provides better utilization of resources when the 
incoming data is skewed. [This 
paper](https://melmeric.files.wordpress.com/2014/11/the-power-of-both-choices-practical-load-balancing-for-distributed-stream-processing-engines.pdf)
 provides a good explanation of how it works and the advantages it provides.
+4. **All grouping**: The stream is replicated across all the bolt's tasks. Use 
this grouping with care.
+5. **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.
+6. **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).
+7. **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).
+8. **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
 
@@ -104,7 +104,7 @@ This is all explained in much more detail in [Guaranteeing 
message processing](G
 
 ### 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). 
+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
 

http://git-wip-us.apache.org/repos/asf/storm/blob/d909db8f/docs/Configuration.md
----------------------------------------------------------------------
diff --git a/docs/Configuration.md b/docs/Configuration.md
index 8e8ca77..83f4ef7 100644
--- a/docs/Configuration.md
+++ b/docs/Configuration.md
@@ -1,9 +1,11 @@
 ---
+title: Configuration
 layout: documentation
+documentation: true
 ---
 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".
+Every configuration has a default value defined in 
[defaults.yaml]({{page.git-blob-base}}/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:
 
@@ -23,7 +25,7 @@ The preference order for configuration values is 
defaults.yaml < storm.yaml < to
 **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
+* [defaults.yaml]({{page.git-blob-base}}/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/d909db8f/docs/Contributing-to-Storm.md
----------------------------------------------------------------------
diff --git a/docs/Contributing-to-Storm.md b/docs/Contributing-to-Storm.md
index dff23fb..fdc5835 100644
--- a/docs/Contributing-to-Storm.md
+++ b/docs/Contributing-to-Storm.md
@@ -1,5 +1,7 @@
 ---
+title: Contributing
 layout: documentation
+documentation: true
 ---
 
 ### Getting started with contributing
@@ -12,7 +14,7 @@ The [Implementation docs](Implementation-docs.html) section 
of the wiki gives de
 
 ### 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.
+Contributions to the Storm codebase should be sent as 
[GitHub](https://github.com/apache/storm) 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:
 

http://git-wip-us.apache.org/repos/asf/storm/blob/d909db8f/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
index feb49b8..35ab1eb 100644
--- a/docs/Creating-a-new-Storm-project.md
+++ b/docs/Creating-a-new-Storm-project.md
@@ -1,18 +1,18 @@
 ---
+title: Creating a New Storm Project
 layout: documentation
+documentation: true
 ---
 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.
+Follow along to see how to set up the 
[storm-starter]({{page.git-blob-base}}/examples/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.
+You'll need the Storm jars on your classpath to develop Storm topologies. 
Using [Maven](Maven.html) is highly recommended. [Here's an 
example]({{page.git-blob-base}}/examples/storm-starter/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.
 
 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.
 

http://git-wip-us.apache.org/repos/asf/storm/blob/d909db8f/docs/DSLs-and-multilang-adapters.md
----------------------------------------------------------------------
diff --git a/docs/DSLs-and-multilang-adapters.md 
b/docs/DSLs-and-multilang-adapters.md
index 31bd453..0ed5450 100644
--- a/docs/DSLs-and-multilang-adapters.md
+++ b/docs/DSLs-and-multilang-adapters.md
@@ -1,9 +1,10 @@
 ---
+title: Storm DSLs and Multi-Lang Adapters
 layout: documentation
+documentation: true
 ---
 * [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
+* [io-storm](https://github.com/dan-blanchard/io-storm): Perl multilang adapter

http://git-wip-us.apache.org/repos/asf/storm/blob/d909db8f/docs/Daemon-Fault-Tolerance.md
----------------------------------------------------------------------
diff --git a/docs/Daemon-Fault-Tolerance.md b/docs/Daemon-Fault-Tolerance.md
new file mode 100644
index 0000000..1af681a
--- /dev/null
+++ b/docs/Daemon-Fault-Tolerance.md
@@ -0,0 +1,30 @@
+---
+title: Daemon Fault Tolerance
+layout: documentation
+documentation: true
+---
+Storm has several different daemon processes.  Nimbus that schedules workers, 
supervisors that launch and kill workers, the log viewer that gives access to 
logs, and the UI that shows the status of a cluster.
+
+## What happens when a worker dies?
+
+When a worker dies, the supervisor will restart it. If it continuously fails 
on startup and is unable to heartbeat to Nimbus, Nimbus will reschedule the 
worker.
+
+## What happens when a node dies?
+
+The tasks assigned to that machine will time-out and Nimbus will reassign 
those tasks to other machines.
+
+## What happens when Nimbus or Supervisor daemons die?
+
+The Nimbus and Supervisor daemons are designed to be fail-fast (process 
self-destructs whenever any unexpected situation is encountered) and stateless 
(all state is kept in Zookeeper or on disk). As described in [Setting up a 
Storm cluster](Setting-up-a-Storm-cluster.html), the Nimbus and Supervisor 
daemons must be run under supervision using a tool like daemontools or monit. 
So if the Nimbus or Supervisor daemons die, they restart like nothing happened.
+
+Most notably, no worker processes are affected by the death of Nimbus or the 
Supervisors. This is in contrast to Hadoop, where if the JobTracker dies, all 
the running jobs are lost. 
+
+## Is Nimbus a single point of failure?
+
+If you lose the Nimbus node, the workers will still continue to function. 
Additionally, supervisors will continue to restart workers if they die. 
However, without Nimbus, workers won't be reassigned to other machines when 
necessary (like if you lose a worker machine). 
+
+So the answer is that Nimbus is "sort of" a SPOF. In practice, it's not a big 
deal since nothing catastrophic happens when the Nimbus daemon dies. There are 
plans to make Nimbus highly available in the future.
+
+## How does Storm guarantee data processing?
+
+Storm provides mechanisms to guarantee data processing even if nodes die or 
messages are lost. See [Guaranteeing message 
processing](Guaranteeing-message-processing.html) for the details.

http://git-wip-us.apache.org/repos/asf/storm/blob/d909db8f/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
index f52f4ab..7096a43 100644
--- a/docs/Defining-a-non-jvm-language-dsl-for-storm.md
+++ b/docs/Defining-a-non-jvm-language-dsl-for-storm.md
@@ -1,7 +1,9 @@
 ---
+title: Defining a Non-JVM DSL for Storm
 layout: documentation
+documentation: true
 ---
-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.
+The right place to start to learn how to make a non-JVM DSL for Storm is 
[storm-core/src/storm.thrift]({{page.git-blob-base}}/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:
 

http://git-wip-us.apache.org/repos/asf/storm/blob/d909db8f/docs/Distributed-RPC.md
----------------------------------------------------------------------
diff --git a/docs/Distributed-RPC.md b/docs/Distributed-RPC.md
index fc75ee4..4af2702 100644
--- a/docs/Distributed-RPC.md
+++ b/docs/Distributed-RPC.md
@@ -1,5 +1,7 @@
 ---
+title: Distributed RPC
 layout: documentation
+documentation: true
 ---
 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. 
 
@@ -116,7 +118,7 @@ The reach of a URL is the number of unique people exposed 
to a URL on Twitter. T
 
 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:
+A sample reach topology is defined in storm-starter 
[here]({{page.git-blob-base}}/examples/storm-starter/src/jvm/storm/starter/ReachTopology.java).
 Here's how you define the reach topology:
 
 ```java
 LinearDRPCTopologyBuilder builder = new LinearDRPCTopologyBuilder("reach");

http://git-wip-us.apache.org/repos/asf/storm/blob/d909db8f/docs/FAQ.md
----------------------------------------------------------------------
diff --git a/docs/FAQ.md b/docs/FAQ.md
index 8ff7a6f..127c95c 100644
--- a/docs/FAQ.md
+++ b/docs/FAQ.md
@@ -1,5 +1,7 @@
 ---
+title: FAQ
 layout: documentation
+documentation: true
 ---
 
 ## Best Practices
@@ -26,7 +28,7 @@ layout: documentation
 
 ### Halp! I cannot see:
 
-* **my logs** Logs by default go to $STORM_HOME/logs. Check that you have 
write permissions to that directory. They are configured in the 
logback/cluster.xml (0.9) and log4j/*.properties in earlier versions.
+* **my logs** Logs by default go to $STORM_HOME/logs. Check that you have 
write permissions to that directory. They are configured in log4j2/{cluster, 
worker}.xml.
 * **final JVM settings** Add the `-XX+PrintFlagsFinal` commandline option in 
the childopts (see the conf file)
 * **final Java system properties** Add `Properties props = 
System.getProperties(); props.list(System.out);` near where you build your 
topology.
 
@@ -60,6 +62,10 @@ You can join streams with join, merge or multiReduce.
 
 At time of writing, you can't emit to multiple output streams from Trident -- 
see [STORM-68](https://issues.apache.org/jira/browse/STORM-68)
 
+### Why am I getting a NotSerializableException/IllegalStateException when my 
topology is being started up?
+
+Within the Storm lifecycle, the topology is instantiated and then serialized 
to byte format to be stored in ZooKeeper, prior to the topology being executed. 
Within this step, if a spout or bolt within the topology has an initialized 
unserializable property, serialization will fail. If there is a need for a 
field that is unserializable, initialize it within the bolt or spout's prepare 
method, which is run after the topology is delivered to the worker.
+
 ## Spouts
 
 ### What is a coordinator, and why are there several?
@@ -72,11 +78,11 @@ You should only store static data, and as little of it as 
possible, into the met
 
 ### How often is the 'emitPartitionBatchNew' function called?
 
-Since the MBC is the actual spout, all the tuples in a batch are just members 
of its tupletree. That means storm's "max spout pending" config effectively 
defines the number of concurrent batches trident runs. The MBC emits a new 
batch if it has fewer than max-spending tuples pending and if at least one 
[trident batch 
interval](https://github.com/apache/incubator-storm/blob/master/conf/defaults.yaml#L115)'s
 worth of seconds has passed since the last batch.
+Since the MBC is the actual spout, all the tuples in a batch are just members 
of its tupletree. That means storm's "max spout pending" config effectively 
defines the number of concurrent batches trident runs. The MBC emits a new 
batch if it has fewer than max-spending tuples pending and if at least one 
[trident batch interval]({{page.git-blob-base}}/conf/defaults.yaml#L115)'s 
worth of seconds has passed since the last batch.
 
 ### If nothing was emitted does Trident slow down the calls?
 
-Yes, there's a pluggable "spout wait strategy"; the default is to sleep for a 
[configurable amount of 
time](https://github.com/apache/incubator-storm/blob/master/conf/defaults.yaml#L110)
+Yes, there's a pluggable "spout wait strategy"; the default is to sleep for a 
[configurable amount of time]({{page.git-blob-base}}/conf/defaults.yaml#L110)
 
 ### OK, then what is the trident batch interval for?
 
@@ -107,7 +113,7 @@ You can't change the overall batch size once generated, but 
you can change the n
 
 ### How do I aggregate events by time?
 
-If have records with an immutable timestamp, and you would like to count, 
average or otherwise aggregate them into discrete time buckets, Trident is an 
excellent and scalable solution.
+If you have records with an immutable timestamp, and you would like to count, 
average or otherwise aggregate them into discrete time buckets, Trident is an 
excellent and scalable solution.
 
 Write an `Each` function that turns the timestamp into a time bucket: if the 
bucket size was "by hour", then the timestamp `2013-08-08 12:34:56` would be 
mapped to the `2013-08-08 12:00:00` time bucket, and so would everything else 
in the twelve o'clock hour. Then group on that timebucket and use a grouped 
persistentAggregate. The persistentAggregate uses a local cacheMap backed by a 
data store. Groups with many records require very few reads from the data 
store, and use efficient bulk reads and writes; as long as your data feed is 
relatively prompt Trident will make very efficient use of memory and network. 
Even if a server drops off line for a day, then delivers that full day's worth 
of data in a rush, the old results will be calmly retrieved and updated -- and 
without interfering with calculating the current results.
 

http://git-wip-us.apache.org/repos/asf/storm/blob/d909db8f/docs/Guaranteeing-message-processing.md
----------------------------------------------------------------------
diff --git a/docs/Guaranteeing-message-processing.md 
b/docs/Guaranteeing-message-processing.md
index 91d4384..932ff0d 100644
--- a/docs/Guaranteeing-message-processing.md
+++ b/docs/Guaranteeing-message-processing.md
@@ -1,7 +1,10 @@
 ---
+title: Guaranteeing Message Processing
 layout: documentation
+documentation: true
 ---
-Storm guarantees that each message coming off a spout will be fully processed. 
This page describes how Storm accomplishes this guarantee and what you have to 
do as a user to benefit from Storm's reliability capabilities.
+Storm offers several different levels of guaranteed message processing, 
includeing best effort, at least once, and exactly once through 
[Trident](Trident-tutorial.html). 
+This page describes how Storm can guarantee at least once processing.
 
 ### What does it mean for a message to be "fully processed"?
 
@@ -129,12 +132,11 @@ In contrast, bolts that do aggregations or joins may 
delay acking a tuple until
 
 ### How do I make my applications work correctly given that tuples can be 
replayed?
 
-As always in software design, the answer is "it depends." Storm 0.7.0 
introduced the "transactional topologies" feature, which enables you to get 
fully fault-tolerant exactly-once messaging semantics for most computations. 
Read more about transactional topologies [here](Transactional-topologies.html). 
-
+As always in software design, the answer is "it depends." If you really want 
exactly once semantics use the [Trident](Trident-tutorial.html) API. In some 
cases, like with a lot of analytics, dropping data is OK so disabling the fault 
tolerance by setting the number of acker bolts to 0 
[Config.TOPOLOGY_ACKERS](javadocs/backtype/storm/Config.html#TOPOLOGY_ACKERS).  
But in some cases you want to be sure that everything was processed at least 
once and nothing was dropped.  This is especially useful if all operations are 
idenpotent or if deduping can happen aferwards.
 
 ### How does Storm implement reliability in an efficient way?
 
-A Storm topology has a set of special "acker" tasks that track the DAG of 
tuples for every spout tuple. When an acker sees that a DAG is complete, it 
sends a message to the spout task that created the spout tuple to ack the 
message. You can set the number of acker tasks for a topology in the topology 
configuration using 
[Config.TOPOLOGY_ACKERS](javadocs/backtype/storm/Config.html#TOPOLOGY_ACKERS). 
Storm defaults TOPOLOGY_ACKERS to one task -- you will need to increase this 
number for topologies processing large amounts of messages. 
+A Storm topology has a set of special "acker" tasks that track the DAG of 
tuples for every spout tuple. When an acker sees that a DAG is complete, it 
sends a message to the spout task that created the spout tuple to ack the 
message. You can set the number of acker tasks for a topology in the topology 
configuration using 
[Config.TOPOLOGY_ACKERS](javadocs/backtype/storm/Config.html#TOPOLOGY_ACKERS). 
Storm defaults TOPOLOGY_ACKERS to one task per worker.
 
 The best way to understand Storm's reliability implementation is to look at 
the lifecycle of tuples and tuple DAGs. When a tuple is created in a topology, 
whether in a spout or a bolt, it is given a random 64 bit id. These ids are 
used by ackers to track the tuple DAG for every spout tuple.
 

http://git-wip-us.apache.org/repos/asf/storm/blob/d909db8f/docs/Hooks.md
----------------------------------------------------------------------
diff --git a/docs/Hooks.md b/docs/Hooks.md
index bbe87a9..01cfa92 100644
--- a/docs/Hooks.md
+++ b/docs/Hooks.md
@@ -1,7 +1,9 @@
 ---
+title: Hooks
 layout: documentation
+documentation: true
 ---
 Storm provides hooks with which you can insert custom code to run on any 
number of events within Storm. You create a hook by extending the 
[BaseTaskHook](javadocs/backtype/storm/hooks/BaseTaskHook.html) class and 
overriding the appropriate method for the event you want to catch. There are 
two ways to register your hook:
 
-1. In the open method of your spout or prepare method of your bolt using the 
[TopologyContext#addTaskHook](javadocs/backtype/storm/task/TopologyContext.html)
 method.
+1. In the open method of your spout or prepare method of your bolt using the 
[TopologyContext](javadocs/backtype/storm/task/TopologyContext.html#addTaskHook)
 method.
 2. Through the Storm configuration using the 
["topology.auto.task.hooks"](javadocs/backtype/storm/Config.html#TOPOLOGY_AUTO_TASK_HOOKS)
 config. These hooks are automatically registered in every spout or bolt, and 
are useful for doing things like integrating with a custom monitoring system.

http://git-wip-us.apache.org/repos/asf/storm/blob/d909db8f/docs/Implementation-docs.md
----------------------------------------------------------------------
diff --git a/docs/Implementation-docs.md b/docs/Implementation-docs.md
index f01083a..15f088e 100644
--- a/docs/Implementation-docs.md
+++ b/docs/Implementation-docs.md
@@ -1,12 +1,13 @@
 ---
+title: Storm Internal Implementation
 layout: documentation
+documentation: true
 ---
 This section of the wiki is dedicated to explaining how Storm is implemented. 
You should have a good grasp of how to use Storm before reading these sections. 
 
 - [Structure of the codebase](Structure-of-the-codebase.html)
 - [Lifecycle of a topology](Lifecycle-of-a-topology.html)
 - [Message passing implementation](Message-passing-implementation.html)
-- [Acking framework implementation](Acking-framework-implementation.html)
 - [Metrics](Metrics.html)
 - How transactional topologies work
    - subtopology for TransactionalSpout

http://git-wip-us.apache.org/repos/asf/storm/blob/d909db8f/docs/Kestrel-and-Storm.md
----------------------------------------------------------------------
diff --git a/docs/Kestrel-and-Storm.md b/docs/Kestrel-and-Storm.md
index e16b0d9..cb80139 100644
--- a/docs/Kestrel-and-Storm.md
+++ b/docs/Kestrel-and-Storm.md
@@ -1,11 +1,13 @@
 ---
+title: Storm and Kestrel
 layout: documentation
+documentation: true
 ---
 This page explains how to use to Storm to consume items from a Kestrel cluster.
 
 ## Preliminaries
 ### Storm
-This tutorial uses examples from the 
[storm-kestrel](https://github.com/nathanmarz/storm-kestrel) project and the 
[storm-starter](https://github.com/nathanmarz/storm-starter) project. It's 
recommended that you clone those projects and follow along with the examples. 
Read [Setting up development 
environment](https://github.com/apache/incubator-storm/wiki/Setting-up-development-environment)
 and [Creating a new Storm 
project](https://github.com/apache/incubator-storm/wiki/Creating-a-new-Storm-project)
 to get your machine set up.
+This tutorial uses examples from the 
[storm-kestrel](https://github.com/nathanmarz/storm-kestrel) project and the 
[storm-starter](http://github.com/apache/storm/blob/{{page.version}}/examples/storm-starter)
 project. It's recommended that you clone those projects and follow along with 
the examples. Read [Setting up development 
environment](Setting-up-development-environment.html) and [Creating a new Storm 
project](Creating-a-new-Storm-project.html) to get your machine set up.
 ### Kestrel
 It assumes you are able to run locally a Kestrel server as described 
[here](https://github.com/nathanmarz/storm-kestrel).
 

http://git-wip-us.apache.org/repos/asf/storm/blob/d909db8f/docs/Lifecycle-of-a-topology.md
----------------------------------------------------------------------
diff --git a/docs/Lifecycle-of-a-topology.md b/docs/Lifecycle-of-a-topology.md
index 4919be8..6436206 100644
--- a/docs/Lifecycle-of-a-topology.md
+++ b/docs/Lifecycle-of-a-topology.md
@@ -1,5 +1,7 @@
 ---
+title: Lifecycle of a Storm Topology
 layout: documentation
+documentation: true
 ---
 (**NOTE**: this page is based on the 0.7.1 code; many things have changed 
since then, including a split between tasks and executors, and a reorganization 
of the code under `storm-core/src` rather than `src/`.)
 
@@ -7,74 +9,74 @@ This page explains in detail the lifecycle of a topology from 
running the "storm
 
 First a couple of important notes about topologies:
 
-1. The actual topology that runs is different than the topology the user 
specifies. The actual topology has implicit streams and an implicit "acker" 
bolt added to manage the acking framework (used to guarantee data processing). 
The implicit topology is created via the 
[system-topology!](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/daemon/common.clj#L188)
 function.
+1. The actual topology that runs is different than the topology the user 
specifies. The actual topology has implicit streams and an implicit "acker" 
bolt added to manage the acking framework (used to guarantee data processing). 
The implicit topology is created via the 
[system-topology!](https://github.com/apache/storm/blob/0.7.1/src/clj/backtype/storm/daemon/common.clj#L188)
 function.
 2. `system-topology!` is used in two places:
-  - when Nimbus is creating tasks for the topology 
[code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/daemon/nimbus.clj#L316)
-  - in the worker so it knows where it needs to route messages to 
[code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/daemon/worker.clj#L90)
+  - when Nimbus is creating tasks for the topology 
[code](https://github.com/apache/storm/blob/0.7.1/src/clj/backtype/storm/daemon/nimbus.clj#L316)
+  - in the worker so it knows where it needs to route messages to 
[code](https://github.com/apache/storm/blob/0.7.1/src/clj/backtype/storm/daemon/worker.clj#L90)
 
 ## Starting a topology
 
-- "storm jar" command executes your class with the specified arguments. The 
only special thing that "storm jar" does is set the "storm.jar" environment 
variable for use by `StormSubmitter` later. 
[code](https://github.com/apache/incubator-storm/blob/0.7.1/bin/storm#L101)
+- "storm jar" command executes your class with the specified arguments. The 
only special thing that "storm jar" does is set the "storm.jar" environment 
variable for use by `StormSubmitter` later. 
[code](https://github.com/apache/storm/blob/0.7.1/bin/storm#L101)
 - When your code uses `StormSubmitter.submitTopology`, `StormSubmitter` takes 
the following actions:
-  - First, `StormSubmitter` uploads the jar if it hasn't been uploaded before. 
[code](https://github.com/apache/incubator-storm/blob/0.7.1/src/jvm/backtype/storm/StormSubmitter.java#L83)
-    - Jar uploading is done via Nimbus's Thrift interface 
[code](https://github.com/apache/incubator-storm/blob/0.7.1/src/storm.thrift#L200)
+  - First, `StormSubmitter` uploads the jar if it hasn't been uploaded before. 
[code](https://github.com/apache/storm/blob/0.7.1/src/jvm/backtype/storm/StormSubmitter.java#L83)
+    - Jar uploading is done via Nimbus's Thrift interface 
[code](https://github.com/apache/storm/blob/0.7.1/src/storm.thrift#L200)
     - `beginFileUpload` returns a path in Nimbus's inbox
     - 15 kilobytes are uploaded at a time through `uploadChunk`
     - `finishFileUpload` is called when it's finished uploading
-    - Here is Nimbus's implementation of those Thrift methods: 
[code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/daemon/nimbus.clj#L694)
-  - Second, `StormSubmitter` calls `submitTopology` on the Nimbus thrift 
interface 
[code](https://github.com/apache/incubator-storm/blob/0.7.1/src/jvm/backtype/storm/StormSubmitter.java#L60)
+    - Here is Nimbus's implementation of those Thrift methods: 
[code](https://github.com/apache/storm/blob/0.7.1/src/clj/backtype/storm/daemon/nimbus.clj#L694)
+  - Second, `StormSubmitter` calls `submitTopology` on the Nimbus thrift 
interface 
[code](https://github.com/apache/storm/blob/0.7.1/src/jvm/backtype/storm/StormSubmitter.java#L60)
     - The topology config is serialized using JSON (JSON is used so that 
writing DSL's in any language is as easy as possible)
     - Notice that the Thrift `submitTopology` call takes in the Nimbus inbox 
path where the jar was uploaded
 
-- Nimbus receives the topology submission. 
[code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/daemon/nimbus.clj#L639)
-- Nimbus normalizes the topology configuration. The main purpose of 
normalization is to ensure that every single task will have the same 
serialization registrations, which is critical for getting serialization 
working correctly. 
[code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/daemon/nimbus.clj#L557)
 
-- Nimbus sets up the static state for the topology 
[code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/daemon/nimbus.clj#L661)
+- Nimbus receives the topology submission. 
[code](https://github.com/apache/storm/blob/0.7.1/src/clj/backtype/storm/daemon/nimbus.clj#L639)
+- Nimbus normalizes the topology configuration. The main purpose of 
normalization is to ensure that every single task will have the same 
serialization registrations, which is critical for getting serialization 
working correctly. 
[code](https://github.com/apache/storm/blob/0.7.1/src/clj/backtype/storm/daemon/nimbus.clj#L557)
+- Nimbus sets up the static state for the topology 
[code](https://github.com/apache/storm/blob/0.7.1/src/clj/backtype/storm/daemon/nimbus.clj#L661)
     - Jars and configs are kept on local filesystem because they're too big 
for Zookeeper. The jar and configs are copied into the path {nimbus local 
dir}/stormdist/{topology id}
     - `setup-storm-static` writes task -> component mapping into ZK
     - `setup-heartbeats` creates a ZK "directory" in which tasks can heartbeat
-- Nimbus calls `mk-assignment` to assign tasks to machines 
[code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/daemon/nimbus.clj#L458)
 
-    - Assignment record definition is here: 
[code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/daemon/common.clj#L25)
+- Nimbus calls `mk-assignment` to assign tasks to machines 
[code](https://github.com/apache/storm/blob/0.7.1/src/clj/backtype/storm/daemon/nimbus.clj#L458)
+    - Assignment record definition is here: 
[code](https://github.com/apache/storm/blob/0.7.1/src/clj/backtype/storm/daemon/common.clj#L25)
     - Assignment contains:
       - `master-code-dir`: used by supervisors to download the correct 
jars/configs for the topology from Nimbus
       - `task->node+port`: Map from a task id to the worker that task should 
be running on. (A worker is identified by a node/port pair)
       - `node->host`: A map from node id to hostname. This is used so workers 
know which machines to connect to to communicate with other workers. Node ids 
are used to identify supervisors so that multiple supervisors can be run on one 
machine. One place this is done is with Mesos integration.
       - `task->start-time-secs`: Contains a map from task id to the timestamp 
at which Nimbus launched that task. This is used by Nimbus when monitoring 
topologies, as tasks are given a longer timeout to heartbeat when they're first 
launched (the launch timeout is configured by "nimbus.task.launch.secs" config)
-- Once topologies are assigned, they're initially in a deactivated mode. 
`start-storm` writes data into Zookeeper so that the cluster knows the topology 
is active and can start emitting tuples from spouts. 
[code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/daemon/nimbus.clj#L504)
+- Once topologies are assigned, they're initially in a deactivated mode. 
`start-storm` writes data into Zookeeper so that the cluster knows the topology 
is active and can start emitting tuples from spouts. 
[code](https://github.com/apache/storm/blob/0.7.1/src/clj/backtype/storm/daemon/nimbus.clj#L504)
 
 - TODO cluster state diagram (show all nodes and what's kept everywhere)
 
 - Supervisor runs two functions in the background:
-    - `synchronize-supervisor`: This is called whenever assignments in 
Zookeeper change and also every 10 seconds. 
[code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/daemon/supervisor.clj#L241)
-      - Downloads code from Nimbus for topologies assigned to this machine for 
which it doesn't have the code yet. 
[code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/daemon/supervisor.clj#L258)
-      - Writes into local filesystem what this node is supposed to be running. 
It writes a map from port -> LocalAssignment. LocalAssignment contains a 
topology id as well as the list of task ids for that worker. 
[code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/daemon/supervisor.clj#L13)
-    - `sync-processes`: Reads from the LFS what `synchronize-supervisor` wrote 
and compares that to what's actually running on the machine. It then 
starts/stops worker processes as necessary to synchronize. 
[code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/daemon/supervisor.clj#L177)
+    - `synchronize-supervisor`: This is called whenever assignments in 
Zookeeper change and also every 10 seconds. 
[code](https://github.com/apache/storm/blob/0.7.1/src/clj/backtype/storm/daemon/supervisor.clj#L241)
+      - Downloads code from Nimbus for topologies assigned to this machine for 
which it doesn't have the code yet. 
[code](https://github.com/apache/storm/blob/0.7.1/src/clj/backtype/storm/daemon/supervisor.clj#L258)
+      - Writes into local filesystem what this node is supposed to be running. 
It writes a map from port -> LocalAssignment. LocalAssignment contains a 
topology id as well as the list of task ids for that worker. 
[code](https://github.com/apache/storm/blob/0.7.1/src/clj/backtype/storm/daemon/supervisor.clj#L13)
+    - `sync-processes`: Reads from the LFS what `synchronize-supervisor` wrote 
and compares that to what's actually running on the machine. It then 
starts/stops worker processes as necessary to synchronize. 
[code](https://github.com/apache/storm/blob/0.7.1/src/clj/backtype/storm/daemon/supervisor.clj#L177)
     
-- Worker processes start up through the `mk-worker` function 
[code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/daemon/worker.clj#L67)
-  - Worker connects to other workers and starts a thread to monitor for 
changes. So if a worker gets reassigned, the worker will automatically 
reconnect to the other worker's new location. 
[code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/daemon/worker.clj#L123)
-  - Monitors whether a topology is active or not and stores that state in the 
`storm-active-atom` variable. This variable is used by tasks to determine 
whether or not to call `nextTuple` on the spouts. 
[code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/daemon/worker.clj#L155)
-  - The worker launches the actual tasks as threads within it 
[code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/daemon/worker.clj#L178)
-- Tasks are set up through the `mk-task` function 
[code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/daemon/task.clj#L160)
-  - Tasks set up routing function which takes in a stream and an output tuple 
and returns a list of task ids to send the tuple to 
[code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/daemon/task.clj#L207)
 (there's also a 3-arity version used for direct streams)
-  - Tasks set up the spout-specific or bolt-specific code with 
[code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/daemon/task.clj#L241)
+- Worker processes start up through the `mk-worker` function 
[code](https://github.com/apache/storm/blob/0.7.1/src/clj/backtype/storm/daemon/worker.clj#L67)
+  - Worker connects to other workers and starts a thread to monitor for 
changes. So if a worker gets reassigned, the worker will automatically 
reconnect to the other worker's new location. 
[code](https://github.com/apache/storm/blob/0.7.1/src/clj/backtype/storm/daemon/worker.clj#L123)
+  - Monitors whether a topology is active or not and stores that state in the 
`storm-active-atom` variable. This variable is used by tasks to determine 
whether or not to call `nextTuple` on the spouts. 
[code](https://github.com/apache/storm/blob/0.7.1/src/clj/backtype/storm/daemon/worker.clj#L155)
+  - The worker launches the actual tasks as threads within it 
[code](https://github.com/apache/storm/blob/0.7.1/src/clj/backtype/storm/daemon/worker.clj#L178)
+- Tasks are set up through the `mk-task` function 
[code](https://github.com/apache/storm/blob/0.7.1/src/clj/backtype/storm/daemon/task.clj#L160)
+  - Tasks set up routing function which takes in a stream and an output tuple 
and returns a list of task ids to send the tuple to 
[code](https://github.com/apache/storm/blob/0.7.1/src/clj/backtype/storm/daemon/task.clj#L207)
 (there's also a 3-arity version used for direct streams)
+  - Tasks set up the spout-specific or bolt-specific code with 
[code](https://github.com/apache/storm/blob/0.7.1/src/clj/backtype/storm/daemon/task.clj#L241)
    
 ## Topology Monitoring
 
 - Nimbus monitors the topology during its lifetime
-   - Schedules recurring task on the timer thread to check the topologies 
[code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/daemon/nimbus.clj#L623)
-   - Nimbus's behavior is represented as a finite state machine 
[code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/daemon/nimbus.clj#L98)
-   - The "monitor" event is called on a topology every 
"nimbus.monitor.freq.secs", which calls `reassign-topology` through 
`reassign-transition` 
[code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/daemon/nimbus.clj#L497)
+   - Schedules recurring task on the timer thread to check the topologies 
[code](https://github.com/apache/storm/blob/0.7.1/src/clj/backtype/storm/daemon/nimbus.clj#L623)
+   - Nimbus's behavior is represented as a finite state machine 
[code](https://github.com/apache/storm/blob/0.7.1/src/clj/backtype/storm/daemon/nimbus.clj#L98)
+   - The "monitor" event is called on a topology every 
"nimbus.monitor.freq.secs", which calls `reassign-topology` through 
`reassign-transition` 
[code](https://github.com/apache/storm/blob/0.7.1/src/clj/backtype/storm/daemon/nimbus.clj#L497)
    - `reassign-topology` calls `mk-assignments`, the same function used to 
assign the topology the first time. `mk-assignments` is also capable of 
incrementally updating a topology
       - `mk-assignments` checks heartbeats and reassigns workers as necessary
       - Any reassignments change the state in ZK, which will trigger 
supervisors to synchronize and start/stop workers
       
 ## Killing a topology
 
-- "storm kill" command runs this code which just calls the Nimbus Thrift 
interface to kill the topology: 
[code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/command/kill_topology.clj)
-- Nimbus receives the kill command 
[code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/daemon/nimbus.clj#L671)
-- Nimbus applies the "kill" transition to the topology 
[code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/daemon/nimbus.clj#L676)
-- The kill transition function changes the status of the topology to "killed" 
and schedules the "remove" event to run "wait time seconds" in the future. 
[code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/daemon/nimbus.clj#L63)
+- "storm kill" command runs this code which just calls the Nimbus Thrift 
interface to kill the topology: 
[code](https://github.com/apache/storm/blob/0.7.1/src/clj/backtype/storm/command/kill_topology.clj)
+- Nimbus receives the kill command 
[code](https://github.com/apache/storm/blob/0.7.1/src/clj/backtype/storm/daemon/nimbus.clj#L671)
+- Nimbus applies the "kill" transition to the topology 
[code](https://github.com/apache/storm/blob/0.7.1/src/clj/backtype/storm/daemon/nimbus.clj#L676)
+- The kill transition function changes the status of the topology to "killed" 
and schedules the "remove" event to run "wait time seconds" in the future. 
[code](https://github.com/apache/storm/blob/0.7.1/src/clj/backtype/storm/daemon/nimbus.clj#L63)
    - The wait time defaults to the topology message timeout but can be 
overridden with the -w flag in the "storm kill" command
    - This causes the topology to be deactivated for the wait time before its 
actually shut down. This gives the topology a chance to finish processing what 
it's currently processing before shutting down the workers
-   - Changing the status during the kill transition ensures that the kill 
protocol is fault-tolerant to Nimbus crashing. On startup, if the status of the 
topology is "killed", Nimbus schedules the remove event to run "wait time 
seconds" in the future 
[code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/daemon/nimbus.clj#L111)
-- Removing a topology cleans out the assignment and static information from ZK 
[code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/daemon/nimbus.clj#L116)
-- A separate cleanup thread runs the `do-cleanup` function which will clean up 
the heartbeat dir and the jars/configs stored locally. 
[code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/daemon/nimbus.clj#L577)
+   - Changing the status during the kill transition ensures that the kill 
protocol is fault-tolerant to Nimbus crashing. On startup, if the status of the 
topology is "killed", Nimbus schedules the remove event to run "wait time 
seconds" in the future 
[code](https://github.com/apache/storm/blob/0.7.1/src/clj/backtype/storm/daemon/nimbus.clj#L111)
+- Removing a topology cleans out the assignment and static information from ZK 
[code](https://github.com/apache/storm/blob/0.7.1/src/clj/backtype/storm/daemon/nimbus.clj#L116)
+- A separate cleanup thread runs the `do-cleanup` function which will clean up 
the heartbeat dir and the jars/configs stored locally. 
[code](https://github.com/apache/storm/blob/0.7.1/src/clj/backtype/storm/daemon/nimbus.clj#L577)

http://git-wip-us.apache.org/repos/asf/storm/blob/d909db8f/docs/Local-mode.md
----------------------------------------------------------------------
diff --git a/docs/Local-mode.md b/docs/Local-mode.md
index 1f98e36..f871a73 100644
--- a/docs/Local-mode.md
+++ b/docs/Local-mode.md
@@ -1,5 +1,7 @@
 ---
+title: Local Mode
 layout: documentation
+documentation: true
 ---
 Local mode simulates a Storm cluster in process and is useful for developing 
and testing topologies. Running topologies in local mode is similar to running 
topologies [on a cluster](Running-topologies-on-a-production-cluster.html). 
 

http://git-wip-us.apache.org/repos/asf/storm/blob/d909db8f/docs/Maven.md
----------------------------------------------------------------------
diff --git a/docs/Maven.md b/docs/Maven.md
index 85828da..0c09c2c 100644
--- a/docs/Maven.md
+++ b/docs/Maven.md
@@ -1,56 +1,22 @@
 ---
+title: Maven
 layout: documentation
+documentation: true
 ---
-To develop topologies, you'll need the Storm jars on your classpath. You 
should either include the unpacked jars in the classpath for your project or 
use Maven to include Storm as a development dependency. Storm is hosted on 
Clojars (a Maven repository). To include Storm in your project as a development 
dependency, add the following to your pom.xml:
+To develop topologies, you'll need the Storm jars on your classpath. You 
should either include the unpacked jars in the classpath for your project or 
use Maven to include Storm as a development dependency. Storm is hosted on 
Maven Central. To include Storm in your project as a development dependency, 
add the following to your pom.xml:
 
-```xml
-<repository>
-  <id>clojars.org</id>
-  <url>http://clojars.org/repo</url>
-</repository>
-```
 
 ```xml
 <dependency>
-  <groupId>storm</groupId>
-  <artifactId>storm</artifactId>
-  <version>0.7.2</version>
-  <scope>test</scope>
+  <groupId>org.apache.storm</groupId>
+  <artifactId>storm-core</artifactId>
+  <version>{{page.version}}</version>
+  <scope>provided</scope>
 </dependency>
 ```
 
-[Here's an 
example](https://github.com/nathanmarz/storm-starter/blob/master/m2-pom.xml) of 
a pom.xml for a Storm project.
-
-If Maven isn't your thing, check out 
[leiningen](https://github.com/technomancy/leiningen). Leiningen is a build 
tool for Clojure, but it can be used for pure Java projects as well. Leiningen 
makes builds and dependency management using Maven dead-simple. Here's an 
example project.clj for a pure-Java Storm project:
-
-```clojure
-(defproject storm-starter "0.0.1-SNAPSHOT"
-  :java-source-path "src/jvm"
-  :javac-options {:debug "true" :fork "true"}
-  :jvm-opts ["-Djava.library.path=/usr/local/lib:/opt/local/lib:/usr/lib"]
-  :dependencies []
-  :dev-dependencies [
-                     [storm "0.7.2"]
-                     ])
-```
-
-You can fetch dependencies using `lein deps`, build the project with `lein 
compile`, and make a jar suitable for submitting to a cluster with `lein 
uberjar`.
-
-### Using Storm as a library
-
-If you want to use Storm as a library (e.g., use the Distributed RPC client) 
and have the Storm dependency jars be distributed with your application, 
there's a separate Maven dependency called "storm/storm-lib". The only 
difference between this dependency and the usual "storm/storm" is that 
storm-lib does not have any logging configured.
+[Here's an example]({{page.git-blob-base}}/examples/storm-starter/pom.xml) of 
a pom.xml for a Storm project.
 
 ### Developing Storm
 
-You will want to
-
-       bash ./bin/install_zmq.sh   # install the jzmq dependency
-       lein sub install
-
-Build javadocs with
-
-       bash ./bin/javadoc.sh
-
-### Building a Storm Release
-
-Use the file `bin/build_release.sh` to make a zipfile like the ones you would 
download (and like what the bin files require in order to run daemons).
+Please refer to [DEVELOPER.md]({{page.git-blob-base}}/DEVELOPER.md) for more 
details.

http://git-wip-us.apache.org/repos/asf/storm/blob/d909db8f/docs/Message-passing-implementation.md
----------------------------------------------------------------------
diff --git a/docs/Message-passing-implementation.md 
b/docs/Message-passing-implementation.md
index f22a5aa..a17f66a 100644
--- a/docs/Message-passing-implementation.md
+++ b/docs/Message-passing-implementation.md
@@ -1,28 +1,30 @@
 ---
+title: Message Passing Implementation
 layout: documentation
+documentation: true
 ---
 (Note: this walkthrough is out of date as of 0.8.0. 0.8.0 revamped the message 
passing infrastructure to be based on the Disruptor)
 
 This page walks through how emitting and transferring tuples works in Storm.
 
 - Worker is responsible for message transfer
-   - `refresh-connections` is called every "task.refresh.poll.secs" or 
whenever assignment in ZK changes. It manages connections to other workers and 
maintains a mapping from task -> worker 
[code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/daemon/worker.clj#L123)
-   - Provides a "transfer function" that is used by tasks to send tuples to 
other tasks. The transfer function takes in a task id and a tuple, and it 
serializes the tuple and puts it onto a "transfer queue". There is a single 
transfer queue for each worker. 
[code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/daemon/worker.clj#L56)
-   - The serializer is thread-safe 
[code](https://github.com/apache/incubator-storm/blob/0.7.1/src/jvm/backtype/storm/serialization/KryoTupleSerializer.java#L26)
-   - The worker has a single thread which drains the transfer queue and sends 
the messages to other workers 
[code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/daemon/worker.clj#L185)
-   - Message sending happens through this protocol: 
[code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/messaging/protocol.clj)
-   - The implementation for distributed mode uses ZeroMQ 
[code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/messaging/zmq.clj)
-   - The implementation for local mode uses in memory Java queues (so that 
it's easy to use Storm locally without needing to get ZeroMQ installed) 
[code](https://github.com/apache/incubator-storm/blob/0.7.1/src/clj/backtype/storm/messaging/local.clj)
+   - `refresh-connections` is called every "task.refresh.poll.secs" or 
whenever assignment in ZK changes. It manages connections to other workers and 
maintains a mapping from task -> worker 
[code](https://github.com/apache/storm/blob/0.7.1/src/clj/backtype/storm/daemon/worker.clj#L123)
+   - Provides a "transfer function" that is used by tasks to send tuples to 
other tasks. The transfer function takes in a task id and a tuple, and it 
serializes the tuple and puts it onto a "transfer queue". There is a single 
transfer queue for each worker. 
[code](https://github.com/apache/storm/blob/0.7.1/src/clj/backtype/storm/daemon/worker.clj#L56)
+   - The serializer is thread-safe 
[code](https://github.com/apache/storm/blob/0.7.1/src/jvm/backtype/storm/serialization/KryoTupleSerializer.java#L26)
+   - The worker has a single thread which drains the transfer queue and sends 
the messages to other workers 
[code](https://github.com/apache/storm/blob/0.7.1/src/clj/backtype/storm/daemon/worker.clj#L185)
+   - Message sending happens through this protocol: 
[code](https://github.com/apache/storm/blob/0.7.1/src/clj/backtype/storm/messaging/protocol.clj)
+   - The implementation for distributed mode uses ZeroMQ 
[code](https://github.com/apache/storm/blob/0.7.1/src/clj/backtype/storm/messaging/zmq.clj)
+   - The implementation for local mode uses in memory Java queues (so that 
it's easy to use Storm locally without needing to get ZeroMQ installed) 
[code](https://github.com/apache/storm/blob/0.7.1/src/clj/backtype/storm/messaging/local.clj)
 - Receiving messages in tasks works differently in local mode and distributed 
mode
-   - In local mode, the tuple is sent directly to an in-memory queue for the 
receiving task 
[code](https://github.com/apache/incubator-storm/blob/master/src/clj/backtype/storm/messaging/local.clj#L21)
-   - In distributed mode, each worker listens on a single TCP port for 
incoming messages and then routes those messages in-memory to tasks. The TCP 
port is called a "virtual port", because it receives [task id, message] and 
then routes it to the actual task. 
[code](https://github.com/apache/incubator-storm/blob/master/src/clj/backtype/storm/daemon/worker.clj#L204)
-      - The virtual port implementation is here: 
[code](https://github.com/apache/incubator-storm/blob/master/src/clj/zilch/virtual_port.clj)
-      - Tasks listen on an in-memory ZeroMQ port for messages from the virtual 
port 
[code](https://github.com/apache/incubator-storm/blob/master/src/clj/backtype/storm/daemon/task.clj#L201)
-        - Bolts listen here: 
[code](https://github.com/apache/incubator-storm/blob/master/src/clj/backtype/storm/daemon/task.clj#L489)
-        - Spouts listen here: 
[code](https://github.com/apache/incubator-storm/blob/master/src/clj/backtype/storm/daemon/task.clj#L382)
+   - In local mode, the tuple is sent directly to an in-memory queue for the 
receiving task 
[code](https://github.com/apache/storm/blob/0.7.1/src/clj/backtype/storm/messaging/local.clj#L21)
+   - In distributed mode, each worker listens on a single TCP port for 
incoming messages and then routes those messages in-memory to tasks. The TCP 
port is called a "virtual port", because it receives [task id, message] and 
then routes it to the actual task. 
[code](https://github.com/apache/storm/blob/0.7.1/src/clj/backtype/storm/daemon/worker.clj#L204)
+      - The virtual port implementation is here: 
[code](https://github.com/apache/storm/blob/0.7.1/src/clj/zilch/virtual_port.clj)
+      - Tasks listen on an in-memory ZeroMQ port for messages from the virtual 
port 
[code](https://github.com/apache/storm/blob/0.7.1/src/clj/backtype/storm/daemon/task.clj#L201)
+        - Bolts listen here: 
[code](https://github.com/apache/storm/blob/0.7.1/src/clj/backtype/storm/daemon/task.clj#L489)
+        - Spouts listen here: 
[code](https://github.com/apache/storm/blob/0.7.1/src/clj/backtype/storm/daemon/task.clj#L382)
 - Tasks are responsible for message routing. A tuple is emitted either to a 
direct stream (where the task id is specified) or a regular stream. In direct 
streams, the message is only sent if that bolt subscribes to that direct 
stream. In regular streams, the stream grouping functions are used to determine 
the task ids to send the tuple to.
-  - Tasks have a routing map from {stream id} -> {component id} -> {stream 
grouping function} 
[code](https://github.com/apache/incubator-storm/blob/master/src/clj/backtype/storm/daemon/task.clj#L198)
-  - The "tasks-fn" returns the task ids to send the tuples to for either 
regular stream emit or direct stream emit 
[code](https://github.com/apache/incubator-storm/blob/master/src/clj/backtype/storm/daemon/task.clj#L207)
+  - Tasks have a routing map from {stream id} -> {component id} -> {stream 
grouping function} 
[code](https://github.com/apache/storm/blob/0.7.1/src/clj/backtype/storm/daemon/task.clj#L198)
+  - The "tasks-fn" returns the task ids to send the tuples to for either 
regular stream emit or direct stream emit 
[code](https://github.com/apache/storm/blob/0.7.1/src/clj/backtype/storm/daemon/task.clj#L207)
   - After getting the output task ids, bolts and spouts use the transfer-fn 
provided by the worker to actually transfer the tuples
-      - Bolt transfer code here: 
[code](https://github.com/apache/incubator-storm/blob/master/src/clj/backtype/storm/daemon/task.clj#L429)
-      - Spout transfer code here: 
[code](https://github.com/apache/incubator-storm/blob/master/src/clj/backtype/storm/daemon/task.clj#L329)
+      - Bolt transfer code here: 
[code](https://github.com/apache/storm/blob/0.7.1/src/clj/backtype/storm/daemon/task.clj#L429)
+      - Spout transfer code here: 
[code](https://github.com/apache/storm/blob/0.7.1/src/clj/backtype/storm/daemon/task.clj#L329)

http://git-wip-us.apache.org/repos/asf/storm/blob/d909db8f/docs/Metrics.md
----------------------------------------------------------------------
diff --git a/docs/Metrics.md b/docs/Metrics.md
index f43f8c7..b2521b1 100644
--- a/docs/Metrics.md
+++ b/docs/Metrics.md
@@ -1,5 +1,7 @@
 ---
+title: Storm Metrics
 layout: documentation
+documentation: true
 ---
 Storm exposes a metrics interface to report summary statistics across the full 
topology.
 It's used internally to track the numbers you see in the Nimbus UI console: 
counts of executes and acks; average process latency per bolt; worker heap 
usage; and so forth.
@@ -10,13 +12,13 @@ Metrics have to implement just one method, 
`getValueAndReset` -- do any remainin
 
 Storm gives you these metric types:
 
-* [AssignableMetric]() -- set the metric to the explicit value you supply. 
Useful if it's an external value or in the case that you are already 
calculating the summary statistic yourself.
-* 
[CombinedMetric](https://github.com/apache/incubator-storm/blob/master/storm-core/src/jvm/backtype/storm/metric/api/CombinedMetric.java)
 -- generic interface for metrics that can be updated associatively. 
-* 
[CountMetric](https://github.com/apache/incubator-storm/blob/master/storm-core/src/jvm/backtype/storm/metric/api/CountMetric.java)
 -- a running total of the supplied values. Call `incr()` to increment by one, 
`incrBy(n)` to add/subtract the given number.
-  - 
[MultiCountMetric](https://github.com/apache/incubator-storm/blob/master/storm-core/src/jvm/backtype/storm/metric/api/MultiCountMetric.java)
 -- a hashmap of count metrics.
-* 
[ReducedMetric](https://github.com/apache/incubator-storm/blob/master/storm-core/src/jvm/backtype/storm/metric/api/ReducedMetric.java)
-  - 
[MeanReducer](https://github.com/apache/incubator-storm/blob/master/storm-core/src/jvm/backtype/storm/metric/api/MeanReducer.java)
 -- track a running average of values given to its `reduce()` method. (It 
accepts `Double`, `Integer` or `Long` values, and maintains the internal 
average as a `Double`.) Despite his reputation, the MeanReducer is actually a 
pretty nice guy in person.
-  - 
[MultiReducedMetric](https://github.com/apache/incubator-storm/blob/master/storm-core/src/jvm/backtype/storm/metric/api/MultiReducedMetric.java)
 -- a hashmap of reduced metrics.
+* 
[AssignableMetric]({{page.git-blob-base}}/storm-core/src/jvm/backtype/storm/metric/api/AssignableMetric.java)
 -- set the metric to the explicit value you supply. Useful if it's an external 
value or in the case that you are already calculating the summary statistic 
yourself.
+* 
[CombinedMetric]({{page.git-blob-base}}/storm-core/src/jvm/backtype/storm/metric/api/CombinedMetric.java)
 -- generic interface for metrics that can be updated associatively. 
+* 
[CountMetric]({{page.git-blob-base}}/storm-core/src/jvm/backtype/storm/metric/api/CountMetric.java)
 -- a running total of the supplied values. Call `incr()` to increment by one, 
`incrBy(n)` to add/subtract the given number.
+  - 
[MultiCountMetric]({{page.git-blob-base}}/storm-core/src/jvm/backtype/storm/metric/api/MultiCountMetric.java)
 -- a hashmap of count metrics.
+* 
[ReducedMetric]({{page.git-blob-base}}/storm-core/src/jvm/backtype/storm/metric/api/ReducedMetric.java)
+  - 
[MeanReducer]({{page.git-blob-base}}/storm-core/src/jvm/backtype/storm/metric/api/MeanReducer.java)
 -- track a running average of values given to its `reduce()` method. (It 
accepts `Double`, `Integer` or `Long` values, and maintains the internal 
average as a `Double`.) Despite his reputation, the MeanReducer is actually a 
pretty nice guy in person.
+  - 
[MultiReducedMetric]({{page.git-blob-base}}/storm-core/src/jvm/backtype/storm/metric/api/MultiReducedMetric.java)
 -- a hashmap of reduced metrics.
 
 
 ### Metric Consumer
@@ -28,7 +30,7 @@ Storm gives you these metric types:
 
 ### Builtin Metrics
 
-The [builtin 
metrics](https://github.com/apache/incubator-storm/blob/46c3ba7/storm-core/src/clj/backtype/storm/daemon/builtin_metrics.clj)
 instrument Storm itself.
+The [builtin 
metrics]({{page.git-blob-base}}/storm-core/src/clj/backtype/storm/daemon/builtin_metrics.clj)
 instrument Storm itself.
 
-[builtin_metrics.clj](https://github.com/apache/incubator-storm/blob/46c3ba7/storm-core/src/clj/backtype/storm/daemon/builtin_metrics.clj)
 sets up data structures for the built-in metrics, and facade methods that the 
other framework components can use to update them. The metrics themselves are 
calculated in the calling code -- see for example 
[`ack-spout-msg`](https://github.com/apache/incubator-storm/blob/46c3ba7/storm-core/src/clj/backtype/storm/daemon/executor.clj#358)
  in `clj/b/s/daemon/daemon/executor.clj`
+[builtin_metrics.clj]({{page.git-blob-base}}/storm-core/src/clj/backtype/storm/daemon/builtin_metrics.clj)
 sets up data structures for the built-in metrics, and facade methods that the 
other framework components can use to update them. The metrics themselves are 
calculated in the calling code -- see for example 
[`ack-spout-msg`]({{page.git-blob-base}}/storm-core/src/clj/backtype/storm/daemon/executor.clj#358)
  in `clj/b/s/daemon/daemon/executor.clj`
 

http://git-wip-us.apache.org/repos/asf/storm/blob/d909db8f/docs/Multilang-protocol.md
----------------------------------------------------------------------
diff --git a/docs/Multilang-protocol.md b/docs/Multilang-protocol.md
index a3cb22c..2a90059 100644
--- a/docs/Multilang-protocol.md
+++ b/docs/Multilang-protocol.md
@@ -1,5 +1,7 @@
 ---
+title: Multi-Lang Protocol
 layout: documentation
+documentation: true
 ---
 This page explains the multilang protocol as of Storm 0.7.1. Versions prior to 
0.7.1 used a somewhat different protocol, documented 
[here](Storm-multi-language-protocol-(versions-0.7.0-and-below\).html).
 
@@ -47,7 +49,7 @@ STDIN and STDOUT.
 
 The initial handshake is the same for both types of shell components:
 
-* STDIN: Setup info. This is a JSON object with the Storm configuration, 
Topology context, and a PID directory, like this:
+* STDIN: Setup info. This is a JSON object with the Storm configuration, a PID 
directory, and a topology context, like this:
 
 ```
 {
@@ -55,15 +57,37 @@ The initial handshake is the same for both types of shell 
components:
         "topology.message.timeout.secs": 3,
         // etc
     },
+    "pidDir": "...",
     "context": {
         "task->component": {
             "1": "example-spout",
             "2": "__acker",
-            "3": "example-bolt"
+            "3": "example-bolt1",
+            "4": "example-bolt2"
         },
-        "taskid": 3
-    },
-    "pidDir": "..."
+        "taskid": 3,
+        // Everything below this line is only available in Storm 0.10.0+
+        "componentid": "example-bolt"
+        "stream->target->grouping": {
+               "default": {
+                       "example-bolt2": {
+                               "type": "SHUFFLE"}}},
+        "streams": ["default"],
+               "stream->outputfields": {"default": ["word"]},
+           "source->stream->grouping": {
+               "example-spout": {
+                       "default": {
+                               "type": "FIELDS",
+                               "fields": ["word"]
+                       }
+               }
+           }
+           "source->stream->fields": {
+               "example-spout": {
+                       "default": ["word"]
+               }
+           }
+       }
 }
 ```
 
@@ -71,6 +95,15 @@ Your script should create an empty file named with its PID 
in this directory. e.
 the PID is 1234, so an empty file named 1234 is created in the directory. This
 file lets the supervisor know the PID so it can shutdown the process later on.
 
+As of Storm 0.10.0, the context sent by Storm to shell components has been
+enhanced substantially to include all aspects of the topology context available
+to JVM components.  One key addition is the ability to determine a shell
+component's source and targets (i.e., inputs and outputs) in the topology via
+the `stream->target->grouping` and `source->stream->grouping` dictionaries.  At
+the innermost level of these nested dictionaries, groupings are represented as
+a dictionary that minimally has a `type` key, but can also have a `fields` key
+to specify which fields are involved in a `FIELDS` grouping.
+
 * STDOUT: Your PID, in a JSON object, like `{"pid": 1234}`. The shell 
component will log the PID to its log.
 
 What happens next depends on the type of component:
@@ -219,3 +252,36 @@ A "log" will log a message in the worker log. It looks 
like:
 
 * Note that, as of version 0.7.1, there is no longer any need for a
   shell bolt to 'sync'.
+
+### Handling Heartbeats (0.9.3 and later)
+
+As of Storm 0.9.3, heartbeats have been between ShellSpout/ShellBolt and their
+multi-lang subprocesses to detect hanging/zombie subprocesses.  Any libraries
+for interfacing with Storm via multi-lang must take the following actions
+regarding hearbeats:
+
+#### Spout
+
+Shell spouts are synchronous, so subprocesses always send `sync` commands at 
the
+end of `next()`,  so you should not have to do much to support heartbeats for
+spouts.  That said, you must not let subprocesses sleep more than the worker
+timeout during `next()`.
+
+#### Bolt
+
+Shell bolts are asynchronous, so a ShellBolt will send heartbeat tuples to its
+subprocess periodically.  Heartbeat tuple looks like:
+
+```
+{
+       "id": "-6955786537413359385",
+       "comp": "1",
+       "stream": "__heartbeat",
+       // this shell bolt's system task id
+       "task": -1,
+       "tuple": []
+}
+```
+
+When subprocess receives heartbeat tuple, it must send a `sync` command back to
+ShellBolt.

http://git-wip-us.apache.org/repos/asf/storm/blob/d909db8f/docs/Rationale.md
----------------------------------------------------------------------
diff --git a/docs/Rationale.md b/docs/Rationale.md
index 214266e..45ff396 100644
--- a/docs/Rationale.md
+++ b/docs/Rationale.md
@@ -1,5 +1,7 @@
 ---
+title: Rationale
 layout: documentation
+documentation: true
 ---
 The past decade has seen a revolution in data processing. MapReduce, Hadoop, 
and related technologies have made it possible to store and process data at 
scales previously unthinkable. Unfortunately, these data processing 
technologies are not realtime systems, nor are they meant to be. There's no 
hack that will turn Hadoop into a realtime system; realtime data processing has 
a fundamentally different set of requirements than batch processing.
 

Reply via email to