You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@storm.apache.org by pt...@apache.org on 2016/01/15 17:22:45 UTC

[1/4] storm git commit: update documentation

Repository: storm
Updated Branches:
  refs/heads/asf-site 573bdcf1d -> 3590e2ee2


http://git-wip-us.apache.org/repos/asf/storm/blob/d63146b7/documentation/ui-rest-api.md
----------------------------------------------------------------------
diff --git a/documentation/ui-rest-api.md b/documentation/ui-rest-api.md
new file mode 100644
index 0000000..7de2bcb
--- /dev/null
+++ b/documentation/ui-rest-api.md
@@ -0,0 +1,996 @@
+# Storm UI REST API
+
+The Storm UI daemon provides a REST API that allows you to interact with a Storm cluster, which includes retrieving
+metrics data and configuration information as well as management operations such as starting or stopping topologies.
+
+
+# Data format
+
+The REST API returns JSON responses and supports JSONP.
+Clients can pass a callback query parameter to wrap JSON in the callback function.
+
+
+# Using the UI REST API
+
+_Note: It is recommended to ignore undocumented elements in the JSON response because future versions of Storm may not_
+_support those elements anymore._
+
+
+## REST API Base URL
+
+The REST API is part of the UI daemon of Storm (started by `storm ui`) and thus runs on the same host and port as the
+Storm UI (the UI daemon is often run on the same host as the Nimbus daemon).  The port is configured by `ui.port`,
+which is set to `8080` by default (see [defaults.yaml](conf/defaults.yaml)).
+
+The API base URL would thus be:
+
+    http://<ui-host>:<ui-port>/api/v1/...
+
+You can use a tool such as `curl` to talk to the REST API:
+
+    # Request the cluster configuration.
+    # Note: We assume ui.port is configured to the default value of 8080.
+    $ curl http://<ui-host>:8080/api/v1/cluster/configuration
+
+##Impersonating a user in secure environment
+In a secure environment an authenticated user can impersonate another user. To impersonate a user the caller must pass
+`doAsUser` param or header with value set to the user that the request needs to be performed as. Please see SECURITY.MD
+to learn more about how to setup impersonation ACLs and authorization. The rest API uses the same configs and acls that
+are used by nimbus.
+
+Examples:
+
+```no-highlight
+ 1. http://ui-daemon-host-name:8080/api/v1/topology/wordcount-1-1425844354\?doAsUser=testUSer1
+ 2. curl 'http://localhost:8080/api/v1/topology/wordcount-1-1425844354/activate' -X POST -H 'doAsUser:testUSer1'
+```
+
+## GET Operations
+
+### /api/v1/cluster/configuration (GET)
+
+Returns the cluster configuration.
+
+Sample response (does not include all the data fields):
+
+```json
+  {
+    "dev.zookeeper.path": "/tmp/dev-storm-zookeeper",
+    "topology.tick.tuple.freq.secs": null,
+    "topology.builtin.metrics.bucket.size.secs": 60,
+    "topology.fall.back.on.java.serialization": true,
+    "topology.max.error.report.per.interval": 5,
+    "zmq.linger.millis": 5000,
+    "topology.skip.missing.kryo.registrations": false,
+    "storm.messaging.netty.client_worker_threads": 1,
+    "ui.childopts": "-Xmx768m",
+    "storm.zookeeper.session.timeout": 20000,
+    "nimbus.reassign": true,
+    "topology.trident.batch.emit.interval.millis": 500,
+    "storm.messaging.netty.flush.check.interval.ms": 10,
+    "nimbus.monitor.freq.secs": 10,
+    "logviewer.childopts": "-Xmx128m",
+    "java.library.path": "/usr/local/lib:/opt/local/lib:/usr/lib",
+    "topology.executor.send.buffer.size": 1024,
+    }
+```
+
+### /api/v1/cluster/summary (GET)
+
+Returns cluster summary information such as nimbus uptime or number of supervisors.
+
+Response fields:
+
+|Field  |Value|Description
+|---	|---	|---
+|stormVersion|String| Storm version|
+|supervisors|Integer| Number of supervisors running|
+|topologies| Integer| Number of topologies running| 
+|slotsTotal| Integer|Total number of available worker slots|
+|slotsUsed| Integer| Number of worker slots used|
+|slotsFree| Integer |Number of worker slots available|
+|executorsTotal| Integer |Total number of executors|
+|tasksTotal| Integer |Total tasks|
+
+Sample response:
+
+```json
+   {
+    "stormVersion": "0.9.2-incubating-SNAPSHOT",
+    "supervisors": 1,
+    "slotsTotal": 4,
+    "slotsUsed": 3,
+    "slotsFree": 1,
+    "executorsTotal": 28,
+    "tasksTotal": 28
+    }
+```
+
+### /api/v1/supervisor/summary (GET)
+
+Returns summary information for all supervisors.
+
+Response fields:
+
+|Field  |Value|Description|
+|---	|---	|---
+|id| String | Supervisor's id|
+|host| String| Supervisor's host name|
+|uptime| String| Shows how long the supervisor is running|
+|uptimeSeconds| Integer| Shows how long the supervisor is running in seconds|
+|slotsTotal| Integer| Total number of available worker slots for this supervisor|
+|slotsUsed| Integer| Number of worker slots used on this supervisor|
+|totalMem| Double| Total memory capacity on this supervisor|
+|totalCpu| Double| Total CPU capacity on this supervisor|
+|usedMem| Double| Used memory capacity on this supervisor|
+|usedCpu| Double| Used CPU capacity on this supervisor|
+
+Sample response:
+
+```json
+{
+    "supervisors": [
+        {
+            "id": "0b879808-2a26-442b-8f7d-23101e0c3696",
+            "host": "10.11.1.7",
+            "uptime": "5m 58s",
+            "uptimeSeconds": 358,
+            "slotsTotal": 4,
+            "slotsUsed": 3,
+            "totalMem": 3000,
+            "totalCpu": 400,
+            "usedMem": 1280,
+            "usedCPU": 160
+        }
+    ],
+    "schedulerDisplayResource": true
+}
+```
+
+### /api/v1/nimbus/summary (GET)
+
+Returns summary information for all nimbus hosts.
+
+Response fields:
+
+|Field  |Value|Description|
+|---	|---	|---
+|host| String | Nimbus' host name|
+|port| int| Nimbus' port number|
+|status| String| Possible values are Leader, Not a Leader, Dead|
+|nimbusUpTime| String| Shows since how long the nimbus has been running|
+|nimbusUpTimeSeconds| String| Shows since how long the nimbus has been running in seconds|
+|nimbusLogLink| String| Logviewer url to view the nimbus.log|
+|version| String| Version of storm this nimbus host is running|
+
+Sample response:
+
+```json
+{
+    "nimbuses":[
+        {
+            "host":"192.168.202.1",
+            "port":6627,
+            "nimbusLogLink":"http:\/\/192.168.202.1:8000\/log?file=nimbus.log",
+            "status":Leader,
+            "version":"0.10.0-SNAPSHOT",
+            "nimbusUpTime":"3m 33s",
+            "nimbusUpTimeSeconds":"213"
+        }
+    ]
+}
+```
+
+### /api/v1/history/summary (GET)
+
+Returns a list of all running topologies' IDs submitted by the current user.
+
+Response fields:
+
+|Field  |Value | Description|
+|---	|---	|---
+|topo-history| List| List of Topologies' IDs|
+
+Sample response:
+
+```json
+{
+    "topo-history":[
+        "wc6-1-1446571009",
+        "wc8-2-1446587178"
+     ]
+}
+```
+
+### /api/v1/topology/summary (GET)
+
+Returns summary information for all topologies.
+
+Response fields:
+
+|Field  |Value | Description|
+|---	|---	|---
+|id| String| Topology Id|
+|name| String| Topology Name|
+|status| String| Topology Status|
+|uptime| String|  Shows how long the topology is running|
+|uptimeSeconds| Integer|  Shows how long the topology is running in seconds|
+|tasksTotal| Integer |Total number of tasks for this topology|
+|workersTotal| Integer |Number of workers used for this topology|
+|executorsTotal| Integer |Number of executors used for this topology|
+|replicationCount| Integer |Number of nimbus hosts on which this topology code is replicated|
+|requestedMemOnHeap| Double|Requested On-Heap Memory by User (MB)
+|requestedMemOffHeap| Double|Requested Off-Heap Memory by User (MB)|
+|requestedTotalMem| Double|Requested Total Memory by User (MB)|
+|requestedCpu| Double|Requested CPU by User (%)|
+|assignedMemOnHeap| Double|Assigned On-Heap Memory by Scheduler (MB)|
+|assignedMemOffHeap| Double|Assigned Off-Heap Memory by Scheduler (MB)|
+|assignedTotalMem| Double|Assigned Total Memory by Scheduler (MB)|
+|assignedCpu| Double|Assigned CPU by Scheduler (%)|
+
+Sample response:
+
+```json
+{
+    "topologies": [
+        {
+            "id": "WordCount3-1-1402960825",
+            "name": "WordCount3",
+            "status": "ACTIVE",
+            "uptime": "6m 5s",
+            "uptimeSeconds": 365,
+            "tasksTotal": 28,
+            "workersTotal": 3,
+            "executorsTotal": 28,
+            "replicationCount": 1,
+            "requestedMemOnHeap": 640,
+            "requestedMemOffHeap": 128,
+            "requestedTotalMem": 768,
+            "requestedCpu": 80,
+            "assignedMemOnHeap": 640,
+            "assignedMemOffHeap": 128,
+            "assignedTotalMem": 768,
+            "assignedCpu": 80
+        }
+    ]
+    "schedulerDisplayResource": true
+}
+```
+
+### /api/v1/topology-workers/:id (GET)
+
+Returns the worker' information (host and port) for a topology.
+
+Response fields:
+
+|Field  |Value | Description|
+|---	|---	|---
+|hostPortList| List| Workers' information for a topology|
+|name| Integer| Logviewer Port|
+
+Sample response:
+
+```json
+{
+    "hostPortList":[
+            {
+                "host":"192.168.202.2",
+                "port":6701
+            },
+            {
+                "host":"192.168.202.2",
+                "port":6702
+            },
+            {
+                "host":"192.168.202.3",
+                "port":6700
+            }
+        ],
+    "logviewerPort":8000
+}
+```
+
+### /api/v1/topology/:id (GET)
+
+Returns topology information and statistics.  Substitute id with topology id.
+
+Request parameters:
+
+|Parameter |Value   |Description  |
+|----------|--------|-------------|
+|id   	   |String (required)| Topology Id  |
+|window    |String. Default value :all-time| Window duration for metrics in seconds|
+|sys       |String. Values 1 or 0. Default value 0| Controls including sys stats part of the response|
+
+
+Response fields:
+
+|Field  |Value |Description|
+|---	|---	|---
+|id| String| Topology Id|
+|name| String |Topology Name|
+|uptime| String |How long the topology has been running|
+|uptimeSeconds| Integer |How long the topology has been running in seconds|
+|status| String |Current status of the topology, e.g. "ACTIVE"|
+|tasksTotal| Integer |Total number of tasks for this topology|
+|workersTotal| Integer |Number of workers used for this topology|
+|executorsTotal| Integer |Number of executors used for this topology|
+|msgTimeout| Integer | Number of seconds a tuple has before the spout considers it failed |
+|windowHint| String | window param value in "hh mm ss" format. Default value is "All Time"|
+|schedulerDisplayResource| Boolean | Whether to display scheduler resource information|
+|topologyStats| Array | Array of all the topology related stats per time window|
+|topologyStats.windowPretty| String |Duration passed in HH:MM:SS format|
+|topologyStats.window| String |User requested time window for metrics|
+|topologyStats.emitted| Long |Number of messages emitted in given window|
+|topologyStats.trasferred| Long |Number messages transferred in given window|
+|topologyStats.completeLatency| String (double value returned in String format) |Total latency for processing the message|
+|topologyStats.acked| Long |Number of messages acked in given window|
+|topologyStats.failed| Long |Number of messages failed in given window|
+|spouts| Array | Array of all the spout components in the topology|
+|spouts.spoutId| String |Spout id|
+|spouts.executors| Integer |Number of executors for the spout|
+|spouts.emitted| Long |Number of messages emitted in given window |
+|spouts.completeLatency| String (double value returned in String format) |Total latency for processing the message|
+|spouts.transferred| Long |Total number of messages  transferred in given window|
+|spouts.tasks| Integer |Total number of tasks for the spout|
+|spouts.lastError| String |Shows the last error happened in a spout|
+|spouts.errorLapsedSecs| Integer | Number of seconds elapsed since that last error happened in a spout|
+|spouts.errorWorkerLogLink| String | Link to the worker log that reported the exception |
+|spouts.acked| Long |Number of messages acked|
+|spouts.failed| Long |Number of messages failed|
+|bolts| Array | Array of bolt components in the topology|
+|bolts.boltId| String |Bolt id|
+|bolts.capacity| String (double value returned in String format) |This value indicates number of messages executed * average execute latency / time window|
+|bolts.processLatency| String (double value returned in String format)  |Average time of the bolt to ack a message after it was received|
+|bolts.executeLatency| String (double value returned in String format) |Average time to run the execute method of the bolt|
+|bolts.executors| Integer |Number of executor tasks in the bolt component|
+|bolts.tasks| Integer |Number of instances of bolt|
+|bolts.acked| Long |Number of tuples acked by the bolt|
+|bolts.failed| Long |Number of tuples failed by the bolt|
+|bolts.lastError| String |Shows the last error occurred in the bolt|
+|bolts.errorLapsedSecs| Integer |Number of seconds elapsed since that last error happened in a bolt|
+|bolts.errorWorkerLogLink| String | Link to the worker log that reported the exception |
+|bolts.emitted| Long |Number of tuples emitted|
+|replicationCount| Integer |Number of nimbus hosts on which this topology code is replicated|
+
+Examples:
+
+```no-highlight
+ 1. http://ui-daemon-host-name:8080/api/v1/topology/WordCount3-1-1402960825
+ 2. http://ui-daemon-host-name:8080/api/v1/topology/WordCount3-1-1402960825?sys=1
+ 3. http://ui-daemon-host-name:8080/api/v1/topology/WordCount3-1-1402960825?window=600
+```
+
+Sample response:
+
+```json
+ {
+    "name": "WordCount3",
+    "id": "WordCount3-1-1402960825",
+    "workersTotal": 3,
+    "window": "600",
+    "status": "ACTIVE",
+    "tasksTotal": 28,
+    "executorsTotal": 28,
+    "uptime": "29m 19s",
+    "uptimeSeconds": 1759,
+    "msgTimeout": 30,
+    "windowHint": "10m 0s",
+    "schedulerDisplayResource": true,
+    "topologyStats": [
+        {
+            "windowPretty": "10m 0s",
+            "window": "600",
+            "emitted": 397960,
+            "transferred": 213380,
+            "completeLatency": "0.000",
+            "acked": 213460,
+            "failed": 0
+        },
+        {
+            "windowPretty": "3h 0m 0s",
+            "window": "10800",
+            "emitted": 1190260,
+            "transferred": 638260,
+            "completeLatency": "0.000",
+            "acked": 638280,
+            "failed": 0
+        },
+        {
+            "windowPretty": "1d 0h 0m 0s",
+            "window": "86400",
+            "emitted": 1190260,
+            "transferred": 638260,
+            "completeLatency": "0.000",
+            "acked": 638280,
+            "failed": 0
+        },
+        {
+            "windowPretty": "All time",
+            "window": ":all-time",
+            "emitted": 1190260,
+            "transferred": 638260,
+            "completeLatency": "0.000",
+            "acked": 638280,
+            "failed": 0
+        }
+    ],
+    "spouts": [
+        {
+            "executors": 5,
+            "emitted": 28880,
+            "completeLatency": "0.000",
+            "transferred": 28880,
+            "acked": 0,
+            "spoutId": "spout",
+            "tasks": 5,
+            "lastError": "",
+            "errorLapsedSecs": null,
+            "failed": 0
+        }
+    ],
+        "bolts": [
+        {
+            "executors": 12,
+            "emitted": 184580,
+            "transferred": 0,
+            "acked": 184640,
+            "executeLatency": "0.048",
+            "tasks": 12,
+            "executed": 184620,
+            "processLatency": "0.043",
+            "boltId": "count",
+            "lastError": "",
+            "errorLapsedSecs": null,
+            "capacity": "0.003",
+            "failed": 0
+        },
+        {
+            "executors": 8,
+            "emitted": 184500,
+            "transferred": 184500,
+            "acked": 28820,
+            "executeLatency": "0.024",
+            "tasks": 8,
+            "executed": 28780,
+            "processLatency": "2.112",
+            "boltId": "split",
+            "lastError": "",
+            "errorLapsedSecs": null,
+            "capacity": "0.000",
+            "failed": 0
+        }
+    ],
+    "configuration": {
+        "storm.id": "WordCount3-1-1402960825",
+        "dev.zookeeper.path": "/tmp/dev-storm-zookeeper",
+        "topology.tick.tuple.freq.secs": null,
+        "topology.builtin.metrics.bucket.size.secs": 60,
+        "topology.fall.back.on.java.serialization": true,
+        "topology.max.error.report.per.interval": 5,
+        "zmq.linger.millis": 5000,
+        "topology.skip.missing.kryo.registrations": false,
+        "storm.messaging.netty.client_worker_threads": 1,
+        "ui.childopts": "-Xmx768m",
+        "storm.zookeeper.session.timeout": 20000,
+        "nimbus.reassign": true,
+        "topology.trident.batch.emit.interval.millis": 500,
+        "storm.messaging.netty.flush.check.interval.ms": 10,
+        "nimbus.monitor.freq.secs": 10,
+        "logviewer.childopts": "-Xmx128m",
+        "java.library.path": "/usr/local/lib:/opt/local/lib:/usr/lib",
+        "topology.executor.send.buffer.size": 1024,
+        "storm.local.dir": "storm-local",
+        "storm.messaging.netty.buffer_size": 5242880,
+        "supervisor.worker.start.timeout.secs": 120,
+        "topology.enable.message.timeouts": true,
+        "nimbus.cleanup.inbox.freq.secs": 600,
+        "nimbus.inbox.jar.expiration.secs": 3600,
+        "drpc.worker.threads": 64,
+        "topology.worker.shared.thread.pool.size": 4,
+        "nimbus.host": "hw10843.local",
+        "storm.messaging.netty.min_wait_ms": 100,
+        "storm.zookeeper.port": 2181,
+        "transactional.zookeeper.port": null,
+        "topology.executor.receive.buffer.size": 1024,
+        "transactional.zookeeper.servers": null,
+        "storm.zookeeper.root": "/storm",
+        "storm.zookeeper.retry.intervalceiling.millis": 30000,
+        "supervisor.enable": true,
+        "storm.messaging.netty.server_worker_threads": 1
+    },
+    "replicationCount": 1
+}
+```
+
+
+### /api/v1/topology/:id/component/:component (GET)
+
+Returns detailed metrics and executor information
+
+|Parameter |Value   |Description  |
+|----------|--------|-------------|
+|id   	   |String (required)| Topology Id  |
+|component |String (required)| Component Id |
+|window    |String. Default value :all-time| window duration for metrics in seconds|
+|sys       |String. Values 1 or 0. Default value 0| controls including sys stats part of the response|
+
+Response fields:
+
+|Field  |Value |Description|
+|---	|---	|---
+|id   | String | Component id|
+|name | String | Topology name|
+|componentType | String | component type: SPOUT or BOLT|
+|windowHint| String | window param value in "hh mm ss" format. Default value is "All Time"|
+|executors| Integer |Number of executor tasks in the component|
+|componentErrors| Array of Errors | List of component errors|
+|componentErrors.errorTime| Long | Timestamp when the exception occurred (Prior to 0.11.0, this field was named 'time'.)|
+|componentErrors.errorHost| String | host name for the error|
+|componentErrors.errorPort| String | port for the error|
+|componentErrors.error| String |Shows the error happened in a component|
+|componentErrors.errorLapsedSecs| Integer | Number of seconds elapsed since the error happened in a component |
+|componentErrors.errorWorkerLogLink| String | Link to the worker log that reported the exception |
+|topologyId| String | Topology id|
+|tasks| Integer |Number of instances of component|
+|window    |String. Default value "All Time" | window duration for metrics in seconds|
+|spoutSummary or boltStats| Array |Array of component stats. **Please note this element tag can be spoutSummary or boltStats depending on the componentType**|
+|spoutSummary.windowPretty| String |Duration passed in HH:MM:SS format|
+|spoutSummary.window| String | window duration for metrics in seconds|
+|spoutSummary.emitted| Long |Number of messages emitted in given window |
+|spoutSummary.completeLatency| String (double value returned in String format) |Total latency for processing the message|
+|spoutSummary.transferred| Long |Total number of messages  transferred in given window|
+|spoutSummary.acked| Long |Number of messages acked|
+|spoutSummary.failed| Long |Number of messages failed|
+|boltStats.windowPretty| String |Duration passed in HH:MM:SS format|
+|boltStats..window| String | window duration for metrics in seconds|
+|boltStats.transferred| Long |Total number of messages  transferred in given window|
+|boltStats.processLatency| String (double value returned in String format)  |Average time of the bolt to ack a message after it was received|
+|boltStats.acked| Long |Number of messages acked|
+|boltStats.failed| Long |Number of messages failed|
+
+Examples:
+
+```no-highlight
+1. http://ui-daemon-host-name:8080/api/v1/topology/WordCount3-1-1402960825/component/spout
+2. http://ui-daemon-host-name:8080/api/v1/topology/WordCount3-1-1402960825/component/spout?sys=1
+3. http://ui-daemon-host-name:8080/api/v1/topology/WordCount3-1-1402960825/component/spout?window=600
+```
+
+Sample response:
+
+```json
+{
+    "name": "WordCount3",
+    "id": "spout",
+    "componentType": "spout",
+    "windowHint": "10m 0s",
+    "executors": 5,
+    "componentErrors":[{"errorTime": 1406006074000,
+                        "errorHost": "10.11.1.70",
+                        "errorPort": 6701,
+                        "errorWorkerLogLink": "http://10.11.1.7:8000/log?file=worker-6701.log",
+                        "errorLapsedSecs": 16,
+                        "error": "java.lang.RuntimeException: java.lang.StringIndexOutOfBoundsException: Some Error\n\tat backtype.storm.utils.DisruptorQueue.consumeBatchToCursor(DisruptorQueue.java:128)\n\tat backtype.storm.utils.DisruptorQueue.consumeBatchWhenAvailable(DisruptorQueue.java:99)\n\tat backtype.storm.disruptor$consume_batch_when_available.invoke(disruptor.clj:80)\n\tat backtype...more.."
+    }],
+    "topologyId": "WordCount3-1-1402960825",
+    "tasks": 5,
+    "window": "600",
+    "spoutSummary": [
+        {
+            "windowPretty": "10m 0s",
+            "window": "600",
+            "emitted": 28500,
+            "transferred": 28460,
+            "completeLatency": "0.000",
+            "acked": 0,
+            "failed": 0
+        },
+        {
+            "windowPretty": "3h 0m 0s",
+            "window": "10800",
+            "emitted": 127640,
+            "transferred": 127440,
+            "completeLatency": "0.000",
+            "acked": 0,
+            "failed": 0
+        },
+        {
+            "windowPretty": "1d 0h 0m 0s",
+            "window": "86400",
+            "emitted": 127640,
+            "transferred": 127440,
+            "completeLatency": "0.000",
+            "acked": 0,
+            "failed": 0
+        },
+        {
+            "windowPretty": "All time",
+            "window": ":all-time",
+            "emitted": 127640,
+            "transferred": 127440,
+            "completeLatency": "0.000",
+            "acked": 0,
+            "failed": 0
+        }
+    ],
+    "outputStats": [
+        {
+            "stream": "__metrics",
+            "emitted": 40,
+            "transferred": 0,
+            "completeLatency": "0",
+            "acked": 0,
+            "failed": 0
+        },
+        {
+            "stream": "default",
+            "emitted": 28460,
+            "transferred": 28460,
+            "completeLatency": "0",
+            "acked": 0,
+            "failed": 0
+        }
+    ],
+    "executorStats": [
+        {
+            "workerLogLink": "http://10.11.1.7:8000/log?file=worker-6701.log",
+            "emitted": 5720,
+            "port": 6701,
+            "completeLatency": "0.000",
+            "transferred": 5720,
+            "host": "10.11.1.7",
+            "acked": 0,
+            "uptime": "43m 4s",
+            "uptimeSeconds": 2584,
+            "id": "[24-24]",
+            "failed": 0
+        },
+        {
+            "workerLogLink": "http://10.11.1.7:8000/log?file=worker-6703.log",
+            "emitted": 5700,
+            "port": 6703,
+            "completeLatency": "0.000",
+            "transferred": 5700,
+            "host": "10.11.1.7",
+            "acked": 0,
+            "uptime": "42m 57s",
+            "uptimeSeconds": 2577,
+            "id": "[25-25]",
+            "failed": 0
+        },
+        {
+            "workerLogLink": "http://10.11.1.7:8000/log?file=worker-6702.log",
+            "emitted": 5700,
+            "port": 6702,
+            "completeLatency": "0.000",
+            "transferred": 5680,
+            "host": "10.11.1.7",
+            "acked": 0,
+            "uptime": "42m 57s",
+            "uptimeSeconds": 2577,
+            "id": "[26-26]",
+            "failed": 0
+        },
+        {
+            "workerLogLink": "http://10.11.1.7:8000/log?file=worker-6701.log",
+            "emitted": 5700,
+            "port": 6701,
+            "completeLatency": "0.000",
+            "transferred": 5680,
+            "host": "10.11.1.7",
+            "acked": 0,
+            "uptime": "43m 4s",
+            "uptimeSeconds": 2584,
+            "id": "[27-27]",
+            "failed": 0
+        },
+        {
+            "workerLogLink": "http://10.11.1.7:8000/log?file=worker-6703.log",
+            "emitted": 5680,
+            "port": 6703,
+            "completeLatency": "0.000",
+            "transferred": 5680,
+            "host": "10.11.1.7",
+            "acked": 0,
+            "uptime": "42m 57s",
+            "uptimeSeconds": 2577,
+            "id": "[28-28]",
+            "failed": 0
+        }
+    ]
+}
+```
+
+## Profiling and Debugging GET Operations
+
+###  /api/v1/topology/:id/profiling/start/:host-port/:timeout (GET)
+
+Request to start profiler on worker with timeout. Returns status and link to profiler artifacts for worker.
+
+|Parameter |Value   |Description  |
+|----------|--------|-------------|
+|id   	   |String (required)| Topology Id  |
+|host-port |String (required)| Worker Id |
+|timeout |String (required)| Time out for profiler to stop in minutes |
+
+Response fields:
+
+|Field  |Value |Description|
+|-----	|----- |-----------|
+|id   | String | Worker id|
+|status | String | Response Status |
+|timeout | String | Requested timeout
+|dumplink | String | Link to logviewer URL for worker profiler documents.|
+
+Examples:
+
+```no-highlight
+1. http://ui-daemon-host-name:8080/api/v1/topology/wordcount-1-1446614150/profiling/start/10.11.1.7:6701/10
+2. http://ui-daemon-host-name:8080/api/v1/topology/wordcount-1-1446614150/profiling/start/10.11.1.7:6701/5
+3. http://ui-daemon-host-name:8080/api/v1/topology/wordcount-1-1446614150/profiling/start/10.11.1.7:6701/20
+```
+
+Sample response:
+
+```json
+{
+   "status": "ok",
+   "id": "10.11.1.7:6701",
+   "timeout": "10",
+   "dumplink": "http:\/\/10.11.1.7:8000\/dumps\/wordcount-1-1446614150\/10.11.1.7%3A6701"
+}
+```
+
+###  /api/v1/topology/:id/profiling/dumpprofile/:host-port (GET)
+
+Request to dump profiler recording on worker. Returns status and worker id for the request.
+
+|Parameter |Value   |Description  |
+|----------|--------|-------------|
+|id   	   |String (required)| Topology Id  |
+|host-port |String (required)| Worker Id |
+
+Response fields:
+
+|Field  |Value |Description|
+|-----	|----- |-----------|
+|id   | String | Worker id|
+|status | String | Response Status |
+
+Examples:
+
+```no-highlight
+1. http://ui-daemon-host-name:8080/api/v1/topology/wordcount-1-1446614150/profiling/dumpprofile/10.11.1.7:6701
+```
+
+Sample response:
+
+```json
+{
+   "status": "ok",
+   "id": "10.11.1.7:6701",
+}
+```
+
+###  /api/v1/topology/:id/profiling/stop/:host-port (GET)
+
+Request to stop profiler on worker. Returns status and worker id for the request.
+
+|Parameter |Value   |Description  |
+|----------|--------|-------------|
+|id   	   |String (required)| Topology Id  |
+|host-port |String (required)| Worker Id |
+
+Response fields:
+
+|Field  |Value |Description|
+|-----	|----- |-----------|
+|id   | String | Worker id|
+|status | String | Response Status |
+
+Examples:
+
+```no-highlight
+1. http://ui-daemon-host-name:8080/api/v1/topology/wordcount-1-1446614150/profiling/stop/10.11.1.7:6701
+```
+
+Sample response:
+
+```json
+{
+   "status": "ok",
+   "id": "10.11.1.7:6701",
+}
+```
+
+###  /api/v1/topology/:id/profiling/dumpjstack/:host-port (GET)
+
+Request to dump jstack on worker. Returns status and worker id for the request.
+
+|Parameter |Value   |Description  |
+|----------|--------|-------------|
+|id   	   |String (required)| Topology Id  |
+|host-port |String (required)| Worker Id |
+
+Response fields:
+
+|Field  |Value |Description|
+|-----	|----- |-----------|
+|id   | String | Worker id|
+|status | String | Response Status |
+
+Examples:
+
+```no-highlight
+1. http://ui-daemon-host-name:8080/api/v1/topology/wordcount-1-1446614150/profiling/dumpjstack/10.11.1.7:6701
+```
+
+Sample response:
+
+```json
+{
+   "status": "ok",
+   "id": "10.11.1.7:6701",
+}
+```
+
+###  /api/v1/topology/:id/profiling/dumpheap/:host-port (GET)
+
+Request to dump heap (jmap) on worker. Returns status and worker id for the request.
+
+|Parameter |Value   |Description  |
+|----------|--------|-------------|
+|id   	   |String (required)| Topology Id  |
+|host-port |String (required)| Worker Id |
+
+Response fields:
+
+|Field  |Value |Description|
+|-----	|----- |-----------|
+|id   | String | Worker id|
+|status | String | Response Status |
+
+Examples:
+
+```no-highlight
+1. http://ui-daemon-host-name:8080/api/v1/topology/wordcount-1-1446614150/profiling/dumpheap/10.11.1.7:6701
+```
+
+Sample response:
+
+```json
+{
+   "status": "ok",
+   "id": "10.11.1.7:6701",
+}
+```
+
+###  /api/v1/topology/:id/profiling/restartworker/:host-port (GET)
+
+Request to request the worker. Returns status and worker id for the request.
+
+|Parameter |Value   |Description  |
+|----------|--------|-------------|
+|id   	   |String (required)| Topology Id  |
+|host-port |String (required)| Worker Id |
+
+Response fields:
+
+|Field  |Value |Description|
+|-----	|----- |-----------|
+|id   | String | Worker id|
+|status | String | Response Status |
+
+Examples:
+
+```no-highlight
+1. http://ui-daemon-host-name:8080/api/v1/topology/wordcount-1-1446614150/profiling/restartworker/10.11.1.7:6701
+```
+
+Sample response:
+
+```json
+{
+   "status": "ok",
+   "id": "10.11.1.7:6701",
+}
+```
+
+## POST Operations
+
+### /api/v1/topology/:id/activate (POST)
+
+Activates a topology.
+
+|Parameter |Value   |Description  |
+|----------|--------|-------------|
+|id   	   |String (required)| Topology Id  |
+
+Sample Response:
+
+```json
+{"topologyOperation":"activate","topologyId":"wordcount-1-1420308665","status":"success"}
+```
+
+
+### /api/v1/topology/:id/deactivate (POST)
+
+Deactivates a topology.
+
+|Parameter |Value   |Description  |
+|----------|--------|-------------|
+|id   	   |String (required)| Topology Id  |
+
+Sample Response:
+
+```json
+{"topologyOperation":"deactivate","topologyId":"wordcount-1-1420308665","status":"success"}
+```
+
+
+### /api/v1/topology/:id/rebalance/:wait-time (POST)
+
+Rebalances a topology.
+
+|Parameter |Value   |Description  |
+|----------|--------|-------------|
+|id   	   |String (required)| Topology Id  |
+|wait-time |String (required)| Wait time before rebalance happens |
+|rebalanceOptions| Json (optional) | topology rebalance options |
+
+
+Sample rebalanceOptions json:
+
+```json
+{"rebalanceOptions" : {"numWorkers" : 2, "executors" : {"spout" :4, "count" : 10}}, "callback" : "foo"}
+```
+
+Examples:
+
+```no-highlight
+curl  -i -b ~/cookiejar.txt -c ~/cookiejar.txt -X POST  
+-H "Content-Type: application/json" 
+-d  '{"rebalanceOptions": {"numWorkers": 2, "executors": { "spout" : "5", "split": 7, "count": 5 }}, "callback":"foo"}' 
+http://localhost:8080/api/v1/topology/wordcount-1-1420308665/rebalance/0
+```
+
+Sample Response:
+
+```json
+{"topologyOperation":"rebalance","topologyId":"wordcount-1-1420308665","status":"success"}
+```
+
+
+
+### /api/v1/topology/:id/kill/:wait-time (POST)
+
+Kills a topology.
+
+|Parameter |Value   |Description  |
+|----------|--------|-------------|
+|id   	   |String (required)| Topology Id  |
+|wait-time |String (required)| Wait time before rebalance happens |
+
+Caution: Small wait times (0-5 seconds) may increase the probability of triggering the bug reported in
+[STORM-112](https://issues.apache.org/jira/browse/STORM-112), which may result in broker Supervisor
+daemons.
+
+Sample Response:
+
+```json
+{"topologyOperation":"kill","topologyId":"wordcount-1-1420308665","status":"success"}
+```
+
+## API errors
+
+The API returns 500 HTTP status codes in case of any errors.
+
+Sample response:
+
+```json
+{
+  "error": "Internal Server Error",
+  "errorMessage": "java.lang.NullPointerException\n\tat clojure.core$name.invoke(core.clj:1505)\n\tat backtype.storm.ui.core$component_page.invoke(core.clj:752)\n\tat backtype.storm.ui.core$fn__7766.invoke(core.clj:782)\n\tat compojure.core$make_route$fn__5755.invoke(core.clj:93)\n\tat compojure.core$if_route$fn__5743.invoke(core.clj:39)\n\tat compojure.core$if_method$fn__5736.invoke(core.clj:24)\n\tat compojure.core$routing$fn__5761.invoke(core.clj:106)\n\tat clojure.core$some.invoke(core.clj:2443)\n\tat compojure.core$routing.doInvoke(core.clj:106)\n\tat clojure.lang.RestFn.applyTo(RestFn.java:139)\n\tat clojure.core$apply.invoke(core.clj:619)\n\tat compojure.core$routes$fn__5765.invoke(core.clj:111)\n\tat ring.middleware.reload$wrap_reload$fn__6880.invoke(reload.clj:14)\n\tat backtype.storm.ui.core$catch_errors$fn__7800.invoke(core.clj:836)\n\tat ring.middleware.keyword_params$wrap_keyword_params$fn__6319.invoke(keyword_params.clj:27)\n\tat ring.middleware.nested_params$wrap_nest
 ed_params$fn__6358.invoke(nested_params.clj:65)\n\tat ring.middleware.params$wrap_params$fn__6291.invoke(params.clj:55)\n\tat ring.middleware.multipart_params$wrap_multipart_params$fn__6386.invoke(multipart_params.clj:103)\n\tat ring.middleware.flash$wrap_flash$fn__6675.invoke(flash.clj:14)\n\tat ring.middleware.session$wrap_session$fn__6664.invoke(session.clj:43)\n\tat ring.middleware.cookies$wrap_cookies$fn__6595.invoke(cookies.clj:160)\n\tat ring.adapter.jetty$proxy_handler$fn__6112.invoke(jetty.clj:16)\n\tat ring.adapter.jetty.proxy$org.mortbay.jetty.handler.AbstractHandler$0.handle(Unknown Source)\n\tat org.mortbay.jetty.handler.HandlerWrapper.handle(HandlerWrapper.java:152)\n\tat org.mortbay.jetty.Server.handle(Server.java:326)\n\tat org.mortbay.jetty.HttpConnection.handleRequest(HttpConnection.java:542)\n\tat org.mortbay.jetty.HttpConnection$RequestHandler.headerComplete(HttpConnection.java:928)\n\tat org.mortbay.jetty.HttpParser.parseNext(HttpParser.java:549)\n\tat org.mortb
 ay.jetty.HttpParser.parseAvailable(HttpParser.java:212)\n\tat org.mortbay.jetty.HttpConnection.handle(HttpConnection.java:404)\n\tat org.mortbay.jetty.bio.SocketConnector$Connection.run(SocketConnector.java:228)\n\tat org.mortbay.thread.QueuedThreadPool$PoolThread.run(QueuedThreadPool.java:582)\n"
+}
+```


[4/4] storm git commit: Merge branch 'asf-site' of github.com:ptgoetz/storm into asf-site

Posted by pt...@apache.org.
Merge branch 'asf-site' of github.com:ptgoetz/storm into asf-site


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

Branch: refs/heads/asf-site
Commit: 3590e2ee2a272d91c0b9d21368aede755f02ea64
Parents: 573bdcf d63146b
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Fri Jan 15 11:21:28 2016 -0500
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Fri Jan 15 11:21:28 2016 -0500

----------------------------------------------------------------------
 documentation/BYLAWS.md                         |  98 ++
 documentation/Contributing-to-Storm.md          |  33 +
 documentation/Documentation.md                  |  56 ++
 documentation/FAQ.md                            |   6 +-
 documentation/Logs.md                           |  30 +
 documentation/Message-passing-implementation.md |  34 +-
 documentation/Pacemaker.md                      | 108 ++
 .../Resource_Aware_Scheduler_overview.md        | 227 +++++
 documentation/Setting-up-a-Storm-cluster.md     |  29 +-
 documentation/State-checkpointing.md            | 147 +++
 documentation/Tutorial.md                       | 320 ++++++
 ...nding-the-parallelism-of-a-Storm-topology.md |   4 +-
 documentation/Windowing.md                      | 235 +++++
 documentation/distcache-blobstore.md            | 735 ++++++++++++++
 documentation/dynamic-log-level-settings.md     |  41 +
 documentation/dynamic-worker-profiling.md       |  33 +
 .../images/dynamic_log_level_settings_1.png     | Bin 0 -> 93689 bytes
 .../images/dynamic_log_level_settings_2.png     | Bin 0 -> 78785 bytes
 .../images/dynamic_profiling_debugging_1.png    | Bin 0 -> 56876 bytes
 .../images/dynamic_profiling_debugging_2.png    | Bin 0 -> 99164 bytes
 .../images/dynamic_profiling_debugging_3.png    | Bin 0 -> 96974 bytes
 .../images/dynamic_profiling_debugging_4.png    | Bin 0 -> 121994 bytes
 documentation/images/hdfs_blobstore.png         | Bin 0 -> 82180 bytes
 documentation/images/local_blobstore.png        | Bin 0 -> 81212 bytes
 documentation/images/nimbus_ha_blobstore.png    | Bin 0 -> 113991 bytes
 documentation/images/search-a-topology.png      | Bin 0 -> 671031 bytes
 .../images/search-for-a-single-worker-log.png   | Bin 0 -> 736579 bytes
 .../storm-metrics-profiling-internal-actions.md |  70 ++
 documentation/ui-rest-api.md                    | 996 +++++++++++++++++++
 29 files changed, 3180 insertions(+), 22 deletions(-)
----------------------------------------------------------------------



[3/4] storm git commit: update documentation

Posted by pt...@apache.org.
update documentation


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

Branch: refs/heads/asf-site
Commit: d63146b7af0aa4db4af7c5f812ca5358f9395aee
Parents: b962d32
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Tue Jan 12 17:31:44 2016 -0500
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Tue Jan 12 17:31:44 2016 -0500

----------------------------------------------------------------------
 documentation/BYLAWS.md                         |  98 ++
 documentation/Contributing-to-Storm.md          |  33 +
 documentation/Documentation.md                  |  56 ++
 documentation/FAQ.md                            |   6 +-
 documentation/Logs.md                           |  30 +
 documentation/Message-passing-implementation.md |  34 +-
 documentation/Pacemaker.md                      | 108 ++
 .../Resource_Aware_Scheduler_overview.md        | 227 +++++
 documentation/Setting-up-a-Storm-cluster.md     |  29 +-
 documentation/State-checkpointing.md            | 147 +++
 documentation/Tutorial.md                       | 320 ++++++
 ...nding-the-parallelism-of-a-Storm-topology.md |   4 +-
 documentation/Windowing.md                      | 235 +++++
 documentation/distcache-blobstore.md            | 735 ++++++++++++++
 documentation/dynamic-log-level-settings.md     |  41 +
 documentation/dynamic-worker-profiling.md       |  33 +
 .../images/dynamic_log_level_settings_1.png     | Bin 0 -> 93689 bytes
 .../images/dynamic_log_level_settings_2.png     | Bin 0 -> 78785 bytes
 .../images/dynamic_profiling_debugging_1.png    | Bin 0 -> 56876 bytes
 .../images/dynamic_profiling_debugging_2.png    | Bin 0 -> 99164 bytes
 .../images/dynamic_profiling_debugging_3.png    | Bin 0 -> 96974 bytes
 .../images/dynamic_profiling_debugging_4.png    | Bin 0 -> 121994 bytes
 documentation/images/hdfs_blobstore.png         | Bin 0 -> 82180 bytes
 documentation/images/local_blobstore.png        | Bin 0 -> 81212 bytes
 documentation/images/nimbus_ha_blobstore.png    | Bin 0 -> 113991 bytes
 documentation/images/search-a-topology.png      | Bin 0 -> 671031 bytes
 .../images/search-for-a-single-worker-log.png   | Bin 0 -> 736579 bytes
 .../storm-metrics-profiling-internal-actions.md |  70 ++
 documentation/ui-rest-api.md                    | 996 +++++++++++++++++++
 29 files changed, 3180 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/d63146b7/documentation/BYLAWS.md
----------------------------------------------------------------------
diff --git a/documentation/BYLAWS.md b/documentation/BYLAWS.md
new file mode 100644
index 0000000..10e857e
--- /dev/null
+++ b/documentation/BYLAWS.md
@@ -0,0 +1,98 @@
+---
+title: Apache Storm Project Bylaws
+layout: documentation
+documentation: true
+---
+
+## Roles and Responsibilities
+
+Apache projects define a set of roles with associated rights and responsibilities. These roles govern what tasks an individual may perform within the project. The roles are defined in the following sections:
+
+### Users:
+
+The most important participants in the project are people who use our software. The majority of our developers start out as users and guide their development efforts from the user's perspective.
+
+Users contribute to the Apache projects by providing feedback to developers in the form of bug reports and feature suggestions. As well, users participate in the Apache community by helping other users on mailing lists and user support forums.
+
+### Contributors:
+
+Contributors are all of the volunteers who are contributing time, code, documentation, or resources to the Storm Project. A contributor that makes sustained, welcome contributions to the project may be invited to become a Committer, though the exact timing of such invitations depends on many factors.
+
+### Committers:
+
+The project's Committers are responsible for the project's technical management. Committers have access to all project source repositories. Committers may cast binding votes on any technical discussion regarding storm.
+
+Committer access is by invitation only and must be approved by lazy consensus of the active PMC members. A Committer is considered emeritus by their own declaration or by not contributing in any form to the project for over six months. An emeritus Committer may request reinstatement of commit access from the PMC. Such reinstatement is subject to lazy consensus approval of active PMC members.
+
+All Apache Committers are required to have a signed Contributor License Agreement (CLA) on file with the Apache Software Foundation. There is a [Committers' FAQ](https://www.apache.org/dev/committers.html) which provides more details on the requirements for Committers.
+
+A Committer who makes a sustained contribution to the project may be invited to become a member of the PMC. The form of contribution is not limited to code. It can also include code review, helping out users on the mailing lists, documentation, testing, etc.
+
+### Project Management Committee(PMC):
+
+The PMC is responsible to the board and the ASF for the management and oversight of the Apache Storm codebase. The responsibilities of the PMC include:
+
+ * Deciding what is distributed as products of the Apache Storm project. In particular all releases must be approved by the PMC.
+ * Maintaining the project's shared resources, including the codebase repository, mailing lists, websites.
+ * Speaking on behalf of the project.
+ * Resolving license disputes regarding products of the project.
+ * Nominating new PMC members and Committers.
+ * Maintaining these bylaws and other guidelines of the project.
+
+Membership of the PMC is by invitation only and must be approved by a consensus approval of active PMC members. A PMC member is considered "emeritus" by their own declaration or by not contributing in any form to the project for over six months. An emeritus member may request reinstatement to the PMC. Such reinstatement is subject to consensus approval of the active PMC members.
+
+The chair of the PMC is appointed by the ASF board. The chair is an office holder of the Apache Software Foundation (Vice President, Apache Storm) and has primary responsibility to the board for the management of the projects within the scope of the Storm PMC. The chair reports to the board quarterly on developments within the Storm project.
+
+The chair of the PMC is rotated annually. When the chair is rotated or if the current chair of the PMC resigns, the PMC votes to recommend a new chair using Single Transferable Vote (STV) voting. See http://wiki.apache.org/general/BoardVoting for specifics. The decision must be ratified by the Apache board.
+
+## Voting
+
+Decisions regarding the project are made by votes on the primary project development mailing list (dev@storm.apache.org). Where necessary, PMC voting may take place on the private Storm PMC mailing list. Votes are clearly indicated by subject line starting with [VOTE]. Votes may contain multiple items for approval and these should be clearly separated. Voting is carried out by replying to the vote mail. Voting may take four flavors:
+	
+| Vote | Meaning |
+|------|---------|
+| +1 | 'Yes,' 'Agree,' or 'the action should be performed.' |
+| +0 | Neutral about the proposed action. |
+| -0 | Mildly negative, but not enough so to want to block it. |
+| -1 |This is a negative vote. On issues where consensus is required, this vote counts as a veto. All vetoes must contain an explanation of why the veto is appropriate. Vetoes with no explanation are void. It may also be appropriate for a -1 vote to include an alternative course of action. |
+
+All participants in the Storm project are encouraged to show their agreement with or against a particular action by voting. For technical decisions, only the votes of active Committers are binding. Non-binding votes are still useful for those with binding votes to understand the perception of an action in the wider Storm community. For PMC decisions, only the votes of active PMC members are binding.
+
+Voting can also be applied to changes already made to the Storm codebase. These typically take the form of a veto (-1) in reply to the commit message sent when the commit is made. Note that this should be a rare occurrence. All efforts should be made to discuss issues when they are still patches before the code is committed.
+
+Only active (i.e. non-emeritus) Committers and PMC members have binding votes.
+
+## Approvals
+
+These are the types of approvals that can be sought. Different actions require different types of approvals
+
+| Approval Type | Criteria |
+|---------------|----------|
+| Consensus Approval | Consensus approval requires 3 binding +1 votes and no binding vetoes. |
+| Majority Approval | Majority approval requires at least 3 binding +1 votes and more +1 votes than -1 votes. |
+| Lazy Consensus | Lazy consensus requires no -1 votes ('silence gives assent'). |
+| 2/3 Majority | 2/3 majority votes requires at least 3 votes and twice as many +1 votes as -1 votes. |
+
+### Vetoes
+
+A valid, binding veto cannot be overruled. If a veto is cast, it must be accompanied by a valid reason explaining the reasons for the veto. The validity of a veto, if challenged, can be confirmed by anyone who has a binding vote. This does not necessarily signify agreement with the veto - merely that the veto is valid.
+
+If you disagree with a valid veto, you must lobby the person casting the veto to withdraw their veto. If a veto is not withdrawn, any action that has been vetoed must be reversed in a timely manner.
+
+## Actions
+
+This section describes the various actions which are undertaken within the project, the corresponding approval required for that action and those who have binding votes over the action.
+
+| Actions | Description | Approval | Binding Votes | Minimum Length | Mailing List |
+|---------|-------------|----------|---------------|----------------|--------------|
+| Code Change | A change made to a source code of the project and committed by a Committer. | A minimum of one +1 from a Committer other than the one who authored the patch, and no -1s. The code can be committed after the first +1. If a -1 is received to the patch within 7 days after the patch was posted, it may be reverted immediately if it was already merged. | Active Committers | 1 day from initial patch (**Note:** Committers should consider allowing more time for review based on the complexity and/or impact of the patch in question.)|JIRA or Github pull ( with notification sent to dev@storm.apache.org) |
+| Non-Code Change | A change made to a repository of the project and committed by a Committer. This includes documentation, website content, etc., but not source code, unless only comments are being modified. | Lazy Consensus | Active Committers | At the discression of the Committer |JIRA or Github pull (with notification sent to dev@storm.apache.org) |
+| Product Release | A vote is required to accept a proposed release as an official release of the project. Any Committer may call for a release vote at any point in time. | Majority Approval | Active PMC members | 3 days | dev@storm.apache.org |
+| Adoption of New Codebase | When the codebase for an existing, released product is to be replaced with an alternative codebase. If such a vote fails to gain approval, the existing code base will continue. This also covers the creation of new sub-projects and submodules within the project as well as merging of feature branches. | 2/3 Majority | Active PMC members | 6 days | dev@storm.apache.org |
+| New Committer | When a new Committer is proposed for the project. | Consensus Approval | Active PMC members | 3 days | private@storm.apache.org |
+| New PMC Member | When a member is proposed for the PMC. | Consensus Approval | Active PMC members | 3 days | private@storm.apache.org |
+| Emeritus PMC Member re-instatement | When an emeritus PMC member requests to be re-instated as an active PMC member. | Consensus Approval | Active PMC members | 6 days | private@storm.apache.org |
+| Emeritus Committer re-instatement | When an emeritus Committer requests to be re-instated as an active Committer. | Consensus Approval | Active PMC members | 6 days | private@storm.apache.org |
+| Committer Removal | When removal of commit privileges is sought. Note: Such actions will also be referred to the ASF board by the PMC chair. | 2/3 Majority | Active PMC members (excluding the Committer in question if a member of the PMC). | 6 Days | private@storm.apache.org |
+| PMC Member Removal | When removal of a PMC member is sought. Note: Such actions will also be referred to the ASF board by the PMC chair. | 2/3 Majority | Active PMC members (excluding the member in question). | 6 Days | private@storm.apache.org |
+| Modifying Bylaws | Modifying this document. | 2/3 Majority | Active PMC members | 6 Days | dev@storm.apache.org |

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

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

http://git-wip-us.apache.org/repos/asf/storm/blob/d63146b7/documentation/FAQ.md
----------------------------------------------------------------------
diff --git a/documentation/FAQ.md b/documentation/FAQ.md
index a69862e..a65da1e 100644
--- a/documentation/FAQ.md
+++ b/documentation/FAQ.md
@@ -65,6 +65,10 @@ You can join streams with join, merge or multiReduce.
 
 At time of writing, you can't emit to multiple output streams from Trident -- see [STORM-68](https://issues.apache.org/jira/browse/STORM-68)
 
+### Why am I getting a NotSerializableException/IllegalStateException when my topology is being started up?
+
+Within the Storm lifecycle, the topology is instantiated and then serialized to byte format to be stored in ZooKeeper, prior to the topology being executed. Within this step, if a spout or bolt within the topology has an initialized unserializable property, serialization will fail. If there is a need for a field that is unserializable, initialize it within the bolt or spout's prepare method, which is run after the topology is delivered to the worker.
+
 ## Spouts
 
 ### What is a coordinator, and why are there several?
@@ -112,7 +116,7 @@ You can't change the overall batch size once generated, but you can change the n
 
 ### How do I aggregate events by time?
 
-If have records with an immutable timestamp, and you would like to count, average or otherwise aggregate them into discrete time buckets, Trident is an excellent and scalable solution.
+If you have records with an immutable timestamp, and you would like to count, average or otherwise aggregate them into discrete time buckets, Trident is an excellent and scalable solution.
 
 Write an `Each` function that turns the timestamp into a time bucket: if the bucket size was "by hour", then the timestamp `2013-08-08 12:34:56` would be mapped to the `2013-08-08 12:00:00` time bucket, and so would everything else in the twelve o'clock hour. Then group on that timebucket and use a grouped persistentAggregate. The persistentAggregate uses a local cacheMap backed by a data store. Groups with many records require very few reads from the data store, and use efficient bulk reads and writes; as long as your data feed is relatively prompt Trident will make very efficient use of memory and network. Even if a server drops off line for a day, then delivers that full day's worth of data in a rush, the old results will be calmly retrieved and updated -- and without interfering with calculating the current results.
 

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

http://git-wip-us.apache.org/repos/asf/storm/blob/d63146b7/documentation/Pacemaker.md
----------------------------------------------------------------------
diff --git a/documentation/Pacemaker.md b/documentation/Pacemaker.md
new file mode 100644
index 0000000..24e3e7f
--- /dev/null
+++ b/documentation/Pacemaker.md
@@ -0,0 +1,108 @@
+# Pacemaker
+
+### 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="nimbus@MY.COMPANY.COM";
+};
+                         
+```
+
+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="pacemaker@MY.COMPANY.COM";
+};
+```
+
+ - 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.

http://git-wip-us.apache.org/repos/asf/storm/blob/d63146b7/documentation/Resource_Aware_Scheduler_overview.md
----------------------------------------------------------------------
diff --git a/documentation/Resource_Aware_Scheduler_overview.md b/documentation/Resource_Aware_Scheduler_overview.md
new file mode 100644
index 0000000..0f5b8cb
--- /dev/null
+++ b/documentation/Resource_Aware_Scheduler_overview.md
@@ -0,0 +1,227 @@
+# Introduction
+
+The purpose of this document is to provide a description of the Resource Aware Scheduler for the Storm distributed real-time computation system.  This document will provide you with both a high level description of the resource aware scheduler in Storm
+
+## Using Resource Aware Scheduler
+
+The user can switch to using the Resource Aware Scheduler by setting the following in *conf/storm.yaml*
+
+    storm.scheduler: “backtype.storm.scheduler.resource.ResourceAwareScheduler”
+
+
+## API Overview
+
+For a Storm Topology, the user can now specify the amount of resources a topology component (i.e. Spout or Bolt) is required to run a single instance of the component.  The user can specify the resource requirement for a topology component by using the following API calls.
+
+### Setting Memory Requirement
+
+API to set component memory requirement:
+
+    public T setMemoryLoad(Number onHeap, Number offHeap)
+
+Parameters:
+* Number onHeap – The amount of on heap memory an instance of this component will consume in megabytes
+* Number offHeap – The amount of off heap memory an instance of this component will consume in megabytes
+
+The user also has to option to just specify the on heap memory requirement if the component does not have an off heap memory need.
+
+    public T setMemoryLoad(Number onHeap)
+
+Parameters:
+* Number onHeap – The amount of on heap memory an instance of this component will consume
+
+If no value is provided for offHeap, 0.0 will be used. If no value is provided for onHeap, or if the API is never called for a component, the default value will be used.
+
+Example of Usage:
+
+    SpoutDeclarer s1 = builder.setSpout("word", new TestWordSpout(), 10);
+    s1.setMemoryLoad(1024.0, 512.0);
+    builder.setBolt("exclaim1", new ExclamationBolt(), 3)
+                .shuffleGrouping("word").setMemoryLoad(512.0);
+
+The entire memory requested for this topology is 16.5 GB. That is from 10 spouts with 1GB on heap memory and 0.5 GB off heap memory each and 3 bolts with 0.5 GB on heap memory each.
+
+### Setting CPU Requirement
+
+
+API to set component CPU requirement:
+
+    public T setCPULoad(Double amount)
+
+Parameters:
+* Number amount – The amount of on CPU an instance of this component will consume.
+
+Currently, the amount of CPU resources a component requires or is available on a node is represented by a point system. CPU usage is a difficult concept to define. Different CPU architectures perform differently depending on the task at hand. They are so complex that expressing all of that in a single precise portable number is impossible. Instead we take a convention over configuration approach and are primarily concerned with rough level of CPU usage while still providing the possibility to specify amounts more fine grained.
+
+By convention a CPU core typically will get 100 points. If you feel that your processors are more or less powerful you can adjust this accordingly. Heavy tasks that are CPU bound will get 100 points, as they can consume an entire core. Medium tasks should get 50, light tasks 25, and tiny tasks 10. In some cases you have a task that spawns other threads to help with processing. These tasks may need to go above 100 points to express the amount of CPU they are using. If these conventions are followed the common case for a single threaded task the reported Capacity * 100 should be the number of CPU points that the task needs.
+
+Example of Usage:
+
+    SpoutDeclarer s1 = builder.setSpout("word", new TestWordSpout(), 10);
+    s1.setCPULoad(15.0);
+    builder.setBolt("exclaim1", new ExclamationBolt(), 3)
+                .shuffleGrouping("word").setCPULoad(10.0);
+    builder.setBolt("exclaim2", new HeavyBolt(), 1)
+                    .shuffleGrouping("exclaim1").setCPULoad(450.0);
+
+###	Limiting the Heap Size per Worker (JVM) Process
+
+
+    public void setTopologyWorkerMaxHeapSize(Number size)
+
+Parameters:
+* Number size – The memory limit a worker process will be allocated in megabytes
+
+The user can limit the amount of memory resources the resource aware scheduler allocates to a single worker on a per topology basis by using the above API.  This API is in place so that the users can spread executors to multiple workers.  However, spreading executors to multiple workers may increase the communication latency since executors will not be able to use Disruptor Queue for intra-process communication.
+
+Example of Usage:
+
+    Config conf = new Config();
+    conf.setTopologyWorkerMaxHeapSize(512.0);
+
+### Setting Available Resources on Node
+
+A storm administrator can specify node resource availability by modifying the *conf/storm.yaml* file located in the storm home directory of that node.
+
+A storm administrator can specify how much available memory a node has in megabytes adding the following to *storm.yaml*
+
+    supervisor.memory.capacity.mb: [amount<Double>]
+
+A storm administrator can also specify how much available CPU resources a node has available adding the following to *storm.yaml*
+
+    supervisor.cpu.capacity: [amount<Double>]
+
+
+Note: that the amount the user can specify for the available CPU is represented using a point system like discussed earlier.
+
+Example of Usage:
+
+    supervisor.memory.capacity.mb: 20480.0
+    supervisor.cpu.capacity: 100.0
+
+
+### Other Configurations
+
+The user can set some default configurations for the Resource Aware Scheduler in *conf/storm.yaml*:
+
+    //default value if on heap memory requirement is not specified for a component 
+    topology.component.resources.onheap.memory.mb: 128.0
+
+    //default value if off heap memory requirement is not specified for a component 
+    topology.component.resources.offheap.memory.mb: 0.0
+
+    //default value if CPU requirement is not specified for a component 
+    topology.component.cpu.pcore.percent: 10.0
+
+    //default value for the max heap size for a worker  
+    topology.worker.max.heap.size.mb: 768.0
+
+# Topology Priorities and Per User Resource 
+
+The Resource Aware Scheduler or RAS also has multitenant capabilities since many Storm users typically share a Storm cluster.  Resource Aware Scheduler can allocate resources on a per user basis.  Each user can be guaranteed a certain amount of resources to run his or her topologies and the Resource Aware Scheduler will meet those guarantees when possible.  When the Storm cluster has extra free resources, Resource Aware Scheduler will to be able allocate additional resources to user in a fair manner. The importance of topologies can also vary.  Topologies can be used for actual production or just experimentation, thus Resource Aware Scheduler will take into account the importance of a topology when determining the order in which to schedule topologies or when to evict topologies
+
+## Setup
+
+The resource guarantees of a user can be specified *conf/user-resource-pools.yaml*.  Specify the resource guarantees of a user in the following format:
+
+    resource.aware.scheduler.user.pools:
+	[UserId]
+		cpu: [Amount of Guarantee CPU Resources]
+		memory: [Amount of Guarantee Memory Resources]
+
+An example of what *user-resource-pools.yaml* can look like:
+
+    resource.aware.scheduler.user.pools:
+        jerry:
+            cpu: 1000
+            memory: 8192.0
+        derek:
+            cpu: 10000.0
+            memory: 32768
+        bobby:
+            cpu: 5000.0
+            memory: 16384.0
+
+Please note that the specified amount of Guaranteed CPU and Memory can be either a integer or double
+
+## API Overview
+### Specifying Topology Priority
+The range of topology priorities can range form 0-29.  The topologies priorities will be partitioned into several priority levels that may contain a range of priorities. 
+For example we can create a priority level mapping:
+
+    PRODUCTION => 0 – 9
+    STAGING => 10 – 19
+    DEV => 20 – 29
+
+Thus, each priority level contains 10 sub priorities. Users can set the priority level of a topology by using the following API
+
+    conf.setTopologyPriority(int priority)
+    
+Parameters:
+* priority – an integer representing the priority of the topology
+
+Please note that the 0-29 range is not a hard limit.  Thus, a user can set a priority number that is higher than 29. However, the property of higher the priority number, lower the importance still holds
+
+### Specifying Scheduling Strategy:
+
+A user can specify on a per topology basis what scheduling strategy to use.  Users can implement the IStrategy interface and define new strategies to schedule specific topologies.  This pluggable interface was created since we realize different topologies might have different scheduling needs.  A user can set the topology strategy within the topology definition by using the API:
+
+    public void setTopologyStrategy(Class<? extends IStrategy> clazz)
+    
+Parameters:
+* clazz – The strategy class that implements the IStrategy interface
+
+Example Usage:
+
+    conf.setTopologyStrategy(backtype.storm.scheduler.resource.strategies.scheduling.DefaultResourceAwareStrategy.class);
+
+A default scheduling is provided.  The DefaultResourceAwareStrategy is implemented based off the scheduling algorithm in the original paper describing resource aware scheduling in Storm:
+
+http://web.engr.illinois.edu/~bpeng/files/r-storm.pdf
+
+### Specifying Topology Prioritization Strategy
+
+The order of scheduling is a pluggable interface in which a user could define a strategy that prioritizes topologies.  For a user to define his or her own prioritization strategy, he or she needs to implement the ISchedulingPriorityStrategy interface.  A user can set the scheduling priority strategy by setting the *Config.RESOURCE_AWARE_SCHEDULER_PRIORITY_STRATEGY* to point to the class that implements the strategy. For instance:
+
+    resource.aware.scheduler.priority.strategy: "backtype.storm.scheduler.resource.strategies.priority.DefaultSchedulingPriorityStrategy"
+    
+A default strategy will be provided.  The following explains how the default scheduling priority strategy works.
+
+**DefaultSchedulingPriorityStrategy**
+
+The order of scheduling should be based on the distance between a user’s current resource allocation and his or her guaranteed allocation.  We should prioritize the users who are the furthest away from their resource guarantee. The difficulty of this problem is that a user may have multiple resource guarantees, and another user can have another set of resource guarantees, so how can we compare them in a fair manner?  Let's use the average percentage of resource guarantees satisfied as a method of comparison.
+
+For example:
+
+|User|Resource Guarantee|Resource Allocated|
+|----|------------------|------------------|
+|A|<10 CPU, 50GB>|<2 CPU, 40 GB>|
+|B|< 20 CPU, 25GB>|<15 CPU, 10 GB>|
+
+User A’s average percentage satisfied of resource guarantee: 
+
+(2/10+40/50)/2  = 0.5
+
+User B’s average percentage satisfied of resource guarantee: 
+
+(15/20+10/25)/2  = 0.575
+
+Thus, in this example User A has a smaller average percentage of his or her resource guarantee satisfied than User B.  Thus, User A should get priority to be allocated more resource, i.e., schedule a topology submitted by User A.
+
+When scheduling, RAS sorts users by the average percentage satisfied of resource guarantee and schedule topologies from users based on that ordering starting from the users with the lowest average percentage satisfied of resource guarantee.  When a user’s resource guarantee is completely satisfied, the user’s average percentage satisfied of resource guarantee will be greater than or equal to 1.
+
+### Specifying Eviction Strategy
+The eviction strategy is used when there are not enough free resources in the cluster to schedule new topologies. If the cluster is full, we need a mechanism to evict topologies so that user resource guarantees can be met and additional resource can be shared fairly among users. The strategy for evicting topologies is also a pluggable interface in which the user can implement his or her own topology eviction strategy.  For a user to implement his or her own eviction strategy, he or she needs to implement the IEvictionStrategy Interface and set *Config.RESOURCE_AWARE_SCHEDULER_EVICTION_STRATEGY* to point to the implemented strategy class. For instance:
+
+    resource.aware.scheduler.eviction.strategy: "backtype.storm.scheduler.resource.strategies.eviction.DefaultEvictionStrategy"
+
+A default eviction strategy is provided.  The following explains how the default topology eviction strategy works
+
+**DefaultEvictionStrategy**
+
+
+To determine if topology eviction should occur we should take into account the priority of the topology that we are trying to schedule and whether the resource guarantees for the owner of the topology have been met.  
+
+We should never evict a topology from a user that does not have his or her resource guarantees satisfied.  The following flow chart should describe the logic for the eviction process.
+
+![Viewing metrics with VisualVM](images/resource_aware_scheduler_default_eviction_strategy.svg)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/d63146b7/documentation/Setting-up-a-Storm-cluster.md
----------------------------------------------------------------------
diff --git a/documentation/Setting-up-a-Storm-cluster.md b/documentation/Setting-up-a-Storm-cluster.md
index 07b4eda..ee4ad15 100644
--- a/documentation/Setting-up-a-Storm-cluster.md
+++ b/documentation/Setting-up-a-Storm-cluster.md
@@ -52,12 +52,18 @@ storm.zookeeper.servers:
 
 If the port that your Zookeeper cluster uses is different than the default, you should set **storm.zookeeper.port** as well.
 
-2) **storm.local.dir**: The Nimbus and Supervisor daemons require a directory on the local disk to store small amounts of state (like jars, confs, and things like that). You should create that directory on each machine, give it proper permissions, and then fill in the directory location using this config. For example:
+2) **storm.local.dir**: The Nimbus and Supervisor daemons require a directory on the local disk to store small amounts of state (like jars, confs, and things like that).
+ You should create that directory on each machine, give it proper permissions, and then fill in the directory location using this config. For example:
 
 ```yaml
 storm.local.dir: "/mnt/storm"
 ```
-
+If you run storm on windows,it could be:
+```yaml
+storm.local.dir: "C:\\storm-local"
+```
+If you use a relative path,it will be relative to where you installed storm(STORM_HOME).
+You can leave it empty with default value `$STORM_HOME/storm-local`
 3) **nimbus.host**: The worker nodes need to know which machine is the master in order to download topology jars and confs. For example:
 
 ```yaml
@@ -74,6 +80,25 @@ supervisor.slots.ports:
     - 6703
 ```
 
+### Monitoring Health of Supervisors
+
+Storm provides a mechanism by which administrators can configure the supervisor to run administrator supplied scripts periodically to determine if a node is healthy or not. Administrators can have the supervisor determine if the node is in a healthy state by performing any checks of their choice in scripts located in storm.health.check.dir. If a script detects the node to be in an unhealthy state, it must print a line to standard output beginning with the string ERROR. The supervisor will periodically run the scripts in the health check dir and check the output. If the script’s output contains the string ERROR, as described above, the supervisor will shut down any workers and exit. 
+
+If the supervisor is running with supervision "/bin/storm node-health-check" can be called to determine if the supervisor should be launched or if the node is unhealthy.
+
+The health check directory location can be configured with:
+
+```yaml
+storm.health.check.dir: "healthchecks"
+
+```
+The scripts must have execute permissions.
+The time to allow any given healthcheck script to run before it is marked failed due to timeout can be configured with:
+
+```yaml
+storm.health.check.timeout.ms: 5000
+```
+
 ### Configure external libraries and environmental variables (optional)
 
 If you need support from external libraries or custom plugins, you can place such jars into the extlib/ and extlib-daemon/ directories. Note that the extlib-daemon/ directory stores jars used only by daemons (Nimbus, Supervisor, DRPC, UI, Logviewer), e.g., HDFS and customized scheduling libraries. Accordingly, two environmental variables STORM_EXT_CLASSPATH and STORM_EXT_CLASSPATH_DAEMON can be configured by users for including the external classpath and daemon-only external classpath.

http://git-wip-us.apache.org/repos/asf/storm/blob/d63146b7/documentation/State-checkpointing.md
----------------------------------------------------------------------
diff --git a/documentation/State-checkpointing.md b/documentation/State-checkpointing.md
new file mode 100644
index 0000000..889b387
--- /dev/null
+++ b/documentation/State-checkpointing.md
@@ -0,0 +1,147 @@
+# State support in core storm
+Storm core has abstractions for bolts to save and retrieve the state of its operations. There is a default in-memory
+based state implementation and also a Redis backed implementation that provides state persistence.
+
+## State management
+Bolts that requires its state to be managed and persisted by the framework should implement the `IStatefulBolt` interface or
+extend the `BaseStatefulBolt` and implement `void initState(T state)` method. The `initState` method is invoked by the framework
+during the bolt initialization with the previously saved state of the bolt. This is invoked after prepare but before the bolt starts
+processing any tuples.
+
+Currently the only kind of `State` implementation that is supported is `KeyValueState` which provides key-value mapping.
+
+For example a word count bolt could use the key value state abstraction for the word counts as follows.
+
+1. Extend the BaseStatefulBolt and type parameterize it with KeyValueState which would store the mapping of word to count.
+2. The bolt gets initialized with its previously saved state in the init method. This will contain the word counts
+last committed by the framework during the previous run.
+3. In the execute method, update the word count.
+
+ ```java
+ public class WordCountBolt extends BaseStatefulBolt<KeyValueState<String, Long>> {
+ private KeyValueState<String, Long> wordCounts;
+ ...
+     @Override
+     public void initState(KeyValueState<String, Long> state) {
+       wordCounts = state;
+     }
+     @Override
+     public void execute(Tuple tuple, BasicOutputCollector collector) {
+       String word = tuple.getString(0);
+       Integer count = wordCounts.get(word, 0);
+       count++;
+       wordCounts.put(word, count);
+       collector.emit(new Values(word, count));
+     }
+ ...
+ }
+ ```
+4. The framework periodically checkpoints the state of the bolt (default every second). The frequency
+can be changed by setting the storm config `topology.state.checkpoint.interval.ms`
+5. For state persistence, use a state provider that supports persistence by setting the `topology.state.provider` in the
+storm config. E.g. for using Redis based key-value state implementation set `topology.state.provider: org.apache.storm.redis.state.RedisKeyValueStateProvider`
+in storm.yaml. The provider implementation jar should be in the class path, which in this case means putting the `storm-redis-*.jar`
+in the extlib directory.
+6. The state provider properties can be overridden by setting `topology.state.provider.config`. For Redis state this is a
+json config with the following properties.
+
+ ```
+ {
+   "keyClass": "Optional fully qualified class name of the Key type.",
+   "valueClass": "Optional fully qualified class name of the Value type.",
+   "keySerializerClass": "Optional Key serializer implementation class.",
+   "valueSerializerClass": "Optional Value Serializer implementation class.",
+   "jedisPoolConfig": {
+     "host": "localhost",
+     "port": 6379,
+     "timeout": 2000,
+     "database": 0,
+     "password": "xyz"
+     }
+ }
+ ```
+
+## Checkpoint mechanism
+Checkpoint is triggered by an internal checkpoint spout at the specified `topology.state.checkpoint.interval.ms`. If there is
+at-least one `IStatefulBolt` in the topology, the checkpoint spout is automatically added by the topology builder . For stateful topologies,
+the topology builder wraps the `IStatefulBolt` in a `StatefulBoltExecutor` which handles the state commits on receiving the checkpoint tuples.
+The non stateful bolts are wrapped in a `CheckpointTupleForwarder` which just forwards the checkpoint tuples so that the checkpoint tuples
+can flow through the topology DAG. The checkpoint tuples flow through a separate internal stream namely `$checkpoint`. The topology builder
+wires the checkpoint stream across the whole topology with the checkpoint spout at the root.
+
+```
+              default                         default               default
+[spout1]   ---------------> [statefulbolt1] ----------> [bolt1] --------------> [statefulbolt2]
+                          |                 ---------->         -------------->
+                          |                   ($chpt)               ($chpt)
+                          |
+[$checkpointspout] _______| ($chpt)
+```
+
+At checkpoint intervals the checkpoint tuples are emitted by the checkpoint spout. On receiving a checkpoint tuple, the state of the bolt
+is saved and then the checkpoint tuple is forwarded to the next component. Each bolt waits for the checkpoint to arrive on all its input
+streams before it saves its state so that the state represents a consistent state across the topology. Once the checkpoint spout receives
+ACK from all the bolts, the state commit is complete and the transaction is recorded as committed by the checkpoint spout.
+
+The state commit works like a three phase commit protocol with a prepare and commit phase so that the state across the topology is saved
+in a consistent and atomic manner.
+
+### Recovery
+The recovery phase is triggered when the topology is started for the first time. If the previous transaction was not successfully
+prepared, a `rollback` message is sent across the topology so that if a bolt has some prepared transactions it can be discarded.
+If the previous transaction was prepared successfully but not committed, a `commit` message is sent across the topology so that
+the prepared transactions can be committed. After these steps are complete, the bolts are initialized with the state.
+
+The recovery is also triggered if one of the bolts fails to acknowledge the checkpoint message or say a worker crashed in
+the middle. Thus when the worker is restarted by the supervisor, the checkpoint mechanism makes sure that the bolt gets
+initialized with its previous state and the checkpointing continues from the point where it left off.
+
+### Guarantee
+Storm relies on the acking mechanism to replay tuples in case of failures. It is possible that the state is committed
+but the worker crashes before acking the tuples. In this case the tuples are replayed causing duplicate state updates.
+Also currently the StatefulBoltExecutor continues to process the tuples from a stream after it has received a checkpoint
+tuple on one stream while waiting for checkpoint to arrive on other input streams for saving the state. This can also cause
+duplicate state updates during recovery.
+
+The state abstraction does not eliminate duplicate evaluations and currently provides only at-least once guarantee.
+
+### IStateful bolt hooks
+IStateful bolt interface provides hook methods where in the stateful bolts could implement some custom actions.
+```java
+    /**
+     * This is a hook for the component to perform some actions just before the
+     * framework commits its state.
+     */
+    void preCommit(long txid);
+
+    /**
+     * This is a hook for the component to perform some actions just before the
+     * framework prepares its state.
+     */
+    void prePrepare(long txid);
+
+    /**
+     * This is a hook for the component to perform some actions just before the
+     * framework rolls back the prepared state.
+     */
+    void preRollback();
+```
+This is optional and stateful bolts are not expected to provide any implementation. This is provided so that other
+system level components can be built on top of the stateful abstractions where we might want to take some actions before the
+stateful bolt's state is prepared, committed or rolled back.
+
+## Providing custom state implementations
+Currently the only kind of `State` implementation supported is `KeyValueState` which provides key-value mapping.
+
+Custom state implementations should provide implementations for the methods defined in the `org.apache.storm.State` interface.
+These are the `void prepareCommit(long txid)`, `void commit(long txid)`, `rollback()` methods. `commit()` method is optional
+and is useful if the bolt manages the state on its own. This is currently used only by the internal system bolts,
+for e.g. the CheckpointSpout to save its state.
+
+`KeyValueState` implementation should also implement the methods defined in the `org.apache.storm.state.KeyValueState` interface.
+
+### State provider
+The framework instantiates the state via the corresponding `StateProvider` implementation. A custom state should also provide
+a `StateProvider` implementation which can load and return the state based on the namespace. Each state belongs to a unique namespace.
+The namespace is typically unique per task so that each task can have its own state. The StateProvider and the corresponding
+State implementation should be available in the class path of Storm (by placing them in the extlib directory).

http://git-wip-us.apache.org/repos/asf/storm/blob/d63146b7/documentation/Tutorial.md
----------------------------------------------------------------------
diff --git a/documentation/Tutorial.md b/documentation/Tutorial.md
new file mode 100644
index 0000000..0d44177
--- /dev/null
+++ b/documentation/Tutorial.md
@@ -0,0 +1,320 @@
+---
+title: Tutorial
+layout: documentation
+documentation: true
+---
+In this tutorial, you'll learn how to create Storm topologies and deploy them to a Storm cluster. Java will be the main language used, but a few examples will use Python to illustrate Storm's multi-language capabilities.
+
+## Preliminaries
+
+This tutorial uses examples from the [storm-starter](https://github.com/apache/storm/blob/master/examples/storm-starter) project. It's recommended that you clone the project and follow along with the examples. Read [Setting up a development environment](Setting-up-development-environment.html) and [Creating a new Storm project](Creating-a-new-Storm-project.html) to get your machine set up.
+
+## Components of a Storm cluster
+
+A Storm cluster is superficially similar to a Hadoop cluster. Whereas on Hadoop you run "MapReduce jobs", on Storm you run "topologies". "Jobs" and "topologies" themselves are very different -- one key difference is that a MapReduce job eventually finishes, whereas a topology processes messages forever (or until you kill it).
+
+There are two kinds of nodes on a Storm cluster: the master node and the worker nodes. The master node runs a daemon called "Nimbus" that is similar to Hadoop's "JobTracker". Nimbus is responsible for distributing code around the cluster, assigning tasks to machines, and monitoring for failures.
+
+Each worker node runs a daemon called the "Supervisor". The supervisor listens for work assigned to its machine and starts and stops worker processes as necessary based on what Nimbus has assigned to it. Each worker process executes a subset of a topology; a running topology consists of many worker processes spread across many machines.
+
+![Storm cluster](images/storm-cluster.png)
+
+All coordination between Nimbus and the Supervisors is done through a [Zookeeper](http://zookeeper.apache.org/) cluster. Additionally, the Nimbus daemon and Supervisor daemons are fail-fast and stateless; all state is kept in Zookeeper or on local disk. This means you can kill -9 Nimbus or the Supervisors and they'll start back up like nothing happened. This design leads to Storm clusters being incredibly stable.
+
+## Topologies
+
+To do realtime computation on Storm, you create what are called "topologies". A topology is a graph of computation. Each node in a topology contains processing logic, and links between nodes indicate how data should be passed around between nodes.
+
+Running a topology is straightforward. First, you package all your code and dependencies into a single jar. Then, you run a command like the following:
+
+```
+storm jar all-my-code.jar backtype.storm.MyTopology arg1 arg2
+```
+
+This runs the class `backtype.storm.MyTopology` with the arguments `arg1` and `arg2`. The main function of the class defines the topology and submits it to Nimbus. The `storm jar` part takes care of connecting to Nimbus and uploading the jar.
+
+Since topology definitions are just Thrift structs, and Nimbus is a Thrift service, you can create and submit topologies using any programming language. The above example is the easiest way to do it from a JVM-based language. See [Running topologies on a production cluster](Running-topologies-on-a-production-cluster.html) for more information on starting and stopping topologies.
+
+## Streams
+
+The core abstraction in Storm is the "stream". A stream is an unbounded sequence of tuples. Storm provides the primitives for transforming a stream into a new stream in a distributed and reliable way. For example, you may transform a stream of tweets into a stream of trending topics.
+
+The basic primitives Storm provides for doing stream transformations are "spouts" and "bolts". Spouts and bolts have interfaces that you implement to run your application-specific logic.
+
+A spout is a source of streams. For example, a spout may read tuples off of a [Kestrel](http://github.com/nathanmarz/storm-kestrel) queue and emit them as a stream. Or a spout may connect to the Twitter API and emit a stream of tweets.
+
+A bolt consumes any number of input streams, does some processing, and possibly emits new streams. Complex stream transformations, like computing a stream of trending topics from a stream of tweets, require multiple steps and thus multiple bolts. Bolts can do anything from run functions, filter tuples, do streaming aggregations, do streaming joins, talk to databases, and more.
+
+Networks of spouts and bolts are packaged into a "topology" which is the top-level abstraction that you submit to Storm clusters for execution. A topology is a graph of stream transformations where each node is a spout or bolt. Edges in the graph indicate which bolts are subscribing to which streams. When a spout or bolt emits a tuple to a stream, it sends the tuple to every bolt that subscribed to that stream.
+
+![A Storm topology](images/topology.png)
+
+Links between nodes in your topology indicate how tuples should be passed around. For example, if there is a link between Spout A and Bolt B, a link from Spout A to Bolt C, and a link from Bolt B to Bolt C, then everytime Spout A emits a tuple, it will send the tuple to both Bolt B and Bolt C. All of Bolt B's output tuples will go to Bolt C as well.
+
+Each node in a Storm topology executes in parallel. In your topology, you can specify how much parallelism you want for each node, and then Storm will spawn that number of threads across the cluster to do the execution.
+
+A topology runs forever, or until you kill it. Storm will automatically reassign any failed tasks. Additionally, Storm guarantees that there will be no data loss, even if machines go down and messages are dropped.
+
+## Data model
+
+Storm uses tuples as its data model. A tuple is a named list of values, and a field in a tuple can be an object of any type. Out of the box, Storm supports all the primitive types, strings, and byte arrays as tuple field values. To use an object of another type, you just need to implement [a serializer](Serialization.html) for the type.
+
+Every node in a topology must declare the output fields for the tuples it emits. For example, this bolt declares that it emits 2-tuples with the fields "double" and "triple":
+
+```java
+public class DoubleAndTripleBolt extends BaseRichBolt {
+    private OutputCollectorBase _collector;
+
+    @Override
+    public void prepare(Map conf, TopologyContext context, OutputCollectorBase collector) {
+        _collector = collector;
+    }
+
+    @Override
+    public void execute(Tuple input) {
+        int val = input.getInteger(0);        
+        _collector.emit(input, new Values(val*2, val*3));
+        _collector.ack(input);
+    }
+
+    @Override
+    public void declareOutputFields(OutputFieldsDeclarer declarer) {
+        declarer.declare(new Fields("double", "triple"));
+    }    
+}
+```
+
+The `declareOutputFields` function declares the output fields `["double", "triple"]` for the component. The rest of the bolt will be explained in the upcoming sections.
+
+## A simple topology
+
+Let's take a look at a simple topology to explore the concepts more and see how the code shapes up. Let's look at the `ExclamationTopology` definition from storm-starter:
+
+```java
+TopologyBuilder builder = new TopologyBuilder();        
+builder.setSpout("words", new TestWordSpout(), 10);        
+builder.setBolt("exclaim1", new ExclamationBolt(), 3)
+        .shuffleGrouping("words");
+builder.setBolt("exclaim2", new ExclamationBolt(), 2)
+        .shuffleGrouping("exclaim1");
+```
+
+This topology contains a spout and two bolts. The spout emits words, and each bolt appends the string "!!!" to its input. The nodes are arranged in a line: the spout emits to the first bolt which then emits to the second bolt. If the spout emits the tuples ["bob"] and ["john"], then the second bolt will emit the words ["bob!!!!!!"] and ["john!!!!!!"].
+
+This code defines the nodes using the `setSpout` and `setBolt` methods. These methods take as input a user-specified id, an object containing the processing logic, and the amount of parallelism you want for the node. In this example, the spout is given id "words" and the bolts are given ids "exclaim1" and "exclaim2". 
+
+The object containing the processing logic implements the [IRichSpout](/javadoc/apidocs/backtype/storm/topology/IRichSpout.html) interface for spouts and the [IRichBolt](/javadoc/apidocs/backtype/storm/topology/IRichBolt.html) interface for bolts.
+
+The last parameter, how much parallelism you want for the node, is optional. It indicates how many threads should execute that component across the cluster. If you omit it, Storm will only allocate one thread for that node.
+
+`setBolt` returns an [InputDeclarer](/javadoc/apidocs/backtype/storm/topology/InputDeclarer.html) object that is used to define the inputs to the Bolt. Here, component "exclaim1" declares that it wants to read all the tuples emitted by component "words" using a shuffle grouping, and component "exclaim2" declares that it wants to read all the tuples emitted by component "exclaim1" using a shuffle grouping. "shuffle grouping" means that tuples should be randomly distributed from the input tasks to the bolt's tasks. There are many ways to group data between components. These will be explained in a few sections.
+
+If you wanted component "exclaim2" to read all the tuples emitted by both component "words" and component "exclaim1", you would write component "exclaim2"'s definition like this:
+
+```java
+builder.setBolt("exclaim2", new ExclamationBolt(), 5)
+            .shuffleGrouping("words")
+            .shuffleGrouping("exclaim1");
+```
+
+As you can see, input declarations can be chained to specify multiple sources for the Bolt.
+
+Let's dig into the implementations of the spouts and bolts in this topology. Spouts are responsible for emitting new messages into the topology. `TestWordSpout` in this topology emits a random word from the list ["nathan", "mike", "jackson", "golda", "bertels"] as a 1-tuple every 100ms. The implementation of `nextTuple()` in TestWordSpout looks like this:
+
+```java
+public void nextTuple() {
+    Utils.sleep(100);
+    final String[] words = new String[] {"nathan", "mike", "jackson", "golda", "bertels"};
+    final Random rand = new Random();
+    final String word = words[rand.nextInt(words.length)];
+    _collector.emit(new Values(word));
+}
+```
+
+As you can see, the implementation is very straightforward.
+
+`ExclamationBolt` appends the string "!!!" to its input. Let's take a look at the full implementation for `ExclamationBolt`:
+
+```java
+public static class ExclamationBolt implements IRichBolt {
+    OutputCollector _collector;
+
+    @Override
+    public void prepare(Map conf, TopologyContext context, OutputCollector collector) {
+        _collector = collector;
+    }
+
+    @Override
+    public void execute(Tuple tuple) {
+        _collector.emit(tuple, new Values(tuple.getString(0) + "!!!"));
+        _collector.ack(tuple);
+    }
+
+    @Override
+    public void cleanup() {
+    }
+
+    @Override
+    public void declareOutputFields(OutputFieldsDeclarer declarer) {
+        declarer.declare(new Fields("word"));
+    }
+    
+    @Override
+    public Map<String, Object> getComponentConfiguration() {
+        return null;
+    }
+}
+```
+
+The `prepare` method provides the bolt with an `OutputCollector` that is used for emitting tuples from this bolt. Tuples can be emitted at anytime from the bolt -- in the `prepare`, `execute`, or `cleanup` methods, or even asynchronously in another thread. This `prepare` implementation simply saves the `OutputCollector` as an instance variable to be used later on in the `execute` method.
+
+The `execute` method receives a tuple from one of the bolt's inputs. The `ExclamationBolt` grabs the first field from the tuple and emits a new tuple with the string "!!!" appended to it. If you implement a bolt that subscribes to multiple input sources, you can find out which component the [Tuple](/javadoc/apidocs/backtype/storm/tuple/Tuple.html) came from by using the `Tuple#getSourceComponent` method.
+
+There's a few other things going on in the `execute` method, namely that the input tuple is passed as the first argument to `emit` and the input tuple is acked on the final line. These are part of Storm's reliability API for guaranteeing no data loss and will be explained later in this tutorial. 
+
+The `cleanup` method is called when a Bolt is being shutdown and should cleanup any resources that were opened. There's no guarantee that this method will be called on the cluster: for example, if the machine the task is running on blows up, there's no way to invoke the method. The `cleanup` method is intended for when you run topologies in [local mode](Local-mode.html) (where a Storm cluster is simulated in process), and you want to be able to run and kill many topologies without suffering any resource leaks.
+
+The `declareOutputFields` method declares that the `ExclamationBolt` emits 1-tuples with one field called "word".
+
+The `getComponentConfiguration` method allows you to configure various aspects of how this component runs. This is a more advanced topic that is explained further on [Configuration](Configuration.html).
+
+Methods like `cleanup` and `getComponentConfiguration` are often not needed in a bolt implementation. You can define bolts more succinctly by using a base class that provides default implementations where appropriate. `ExclamationBolt` can be written more succinctly by extending `BaseRichBolt`, like so:
+
+```java
+public static class ExclamationBolt extends BaseRichBolt {
+    OutputCollector _collector;
+
+    @Override
+    public void prepare(Map conf, TopologyContext context, OutputCollector collector) {
+        _collector = collector;
+    }
+
+    @Override
+    public void execute(Tuple tuple) {
+        _collector.emit(tuple, new Values(tuple.getString(0) + "!!!"));
+        _collector.ack(tuple);
+    }
+
+    @Override
+    public void declareOutputFields(OutputFieldsDeclarer declarer) {
+        declarer.declare(new Fields("word"));
+    }    
+}
+```
+
+## Running ExclamationTopology in local mode
+
+Let's see how to run the `ExclamationTopology` in local mode and see that it's working.
+
+Storm has two modes of operation: local mode and distributed mode. In local mode, Storm executes completely in process by simulating worker nodes with threads. Local mode is useful for testing and development of topologies. When you run the topologies in storm-starter, they'll run in local mode and you'll be able to see what messages each component is emitting. You can read more about running topologies in local mode on [Local mode](Local-mode.html).
+
+In distributed mode, Storm operates as a cluster of machines. When you submit a topology to the master, you also submit all the code necessary to run the topology. The master will take care of distributing your code and allocating workers to run your topology. If workers go down, the master will reassign them somewhere else. You can read more about running topologies on a cluster on [Running topologies on a production cluster](Running-topologies-on-a-production-cluster.html)]. 
+
+Here's the code that runs `ExclamationTopology` in local mode:
+
+```java
+Config conf = new Config();
+conf.setDebug(true);
+conf.setNumWorkers(2);
+
+LocalCluster cluster = new LocalCluster();
+cluster.submitTopology("test", conf, builder.createTopology());
+Utils.sleep(10000);
+cluster.killTopology("test");
+cluster.shutdown();
+```
+
+First, the code defines an in-process cluster by creating a `LocalCluster` object. Submitting topologies to this virtual cluster is identical to submitting topologies to distributed clusters. It submits a topology to the `LocalCluster` by calling `submitTopology`, which takes as arguments a name for the running topology, a configuration for the topology, and then the topology itself.
+
+The name is used to identify the topology so that you can kill it later on. A topology will run indefinitely until you kill it.
+
+The configuration is used to tune various aspects of the running topology. The two configurations specified here are very common:
+
+1. **TOPOLOGY_WORKERS** (set with `setNumWorkers`) specifies how many _processes_ you want allocated around the cluster to execute the topology. Each component in the topology will execute as many _threads_. The number of threads allocated to a given component is configured through the `setBolt` and `setSpout` methods. Those _threads_ exist within worker _processes_. Each worker _process_ contains within it some number of _threads_ for some number of components. For instance, you may have 300 threads specified across all your components and 50 worker processes specified in your config. Each worker process will execute 6 threads, each of which of could belong to a different component. You tune the performance of Storm topologies by tweaking the parallelism for each component and the number of worker processes those threads should run within.
+2. **TOPOLOGY_DEBUG** (set with `setDebug`), when set to true, tells Storm to log every message every emitted by a component. This is useful in local mode when testing topologies, but you probably want to keep this turned off when running topologies on the cluster.
+
+There's many other configurations you can set for the topology. The various configurations are detailed on [the Javadoc for Config](/javadoc/apidocs/backtype/storm/Config.html).
+
+To learn about how to set up your development environment so that you can run topologies in local mode (such as in Eclipse), see [Creating a new Storm project](Creating-a-new-Storm-project.html).
+
+## Stream groupings
+
+A stream grouping tells a topology how to send tuples between two components. Remember, spouts and bolts execute in parallel as many tasks across the cluster. If you look at how a topology is executing at the task level, it looks something like this:
+
+![Tasks in a topology](images/topology-tasks.png)
+
+When a task for Bolt A emits a tuple to Bolt B, which task should it send the tuple to?
+
+A "stream grouping" answers this question by telling Storm how to send tuples between sets of tasks. Before we dig into the different kinds of stream groupings, let's take a look at another topology from [storm-starter](http://github.com/apache/storm/blob/master/examples/storm-starter). This [WordCountTopology](https://github.com/apache/storm/blob/master/examples/storm-starter/src/jvm/storm/starter/WordCountTopology.java) reads sentences off of a spout and streams out of `WordCountBolt` the total number of times it has seen that word before:
+
+```java
+TopologyBuilder builder = new TopologyBuilder();
+        
+builder.setSpout("sentences", new RandomSentenceSpout(), 5);        
+builder.setBolt("split", new SplitSentence(), 8)
+        .shuffleGrouping("sentences");
+builder.setBolt("count", new WordCount(), 12)
+        .fieldsGrouping("split", new Fields("word"));
+```
+
+`SplitSentence` emits a tuple for each word in each sentence it receives, and `WordCount` keeps a map in memory from word to count. Each time `WordCount` receives a word, it updates its state and emits the new word count.
+
+There's a few different kinds of stream groupings.
+
+The simplest kind of grouping is called a "shuffle grouping" which sends the tuple to a random task. A shuffle grouping is used in the `WordCountTopology` to send tuples from `RandomSentenceSpout` to the `SplitSentence` bolt. It has the effect of evenly distributing the work of processing the tuples across all of `SplitSentence` bolt's tasks.
+
+A more interesting kind of grouping is the "fields grouping". A fields grouping is used between the `SplitSentence` bolt and the `WordCount` bolt. It is critical for the functioning of the `WordCount` bolt that the same word always go to the same task. Otherwise, more than one task will see the same word, and they'll each emit incorrect values for the count since each has incomplete information. A fields grouping lets you group a stream by a subset of its fields. This causes equal values for that subset of fields to go to the same task. Since `WordCount` subscribes to `SplitSentence`'s output stream using a fields grouping on the "word" field, the same word always goes to the same task and the bolt produces the correct output.
+
+Fields groupings are the basis of implementing streaming joins and streaming aggregations as well as a plethora of other use cases. Underneath the hood, fields groupings are implemented using mod hashing.
+
+There's a few other kinds of stream groupings. You can read more about them on [Concepts](Concepts.html). 
+
+## Defining Bolts in other languages
+
+Bolts can be defined in any language. Bolts written in another language are executed as subprocesses, and Storm communicates with those subprocesses with JSON messages over stdin/stdout. The communication protocol just requires an ~100 line adapter library, and Storm ships with adapter libraries for Ruby, Python, and Fancy. 
+
+Here's the definition of the `SplitSentence` bolt from `WordCountTopology`:
+
+```java
+public static class SplitSentence extends ShellBolt implements IRichBolt {
+    public SplitSentence() {
+        super("python", "splitsentence.py");
+    }
+
+    public void declareOutputFields(OutputFieldsDeclarer declarer) {
+        declarer.declare(new Fields("word"));
+    }
+}
+```
+
+`SplitSentence` overrides `ShellBolt` and declares it as running using `python` with the arguments `splitsentence.py`. Here's the implementation of `splitsentence.py`:
+
+```python
+import storm
+
+class SplitSentenceBolt(storm.BasicBolt):
+    def process(self, tup):
+        words = tup.values[0].split(" ")
+        for word in words:
+          storm.emit([word])
+
+SplitSentenceBolt().run()
+```
+
+For more information on writing spouts and bolts in other languages, and to learn about how to create topologies in other languages (and avoid the JVM completely), see [Using non-JVM languages with Storm](Using-non-JVM-languages-with-Storm.html).
+
+## Guaranteeing message processing
+
+Earlier on in this tutorial, we skipped over a few aspects of how tuples are emitted. Those aspects were part of Storm's reliability API: how Storm guarantees that every message coming off a spout will be fully processed. See [Guaranteeing message processing](Guaranteeing-message-processing.html) for information on how this works and what you have to do as a user to take advantage of Storm's reliability capabilities.
+
+## Transactional topologies
+
+Storm guarantees that every message will be played through the topology at least once. A common question asked is "how do you do things like counting on top of Storm? Won't you overcount?" Storm has a feature called transactional topologies that let you achieve exactly-once messaging semantics for most computations. Read more about transactional topologies [here](Transactional-topologies.html). 
+
+## Distributed RPC
+
+This tutorial showed how to do basic stream processing on top of Storm. There's lots more things you can do with Storm's primitives. One of the most interesting applications of Storm is Distributed RPC, where you parallelize the computation of intense functions on the fly. Read more about Distributed RPC [here](Distributed-RPC.html). 
+
+## Conclusion
+
+This tutorial gave a broad overview of developing, testing, and deploying Storm topologies. The rest of the documentation dives deeper into all the aspects of using Storm.

http://git-wip-us.apache.org/repos/asf/storm/blob/d63146b7/documentation/Understanding-the-parallelism-of-a-Storm-topology.md
----------------------------------------------------------------------
diff --git a/documentation/Understanding-the-parallelism-of-a-Storm-topology.md b/documentation/Understanding-the-parallelism-of-a-Storm-topology.md
index 455b229..9b1e006 100644
--- a/documentation/Understanding-the-parallelism-of-a-Storm-topology.md
+++ b/documentation/Understanding-the-parallelism-of-a-Storm-topology.md
@@ -116,7 +116,7 @@ $ storm rebalance mytopology -n 5 -e blue-spout=3 -e yellow-bolt=10
 
 * [Concepts](Concepts.html)
 * [Configuration](Configuration.html)
-* [Running topologies on a production cluster](Running-topologies-on-a-production-cluster.html)
+* [Running topologies on a production cluster](Running-topologies-on-a-production-cluster.html)]
 * [Local mode](Local-mode.html)
-* [Tutorial](/tutorial.html)
+* [Tutorial](Tutorial.html)
 * [Storm API documentation](/javadoc/apidocs/), most notably the class ``Config``


[2/4] storm git commit: update documentation

Posted by pt...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/d63146b7/documentation/Windowing.md
----------------------------------------------------------------------
diff --git a/documentation/Windowing.md b/documentation/Windowing.md
new file mode 100644
index 0000000..803e5ca
--- /dev/null
+++ b/documentation/Windowing.md
@@ -0,0 +1,235 @@
+# Windowing support in core storm
+
+Storm core has support for processing a group of tuples that falls within a window. Windows are specified with the 
+following two parameters,
+
+1. Window length - the length or duration of the window
+2. Sliding interval - the interval at which the windowing slides
+
+## Sliding Window
+
+Tuples are grouped in windows and window slides every sliding interval. A tuple can belong to more than one window.
+
+For example a time duration based sliding window with length 10 secs and sliding interval of 5 seconds.
+
+```
+| e1 e2 | e3 e4 e5 e6 | e7 e8 e9 |...
+0       5             10         15    -> time
+
+|<------- w1 -------->|
+        |------------ w2 ------->|
+```
+
+The window is evaluated every 5 seconds and some of the tuples in the first window overlaps with the second one.
+		
+
+## Tumbling Window
+
+Tuples are grouped in a single window based on time or count. Any tuple belongs to only one of the windows.
+
+For example a time duration based tumbling window with length 5 secs.
+
+```
+| e1 e2 | e3 e4 e5 e6 | e7 e8 e9 |...
+0       5             10         15    -> time
+   w1         w2            w3
+```
+
+The window is evaluated every five seconds and none of the windows overlap.
+
+Storm supports specifying the window length and sliding intervals as a count of the number of tuples or as a time duration.
+
+The bolt interface `IWindowedBolt` is implemented by bolts that needs windowing support.
+
+```java
+public interface IWindowedBolt extends IComponent {
+    void prepare(Map stormConf, TopologyContext context, OutputCollector collector);
+    /**
+     * Process tuples falling within the window and optionally emit 
+     * new tuples based on the tuples in the input window.
+     */
+    void execute(TupleWindow inputWindow);
+    void cleanup();
+}
+```
+
+Every time the window activates, the `execute` method is invoked. The TupleWindow parameter gives access to the current tuples
+in the window, the tuples that expired and the new tuples that are added since last window was computed which will be useful 
+for efficient windowing computations.
+
+Bolts that needs windowing support typically would extend `BaseWindowedBolt` which has the apis for specifying the
+window length and sliding intervals.
+
+E.g. 
+
+```java
+public class SlidingWindowBolt extends BaseWindowedBolt {
+	private OutputCollector collector;
+	
+    @Override
+    public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
+    	this.collector = collector;
+    }
+	
+    @Override
+    public void execute(TupleWindow inputWindow) {
+	  for(Tuple tuple: inputWindow.get()) {
+	    // do the windowing computation
+		...
+	  }
+	  // emit the results
+	  collector.emit(new Values(computedValue));
+    }
+}
+
+public static void main(String[] args) {
+    TopologyBuilder builder = new TopologyBuilder();
+     builder.setSpout("spout", new RandomSentenceSpout(), 1);
+     builder.setBolt("slidingwindowbolt", 
+                     new SlidingWindowBolt().withWindow(new Count(30), new Count(10)),
+                     1).shuffleGrouping("spout");
+    Config conf = new Config();
+    conf.setDebug(true);
+    conf.setNumWorkers(1);
+
+    StormSubmitter.submitTopologyWithProgressBar(args[0], conf, builder.createTopology());
+	
+}
+```
+
+The following window configurations are supported.
+
+```java
+withWindow(Count windowLength, Count slidingInterval)
+Tuple count based sliding window that slides after `slidingInterval` number of tuples.
+
+withWindow(Count windowLength)
+Tuple count based window that slides with every incoming tuple.
+
+withWindow(Count windowLength, Duration slidingInterval)
+Tuple count based sliding window that slides after `slidingInterval` time duration.
+
+withWindow(Duration windowLength, Duration slidingInterval)
+Time duration based sliding window that slides after `slidingInterval` time duration.
+
+withWindow(Duration windowLength)
+Time duration based window that slides with every incoming tuple.
+
+withWindow(Duration windowLength, Count slidingInterval)
+Time duration based sliding window configuration that slides after `slidingInterval` number of tuples.
+
+withTumblingWindow(BaseWindowedBolt.Count count)
+Count based tumbling window that tumbles after the specified count of tuples.
+
+withTumblingWindow(BaseWindowedBolt.Duration duration)
+Time duration based tumbling window that tumbles after the specified time duration.
+
+```
+
+## Tuple timestamp and out of order tuples
+By default the timestamp tracked in the window is the time when the tuple is processed by the bolt. The window calculations
+are performed based on the processing timestamp. Storm has support for tracking windows based on the source generated timestamp.
+
+```java
+/**
+* Specify a field in the tuple that represents the timestamp as a long value. If this
+* field is not present in the incoming tuple, an {@link IllegalArgumentException} will be thrown.
+*
+* @param fieldName the name of the field that contains the timestamp
+*/
+public BaseWindowedBolt withTimestampField(String fieldName)
+```
+
+The value for the above `fieldName` will be looked up from the incoming tuple and considered for windowing calculations. 
+If the field is not present in the tuple an exception will be thrown. Along with the timestamp field name, a time lag parameter 
+can also be specified which indicates the max time limit for tuples with out of order timestamps. 
+
+E.g. If the lag is 5 secs and a tuple `t1` arrived with timestamp `06:00:05` no tuples may arrive with tuple timestamp earlier than `06:00:00`. If a tuple
+arrives with timestamp 05:59:59 after `t1` and the window has moved past `t1`, it will be treated as a late tuple and not processed. Currently the late
+ tuples are just logged in the worker log files at INFO level.
+
+```java
+/**
+* Specify the maximum time lag of the tuple timestamp in milliseconds. It means that the tuple timestamps
+* cannot be out of order by more than this amount.
+*
+* @param duration the max lag duration
+*/
+public BaseWindowedBolt withLag(Duration duration)
+```
+
+### Watermarks
+For processing tuples with timestamp field, storm internally computes watermarks based on the incoming tuple timestamp. Watermark is 
+the minimum of the latest tuple timestamps (minus the lag) across all the input streams. At a higher level this is similar to the watermark concept
+used by Flink and Google's MillWheel for tracking event based timestamps.
+
+Periodically (default every sec), the watermark timestamps are emitted and this is considered as the clock tick for the window calculation if 
+tuple based timestamps are in use. The interval at which watermarks are emitted can be changed with the below api.
+ 
+```java
+/**
+* Specify the watermark event generation interval. For tuple based timestamps, watermark events
+* are used to track the progress of time
+*
+* @param interval the interval at which watermark events are generated
+*/
+public BaseWindowedBolt withWatermarkInterval(Duration interval)
+```
+
+
+When a watermark is received, all windows up to that timestamp will be evaluated.
+
+For example, consider tuple timestamp based processing with following window parameters,
+
+`Window length = 20s, sliding interval = 10s, watermark emit frequency = 1s, max lag = 5s`
+
+```
+|-----|-----|-----|-----|-----|-----|-----|
+0     10    20    30    40    50    60    70
+````
+
+Current ts = `09:00:00`
+
+Tuples `e1(6:00:03), e2(6:00:05), e3(6:00:07), e4(6:00:18), e5(6:00:26), e6(6:00:36)` are received between `9:00:00` and `9:00:01`
+
+At time t = `09:00:01`, watermark w1 = `6:00:31` is emitted since no tuples earlier than `6:00:31` can arrive.
+
+Three windows will be evaluated. The first window end ts (06:00:10) is computed by taking the earliest event timestamp (06:00:03) 
+and computing the ceiling based on the sliding interval (10s).
+
+1. `5:59:50 - 06:00:10` with tuples e1, e2, e3
+2. `6:00:00 - 06:00:20` with tuples e1, e2, e3, e4
+3. `6:00:10 - 06:00:30` with tuples e4, e5
+
+e6 is not evaluated since watermark timestamp `6:00:31` is older than the tuple ts `6:00:36`.
+
+Tuples `e7(8:00:25), e8(8:00:26), e9(8:00:27), e10(8:00:39)` are received between `9:00:01` and `9:00:02`
+
+At time t = `09:00:02` another watermark w2 = `08:00:34` is emitted since no tuples earlier than `8:00:34` can arrive now.
+
+Three windows will be evaluated,
+
+1. `6:00:20 - 06:00:40` with tuples e5, e6 (from earlier batch)
+2. `6:00:30 - 06:00:50` with tuple e6 (from earlier batch)
+3. `8:00:10 - 08:00:30` with tuples e7, e8, e9
+
+e10 is not evaluated since the tuple ts `8:00:39` is beyond the watermark time `8:00:34`.
+
+The window calculation considers the time gaps and computes the windows based on the tuple timestamp.
+
+## Guarentees
+The windowing functionality in storm core currently provides at-least once guarentee. The values emitted from the bolts
+`execute(TupleWindow inputWindow)` method are automatically anchored to all the tuples in the inputWindow. The downstream
+bolts are expected to ack the received tuple (i.e the tuple emitted from the windowed bolt) to complete the tuple tree. 
+If not the tuples will be replayed and the windowing computation will be re-evaluated. 
+
+The tuples in the window are automatically acked when the expire, i.e. when they fall out of the window after 
+`windowLength + slidingInterval`. Note that the configuration `topology.message.timeout.secs` should be sufficiently more 
+than `windowLength + slidingInterval` for time based windows; otherwise the tuples will timeout and get replayed and can result
+in duplicate evaluations. For count based windows, the configuration should be adjusted such that `windowLength + slidingInterval`
+tuples can be received within the timeout period.
+
+## Example topology
+An example toplogy `SlidingWindowTopology` shows how to use the apis to compute a sliding window sum and a tumbling window 
+average.
+

http://git-wip-us.apache.org/repos/asf/storm/blob/d63146b7/documentation/distcache-blobstore.md
----------------------------------------------------------------------
diff --git a/documentation/distcache-blobstore.md b/documentation/distcache-blobstore.md
new file mode 100644
index 0000000..2011ce3
--- /dev/null
+++ b/documentation/distcache-blobstore.md
@@ -0,0 +1,735 @@
+# Storm Distributed Cache API
+
+The distributed cache feature in storm is used to efficiently distribute files
+(or blobs, which is the equivalent terminology for a file in the distributed
+cache and is used interchangeably in this document) that are large and can
+change during the lifetime of a topology, such as geo-location data,
+dictionaries, etc. Typical use cases include phrase recognition, entity
+extraction, document classification, URL re-writing, location/address detection
+and so forth. Such files may be several KB to several GB in size. For small
+datasets that don't need dynamic updates, including them in the topology jar
+could be fine. But for large files, the startup times could become very large.
+In these cases, the distributed cache feature can provide fast topology startup,
+especially if the files were previously downloaded for the same submitter and
+are still in the cache. This is useful with frequent deployments, sometimes few
+times a day with updated jars, because the large cached files will remain available
+without changes. The large cached blobs that do not change frequently will
+remain available in the distributed cache.
+
+At the starting time of a topology, the user specifies the set of files the
+topology needs. Once a topology is running, the user at any time can request for
+any file in the distributed cache to be updated with a newer version. The
+updating of blobs happens in an eventual consistency model. If the topology
+needs to know what version of a file it has access to, it is the responsibility
+of the user to find this information out. The files are stored in a cache with
+Least-Recently Used (LRU) eviction policy, where the supervisor decides which
+cached files are no longer needed and can delete them to free disk space. The
+blobs can be compressed, and the user can request the blobs to be uncompressed
+before it accesses them.
+
+## Motivation for Distributed Cache
+* Allows sharing blobs among topologies.
+* Allows updating the blobs from the command line.
+
+## Distributed Cache Implementations
+The current BlobStore interface has the following two implementations
+* LocalFsBlobStore
+* HdfsBlobStore
+
+Appendix A contains the interface for blobstore implementation.
+
+## LocalFsBlobStore
+![LocalFsBlobStore](images/local_blobstore.png)
+
+Local file system implementation of Blobstore can be depicted in the above timeline diagram.
+
+There are several stages from blob creation to blob download and corresponding execution of a topology. 
+The main stages can be depicted as follows
+
+### Blob Creation Command
+Blobs in the blobstore can be created through command line using the following command.
+
+```
+storm blobstore create --file README.txt --acl o::rwa --repl-fctr 4 key1
+```
+
+The above command creates a blob with a key name “key1” corresponding to the file README.txt. 
+The access given to all users being read, write and admin with a replication factor of 4.
+
+### Topology Submission and Blob Mapping
+Users can submit their topology with the following command. The command includes the 
+topology map configuration. The configuration holds two keys “key1” and “key2” with the 
+key “key1” having a local file name mapping named “blob_file” and it is not compressed.
+
+```
+storm jar /home/y/lib/storm-starter/current/storm-starter-jar-with-dependencies.jar 
+storm.starter.clj.word_count test_topo -c topology.blobstore.map='{"key1":{"localname":"blob_file", "uncompress":"false"},"key2":{}}'
+```
+
+### Blob Creation Process
+The creation of the blob takes place through the interface “ClientBlobStore”. Appendix B contains the “ClientBlobStore” interface. 
+The concrete implementation of this interface is the  “NimbusBlobStore”. In the case of local file system the client makes a 
+call to the nimbus to create the blobs within the local file system. The nimbus uses the local file system implementation to create these blobs. 
+When a user submits a topology, the jar, configuration and code files are uploaded as blobs with the help of blobstore. 
+Also, all the other blobs specified by the topology are mapped to it with the help of topology.blobstore.map configuration.
+
+### Blob Download by the Supervisor
+Finally, the blobs corresponding to a topology are downloaded by the supervisor once it receives the assignments from the nimbus through 
+the same “NimbusBlobStore” thrift client that uploaded the blobs. The supervisor downloads the code, jar and conf blobs by calling the 
+“NimbusBlobStore” client directly while the blobs specified in the topology.blobstore.map are downloaded and mapped locally with the help 
+of the Localizer. The Localizer talks to the “NimbusBlobStore” thrift client to download the blobs and adds the blob compression and local 
+blob name mapping logic to suit the implementation of a topology. Once all the blobs have been downloaded the workers are launched to run 
+the topologies.
+
+## HdfsBlobStore
+![HdfsBlobStore](images/hdfs_blobstore.png)
+
+The HdfsBlobStore functionality has a similar implementation and blob creation and download procedure barring how the replication 
+is handled in the two blobstore implementations. The replication in HDFS blobstore is obvious as HDFS is equipped to handle replication 
+and it requires no state to be stored inside the zookeeper. On the other hand, the local file system blobstore requires the state to be 
+stored on the zookeeper in order for it to work with nimbus HA. Nimbus HA allows the local filesystem to implement the replication feature 
+seamlessly by storing the state in the zookeeper about the running topologies and syncing the blobs on various nimbuses. On the supervisor’s 
+end, the supervisor and localizer talks to HdfsBlobStore through “HdfsClientBlobStore” implementation.
+
+## Additional Features and Documentation
+```
+storm jar /home/y/lib/storm-starter/current/storm-starter-jar-with-dependencies.jar storm.starter.clj.word_count test_topo 
+-c topology.blobstore.map='{"key1":{"localname":"blob_file", "uncompress":"false"},"key2":{}}'
+```
+ 
+### Compression
+The blobstore allows the user to specify the “uncompress” configuration to true or false. This configuration can be specified 
+in the topology.blobstore.map mentioned in the above command. This allows the user to upload a compressed file like a tarball/zip. 
+In local file system blobstore, the compressed blobs are stored on the nimbus node. The localizer code takes the responsibility to 
+uncompress the blob and store it on the supervisor node. Symbolic links to the blobs on the supervisor node are created within the worker 
+before the execution starts.
+
+### Local File Name Mapping
+Apart from compression the blobstore helps to give the blob a name that can be used by the workers. The localizer takes 
+the responsibility of mapping the blob to a local name on the supervisor node.
+
+## Additional Blobstore Implementation Details
+Blobstore uses a hashing function to create the blobs based on the key. The blobs are generally stored inside the directory specified by
+the blobstore.dir configuration. By default, it is stored under “storm.local.dir/nimbus/blobs” for local file system and a similar path on 
+hdfs file system.
+
+Once a file is submitted, the blobstore reads the configs and creates a metadata for the blob with all the access control details. The metadata 
+is generally used for authorization while accessing the blobs. The blob key and version contribute to the hash code and there by the directory 
+under “storm.local.dir/nimbus/blobs/data” where the data is placed. The blobs are generally placed in a positive number directory like 193,822 etc.
+
+Once the topology is launched and the relevant blobs have been created, the supervisor downloads blobs related to the storm.conf, storm.ser 
+and storm.code first and all the blobs uploaded by the command line separately using the localizer to uncompress and map them to a local name 
+specified in the topology.blobstore.map configuration. The supervisor periodically updates blobs by checking for the change of version. 
+This allows updating the blobs on the fly and thereby making it a very useful feature.
+
+For a local file system, the distributed cache on the supervisor node is set to 10240 MB as a soft limit and the clean up code attempts 
+to clean anything over the soft limit every 600 seconds based on LRU policy.
+
+The HDFS blobstore implementation handles load better by removing the burden on the nimbus to store the blobs, which avoids it becoming a bottleneck. Moreover, it provides seamless replication of blobs. On the other hand, the local file system blobstore is not very efficient in 
+replicating the blobs and is limited by the number of nimbuses. Moreover, the supervisor talks to the HDFS blobstore directly without the 
+involvement of the nimbus and thereby reduces the load and dependency on nimbus.
+
+## Highly Available Nimbus
+### Problem Statement:
+Currently the storm master aka nimbus, is a process that runs on a single machine under supervision. In most cases, the 
+nimbus failure is transient and it is restarted by the process that does supervision. However sometimes when disks fail and networks 
+partitions occur, nimbus goes down. Under these circumstances, the topologies run normally but no new topologies can be 
+submitted, no existing topologies can be killed/deactivated/activated and if a supervisor node fails then the 
+reassignments are not performed resulting in performance degradation or topology failures. With this project we intend, 
+to resolve this problem by running nimbus in a primary backup mode to guarantee that even if a nimbus server fails one 
+of the backups will take over. 
+
+### Requirements for Highly Available Nimbus:
+* Increase overall availability of nimbus.
+* Allow nimbus hosts to leave and join the cluster at will any time. A newly joined host should auto catch up and join 
+the list of potential leaders automatically. 
+* No topology resubmissions required in case of nimbus fail overs.
+* No active topology should ever be lost. 
+
+#### Leader Election:
+The nimbus server will use the following interface:
+
+```java
+public interface ILeaderElector {
+    /**
+     * queue up for leadership lock. The call returns immediately and the caller                     
+     * must check isLeader() to perform any leadership action.
+     */
+    void addToLeaderLockQueue();
+
+    /**
+     * Removes the caller from the leader lock queue. If the caller is leader
+     * also releases the lock.
+     */
+    void removeFromLeaderLockQueue();
+
+    /**
+     *
+     * @return true if the caller currently has the leader lock.
+     */
+    boolean isLeader();
+
+    /**
+     *
+     * @return the current leader's address , throws exception if noone has has    lock.
+     */
+    InetSocketAddress getLeaderAddress();
+
+    /**
+     * 
+     * @return list of current nimbus addresses, includes leader.
+     */
+    List<InetSocketAddress> getAllNimbusAddresses();
+}
+```
+Once a nimbus comes up it calls addToLeaderLockQueue() function. The leader election code selects a leader from the queue.
+If the topology code, jar or config blobs are missing, it would download the blobs from any other nimbus which is up and running.
+
+The first implementation will be Zookeeper based. If the zookeeper connection is lost/reset resulting in loss of lock
+or the spot in queue the implementation will take care of updating the state such that isLeader() will reflect the 
+current status. The leader like actions must finish in less than minimumOf(connectionTimeout, SessionTimeout) to ensure
+the lock was held by nimbus for the entire duration of the action (Not sure if we want to just state this expectation 
+and ensure that zk configurations are set high enough which will result in higher failover time or we actually want to 
+create some sort of rollback mechanism for all actions, the second option needs a lot of code). If a nimbus that is not 
+leader receives a request that only a leader can perform,  it will throw a RunTimeException.
+
+### Nimbus state store:
+
+To achieve fail over from primary to backup servers nimbus state/data needs to be replicated across all nimbus hosts or 
+needs to be stored in a distributed storage. Replicating the data correctly involves state management, consistency checks
+and it is hard to test for correctness. However many storm users do not want to take extra dependency on another replicated
+storage system like HDFS and still need high availability. The blobstore implementation along with the state storage helps
+to overcome the failover scenarios in case a leader nimbus goes down.
+
+To support replication we will allow the user to define a code replication factor which would reflect number of nimbus 
+hosts to which the code must be replicated before starting the topology. With replication comes the issue of consistency. 
+The topology is launched once the code, jar and conf blob files are replicated based on the "topology.min.replication" config.
+Maintaining state for failover scenarios is important for local file system. The current implementation makes sure one of the
+available nimbus is elected as a leader in the case of a failure. If the topology specific blobs are missing, the leader nimbus
+tries to download them as and when they are needed. With this current architecture, we do not have to download all the blobs 
+required for a topology for a nimbus to accept leadership. This helps us in case the blobs are very large and avoid causing any 
+inadvertant delays in electing a leader.
+
+The state for every blob is relevant for the local blobstore implementation. For HDFS blobstore the replication
+is taken care by the HDFS. For handling the fail over scenarios for a local blobstore we need to store the state of the leader and
+non-leader nimbuses within the zookeeper.
+
+The state is stored under /storm/blobstore/key/nimbusHostPort:SequenceNumber for the blobstore to work to make nimbus highly available. 
+This state is used in the local file system blobstore to support replication. The HDFS blobstore does not have to store the state inside the 
+zookeeper.
+
+* NimbusHostPort: This piece of information generally contains the parsed string holding the hostname and port of the nimbus. 
+  It uses the same class “NimbusHostPortInfo” used earlier by the code-distributor interface to store the state and parse the data.
+
+* SequenceNumber: This is the blob sequence number information. The SequenceNumber information is implemented by a KeySequenceNumber class. 
+The sequence numbers are generated for every key. For every update, the sequence numbers are assigned based ona global sequence number 
+stored under /storm/blobstoremaxsequencenumber/key. For more details about how the numbers are generated you can look at the java docs for KeySequenceNumber.
+
+![Nimbus High Availability - BlobStore](images/nimbus_ha_blobstore.png)
+
+The sequence diagram proposes how the blobstore works and the state storage inside the zookeeper makes the nimbus highly available.
+Currently, the thread to sync the blobs on a non-leader is within the nimbus. In the future, it will be nice to move the thread around
+to the blobstore to make the blobstore coordinate the state change and blob download as per the sequence diagram.
+
+## Thrift and Rest API 
+In order to avoid workers/supervisors/ui talking to zookeeper for getting master nimbus address we are going to modify the 
+`getClusterInfo` API so it can also return nimbus information. getClusterInfo currently returns `ClusterSummary` instance
+which has a list of `supervisorSummary` and a list of `topologySummary` instances. We will add a list of `NimbusSummary` 
+to the `ClusterSummary`. See the structures below:
+
+```thrift
+struct ClusterSummary {
+  1: required list<SupervisorSummary> supervisors;
+  3: required list<TopologySummary> topologies;
+  4: required list<NimbusSummary> nimbuses;
+}
+
+struct NimbusSummary {
+  1: required string host;
+  2: required i32 port;
+  3: required i32 uptime_secs;
+  4: required bool isLeader;
+  5: required string version;
+}
+```
+
+This will be used by StormSubmitter, Nimbus clients, supervisors and ui to discover the current leaders and participating 
+nimbus hosts. Any nimbus host will be able to respond to these requests. The nimbus hosts can read this information once 
+from zookeeper and cache it and keep updating the cache when the watchers are fired to indicate any changes,which should 
+be rare in general case.
+
+Note: All nimbus hosts have watchers on zookeeper to be notified immediately as soon as a new blobs is available for download, the callback may or may not download
+the code. Therefore, a background thread is triggered to download the respective blobs to run the topologies. The replication is achieved when the blobs are downloaded
+onto non-leader nimbuses. So you should expect your topology submission time to be somewhere between 0 to (2 * nimbus.code.sync.freq.secs) for any 
+nimbus.min.replication.count > 1.
+
+## Configuration
+
+```
+blobstore.dir: The directory where all blobs are stored. For local file system it represents the directory on the nimbus
+node and for HDFS file system it represents the hdfs file system path.
+
+supervisor.blobstore.class: This configuration is meant to set the client for  the supervisor  in order to talk to the blobstore. 
+For a local file system blobstore it is set to “backtype.storm.blobstore.NimbusBlobStore” and for the HDFS blobstore it is set 
+to “backtype.storm.blobstore.HdfsClientBlobStore”.
+
+supervisor.blobstore.download.thread.count: This configuration spawns multiple threads for from the supervisor in order download 
+blobs concurrently. The default is set to 5
+
+supervisor.blobstore.download.max_retries: This configuration is set to allow the supervisor to retry for the blob download. 
+By default it is set to 3.
+
+supervisor.localizer.cache.target.size.mb: The jvm opts provided to workers launched by this supervisor. All "%ID%" substrings 
+are replaced with an identifier for this worker. Also, "%WORKER-ID%", "%STORM-ID%" and "%WORKER-PORT%" are replaced with 
+appropriate runtime values for this worker. The distributed cache target size in MB. This is a soft limit to the size 
+of the distributed cache contents. It is set to 10240 MB.
+
+supervisor.localizer.cleanup.interval.ms: The distributed cache cleanup interval. Controls how often it scans to attempt to 
+cleanup anything over the cache target size. By default it is set to 600000 milliseconds.
+
+nimbus.blobstore.class:  Sets the blobstore implementation nimbus uses. It is set to "backtype.storm.blobstore.LocalFsBlobStore"
+
+nimbus.blobstore.expiration.secs: During operations with the blobstore, via master, how long a connection is idle before nimbus 
+considers it dead and drops the session and any associated connections. The default is set to 600.
+
+storm.blobstore.inputstream.buffer.size.bytes: The buffer size it uses for blobstore upload. It is set to 65536 bytes.
+
+client.blobstore.class: The blobstore implementation the storm client uses. The current implementation uses the default 
+config "backtype.storm.blobstore.NimbusBlobStore".
+
+blobstore.replication.factor: It sets the replication for each blob within the blobstore. The “topology.min.replication.count” 
+ensures the minimum replication the topology specific blobs are set before launching the topology. You might want to set the 
+“topology.min.replication.count <= blobstore.replication”. The default is set to 3.
+
+topology.min.replication.count : Minimum number of nimbus hosts where the code must be replicated before leader nimbus
+can mark the topology as active and create assignments. Default is 1.
+
+topology.max.replication.wait.time.sec: Maximum wait time for the nimbus host replication to achieve the nimbus.min.replication.count.
+Once this time is elapsed nimbus will go ahead and perform topology activation tasks even if required nimbus.min.replication.count is not achieved. 
+The default is 60 seconds, a value of -1 indicates to wait for ever.
+* nimbus.code.sync.freq.secs: Frequency at which the background thread on nimbus which syncs code for locally missing blobs. Default is 2 minutes.
+```
+
+## Using the Distributed Cache API, Command Line Interface (CLI)
+
+### Creating blobs 
+
+To use the distributed cache feature, the user first has to "introduce" files
+that need to be cached and bind them to key strings. To achieve this, the user
+uses the "blobstore create" command of the storm executable, as follows:
+
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+storm blobstore create [-f|--file FILE] [-a|--acl ACL1,ACL2,...] [--repl-fctr NUMBER] [keyname]
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+The contents come from a FILE, if provided by -f or --file option, otherwise
+from STDIN.  
+The ACLs, which can also be a comma separated list of many ACLs, is of the
+following format:
+
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+> [u|o]:[username]:[r-|w-|a-|_]
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+where:  
+
+* u = user  
+* o = other  
+* username = user for this particular ACL  
+* r = read access  
+* w = write access  
+* a = admin access  
+* _ = ignored  
+
+The replication factor can be set to a value greater than 1 using --repl-fctr.
+
+Note: The replication right now is configurable for a hdfs blobstore but for a
+local blobstore the replication always stays at 1. For a hdfs blobstore
+the default replication is set to 3.
+
+###### Example:  
+
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+storm blobstore create --file README.txt --acl o::rwa --repl-fctr 4 key1
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+In the above example, the *README.txt* file is added to the distributed cache.
+It can be accessed using the key string "*key1*" for any topology that needs
+it. The file is set to have read/write/admin access for others, a.k.a world
+everything and the replication is set to 4.
+
+###### Example:  
+
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+storm blobstore create mytopo:data.tgz -f data.tgz -a u:alice:rwa,u:bob:rw,o::r  
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+The above example createss a mytopo:data.tgz key using the data stored in
+data.tgz.  User alice would have full access, bob would have read/write access
+and everyone else would have read access.
+
+### Making dist. cache files accessible to topologies
+
+Once a blob is created, we can use it for topologies. This is generally achieved
+by including the key string among the configurations of a topology, with the
+following format. A shortcut is to add the configuration item on the command
+line when starting a topology by using the **-c** command:
+
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+-c topology.blobstore.map='{"[KEY]":{"localname":"[VALUE]", "uncompress":"[true|false]"}}'
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+Note: Please take care of the quotes.
+
+The cache file would then be accessible to the topology as a local file with the
+name [VALUE].  
+The localname parameter is optional, if omitted the local cached file will have
+the same name as [KEY].  
+The uncompress parameter is optional, if omitted the local cached file will not
+be uncompressed.  Note that the key string needs to have the appropriate
+file-name-like format and extension, so it can be uncompressed correctly.
+
+###### Example:  
+
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+storm jar /home/y/lib/storm-starter/current/storm-starter-jar-with-dependencies.jar storm.starter.clj.word_count test_topo -c topology.blobstore.map='{"key1":{"localname":"blob_file", "uncompress":"false"},"key2":{}}'
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+Note: Please take care of the quotes.
+
+In the above example, we start the *word_count* topology (stored in the
+*storm-starter-jar-with-dependencies.jar* file), and ask it to have access
+to the cached file stored with key string = *key1*. This file would then be
+accessible to the topology as a local file called *blob_file*, and the
+supervisor will not try to uncompress the file. Note that in our example, the
+file's content originally came from *README.txt*. We also ask for the file
+stored with the key string = *key2* to be accessible to the topology. Since
+both the optional parameters are omitted, this file will get the local name =
+*key2*, and will not be uncompressed.
+
+### Updating a cached file
+
+It is possible for the cached files to be updated while topologies are running.
+The update happens in an eventual consistency model, where the supervisors poll
+Nimbus every 30 seconds, and update their local copies. In the current version,
+it is the user's responsibility to check whether a new file is available.
+
+To update a cached file, use the following command. Contents come from a FILE or
+STDIN. Write access is required to be able to update a cached file.
+
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+storm blobstore update [-f|--file NEW_FILE] [KEYSTRING]
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+###### Example:  
+
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+storm blobstore update -f updates.txt key1
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+In the above example, the topologies will be presented with the contents of the
+file *updates.txt* instead of *README.txt* (from the previous example), even
+though their access by the topology is still through a file called
+*blob_file*.
+
+### Removing a cached file
+
+To remove a file from the distributed cache, use the following command. Removing
+a file requires write access.
+
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+storm blobstore delete [KEYSTRING]
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+### Listing Blobs currently in the distributed cache blobstore
+
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+storm blobstore list [KEY...]
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+lists blobs currently in the blobstore
+
+### Reading the contents of a blob
+
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+storm blobstore cat [-f|--file FILE] KEY
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+read a blob and then either write it to a file, or STDOUT. Reading a blob
+requires read access.
+
+### Setting the access control for a blob
+
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+set-acl [-s ACL] KEY
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+ACL is in the form [uo]:[username]:[r-][w-][a-] can be comma  separated list
+(requires admin access).
+
+### Update the replication factor for a blob
+
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+storm blobstore replication --update --repl-fctr 5 key1
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+### Read the replication factor of a blob
+
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+storm blobstore replication --read key1
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+### Command line help
+
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+storm help blobstore
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+## Using the Distributed Cache API from Java
+
+We start by getting a ClientBlobStore object by calling this function:
+
+``` java
+Config theconf = new Config();
+theconf.putAll(Utils.readStormConfig());
+ClientBlobStore clientBlobStore = Utils.getClientBlobStore(theconf);
+```
+
+The required Utils package can by imported by:
+
+```java
+import backtype.storm.utils.Utils;
+```
+
+ClientBlobStore and other blob-related classes can be imported by:
+
+```java
+import backtype.storm.blobstore.ClientBlobStore;
+import backtype.storm.blobstore.AtomicOutputStream;
+import backtype.storm.blobstore.InputStreamWithMeta;
+import backtype.storm.blobstore.BlobStoreAclHandler;
+import backtype.storm.generated.*;
+```
+
+### Creating ACLs to be used for blobs
+
+```java
+String stringBlobACL = "u:username:rwa";
+AccessControl blobACL = BlobStoreAclHandler.parseAccessControl(stringBlobACL);
+List<AccessControl> acls = new LinkedList<AccessControl>();
+acls.add(blobACL); // more ACLs can be added here
+SettableBlobMeta settableBlobMeta = new SettableBlobMeta(acls);
+settableBlobMeta.set_replication_factor(4); // Here we can set the replication factor
+```
+
+The settableBlobMeta object is what we need to create a blob in the next step. 
+
+### Creating a blob
+
+```java
+AtomicOutputStream blobStream = clientBlobStore.createBlob("some_key", settableBlobMeta);
+blobStream.write("Some String or input data".getBytes());
+blobStream.close();
+```
+
+Note that the settableBlobMeta object here comes from the last step, creating ACLs.
+It is recommended that for very large files, the user writes the bytes in smaller chunks (for example 64 KB, up to 1 MB chunks).
+
+### Updating a blob
+
+Similar to creating a blob, but we get the AtomicOutputStream in a different way:
+
+```java
+String blobKey = "some_key";
+AtomicOutputStream blobStream = clientBlobStore.updateBlob(blobKey);
+```
+
+Pass a byte stream to the returned AtomicOutputStream as before. 
+
+### Updating the ACLs of a blob
+
+```java
+String blobKey = "some_key";
+AccessControl updateAcl = BlobStoreAclHandler.parseAccessControl("u:USER:--a");
+List<AccessControl> updateAcls = new LinkedList<AccessControl>();
+updateAcls.add(updateAcl);
+SettableBlobMeta modifiedSettableBlobMeta = new SettableBlobMeta(updateAcls);
+clientBlobStore.setBlobMeta(blobKey, modifiedSettableBlobMeta);
+
+//Now set write only
+updateAcl = BlobStoreAclHandler.parseAccessControl("u:USER:-w-");
+updateAcls = new LinkedList<AccessControl>();
+updateAcls.add(updateAcl);
+modifiedSettableBlobMeta = new SettableBlobMeta(updateAcls);
+clientBlobStore.setBlobMeta(blobKey, modifiedSettableBlobMeta);
+```
+
+### Updating and Reading the replication of a blob
+
+```java
+String blobKey = "some_key";
+BlobReplication replication = clientBlobStore.updateBlobReplication(blobKey, 5);
+int replication_factor = replication.get_replication();
+```
+
+Note: The replication factor gets updated and reflected only for hdfs blobstore
+
+### Reading a blob
+
+```java
+String blobKey = "some_key";
+InputStreamWithMeta blobInputStream = clientBlobStore.getBlob(blobKey);
+BufferedReader r = new BufferedReader(new InputStreamReader(blobInputStream));
+String blobContents =  r.readLine();
+```
+
+### Deleting a blob
+
+```java
+String blobKey = "some_key";
+clientBlobStore.deleteBlob(blobKey);
+```
+
+### Getting a list of blob keys already in the blobstore
+
+```java
+Iterator <String> stringIterator = clientBlobStore.listKeys();
+```
+
+## Appendix A
+
+```java
+public abstract void prepare(Map conf, String baseDir);
+
+public abstract AtomicOutputStream createBlob(String key, SettableBlobMeta meta, Subject who) throws AuthorizationException, KeyAlreadyExistsException;
+
+public abstract AtomicOutputStream updateBlob(String key, Subject who) throws AuthorizationException, KeyNotFoundException;
+
+public abstract ReadableBlobMeta getBlobMeta(String key, Subject who) throws AuthorizationException, KeyNotFoundException;
+
+public abstract void setBlobMeta(String key, SettableBlobMeta meta, Subject who) throws AuthorizationException, KeyNotFoundException;
+
+public abstract void deleteBlob(String key, Subject who) throws AuthorizationException, KeyNotFoundException;
+
+public abstract InputStreamWithMeta getBlob(String key, Subject who) throws AuthorizationException, KeyNotFoundException;
+
+public abstract Iterator<String> listKeys(Subject who);
+
+public abstract BlobReplication getBlobReplication(String key, Subject who) throws Exception;
+
+public abstract BlobReplication updateBlobReplication(String key, int replication, Subject who) throws AuthorizationException, KeyNotFoundException, IOException
+```
+
+## Appendix B
+
+```java
+public abstract void prepare(Map conf);
+
+protected abstract AtomicOutputStream createBlobToExtend(String key, SettableBlobMeta meta) throws AuthorizationException, KeyAlreadyExistsException;
+
+public abstract AtomicOutputStream updateBlob(String key) throws AuthorizationException, KeyNotFoundException;
+
+public abstract ReadableBlobMeta getBlobMeta(String key) throws AuthorizationException, KeyNotFoundException;
+
+protected abstract void setBlobMetaToExtend(String key, SettableBlobMeta meta) throws AuthorizationException, KeyNotFoundException;
+
+public abstract void deleteBlob(String key) throws AuthorizationException, KeyNotFoundException;
+
+public abstract InputStreamWithMeta getBlob(String key) throws AuthorizationException, KeyNotFoundException;
+
+public abstract Iterator<String> listKeys();
+
+public abstract void watchBlob(String key, IBlobWatcher watcher) throws AuthorizationException;
+
+public abstract void stopWatchingBlob(String key) throws AuthorizationException;
+
+public abstract BlobReplication getBlobReplication(String Key) throws AuthorizationException, KeyNotFoundException;
+
+public abstract BlobReplication updateBlobReplication(String Key, int replication) throws AuthorizationException, KeyNotFoundException
+```
+
+## Appendix C
+
+``` thrift
+service Nimbus {
+...
+string beginCreateBlob(1: string key, 2: SettableBlobMeta meta) throws (1: AuthorizationException aze, 2: KeyAlreadyExistsException kae);
+
+string beginUpdateBlob(1: string key) throws (1: AuthorizationException aze, 2: KeyNotFoundException knf);
+
+void uploadBlobChunk(1: string session, 2: binary chunk) throws (1: AuthorizationException aze);
+
+void finishBlobUpload(1: string session) throws (1: AuthorizationException aze);
+
+void cancelBlobUpload(1: string session) throws (1: AuthorizationException aze);
+
+ReadableBlobMeta getBlobMeta(1: string key) throws (1: AuthorizationException aze, 2: KeyNotFoundException knf);
+
+void setBlobMeta(1: string key, 2: SettableBlobMeta meta) throws (1: AuthorizationException aze, 2: KeyNotFoundException knf);
+
+BeginDownloadResult beginBlobDownload(1: string key) throws (1: AuthorizationException aze, 2: KeyNotFoundException knf);
+
+binary downloadBlobChunk(1: string session) throws (1: AuthorizationException aze);
+
+void deleteBlob(1: string key) throws (1: AuthorizationException aze, 2: KeyNotFoundException knf);
+
+ListBlobsResult listBlobs(1: string session);
+
+BlobReplication getBlobReplication(1: string key) throws (1: AuthorizationException aze, 2: KeyNotFoundException knf);
+
+BlobReplication updateBlobReplication(1: string key, 2: i32 replication) throws (1: AuthorizationException aze, 2: KeyNotFoundException knf);
+...
+}
+
+struct BlobReplication {
+1: required i32 replication;
+}
+
+exception AuthorizationException {
+ 1: required string msg;
+}
+
+exception KeyNotFoundException {
+ 1: required string msg;
+}
+
+exception KeyAlreadyExistsException {
+ 1: required string msg;
+}
+
+enum AccessControlType {
+ OTHER = 1,
+ USER = 2
+ //eventually ,GROUP=3
+}
+
+struct AccessControl {
+ 1: required AccessControlType type;
+ 2: optional string name; //Name of user or group in ACL
+ 3: required i32 access; //bitmasks READ=0x1, WRITE=0x2, ADMIN=0x4
+}
+
+struct SettableBlobMeta {
+ 1: required list<AccessControl> acl;
+ 2: optional i32 replication_factor
+}
+
+struct ReadableBlobMeta {
+ 1: required SettableBlobMeta settable;
+ //This is some indication of a version of a BLOB.  The only guarantee is
+ // if the data changed in the blob the version will be different.
+ 2: required i64 version;
+}
+
+struct ListBlobsResult {
+ 1: required list<string> keys;
+ 2: required string session;
+}
+
+struct BeginDownloadResult {
+ //Same version as in ReadableBlobMeta
+ 1: required i64 version;
+ 2: required string session;
+ 3: optional i64 data_size;
+}
+```

http://git-wip-us.apache.org/repos/asf/storm/blob/d63146b7/documentation/dynamic-log-level-settings.md
----------------------------------------------------------------------
diff --git a/documentation/dynamic-log-level-settings.md b/documentation/dynamic-log-level-settings.md
new file mode 100644
index 0000000..f38b708
--- /dev/null
+++ b/documentation/dynamic-log-level-settings.md
@@ -0,0 +1,41 @@
+Dynamic Log Level Settings
+==========================
+
+We have added the ability to set log level settings for a running topology using the Storm UI and the Storm CLI. 
+
+The log level settings apply the same way as you'd expect from log4j, as all we are doing is telling log4j to set the level of the logger you provide. If you set the log level of a parent logger, the children loggers start using that level (unless the children have a more restrictive level already). A timeout can optionally be provided (except for DEBUG mode, where it’s required in the UI), if workers should reset log levels automatically.
+
+This revert action is triggered using a polling mechanism (every 30 seconds, but this is configurable), so you should expect your timeouts to be the value you provided plus anywhere between 0 and the setting's value.
+
+Using the Storm UI
+-------------
+
+In order to set a level, click on a running topology, and then click on “Change Log Level” in the Topology Actions section.
+
+![Change Log Level dialog](images/dynamic_log_level_settings_1.png "Change Log Level dialog")
+
+Next, provide the logger name, select the level you expect (e.g. WARN), and a timeout in seconds (or 0 if not needed). Then click on “Add”.
+
+![After adding a log level setting](images/dynamic_log_level_settings_2.png "After adding a log level setting")
+
+To clear the log level click on the “Clear” button. This reverts the log level back to what it was before you added the setting. The log level line will disappear from the UI.
+
+While there is a delay resetting log levels back, setting the log level in the first place is immediate (or as quickly as the message can travel from the UI/CLI to the workers by way of nimbus and zookeeper).
+
+Using the CLI
+-------------
+
+Using the CLI, issue the command:
+
+`./bin/storm set_log_level [topology name] -l [logger name]=[LEVEL]:[TIMEOUT]`
+
+For example:
+
+`./bin/storm set_log_level my_topology -l ROOT=DEBUG:30`
+
+Sets the ROOT logger to DEBUG for 30 seconds.
+
+`./bin/storm set_log_level my_topology -r ROOT`
+
+Clears the ROOT logger dynamic log level, resetting it to its original value.
+

http://git-wip-us.apache.org/repos/asf/storm/blob/d63146b7/documentation/dynamic-worker-profiling.md
----------------------------------------------------------------------
diff --git a/documentation/dynamic-worker-profiling.md b/documentation/dynamic-worker-profiling.md
new file mode 100644
index 0000000..088a232
--- /dev/null
+++ b/documentation/dynamic-worker-profiling.md
@@ -0,0 +1,33 @@
+Dynamic Worker Profiling
+==========================
+
+In multi-tenant mode, storm launches long-running JVMs across cluster without sudo access to user. Self-serving of Java heap-dumps, jstacks and java profiling of these JVMs would improve users' ability to analyze and debug issues when monitoring it actively.
+
+The storm dynamic profiler lets you dynamically take heap-dumps, jprofile or jstack for a worker jvm running on stock cluster. It let user download these dumps from the browser and use your favorite tools to analyze it  The UI component page provides list workers for the component and action buttons. The logviewer lets you download the dumps generated by these logs. Please see the screenshots for more information.
+
+Using the Storm UI
+-------------
+
+In order to request for heap-dump, jstack, start/stop/dump jprofile or restart a worker, click on a running topology, then click on specific component, then you can select workers by checking the box of any of the worker's executors in the Executors table, and then click on “Start","Heap", "Jstack" or "Restart Worker" in the "Profiling and Debugging" section.
+
+![Selecting Workers](images/dynamic_profiling_debugging_4.png "Selecting Workers")
+
+In the Executors table, click the checkbox in the Actions column next to any executor, and any other executors belonging to the same worker are automatically selected. When the action has completed, any output files created will available at the link in the Actions column.
+
+![Profiling and Debugging](images/dynamic_profiling_debugging_1.png "Profiling and Debugging")
+
+For start jprofile, provide a timeout in minutes (or 10 if not needed). Then click on “Start”.
+
+![After starting jprofile for worker](images/dynamic_profiling_debugging_2.png "After jprofile for worker ")
+
+To stop the jprofile logging click on the “Stop” button. This dumps the jprofile stats and stops the profiling. Refresh the page for the line to disappear from the UI.
+
+Click on "My Dump Files" to go the logviewer UI for list of worker specific dump files.
+
+![Dump Files Links for worker](images/dynamic_profiling_debugging_3.png "Dump Files Links for worker")
+
+Configuration
+-------------
+
+The "worker.profiler.command" can be configured to point to specific pluggable profiler, heapdump commands. The "worker.profiler.enabled" can be disabled if plugin is not available or jdk does not support Jprofile flight recording so that worker JVM options will not have "worker.profiler.childopts". To use different profiler plugin, you can change these configuration.
+

http://git-wip-us.apache.org/repos/asf/storm/blob/d63146b7/documentation/images/dynamic_log_level_settings_1.png
----------------------------------------------------------------------
diff --git a/documentation/images/dynamic_log_level_settings_1.png b/documentation/images/dynamic_log_level_settings_1.png
new file mode 100644
index 0000000..71d42e7
Binary files /dev/null and b/documentation/images/dynamic_log_level_settings_1.png differ

http://git-wip-us.apache.org/repos/asf/storm/blob/d63146b7/documentation/images/dynamic_log_level_settings_2.png
----------------------------------------------------------------------
diff --git a/documentation/images/dynamic_log_level_settings_2.png b/documentation/images/dynamic_log_level_settings_2.png
new file mode 100644
index 0000000..d0e61a7
Binary files /dev/null and b/documentation/images/dynamic_log_level_settings_2.png differ

http://git-wip-us.apache.org/repos/asf/storm/blob/d63146b7/documentation/images/dynamic_profiling_debugging_1.png
----------------------------------------------------------------------
diff --git a/documentation/images/dynamic_profiling_debugging_1.png b/documentation/images/dynamic_profiling_debugging_1.png
new file mode 100644
index 0000000..6be1f86
Binary files /dev/null and b/documentation/images/dynamic_profiling_debugging_1.png differ

http://git-wip-us.apache.org/repos/asf/storm/blob/d63146b7/documentation/images/dynamic_profiling_debugging_2.png
----------------------------------------------------------------------
diff --git a/documentation/images/dynamic_profiling_debugging_2.png b/documentation/images/dynamic_profiling_debugging_2.png
new file mode 100644
index 0000000..342ad94
Binary files /dev/null and b/documentation/images/dynamic_profiling_debugging_2.png differ

http://git-wip-us.apache.org/repos/asf/storm/blob/d63146b7/documentation/images/dynamic_profiling_debugging_3.png
----------------------------------------------------------------------
diff --git a/documentation/images/dynamic_profiling_debugging_3.png b/documentation/images/dynamic_profiling_debugging_3.png
new file mode 100644
index 0000000..5706d7e
Binary files /dev/null and b/documentation/images/dynamic_profiling_debugging_3.png differ

http://git-wip-us.apache.org/repos/asf/storm/blob/d63146b7/documentation/images/dynamic_profiling_debugging_4.png
----------------------------------------------------------------------
diff --git a/documentation/images/dynamic_profiling_debugging_4.png b/documentation/images/dynamic_profiling_debugging_4.png
new file mode 100644
index 0000000..0afe9f4
Binary files /dev/null and b/documentation/images/dynamic_profiling_debugging_4.png differ

http://git-wip-us.apache.org/repos/asf/storm/blob/d63146b7/documentation/images/hdfs_blobstore.png
----------------------------------------------------------------------
diff --git a/documentation/images/hdfs_blobstore.png b/documentation/images/hdfs_blobstore.png
new file mode 100644
index 0000000..11c5c10
Binary files /dev/null and b/documentation/images/hdfs_blobstore.png differ

http://git-wip-us.apache.org/repos/asf/storm/blob/d63146b7/documentation/images/local_blobstore.png
----------------------------------------------------------------------
diff --git a/documentation/images/local_blobstore.png b/documentation/images/local_blobstore.png
new file mode 100644
index 0000000..ff8001e
Binary files /dev/null and b/documentation/images/local_blobstore.png differ

http://git-wip-us.apache.org/repos/asf/storm/blob/d63146b7/documentation/images/nimbus_ha_blobstore.png
----------------------------------------------------------------------
diff --git a/documentation/images/nimbus_ha_blobstore.png b/documentation/images/nimbus_ha_blobstore.png
new file mode 100644
index 0000000..26e8c2a
Binary files /dev/null and b/documentation/images/nimbus_ha_blobstore.png differ

http://git-wip-us.apache.org/repos/asf/storm/blob/d63146b7/documentation/images/search-a-topology.png
----------------------------------------------------------------------
diff --git a/documentation/images/search-a-topology.png b/documentation/images/search-a-topology.png
new file mode 100644
index 0000000..8d6153c
Binary files /dev/null and b/documentation/images/search-a-topology.png differ

http://git-wip-us.apache.org/repos/asf/storm/blob/d63146b7/documentation/images/search-for-a-single-worker-log.png
----------------------------------------------------------------------
diff --git a/documentation/images/search-for-a-single-worker-log.png b/documentation/images/search-for-a-single-worker-log.png
new file mode 100644
index 0000000..8c6f423
Binary files /dev/null and b/documentation/images/search-for-a-single-worker-log.png differ

http://git-wip-us.apache.org/repos/asf/storm/blob/d63146b7/documentation/storm-metrics-profiling-internal-actions.md
----------------------------------------------------------------------
diff --git a/documentation/storm-metrics-profiling-internal-actions.md b/documentation/storm-metrics-profiling-internal-actions.md
new file mode 100644
index 0000000..e549c0c
--- /dev/null
+++ b/documentation/storm-metrics-profiling-internal-actions.md
@@ -0,0 +1,70 @@
+# Storm Metrics for Profiling Various Storm Internal Actions
+
+With the addition of these metrics, Storm users can collect, view, and analyze the performance of various internal actions.  The actions that are profiled include thrift rpc calls and http quests within the storm daemons. For instance, in the Storm Nimbus daemon, the following thrift calls defined in the Nimbus$Iface are profiled:
+
+- submitTopology
+- submitTopologyWithOpts
+- killTopology
+- killTopologyWithOpts
+- activate
+- deactivate
+- rebalance
+- setLogConfig
+- getLogConfig
+
+Various HTTP GET and POST requests are marked for profiling as well such as the GET and POST requests for the Storm UI daemon (ui/core.cj)
+To implement these metrics the following packages are used: 
+- io.dropwizard.metrics
+- metrics-clojure
+
+## How it works
+
+By using packages io.dropwizard.metrics and metrics-clojure (clojure wrapper for the metrics Java API), we can mark functions to profile by declaring (defmeter num-some-func-calls) and then adding the (mark! num-some-func-calls) to where the function is invoked. For example:
+
+    (defmeter num-some-func-calls)
+    (defn some-func [args]
+        (mark! num-some-func-calls)
+        (body))
+        
+What essentially the mark! API call does is increment a counter that represents how many times a certain action occured.  For instantanous measurements user can use gauges.  For example: 
+
+    (defgauge nimbus:num-supervisors
+         (fn [] (.size (.supervisors (:storm-cluster-state nimbus) nil))))
+         
+The above example will get the number of supervisors in the cluster.  This metric is not accumlative like one previously discussed.
+
+A metrics reporting server needs to also be activated to collect the metrics. You can do this by calling the following function:
+
+    (defn start-metrics-reporters []
+        (jmx/start (jmx/reporter {})))
+
+## How to collect the metrics
+
+Metrics can be reported via JMX or HTTP.  A user can use JConsole or VisualVM to connect to the jvm process and view the stats.
+
+To view the metrics in a GUI use VisualVM or JConsole.  Screenshot of using VisualVm for metrics: 
+
+![Viewing metrics with VisualVM](images/viewing_metrics_with_VisualVM.png)
+
+For detailed information regarding how to collect the metrics please reference: 
+
+https://dropwizard.github.io/metrics/3.1.0/getting-started/
+
+If you want use JMX and view metrics through JConsole or VisualVM, remember launch JVM processes your want to profile with the correct JMX configurations.  For example in Storm you would add the following to conf/storm.yaml
+
+    nimbus.childopts: "-Xmx1024m -Dcom.sun.management.jmxremote.local.only=false -Dcom.sun.management.jmxremote -Dcom.sun.management.jmxremote.port=3333  -Dcom.sun.management.jmxremote.ssl=false -Dcom.sun.management.jmxremote.authenticate=false"
+    
+    ui.childopts: "-Xmx768m -Dcom.sun.management.jmxremote.port=3334 -Dcom.sun.management.jmxremote.local.only=false -Dcom.sun.management.jmxremote -Dcom.sun.management.jmxremote.ssl=false -Dcom.sun.management.jmxremote.authenticate=false"
+    
+    logviewer.childopts: "-Xmx128m -Dcom.sun.management.jmxremote.port=3335 -Dcom.sun.management.jmxremote.local.only=false -Dcom.sun.management.jmxremote -Dcom.sun.management.jmxremote.ssl=false -Dcom.sun.management.jmxremote.authenticate=false"
+    
+    drpc.childopts: "-Xmx768m -Dcom.sun.management.jmxremote.port=3336 -Dcom.sun.management.jmxremote.local.only=false -Dcom.sun.management.jmxremote -Dcom.sun.management.jmxremote.ssl=false -Dcom.sun.management.jmxremote.authenticate=false"
+   
+    supervisor.childopts: "-Xmx256m -Dcom.sun.management.jmxremote.port=3337 -Dcom.sun.management.jmxremote.local.only=false -Dcom.sun.management.jmxremote -Dcom.sun.management.jmxremote.ssl=false -Dcom.sun.management.jmxremote.authenticate=false"
+
+### Please Note:
+Since we shade all of the packages we use, additional plugins for collecting metrics might not work at this time.  Currently collecting the metrics via JMX is supported.
+   
+For more information about io.dropwizard.metrics and metrics-clojure packages please reference their original documentation:
+- https://dropwizard.github.io/metrics/3.1.0/
+- http://metrics-clojure.readthedocs.org/en/latest/
\ No newline at end of file