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

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

http://git-wip-us.apache.org/repos/asf/storm/blob/cf0cdbb0/docs/STORM-UI-REST-API.md
----------------------------------------------------------------------
diff --git a/docs/STORM-UI-REST-API.md b/docs/STORM-UI-REST-API.md
new file mode 100644
index 0000000..2109ab2
--- /dev/null
+++ b/docs/STORM-UI-REST-API.md
@@ -0,0 +1,678 @@
+---
+title: Storm UI REST API
+layout: documentation
+documentation: true
+---
+
+# 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|
+|nimbusUptime|String| Shows how long the cluster is running|
+|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",
+    "nimbusUptime": "3m 53s",
+    "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|
+|slotsTotal| Integer| Total number of available worker slots for this supervisor|
+|slotsUsed| Integer| Number of worker slots used on this supervisor|
+
+Sample response:
+
+```json
+{
+    "supervisors": [
+        {
+            "id": "0b879808-2a26-442b-8f7d-23101e0c3696",
+            "host": "10.11.1.7",
+            "uptime": "5m 58s",
+            "slotsTotal": 4,
+            "slotsUsed": 3
+        }
+    ]
+}
+```
+
+### /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|
+|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|
+
+Sample response:
+
+```json
+{
+    "topologies": [
+        {
+            "id": "WordCount3-1-1402960825",
+            "name": "WordCount3",
+            "status": "ACTIVE",
+            "uptime": "6m 5s",
+            "tasksTotal": 28,
+            "workersTotal": 3,
+            "executorsTotal": 28
+        }
+    ]
+}
+```
+
+### /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|
+|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"|
+|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|
+
+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",
+    "msgTimeout": 30,
+    "windowHint": "10m 0s",
+    "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
+    }
+}
+```
+
+
+### /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.time| Long | Timestamp when the exception occurred |
+|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":[{"time": 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",
+            "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",
+            "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",
+            "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",
+            "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",
+            "id": "[28-28]",
+            "failed": 0
+        }
+    ]
+}
+```
+
+## 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"
+}
+```

http://git-wip-us.apache.org/repos/asf/storm/blob/cf0cdbb0/docs/Serialization-(prior-to-0.6.0).md
----------------------------------------------------------------------
diff --git a/docs/Serialization-(prior-to-0.6.0).md b/docs/Serialization-(prior-to-0.6.0).md
new file mode 100644
index 0000000..e4a0d4f
--- /dev/null
+++ b/docs/Serialization-(prior-to-0.6.0).md
@@ -0,0 +1,50 @@
+---
+layout: documentation
+---
+Tuples can be comprised of objects of any types. Since Storm is a distributed system, it needs to know how to serialize and deserialize objects when they're passed between tasks. By default Storm can serialize ints, shorts, longs, floats, doubles, bools, bytes, strings, and byte arrays, but if you want to use another type in your tuples, you'll need to implement a custom serializer.
+
+### Dynamic typing
+
+There are no type declarations for fields in a Tuple. You put objects in fields and Storm figures out the serialization dynamically. Before we get to the interface for serialization, let's spend a moment understanding why Storm's tuples are dynamically typed.
+
+Adding static typing to tuple fields would add large amount of complexity to Storm's API. Hadoop, for example, statically types its keys and values but requires a huge amount of annotations on the part of the user. Hadoop's API is a burden to use and the "type safety" isn't worth it. Dynamic typing is simply easier to use.
+
+Further than that, it's not possible to statically type Storm's tuples in any reasonable way. Suppose a Bolt subscribes to multiple streams. The tuples from all those streams may have different types across the fields. When a Bolt receives a `Tuple` in `execute`, that tuple could have come from any stream and so could have any combination of types. There might be some reflection magic you can do to declare a different method for every tuple stream a bolt subscribes to, but Storm opts for the simpler, straightforward approach of dynamic typing.
+
+Finally, another reason for using dynamic typing is so Storm can be used in a straightforward manner from dynamically typed languages like Clojure and JRuby.
+
+### Custom serialization
+
+Let's dive into Storm's API for defining custom serializations. There are two steps you need to take as a user to create a custom serialization: implement the serializer, and register the serializer to Storm.
+
+#### Creating a serializer
+
+Custom serializers implement the [ISerialization](javadocs/backtype/storm/serialization/ISerialization.html) interface. Implementations specify how to serialize and deserialize types into a binary format.
+
+The interface looks like this:
+
+```java
+public interface ISerialization<T> {
+    public boolean accept(Class c);
+    public void serialize(T object, DataOutputStream stream) throws IOException;
+    public T deserialize(DataInputStream stream) throws IOException;
+}
+```
+
+Storm uses the `accept` method to determine if a type can be serialized by this serializer. Remember, Storm's tuples are dynamically typed so Storm determines what serializer to use at runtime.
+
+`serialize` writes the object out to the output stream in binary format. The field must be written in a way such that it can be deserialized later. For example, if you're writing out a list of objects, you'll need to write out the size of the list first so that you know how many elements to deserialize.
+
+`deserialize` reads the serialized object off of the stream and returns it.
+
+You can see example serialization implementations in the source for [SerializationFactory](https://github.com/apache/incubator-storm/blob/0.5.4/src/jvm/backtype/storm/serialization/SerializationFactory.java)
+
+#### Registering a serializer
+
+Once you create a serializer, you need to tell Storm it exists. This is done through the Storm configuration (See [Concepts](Concepts.html) for information about how configuration works in Storm). You can register serializations either through the config given when submitting a topology or in the storm.yaml files across your cluster.
+
+Serializer registrations are done through the Config.TOPOLOGY_SERIALIZATIONS config and is simply a list of serialization class names.
+
+Storm provides helpers for registering serializers in a topology config. The [Config](javadocs/backtype/storm/Config.html) class has a method called `addSerialization` that takes in a serializer class to add to the config.
+
+There's an advanced config called Config.TOPOLOGY_SKIP_MISSING_SERIALIZATIONS. If you set this to true, Storm will ignore any serializations that are registered but do not have their code available on the classpath. Otherwise, Storm will throw errors when it can't find a serialization. This is useful if you run many topologies on a cluster that each have different serializations, but you want to declare all the serializations across all topologies in the `storm.yaml` files.

http://git-wip-us.apache.org/repos/asf/storm/blob/cf0cdbb0/docs/Serialization.md
----------------------------------------------------------------------
diff --git a/docs/Serialization.md b/docs/Serialization.md
new file mode 100644
index 0000000..4c271b4
--- /dev/null
+++ b/docs/Serialization.md
@@ -0,0 +1,60 @@
+---
+layout: documentation
+---
+This page is about how the serialization system in Storm works for versions 0.6.0 and onwards. Storm used a different serialization system prior to 0.6.0 which is documented on [Serialization (prior to 0.6.0)](Serialization-\(prior-to-0.6.0\).html). 
+
+Tuples can be comprised of objects of any types. Since Storm is a distributed system, it needs to know how to serialize and deserialize objects when they're passed between tasks.
+
+Storm uses [Kryo](http://code.google.com/p/kryo/) for serialization. Kryo is a flexible and fast serialization library that produces small serializations.
+
+By default, Storm can serialize primitive types, strings, byte arrays, ArrayList, HashMap, HashSet, and the Clojure collection types. If you want to use another type in your tuples, you'll need to register a custom serializer.
+
+### Dynamic typing
+
+There are no type declarations for fields in a Tuple. You put objects in fields and Storm figures out the serialization dynamically. Before we get to the interface for serialization, let's spend a moment understanding why Storm's tuples are dynamically typed.
+
+Adding static typing to tuple fields would add large amount of complexity to Storm's API. Hadoop, for example, statically types its keys and values but requires a huge amount of annotations on the part of the user. Hadoop's API is a burden to use and the "type safety" isn't worth it. Dynamic typing is simply easier to use.
+
+Further than that, it's not possible to statically type Storm's tuples in any reasonable way. Suppose a Bolt subscribes to multiple streams. The tuples from all those streams may have different types across the fields. When a Bolt receives a `Tuple` in `execute`, that tuple could have come from any stream and so could have any combination of types. There might be some reflection magic you can do to declare a different method for every tuple stream a bolt subscribes to, but Storm opts for the simpler, straightforward approach of dynamic typing.
+
+Finally, another reason for using dynamic typing is so Storm can be used in a straightforward manner from dynamically typed languages like Clojure and JRuby.
+
+### Custom serialization
+
+As mentioned, Storm uses Kryo for serialization. To implement custom serializers, you need to register new serializers with Kryo. It's highly recommended that you read over [Kryo's home page](http://code.google.com/p/kryo/) to understand how it handles custom serialization.
+
+Adding custom serializers is done through the "topology.kryo.register" property in your topology config. It takes a list of registrations, where each registration can take one of two forms:
+
+1. The name of a class to register. In this case, Storm will use Kryo's `FieldsSerializer` to serialize the class. This may or may not be optimal for the class -- see the Kryo docs for more details.
+2. A map from the name of a class to register to an implementation of [com.esotericsoftware.kryo.Serializer](http://code.google.com/p/kryo/source/browse/trunk/src/com/esotericsoftware/kryo/Serializer.java).
+
+Let's look at an example.
+
+```
+topology.kryo.register:
+  - com.mycompany.CustomType1
+  - com.mycompany.CustomType2: com.mycompany.serializer.CustomType2Serializer
+  - com.mycompany.CustomType3
+```
+
+`com.mycompany.CustomType1` and `com.mycompany.CustomType3` will use the `FieldsSerializer`, whereas `com.mycompany.CustomType2` will use `com.mycompany.serializer.CustomType2Serializer` for serialization.
+
+Storm provides helpers for registering serializers in a topology config. The [Config](javadocs/backtype/storm/Config.html) class has a method called `registerSerialization` that takes in a registration to add to the config.
+
+There's an advanced config called `Config.TOPOLOGY_SKIP_MISSING_KRYO_REGISTRATIONS`. If you set this to true, Storm will ignore any serializations that are registered but do not have their code available on the classpath. Otherwise, Storm will throw errors when it can't find a serialization. This is useful if you run many topologies on a cluster that each have different serializations, but you want to declare all the serializations across all topologies in the `storm.yaml` files.
+
+### Java serialization
+
+If Storm encounters a type for which it doesn't have a serialization registered, it will use Java serialization if possible. If the object can't be serialized with Java serialization, then Storm will throw an error.
+
+Beware that Java serialization is extremely expensive, both in terms of CPU cost as well as the size of the serialized object. It is highly recommended that you register custom serializers when you put the topology in production. The Java serialization behavior is there so that it's easy to prototype new topologies.
+
+You can turn off the behavior to fall back on Java serialization by setting the `Config.TOPOLOGY_FALL_BACK_ON_JAVA_SERIALIZATION` config to false.
+
+### Component-specific serialization registrations
+
+Storm 0.7.0 lets you set component-specific configurations (read more about this at [Configuration](Configuration.html)). Of course, if one component defines a serialization that serialization will need to be available to other bolts -- otherwise they won't be able to receive messages from that component!
+
+When a topology is submitted, a single set of serializations is chosen to be used by all components in the topology for sending messages. This is done by merging the component-specific serializer registrations with the regular set of serialization registrations. If two components define serializers for the same class, one of the serializers is chosen arbitrarily.
+
+To force a serializer for a particular class if there's a conflict between two component-specific registrations, just define the serializer you want to use in the topology-specific configuration. The topology-specific configuration has precedence over component-specific configurations for serialization registrations.

http://git-wip-us.apache.org/repos/asf/storm/blob/cf0cdbb0/docs/Serializers.md
----------------------------------------------------------------------
diff --git a/docs/Serializers.md b/docs/Serializers.md
new file mode 100644
index 0000000..071c885
--- /dev/null
+++ b/docs/Serializers.md
@@ -0,0 +1,4 @@
+---
+layout: documentation
+---
+* [storm-json](https://github.com/rapportive-oss/storm-json): Simple JSON serializer for Storm

http://git-wip-us.apache.org/repos/asf/storm/blob/cf0cdbb0/docs/Setting-up-a-Storm-cluster.md
----------------------------------------------------------------------
diff --git a/docs/Setting-up-a-Storm-cluster.md b/docs/Setting-up-a-Storm-cluster.md
new file mode 100644
index 0000000..e139523
--- /dev/null
+++ b/docs/Setting-up-a-Storm-cluster.md
@@ -0,0 +1,83 @@
+---
+layout: documentation
+---
+This page outlines the steps for getting a Storm cluster up and running. If you're on AWS, you should check out the [storm-deploy](https://github.com/nathanmarz/storm-deploy/wiki) project. [storm-deploy](https://github.com/nathanmarz/storm-deploy/wiki) completely automates the provisioning, configuration, and installation of Storm clusters on EC2. It also sets up Ganglia for you so you can monitor CPU, disk, and network usage.
+
+If you run into difficulties with your Storm cluster, first check for a solution is in the [Troubleshooting](Troubleshooting.html) page. Otherwise, email the mailing list.
+
+Here's a summary of the steps for setting up a Storm cluster:
+
+1. Set up a Zookeeper cluster
+2. Install dependencies on Nimbus and worker machines
+3. Download and extract a Storm release to Nimbus and worker machines
+4. Fill in mandatory configurations into storm.yaml
+5. Launch daemons under supervision using "storm" script and a supervisor of your choice
+
+### Set up a Zookeeper cluster
+
+Storm uses Zookeeper for coordinating the cluster. Zookeeper **is not** used for message passing, so the load Storm places on Zookeeper is quite low. Single node Zookeeper clusters should be sufficient for most cases, but if you want failover or are deploying large Storm clusters you may want larger Zookeeper clusters. Instructions for deploying Zookeeper are [here](http://zookeeper.apache.org/doc/r3.3.3/zookeeperAdmin.html). 
+
+A few notes about Zookeeper deployment:
+
+1. It's critical that you run Zookeeper under supervision, since Zookeeper is fail-fast and will exit the process if it encounters any error case. See [here](http://zookeeper.apache.org/doc/r3.3.3/zookeeperAdmin.html#sc_supervision) for more details. 
+2. It's critical that you set up a cron to compact Zookeeper's data and transaction logs. The Zookeeper daemon does not do this on its own, and if you don't set up a cron, Zookeeper will quickly run out of disk space. See [here](http://zookeeper.apache.org/doc/r3.3.3/zookeeperAdmin.html#sc_maintenance) for more details.
+
+### Install dependencies on Nimbus and worker machines
+
+Next you need to install Storm's dependencies on Nimbus and the worker machines. These are:
+
+1. Java 6
+2. Python 2.6.6
+
+These are the versions of the dependencies that have been tested with Storm. Storm may or may not work with different versions of Java and/or Python.
+
+
+### Download and extract a Storm release to Nimbus and worker machines
+
+Next, download a Storm release and extract the zip file somewhere on Nimbus and each of the worker machines. The Storm releases can be downloaded [from here](http://github.com/apache/incubator-storm/downloads).
+
+### Fill in mandatory configurations into storm.yaml
+
+The Storm release contains a file at `conf/storm.yaml` that configures the Storm daemons. You can see the default configuration values [here](https://github.com/apache/incubator-storm/blob/master/conf/defaults.yaml). storm.yaml overrides anything in defaults.yaml. There's a few configurations that are mandatory to get a working cluster:
+
+1) **storm.zookeeper.servers**: This is a list of the hosts in the Zookeeper cluster for your Storm cluster. It should look something like:
+
+```yaml
+storm.zookeeper.servers:
+  - "111.222.333.444"
+  - "555.666.777.888"
+```
+
+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:
+
+```yaml
+storm.local.dir: "/mnt/storm"
+```
+
+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
+nimbus.host: "111.222.333.44"
+```
+
+4) **supervisor.slots.ports**: For each worker machine, you configure how many workers run on that machine with this config. Each worker uses a single port for receiving messages, and this setting defines which ports are open for use. If you define five ports here, then Storm will allocate up to five workers to run on this machine. If you define three ports, Storm will only run up to three. By default, this setting is configured to run 4 workers on the ports 6700, 6701, 6702, and 6703. For example:
+
+```yaml
+supervisor.slots.ports:
+    - 6700
+    - 6701
+    - 6702
+    - 6703
+```
+
+### Launch daemons under supervision using "storm" script and a supervisor of your choice
+
+The last step is to launch all the Storm daemons. It is critical that you run each of these daemons under supervision. Storm is a __fail-fast__ system which means the processes will halt whenever an unexpected error is encountered. Storm is designed so that it can safely halt at any point and recover correctly when the process is restarted. This is why Storm keeps no state in-process -- if Nimbus or the Supervisors restart, the running topologies are unaffected. Here's how to run the Storm daemons:
+
+1. **Nimbus**: Run the command "bin/storm nimbus" under supervision on the master machine.
+2. **Supervisor**: Run the command "bin/storm supervisor" under supervision on each worker machine. The supervisor daemon is responsible for starting and stopping worker processes on that machine.
+3. **UI**: Run the Storm UI (a site you can access from the browser that gives diagnostics on the cluster and topologies) by running the command "bin/storm ui" under supervision. The UI can be accessed by navigating your web browser to http://{nimbus host}:8080. 
+
+As you can see, running the daemons is very straightforward. The daemons will log to the logs/ directory in wherever you extracted the Storm release.

http://git-wip-us.apache.org/repos/asf/storm/blob/cf0cdbb0/docs/Setting-up-a-Storm-project-in-Eclipse.md
----------------------------------------------------------------------
diff --git a/docs/Setting-up-a-Storm-project-in-Eclipse.md b/docs/Setting-up-a-Storm-project-in-Eclipse.md
new file mode 100644
index 0000000..5137cd9
--- /dev/null
+++ b/docs/Setting-up-a-Storm-project-in-Eclipse.md
@@ -0,0 +1 @@
+- fill me in
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/cf0cdbb0/docs/Setting-up-development-environment.md
----------------------------------------------------------------------
diff --git a/docs/Setting-up-development-environment.md b/docs/Setting-up-development-environment.md
new file mode 100644
index 0000000..07ba670
--- /dev/null
+++ b/docs/Setting-up-development-environment.md
@@ -0,0 +1,39 @@
+---
+layout: documentation
+---
+This page outlines what you need to do to get a Storm development environment set up. In summary, the steps are:
+
+1. Download a [Storm release](/releases.html) , unpack it, and put the unpacked `bin/` directory on your PATH
+2. To be able to start and stop topologies on a remote cluster, put the cluster information in `~/.storm/storm.yaml`
+
+More detail on each of these steps is below.
+
+### What is a development environment?
+
+Storm has two modes of operation: local mode and remote mode. In local mode, you can develop and test topologies completely in process on your local machine. In remote mode, you submit topologies for execution on a cluster of machines.
+
+A Storm development environment has everything installed so that you can develop and test Storm topologies in local mode, package topologies for execution on a remote cluster, and submit/kill topologies on a remote cluster.
+
+Let's quickly go over the relationship between your machine and a remote cluster. A Storm cluster is managed by a master node called "Nimbus". Your machine communicates with Nimbus to submit code (packaged as a jar) and topologies for execution on the cluster, and Nimbus will take care of distributing that code around the cluster and assigning workers to run your topology. Your machine uses a command line client called `storm` to communicate with Nimbus. The `storm` client is only used for remote mode; it is not used for developing and testing topologies in local mode.
+
+### Installing a Storm release locally
+
+If you want to be able to submit topologies to a remote cluster from your machine, you should install a Storm release locally. Installing a Storm release will give you the `storm` client that you can use to interact with remote clusters. To install Storm locally, download a release [from here](/releases.html) and unzip it somewhere on your computer. Then add the unpacked `bin/` directory onto your `PATH` and make sure the `bin/storm` script is executable.
+
+Installing a Storm release locally is only for interacting with remote clusters. For developing and testing topologies in local mode, it is recommended that you use Maven to include Storm as a dev dependency for your project. You can read more about using Maven for this purpose on [Maven](Maven.html). 
+
+### Starting and stopping topologies on a remote cluster
+
+The previous step installed the `storm` client on your machine which is used to communicate with remote Storm clusters. Now all you have to do is tell the client which Storm cluster to talk to. To do this, all you have to do is put the host address of the master in the `~/.storm/storm.yaml` file. It should look something like this:
+
+```
+nimbus.host: "123.45.678.890"
+```
+
+Alternatively, if you use the [storm-deploy](https://github.com/nathanmarz/storm-deploy) project to provision Storm clusters on AWS, it will automatically set up your ~/.storm/storm.yaml file. You can manually attach to a Storm cluster (or switch between multiple clusters) using the "attach" command, like so:
+
+```
+lein run :deploy --attach --name mystormcluster
+```
+
+More information is on the storm-deploy [wiki](https://github.com/nathanmarz/storm-deploy/wiki)

http://git-wip-us.apache.org/repos/asf/storm/blob/cf0cdbb0/docs/Spout-implementations.md
----------------------------------------------------------------------
diff --git a/docs/Spout-implementations.md b/docs/Spout-implementations.md
new file mode 100644
index 0000000..10ddd42
--- /dev/null
+++ b/docs/Spout-implementations.md
@@ -0,0 +1,8 @@
+---
+layout: documentation
+---
+* [storm-kestrel](https://github.com/nathanmarz/storm-kestrel): Adapter to use Kestrel as a spout
+* [storm-amqp-spout](https://github.com/rapportive-oss/storm-amqp-spout): Adapter to use AMQP source as a spout
+* [storm-jms](https://github.com/ptgoetz/storm-jms): Adapter to use a JMS source as a spout
+* [storm-redis-pubsub](https://github.com/sorenmacbeth/storm-redis-pubsub): A spout that subscribes to a Redis pubsub stream
+* [storm-beanstalkd-spout](https://github.com/haitaoyao/storm-beanstalkd-spout): A spout that subscribes to a beanstalkd queue

http://git-wip-us.apache.org/repos/asf/storm/blob/cf0cdbb0/docs/Storm-multi-language-protocol-(versions-0.7.0-and-below).md
----------------------------------------------------------------------
diff --git a/docs/Storm-multi-language-protocol-(versions-0.7.0-and-below).md b/docs/Storm-multi-language-protocol-(versions-0.7.0-and-below).md
new file mode 100644
index 0000000..1d4422f
--- /dev/null
+++ b/docs/Storm-multi-language-protocol-(versions-0.7.0-and-below).md
@@ -0,0 +1,122 @@
+---
+layout: documentation
+---
+This page explains the multilang protocol for versions 0.7.0 and below. The protocol changed in version 0.7.1.
+
+# Storm Multi-Language Protocol
+
+## The ShellBolt
+
+Support for multiple languages is implemented via the ShellBolt class.  This
+class implements the IBolt interfaces and implements the protocol for
+executing a script or program via the shell using Java's ProcessBuilder class.
+
+## Output fields
+
+Output fields are part of the Thrift definition of the topology. This means that when you multilang in Java, you need to create a bolt that extends ShellBolt, implements IRichBolt, and declared the fields in `declareOutputFields`. 
+You can learn more about this on [Concepts](Concepts.html)
+
+## Protocol Preamble
+
+A simple protocol is implemented via the STDIN and STDOUT of the executed
+script or program. A mix of simple strings and JSON encoded data are exchanged
+with the process making support possible for pretty much any language.
+
+# Packaging Your Stuff
+
+To run a ShellBolt on a cluster, the scripts that are shelled out to must be
+in the `resources/` directory within the jar submitted to the master.
+
+However, During development or testing on a local machine, the resources
+directory just needs to be on the classpath.
+
+## The Protocol
+
+Notes:
+* Both ends of this protocol use a line-reading mechanism, so be sure to
+trim off newlines from the input and to append them to your output.
+* All JSON inputs and outputs are terminated by a single line contained "end".
+* The bullet points below are written from the perspective of the script writer's
+STDIN and STDOUT.
+
+
+* Your script will be executed by the Bolt.
+* STDIN: A string representing a path. This is a PID directory.
+Your script should create an empty file named with it's pid in this directory. e.g.
+the PID is 1234, so an empty file named 1234 is created in the directory. This
+file lets the supervisor know the PID so it can shutdown the process later on.
+* STDOUT: Your PID. This is not JSON encoded, just a string. ShellBolt will log the PID to its log.
+* STDIN: (JSON) The Storm configuration.  Various settings and properties.
+* STDIN: (JSON) The Topology context
+* The rest happens in a while(true) loop
+* STDIN: A tuple! This is a JSON encoded structure like this:
+
+```
+{
+    // The tuple's id
+	"id": -6955786537413359385,
+	// The id of the component that created this tuple
+	"comp": 1,
+	// The id of the stream this tuple was emitted to
+	"stream": 1,
+	// The id of the task that created this tuple
+	"task": 9,
+	// All the values in this tuple
+	"tuple": ["snow white and the seven dwarfs", "field2", 3]
+}
+```
+
+* STDOUT: The results of your bolt, JSON encoded. This can be a sequence of acks, fails, emits, and/or logs. Emits look like:
+
+```
+{
+	"command": "emit",
+	// The ids of the tuples this output tuples should be anchored to
+	"anchors": [1231231, -234234234],
+	// The id of the stream this tuple was emitted to. Leave this empty to emit to default stream.
+	"stream": 1,
+	// If doing an emit direct, indicate the task to sent the tuple to
+	"task": 9,
+	// All the values in this tuple
+	"tuple": ["field1", 2, 3]
+}
+```
+
+An ack looks like:
+
+```
+{
+	"command": "ack",
+	// the id of the tuple to ack
+	"id": 123123
+}
+```
+
+A fail looks like:
+
+```
+{
+	"command": "fail",
+	// the id of the tuple to fail
+	"id": 123123
+}
+```
+
+A "log" will log a message in the worker log. It looks like:
+
+```
+{
+	"command": "log",
+	// the message to log
+	"msg": "hello world!"
+
+}
+```
+
+* STDOUT: emit "sync" as a single line by itself when the bolt has finished emitting/acking/failing and is ready for the next input
+
+### sync
+
+Note: This command is not JSON encoded, it is sent as a simple string.
+
+This lets the parent bolt know that the script has finished processing and is ready for another tuple.

http://git-wip-us.apache.org/repos/asf/storm/blob/cf0cdbb0/docs/Structure-of-the-codebase.md
----------------------------------------------------------------------
diff --git a/docs/Structure-of-the-codebase.md b/docs/Structure-of-the-codebase.md
new file mode 100644
index 0000000..8ac66f4
--- /dev/null
+++ b/docs/Structure-of-the-codebase.md
@@ -0,0 +1,140 @@
+---
+layout: documentation
+---
+There are three distinct layers to Storm's codebase.
+
+First, Storm was designed from the very beginning to be compatible with multiple languages. Nimbus is a Thrift service and topologies are defined as Thrift structures. The usage of Thrift allows Storm to be used from any language.
+
+Second, all of Storm's interfaces are specified as Java interfaces. So even though there's a lot of Clojure in Storm's implementation, all usage must go through the Java API. This means that every feature of Storm is always available via Java.
+
+Third, Storm's implementation is largely in Clojure. Line-wise, Storm is about half Java code, half Clojure code. But Clojure is much more expressive, so in reality the great majority of the implementation logic is in Clojure. 
+
+The following sections explain each of these layers in more detail.
+
+### storm.thrift
+
+The first place to look to understand the structure of Storm's codebase is the [storm.thrift](https://github.com/apache/incubator-storm/blob/master/storm-core/src/storm.thrift) file.
+
+Storm uses [this fork](https://github.com/nathanmarz/thrift/tree/storm) of Thrift (branch 'storm') to produce the generated code. This "fork" is actually Thrift 7 with all the Java packages renamed to be `org.apache.thrift7`. Otherwise, it's identical to Thrift 7. This fork was done because of the lack of backwards compatibility in Thrift and the need for many people to use other versions of Thrift in their Storm topologies.
+
+Every spout or bolt in a topology is given a user-specified identifier called the "component id". The component id is used to specify subscriptions from a bolt to the output streams of other spouts or bolts. A [StormTopology](https://github.com/apache/incubator-storm/blob/master/storm-core/src/storm.thrift#L91) structure contains a map from component id to component for each type of component (spouts and bolts).
+
+Spouts and bolts have the same Thrift definition, so let's just take a look at the [Thrift definition for bolts](https://github.com/apache/incubator-storm/blob/master/storm-core/src/storm.thrift#L79). It contains a `ComponentObject` struct and a `ComponentCommon` struct.
+
+The `ComponentObject` defines the implementation for the bolt. It can be one of three types:
+
+1. A serialized java object (that implements [IBolt](https://github.com/apache/incubator-storm/blob/master/storm-core/src/jvm/backtype/storm/task/IBolt.java))
+2. A `ShellComponent` object that indicates the implementation is in another language. Specifying a bolt this way will cause Storm to instantiate a [ShellBolt](https://github.com/apache/incubator-storm/blob/master/storm-core/src/jvm/backtype/storm/task/ShellBolt.java) object to handle the communication between the JVM-based worker process and the non-JVM-based implementation of the component.
+3. A `JavaObject` structure which tells Storm the classname and constructor arguments to use to instantiate that bolt. This is useful if you want to define a topology in a non-JVM language. This way, you can make use of JVM-based spouts and bolts without having to create and serialize a Java object yourself.
+
+`ComponentCommon` defines everything else for this component. This includes:
+
+1. What streams this component emits and the metadata for each stream (whether it's a direct stream, the fields declaration)
+2. What streams this component consumes (specified as a map from component_id:stream_id to the stream grouping to use)
+3. The parallelism for this component
+4. The component-specific [configuration](https://github.com/apache/incubator-storm/wiki/Configuration) for this component
+
+Note that the structure spouts also have a `ComponentCommon` field, and so spouts can also have declarations to consume other input streams. Yet the Storm Java API does not provide a way for spouts to consume other streams, and if you put any input declarations there for a spout you would get an error when you tried to submit the topology. The reason that spouts have an input declarations field is not for users to use, but for Storm itself to use. Storm adds implicit streams and bolts to the topology to set up the [acking framework](https://github.com/apache/incubator-storm/wiki/Guaranteeing-message-processing), and two of these implicit streams are from the acker bolt to each spout in the topology. The acker sends "ack" or "fail" messages along these streams whenever a tuple tree is detected to be completed or failed. The code that transforms the user's topology into the runtime topology is located [here](https://github.com/apache/incubator-storm/blob/master/storm-core/src/clj/back
 type/storm/daemon/common.clj#L279).
+
+### Java interfaces
+
+The interfaces for Storm are generally specified as Java interfaces. The main interfaces are:
+
+1. [IRichBolt](javadocs/backtype/storm/topology/IRichBolt.html)
+2. [IRichSpout](javadocs/backtype/storm/topology/IRichSpout.html)
+3. [TopologyBuilder](javadocs/backtype/storm/topology/TopologyBuilder.html)
+
+The strategy for the majority of the interfaces is to:
+
+1. Specify the interface using a Java interface
+2. Provide a base class that provides default implementations when appropriate
+
+You can see this strategy at work with the [BaseRichSpout](javadocs/backtype/storm/topology/base/BaseRichSpout.html) class. 
+
+Spouts and bolts are serialized into the Thrift definition of the topology as described above. 
+
+One subtle aspect of the interfaces is the difference between `IBolt` and `ISpout` vs. `IRichBolt` and `IRichSpout`. The main difference between them is the addition of the `declareOutputFields` method in the "Rich" versions of the interfaces. The reason for the split is that the output fields declaration for each output stream needs to be part of the Thrift struct (so it can be specified from any language), but as a user you want to be able to declare the streams as part of your class. What `TopologyBuilder` does when constructing the Thrift representation is call `declareOutputFields` to get the declaration and convert it into the Thrift structure. The conversion happens [at this portion](https://github.com/apache/incubator-storm/blob/master/storm-core/src/jvm/backtype/storm/topology/TopologyBuilder.java#L205) of the `TopologyBuilder` code. 
+
+
+### Implementation
+
+Specifying all the functionality via Java interfaces ensures that every feature of Storm is available via Java. Moreso, the focus on Java interfaces ensures that the user experience from Java-land is pleasant as well.
+
+The implementation of Storm, on the other hand, is primarily in Clojure. While the codebase is about 50% Java and 50% Clojure in terms of LOC, most of the implementation logic is in Clojure. There are two notable exceptions to this, and that is the [DRPC](https://github.com/apache/incubator-storm/wiki/Distributed-RPC) and [transactional topologies](https://github.com/apache/incubator-storm/wiki/Transactional-topologies) implementations. These are implemented purely in Java. This was done to serve as an illustration for how to implement a higher level abstraction on Storm. The DRPC and transactional topologies implementations are in the [backtype.storm.coordination](https://github.com/apache/incubator-storm/tree/master/storm-core/src/jvm/backtype/storm/coordination), [backtype.storm.drpc](https://github.com/apache/incubator-storm/tree/master/storm-core/src/jvm/backtype/storm/drpc), and [backtype.storm.transactional](https://github.com/apache/incubator-storm/tree/master/storm-core/src
 /jvm/backtype/storm/transactional) packages.
+
+Here's a summary of the purpose of the main Java packages and Clojure namespace:
+
+#### Java packages
+
+[backtype.storm.coordination](https://github.com/apache/incubator-storm/tree/master/storm-core/src/jvm/backtype/storm/coordination): Implements the pieces required to coordinate batch-processing on top of Storm, which both DRPC and transactional topologies use. `CoordinatedBolt` is the most important class here.
+
+[backtype.storm.drpc](https://github.com/apache/incubator-storm/tree/master/storm-core/src/jvm/backtype/storm/drpc): Implementation of the DRPC higher level abstraction
+
+[backtype.storm.generated](https://github.com/apache/incubator-storm/tree/master/storm-core/src/jvm/backtype/storm/generated): The generated Thrift code for Storm (generated using [this fork](https://github.com/nathanmarz/thrift) of Thrift, which simply renames the packages to org.apache.thrift7 to avoid conflicts with other Thrift versions)
+
+[backtype.storm.grouping](https://github.com/apache/incubator-storm/tree/master/storm-core/src/jvm/backtype/storm/grouping): Contains interface for making custom stream groupings
+
+[backtype.storm.hooks](https://github.com/apache/incubator-storm/tree/master/storm-core/src/jvm/backtype/storm/hooks): Interfaces for hooking into various events in Storm, such as when tasks emit tuples, when tuples are acked, etc. User guide for hooks is [here](https://github.com/apache/incubator-storm/wiki/Hooks).
+
+[backtype.storm.serialization](https://github.com/apache/incubator-storm/tree/master/storm-core/src/jvm/backtype/storm/serialization): Implementation of how Storm serializes/deserializes tuples. Built on top of [Kryo](http://code.google.com/p/kryo/).
+
+[backtype.storm.spout](https://github.com/apache/incubator-storm/tree/master/storm-core/src/jvm/backtype/storm/spout): Definition of spout and associated interfaces (like the `SpoutOutputCollector`). Also contains `ShellSpout` which implements the protocol for defining spouts in non-JVM languages.
+
+[backtype.storm.task](https://github.com/apache/incubator-storm/tree/master/storm-core/src/jvm/backtype/storm/task): Definition of bolt and associated interfaces (like `OutputCollector`). Also contains `ShellBolt` which implements the protocol for defining bolts in non-JVM languages. Finally, `TopologyContext` is defined here as well, which is provided to spouts and bolts so they can get data about the topology and its execution at runtime.
+
+[backtype.storm.testing](https://github.com/apache/incubator-storm/tree/master/storm-core/src/jvm/backtype/storm/testing): Contains a variety of test bolts and utilities used in Storm's unit tests.
+
+[backtype.storm.topology](https://github.com/apache/incubator-storm/tree/master/storm-core/src/jvm/backtype/storm/topology): Java layer over the underlying Thrift structure to provide a clean, pure-Java API to Storm (users don't have to know about Thrift). `TopologyBuilder` is here as well as the helpful base classes for the different spouts and bolts. The slightly-higher level `IBasicBolt` interface is here, which is a simpler way to write certain kinds of bolts.
+
+[backtype.storm.transactional](https://github.com/apache/incubator-storm/tree/master/storm-core/src/jvm/backtype/storm/transactional): Implementation of transactional topologies.
+
+[backtype.storm.tuple](https://github.com/apache/incubator-storm/tree/master/storm-core/src/jvm/backtype/storm/tuple): Implementation of Storm's tuple data model.
+
+[backtype.storm.utils](https://github.com/apache/incubator-storm/tree/master/storm-core/src/jvm/backtype/storm/tuple): Data structures and miscellaneous utilities used throughout the codebase.
+
+
+#### Clojure namespaces
+
+[backtype.storm.bootstrap](https://github.com/apache/incubator-storm/blob/master/storm-core/src/clj/backtype/storm/bootstrap.clj): Contains a helpful macro to import all the classes and namespaces that are used throughout the codebase.
+
+[backtype.storm.clojure](https://github.com/apache/incubator-storm/blob/master/storm-core/src/clj/backtype/storm/clojure.clj): Implementation of the Clojure DSL for Storm.
+
+[backtype.storm.cluster](https://github.com/apache/incubator-storm/blob/master/storm-core/src/clj/backtype/storm/cluster.clj): All Zookeeper logic used in Storm daemons is encapsulated in this file. This code manages how cluster state (like what tasks are running where, what spout/bolt each task runs as) is mapped to the Zookeeper "filesystem" API.
+
+[backtype.storm.command.*](https://github.com/apache/incubator-storm/blob/master/storm-core/src/clj/backtype/storm/command): These namespaces implement various commands for the `storm` command line client. These implementations are very short.
+
+[backtype.storm.config](https://github.com/apache/incubator-storm/blob/master/storm-core/src/clj/backtype/storm/config.clj): Implementation of config reading/parsing code for Clojure. Also has utility functions for determining what local path nimbus/supervisor/daemons should be using for various things. e.g. the `master-inbox` function will return the local path that Nimbus should use when jars are uploaded to it.
+
+[backtype.storm.daemon.acker](https://github.com/apache/incubator-storm/blob/master/storm-core/src/clj/backtype/storm/daemon/acker.clj): Implementation of the "acker" bolt, which is a key part of how Storm guarantees data processing.
+
+[backtype.storm.daemon.common](https://github.com/apache/incubator-storm/blob/master/storm-core/src/clj/backtype/storm/daemon/common.clj): Implementation of common functions used in Storm daemons, like getting the id for a topology based on the name, mapping a user's topology into the one that actually executes (with implicit acking streams and acker bolt added - see `system-topology!` function), and definitions for the various heartbeat and other structures persisted by Storm.
+
+[backtype.storm.daemon.drpc](https://github.com/apache/incubator-storm/blob/master/storm-core/src/clj/backtype/storm/daemon/drpc.clj): Implementation of the DRPC server for use with DRPC topologies.
+
+[backtype.storm.daemon.nimbus](https://github.com/apache/incubator-storm/blob/master/storm-core/src/clj/backtype/storm/daemon/nimbus.clj): Implementation of Nimbus.
+
+[backtype.storm.daemon.supervisor](https://github.com/apache/incubator-storm/blob/master/storm-core/src/clj/backtype/storm/daemon/supervisor.clj): Implementation of Supervisor.
+
+[backtype.storm.daemon.task](https://github.com/apache/incubator-storm/blob/master/storm-core/src/clj/backtype/storm/daemon/task.clj): Implementation of an individual task for a spout or bolt. Handles message routing, serialization, stats collection for the UI, as well as the spout-specific and bolt-specific execution implementations.
+
+[backtype.storm.daemon.worker](https://github.com/apache/incubator-storm/blob/master/storm-core/src/clj/backtype/storm/daemon/worker.clj): Implementation of a worker process (which will contain many tasks within). Implements message transferring and task launching.
+
+[backtype.storm.event](https://github.com/apache/incubator-storm/blob/master/storm-core/src/clj/backtype/storm/event.clj): Implements a simple asynchronous function executor. Used in various places in Nimbus and Supervisor to make functions execute in serial to avoid any race conditions.
+
+[backtype.storm.log](https://github.com/apache/incubator-storm/blob/master/storm-core/src/clj/backtype/storm/log.clj): Defines the functions used to log messages to log4j.
+
+[backtype.storm.messaging.*](https://github.com/apache/incubator-storm/blob/master/storm-core/src/clj/backtype/storm/messaging): Defines a higher level interface to implementing point to point messaging. In local mode Storm uses in-memory Java queues to do this; on a cluster, it uses ZeroMQ. The generic interface is defined in protocol.clj.
+
+[backtype.storm.stats](https://github.com/apache/incubator-storm/blob/master/storm-core/src/clj/backtype/storm/stats.clj): Implementation of stats rollup routines used when sending stats to ZK for use by the UI. Does things like windowed and rolling aggregations at multiple granularities.
+
+[backtype.storm.testing](https://github.com/apache/incubator-storm/blob/master/storm-core/src/clj/backtype/storm/testing.clj): Implementation of facilities used to test Storm topologies. Includes time simulation, `complete-topology` for running a fixed set of tuples through a topology and capturing the output, tracker topologies for having fine grained control over detecting when a cluster is "idle", and other utilities.
+
+[backtype.storm.thrift](https://github.com/apache/incubator-storm/blob/master/storm-core/src/clj/backtype/storm/thrift.clj): Clojure wrappers around the generated Thrift API to make working with Thrift structures more pleasant.
+
+[backtype.storm.timer](https://github.com/apache/incubator-storm/blob/master/storm-core/src/clj/backtype/storm/timer.clj): Implementation of a background timer to execute functions in the future or on a recurring interval. Storm couldn't use the [Timer](http://docs.oracle.com/javase/1.4.2/docs/api/java/util/Timer.html) class because it needed integration with time simulation in order to be able to unit test Nimbus and the Supervisor.
+
+[backtype.storm.ui.*](https://github.com/apache/incubator-storm/blob/master/storm-core/src/clj/backtype/storm/ui): Implementation of Storm UI. Completely independent from rest of code base and uses the Nimbus Thrift API to get data.
+
+[backtype.storm.util](https://github.com/apache/incubator-storm/blob/master/storm-core/src/clj/backtype/storm/util.clj): Contains generic utility functions used throughout the code base.
+ 
+[backtype.storm.zookeeper](https://github.com/apache/incubator-storm/blob/master/storm-core/src/clj/backtype/storm/zookeeper.clj): Clojure wrapper around the Zookeeper API and implements some "high-level" stuff like "mkdirs" and "delete-recursive".

http://git-wip-us.apache.org/repos/asf/storm/blob/cf0cdbb0/docs/Support-for-non-java-languages.md
----------------------------------------------------------------------
diff --git a/docs/Support-for-non-java-languages.md b/docs/Support-for-non-java-languages.md
new file mode 100644
index 0000000..724d106
--- /dev/null
+++ b/docs/Support-for-non-java-languages.md
@@ -0,0 +1,7 @@
+---
+layout: documentation
+---
+* [Scala DSL](https://github.com/velvia/ScalaStorm)
+* [JRuby DSL](https://github.com/colinsurprenant/storm-jruby)
+* [Clojure DSL](Clojure-DSL.html)
+* [io-storm](https://github.com/gphat/io-storm): Perl multilang adapter