You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@storm.apache.org by sr...@apache.org on 2018/05/12 15:58:53 UTC

[50/51] [abbrv] [partial] storm-site git commit: Publish up to date 2.0.0-SNAPSHOT documentation

http://git-wip-us.apache.org/repos/asf/storm-site/blob/ef81b5ca/releases/2.0.0-SNAPSHOT/Acking-framework-implementation.md
----------------------------------------------------------------------
diff --git a/releases/2.0.0-SNAPSHOT/Acking-framework-implementation.md b/releases/2.0.0-SNAPSHOT/Acking-framework-implementation.md
index 5ca5d93..f181e98 100644
--- a/releases/2.0.0-SNAPSHOT/Acking-framework-implementation.md
+++ b/releases/2.0.0-SNAPSHOT/Acking-framework-implementation.md
@@ -1,13 +1,16 @@
 ---
+title: Acking framework implementation
 layout: documentation
+documentation: true
 ---
-[Storm's acker](https://github.com/apache/incubator-storm/blob/46c3ba7/storm-core/src/clj/backtype/storm/daemon/acker.clj#L28) tracks completion of each tupletree with a checksum hash: each time a tuple is sent, its value is XORed into the checksum, and each time a tuple is acked its value is XORed in again. If all tuples have been successfully acked, the checksum will be zero (the odds that the checksum will be zero otherwise are vanishingly small).
+
+[Storm's acker]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/daemon/Acker.java) tracks completion of each tupletree with a checksum hash: each time a tuple is sent, its value is XORed into the checksum, and each time a tuple is acked its value is XORed in again. If all tuples have been successfully acked, the checksum will be zero (the odds that the checksum will be zero otherwise are vanishingly small).
 
 You can read a bit more about the [reliability mechanism](Guaranteeing-message-processing.html#what-is-storms-reliability-api) elsewhere on the wiki -- this explains the internal details.
 
 ### acker `execute()`
 
-The acker is actually a regular bolt, with its  [execute method](https://github.com/apache/incubator-storm/blob/46c3ba7/storm-core/src/clj/backtype/storm/daemon/acker.clj#L36) defined withing `mk-acker-bolt`.  When a new tupletree is born, the spout sends the XORed edge-ids of each tuple recipient, which the acker records in its `pending` ledger. Every time an executor acks a tuple, the acker receives a partial checksum that is the XOR of the tuple's own edge-id (clearing it from the ledger) and the edge-id of each downstream tuple the executor emitted (thus entering them into the ledger).
+The acker is actually a regular bolt.  When a new tupletree is born, the spout sends the XORed edge-ids of each tuple recipient, which the acker records in its `pending` ledger. Every time an executor acks a tuple, the acker receives a partial checksum that is the XOR of the tuple's own edge-id (clearing it from the ledger) and the edge-id of each downstream tuple the executor emitted (thus entering them into the ledger).
 
 This is accomplished as follows.
 
@@ -17,7 +20,7 @@ On a tick tuple, just advance pending tupletree checksums towards death and retu
 * on ack:  xor the partial checksum into the existing checksum value
 * on fail: just mark it as failed
 
-Next, [put the record](https://github.com/apache/incubator-storm/blob/46c3ba7/storm-core/src/clj/backtype/storm/daemon/acker.clj#L50)),  into the RotatingMap (thus resetting is countdown to expiry) and take action:
+Next, put the record into the RotatingMap (thus resetting is countdown to expiry) and take action:
 
 * if the total checksum is zero, the tupletree is complete: remove it from the pending collection and notify the spout of success
 * if the tupletree has failed, it is also complete:   remove it from the pending collection and notify the spout of failure
@@ -26,7 +29,7 @@ Finally, pass on an ack of our own.
 
 ### Pending tuples and the `RotatingMap`
 
-The acker stores pending tuples in a [`RotatingMap`](https://github.com/apache/incubator-storm/blob/master/storm-core/src/jvm/backtype/storm/utils/RotatingMap.java#L19), a simple device used in several places within Storm to efficiently time-expire a process.
+The acker stores pending tuples in a [`RotatingMap`]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/utils/RotatingMap.java), a simple device used in several places within Storm to efficiently time-expire a process.
 
 The RotatingMap behaves as a HashMap, and offers the same O(1) access guarantees.
 

http://git-wip-us.apache.org/repos/asf/storm-site/blob/ef81b5ca/releases/2.0.0-SNAPSHOT/Classpath-handling.md
----------------------------------------------------------------------
diff --git a/releases/2.0.0-SNAPSHOT/Classpath-handling.md b/releases/2.0.0-SNAPSHOT/Classpath-handling.md
new file mode 100644
index 0000000..d48517e
--- /dev/null
+++ b/releases/2.0.0-SNAPSHOT/Classpath-handling.md
@@ -0,0 +1,29 @@
+---
+title: Classpath Handling
+layout: documentation
+documentation: true
+---
+### Storm is an Application Container
+
+Storm provides an application container environment, a la Apache Tomcat, which creates potential for classpath conflicts between Storm and your application.  The most common way of using Storm involves submitting an "uber JAR" containing your application code with all of its dependencies bundled in, and then Storm distributes this JAR to Worker nodes.  Then Storm runs your application within a Storm process called a `Worker` -- thus the JVM's classpath contains the dependencies of your JAR as well as whatever dependencies the Worker itself has.  So careful handling of classpaths and dependencies is critical for the correct functioning of Storm.
+
+### Adding Extra Dependencies to Classpath
+
+You no longer *need* to bundle your dependencies into your topology and create an uber JAR, there are now facilities for separately handling your topology's dependencies.  Furthermore, there are facilities for adding external dependencies into the Storm daemons.
+
+The `storm.py` launcher script allows you to include dependencies into the launched program's classpath via a few different mechanisms:
+
+1. The `--jar` and `--artifacts` options for the `storm jar` command: allow inclusion of non-bundled dependencies with your topology; i.e., allowing specification of JARs that were not bundled into the topology uber-jar.  This is required when using the `storm sql` command, which constructs a topology automatically without needing you to write code and build a topology JAR.
+2. The `${STORM_DIR}/extlib/` and `${STORM_DIR}/extlib-daemon/` directories can have dependencies added to them for inclusion of plugins & 3rd-party libraries into the Storm daemons (e.g., Nimbus, UI, Supervisor, etc. -- use `extlib-daemon/`) and other commands launched via the `storm.py` script, e.g., `storm sql` and `storm jar` (use `extlib`). Notably, this means that the Storm Worker process does not include the `extlib-daemon/` directory into its classpath.
+3. The `STORM_EXT_CLASSPATH` and `STORM_EXT_CLASSPATH_DAEMON` environment variables provide a similar functionality as those directories, but allows the user to place their external dependencies in alternative locations.
+ * There is a wrinkle here: because the Supervisor daemon launches the Worker process, if you want `STORM_EXT_CLASSPATH` to impact your Workers, you will need to specify the `STORM_EXT_CLASSPATH` for the Supervisor daemon.  That will allow the Supervisor to consult this environment variable as it constructs the classpath of the Worker processes.
+
+#### Which Facility to Choose?
+
+You might have noticed the overlap between the first mechanism and the others. If you consider the `--jar` / `--artifacts` option versus the `extlib/` / `STORM_EXT_CLASSPATH` it is not obvious which one you should choose for using dependencies with your Worker processes. i.e., both mechanisms allow including JARs to be used for running your Worker processes. Here is my understanding of the difference: `--jar` / `--artifacts` will result in the dependencies being used for running the `storm jar/sql` command, *and* the dependencies will be uploaded and available in the classpath of the topology's `Worker` processes. Whereas the use of `extlib/` / `STORM_EXT_CLASSPATH` requires you to have distributed your JAR dependencies out to all Worker nodes.  Another difference is that `extlib/` / `STORM_EXT_CLASSPATH` would impact all topologies, whereas `--jar` / `--artifacts` is a topology-specific option.
+
+### Abbreviation of Classpaths and Process Commands
+
+When the `storm.py` script launches a `java` command, it first constructs the classpath from the optional settings mentioned above, as well as including some default locations such as the `${STORM_DIR}/`, `${STORM_DIR}/lib/`, `${STORM_DIR}/extlib/` and `${STORM_DIR}/extlib-daemon/` directories.  In past releases, Storm would enumerate all JARs in those directories and then explicitly add all of those JARs into the `-cp` / `--classpath` argument to the launched `java` commands.  As such, the classpath would get so long that the `java` commands could breach the Linux Kernel process table limit of 4096 bytes for recording commands.  That led to truncated commands in `ps` output, making it hard to operate Storm clusters because you could not easily differentiate the processes nor easily see from `ps` which port a worker is listening to.
+
+After Storm dropped support for Java 5, this classpath expansion was no longer necessary, because Java 6 supports classpath wildcards. Classpath wildcards allow you to specify a directory ending with a `*` element, such as `foo/bar/*`, and the JVM will automatically expand the classpath to include all `.jar` files in the wildcard directory.  As of [STORM-2191](https://issues.apache.org/jira/browse/STORM-2191) Storm just uses classpath wildcards instead of explicitly listing all JARs, thereby shortening all of the commands and making operating Storm clusters a bit easier.

http://git-wip-us.apache.org/repos/asf/storm-site/blob/ef81b5ca/releases/2.0.0-SNAPSHOT/Clojure-DSL.md
----------------------------------------------------------------------
diff --git a/releases/2.0.0-SNAPSHOT/Clojure-DSL.md b/releases/2.0.0-SNAPSHOT/Clojure-DSL.md
index 1aa3393..56bb54f 100644
--- a/releases/2.0.0-SNAPSHOT/Clojure-DSL.md
+++ b/releases/2.0.0-SNAPSHOT/Clojure-DSL.md
@@ -252,7 +252,7 @@ The following example illustrates how to use this spout in a `spout-spec`:
 
 ### Running topologies in local mode or on a cluster
 
-That's all there is to the Clojure DSL. To submit topologies in remote mode or local mode, just use the `StormSubmitter` or `LocalCluster` classes just like you would from Java.
+That's all there is to the Clojure DSL. To submit topologies in remote mode or local mode, just use the `StormSubmitter` class just like you would from Java.
 
 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:
 

http://git-wip-us.apache.org/repos/asf/storm-site/blob/ef81b5ca/releases/2.0.0-SNAPSHOT/Command-line-client.md
----------------------------------------------------------------------
diff --git a/releases/2.0.0-SNAPSHOT/Command-line-client.md b/releases/2.0.0-SNAPSHOT/Command-line-client.md
index f97ca33..7bc678d 100644
--- a/releases/2.0.0-SNAPSHOT/Command-line-client.md
+++ b/releases/2.0.0-SNAPSHOT/Command-line-client.md
@@ -3,7 +3,7 @@ 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-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). See [Classpath handling](Classpath-handling.html) for details on using external libraries in these commands.
 
 These commands are:
 
@@ -21,6 +21,7 @@ These commands are:
 1. supervisor
 1. ui
 1. drpc
+1. drpc-client
 1. blobstore
 1. dev-zookeeper
 1. get-errors
@@ -47,7 +48,9 @@ When you want to ship other jars which is not included to application jar, you c
 For example, --jars "your-local-jar.jar,your-local-jar2.jar" will load your-local-jar.jar and your-local-jar2.jar.
 And when you want to ship maven artifacts and its transitive dependencies, you can pass them to `--artifacts` with comma-separated string. You can also exclude some dependencies like what you're doing in maven pom. Please add exclusion artifacts with '^' separated string after the artifact. For example, `--artifacts "redis.clients:jedis:2.9.0,org.apache.kafka:kafka_2.10:0.8.2.2^org.slf4j:slf4j-log4j12"` will load jedis and kafka artifact and all of transitive dependencies but exclude slf4j-log4j12 from kafka.
 
-Complete example of both options is here: `./bin/storm jar example/storm-starter/storm-starter-topologies-*.jar org.apache.storm.starter.RollingTopWords blobstore-remote2 remote --jars "./external/storm-redis/storm-redis-1.1.0.jar,./external/storm-kafka/storm-kafka-1.1.0.jar" --artifacts "redis.clients:jedis:2.9.0,org.apache.kafka:kafka_2.10:0.8.2.2^org.slf4j:slf4j-log4j12"`
+When you need to pull the artifacts from other than Maven Central, you can pass remote repositories to --artifactRepositories option with comma-separated string. Repository format is "<name>^<url>". '^' is taken as separator because URL allows various characters. For example, --artifactRepositories "jboss-repository^http://repository.jboss.com/maven2,HDPRepo^http://repo.hortonworks.com/content/groups/public/" will add JBoss and HDP repositories for dependency resolver.
+
+Complete example of both options is here: `./bin/storm jar example/storm-starter/storm-starter-topologies-*.jar org.apache.storm.starter.RollingTopWords blobstore-remote2 remote --jars "./external/storm-redis/storm-redis-1.1.0.jar,./external/storm-kafka/storm-kafka-1.1.0.jar" --artifacts "redis.clients:jedis:2.9.0,org.apache.kafka:kafka_2.10:0.8.2.2^org.slf4j:slf4j-log4j12" --artifactRepositories "jboss-repository^http://repository.jboss.com/maven2,HDPRepo^http://repo.hortonworks.com/content/groups/public/"`
 
 When you pass jars and/or artifacts options, StormSubmitter will upload them when the topology is submitted, and they will be included to classpath of both the process which runs the class, and also workers for that topology.
 
@@ -57,7 +60,7 @@ Syntax: `storm sql sql-file topology-name`
 
 Compiles the SQL statements into a Trident topology and submits it to Storm.
 
-`--jars` and `--artifacts` options are also applied to `sql` command. You normally want to pass these options since you need to set data source to your sql which is an external storage in many cases.
+`--jars` and `--artifacts`, and `--artifactRepositories` options available for jar are also applied to sql command. Please refer "help jar" to see how to use --jars and --artifacts, and --artifactRepositories options. You normally want to pass these options since you need to set data source to your sql which is an external storage in many cases.
 
 ### kill
 
@@ -135,6 +138,40 @@ Syntax: `storm drpc`
 
 Launches a DRPC daemon. This command should be run under supervision with a tool like [daemontools](http://cr.yp.to/daemontools.html) or [monit](http://mmonit.com/monit/). See [Distributed RPC](Distributed-RPC.html) for more information.
 
+### drpc-client
+
+Syntax: `storm drpc-client [options] ([function argument]*)|(argument*)`
+
+Provides a very simple way to send DRPC requests. If a `-f` argument is supplied, to set the function name, all of the arguments are treated
+as arguments to the function.  If no function is given the arguments must be pairs of function argument.
+
+*NOTE:* This is not really intended for production use.  This is mostly because parsing out the results can be a pain.
+
+Creating an actuall DRPC client only takes a few lines, so for production please go with that.
+
+```java
+Config conf = new Config();
+try (DRPCClient drpc = DRPCClient.getConfiguredClient(conf)) {
+  //User the drpc client
+  String result = drpc.execute(function, argument);
+}
+```
+
+#### Examples
+
+`storm drpc-client exclaim a exclaim b test bar`
+
+This will submit 3 separate DRPC request.
+1. funciton = "exclaim" args = "a"
+2. function = "exclaim" args = "b"
+3. function = "test" args = "bar"
+
+`storm drpc-client -f exclaim a b`
+
+This will submit 2 separate DRPC request.
+1. funciton = "exclaim" args = "a"
+2. function = "exclaim" args = "b"
+
 ### blobstore
 
 Syntax: `storm blobstore cmd`

http://git-wip-us.apache.org/repos/asf/storm-site/blob/ef81b5ca/releases/2.0.0-SNAPSHOT/Common-patterns.md
----------------------------------------------------------------------
diff --git a/releases/2.0.0-SNAPSHOT/Common-patterns.md b/releases/2.0.0-SNAPSHOT/Common-patterns.md
index b98fe90..e0e8c1f 100644
--- a/releases/2.0.0-SNAPSHOT/Common-patterns.md
+++ b/releases/2.0.0-SNAPSHOT/Common-patterns.md
@@ -6,28 +6,12 @@ documentation: true
 
 This page lists a variety of common patterns in Storm topologies.
 
-1. Streaming joins
-2. Batching
-3. BasicBolt
-4. In-memory caching + fields grouping combo
-5. Streaming top N
-6. TimeCacheMap for efficiently keeping a cache of things that have been recently updated
-7. CoordinatedBolt and KeyedFairBolt for Distributed RPC
-
-### Joins
-
-A streaming join combines two or more data streams together based on some common field. Whereas a normal database join has finite input and clear semantics for a join, a streaming join has infinite input and unclear semantics for what a join should be.
-
-The join type you need will vary per application. Some applications join all tuples for two streams over a finite window of time, whereas other applications expect exactly one tuple for each side of the join for each join field. Other applications may do the join completely differently. The common pattern among all these join types is partitioning multiple input streams in the same way. This is easily accomplished in Storm by using a fields grouping on the same fields for many input streams to the joiner bolt. For example:
-
-```java
-builder.setBolt("join", new MyJoiner(), parallelism)
-  .fieldsGrouping("1", new Fields("joinfield1", "joinfield2"))
-  .fieldsGrouping("2", new Fields("joinfield1", "joinfield2"))
-  .fieldsGrouping("3", new Fields("joinfield1", "joinfield2"));
-```
-
-The different streams don't have to have the same field names, of course.
+1. Batching
+2. BasicBolt
+3. In-memory caching + fields grouping combo
+4. Streaming top N
+5. TimeCacheMap for efficiently keeping a cache of things that have been recently updated
+6. CoordinatedBolt and KeyedFairBolt for Distributed RPC
 
 
 ### Batching

http://git-wip-us.apache.org/repos/asf/storm-site/blob/ef81b5ca/releases/2.0.0-SNAPSHOT/Concepts.md
----------------------------------------------------------------------
diff --git a/releases/2.0.0-SNAPSHOT/Concepts.md b/releases/2.0.0-SNAPSHOT/Concepts.md
index d46033c..8516b9e 100644
--- a/releases/2.0.0-SNAPSHOT/Concepts.md
+++ b/releases/2.0.0-SNAPSHOT/Concepts.md
@@ -113,3 +113,8 @@ Topologies execute across one or more worker processes. Each worker process is a
 **Resources:**
 
 * [Config.TOPOLOGY_WORKERS](javadocs/org/apache/storm/Config.html#TOPOLOGY_WORKERS): this config sets the number of workers to allocate for executing the topology
+
+### Performance Tuning
+
+Refer to [performance tuning guide](Performance.md)
+

http://git-wip-us.apache.org/repos/asf/storm-site/blob/ef81b5ca/releases/2.0.0-SNAPSHOT/Configuration.md
----------------------------------------------------------------------
diff --git a/releases/2.0.0-SNAPSHOT/Configuration.md b/releases/2.0.0-SNAPSHOT/Configuration.md
index 979ac9a..83cb28a 100644
--- a/releases/2.0.0-SNAPSHOT/Configuration.md
+++ b/releases/2.0.0-SNAPSHOT/Configuration.md
@@ -21,10 +21,20 @@ The Java API lets you specify component specific configurations in two ways:
 
 The preference order for configuration values is defaults.yaml < storm.yaml < topology specific configuration < internal component specific configuration < external component specific configuration. 
 
+# Bolts, Spouts, and Plugins
+In almost all cases configuration for a bolt or a spout should be done though setters on the bolt or spout implementation and not the topology conf.  In some rare cases it may make since to
+expose topology wide configurations that are not currently a part of [Config](javadocs/org/apache/storm/Config.html) or [DaemonConfig](javadocs/org/apache/storm/DaemonConfig.html) such as
+when writing a custom scheduler or a plugin to some part of storm.  In those
+cases you can create your own class like Config but implements [Validated](javadocs/org/apache/storm/validation/Validated.html). Any `public static final String` field declared in this
+class will be treated as a config and annotations from the `org.apache.storm.validation.ConfigValidationAnnotations` class can be used to enforce what is stored in that config.
+To let the validator know about this class you need to treat the class
+like a service that will be loaded through a ServiceLoader for the Validated class and include a `META-INF/services/org.apache.storm.validation.Validated` file in your jar that holds
+the name of your Config class.
 
 **Resources:**
 
-* [Config](javadocs/org/apache/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 client configurations as well as a helper class for creating topology specific configurations
+* [DaemonConfig](javadocs/org/apache/storm/DaemonConfig.html): a listing of Storm Daemon 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-site/blob/ef81b5ca/releases/2.0.0-SNAPSHOT/Contributing-to-Storm.md
----------------------------------------------------------------------
diff --git a/releases/2.0.0-SNAPSHOT/Contributing-to-Storm.md b/releases/2.0.0-SNAPSHOT/Contributing-to-Storm.md
index fdc5835..2c6ad1f 100644
--- a/releases/2.0.0-SNAPSHOT/Contributing-to-Storm.md
+++ b/releases/2.0.0-SNAPSHOT/Contributing-to-Storm.md
@@ -6,7 +6,7 @@ documentation: true
 
 ### Getting started with contributing
 
-Some of the issues on the [issue tracker](https://issues.apache.org/jira/browse/STORM) are marked with the "Newbie" label. If you're interesting in contributing to Storm but don't know where to begin, these are good issues to start with. These issues are a great way to get your feet wet with learning the codebase because they require learning about only an isolated portion of the codebase and are a relatively small amount of work.
+Some of the issues on the [issue tracker](https://issues.apache.org/jira/browse/STORM) are marked with the ["Newbie"](https://issues.apache.org/jira/browse/STORM-2891?jql=project%20%3D%20STORM%20AND%20status%20%3D%20Open%20AND%20labels%20in%20(newbie%2C%20%22newbie%2B%2B%22)) label. If you're interested in contributing to Storm but don't know where to begin, these are good issues to start with. These issues are a great way to get your feet wet with learning the codebase because they require learning about only an isolated portion of the codebase and are a relatively small amount of work.
 
 ### Learning the codebase
 

http://git-wip-us.apache.org/repos/asf/storm-site/blob/ef81b5ca/releases/2.0.0-SNAPSHOT/DSLs-and-multilang-adapters.md
----------------------------------------------------------------------
diff --git a/releases/2.0.0-SNAPSHOT/DSLs-and-multilang-adapters.md b/releases/2.0.0-SNAPSHOT/DSLs-and-multilang-adapters.md
index 0ed5450..917b419 100644
--- a/releases/2.0.0-SNAPSHOT/DSLs-and-multilang-adapters.md
+++ b/releases/2.0.0-SNAPSHOT/DSLs-and-multilang-adapters.md
@@ -3,8 +3,9 @@ title: Storm DSLs and Multi-Lang Adapters
 layout: documentation
 documentation: true
 ---
+* [Clojure DSL](Clojure-DSL.html)
 * [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/dan-blanchard/io-storm): Perl multilang adapter
+* [FsShelter](https://github.com/Prolucid/FsShelter): F# DSL and runtime with protobuf multilang

http://git-wip-us.apache.org/repos/asf/storm-site/blob/ef81b5ca/releases/2.0.0-SNAPSHOT/Defining-a-non-jvm-language-dsl-for-storm.md
----------------------------------------------------------------------
diff --git a/releases/2.0.0-SNAPSHOT/Defining-a-non-jvm-language-dsl-for-storm.md b/releases/2.0.0-SNAPSHOT/Defining-a-non-jvm-language-dsl-for-storm.md
index 7096a43..311577b 100644
--- a/releases/2.0.0-SNAPSHOT/Defining-a-non-jvm-language-dsl-for-storm.md
+++ b/releases/2.0.0-SNAPSHOT/Defining-a-non-jvm-language-dsl-for-storm.md
@@ -3,7 +3,7 @@ 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]({{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.
+The right place to start to learn how to make a non-JVM DSL for Storm is [storm-client/src/storm.thrift]({{page.git-blob-base}}/storm-client/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-site/blob/ef81b5ca/releases/2.0.0-SNAPSHOT/Distributed-RPC.md
----------------------------------------------------------------------
diff --git a/releases/2.0.0-SNAPSHOT/Distributed-RPC.md b/releases/2.0.0-SNAPSHOT/Distributed-RPC.md
index b20419a..9931e69 100644
--- a/releases/2.0.0-SNAPSHOT/Distributed-RPC.md
+++ b/releases/2.0.0-SNAPSHOT/Distributed-RPC.md
@@ -12,7 +12,19 @@ DRPC is not so much a feature of Storm as it is a pattern expressed from Storm's
 Distributed RPC is coordinated by a "DRPC server" (Storm comes packaged with an implementation of this). The DRPC server coordinates receiving an RPC request, sending the request to the Storm topology, receiving the results from the Storm topology, and sending the results back to the waiting client. From a client's perspective, a distributed RPC call looks just like a regular RPC call. For example, here's how a client would compute the results for the "reach" function with the argument "http://twitter.com":
 
 ```java
-DRPCClient client = new DRPCClient("drpc-host", 3772);
+Config conf = new Config();
+conf.put("storm.thrift.transport", "org.apache.storm.security.auth.plain.PlainSaslTransportPlugin");
+conf.put(Config.STORM_NIMBUS_RETRY_TIMES, 3);
+conf.put(Config.STORM_NIMBUS_RETRY_INTERVAL, 10);
+conf.put(Config.STORM_NIMBUS_RETRY_INTERVAL_CEILING, 20);
+DRPCClient client = new DRPCClient(conf, "drpc-host", 3772);
+String result = client.execute("reach", "http://twitter.com");
+```
+
+or if you just want to use a preconfigured client you can call.  The exact host will be selected randomly from the configured set of hosts, if the host appears to be down it will loop through all configured hosts looking for one that works.
+
+```java
+DRPCClient client = DRPCClient.getConfiguredClient(conf);
 String result = client.execute("reach", "http://twitter.com");
 ```
 
@@ -57,23 +69,9 @@ In this example, `ExclaimBolt` simply appends a "!" to the second field of the t
 
 ### Local mode DRPC
 
-DRPC can be run in local mode. Here's how to run the above example in local mode:
-
-```java
-LocalDRPC drpc = new LocalDRPC();
-LocalCluster cluster = new LocalCluster();
-
-cluster.submitTopology("drpc-demo", conf, builder.createLocalTopology(drpc));
-
-System.out.println("Results for 'hello':" + drpc.execute("exclamation", "hello"));
-
-cluster.shutdown();
-drpc.shutdown();
-```
-
-First you create a `LocalDRPC` object. This object simulates a DRPC server in process, just like how `LocalCluster` simulates a Storm cluster in process. Then you create the `LocalCluster` to run the topology in local mode. `LinearDRPCTopologyBuilder` has separate methods for creating local topologies and remote topologies. In local mode the `LocalDRPC` object does not bind to any ports so the topology needs to know about the object to communicate with it. This is why `createLocalTopology` takes in the `LocalDRPC` object as input.
-
-After launching the topology, you can do DRPC invocations using the `execute` method on `LocalDRPC`.
+In the past to use DRPC in local mode it took creating a special LocalDRPC instance.  This can still be used when writing tests for your code, but in the current version of storm when you run in local mode a LocalDRPC
+instance is also created, and any DRPCClient created will link to it instead of the outside world.  This means that any interaction you want to test needs to be a part of the script that launches the topology, just like
+with LocalDRPC.
 
 ### Remote mode DRPC
 
@@ -89,12 +87,14 @@ Launching a DRPC server can be done with the `storm` script and is just like lau
 bin/storm drpc
 ```
 
-Next, you need to configure your Storm cluster to know the locations of the DRPC server(s). This is how `DRPCSpout` knows from where to read function invocations. This can be done through the `storm.yaml` file or the topology configurations. Configuring this through the `storm.yaml` looks something like this:
+Next, you need to configure your Storm cluster to know the locations of the DRPC server(s). This is how `DRPCSpout` knows from where to read function invocations. This can be done through the `storm.yaml` file or the topology configurations. You should also specify storm.thrift.transport property to match DRPCClient settings. Configuring this through the `storm.yaml` looks something like this:
 
 ```yaml
 drpc.servers:
   - "drpc1.foo.com"
   - "drpc2.foo.com"
+drpc.http.port: 8081
+storm.thrift.transport: "org.apache.storm.security.auth.plain.PlainSaslTransportPlugin"
 ```
 
 Finally, you launch DRPC topologies using `StormSubmitter` just like you launch any other topology. To run the above example in remote mode, you do something like this:
@@ -105,6 +105,23 @@ StormSubmitter.submitTopology("exclamation-drpc", conf, builder.createRemoteTopo
 
 `createRemoteTopology` is used to create topologies suitable for Storm clusters.
 
+Assuming that the topology is listening on the `exclaim` function you can execute something several differnt ways.
+
+Programatically:
+```java
+Config conf = new Config();
+try (DRPCClient drpc = DRPCClient.getConfiguredClient(conf)) {
+  //User the drpc client
+  String result = drpc.execute("exclaim", "argument");
+}
+```
+
+through curl:
+```curl http://hostname:8081/drpc/exclaim/argument```
+
+Through the command line:
+```bin/storm drpc-client exclaim argument```
+
 ### A more complex example
 
 The exclamation DRPC example was a toy example for illustrating the concepts of DRPC. Let's look at a more complex example which really needs the parallelism a Storm cluster provides for computing the DRPC function. The example we'll look at is computing the reach of a URL on Twitter.

http://git-wip-us.apache.org/repos/asf/storm-site/blob/ef81b5ca/releases/2.0.0-SNAPSHOT/Eventlogging.md
----------------------------------------------------------------------
diff --git a/releases/2.0.0-SNAPSHOT/Eventlogging.md b/releases/2.0.0-SNAPSHOT/Eventlogging.md
index 0895232..6fbf1ed 100644
--- a/releases/2.0.0-SNAPSHOT/Eventlogging.md
+++ b/releases/2.0.0-SNAPSHOT/Eventlogging.md
@@ -68,7 +68,8 @@ Eventlogging works by sending the events (tuples) from each component to an inte
 
 
 ## Extending eventlogging
-Storm provides an `IEventLogger` interface which is used by the event logger bolt to log the events. The default implementation for this is a FileBasedEventLogger which logs the events to an events.log file ( `logs/workers-artifacts/<topology-id>/<worker-port>/events.log`). Alternate implementations of the `IEventLogger` interface can be added to extend the event logging functionality (say build a search index or log the events in a database etc)
+
+Storm provides an `IEventLogger` interface which is used by the event logger bolt to log the events.
 
 ```java
 /**
@@ -79,7 +80,7 @@ public interface IEventLogger {
     /**
     * Invoked during eventlogger bolt prepare.
     */
-    void prepare(Map stormConf, TopologyContext context);
+    void prepare(Map stormConf, Map<String, Object> arguments, TopologyContext context);
 
     /**
      * Invoked when the {@link EventLoggerBolt} receives a tuple from the spouts or bolts that has event logging enabled.
@@ -94,3 +95,31 @@ public interface IEventLogger {
     void close();
 }
 ```
+
+The default implementation for this is a FileBasedEventLogger which logs the events to an events.log file ( `logs/workers-artifacts/<topology-id>/<worker-port>/events.log`).
+Alternate implementations of the `IEventLogger` interface can be added to extend the event logging functionality (say build a search index or log the events in a database etc)
+
+If you just want to use FileBasedEventLogger but with changing the log format, you can simply implement your own by extending FileBasedEventLogger and override `buildLogMessage(EventInfo)` to provide log line explicitly.
+
+To register event logger to your topology, add to your topology's configuration like:
+
+```java
+conf.registerEventLogger(org.apache.storm.metric.FileBasedEventLogger.class);
+```
+
+You can refer [Config#registerEventLogger](javadocs/org/apache/storm/Config.html#registerEventLogger-java.lang.Class-) and overloaded methods from javadoc.
+
+Otherwise edit the storm.yaml config file:
+
+```yaml
+topology.event.logger.register:
+  - class: "org.apache.storm.metric.FileBasedEventLogger"
+  - class: "org.mycompany.MyEventLogger"
+    arguments:
+      endpoint: "event-logger.mycompany.org"
+```
+
+When you implement your own event logger, `arguments` is passed to Map<String, Object> when [IEventLogger#prepare](javadocs/org/apache/storm/metric/IEventLogger.html#prepare-java.util.Map-java.lang.Map-org.apache.storm.task.TopologyContext-) is called.
+
+Please keep in mind that EventLoggerBolt is just a kind of Bolt, so whole throughput of the topology will go down when registered event loggers cannot keep up handling incoming events, so you may want to take care of the Bolt like normal Bolt.
+One of idea to avoid this is making your implementation of IEventLogger as `non-blocking` fashion.

http://git-wip-us.apache.org/repos/asf/storm-site/blob/ef81b5ca/releases/2.0.0-SNAPSHOT/FAQ.md
----------------------------------------------------------------------
diff --git a/releases/2.0.0-SNAPSHOT/FAQ.md b/releases/2.0.0-SNAPSHOT/FAQ.md
index 127c95c..ce9130e 100644
--- a/releases/2.0.0-SNAPSHOT/FAQ.md
+++ b/releases/2.0.0-SNAPSHOT/FAQ.md
@@ -123,5 +123,5 @@ You cannot know that all events are collected -- this is an epistemological chal
 
 * Set a time limit using domain knowledge
 * Introduce a _punctuation_: a record known to come after all records in the given time bucket. Trident uses this scheme to know when a batch is complete. If you for instance receive records from a set of sensors, each in order for that sensor, then once all sensors have sent you a 3:02:xx or later timestamp lets you know you can commit. 
-* When possible, make your process incremental: each value that comes in makes the answer more an more true. A Trident ReducerAggregator is an operator that takes a prior result and a set of new records and returns a new result. This lets the result be cached and serialized to a datastore; if a server drops off line for a day and then comes back with a full day's worth of data in a rush, the old results will be calmly retrieved and updated.
+* When possible, make your process incremental: each value that comes in makes the answer more and more true. A Trident ReducerAggregator is an operator that takes a prior result and a set of new records and returns a new result. This lets the result be cached and serialized to a datastore; if a server drops off line for a day and then comes back with a full day's worth of data in a rush, the old results will be calmly retrieved and updated.
 * Lambda architecture: Record all events into an archival store (S3, HBase, HDFS) on receipt. in the fast layer, once the time window is clear, process the bucket to get an actionable answer, and ignore everything older than the time window. Periodically run a global aggregation to calculate a "correct" answer.

http://git-wip-us.apache.org/repos/asf/storm-site/blob/ef81b5ca/releases/2.0.0-SNAPSHOT/IConfigLoader.md
----------------------------------------------------------------------
diff --git a/releases/2.0.0-SNAPSHOT/IConfigLoader.md b/releases/2.0.0-SNAPSHOT/IConfigLoader.md
new file mode 100644
index 0000000..3c2f4de
--- /dev/null
+++ b/releases/2.0.0-SNAPSHOT/IConfigLoader.md
@@ -0,0 +1,51 @@
+---
+title: IConfigLoader
+layout: documentation
+documentation: true
+---
+
+
+### Introduction
+IConfigLoader is an interface designed to allow dynamic loading of scheduler resource constraints. Currently, the MultiTenant scheduler uses this interface to dynamically load the number of isolated nodes a given user has been guaranteed, and the ResoureAwareScheduler uses the interface to dynamically load per user resource guarantees.
+
+The following interface is provided for users to create an IConfigLoader instance based on the scheme of the `scheduler.config.loader.uri`.
+```
+ConfigLoaderFactoryService.createConfigLoader(Map<String, Object> conf)
+``` 
+
+------
+
+### Interface
+```
+public interface IConfigLoader {
+    Map<?,?> load();
+};
+```
+#### Description
+  - load is called by the scheduler whenever it wishes to retrieve the most recent configuration map.
+ 
+#### Loader Configuration
+The loaders are dynamically selected and dynamically configured through configuration items in the scheduler implementations.
+
+##### Example
+```
+scheduler.config.loader.uri: "artifactory+http://artifactory.my.company.com:8000/artifactory/configurations/clusters/my_cluster/ras_pools"
+scheduler.config.loader.timeout.sec: 30
+```
+Or
+```
+scheduler.config.loader.uri: "file:///path/to/my/config.yaml"
+```
+### Implementations
+
+There are currently two implemenations of IConfigLoader
+ - org.apache.storm.scheduler.utils.ArtifactoryConfigLoader: Load configurations from an Artifactory server. 
+ It will be used if users add `artifactory+` to the scheme of the real URI and set to `scheduler.config.loader.uri`.
+ - org.apache.storm.scheduler.utils.FileConfigLoader: Load configurations from a local file. It will be used if users use `file` scheme.
+
+#### Configurations
+ - scheduler.config.loader.uri: For `ArtifactoryConfigLoader`, this can either be a reference to an individual file in Artifactory or to a directory.  If it is a directory, the file with the largest lexographic name will be returned.
+ For `FileConfigLoader`, this is the URI pointing to a file.
+ - scheduler.config.loader.timeout.secs: Currently only used in `ArtifactoryConfigLoader`. It is the amount of time an http connection to the artifactory server will wait before timing out. The default is 10.
+ - scheduler.config.loader.polltime.secs: Currently only used in `ArtifactoryConfigLoader`. It is the frequency at which the plugin will call out to artifactory instead of returning the most recently cached result. The default is 600 seconds.
+ - scheduler.config.loader.artifactory.base.directory: Only used in `ArtifactoryConfigLoader`. It is the part of the uri, configurable in Artifactory, which represents the top of the directory tree. It defaults to "/artifactory".
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm-site/blob/ef81b5ca/releases/2.0.0-SNAPSHOT/Implementation-docs.md
----------------------------------------------------------------------
diff --git a/releases/2.0.0-SNAPSHOT/Implementation-docs.md b/releases/2.0.0-SNAPSHOT/Implementation-docs.md
index 9eb91f5..93459f3 100644
--- a/releases/2.0.0-SNAPSHOT/Implementation-docs.md
+++ b/releases/2.0.0-SNAPSHOT/Implementation-docs.md
@@ -8,6 +8,7 @@ This section of the wiki is dedicated to explaining how Storm is implemented. Yo
 - [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)
 - [Nimbus HA](nimbus-ha-design.html)
 - [Storm SQL](storm-sql-internal.html)

http://git-wip-us.apache.org/repos/asf/storm-site/blob/ef81b5ca/releases/2.0.0-SNAPSHOT/Joins.md
----------------------------------------------------------------------
diff --git a/releases/2.0.0-SNAPSHOT/Joins.md b/releases/2.0.0-SNAPSHOT/Joins.md
new file mode 100644
index 0000000..9efb7c6
--- /dev/null
+++ b/releases/2.0.0-SNAPSHOT/Joins.md
@@ -0,0 +1,139 @@
+---
+title: Joining Streams in Storm Core
+layout: documentation
+documentation: true
+---
+
+Storm core supports joining multiple data streams into one with the help of `JoinBolt`.
+`JoinBolt` is a Windowed bolt, i.e. it waits for the configured window duration to match up the
+tuples among the streams being joined. This helps align the streams within a Window boundary.
+
+Each of `JoinBolt`'s incoming data streams must be Fields Grouped on a single field. A stream 
+should only be joined with the other streams using the field on which it has been FieldsGrouped.  
+Knowing this will help understand the join syntax described below.  
+
+## Performing Joins
+Consider the following SQL join involving 4 tables:
+
+```sql
+select  userId, key4, key2, key3
+from        table1
+inner join  table2  on table2.userId =  table1.key1
+inner join  table3  on table3.key3   =  table2.userId
+left join   table4  on table4.key4   =  table3.key3
+```
+
+Similar joins could be expressed on tuples generated by 4 spouts using `JoinBolt`:
+
+```java
+JoinBolt jbolt =  new JoinBolt("spout1", "key1")                   // from        spout1  
+                    .join     ("spout2", "userId",  "spout1")      // inner join  spout2  on spout2.userId = spout1.key1
+                    .join     ("spout3", "key3",    "spout2")      // inner join  spout3  on spout3.key3   = spout2.userId   
+                    .leftJoin ("spout4", "key4",    "spout3")      // left join   spout4  on spout4.key4   = spout3.key3
+                    .select  ("userId, key4, key2, spout3:key3")   // chose output fields
+                    .withTumblingWindow( new Duration(10, TimeUnit.MINUTES) ) ;
+
+topoBuilder.setBolt("joiner", jbolt, 1)
+            .fieldsGrouping("spout1", new Fields("key1") )
+            .fieldsGrouping("spout2", new Fields("userId") )
+            .fieldsGrouping("spout3", new Fields("key3") )
+            .fieldsGrouping("spout4", new Fields("key4") );
+```
+
+The bolt constructor takes two arguments. The 1st argument introduces the data from `spout1`
+to be the first stream and specifies that it will always use field `key1` when joining this with the others streams.
+The name of the component specified must refer to the spout or bolt that is directly connected to the Join bolt. 
+Here data received from `spout1` must be fields grouped on `key1`. Similarly, each of the `leftJoin()` and `join()` method
+calls introduce a new stream along with the field to use for the join. As seen in above example, the same FieldsGrouping
+requirement applies to these streams as well. The 3rd argument to the join methods refers to another stream with which
+to join.
+
+The `select()` method is used to specify the output fields. The argument to `select` is a comma separated list of fields.
+Individual field names can be prefixed with a stream name to disambiguate between the same field name occurring in
+multiple streams as follows:  `.select("spout3:key3, spout4:key3")`. Nested tuple types are supported if the
+nesting has been done using `Map`s. For example  `outer.inner.innermost` refers to a field that is nested three levels
+deep where `outer` and `inner` are of type `Map`.   
+
+Stream name prefix is not allowed for the fields in any of the join() arguments, but nested fields are supported. 
+
+The call to `withTumblingWindow()` above, configures the join window to be a 10 minute tumbling window. Since `JoinBolt` 
+is a Windowed Bolt, we can also use the `withWindow` method to configure it as a sliding window (see tips section below). 
+
+## Stream names and Join order
+* Stream names must be introduced (in constructor or as 1st arg to various join methods) before being referred
+to (in the 3rd argument of the join methods). Forward referencing of stream names, as shown below, is not allowed:
+
+```java
+new JoinBolt( "spout1", "key1")                 
+  .join     ( "spout2", "userId",  "spout3") //not allowed. spout3 not yet introduced
+  .join     ( "spout3", "key3",    "spout1")
+```
+* Internally, the joins will be performed in the order expressed by the user.
+
+## Joining based on Stream names
+
+For simplicity, Storm topologies often use the `default` stream. Topologies can also use named streams 
+instead of `default` streams.  To support such topologies, `JoinBolt` can be configured to use stream
+names,  instead of source component (spout/bolt) names, via the constructor's first argument:
+
+```java
+new JoinBolt(JoinBolt.Selector.STREAM,  "stream1", "key1")
+                                  .join("stream2", "key2")
+    ...
+```
+The first argument `JoinBolt.Selector.STREAM` informs the bolt that `stream1/2/3/4` refer to named streams
+(as opposed to names of upstream spouts/bolts).
+
+
+The below example joins two named streams from four spouts:
+
+```java
+new JoinBolt(JoinBolt.Selector.STREAM,  "stream1", "key1") 
+                             .join     ("stream2", "userId",  "stream1" )
+                             .select ("userId, key1, key2")
+                             .withTumblingWindow( new Duration(10, TimeUnit.MINUTES) ) ;
+                             
+topoBuilder.setBolt("joiner", jbolt, 1)
+            .fieldsGrouping("bolt1", "stream1", new Fields("key1") )
+            .fieldsGrouping("bolt2", "stream1", new Fields("key1") )
+            .fieldsGrouping("bolt3", "stream2", new Fields("userId") )
+            .fieldsGrouping("bolt4", "stream1", new Fields("key1") );
+```
+
+In the above example, it is possible that `bolt1`, for example, is emitting other streams also. But the join bolt 
+is only subscribing to `stream1` & `stream2` from the different bolts. `stream1` from `bolt1`, `bolt2` and `bolt4` 
+is treated as a single stream and joined against `stream2` from `bolt3`.
+
+## Limitations: 
+1. Currently only INNER and LEFT joins are supported. 
+
+2. Unlike SQL, which allows joining the same table on different keys to different tables, here the same one field must be used
+   on a stream. Fields Grouping ensures the right tuples are routed to the right instances of a Join Bolt. Consequently the 
+   FieldsGrouping field must be same as the join field, for correct results. To perform joins on multiple fields, the fields 
+   can be combined into one field and then sent to the Join bolt.  
+
+
+## Tips:
+
+1. Joins can be CPU and memory intensive. The larger the data accumulated in the current window (proportional to window
+   length), the longer it takes to do the join. Having a short sliding interval (few seconds for example) triggers frequent
+    joins. Consequently performance can suffer if using large window lengths or small sliding intervals or both.
+
+2. Duplication of joined records across windows is possible when using Sliding Windows. This is because the tuples continue to exist
+   across multiple windows when using Sliding Windows.
+
+3. If message timeouts are enabled, ensure the timeout setting (topology.message.timeout.secs) is large enough to comfortably
+   accommodate the window size, plus the additional processing by other spouts and bolts.
+
+4. Joining a window of two streams with M and N elements each, *in the worst case*, can produce MxN elements with every output tuple
+   anchored to one tuple from each input stream. This can mean a lot of output tuples from JoinBolt and even more ACKs for downstream bolts
+   to emit. This can place a substantial pressure on the messaging system and dramatically slowdown the topology if not careful.
+   To manage the load on the messaging subsystem, it is advisable to:
+   * Increase the worker's heap (topology.worker.max.heap.size.mb).
+   * **If** ACKing is not necessary for your topology, disable ACKers (topology.acker.executors=0).
+   * Disable event logger (topology.eventlogger.executors=0).
+   * Turn of topology debugging (topology.debug=false).
+   * Set topology.max.spout.pending to a value large enough to accommodate an estimated full window worth of tuples plus some more for headroom.
+     This helps mitigate the possibility of spouts emitting excessive tuples when messaging subsystem is experiencing excessive load. This situation
+     can occur when its value is set to null.
+   * Lastly, keep the window size to the minimum value necessary for solving the problem at hand.

http://git-wip-us.apache.org/repos/asf/storm-site/blob/ef81b5ca/releases/2.0.0-SNAPSHOT/Lifecycle-of-a-topology.md
----------------------------------------------------------------------
diff --git a/releases/2.0.0-SNAPSHOT/Lifecycle-of-a-topology.md b/releases/2.0.0-SNAPSHOT/Lifecycle-of-a-topology.md
index dba1457..fe785f1 100644
--- a/releases/2.0.0-SNAPSHOT/Lifecycle-of-a-topology.md
+++ b/releases/2.0.0-SNAPSHOT/Lifecycle-of-a-topology.md
@@ -3,7 +3,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/`.)
+(**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-client/src` rather than `src/`.)
 
 This page explains in detail the lifecycle of a topology from running the "storm jar" command to uploading the topology to Nimbus to the supervisors starting/stopping workers to workers and tasks setting themselves up. It also explains how Nimbus monitors topologies and how topologies are shutdown when they are killed.
 

http://git-wip-us.apache.org/repos/asf/storm-site/blob/ef81b5ca/releases/2.0.0-SNAPSHOT/Local-mode.md
----------------------------------------------------------------------
diff --git a/releases/2.0.0-SNAPSHOT/Local-mode.md b/releases/2.0.0-SNAPSHOT/Local-mode.md
index e3d9666..199d690 100644
--- a/releases/2.0.0-SNAPSHOT/Local-mode.md
+++ b/releases/2.0.0-SNAPSHOT/Local-mode.md
@@ -3,27 +3,82 @@ 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). 
+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).
+
+To run a topology in local mode you have two options.  The most common option is to run your topology with `storm local` instead of `storm jar`
+
+This will bring up a local simulated cluster and force all interactions with nimbus to go through the simulated cluster instead of going to a separate process.
+
+If you want to do some automated testing but without actually launching a storm cluster you can use the same classes internally that `storm local` does.
+
+To do this you first need to pull in the dependencies needed to access these classes.  For the java API you should depend on `storm-server` as a `test` dependency.
 
 To create an in-process cluster, simply use the `LocalCluster` class. For example:
 
 ```java
 import org.apache.storm.LocalCluster;
 
-LocalCluster cluster = new LocalCluster();
+...
+
+try (LocalCluster cluster = new LocalCluster()) {
+    //Interact with the cluster...
+}
 ```
 
 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:
+The `LocalCluster` is an `AutoCloseable` and will shut down when close is called. 
+
+many of the Nimbus APIs are also available through the LocalCluster.
+
+### DRPC
+
+DRPC can be run in local mode as well. Here's how to run the above example in local mode:
 
 ```java
-cluster.shutdown();
+try (LocalDRPC drpc = new LocalDRPC();
+     LocalCluster cluster = new LocalCluster();
+     LocalTopology topo = cluster.submitTopology("drpc-demo", conf, builder.createLocalTopology(drpc))) {
+
+    System.out.println("Results for 'hello':" + drpc.execute("exclamation", "hello"));
+}
 ```
 
+First you create a `LocalDRPC` object. This object simulates a DRPC server in process, just like how `LocalCluster` simulates a Storm cluster in process. Then you create the `LocalCluster` to run the topology in local mode. `LinearDRPCTopologyBuilder` has separate methods for creating local topologies and remote topologies. In local mode the `LocalDRPC` object does not bind to any ports so the topology needs to know about the object to communicate with it. This is why `createLocalTopology` takes in the `LocalDRPC` object as input.
+
+After launching the topology, you can do DRPC invocations using the `execute` method on `LocalDRPC`.
+
+Because all of the objects used are instances of AutoCloseable when the try blocks scope ends the topology is killed, the cluster is shut down and the drpc server also shuts down.
+
+### Clojure API
+
+Storm also offers a clojure API for testing.
+
+[This blog post](http://www.pixelmachine.org/2011/12/21/Testing-Storm-Topologies-Part-2.html) talk about this, but is a little out of date.  To get this functionality you need to include the `storm-clojure-test` dependency.  This will pull in a lot of storm itself that should not be packaged with your topology, sp please make sure it is a test dependency only,.
+
+### Debugging your topology with an IDE
+
+One of the great use cases for local mode is to be able to walk through the code execution of your bolts and spouts using an IDE.  You can do this on the command line by adding the `--java-debug` option followed by the parameter you would pass to jdwp. This makes it simple to launch the local cluster with `-agentlib:jdwp=` turned on.
+
+When running from within an IDE itself you can modify your code run run withing a call to `LocalCluster.withLocalModeOverride`
+
+```java
+public static void main(final String args[]) {
+    LocalCluster.withLocalModeOverride(() -> originalMain(args), 10);
+}
+```
+
+Or you could also modify the IDE to run "org.apache.storm.LocalCluster" instead of your main class when launching, and pass in the name of the class as an argument to it.  This will also trigger local mode, and is what `storm local` does behind the scenes. 
+
 ### Common configurations for local mode
 
 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.
+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.A
+
+These, like all other configs, can be set on the command line when launching your toplogy with the `-c` flag.  The flag is of the form `-c <conf_name>=<JSON_VALUE>`  so to enable debugging when launching your topology in local mode you could run
+
+```
+storm local topology.jar <MY_MAIN_CLASS> -c topology.debug=true
+``` 

http://git-wip-us.apache.org/repos/asf/storm-site/blob/ef81b5ca/releases/2.0.0-SNAPSHOT/Maven.md
----------------------------------------------------------------------
diff --git a/releases/2.0.0-SNAPSHOT/Maven.md b/releases/2.0.0-SNAPSHOT/Maven.md
index 0c09c2c..47fd1b1 100644
--- a/releases/2.0.0-SNAPSHOT/Maven.md
+++ b/releases/2.0.0-SNAPSHOT/Maven.md
@@ -9,7 +9,7 @@ To develop topologies, you'll need the Storm jars on your classpath. You should
 ```xml
 <dependency>
   <groupId>org.apache.storm</groupId>
-  <artifactId>storm-core</artifactId>
+  <artifactId>storm-client</artifactId>
   <version>{{page.version}}</version>
   <scope>provided</scope>
 </dependency>

http://git-wip-us.apache.org/repos/asf/storm-site/blob/ef81b5ca/releases/2.0.0-SNAPSHOT/Metrics.md
----------------------------------------------------------------------
diff --git a/releases/2.0.0-SNAPSHOT/Metrics.md b/releases/2.0.0-SNAPSHOT/Metrics.md
index 8537c7c..09ed2fc 100644
--- a/releases/2.0.0-SNAPSHOT/Metrics.md
+++ b/releases/2.0.0-SNAPSHOT/Metrics.md
@@ -4,22 +4,23 @@ 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.
+The numbers you see on the UI come from some of these built in metrics, but are reported through the worker heartbeats instead of through the IMetricsConsumer described below.
 
 ### Metric Types
 
-Metrics have to implement [`IMetric`]({{page.git-blob-base}}/storm-core/src/jvm/org/apache/storm/metric/api/IMetric.java) which contains just one method, `getValueAndReset` -- do any remaining work to find the summary value, and reset back to an initial state. For example, the MeanReducer divides the running total by its running count to find the mean, then initializes both values back to zero.
+Metrics have to implement [`IMetric`]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/metric/api/IMetric.java) which contains just one method, `getValueAndReset` -- do any remaining work to find the summary value, and reset back to an initial state. For example, the MeanReducer divides the running total by its running count to find the mean, then initializes both values back to zero.
 
 Storm gives you these metric types:
 
-* [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.
+* [AssignableMetric]({{page.git-blob-base}}/storm-client/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-client/src/jvm/org/apache/storm/metric/api/CombinedMetric.java) -- generic interface for metrics that can be updated associatively. 
+* [CountMetric]({{page.git-blob-base}}/storm-client/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-client/src/jvm/org/apache/storm/metric/api/MultiCountMetric.java) -- a hashmap of count metrics.
+* [ReducedMetric]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/metric/api/ReducedMetric.java)
+  - [MeanReducer]({{page.git-blob-base}}/storm-client/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-client/src/jvm/org/apache/storm/metric/api/MultiReducedMetric.java) -- a hashmap of reduced metrics.
 
+Be aware that even though `getValueAndReset` can return an object returning any object makes it very difficult for an `IMetricsConsumer` to know how to translate it into something usable.  Also note that because it is sent to the `IMetricsConsumer` as a part of a tuple the values returned need to be able to be [serialized](Serialization.html) by your topology.
 
 ### Metrics Consumer
 
@@ -44,14 +45,14 @@ topology.metrics.consumer.register:
     argument: "http://example.com:8080/metrics/my-topology/"
 ```
 
-Storm appends MetricsConsumerBolt to your topology per each registered metrics consumer internally, and each MetricsConsumerBolt subscribes to receive metrics from all tasks. The parallelism for that Bolt is set to `parallelism.hint` and `component id` for that Bolt is set to `__metrics_<metrics consumer class name>`. If you register same class name more than once, postfix `#<sequence number>` is appended to component id.
+Storm adds a MetricsConsumerBolt to your topolology for each class in the `topology.metrics.consumer.register` list. Each MetricsConsumerBolt subscribes to receive metrics from all tasks in the topology. The parallelism for each Bolt is set to `parallelism.hint` and `component id` for that Bolt is set to `__metrics_<metrics consumer class name>`. If you register the same class name more than once, postfix `#<sequence number>` is appended to component id.
 
 Storm provides some built-in metrics consumers for you to try out to see which metrics are provided in your topology.
 
-* [`LoggingMetricsConsumer`]({{page.git-blob-base}}/storm-core/src/jvm/org/apache/storm/metric/LoggingMetricsConsumer.java) -- listens for all metrics and dumps them to log file with TSV (Tab Separated Values).
-* [`HttpForwardingMetricsConsumer`]({{page.git-blob-base}}/storm-core/src/jvm/org/apache/storm/metric/HttpForwardingMetricsConsumer.java) -- listens for all metrics and POSTs them serialized to a configured URL via HTTP. Storm also provides [`HttpForwardingMetricsServer`]({{page.git-blob-base}}/storm-core/src/jvm/org/apache/storm/metric/HttpForwardingMetricsServer.java) as abstract class so you can extend this class and run as a HTTP server, and handle metrics sent by HttpForwardingMetricsConsumer.
+* [`LoggingMetricsConsumer`]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/metric/LoggingMetricsConsumer.java) -- listens for all metrics and dumps them to log file with TSV (Tab Separated Values).
+* [`HttpForwardingMetricsConsumer`]({{page.git-blob-base}}/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/HttpForwardingMetricsConsumer.java) -- listens for all metrics and POSTs them serialized to a configured URL via HTTP. Storm also provides [`HttpForwardingMetricsServer`]({{page.git-blob-base}}/storm-core/src/jvm/org/apache/storm/metric/HttpForwardingMetricsServer.java) as abstract class so you can extend this class and run as a HTTP server, and handle metrics sent by HttpForwardingMetricsConsumer.
 
-Also, Storm exposes the interface [`IMetricsConsumer`]({{page.git-blob-base}}/storm-core/src/jvm/org/apache/storm/metric/api/IMetricsConsumer.java) for implementing Metrics Consumer so you can create custom metrics consumers and attach to their topologies, or use other great implementation of Metrics Consumers provided by Storm community. Some of examples are [versign/storm-graphite](https://github.com/verisign/storm-graphite), and [storm-metrics-statsd](https://github.com/endgameinc/storm-metrics-statsd).
+Also, Storm exposes the interface [`IMetricsConsumer`]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/metric/api/IMetricsConsumer.java) for implementing Metrics Consumer so you can create custom metrics consumers and attach to their topologies, or use other great implementation of Metrics Consumers provided by Storm community. Some of examples are [versign/storm-graphite](https://github.com/verisign/storm-graphite), and [storm-metrics-statsd](https://github.com/endgameinc/storm-metrics-statsd).
 
 When you implement your own metrics consumer, `argument` is passed to Object when [IMetricsConsumer#prepare](javadocs/org/apache/storm/metric/api/IMetricsConsumer.html#prepare-java.util.Map-java.lang.Object-org.apache.storm.task.TopologyContext-org.apache.storm.task.IErrorReporter-) is called, so you need to infer the Java type of configured value on yaml, and do explicit type casting.
 
@@ -75,7 +76,7 @@ Next, let's initialize and register the metric instance.
 ```java
 @Override
 public void prepare(Map conf, TopologyContext context, OutputCollector collector) {
-	// other intialization here.
+	// other initialization here.
 	countMetric = new CountMetric();
 	context.registerMetric("execute_count", countMetric, 60);
 }
@@ -111,7 +112,7 @@ worker.metrics:
 
 or put `Map<String, String>` (metric name, metric class name) with key `Config.TOPOLOGY_WORKER_METRICS` to config map.
 
-There're some restrictions for worker level metric instances: 
+There are some restrictions for worker level metric instances: 
 
 A) Metrics for worker level should be kind of gauge since it is initialized and registered from SystemBolt and not exposed to user tasks.
 
@@ -121,7 +122,207 @@ C) Bucket size (seconds) for metrics is fixed to `Config.TOPOLOGY_BUILTIN_METRIC
 
 ### Builtin Metrics
 
-The [builtin metrics]({{page.git-blob-base}}/storm-core/src/clj/org/apache/storm/daemon/builtin_metrics.clj) instrument Storm itself.
+The [builtin metrics]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/daemon/metrics/BuiltinMetricsUtil.java) instrument Storm itself.
 
-[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`
+[BuiltinMetricsUtil.java]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/daemon/metrics/BuiltinMetricsUtil.java) 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 [`ackSpoutMsg`]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/executor/Executor.java).
 
+#### Reporting Rate
+
+The rate at which built in metrics are reported is configurable through the `topology.builtin.metrics.bucket.size.secs` config.  If you set this too low it can overload the consumers,
+so please use caution when modifying it.
+
+#### Tuple Counting Metrics
+
+There are several different metrics related to counting what a bolt or spout does to a tuple. These include things like emitting, transferring, acking, and failing of tuples.
+
+In general all of these tuple count metrics are randomly sub-sampled unless otherwise stated.  This means that the counts you see both on the UI and from the built in metrics are not necessarily exact.  In fact by default we sample only 5% of the events and estimate the total number of events from that.  The sampling percentage is configurable per topology through the `topology.stats.sample.rate` config.  Setting it to 1.0 will make the counts exact, but be aware that the more events we sample the slower your topology will run (as the metrics are counted in the same code path as tuples are processed).  This is why we have a 5% sample rate as the default.
+
+The tuple counting metrics are generally reported to the metrics consumers as maps unless explicitly stated otherwise.  They break down each count for finer grained reporting.
+The keys to these maps fall into two categories `"${stream_name}"` or `"${upstream_component}:${stream_name}"`.  The former is used for all spout metrics and for outgoing bolt metrics (`__emit-count` and `__transfer-count`).  The latter is used for bolt metrics that deal with incoming tuples.
+
+So for a word count topology the count bolt might show something like the following for the `__ack-count` metric
+
+```
+{
+    "split:default": 80080
+}
+```
+
+But the spout instead would show something like the following for the `__ack-count` metric.
+
+```
+{
+    "default": 12500
+}
+```
+
+
+##### `__ack-count`
+
+For bolts it is the number of incoming tuples that had the `ack` method called on them.  For spouts it is the number of tuples trees that were fully acked. See Guaranteeing Message Processing[](Guaranteeing-message-processing.html) for more information about what a tuple tree is. If acking is disabled this metric is still reported, but it is not really meaningful.
+
+##### `__fail-count`
+
+For bolts this is the number of incoming tuples that had the `fail` method called on them.  For spouts this is the number of tuple trees that failed.  Tuple trees may fail from timing out or because a bolt called fail on it.  The two are not separated out by this metric.
+
+##### `__emit-count`
+
+This is the total number of times the `emit` method was called to send a tuple.  This is the same for both bolts and spouts.
+
+##### `__transfer-count`
+
+This is the total number of tuples transferred to a downstream bolt/spout for processing. This number will not always match `__emit_count`.  If nothing is registered to receive a tuple down stream the number will be 0 even if tuples were emitted.  Similarly if there are multiple down stream consumers it may be a multiple of the number emitted.  The grouping also can play a role if it sends the tuple to multiple instances of a single bolt down stream.
+
+##### `__execute-count`
+
+This count metric is bolt specific.  It counts the number of times that a bolt's `execute` method was called.
+
+#### Tuple Latency Metrics
+
+Similar to the tuple counting metrics storm also collects average latency metrics for bolts and spouts.  These follow the same structure as the bolt/spout maps and are sub-sampled in the same way as well.  In all cases the latency is measured in milliseconds.
+
+##### `__complete-latency`
+
+The complete latency is just for spouts.  It is the average amount of time it took for `ack` or `fail` to be called for a tuple after it was emitted.  If acking is disabled this metric is likely to be blank or 0 for all values, and should be ignored.
+
+##### `__execute-latency`
+
+This is just for bolts.  It is the average amount of time that the bolt spent in the call to the `execute` method.  The higher this gets, the lower the throughput of tuples per bolt instance.
+
+##### `__process-latency`
+
+This is also just for bolts.  It is the average amount of time between when `execute` was called to start processing a tuple, to when it was acked or failed by the bolt.  If your bolt is a very simple bolt and the processing is synchronous then `__process-latency` and `__execute-latency` should be very close to one another, with process latency being slightly smaller.  If you are doing a join or have asynchronous processing then it may take a while for a tuple to be acked so the process latency would be higher than the execute latency.
+
+##### `__skipped-max-spout-ms`
+
+This metric records how much time a spout was idle because more tuples than `topology.max.spout.pending` were still outstanding.  This is the total time in milliseconds, not the average amount of time and is not sub-sampled.
+
+
+##### `__skipped-backpressure-ms`
+
+This metric records how much time a spout was idle because back-pressure indicated that downstream queues in the topology were too full.  This is the total time in milliseconds, not the average amount of time and is not sub-sampled. This is similar to skipped-throttle-ms in Storm 1.x.
+
+##### `skipped-inactive-ms`
+
+This metric records how much time a spout was idle because the topology was deactivated.  This is the total time in milliseconds, not the average amount of time and is not sub-sampled.
+
+#### Error Reporting Metrics
+
+Storm also collects error reporting metrics for bolts and spouts.
+
+##### `__reported-error-count`
+
+This metric records how many errors were reported by a spout/bolt. It is the total number of times the `reportError` method was called.
+
+#### Queue Metrics
+
+Each bolt or spout instance in a topology has a receive queue and a send queue.  Each worker also has a queue for sending messages to other workers.  All of these have metrics that are reported.
+
+The receive queue metrics are reported under the `__receive` name and send queue metrics are reported under the `__sendqueue` for the given bolt/spout they are a part of.  The metrics for the queue that sends messages to other workers is under the `__transfer` metric name for the system bolt (`__system`).
+
+They all have the form.
+
+```
+{
+    "arrival_rate_secs": 1229.1195171893523,
+    "overflow": 0,
+    "read_pos": 103445,
+    "write_pos": 103448,
+    "sojourn_time_ms": 2.440771591407277,
+    "capacity": 1024,
+    "population": 19
+    "tuple_population": 200
+}
+```
+In storm we sometimes batch multiple tuples into a single entry in the disruptor queue. This batching is an optimization that has been in storm in some form since the beginning, but the metrics did not always reflect this so be careful with how you interpret the metrics and pay attention to which metrics are for tuples and which metrics are for entries in the disruptor queue. The `__receive` and `__transfer` queues can have batching but the `__sendqueue` should not.
+
+`arrival_rate_secs` is an estimation of the number of tuples that are inserted into the queue in one second, although it is actually the dequeue rate.
+The `sojourn_time_ms` is calculated from the arrival rate and is an estimate of how many milliseconds each tuple sits in the queue before it is processed.
+Prior to STORM-2621 (v1.1.1, v1.2.0, and v2.0.0) these were the rate of entries, not of tuples.
+
+A disruptor queue has a set maximum number of entries.  If the regular queue fills up an overflow queue takes over.  The number of tuple batches stored in this overflow section are represented by the `overflow` metric.  Storm also does some micro batching of tuples for performance/efficiency reasons so you may see the overflow with a very small number in it even if the queue is not full.
+
+`read_pos` and `write_pos` are internal disruptor accounting numbers.  You can think of them almost as the total number of entries written (`write_pos`) or read (`read_pos`) since the queue was created.  They allow for integer overflow so if you use them please take that into account.
+
+`capacity` is the maximum number of entries in the disruptor queue. `population` is the number of entries currently filled in the queue.
+
+`tuple_population` is the number of tuples currently in the queue as opposed to the number of entries.  This was added at the same time as STORM-2621 (v1.1.1, v1.2.0, and v2.0.0)
+
+#### System Bolt (Worker) Metrics
+
+The System Bolt `__system` provides lots of metrics for different worker wide things.  The one metric not described here is the `__transfer` queue metric, because it fits with the other disruptor metrics described above.
+
+Be aware that the `__system` bolt is an actual bolt so regular bolt metrics described above also will be reported for it.
+
+##### Receive (NettyServer)
+`__recv-iconnection` reports stats for the netty server on the worker.  This is what gets messages from other workers.  It is of the form
+
+```
+{
+    "dequeuedMessages": 0,
+    "enqueued": {
+      "/127.0.0.1:49952": 389951
+    }
+}
+```
+
+`dequeuedMessages` is a throwback to older code where there was an internal queue between the server and the bolts/spouts.  That is no longer the case and the value can be ignored.
+`enqueued` is a map between the address of the remote worker and the number of tuples that were sent from it to this worker.
+
+##### Send (Netty Client)
+
+The `__send-iconnection` metric holds information about all of the clients for this worker.  It is of the form
+
+```
+{
+    NodeInfo(node:7decee4b-c314-41f4-b362-fd1358c985b3-127.0.01, port:[6701]): {
+        "reconnects": 0,
+        "src": "/127.0.0.1:49951",
+        "pending": 0,
+        "dest": "localhost/127.0.0.1:6701",
+        "sent": 420779,
+        "lostOnSend": 0
+    }
+}
+```
+
+The value is a map where the key is a NodeInfo class for the downstream worker it is sending messages to.  This is the SupervisorId + port.  The value is another map with the fields
+
+ * `src`  What host/port this client has used to connect to the receiving worker.
+ * `dest` What host/port this client has connected to.
+ * `reconnects` the number of reconnections that have happened.
+ * `pending` the number of messages that have not been sent.  (This corresponds to messages, not tuples)
+ * `sent` the number of messages that have been sent.  (This is messages not tuples)
+ * `lostOnSend`.  This is the number of messages that were lost because of connection issues. (This is messages not tuples). 
+
+##### JVM Memory
+
+JVM memory usage is reported through `memory/nonHeap` for off heap memory and `memory/heap` for on heap memory.  These values come from the [MemoryUsage](https://docs.oracle.com/javase/8/docs/api/index.html?java/lang/management/MemoryUsage.html) mxbean.  Each of the metrics are reported as a map with the following keys, and values returned by the corresponding java code.
+
+| Key | Corresponding Code |
+|--------|--------------------|
+| `maxBytes` | `memUsage.getMax()` |
+| `committedBytes` | `memUsage.getCommitted()` |
+| `initBytes` | `memUsage.getInit()` |
+| `usedBytes` | `memUsage.getUsed()` |
+| `virtualFreeBytes` | `memUsage.getMax() - memUsage.getUsed()` |
+| `unusedBytes` | `memUsage.getCommitted() - memUsage.getUsed()` |
+
+##### JVM Garbage Collection
+
+The exact GC metric name depends on the garbage collector that your worker uses.  The data is all collected from `ManagementFactory.getGarbageCollectorMXBeans()` and the name of the metrics is `"GC/"` followed by the name of the returned bean with white space removed.  The reported metrics are just
+
+* `count` the number of gc events that happened and
+* `timeMs` the total number of milliseconds that were spent doing gc.  
+
+Please refer to the [JVM documentation](https://docs.oracle.com/javase/8/docs/api/java/lang/management/ManagementFactory.html#getGarbageCollectorMXBeans--) for more details.
+
+##### JVM Misc
+
+* `threadCount` is the number of threads currently in the JVM.
+
+##### Uptime
+
+* `uptimeSecs` reports the number of seconds the worker has been up for
+* `newWorkerEvent` is 1 when a worker is first started and 0 all other times.  This can be used to tell when a worker has crashed and is restarted.
+* `startTimeSecs` is when the worker started in seconds since the epoch

http://git-wip-us.apache.org/repos/asf/storm-site/blob/ef81b5ca/releases/2.0.0-SNAPSHOT/Multilang-protocol.md
----------------------------------------------------------------------
diff --git a/releases/2.0.0-SNAPSHOT/Multilang-protocol.md b/releases/2.0.0-SNAPSHOT/Multilang-protocol.md
index ea010f2..45ad2bc 100644
--- a/releases/2.0.0-SNAPSHOT/Multilang-protocol.md
+++ b/releases/2.0.0-SNAPSHOT/Multilang-protocol.md
@@ -140,7 +140,7 @@ What happens next depends on the type of component:
 
 Shell spouts are synchronous. The rest happens in a while(true) loop:
 
-* STDIN: Either a next, ack, or fail command.
+* STDIN: Either a next, ack, activate, deactivate or fail command.
 
 "next" is the equivalent of ISpout's `nextTuple`. It looks like:
 
@@ -154,6 +154,16 @@ Shell spouts are synchronous. The rest happens in a while(true) loop:
 {"command": "ack", "id": "1231231"}
 ```
 
+"activate" is the equivalent of ISpout's `activate`:
+```
+{"command": "activate"}
+```
+
+"deactivate" is the equivalent of ISpout's `deactivate`:
+```
+{"command": "deactivate"}
+```
+
 "fail" looks like:
 
 ```