STORM-1617: added in 1.x specific doc changes

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

Branch: refs/heads/master
Commit: 69dfb5322f896057c21cf6bf758938cb14bc3984
Parents: d909db8
Author: Robert (Bobby) Evans <[email protected]>
Authored: Sat Mar 19 12:56:04 2016 -0500
Committer: Robert (Bobby) Evans <[email protected]>
Committed: Sat Mar 19 12:58:54 2016 -0500

----------------------------------------------------------------------
 docs/Clojure-DSL.md                             |    8 +-
 docs/Command-line-client.md                     |    2 +-
 docs/Common-patterns.md                         |    6 +-
 docs/Concepts.md                                |   42 +-
 docs/Configuration.md                           |    4 +-
 docs/Distributed-RPC.md                         |    2 +-
 docs/Guaranteeing-message-processing.md         |    8 +-
 docs/Hooks.md                                   |    6 +-
 docs/Implementation-docs.md                     |   10 +-
 docs/Kestrel-and-Storm.md                       |    8 +-
 docs/Lifecycle-of-a-topology.md                 |   66 +-
 docs/Local-mode.md                              |    6 +-
 docs/Logs.md                                    |   30 +
 docs/Message-passing-implementation.md          |   32 +-
 docs/Metrics.md                                 |   18 +-
 docs/Pacemaker.md                               |  113 +
 docs/Resource_Aware_Scheduler_overview.md       |  232 ++
 ...unning-topologies-on-a-production-cluster.md |    6 +-
 docs/SECURITY.md                                |   14 +-
 docs/STORM-UI-REST-API.md                       |  363 ++-
 docs/Serialization.md                           |    2 +-
 docs/State-checkpointing.md                     |  160 ++
 docs/Structure-of-the-codebase.md               |   88 +-
 docs/Transactional-topologies.md                |   20 +-
 docs/Trident-API-Overview.md                    |    4 +-
 docs/Trident-spouts.md                          |    8 +-
 docs/Trident-state.md                           |    8 +-
 docs/Troubleshooting.md                         |   56 +-
 docs/Tutorial.md                                |   12 +-
 ...nding-the-parallelism-of-a-Storm-topology.md |   14 +-
 docs/Windowing.md                               |  239 ++
 docs/_sass/_syntax-highlighting.scss            |   70 +
 docs/assets/css/bootstrap-theme.css             |  470 ++++
 docs/assets/css/bootstrap-theme.css.map         |    1 +
 docs/assets/css/bootstrap-theme.min.css         |    5 +
 docs/assets/css/bootstrap.min.css               |    5 +
 docs/assets/css/theme.css                       |   18 +
 docs/assets/favicon.ico                         |  Bin 0 -> 1150 bytes
 .../fonts/glyphicons-halflings-regular.eot      |  Bin 0 -> 20335 bytes
 .../fonts/glyphicons-halflings-regular.svg      |  229 ++
 .../fonts/glyphicons-halflings-regular.ttf      |  Bin 0 -> 41280 bytes
 .../fonts/glyphicons-halflings-regular.woff     |  Bin 0 -> 23320 bytes
 docs/assets/js/bootstrap.js                     | 2320 ++++++++++++++++++
 docs/assets/js/npm.js                           |   13 +
 docs/cgroups_in_storm.md                        |   71 +
 docs/distcache-blobstore.md                     |  740 ++++++
 docs/dynamic-log-level-settings.md              |   45 +
 docs/dynamic-worker-profiling.md                |   37 +
 docs/flux.md                                    |   48 +-
 docs/images/architecture.png                    |  Bin 0 -> 69825 bytes
 docs/images/architecture.svg                    | 1458 +++++++++++
 docs/images/bolt.png                            |  Bin 0 -> 24796 bytes
 docs/images/bolt.svg                            |  743 ++++++
 docs/images/bullet.gif                          |  Bin 0 -> 82 bytes
 docs/images/download.png                        |  Bin 0 -> 16272 bytes
 docs/images/dynamic_log_level_settings_1.png    |  Bin 0 -> 93689 bytes
 docs/images/dynamic_log_level_settings_2.png    |  Bin 0 -> 78785 bytes
 docs/images/dynamic_profiling_debugging_1.png   |  Bin 0 -> 56876 bytes
 docs/images/dynamic_profiling_debugging_2.png   |  Bin 0 -> 99164 bytes
 docs/images/dynamic_profiling_debugging_3.png   |  Bin 0 -> 96974 bytes
 docs/images/dynamic_profiling_debugging_4.png   |  Bin 0 -> 121994 bytes
 docs/images/hdfs_blobstore.png                  |  Bin 0 -> 82180 bytes
 docs/images/incubator-logo.png                  |  Bin 0 -> 11651 bytes
 docs/images/local_blobstore.png                 |  Bin 0 -> 81212 bytes
 docs/images/mailinglist.png                     |  Bin 0 -> 4245 bytes
 docs/images/nimbus_ha_blobstore.png             |  Bin 0 -> 113991 bytes
 .../nimbus_ha_leader_election_and_failover.png  |  Bin 0 -> 154316 bytes
 docs/images/nimbus_ha_topology_submission.png   |  Bin 0 -> 134180 bytes
 docs/images/search-a-topology.png               |  Bin 0 -> 671031 bytes
 docs/images/search-for-a-single-worker-log.png  |  Bin 0 -> 736579 bytes
 docs/images/security.png                        |  Bin 0 -> 72415 bytes
 docs/images/security.svg                        | 1779 ++++++++++++++
 docs/images/spout.png                           |  Bin 0 -> 22911 bytes
 docs/images/spout.svg                           |  833 +++++++
 docs/images/storm-sql-internal-example.png      |  Bin 0 -> 28377 bytes
 docs/images/storm-sql-internal-workflow.png     |  Bin 0 -> 20020 bytes
 docs/images/storm.svg                           | 1326 ++++++++++
 docs/images/storm_header.png                    |  Bin 0 -> 17291 bytes
 docs/images/storm_logo_tagline_color.png        |  Bin 0 -> 33568 bytes
 docs/images/top_bg.gif                          |  Bin 0 -> 113 bytes
 docs/images/topology.svg                        | 1044 ++++++++
 docs/images/topology_dark.png                   |  Bin 0 -> 49692 bytes
 docs/images/topology_dark.svg                   | 1101 +++++++++
 docs/images/ui_topology_viz.png                 |  Bin 0 -> 112831 bytes
 docs/index.md                                   |   24 +-
 docs/nimbus-ha-design.md                        |  222 ++
 docs/storm-kafka.md                             |    8 +-
 .../storm-metrics-profiling-internal-actions.md |   70 +
 docs/storm-solr.md                              |  184 ++
 docs/storm-sql-internal.md                      |   55 +
 docs/storm-sql.md                               |   97 +
 91 files changed, 14331 insertions(+), 272 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/69dfb532/docs/Clojure-DSL.md
----------------------------------------------------------------------
diff --git a/docs/Clojure-DSL.md b/docs/Clojure-DSL.md
index 234febe..816a5e3 100644
--- a/docs/Clojure-DSL.md
+++ b/docs/Clojure-DSL.md
@@ -3,7 +3,7 @@ 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]({{page.git-blob-base}}/storm-core/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 
[org.apache.storm.clojure]({{page.git-blob-base}}/storm-core/src/clj/org/apache/storm/clojure.clj)
 namespace.
 
 This page outlines all the pieces of the Clojure DSL, including:
 
@@ -38,11 +38,11 @@ The maps of spout and bolt specs are maps from the 
component id to the correspon
 
 #### 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.
+`spout-spec` takes as arguments the spout implementation (an object that 
implements [IRichSpout](javadocs/org/apache/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.
+`bolt-spec` takes as arguments the input declaration for the bolt, the bolt 
implementation (an object that implements 
[IRichBolt](javadocs/org/apache/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:
 
@@ -254,7 +254,7 @@ The following example illustrates how to use this spout in 
a `spout-spec`:
 
 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:
+To create topology configs, it's easiest to use the `org.apache.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

http://git-wip-us.apache.org/repos/asf/storm/blob/69dfb532/docs/Command-line-client.md
----------------------------------------------------------------------
diff --git a/docs/Command-line-client.md b/docs/Command-line-client.md
index 4634467..e97b49c 100644
--- a/docs/Command-line-client.md
+++ b/docs/Command-line-client.md
@@ -25,7 +25,7 @@ These commands are:
 
 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.
+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/org/apache/storm/StormSubmitter.html) will 
upload the jar at `topology-jar-path` when the topology is submitted.
 
 ### kill
 

http://git-wip-us.apache.org/repos/asf/storm/blob/69dfb532/docs/Common-patterns.md
----------------------------------------------------------------------
diff --git a/docs/Common-patterns.md b/docs/Common-patterns.md
index 1c97f6d..9f5ffe7 100644
--- a/docs/Common-patterns.md
+++ b/docs/Common-patterns.md
@@ -39,7 +39,7 @@ If you want reliability in your data processing, the right 
way to do this is to
 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.
+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/org/apache/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
 
@@ -87,11 +87,11 @@ The topology needs an extra layer of processing to 
aggregate the partial counts
 
 ### 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.
+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/org/apache/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.
+When building distributed RPC applications on top of Storm, there are two 
common patterns that are usually needed. These are encapsulated by 
[CoordinatedBolt](javadocs/org/apache/storm/task/CoordinatedBolt.html) and 
[KeyedFairBolt](javadocs/org/apache/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.
 

http://git-wip-us.apache.org/repos/asf/storm/blob/69dfb532/docs/Concepts.md
----------------------------------------------------------------------
diff --git a/docs/Concepts.md b/docs/Concepts.md
index 01dbd11..d46033c 100644
--- a/docs/Concepts.md
+++ b/docs/Concepts.md
@@ -21,7 +21,7 @@ The logic for a realtime application is packaged into a Storm 
topology. A Storm
 
 **Resources:**
 
-* [TopologyBuilder](javadocs/backtype/storm/topology/TopologyBuilder.html): 
use this class to construct topologies in Java
+* [TopologyBuilder](javadocs/org/apache/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.
 
@@ -29,28 +29,28 @@ The logic for a realtime application is packaged into a 
Storm topology. A Storm
 
 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".
+Every stream is given an id when declared. Since single-stream spouts and 
bolts are so common, 
[OutputFieldsDeclarer](javadocs/org/apache/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
+* [Tuple](javadocs/org/apache/storm/tuple/Tuple.html): streams are composed of 
tuples
+* 
[OutputFieldsDeclarer](javadocs/org/apache/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
 
 ### 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/org/apache/storm/topology/OutputFieldsDeclarer.html)
 and specify the stream to emit to when using the `emit` method on 
[SpoutOutputCollector](javadocs/org/apache/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.
+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/org/apache/storm/spout/ISpout.html) for more information.
 
 **Resources:**
 
-* [IRichSpout](javadocs/backtype/storm/topology/IRichSpout.html): this is the 
interface that spouts must implement.
+* [IRichSpout](javadocs/org/apache/storm/topology/IRichSpout.html): this is 
the interface that spouts must implement.
 * [Guaranteeing message processing](Guaranteeing-message-processing.html)
 
 ### Bolts
@@ -59,26 +59,26 @@ All processing in topologies is done in bolts. Bolts can do 
anything from filter
 
 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).
+Bolts can emit more than one stream. To do so, declare multiple streams using 
the `declareStream` method of 
[OutputFieldsDeclarer](javadocs/org/apache/storm/topology/OutputFieldsDeclarer.html)
 and specify the stream to emit to when using the `emit` method on 
[OutputCollector](javadocs/org/apache/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/org/apache/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.
+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/org/apache/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/org/apache/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.
+Its perfectly fine to launch new threads in bolts that do processing 
asynchronously. 
[OutputCollector](javadocs/org/apache/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
+* [IRichBolt](javadocs/org/apache/storm/topology/IRichBolt.html): this is 
general interface for bolts.
+* [IBasicBolt](javadocs/org/apache/storm/topology/IBasicBolt.html): this is a 
convenience interface for defining bolts that do filtering or simple functions.
+* [OutputCollector](javadocs/org/apache/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 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:
+There are eight built-in stream groupings in Storm, and you can implement a 
custom stream grouping by implementing the 
[CustomStreamGrouping](javadocs/org/apache/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.
@@ -86,25 +86,25 @@ There are eight built-in stream groupings in Storm, and you 
can implement a cust
 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).
+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/org/apache/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/org/apache/storm/task/TopologyContext.html) or by 
keeping track of the output of the `emit` method in 
[OutputCollector](javadocs/org/apache/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
+* [TopologyBuilder](javadocs/org/apache/storm/topology/TopologyBuilder.html): 
use this class to define topologies
+* [InputDeclarer](javadocs/org/apache/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
 
 ### 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.
+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/org/apache/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).
+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/org/apache/storm/topology/TopologyBuilder.html).
 
 ### Workers
 
@@ -112,4 +112,4 @@ Topologies execute across one or more worker processes. 
Each worker process is a
 
 **Resources:**
 
-* 
[Config.TOPOLOGY_WORKERS](javadocs/backtype/storm/Config.html#TOPOLOGY_WORKERS):
 this config sets the number of workers to allocate for executing the topology
+* 
[Config.TOPOLOGY_WORKERS](javadocs/org/apache/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/69dfb532/docs/Configuration.md
----------------------------------------------------------------------
diff --git a/docs/Configuration.md b/docs/Configuration.md
index 83f4ef7..979ac9a 100644
--- a/docs/Configuration.md
+++ b/docs/Configuration.md
@@ -5,7 +5,7 @@ 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]({{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".
+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/org/apache/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:
 
@@ -24,7 +24,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
+* [Config](javadocs/org/apache/storm/Config.html): a listing of all 
configurations as well as a helper class for creating topology specific 
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

http://git-wip-us.apache.org/repos/asf/storm/blob/69dfb532/docs/Distributed-RPC.md
----------------------------------------------------------------------
diff --git a/docs/Distributed-RPC.md b/docs/Distributed-RPC.md
index 4af2702..2ad63e5 100644
--- a/docs/Distributed-RPC.md
+++ b/docs/Distributed-RPC.md
@@ -24,7 +24,7 @@ A client sends the DRPC server the name of the function to 
execute and the argum
 
 ### 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:
+Storm comes with a topology builder called 
[LinearDRPCTopologyBuilder](javadocs/org/apache/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

http://git-wip-us.apache.org/repos/asf/storm/blob/69dfb532/docs/Guaranteeing-message-processing.md
----------------------------------------------------------------------
diff --git a/docs/Guaranteeing-message-processing.md 
b/docs/Guaranteeing-message-processing.md
index 932ff0d..0e40536 100644
--- a/docs/Guaranteeing-message-processing.md
+++ b/docs/Guaranteeing-message-processing.md
@@ -26,11 +26,11 @@ This topology reads sentences off of a Kestrel queue, 
splits the sentences into
 
 ![Tuple tree](images/tuple_tree.png)
 
-Storm considers a tuple coming off a spout "fully processed" when the tuple 
tree has been exhausted and every message in the tree has been processed. A 
tuple is considered failed when its tree of messages fails to be fully 
processed within a specified timeout. This timeout can be configured on a 
topology-specific basis using the 
[Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS](javadocs/backtype/storm/Config.html#TOPOLOGY_MESSAGE_TIMEOUT_SECS)
 configuration and defaults to 30 seconds.
+Storm considers a tuple coming off a spout "fully processed" when the tuple 
tree has been exhausted and every message in the tree has been processed. A 
tuple is considered failed when its tree of messages fails to be fully 
processed within a specified timeout. This timeout can be configured on a 
topology-specific basis using the 
[Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS](javadocs/org/apache/storm/Config.html#TOPOLOGY_MESSAGE_TIMEOUT_SECS)
 configuration and defaults to 30 seconds.
 
 ### What happens if a message is fully processed or fails to be fully 
processed?
 
-To understand this question, let's take a look at the lifecycle of a tuple 
coming off of a spout. For reference, here is the interface that spouts 
implement (see the [Javadoc](javadocs/backtype/storm/spout/ISpout.html) for 
more information):
+To understand this question, let's take a look at the lifecycle of a tuple 
coming off of a spout. For reference, here is the interface that spouts 
implement (see the [Javadoc](javadocs/org/apache/storm/spout/ISpout.html) for 
more information):
 
 ```java
 public interface ISpout extends Serializable {
@@ -132,11 +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." 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.
+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/org/apache/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 per worker.
+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/org/apache/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/69dfb532/docs/Hooks.md
----------------------------------------------------------------------
diff --git a/docs/Hooks.md b/docs/Hooks.md
index 01cfa92..3c85407 100644
--- a/docs/Hooks.md
+++ b/docs/Hooks.md
@@ -3,7 +3,7 @@ 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:
+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/org/apache/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](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.
+1. In the open method of your spout or prepare method of your bolt using the 
[TopologyContext](javadocs/org/apache/storm/task/TopologyContext.html#addTaskHook)
 method.
+2. Through the Storm configuration using the 
["topology.auto.task.hooks"](javadocs/org/apache/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/69dfb532/docs/Implementation-docs.md
----------------------------------------------------------------------
diff --git a/docs/Implementation-docs.md b/docs/Implementation-docs.md
index 15f088e..9eb91f5 100644
--- a/docs/Implementation-docs.md
+++ b/docs/Implementation-docs.md
@@ -9,11 +9,5 @@ This section of the wiki is dedicated to explaining how Storm 
is implemented. Yo
 - [Lifecycle of a topology](Lifecycle-of-a-topology.html)
 - [Message passing implementation](Message-passing-implementation.html)
 - [Metrics](Metrics.html)
-- How transactional topologies work
-   - subtopology for TransactionalSpout
-   - how state is stored in ZK
-   - subtleties around what to do when emitting batches out of order
-- Unit testing
-  - time simulation
-  - complete-topology
-  - tracker clusters
+- [Nimbus HA](nimbus-ha-design.html)
+- [Storm SQL](storm-sql-internal.html)

http://git-wip-us.apache.org/repos/asf/storm/blob/69dfb532/docs/Kestrel-and-Storm.md
----------------------------------------------------------------------
diff --git a/docs/Kestrel-and-Storm.md b/docs/Kestrel-and-Storm.md
index cb80139..cd584ff 100644
--- a/docs/Kestrel-and-Storm.md
+++ b/docs/Kestrel-and-Storm.md
@@ -12,7 +12,7 @@ This tutorial uses examples from the 
[storm-kestrel](https://github.com/nathanma
 It assumes you are able to run locally a Kestrel server as described 
[here](https://github.com/nathanmarz/storm-kestrel).
 
 ## Kestrel Server and Queue
-A single kestrel server has a set of queues. A Kestrel queue is a very simple 
message queue that runs on the JVM and uses the memcache protocol (with some 
extensions) to talk to clients. For details, look at the implementation of the 
[KestrelThriftClient](https://github.com/nathanmarz/storm-kestrel/blob/master/src/jvm/backtype/storm/spout/KestrelThriftClient.java)
 class provided in [storm-kestrel](https://github.com/nathanmarz/storm-kestrel) 
project.
+A single kestrel server has a set of queues. A Kestrel queue is a very simple 
message queue that runs on the JVM and uses the memcache protocol (with some 
extensions) to talk to clients. For details, look at the implementation of the 
[KestrelThriftClient](https://github.com/nathanmarz/storm-kestrel/blob/master/src/jvm/org/apache/storm/spout/KestrelThriftClient.java)
 class provided in [storm-kestrel](https://github.com/nathanmarz/storm-kestrel) 
project.
 
 Each queue is strictly ordered following the FIFO (first in, first out) 
principle. To keep up with performance items are cached in system memory; 
though, only the first 128MB is kept in memory. When stopping the server, the 
queue state is stored in a journal file.
 
@@ -120,9 +120,9 @@ In order to stop it type a closing bracket char ']' in 
console and hit 'Enter'.
     import java.io.InputStream;
     import java.util.Random;
 
-    import backtype.storm.spout.KestrelClient;
-    import backtype.storm.spout.KestrelClient.Item;
-    import backtype.storm.spout.KestrelClient.ParseError;
+    import org.apache.storm.spout.KestrelClient;
+    import org.apache.storm.spout.KestrelClient.Item;
+    import org.apache.storm.spout.KestrelClient.ParseError;
 
     public class AddSentenceItemsToKestrel {
 

http://git-wip-us.apache.org/repos/asf/storm/blob/69dfb532/docs/Lifecycle-of-a-topology.md
----------------------------------------------------------------------
diff --git a/docs/Lifecycle-of-a-topology.md b/docs/Lifecycle-of-a-topology.md
index 6436206..dba1457 100644
--- a/docs/Lifecycle-of-a-topology.md
+++ b/docs/Lifecycle-of-a-topology.md
@@ -9,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/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/org/apache/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/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)
+  - when Nimbus is creating tasks for the topology 
[code](https://github.com/apache/storm/blob/0.7.1/src/clj/org/apache/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/org/apache/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/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/storm/blob/0.7.1/src/jvm/backtype/storm/StormSubmitter.java#L83)
+  - First, `StormSubmitter` uploads the jar if it hasn't been uploaded before. 
[code](https://github.com/apache/storm/blob/0.7.1/src/jvm/org/apache/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/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)
+    - Here is Nimbus's implementation of those Thrift methods: 
[code](https://github.com/apache/storm/blob/0.7.1/src/clj/org/apache/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/org/apache/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/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)
+- Nimbus receives the topology submission. 
[code](https://github.com/apache/storm/blob/0.7.1/src/clj/org/apache/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/org/apache/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/org/apache/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/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)
+- Nimbus calls `mk-assignment` to assign tasks to machines 
[code](https://github.com/apache/storm/blob/0.7.1/src/clj/org/apache/storm/daemon/nimbus.clj#L458)
+    - Assignment record definition is here: 
[code](https://github.com/apache/storm/blob/0.7.1/src/clj/org/apache/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/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/org/apache/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/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)
+    - `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/org/apache/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/org/apache/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/org/apache/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/org/apache/storm/daemon/supervisor.clj#L177)
     
-- 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)
+- Worker processes start up through the `mk-worker` function 
[code](https://github.com/apache/storm/blob/0.7.1/src/clj/org/apache/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/org/apache/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/org/apache/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/org/apache/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/org/apache/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/org/apache/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/org/apache/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/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)
+   - Schedules recurring task on the timer thread to check the topologies 
[code](https://github.com/apache/storm/blob/0.7.1/src/clj/org/apache/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/org/apache/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/org/apache/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/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)
+- "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/org/apache/storm/command/kill_topology.clj)
+- Nimbus receives the kill command 
[code](https://github.com/apache/storm/blob/0.7.1/src/clj/org/apache/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/org/apache/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/org/apache/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/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)
+   - 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/org/apache/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/org/apache/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/org/apache/storm/daemon/nimbus.clj#L577)

http://git-wip-us.apache.org/repos/asf/storm/blob/69dfb532/docs/Local-mode.md
----------------------------------------------------------------------
diff --git a/docs/Local-mode.md b/docs/Local-mode.md
index f871a73..e3d9666 100644
--- a/docs/Local-mode.md
+++ b/docs/Local-mode.md
@@ -8,12 +8,12 @@ Local mode simulates a Storm cluster in process and is useful 
for developing and
 To create an in-process cluster, simply use the `LocalCluster` class. For 
example:
 
 ```java
-import backtype.storm.LocalCluster;
+import org.apache.storm.LocalCluster;
 
 LocalCluster cluster = new LocalCluster();
 ```
 
-You can then submit topologies using the `submitTopology` method on the 
`LocalCluster` object. Just like the corresponding method on 
[StormSubmitter](javadocs/backtype/storm/StormSubmitter.html), `submitTopology` 
takes a name, a topology configuration, and the topology object. You can then 
kill a topology using the `killTopology` method which takes the topology name 
as an argument.
+You can then submit topologies using the `submitTopology` method on the 
`LocalCluster` object. Just like the corresponding method on 
[StormSubmitter](javadocs/org/apache/storm/StormSubmitter.html), 
`submitTopology` takes a name, a topology configuration, and the topology 
object. You can then kill a topology using the `killTopology` method which 
takes the topology name as an argument.
 
 To shutdown a local cluster, simple call:
 
@@ -23,7 +23,7 @@ cluster.shutdown();
 
 ### Common configurations for local mode
 
-You can see a full list of configurations 
[here](javadocs/backtype/storm/Config.html).
+You can see a full list of configurations 
[here](javadocs/org/apache/storm/Config.html).
 
 1. **Config.TOPOLOGY_MAX_TASK_PARALLELISM**: This config puts a ceiling on the 
number of threads spawned for a single component. Oftentimes production 
topologies have a lot of parallelism (hundreds of threads) which places 
unreasonable load when trying to test the topology in local mode. This config 
lets you easy control that parallelism.
 2. **Config.TOPOLOGY_DEBUG**: When this is set to true, Storm will log a 
message every time a tuple is emitted from any spout or bolt. This is extremely 
useful for debugging.

http://git-wip-us.apache.org/repos/asf/storm/blob/69dfb532/docs/Logs.md
----------------------------------------------------------------------
diff --git a/docs/Logs.md b/docs/Logs.md
new file mode 100644
index 0000000..28e6693
--- /dev/null
+++ b/docs/Logs.md
@@ -0,0 +1,30 @@
+---
+title: Storm Logs
+layout: documentation
+documentation: true
+---
+Logs in Storm are essential for tracking the status, operations, error 
messages and debug information for all the 
+daemons (e.g., nimbus, supervisor, logviewer, drpc, ui, pacemaker) and 
topologies' workers.
+
+### Location of the Logs
+All the daemon logs are placed under ${storm.log.dir} directory, which an 
administrator can set in the System properties or
+in the cluster configuration. By default, ${storm.log.dir} points to 
${storm.home}/logs.
+
+All the worker logs are placed under the workers-artifacts directory in a 
hierarchical manner, e.g.,
+${workers-artifacts}/${topologyId}/${port}/worker.log. Users can set the 
workers-artifacts directory
+by configuring the variable "storm.workers.artifacts.dir". By default, 
workers-artifacts directory
+locates at ${storm.log.dir}/logs/workers-artifacts.
+
+### Using the Storm UI for Log View/Download and Log Search
+Daemon and worker logs are allowed to view and download through Storm UI by 
authorized users.
+
+To improve the debugging of Storm, we provide the Log Search feature.
+Log Search supports searching in a certain log file or in all of a topology's 
log files:
+
+String search in a log file: In the log page for a worker, a user can search a 
certain string, e.g., "Exception", in a certain worker log. This search can 
happen for both normal text log or rolled zip log files. In the results, the 
offset and matched lines will be displayed.
+
+![Search in a log](images/search-for-a-single-worker-log.png "Search in a log")
+
+Search in a topology: a user can also search a string for a certain topology 
by clicking the icon of magnifying lens at the top right corner of the UI page. 
This means the UI will try to search on all the supervisor nodes in a 
distributed way to find the matched string in all logs for this topology. The 
search can happen for either normal text log files or rolled zip log files by 
checking/unchecking the "Search archived logs:" box. Then the matched results 
can be shown on the UI with url links, directing the user to the certain logs 
on each supervisor node. This powerful feature is very helpful for users to 
find certain problematic supervisor nodes running this topology.
+
+![Search in a topology](images/search-a-topology.png "Search in a topology")

http://git-wip-us.apache.org/repos/asf/storm/blob/69dfb532/docs/Message-passing-implementation.md
----------------------------------------------------------------------
diff --git a/docs/Message-passing-implementation.md 
b/docs/Message-passing-implementation.md
index a17f66a..fd4cf2c 100644
--- a/docs/Message-passing-implementation.md
+++ b/docs/Message-passing-implementation.md
@@ -8,23 +8,23 @@ documentation: true
 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/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)
+   - `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/org/apache/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/org/apache/storm/daemon/worker.clj#L56)
+   - The serializer is thread-safe 
[code](https://github.com/apache/storm/blob/0.7.1/src/jvm/org/apache/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/org/apache/storm/daemon/worker.clj#L185)
+   - Message sending happens through this protocol: 
[code](https://github.com/apache/storm/blob/0.7.1/src/clj/org/apache/storm/messaging/protocol.clj)
+   - The implementation for distributed mode uses ZeroMQ 
[code](https://github.com/apache/storm/blob/0.7.1/src/clj/org/apache/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/org/apache/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/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)
+   - 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/org/apache/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/org/apache/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 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/org/apache/storm/daemon/task.clj#L201)
+        - Bolts listen here: 
[code](https://github.com/apache/storm/blob/0.7.1/src/clj/org/apache/storm/daemon/task.clj#L489)
+        - Spouts listen here: 
[code](https://github.com/apache/storm/blob/0.7.1/src/clj/org/apache/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/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)
+  - 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/org/apache/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/org/apache/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/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)
+      - Bolt transfer code here: 
[code](https://github.com/apache/storm/blob/0.7.1/src/clj/org/apache/storm/daemon/task.clj#L429)
+      - Spout transfer code here: 
[code](https://github.com/apache/storm/blob/0.7.1/src/clj/org/apache/storm/daemon/task.clj#L329)

http://git-wip-us.apache.org/repos/asf/storm/blob/69dfb532/docs/Metrics.md
----------------------------------------------------------------------
diff --git a/docs/Metrics.md b/docs/Metrics.md
index b2521b1..5c2aa24 100644
--- a/docs/Metrics.md
+++ b/docs/Metrics.md
@@ -12,13 +12,13 @@ Metrics have to implement just one method, 
`getValueAndReset` -- do any remainin
 
 Storm gives you these metric types:
 
-* 
[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.
+* 
[AssignableMetric]({{page.git-blob-base}}/storm-core/src/jvm/org/apache/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/org/apache/storm/metric/api/CombinedMetric.java)
 -- generic interface for metrics that can be updated associatively. 
+* 
[CountMetric]({{page.git-blob-base}}/storm-core/src/jvm/org/apache/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/org/apache/storm/metric/api/MultiCountMetric.java)
 -- a hashmap of count metrics.
+* 
[ReducedMetric]({{page.git-blob-base}}/storm-core/src/jvm/org/apache/storm/metric/api/ReducedMetric.java)
+  - 
[MeanReducer]({{page.git-blob-base}}/storm-core/src/jvm/org/apache/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/org/apache/storm/metric/api/MultiReducedMetric.java)
 -- a hashmap of reduced metrics.
 
 
 ### Metric Consumer
@@ -30,7 +30,7 @@ Storm gives you these metric types:
 
 ### Builtin Metrics
 
-The [builtin 
metrics]({{page.git-blob-base}}/storm-core/src/clj/backtype/storm/daemon/builtin_metrics.clj)
 instrument Storm itself.
+The [builtin 
metrics]({{page.git-blob-base}}/storm-core/src/clj/org/apache/storm/daemon/builtin_metrics.clj)
 instrument Storm itself.
 
-[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`
+[builtin_metrics.clj]({{page.git-blob-base}}/storm-core/src/clj/org/apache/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/org/apache/storm/daemon/executor.clj#358)
  in `clj/b/s/daemon/daemon/executor.clj`
 

http://git-wip-us.apache.org/repos/asf/storm/blob/69dfb532/docs/Pacemaker.md
----------------------------------------------------------------------
diff --git a/docs/Pacemaker.md b/docs/Pacemaker.md
new file mode 100644
index 0000000..39e3014
--- /dev/null
+++ b/docs/Pacemaker.md
@@ -0,0 +1,113 @@
+---
+title: Pacemaker
+layout: documentation
+documentation: true
+---
+
+
+### Introduction
+Pacemaker is a storm daemon designed to process heartbeats from workers. As 
Storm is scaled up, ZooKeeper begins to become a bottleneck due to high volumes 
of writes from workers doing heartbeats. Lots of writes to disk and too much 
traffic across the network is generated as ZooKeeper tries to maintain 
consistency.
+
+Because heartbeats are of an ephemeral nature, they do not need to be 
persisted to disk or synced across nodes; an in-memory store will do. This is 
the role of Pacemaker. Pacemaker functions as a simple in-memory key/value 
store with ZooKeeper-like, directory-style keys and byte array values.
+
+The corresponding Pacemaker client is a plugin for the `ClusterState` 
interface, `org.apache.storm.pacemaker.pacemaker_state_factory`. Heartbeat 
calls are funneled by the `ClusterState` produced by `pacemaker_state_factory` 
into the Pacemaker daemon, while other set/get operations are forwarded to 
ZooKeeper.
+
+------
+
+### Configuration
+
+ - `pacemaker.host` : The host that the Pacemaker daemon is running on
+ - `pacemaker.port` : The port that Pacemaker will listen on
+ - `pacemaker.max.threads` : Maximum number of threads Pacemaker daemon will 
use to handle requests.
+ - `pacemaker.childopts` : Any JVM parameters that need to go to the 
Pacemaker. (used by storm-deploy project)
+ - `pacemaker.auth.method` : The authentication method that is used (more info 
below)
+
+#### Example
+
+To get Pacemaker up and running, set the following option in the cluster 
config on all nodes:
+```
+storm.cluster.state.store: "org.apache.storm.pacemaker.pacemaker_state_factory"
+```
+
+The Pacemaker host also needs to be set on all nodes:
+```
+pacemaker.host: somehost.mycompany.com
+```
+
+And then start all of your daemons
+
+(including Pacemaker):
+```
+$ storm pacemaker
+```
+
+The Storm cluster should now be pushing all worker heartbeats through 
Pacemaker.
+
+### Security
+
+Currently digest (password-based) and Kerberos security are supported. 
Security is currently only around reads, not writes. Writes may be performed by 
anyone, whereas reads may only be performed by authorized and authenticated 
users. This is an area for future development, as it leaves the cluster open to 
DoS attacks, but it prevents any sensitive information from reaching 
unauthorized eyes, which was the main goal.
+
+#### Digest
+To configure digest authentication, set `pacemaker.auth.method: DIGEST` in the 
cluster config on the nodes hosting Nimbus and Pacemaker.
+The nodes must also have `java.security.auth.login.config` set to point to a 
JAAS config file containing the following structure:
+```
+PacemakerDigest {
+    username="some username"
+    password="some password";
+};
+```
+
+Any node with these settings configured will be able to read from Pacemaker.
+Worker nodes need not have these configs set, and may keep 
`pacemaker.auth.method: NONE` set, since they do not need to read from the 
Pacemaker daemon.
+
+#### Kerberos
+To configure Kerberos authentication, set `pacemaker.auth.method: KERBEROS` in 
the cluster config on the nodes hosting Nimbus and Pacemaker.
+The nodes must also have `java.security.auth.login.config` set to point to a 
JAAS config.
+
+The JAAS config on Nimbus must look something like this:
+```
+PacemakerClient {
+    com.sun.security.auth.module.Krb5LoginModule required
+    useKeyTab=true
+    keyTab="/etc/keytabs/nimbus.keytab"
+    storeKey=true
+    useTicketCache=false
+    serviceName="pacemaker"
+    principal="[email protected]";
+};
+                         
+```
+
+The JAAS config on Pacemaker must look something like this:
+```
+PacemakerServer {
+   com.sun.security.auth.module.Krb5LoginModule required
+   useKeyTab=true
+   keyTab="/etc/keytabs/pacemaker.keytab"
+   storeKey=true
+   useTicketCache=false
+   principal="[email protected]";
+};
+```
+
+ - The client's user principal in the `PacemakerClient` section on the Nimbus 
host must match the `nimbus.daemon.user` storm cluster config value.
+ - The client's `serviceName` value must match the server's user principal in 
the `PacemakerServer` section on the Pacemaker host.
+
+
+### Fault Tolerance
+
+Pacemaker runs as a single daemon instance, making it a potential Single Point 
of Failure.
+
+If Pacemaker becomes unreachable by Nimbus, through crash or network 
partition, the workers will continue to run, and Nimbus will repeatedly attempt 
to reconnect. Nimbus functionality will be disrupted, but the topologies 
themselves will continue to run.
+In case of partition of the cluster where Nimbus and Pacemaker are on the same 
side of the partition, the workers that are on the other side of the partition 
will not be able to heartbeat, and Nimbus will reschedule the tasks elsewhere. 
This is probably what we want to happen anyway.
+
+
+### ZooKeeper Comparison
+Compared to ZooKeeper, Pacemaker uses less CPU, less memory, and of course no 
disk for the same load, thanks to lack of overhead from maintaining consistency 
between nodes.
+On Gigabit networking, there is a theoretical limit of about 6000 nodes. 
However, the real limit is likely around 2000-3000 nodes. These limits have not 
yet been tested.
+On a 270 supervisor cluster, fully scheduled with topologies, Pacemaker 
resource utilization was 70% of one core and nearly 1GiB of RAM on a machine 
with 4 `Intel(R) Xeon(R) CPU E5530 @ 2.40GHz` and 24GiB of RAM.
+
+
+There is an easy route to HA for Pacemaker. Unlike ZooKeeper, Pacemaker should 
be able to scale horizontally without overhead. By contrast, with ZooKeeper, 
there are diminishing returns when adding ZK nodes.
+
+In short, a single Pacemaker node should be able to handle many times the load 
that a ZooKeeper cluster can, and future HA work allowing horizontal scaling 
will increase that even farther.

Reply via email to