http://git-wip-us.apache.org/repos/asf/helix/blob/e8125e8b/website/0.8.3/src/site/markdown/design_crushed.md
----------------------------------------------------------------------
diff --git a/website/0.8.3/src/site/markdown/design_crushed.md 
b/website/0.8.3/src/site/markdown/design_crushed.md
new file mode 100644
index 0000000..db755f6
--- /dev/null
+++ b/website/0.8.3/src/site/markdown/design_crushed.md
@@ -0,0 +1,173 @@
+CrushED (Crush-based rebalancer with Even Distribution)
+============================================
+
+## Overview
+
+Helix provides AutoRebalanceStrategy which is based on card dealing strategy. 
This strategy takes the current mapping as an input, and computes new mappings 
only for the partitions that need to be moved. This provides minimum partition 
movement, but the mapping is not deterministic, and moreover, fault-zone aware 
mapping (i.e. rack-aware partitioning) is not possible. 
+
+CRUSH-based partitioning scheme was implemented to provide fault-zone aware 
mapping and deterministic partition assignment. CrushRebalanceStrategy (and 
MultiRoundCrushRebalanceStrategy) algorithm uses pseudo-random partition 
placement to ensure consistent partition distribution. As the number of placed 
items (i.e partitions) approaches infinity, the distribution will be perfectly 
uniform. However, with a small number of placed items, especially for resources 
(i.e. databases) with a small number of partitions, the placement algorithm may 
result in fairly  uneven partition distribution.  
+
+We want to provide a new rebalance strategy that provides a deterministic and 
fault-zone aware mapping while providing even partition distribution in all 
cases. In this document, we propose a hybrid algorithm that uses CRUSH, card 
dealing strategy, and consistent hashing to ensure both even distribution and 
minimal partition movement (while cluster topology remains the same). We call 
it CrushED (Crush w/ Even Distribution). Compared to CRUSH, CrushED results in 
a much more uniform distribution and minimal partition movements as long as 
topology remains the same, at the cost of additional run time computation.  
+
+## Design
+
+In addition to what we already achieved in CrushRebalanceStrategy, we have 2 
high level goals :
+
+1.  Even distribution.
+2.  Minimize partition movements when instances go up/down.
+
+CrushRebalanceStrategy has very small movement count, but the distribution is 
not optimal. MultiRoundCrushRebalanceStrategy was designed to solve this 
problem by running CRUSH multiple times on partition assignments that 
contribute to uneven mapping. However, due to potentially high number of 
rounds, computation cost is high, we observed significantly more partition 
movements when the cluster topology is changed.
+
+Since we have a good base strategy, CrushRebalanceStrategy, we built 
CrushEDRebalanceStrategy on top of it. Sample mapping of both strategies are as 
following. Note that blue parts remain unchanged before and after.
+
+Before (CRUSH)
+
+![Before (CRUSH)](images/design/crushed/before-using-crush.png)
+
+After (new strategy)
+
+![After (new strategy)](images/design/crushed/after-using-crushed.png)
+
+Since the problem is NP-hard. We are not expecting the best assignment. A 
greedy algorithm works good enough.  
+After we tried different designs, we found it's hard to achieve both goals 
(even distribution and fewer movements) using a single strategy. So we decided 
to apply a hybrid algorithm that finishes the work step by step.
+
+**Step 1, run CRUSH to get a base assignment.**  
+The base assignment usually contains a certain number of uneven 
partitions(i.e. extra partitions above perfect distribution), so we need the 
following steps to re-distribute them.
+
+**Step 2, run a card dealing algorithm on the uneven parts.**  
+Assign extra partitions to under-loaded nodes, using card dealing strategy. 
This algorithm is conceptually simple. The result ensures that all partitions 
are assigned to instances with minimum difference. When gauranteeing fault-zone 
safe assignment, our greedy algorithm may not be able to calculate possible 
results because of fault-zone conflict. 
+
+Example of assignments after step 2,
+
+![Example](images/design/crushed/example-cluster-partition-dist.png)
+
+**Step 3, Shuffle partitions' preference lists.**  
+State assignments (i.e. Master, Slave, Online, Offline, etc)  are made 
according to preflist, ordered node. When using randomly ordered lists, State 
assignment is also random, and it may result in uneven state distribution. To 
resolve this issue, CrushED assigns scores to nodes as it computes pref list, 
to give all nodes equal chances in appearing at the top of the pref list. This 
operation results in a much more even state distribution.
+
+Example of master distribution before step 3,
+
+![Example](images/design/crushed/example-cluster-master-dist.png)
+
+Example of master distribution after step 3,
+
+![Example](images/design/crushed/example-cluster-master-dist-after.png)
+
+**Step 4, re-calculate the assignment for the partitions on temporarily 
disabled nodes using a consistent hashing algorithm.**  
+Consistent hashing ensures minimize partition movement.  
+Note that the first 3 steps are using full node list, regardless of disabled 
or offline nodes. So the assignment will be stable even the algorithm contains 
random factors such hashCode. Then step 4 ensures all the disabled nodes are 
handled correctly without causing huge partition movements.
+
+Pseudocode of above algorithm is as follows :
+
+**Pseudo Code** 
+
+    // Round 1: Calculate mapping using the base strategy.
+    // Note to use all nodes for minimizing the influence of live node changes.
+    origPartitionMap = 
getBaseRebalanceStrategy().computePartitionAssignment(allNodes, clusterData);
+    
+    // Transform current assignment to instance->partitions map, and get total 
partitions
+    nodeToPartitionMap = convertMap(origPartitionMap);
+
+    // Round 2: Rebalance mapping using card dealing algorithm.
+    Topology allNodeTopo = new Topology(allNodes, clusterData);
+    cardDealer.computeMapping(allNodeTopo, nodeToPartitionMap);
+
+    // Since states are assigned according to preference list order, shuffle 
preference list for even states distribution.
+    shufflePreferenceList(nodeToPartitionMap);
+
+    // Round 3: Re-mapping the partitions on non-live nodes using consistent 
hashing for reducing movement.
+    // Consistent hashing ensures minimum movements when nodes are disabled 
unexpectedly.
+    if (!liveNodes.containsAll(allNodes)) {
+      Topology liveNodeTopo = new Topology(liveNodes, clusterData);
+      hashPlacement.computeMapping(liveNodeTopo, nodeToPartitionMap);
+    }
+
+    if (!nodeToPartitionMap.isEmpty()) {
+      // Round 2 and 3 is done successfully
+      return convertMap(nodeToPartitionMap);
+    } else {
+      return getBaseRebalanceStrategy().computePartitionAssignment(liveNodes, 
clusterData);
+    }
+
+
+### Maximum uneven partition assignment using CrushED
+
+Helix cluster typically manages 1 or more resources (i.e. databases). For each 
resource, CrushED makes the best effort to ensure the partition count 
difference is at most 1 across all the instances. Assuming such assignment is 
possible considering fault-zone configuration, the worst partition distribution 
happens when all one off partitions are located in one node. So N resources in 
a cluster can theoretically have their extra partitions in one node, so the 
node will have N additional partitions in total. Thus, the maximum difference 
between the most heavily loaded node and the least is **the number of 
resources** in a cluster.
+
+## Experiment
+
+We tested CrushED by simulating real production cluster topology data. And we 
tested multiple scenarios:
+
+*   Distribution based on cluster topology.
+*   Disabling hosts to simulate hosts down.
+*   Adding hosts to simulate expansion.
+*   Rolling upgrade.
+
+All results show that CrushED generates more uniform global distribution 
compared with CRUSH.  
+Moreover, partition movements in most scenarios are minimized. When topology 
changes (i.e. cluster expansion), there can be significantly more partition 
movements, but we can control the impact by using State Transition Throttling 
feature. 
+
+### Partition Distribution
+
+Following charts demonstrate the worst cases (min load vs. max load) and 
STDEVs of partition/master distributions from some sample clusters data.  
+If we measure the improvement by STDEV, CrushED improves the partition 
distribution evenness by 87% on average compared with CRUSH. And for state 
assignment (i.e. Mastership assignment) the evenness improvement is 68% on 
average.
+
+![Example](images/design/crushed/cursh-partition-dist.png)![Example](images/design/crushed/crushed-partition-dist.png)
+
+![Example](images/design/crushed/cursh-master-dist.png)![Example](images/design/crushed/crushed-master-dist.png)
+
+### Disabling Nodes
+
+When nodes are offline or disabled, CrushED will re-assign the partitions to 
other live nodes. The algorithm move only the necessary partitions.  
+We simulated disabling nodes, and measured partition movement changes and 
mastership changes. We also used the expected movement (the partitions/masters 
count on the disabled nodes) as a baseline to measure extra movements.
+
+The results show that movement is highly correlated to the number of disabled 
nodes, and extra movements are minor (in most cases 0 movements).
+
+Note that **Rate** in this document is **the changed number / total partition 
or master count**.
+
+![Example](images/design/crushed/node-down-partition-move.png)![Example](images/design/crushed/node-down-master-move.png)
+
+### Rolling upgrade
+
+Rolling upgrade is different from disabling nodes. Since nodes are reset one 
by one, in this test we assume the difference could be 2 nodes in maximum (for 
example, upgrading Node A then upgrading Node B).  
+In this case, movements are still minimized. Even in the worst case scenario, 
extra partition movements and mastership changes are still close to 0%.
+
+Note that in real production clusters, we can completely avoid partition 
movements while doing rolling upgrade, by enabling Delayed Rebalancing.
+
+### Adding Nodes
+
+Adding nodes (i.e. cluster expansion) changes topology. CrushED uses card 
dealing strategy to provide even distribution, so when topology changes, there 
are a lot of additional partition movements than CRUSH.  
+
+Note that the extra change rate is not correlated with the number of 
additional nodes. So our recommendation is finishing expansion in one operation 
so as to do only one partition shuffling.
+
+### Algorithm Performance
+
+We compared CrushED with CRUSH algorithms using different instance numbers. 
The tests are executed multiple times and we recorded median computation time.  
+CrushED does not cost much additional computation time for regular 
rebalancing. In some of the worst cases, 30% more runtime was observed, 
compared with CRUSH, but it is quicker than MultiRoundCRUSH.
+
+However, when there are down nodes since CrushED needs to run an additional 
consistent hashing based re-distribution, the computation time will be much 
longer. In some cases, we saw more than 3 times compared to CRUSH.
+
+With some **performance improvements**, such as using cache to avoid duplicate 
calculation, we achieved to greatly reduce CrushED's running time. According to 
our experiment, it is now close to MultiRound CRUSH.
+
+![Example](images/design/crushed/performance.png)
+
+## Conclusion
+
+CrushED achieves more uniform distribution compared with CRUSH at the cost of 
higher rebalance computation and more partition movement when the cluster 
topology changes.
+
+## Simple User Guide
+
+1.  Ensure the resouce is using FULL_AUTO mode.
+2.  Set rebalance strategy to be 
"org.apache.helix.controller.rebalancer.strategy.CrushEdRebalanceStrategy".
+3.  Expect more partition movement on topology changes when using the new 
strategy.
+
+**IdeaState SimpleFields Example** 
+
+    HELIX_ENABLED : "true"
+    IDEAL\_STATE\_MODE : "AUTO_REBALANCE"
+    REBALANCE\_MODE : "FULL\_AUTO"
+    REBALANCE_STRATEGY : 
"org.apache.helix.controller.rebalancer.strategy.CrushRebalanceStrategy"
+    MIN\_ACTIVE\_REPLICAS : "0"
+    NUM_PARTITIONS : "64"
+    REBALANCER\_CLASS\_NAME : 
"org.apache.helix.controller.rebalancer.DelayedAutoRebalancer"
+    REPLICAS : "1"
+    STATE\_MODEL\_DEF_REF : "LeaderStandby"
+

http://git-wip-us.apache.org/repos/asf/helix/blob/e8125e8b/website/0.8.3/src/site/markdown/index.md
----------------------------------------------------------------------
diff --git a/website/0.8.3/src/site/markdown/index.md 
b/website/0.8.3/src/site/markdown/index.md
new file mode 100644
index 0000000..c7d755c
--- /dev/null
+++ b/website/0.8.3/src/site/markdown/index.md
@@ -0,0 +1,58 @@
+<!---
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License.
+-->
+
+<head>
+  <title>Helix 0.8.3 Documentation</title>
+</head>
+
+### Get Helix
+
+[Download](./download.html)
+
+[Building](./Building.html)
+
+[Release Notes](./releasenotes/release-0.8.3.html)
+
+### Hands-On
+
+[Quickstart](./Quickstart.html)
+
+[Tutorial](./Tutorial.html)
+
+[Javadocs](http://helix.apache.org/javadocs/0.8.3/index.html)
+
+### Recipes
+
+[Distributed lock manager](./recipes/lock_manager.html)
+
+[Rabbit MQ consumer group](./recipes/rabbitmq_consumer_group.html)
+
+[Rsync replicated file store](./recipes/rsync_replicated_file_store.html)
+
+[Service discovery](./recipes/service_discovery.html)
+
+[Distributed task DAG execution](./recipes/task_dag_execution.html)
+
+### Operation
+
+[Monitoring Metrics](./Metrics.html)
+
+### Design
+
+[CRUSH-ed for even distribution](./design_crushed.html)

http://git-wip-us.apache.org/repos/asf/helix/blob/e8125e8b/website/0.8.3/src/site/markdown/recipes/lock_manager.md
----------------------------------------------------------------------
diff --git a/website/0.8.3/src/site/markdown/recipes/lock_manager.md 
b/website/0.8.3/src/site/markdown/recipes/lock_manager.md
new file mode 100644
index 0000000..3a6f4aa
--- /dev/null
+++ b/website/0.8.3/src/site/markdown/recipes/lock_manager.md
@@ -0,0 +1,236 @@
+<!---
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License.
+-->
+Distributed Lock Manager
+------------------------
+Distributed locks are used to synchronize accesses shared resources. Most 
applications today use ZooKeeper to model distributed locks.
+
+The simplest way to model a lock using ZooKeeper is (See ZooKeeper leader 
recipe for an exact and more advanced solution)
+
+* Each process tries to create an emphemeral node
+* If the node is successfully created, the process acquires the lock
+* Otherwise, it will watch the ZNode and try to acquire the lock again if the 
current lock holder disappears
+
+This is good enough if there is only one lock. But in practice, an application 
will need many such locks. Distributing and managing the locks among difference 
process becomes challenging. Extending such a solution to many locks will 
result in:
+
+* Uneven distribution of locks among nodes; the node that starts first will 
acquire all the locks. Nodes that start later will be idle.
+* When a node fails, how the locks will be distributed among remaining nodes 
is not predicable.
+* When new nodes are added the current nodes don\'t relinquish the locks so 
that new nodes can acquire some locks
+
+In other words we want a system to satisfy the following requirements.
+
+* Distribute locks evenly among all nodes to get better hardware utilization
+* If a node fails, the locks that were acquired by that node should be evenly 
distributed among other nodes
+* If nodes are added, locks must be evenly re-distributed among nodes.
+
+Helix provides a simple and elegant solution to this problem. Simply specify 
the number of locks and Helix will ensure that above constraints are satisfied.
+
+To quickly see this working run the `lock-manager-demo` script where 12 locks 
are evenly distributed among three nodes, and when a node fails, the locks get 
re-distributed among remaining two nodes. Note that Helix does not re-shuffle 
the locks completely, instead it simply distributes the locks relinquished by 
dead node among 2 remaining nodes evenly.
+
+----------------------------------------------------------------------------------------
+
+### Short Version
+This version starts multiple threads within the same process to simulate a 
multi node deployment. Try the long version to get a better idea of how it 
works.
+
+```
+git clone https://git-wip-us.apache.org/repos/asf/helix.git
+cd helix
+git checkout tags/helix-0.8.3
+mvn clean install package -DskipTests
+cd recipes/distributed-lock-manager/target/distributed-lock-manager-pkg/bin
+chmod +x *
+./lock-manager-demo
+```
+
+#### Output
+
+```
+./lock-manager-demo
+STARTING localhost_12000
+STARTING localhost_12002
+STARTING localhost_12001
+STARTED localhost_12000
+STARTED localhost_12002
+STARTED localhost_12001
+localhost_12001 acquired lock:lock-group_3
+localhost_12000 acquired lock:lock-group_8
+localhost_12001 acquired lock:lock-group_2
+localhost_12001 acquired lock:lock-group_4
+localhost_12002 acquired lock:lock-group_1
+localhost_12002 acquired lock:lock-group_10
+localhost_12000 acquired lock:lock-group_7
+localhost_12001 acquired lock:lock-group_5
+localhost_12002 acquired lock:lock-group_11
+localhost_12000 acquired lock:lock-group_6
+localhost_12002 acquired lock:lock-group_0
+localhost_12000 acquired lock:lock-group_9
+lockName    acquired By
+======================================
+lock-group_0    localhost_12002
+lock-group_1    localhost_12002
+lock-group_10    localhost_12002
+lock-group_11    localhost_12002
+lock-group_2    localhost_12001
+lock-group_3    localhost_12001
+lock-group_4    localhost_12001
+lock-group_5    localhost_12001
+lock-group_6    localhost_12000
+lock-group_7    localhost_12000
+lock-group_8    localhost_12000
+lock-group_9    localhost_12000
+Stopping localhost_12000
+localhost_12000 Interrupted
+localhost_12001 acquired lock:lock-group_9
+localhost_12001 acquired lock:lock-group_8
+localhost_12002 acquired lock:lock-group_6
+localhost_12002 acquired lock:lock-group_7
+lockName    acquired By
+======================================
+lock-group_0    localhost_12002
+lock-group_1    localhost_12002
+lock-group_10    localhost_12002
+lock-group_11    localhost_12002
+lock-group_2    localhost_12001
+lock-group_3    localhost_12001
+lock-group_4    localhost_12001
+lock-group_5    localhost_12001
+lock-group_6    localhost_12002
+lock-group_7    localhost_12002
+lock-group_8    localhost_12001
+lock-group_9    localhost_12001
+
+```
+
+----------------------------------------------------------------------------------------
+
+### Long version
+This provides more details on how to setup the cluster and where to plugin 
application code.
+
+#### Start ZooKeeper
+
+```
+./start-standalone-zookeeper 2199
+```
+
+#### Create a Cluster
+
+```
+./helix-admin --zkSvr localhost:2199 --addCluster lock-manager-demo
+```
+
+#### Create a Lock Group
+
+Create a lock group and specify the number of locks in the lock group.
+
+```
+./helix-admin --zkSvr localhost:2199  --addResource lock-manager-demo 
lock-group 6 OnlineOffline --mode AUTO_REBALANCE
+```
+
+#### Start the Nodes
+
+Create a Lock class that handles the callbacks.
+
+```
+public class Lock extends StateModel {
+  private String lockName;
+
+  public Lock(String lockName) {
+    this.lockName = lockName;
+  }
+
+  public void lock(Message m, NotificationContext context) {
+    System.out.println(" acquired lock:"+ lockName );
+  }
+
+  public void release(Message m, NotificationContext context) {
+    System.out.println(" releasing lock:"+ lockName );
+  }
+
+}
+```
+
+and a LockFactory that creates Locks
+
+```
+public class LockFactory extends StateModelFactory<Lock> {
+    /* Instantiates the lock handler, one per lockName */
+    public Lock create(String lockName) {
+        return new Lock(lockName);
+    }
+}
+```
+
+At node start up, simply join the cluster and Helix will invoke the 
appropriate callbacks on the appropriate Lock instance. One can start any 
number of nodes and Helix detects that a new node has joined the cluster and 
re-distributes the locks automatically.
+
+```
+public class LockProcess {
+  public static void main(String args) {
+    String zkAddress= "localhost:2199";
+    String clusterName = "lock-manager-demo";
+    //Give a unique id to each process, most commonly used format hostname_port
+    String instanceName ="localhost_12000";
+    ZKHelixAdmin helixAdmin = new ZKHelixAdmin(zkAddress);
+    //configure the instance and provide some metadata
+    InstanceConfig config = new InstanceConfig(instanceName);
+    config.setHostName("localhost");
+    config.setPort("12000");
+    admin.addInstance(clusterName, config);
+    //join the cluster
+    HelixManager manager;
+    manager = HelixManagerFactory.getHelixManager(clusterName,
+                                                  instanceName,
+                                                  InstanceType.PARTICIPANT,
+                                                  zkAddress);
+    manager.getStateMachineEngine().registerStateModelFactory("OnlineOffline", 
modelFactory);
+    manager.connect();
+    Thread.currentThread.join();
+  }
+}
+```
+
+#### Start the Controller
+
+The controller can be started either as a separate process or can be embedded 
within each node process
+
+##### Separate Process
+This is recommended when number of nodes in the cluster \> 100. For fault 
tolerance, you can run multiple controllers on different boxes.
+
+```
+./run-helix-controller --zkSvr localhost:2199 --cluster lock-manager-demo 2>&1 
> /tmp/controller.log &
+```
+
+##### Embedded Within the Node Process
+This is recommended when the number of nodes in the cluster is less than 100. 
To start a controller from each process, simply add the following lines to 
MyClass
+
+```
+public class LockProcess {
+  public static void main(String args) {
+    String zkAddress= "localhost:2199";
+    String clusterName = "lock-manager-demo";
+    // .
+    // .
+    manager.connect();
+    HelixManager controller;
+    controller = HelixControllerMain.startHelixController(zkAddress,
+                                                          clusterName,
+                                                          "controller",
+                                                          
HelixControllerMain.STANDALONE);
+    Thread.currentThread.join();
+  }
+}
+```

http://git-wip-us.apache.org/repos/asf/helix/blob/e8125e8b/website/0.8.3/src/site/markdown/recipes/rabbitmq_consumer_group.md
----------------------------------------------------------------------
diff --git a/website/0.8.3/src/site/markdown/recipes/rabbitmq_consumer_group.md 
b/website/0.8.3/src/site/markdown/recipes/rabbitmq_consumer_group.md
new file mode 100644
index 0000000..e0283fd
--- /dev/null
+++ b/website/0.8.3/src/site/markdown/recipes/rabbitmq_consumer_group.md
@@ -0,0 +1,213 @@
+<!---
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License.
+-->
+
+
+RabbitMQ Consumer Group
+-----------------------
+
+[RabbitMQ](http://www.rabbitmq.com/) is well-known open source software the 
provides robust messaging for applications.
+
+One of the commonly implemented recipes using this software is a work queue.  
[http://www.rabbitmq.com/tutorials/tutorial-four-java.html](http://www.rabbitmq.com/tutorials/tutorial-four-java.html)
 describes the use case where
+
+* A producer sends a message with a routing key
+* The message is routed to the queue whose binding key exactly matches the 
routing key of the message
+* There are multiple consumers and each consumer is interested in processing 
only a subset of the messages by binding to the interested keys
+
+The example provided 
[here](http://www.rabbitmq.com/tutorials/tutorial-four-java.html) describes how 
multiple consumers can be started to process all the messages.
+
+While this works, in production systems one needs the following:
+
+* Ability to handle failures: when a consumer fails, another consumer must be 
started or the other consumers must start processing these messages that should 
have been processed by the failed consumer
+* When the existing consumers cannot keep up with the task generation rate, 
new consumers will be added. The tasks must be redistributed among all the 
consumers
+
+In this recipe, we demonstrate handling of consumer failures and new consumer 
additions using Helix.
+
+Mapping this usecase to Helix is pretty easy as the binding key/routing key is 
equivalent to a partition.
+
+Let's take an example. Lets say the queue has 6 partitions, and we have 2 
consumers to process all the queues.
+What we want is all 6 queues to be evenly divided among 2 consumers.
+Eventually when the system scales, we add more consumers to keep up. This will 
make each consumer process tasks from 2 queues.
+Now let's say that a consumer failed, reducing the number of active consumers 
to 2. This means each consumer must process 3 queues.
+
+We showcase how such a dynamic application can be developed using Helix. Even 
though we use RabbitMQ as the pub/sub system one can extend this solution to 
other pub/sub systems.
+
+### Try It
+
+```
+git clone https://git-wip-us.apache.org/repos/asf/helix.git
+cd helix
+git checkout tags/helix-0.8.3
+mvn clean install package -DskipTests
+cd recipes/rabbitmq-consumer-group/bin
+chmod +x *
+export HELIX_PKG_ROOT=`pwd`/helix-core/target/helix-core-pkg
+export HELIX_RABBITMQ_ROOT=`pwd`/recipes/rabbitmq-consumer-group/
+chmod +x $HELIX_PKG_ROOT/bin/*
+chmod +x $HELIX_RABBITMQ_ROOT/bin/*
+```
+
+#### Install RabbitMQ
+
+Setting up RabbitMQ on a local box is straightforward. You can find the 
instructions here
+http://www.rabbitmq.com/download.html
+
+#### Start ZK
+
+Start ZooKeeper at port 2199
+
+```
+$HELIX_PKG_ROOT/bin/start-standalone-zookeeper 2199
+```
+
+#### Setup the Consumer Group Cluster
+
+This will setup the cluster by creating a "rabbitmq-consumer-group" cluster 
and adds a "topic" with "6" queues.
+
+```
+$HELIX_RABBITMQ_ROOT/bin/setup-cluster.sh localhost:2199
+```
+
+#### Add Consumers
+
+Start 2 consumers in 2 different terminals. Each consumer is given a unique ID.
+
+```
+//start-consumer.sh zookeeperAddress (e.g. localhost:2181) consumerId , 
rabbitmqServer (e.g. localhost)
+$HELIX_RABBITMQ_ROOT/bin/start-consumer.sh localhost:2199 0 localhost
+$HELIX_RABBITMQ_ROOT/bin/start-consumer.sh localhost:2199 1 localhost
+
+```
+
+#### Start the Helix Controller
+
+Now start a Helix controller that starts managing the 
"rabbitmq-consumer-group" cluster.
+
+```
+$HELIX_RABBITMQ_ROOT/bin/start-cluster-manager.sh localhost:2199
+```
+
+#### Send Messages to the Topic
+
+Start sending messages to the topic. This script randomly selects a routing 
key (1-6) and sends the message to topic.
+Based on the key, messages gets routed to the appropriate queue.
+
+```
+$HELIX_RABBITMQ_ROOT/bin/send-message.sh localhost 20
+```
+
+After running this, you should see all 20 messages being processed by 2 
consumers.
+
+#### Add Another Consumer
+
+Once a new consumer is started, Helix detects it. In order to balance the load 
between 3 consumers, it deallocates 1 partition from the existing consumers and 
allocates it to the new consumer. We see that
+each consumer is now processing only 2 queues.
+Helix makes sure that old nodes are asked to stop consuming before the new 
consumer is asked to start consuming for a given partition. But the transitions 
for each partition can happen in parallel.
+
+```
+$HELIX_RABBITMQ_ROOT/bin/start-consumer.sh localhost:2199 2 localhost
+```
+
+Send messages again to the topic
+
+```
+$HELIX_RABBITMQ_ROOT/bin/send-message.sh localhost 100
+```
+
+You should see that messages are now received by all 3 consumers.
+
+#### Stop a Consumer
+
+In any terminal press CTRL^C and notice that Helix detects the consumer 
failure and distributes the 2 partitions that were processed by failed consumer 
to the remaining 2 active consumers.
+
+
+### How does this work?
+
+Find the entire code 
[here](https://git-wip-us.apache.org/repos/asf?p=helix.git;a=tree;f=recipes/rabbitmq-consumer-group/src/main/java/org/apache/helix/recipes/rabbitmq).
+
+#### Cluster Setup
+
+This step creates ZNode on ZooKeeper for the cluster and adds the state model. 
We use online offline state model since there is no need for other states. The 
consumer is either processing a queue or it is not.
+
+It creates a resource called "rabbitmq-consumer-group" with 6 partitions. The 
execution mode is set to AUTO_REBALANCE. This means that the Helix controls the 
assignment of partition to consumers and automatically distributes the 
partitions evenly among the active consumers. When a consumer is added or 
removed, it ensures that a minimum number of partitions are shuffled.
+
+```
+zkclient = new ZkClient(zkAddr, ZkClient.DEFAULT_SESSION_TIMEOUT,
+    ZkClient.DEFAULT_CONNECTION_TIMEOUT, new ZNRecordSerializer());
+ZKHelixAdmin admin = new ZKHelixAdmin(zkclient);
+
+// add cluster
+admin.addCluster(clusterName, true);
+
+// add state model definition
+StateModelConfigGenerator generator = new StateModelConfigGenerator();
+admin.addStateModelDef(clusterName, "OnlineOffline",
+    new StateModelDefinition(generator.generateConfigForOnlineOffline()));
+
+// add resource "topic" which has 6 partitions
+String resourceName = "rabbitmq-consumer-group";
+admin.addResource(clusterName, resourceName, 6, "OnlineOffline", 
"AUTO_REBALANCE");
+```
+
+### Starting the Consumers
+
+The only thing consumers need to know is the ZooKeeper address, cluster name 
and consumer ID. It does not need to know anything else.
+
+```
+_manager = HelixManagerFactory.getZKHelixManager(_clusterName,
+                                                 _consumerId,
+                                                 InstanceType.PARTICIPANT,
+                                                 _zkAddr);
+
+StateMachineEngine stateMach = _manager.getStateMachineEngine();
+ConsumerStateModelFactory modelFactory =
+    new ConsumerStateModelFactory(_consumerId, _mqServer);
+stateMach.registerStateModelFactory("OnlineOffline", modelFactory);
+
+_manager.connect();
+```
+
+Once the consumer has registered the state model and the controller is 
started, the consumer starts getting callbacks (onBecomeOnlineFromOffline) for 
the partition it needs to host. All it needs to do as part of the callback is 
to start consuming messages from the appropriate queue. Similarly, when the 
controller deallocates a partitions from a consumer, it fires 
onBecomeOfflineFromOnline for the same partition.
+As a part of this transition, the consumer will stop consuming from a that 
queue.
+
+```
+@Transition(to = "ONLINE", from = "OFFLINE")
+public void onBecomeOnlineFromOffline(Message message, NotificationContext 
context) {
+  LOG.debug(_consumerId + " becomes ONLINE from OFFLINE for " + _partition);
+  if (_thread == null) {
+    LOG.debug("Starting ConsumerThread for " + _partition + "...");
+    _thread = new ConsumerThread(_partition, _mqServer, _consumerId);
+    _thread.start();
+    LOG.debug("Starting ConsumerThread for " + _partition + " done");
+
+  }
+}
+
+@Transition(to = "OFFLINE", from = "ONLINE")
+public void onBecomeOfflineFromOnline(Message message, NotificationContext 
context)
+    throws InterruptedException {
+  LOG.debug(_consumerId + " becomes OFFLINE from ONLINE for " + _partition);
+  if (_thread != null) {
+    LOG.debug("Stopping " + _consumerId + " for " + _partition + "...");
+    _thread.interrupt();
+    _thread.join(2000);
+    _thread = null;
+    LOG.debug("Stopping " +  _consumerId + " for " + _partition + " done");
+  }
+}
+```

http://git-wip-us.apache.org/repos/asf/helix/blob/e8125e8b/website/0.8.3/src/site/markdown/recipes/rsync_replicated_file_store.md
----------------------------------------------------------------------
diff --git 
a/website/0.8.3/src/site/markdown/recipes/rsync_replicated_file_store.md 
b/website/0.8.3/src/site/markdown/recipes/rsync_replicated_file_store.md
new file mode 100644
index 0000000..107c937
--- /dev/null
+++ b/website/0.8.3/src/site/markdown/recipes/rsync_replicated_file_store.md
@@ -0,0 +1,162 @@
+<!---
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License.
+-->
+
+Near-Realtime Rsync Replicated File System
+------------------------------------------
+
+### Quick Demo
+
+* This demo starts 3 instances with id's as ```localhost_12001, 
localhost_12002, localhost_12003```
+* Each instance stores its files under ```/tmp/<id>/filestore```
+* ```localhost_12001``` is designated as the master, and ```localhost_12002``` 
and ```localhost_12003``` are the slaves
+* Files written to the master are replicated to the slaves automatically. In 
this demo, a.txt and b.txt are written to ```/tmp/localhost_12001/filestore``` 
and they get replicated to other folders.
+* When the master is stopped, ```localhost_12002``` is promoted to master.
+* The other slave ```localhost_12003``` stops replicating from 
```localhost_12001``` and starts replicating from new master 
```localhost_12002```
+* Files written to new master ```localhost_12002``` are replicated to 
```localhost_12003```
+* In the end state of this quick demo, ```localhost_12002``` is the master and 
```localhost_12003``` is the slave. Manually create files under 
```/tmp/localhost_12002/filestore``` and see that appear in 
```/tmp/localhost_12003/filestore```
+* Ignore the interrupted exceptions on the console :-)
+
+
+```
+git clone https://git-wip-us.apache.org/repos/asf/helix.git
+cd helix
+git checkout tags/helix-0.8.3
+cd recipes/rsync-replicated-file-system/
+mvn clean install package -DskipTests
+cd target/rsync-replicated-file-system-pkg/bin
+chmod +x *
+./quickdemo
+
+```
+
+### Overview
+
+There are many applications that require storage for storing large number of 
relatively small data files. Examples include media stores to store small 
videos, images, mail attachments etc. Each of these objects is typically 
kilobytes, often no larger than a few megabytes. An additional distinguishing 
feature of these use cases is that files are typically only added or deleted, 
rarely updated. When there are updates, they do not have any concurrency 
requirements.
+
+These are much simpler requirements than what general purpose distributed file 
system have to satisfy; these would include concurrent access to files, random 
access for reads and updates, posix compliance, and others. To satisfy those 
requirements, general DFSs are also pretty complex that are expensive to build 
and maintain.
+
+A different implementation of a distributed file system includes HDFS which is 
inspired by Google's GFS. This is one of the most widely used distributed file 
system that forms the main data storage platform for Hadoop. HDFS is primary 
aimed at processing very large data sets and distributes files across a cluster 
of commodity servers by splitting up files in fixed size chunks. HDFS is not 
particularly well suited for storing a very large number of relatively tiny 
files.
+
+### File Store
+
+It's possible to build a vastly simpler system for the class of applications 
that have simpler requirements as we have pointed out.
+
+* Large number of files but each file is relatively small
+* Access is limited to create, delete and get entire files
+* No updates to files that are already created (or it's feasible to delete the 
old file and create a new one)
+
+
+We call this system a Partitioned File Store (PFS) to distinguish it from 
other distributed file systems. This system needs to provide the following 
features:
+
+* CRD access to large number of small files
+* Scalability: Files should be distributed across a large number of commodity 
servers based on the storage requirement
+* Fault-tolerance: Each file should be replicated on multiple servers so that 
individual server failures do not reduce availability
+* Elasticity: It should be possible to add capacity to the cluster easily
+
+
+Apache Helix is a generic cluster management framework that makes it very easy 
to provide scalability, fault-tolerance and elasticity features.
+rsync can be easily used as a replication channel between servers so that each 
file gets replicated on multiple servers.
+
+### Design
+
+#### High Level
+
+* Partition the file system based on the file name
+* At any time a single writer can write, we call this a master
+* For redundancy, we need to have additional replicas called slave. Slaves can 
optionally serve reads
+* Slave replicates data from the master
+* When a master fails, a slave gets promoted to master
+
+#### Transaction Log
+
+Every write on the master will result in creation/deletion of one or more 
files. In order to maintain timeline consistency slaves need to apply the 
changes in the same order
+To facilitate this, the master logs each transaction in a file and each 
transaction is associated with an 64 bit ID in which the 32 LSB represents a 
sequence number and MSB represents the generation number
+The sequence number gets incremented on every transaction and the generation 
is incremented when a new master is elected
+
+#### Replication
+
+Replication is required for slaves to keep up with changes on the master. 
Every time the slave applies a change it checkpoints the last applied 
transaction ID.
+During restarts, this allows the slave to pull changes from the last 
checkpointed ID. Similar to master, the slave logs each transaction to the 
transaction logs but instead of generating new transaction ID, it uses the same 
ID generated by the master.
+
+
+#### Failover
+
+When a master fails, a new slave will be promoted to master. If the previous 
master node is reachable, then the new master will flush all the
+changes from previous the master before taking up mastership. The new master 
will record the end transaction ID of the current generation and then start a 
new generation
+with sequence starting from 1. After this the master will begin accepting 
writes.
+
+![Partitioned File Store](../images/PFS-Generic.png)
+
+
+
+### Rsync-based Solution
+
+![Rsync based File Store](../images/RSYNC_BASED_PFS.png)
+
+
+This application demonstrates a file store that uses rsync as the replication 
mechanism. One can envision a similar system where instead of using rsync, one
+can implement a custom solution to notify the slave of the changes and also 
provide an api to pull the change files.
+
+#### Concepts
+* file_store_dir: Root directory for the actual data files
+* change_log_dir: The transaction logs are generated under this folder
+* check_point_dir: The slave stores the check points ( last processed 
transaction) here
+
+#### Master
+* File server: This component supports file uploads and downloads and writes 
the files to ```file_store_dir```. This is not included in this application. 
The idea is that most applications have different ways of implementing this 
component and have some associated business logic. It is not hard to come up 
with such a component if needed.
+* File store watcher: This component watches the ```file_store_dir``` 
directory on the local file system for any changes and notifies the registered 
listeners of the changes
+* Change log generator: This registers as a listener of the file store watcher 
and on each notification logs the changes into a file under ```change_log_dir```
+
+#### Slave
+* File server: This component on the slave will only support reads
+* Cluster state observer: Slave observes the cluster state and is able to know 
who is the current master
+* Replicator: This has two subcomponents
+    - Periodic rsync of change log: This is a background process that 
periodically rsyncs the ```change_log_dir``` of the master to its local 
directory
+    - Change Log Watcher: This watches the ```change_log_dir``` for changes 
and notifies the registered listeners of the change
+    - On demand rsync invoker: This is registered as a listener to change log 
watcher and on every change invokes rsync to sync only the changed file
+
+#### Coordination
+
+The coordination between nodes is done by Helix. Helix does the partition 
management and assigns the partition to multiple nodes based on the replication 
factor. It elects one the nodes as master and designates others as slaves.
+It provides notifications to each node in the form of state transitions 
(Offline to Slave, Slave to Master). It also provides notifications when there 
is change is cluster state.
+This allows the slave to stop replicating from current master and start 
replicating from new master.
+
+In this application, we have only one partition but its very easy to extend it 
to support multiple partitions. By partitioning the file store, one can add new 
nodes and Helix will automatically
+re-distribute partitions among the nodes. To summarize, Helix provides 
partition management, fault tolerance and facilitates automated cluster 
expansion.
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+

http://git-wip-us.apache.org/repos/asf/helix/blob/e8125e8b/website/0.8.3/src/site/markdown/recipes/service_discovery.md
----------------------------------------------------------------------
diff --git a/website/0.8.3/src/site/markdown/recipes/service_discovery.md 
b/website/0.8.3/src/site/markdown/recipes/service_discovery.md
new file mode 100644
index 0000000..fe80bf0
--- /dev/null
+++ b/website/0.8.3/src/site/markdown/recipes/service_discovery.md
@@ -0,0 +1,184 @@
+<!---
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License.
+-->
+Service Discovery
+-----------------
+
+One of the common usage of ZooKeeper is to enable service discovery.
+The basic idea is that when a server starts up it advertises its 
configuration/metadata such as its hostname and port on ZooKeeper.
+This allows clients to dynamically discover the servers that are currently 
active. One can think of this like a service registry to which a server 
registers when it starts and
+is automatically deregistered when it shutdowns or crashes. In many cases it 
serves as an alternative to VIPs.
+
+The core idea behind this is to use ZooKeeper ephemeral nodes. The ephemeral 
nodes are created when the server registers and all its metadata is put into a 
ZNode.
+When the server shutdowns, ZooKeeper automatically removes this ZNode.
+
+There are two ways the clients can dynamically discover the active servers:
+
+### ZooKeeper Watch
+
+Clients can set a child watch under specific path on ZooKeeper.
+When a new service is registered/deregistered, ZooKeeper notifies the client 
via a watch event and the client can read the list of services. Even though 
this looks trivial,
+there are lot of things one needs to keep in mind like ensuring that you first 
set the watch back on ZooKeeper before reading data.
+
+
+### Poll
+
+Another approach is for the client to periodically read the ZooKeeper path and 
get the list of services.
+
+Both approaches have pros and cons, for example setting a watch might trigger 
herd effect if there are large number of clients. This is problematic, 
especially when servers are starting up.
+But the advantage to setting watches is that clients are immediately notified 
of a change which is not true in case of polling.
+In some cases, having both watches and polls makes sense; watch allows one to 
get notifications as soon as possible while poll provides a safety net if a 
watch event is missed because of code bug or ZooKeeper fails to notify.
+
+### Other Developer Considerations
+* What happens when the ZooKeeper session expires? All the watches and 
ephemeral nodes previously added or created by this server are lost. One needs 
to add the watches again, recreate the ephemeral nodes, and so on.
+* Due to network issues or Java GC pauses session expiry might happen again 
and again; this phenomenon is known as flapping. It\'s important for the server 
to detect this and deregister itself.
+
+### Other Operational Considerations
+* What if the node is behaving badly? One might kill the server, but it will 
lose the ability to debug. It would be nice to have the ability to mark a 
server as disabled and clients know that a node is disabled and will not 
contact that node.
+
+### Configuration Ownership
+
+This is an important aspect that is often ignored in the initial stages of 
your development. Typically, the service discovery pattern means that servers 
start up with some configuration which it simply puts into ZooKeeper. While 
this works well in the beginning, configuration management becomes very 
difficult since the servers themselves are statically configured. Any change in 
server configuration implies restarting the server. Ideally, it will be nice to 
have the ability to change configuration dynamically without having to restart 
a server.
+
+Ideally you want a hybrid solution, a node starts with minimal configuration 
and gets the rest of configuration from ZooKeeper.
+
+### Using Helix for Service Discovery
+
+Even though Helix has a higher-level abstraction in terms of state machines, 
constraints and objectives, service discovery is one of things has been a 
prevalent use case from the start.
+The controller uses the exact mechanism we described above to discover when 
new servers join the cluster. We create these ZNodes under 
/CLUSTERNAME/LIVEINSTANCES.
+Since at any time there is only one controller, we use a ZK watch to track the 
liveness of a server.
+
+This recipe simply demonstrates how one can re-use that part for implementing 
service discovery. This demonstrates multiple modes of service discovery:
+
+* POLL: The client reads from zookeeper at regular intervals 30 seconds. Use 
this if you have 100's of clients
+* WATCH: The client sets up watcher and gets notified of the changes. Use this 
if you have 10's of clients
+* NONE: This does neither of the above, but reads directly from zookeeper when 
ever needed
+
+Helix provides these additional features compared to other implementations 
available elsewhere:
+
+* It has the concept of disabling a node which means that a badly behaving 
node can be disabled using the Helix admin API
+* It automatically detects if a node connects/disconnects from zookeeper 
repeatedly and disables the node
+* Configuration management
+    * Allows one to set configuration via the admin API at various granulaties 
like cluster, instance, resource, partition
+    * Configurations can be dynamically changed
+    * The server is notified when configurations change
+
+
+### Checkout and Build
+
+```
+git clone https://git-wip-us.apache.org/repos/asf/helix.git
+cd helix
+git checkout tags/helix-0.8.3
+mvn clean install package -DskipTests
+cd recipes/service-discovery/target/service-discovery-pkg/bin
+chmod +x *
+```
+
+### Start ZooKeeper
+
+```
+./start-standalone-zookeeper 2199
+```
+
+### Run the Demo
+
+```
+./service-discovery-demo.sh
+```
+
+### Output
+
+```
+START:Service discovery demo mode:WATCH
+       Registering service
+               host.x.y.z_12000
+               host.x.y.z_12001
+               host.x.y.z_12002
+               host.x.y.z_12003
+               host.x.y.z_12004
+       SERVICES AVAILABLE
+               SERVICENAME     HOST                    PORT
+               myServiceName   host.x.y.z              12000
+               myServiceName   host.x.y.z              12001
+               myServiceName   host.x.y.z              12002
+               myServiceName   host.x.y.z              12003
+               myServiceName   host.x.y.z              12004
+       Deregistering service:
+               host.x.y.z_12002
+       SERVICES AVAILABLE
+               SERVICENAME     HOST                    PORT
+               myServiceName   host.x.y.z              12000
+               myServiceName   host.x.y.z              12001
+               myServiceName   host.x.y.z              12003
+               myServiceName   host.x.y.z              12004
+       Registering service:host.x.y.z_12002
+END:Service discovery demo mode:WATCH
+=============================================
+START:Service discovery demo mode:POLL
+       Registering service
+               host.x.y.z_12000
+               host.x.y.z_12001
+               host.x.y.z_12002
+               host.x.y.z_12003
+               host.x.y.z_12004
+       SERVICES AVAILABLE
+               SERVICENAME     HOST                    PORT
+               myServiceName   host.x.y.z              12000
+               myServiceName   host.x.y.z              12001
+               myServiceName   host.x.y.z              12002
+               myServiceName   host.x.y.z              12003
+               myServiceName   host.x.y.z              12004
+       Deregistering service:
+               host.x.y.z_12002
+       Sleeping for poll interval:30000
+       SERVICES AVAILABLE
+               SERVICENAME     HOST                    PORT
+               myServiceName   host.x.y.z              12000
+               myServiceName   host.x.y.z              12001
+               myServiceName   host.x.y.z              12003
+               myServiceName   host.x.y.z              12004
+       Registering service:host.x.y.z_12002
+END:Service discovery demo mode:POLL
+=============================================
+START:Service discovery demo mode:NONE
+       Registering service
+               host.x.y.z_12000
+               host.x.y.z_12001
+               host.x.y.z_12002
+               host.x.y.z_12003
+               host.x.y.z_12004
+       SERVICES AVAILABLE
+               SERVICENAME     HOST                    PORT
+               myServiceName   host.x.y.z              12000
+               myServiceName   host.x.y.z              12001
+               myServiceName   host.x.y.z              12002
+               myServiceName   host.x.y.z              12003
+               myServiceName   host.x.y.z              12004
+       Deregistering service:
+               host.x.y.z_12000
+       SERVICES AVAILABLE
+               SERVICENAME     HOST                    PORT
+               myServiceName   host.x.y.z              12001
+               myServiceName   host.x.y.z              12002
+               myServiceName   host.x.y.z              12003
+               myServiceName   host.x.y.z              12004
+       Registering service:host.x.y.z_12000
+END:Service discovery demo mode:NONE
+=============================================
+```

http://git-wip-us.apache.org/repos/asf/helix/blob/e8125e8b/website/0.8.3/src/site/markdown/recipes/task_dag_execution.md
----------------------------------------------------------------------
diff --git a/website/0.8.3/src/site/markdown/recipes/task_dag_execution.md 
b/website/0.8.3/src/site/markdown/recipes/task_dag_execution.md
new file mode 100644
index 0000000..12a4e6e
--- /dev/null
+++ b/website/0.8.3/src/site/markdown/recipes/task_dag_execution.md
@@ -0,0 +1,199 @@
+<!---
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License.
+-->
+
+Distributed Task Execution
+--------------------------
+
+This recipe is intended to demonstrate how task dependencies can be modeled 
using primitives provided by Helix. A given task can be run with the desired 
amount of parallelism and will start only when upstream dependencies are met. 
The demo executes the task DAG described below using 10 workers. Although the 
demo starts the workers as threads, there is no requirement that all the 
workers need to run in the same process. In reality, these workers run on many 
different boxes on a cluster.  When worker fails, Helix takes care of 
re-assigning a failed task partition to a new worker.
+
+Redis is used as a result store. Any other suitable implementation for 
TaskResultStore can be plugged in.
+
+### Workflow
+
+#### Input
+
+10000 impression events and around 100 click events are pre-populated in task 
result store (redis).
+
+* **ImpEvent**: format: id,isFraudulent,country,gender
+
+* **ClickEvent**: format: id,isFraudulent,impEventId
+
+#### Stages
+
++ **FilterImps**: Filters impression where isFraudulent=true.
+
++ **FilterClicks**: Filters clicks where isFraudulent=true
+
++ **impCountsByGender**: Generates impression counts grouped by gender. It 
does this by incrementing the count for 
'impression_gender_counts:<gender_value>' in the task result store (redis 
hash). Depends on: **FilterImps**
+
++ **impCountsByCountry**: Generates impression counts grouped by country. It 
does this by incrementing the count for 
'impression_country_counts:<country_value>' in the task result store (redis 
hash). Depends on: **FilterClicks**
+
++ **impClickJoin**: Joins clicks with corresponding impression event using 
impEventId as the join key. Join is needed to pull dimensions not present in 
click event. Depends on: **FilterImps, FilterClicks**
+
++ **clickCountsByGender**: Generates click counts grouped by gender. It does 
this by incrementing the count for click_gender_counts:<gender_value> in the 
task result store (redis hash). Depends on: **impClickJoin**
+
++ **clickCountsByGender**: Generates click counts grouped by country. It does 
this by incrementing the count for click_country_counts:<country_value> in the 
task result store (redis hash). Depends on: **impClickJoin**
+
++ **report**: Reads from all aggregates generated by previous stages and 
prints them. Depends on: **impCountsByGender, impCountsByCountry, 
clickCountsByGender,clickCountsByGender**
+
+
+### Creating a DAG
+
+Each stage is represented as a Node along with the upstream dependency and 
desired parallelism.  Each stage is modeled as a resource in Helix using 
OnlineOffline state model. As part of an Offline to Online transition, we watch 
the external view of upstream resources and wait for them to transition to the 
online state. See Task.java for additional info.
+
+```
+Dag dag = new Dag();
+dag.addNode(new Node("filterImps", 10, ""));
+dag.addNode(new Node("filterClicks", 5, ""));
+dag.addNode(new Node("impClickJoin", 10, "filterImps,filterClicks"));
+dag.addNode(new Node("impCountsByGender", 10, "filterImps"));
+dag.addNode(new Node("impCountsByCountry", 10, "filterImps"));
+dag.addNode(new Node("clickCountsByGender", 5, "impClickJoin"));
+dag.addNode(new Node("clickCountsByCountry", 5, "impClickJoin"));
+dag.addNode(new 
Node("report",1,"impCountsByGender,impCountsByCountry,clickCountsByGender,clickCountsByCountry"));
+```
+
+### Demo
+
+In order to run the demo, use the following steps
+
+See http://redis.io/topics/quickstart on how to install redis server
+
+```
+Start redis e.g:
+./redis-server --port 6379
+
+git clone https://git-wip-us.apache.org/repos/asf/helix.git
+cd helix
+git checkout helix-0.8.3
+cd recipes/task-execution
+mvn clean install package -DskipTests
+cd target/task-execution-pkg/bin
+chmod +x task-execution-demo.sh
+./task-execution-demo.sh 2181 localhost 6379
+
+```
+
+Here\'s a visual representation of the DAG.
+
+```
+
+
+
+
+
+                       +-----------------+       +----------------+
+                       |   filterImps    |       |  filterClicks  |
+                       | (parallelism=10)|       | (parallelism=5)|
+                       +----------+-----++       +-------+--------+
+                       |          |     |                |
+                       |          |     |                |
+                       |          |     |                |
+                       |          |     +------->--------v------------+
+      +--------------<-+   +------v-------+    |  impClickJoin        |
+      |impCountsByGender   |impCountsByCountry | (parallelism=10)     |
+      |(parallelism=10)    |(parallelism=10)   ++-------------------+-+
+      +-----------+--+     +---+----------+     |                   |
+                  |            |                |                   |
+                  |            |                |                   |
+                  |            |       +--------v---------+       
+-v-------------------+
+                  |            |       |clickCountsByGender       
|clickCountsByCountry |
+                  |            |       |(parallelism=5)   |       
|(parallelism=5)      |
+                  |            |       +----+-------------+       
+---------------------+
+                  |            |            |                     |
+                  |            |            |                     |
+                  |            |            |                     |
+                  +----->+-----+>-----------v----+<---------------+
+                         | report                |
+                         |(parallelism=1)        |
+                         +-----------------------+
+
+```
+
+(credit for above ascii art: http://www.asciiflow.com)
+
+#### Output
+
+```
+Done populating dummy data
+Executing filter task for filterImps_3 for impressions_demo
+Executing filter task for filterImps_2 for impressions_demo
+Executing filter task for filterImps_0 for impressions_demo
+Executing filter task for filterImps_1 for impressions_demo
+Executing filter task for filterImps_4 for impressions_demo
+Executing filter task for filterClicks_3 for clicks_demo
+Executing filter task for filterClicks_1 for clicks_demo
+Executing filter task for filterImps_8 for impressions_demo
+Executing filter task for filterImps_6 for impressions_demo
+Executing filter task for filterClicks_2 for clicks_demo
+Executing filter task for filterClicks_0 for clicks_demo
+Executing filter task for filterImps_7 for impressions_demo
+Executing filter task for filterImps_5 for impressions_demo
+Executing filter task for filterClicks_4 for clicks_demo
+Executing filter task for filterImps_9 for impressions_demo
+Running AggTask for impCountsByGender_3 for filtered_impressions_demo gender
+Running AggTask for impCountsByGender_2 for filtered_impressions_demo gender
+Running AggTask for impCountsByGender_0 for filtered_impressions_demo gender
+Running AggTask for impCountsByGender_9 for filtered_impressions_demo gender
+Running AggTask for impCountsByGender_1 for filtered_impressions_demo gender
+Running AggTask for impCountsByGender_4 for filtered_impressions_demo gender
+Running AggTask for impCountsByCountry_4 for filtered_impressions_demo country
+Running AggTask for impCountsByGender_5 for filtered_impressions_demo gender
+Executing JoinTask for impClickJoin_2
+Running AggTask for impCountsByCountry_3 for filtered_impressions_demo country
+Running AggTask for impCountsByCountry_1 for filtered_impressions_demo country
+Running AggTask for impCountsByCountry_0 for filtered_impressions_demo country
+Running AggTask for impCountsByCountry_2 for filtered_impressions_demo country
+Running AggTask for impCountsByGender_6 for filtered_impressions_demo gender
+Executing JoinTask for impClickJoin_1
+Executing JoinTask for impClickJoin_0
+Executing JoinTask for impClickJoin_3
+Running AggTask for impCountsByGender_8 for filtered_impressions_demo gender
+Executing JoinTask for impClickJoin_4
+Running AggTask for impCountsByGender_7 for filtered_impressions_demo gender
+Running AggTask for impCountsByCountry_5 for filtered_impressions_demo country
+Running AggTask for impCountsByCountry_6 for filtered_impressions_demo country
+Executing JoinTask for impClickJoin_9
+Running AggTask for impCountsByCountry_8 for filtered_impressions_demo country
+Running AggTask for impCountsByCountry_7 for filtered_impressions_demo country
+Executing JoinTask for impClickJoin_5
+Executing JoinTask for impClickJoin_6
+Running AggTask for impCountsByCountry_9 for filtered_impressions_demo country
+Executing JoinTask for impClickJoin_8
+Executing JoinTask for impClickJoin_7
+Running AggTask for clickCountsByCountry_1 for joined_clicks_demo country
+Running AggTask for clickCountsByCountry_0 for joined_clicks_demo country
+Running AggTask for clickCountsByCountry_2 for joined_clicks_demo country
+Running AggTask for clickCountsByCountry_3 for joined_clicks_demo country
+Running AggTask for clickCountsByGender_1 for joined_clicks_demo gender
+Running AggTask for clickCountsByCountry_4 for joined_clicks_demo country
+Running AggTask for clickCountsByGender_3 for joined_clicks_demo gender
+Running AggTask for clickCountsByGender_2 for joined_clicks_demo gender
+Running AggTask for clickCountsByGender_4 for joined_clicks_demo gender
+Running AggTask for clickCountsByGender_0 for joined_clicks_demo gender
+Running reports task
+Impression counts per country
+{CANADA=1940, US=1958, CHINA=2014, UNKNOWN=2022, UK=1946}
+Click counts per country
+{US=24, CANADA=14, CHINA=26, UNKNOWN=14, UK=22}
+Impression counts per gender
+{F=3325, UNKNOWN=3259, M=3296}
+Click counts per gender
+{F=33, UNKNOWN=32, M=35}
+```

http://git-wip-us.apache.org/repos/asf/helix/blob/e8125e8b/website/0.8.3/src/site/markdown/tutorial_admin.md
----------------------------------------------------------------------
diff --git a/website/0.8.3/src/site/markdown/tutorial_admin.md 
b/website/0.8.3/src/site/markdown/tutorial_admin.md
new file mode 100644
index 0000000..c19b407
--- /dev/null
+++ b/website/0.8.3/src/site/markdown/tutorial_admin.md
@@ -0,0 +1,408 @@
+<!---
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License.
+-->
+
+<head>
+  <title>Tutorial - Admin Operations</title>
+</head>
+
+## [Helix Tutorial](./Tutorial.html): Admin Operations
+
+Helix provides a set of admin APIs for cluster management operations. They are 
supported via:
+
+* Java API
+* Command Line Interface
+* REST Interface via helix-admin-webapp
+
+### Java API
+See interface 
[_org.apache.helix.HelixAdmin_](http://helix.apache.org/javadocs/0.8.3/reference/org/apache/helix/HelixAdmin.html)
+
+### Command Line Interface
+The command line tool comes with helix-core package:
+
+Get the command line tool:
+
+```
+git clone https://git-wip-us.apache.org/repos/asf/helix.git
+cd helix
+git checkout tags/helix-0.8.3
+./build
+cd helix-core/target/helix-core-pkg/bin
+chmod +x *.sh
+```
+
+Get help:
+
+```
+./helix-admin.sh --help
+```
+
+All other commands have this form:
+
+```
+./helix-admin.sh --zkSvr <ZookeeperServerAddress> <command> <parameters>
+```
+
+#### Supported Commands
+
+| Command Syntax | Description |
+| -------------- | ----------- |
+| _\-\-activateCluster \<clusterName controllerCluster true/false\>_ | 
Enable/disable a cluster in distributed controller mode |
+| _\-\-addCluster \<clusterName\>_ | Add a new cluster |
+| _\-\-addIdealState \<clusterName resourceName fileName.json\>_ | Add an 
ideal state to a cluster |
+| _\-\-addInstanceTag \<clusterName instanceName tag\>_ | Add a tag to an 
instance |
+| _\-\-addNode \<clusterName instanceId\>_ | Add an instance to a cluster |
+| _\-\-addResource \<clusterName resourceName partitionNumber 
stateModelName\>_ | Add a new resource to a cluster |
+| _\-\-addResourceProperty \<clusterName resourceName propertyName 
propertyValue\>_ | Add a resource property |
+| _\-\-addStateModelDef \<clusterName fileName.json\>_ | Add a State model 
definition to a cluster |
+| _\-\-dropCluster \<clusterName\>_ | Delete a cluster |
+| _\-\-dropNode \<clusterName instanceId\>_ | Remove a node from a cluster |
+| _\-\-dropResource \<clusterName resourceName\>_ | Remove an existing 
resource from a cluster |
+| _\-\-enableCluster \<clusterName true/false\>_ | Enable/disable a cluster |
+| _\-\-enableInstance \<clusterName instanceId true/false\>_ | Enable/disable 
an instance |
+| _\-\-enablePartition \<true/false clusterName nodeId resourceName 
partitionName\>_ | Enable/disable a partition |
+| _\-\-getConfig \<configScope configScopeArgs configKeys\>_ | Get user 
configs |
+| _\-\-getConstraints \<clusterName constraintType\>_ | Get constraints |
+| _\-\-help_ | print help information |
+| _\-\-instanceGroupTag \<instanceTag\>_ | Specify instance group tag, used 
with rebalance command |
+| _\-\-listClusterInfo \<clusterName\>_ | Show information of a cluster |
+| _\-\-listClusters_ | List all clusters |
+| _\-\-listInstanceInfo \<clusterName instanceId\>_ | Show information of an 
instance |
+| _\-\-listInstances \<clusterName\>_ | List all instances in a cluster |
+| _\-\-listPartitionInfo \<clusterName resourceName partitionName\>_ | Show 
information of a partition |
+| _\-\-listResourceInfo \<clusterName resourceName\>_ | Show information of a 
resource |
+| _\-\-listResources \<clusterName\>_ | List all resources in a cluster |
+| _\-\-listStateModel \<clusterName stateModelName\>_ | Show information of a 
state model |
+| _\-\-listStateModels \<clusterName\>_ | List all state models in a cluster |
+| _\-\-maxPartitionsPerNode \<maxPartitionsPerNode\>_ | Specify the max 
partitions per instance, used with addResourceGroup command |
+| _\-\-rebalance \<clusterName resourceName replicas\>_ | Rebalance a resource 
|
+| _\-\-removeConfig \<configScope configScopeArgs configKeys\>_ | Remove user 
configs |
+| _\-\-removeConstraint \<clusterName constraintType constraintId\>_ | Remove 
a constraint |
+| _\-\-removeInstanceTag \<clusterName instanceId tag\>_ | Remove a tag from 
an instance |
+| _\-\-removeResourceProperty \<clusterName resourceName propertyName\>_ | 
Remove a resource property |
+| _\-\-resetInstance \<clusterName instanceId\>_ | Reset all erroneous 
partitions on an instance |
+| _\-\-resetPartition \<clusterName instanceId resourceName partitionName\>_ | 
Reset an erroneous partition |
+| _\-\-resetResource \<clusterName resourceName\>_ | Reset all erroneous 
partitions of a resource |
+| _\-\-setConfig \<configScope configScopeArgs configKeyValueMap\>_ | Set user 
configs |
+| _\-\-setConstraint \<clusterName constraintType constraintId 
constraintKeyValueMap\>_ | Set a constraint |
+| _\-\-swapInstance \<clusterName oldInstance newInstance\>_ | Swap an old 
instance with a new instance |
+| _\-\-zkSvr \<ZookeeperServerAddress\>_ | Provide zookeeper address |
+
+### REST Interface
+
+The REST interface comes wit helix-admin-webapp package:
+
+```
+git clone https://git-wip-us.apache.org/repos/asf/helix.git
+cd helix
+git checkout tags/helix-0.8.3
+./build
+cd helix-admin-webapp/target/helix-admin-webapp-pkg/bin
+chmod +x *.sh
+./run-rest-admin.sh --zkSvr <zookeeperAddress> --port <port> // make sure 
ZooKeeper is running
+```
+
+#### URL and support methods
+
+* _/clusters_
+    * List all clusters
+
+    ```
+    curl http://localhost:8100/clusters
+    ```
+
+    * Add a cluster
+
+    ```
+    curl -d 
'jsonParameters={"command":"addCluster","clusterName":"MyCluster"}' -H 
"Content-Type: application/json" http://localhost:8100/clusters
+    ```
+
+* _/clusters/{clusterName}_
+    * List cluster information
+
+    ```
+    curl http://localhost:8100/clusters/MyCluster
+    ```
+
+    * Enable/disable a cluster in distributed controller mode
+
+    ```
+    curl -d 
'jsonParameters={"command":"activateCluster","grandCluster":"MyControllerCluster","enabled":"true"}'
 -H "Content-Type: application/json" http://localhost:8100/clusters/MyCluster
+    ```
+
+    * Remove a cluster
+
+    ```
+    curl -X DELETE http://localhost:8100/clusters/MyCluster
+    ```
+
+* _/clusters/{clusterName}/resourceGroups_
+    * List all resources in a cluster
+
+    ```
+    curl http://localhost:8100/clusters/MyCluster/resourceGroups
+    ```
+
+    * Add a resource to cluster
+
+    ```
+    curl -d 
'jsonParameters={"command":"addResource","resourceGroupName":"MyDB","partitions":"8","stateModelDefRef":"MasterSlave"
 }' -H "Content-Type: application/json" 
http://localhost:8100/clusters/MyCluster/resourceGroups
+    ```
+
+* _/clusters/{clusterName}/resourceGroups/{resourceName}_
+    * List resource information
+
+    ```
+    curl http://localhost:8100/clusters/MyCluster/resourceGroups/MyDB
+    ```
+
+    * Drop a resource
+
+    ```
+    curl -X DELETE http://localhost:8100/clusters/MyCluster/resourceGroups/MyDB
+    ```
+
+    * Reset all erroneous partitions of a resource
+
+    ```
+    curl -d 'jsonParameters={"command":"resetResource"}' -H "Content-Type: 
application/json" http://localhost:8100/clusters/MyCluster/resourceGroups/MyDB
+    ```
+
+* _/clusters/{clusterName}/resourceGroups/{resourceName}/idealState_
+    * Rebalance a resource
+
+    ```
+    curl -d 'jsonParameters={"command":"rebalance","replicas":"3"}' -H 
"Content-Type: application/json" 
http://localhost:8100/clusters/MyCluster/resourceGroups/MyDB/idealState
+    ```
+
+    * Add an ideal state
+
+    ```
+    echo jsonParameters={
+    "command":"addIdealState"
+       }&newIdealState={
+      "id" : "MyDB",
+      "simpleFields" : {
+        "IDEAL_STATE_MODE" : "AUTO",
+        "NUM_PARTITIONS" : "8",
+        "REBALANCE_MODE" : "SEMI_AUTO",
+        "REPLICAS" : "0",
+        "STATE_MODEL_DEF_REF" : "MasterSlave",
+        "STATE_MODEL_FACTORY_NAME" : "DEFAULT"
+      },
+      "listFields" : {
+      },
+      "mapFields" : {
+        "MyDB_0" : {
+          "localhost_1001" : "MASTER",
+          "localhost_1002" : "SLAVE"
+        }
+      }
+    }
+    > newIdealState.json
+    curl -d @'./newIdealState.json' -H 'Content-Type: application/json' 
http://localhost:8100/clusters/MyCluster/resourceGroups/MyDB/idealState
+    ```
+
+    * Add resource property
+
+    ```
+    curl -d 
'jsonParameters={"command":"addResourceProperty","REBALANCE_TIMER_PERIOD":"500"}'
 -H "Content-Type: application/json" 
http://localhost:8100/clusters/MyCluster/resourceGroups/MyDB/idealState
+    ```
+
+* _/clusters/{clusterName}/resourceGroups/{resourceName}/externalView_
+    * Show resource external view
+
+    ```
+    curl 
http://localhost:8100/clusters/MyCluster/resourceGroups/MyDB/externalView
+    ```
+* _/clusters/{clusterName}/instances_
+    * List all instances
+
+    ```
+    curl http://localhost:8100/clusters/MyCluster/instances
+    ```
+
+    * Add an instance
+
+    ```
+    curl -d 
'jsonParameters={"command":"addInstance","instanceNames":"localhost_1001"}' -H 
"Content-Type: application/json" 
http://localhost:8100/clusters/MyCluster/instances
+    ```
+
+    * Swap an instance
+
+    ```
+    curl -d 
'jsonParameters={"command":"swapInstance","oldInstance":"localhost_1001", 
"newInstance":"localhost_1002"}' -H "Content-Type: application/json" 
http://localhost:8100/clusters/MyCluster/instances
+    ```
+* _/clusters/{clusterName}/instances/{instanceName}_
+    * Show instance information
+
+    ```
+    curl http://localhost:8100/clusters/MyCluster/instances/localhost_1001
+    ```
+
+    * Enable/disable an instance
+
+    ```
+    curl -d 'jsonParameters={"command":"enableInstance","enabled":"false"}' -H 
"Content-Type: application/json" 
http://localhost:8100/clusters/MyCluster/instances/localhost_1001
+    ```
+
+    * Drop an instance
+
+    ```
+    curl -X DELETE 
http://localhost:8100/clusters/MyCluster/instances/localhost_1001
+    ```
+
+    * Disable/enable partitions on an instance
+
+    ```
+    curl -d 'jsonParameters={"command":"enablePartition","resource": 
"MyDB","partition":"MyDB_0",  "enabled" : "false"}' -H "Content-Type: 
application/json" 
http://localhost:8100/clusters/MyCluster/instances/localhost_1001
+    ```
+
+    * Reset an erroneous partition on an instance
+
+    ```
+    curl -d 'jsonParameters={"command":"resetPartition","resource": 
"MyDB","partition":"MyDB_0"}' -H "Content-Type: application/json" 
http://localhost:8100/clusters/MyCluster/instances/localhost_1001
+    ```
+
+    * Reset all erroneous partitions on an instance
+
+    ```
+    curl -d 'jsonParameters={"command":"resetInstance"}' -H "Content-Type: 
application/json" 
http://localhost:8100/clusters/MyCluster/instances/localhost_1001
+    ```
+
+* _/clusters/{clusterName}/configs_
+    * Get user cluster level config
+
+    ```
+    curl http://localhost:8100/clusters/MyCluster/configs/cluster
+    ```
+
+    * Set user cluster level config
+
+    ```
+    curl -d 
'jsonParameters={"command":"setConfig","configs":"key1=value1,key2=value2"}' -H 
"Content-Type: application/json" 
http://localhost:8100/clusters/MyCluster/configs/cluster
+    ```
+
+    * Remove user cluster level config
+
+    ```
+    curl -d 'jsonParameters={"command":"removeConfig","configs":"key1,key2"}' 
-H "Content-Type: application/json" 
http://localhost:8100/clusters/MyCluster/configs/cluster
+    ```
+
+    * Get/set/remove user participant level config
+
+    ```
+    curl -d 
'jsonParameters={"command":"setConfig","configs":"key1=value1,key2=value2"}' -H 
"Content-Type: application/json" 
http://localhost:8100/clusters/MyCluster/configs/participant/localhost_1001
+    ```
+
+    * Get/set/remove resource level config
+
+    ```
+    curl -d 
'jsonParameters={"command":"setConfig","configs":"key1=value1,key2=value2"}' -H 
"Content-Type: application/json" 
http://localhost:8100/clusters/MyCluster/configs/resource/MyDB
+    ```
+
+* _/clusters/{clusterName}/controller_
+    * Show controller information
+
+    ```
+    curl http://localhost:8100/clusters/MyCluster/Controller
+    ```
+
+    * Enable/disable cluster
+
+    ```
+    curl -d 'jsonParameters={"command":"enableCluster","enabled":"false"}' -H 
"Content-Type: application/json" 
http://localhost:8100/clusters/MyCluster/Controller
+    ```
+
+* _/zkPath/{path}_
+    * Get information for zookeeper path
+
+    ```
+    curl http://localhost:8100/zkPath/MyCluster
+    ```
+
+* _/clusters/{clusterName}/StateModelDefs_
+    * Show all state model definitions
+
+    ```
+    curl http://localhost:8100/clusters/MyCluster/StateModelDefs
+    ```
+
+    * Add a state mdoel definition
+
+    ```
+    echo jsonParameters={
+      "command":"addStateModelDef"
+    }&newStateModelDef={
+      "id" : "OnlineOffline",
+      "simpleFields" : {
+        "INITIAL_STATE" : "OFFLINE"
+      },
+      "listFields" : {
+        "STATE_PRIORITY_LIST" : [ "ONLINE", "OFFLINE", "DROPPED" ],
+        "STATE_TRANSITION_PRIORITYLIST" : [ "OFFLINE-ONLINE", 
"ONLINE-OFFLINE", "OFFLINE-DROPPED" ]
+      },
+      "mapFields" : {
+        "DROPPED.meta" : {
+          "count" : "-1"
+        },
+        "OFFLINE.meta" : {
+          "count" : "-1"
+        },
+        "OFFLINE.next" : {
+          "DROPPED" : "DROPPED",
+          "ONLINE" : "ONLINE"
+        },
+        "ONLINE.meta" : {
+          "count" : "R"
+        },
+        "ONLINE.next" : {
+          "DROPPED" : "OFFLINE",
+          "OFFLINE" : "OFFLINE"
+        }
+      }
+    }
+    > newStateModelDef.json
+    curl -d @'./untitled.txt' -H 'Content-Type: application/json' 
http://localhost:8100/clusters/MyCluster/StateModelDefs
+    ```
+
+* _/clusters/{clusterName}/StateModelDefs/{stateModelDefName}_
+    * Show a state model definition
+
+    ```
+    curl http://localhost:8100/clusters/MyCluster/StateModelDefs/OnlineOffline
+    ```
+
+* _/clusters/{clusterName}/constraints/{constraintType}_
+    * Show all contraints
+
+    ```
+    curl 
http://localhost:8100/clusters/MyCluster/constraints/MESSAGE_CONSTRAINT
+    ```
+
+    * Set a contraint
+
+    ```
+    curl -d 
'jsonParameters={"constraintAttributes":"RESOURCE=MyDB,CONSTRAINT_VALUE=1"}' -H 
"Content-Type: application/json" 
http://localhost:8100/clusters/MyCluster/constraints/MESSAGE_CONSTRAINT/MyConstraint
+    ```
+
+    * Remove a constraint
+
+    ```
+    curl -X DELETE 
http://localhost:8100/clusters/MyCluster/constraints/MESSAGE_CONSTRAINT/MyConstraint
+    ```

http://git-wip-us.apache.org/repos/asf/helix/blob/e8125e8b/website/0.8.3/src/site/markdown/tutorial_agent.md
----------------------------------------------------------------------
diff --git a/website/0.8.3/src/site/markdown/tutorial_agent.md 
b/website/0.8.3/src/site/markdown/tutorial_agent.md
new file mode 100644
index 0000000..4d5965b
--- /dev/null
+++ b/website/0.8.3/src/site/markdown/tutorial_agent.md
@@ -0,0 +1,169 @@
+<!---
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License.
+-->
+
+<head>
+  <title>Tutorial - Helix Agent</title>
+</head>
+
+## [Helix Tutorial](./Tutorial.html): Helix Agent (for non-JVM systems)
+
+Not every distributed system is written on the JVM, but many systems would 
benefit from the cluster management features that Helix provides. To make a 
non-JVM system work with Helix, you can use the Helix Agent module.
+
+### What is Helix Agent?
+
+Helix is built on the following assumption: if your distributed resource is 
modeled by a finite state machine, then Helix can tell participants when they 
should transition between states. In the Java API, this means implementing 
transition callbacks. In the Helix agent API, this means providing commands 
than can run for each transition.
+
+These commands could do anything behind the scenes; Helix only requires that 
they exit once the state transition is complete.
+
+### Configuring Transition Commands
+
+Here's how to tell Helix which commands to run on state transitions:
+
+#### Java
+
+Using the Java API, first get a configuration scope (the Helix agent supports 
both cluster and resource scopes, picking resource first if it is available):
+
+```
+// Cluster scope
+HelixConfigScope scope =
+    new 
HelixConfigScopeBuilder(ConfigScopeProperty.CLUSTER).forCluster(clusterName).build();
+
+// Resource scope
+HelixConfigScope scope =
+    new 
HelixConfigScopeBuilder(ConfigScopeProperty.RESOURCE).forCluster(clusterName).forResource(resourceName).build();
+```
+
+Then, specify the command to run for each state transition:
+
+```
+// Get the configuration accessor
+ConfigAccessor configAccessor = new ConfigAccessor(_gZkClient);
+
+// Specify the script for OFFLINE --> ONLINE
+CommandConfig.Builder builder = new CommandConfig.Builder();
+CommandConfig cmdConfig =
+    builder.setTransition("OFFLINE", "ONLINE").setCommand("simpleHttpClient.py 
OFFLINE-ONLINE")
+        .setCommandWorkingDir(workingDir)
+        .setCommandTimeout("5000L") // optional: ms to wait before failing
+        .setPidFile(pidFile) // optional: for daemon-like systems that will 
write the process id to a file
+        .build();
+configAccessor.set(scope, cmdConfig.toKeyValueMap());
+
+// Specify the script for ONLINE --> OFFLINE
+builder = new CommandConfig.Builder();
+cmdConfig =
+    builder.setTransition("ONLINE", "OFFLINE").setCommand("simpleHttpClient.py 
ONLINE-OFFLINE")
+        .setCommandWorkingDir(workingDir)
+        .build();
+configAccessor.set(scope, cmdConfig.toKeyValueMap());
+
+// Specify NOP for OFFLINE --> DROPPED
+builder = new CommandConfig.Builder();
+cmdConfig =
+    builder.setTransition("OFFLINE", "DROPPED")
+        .setCommand(CommandAttribute.NOP.getName())
+        .build();
+configAccessor.set(scope, cmdConfig.toKeyValueMap());
+```
+
+In this example, we have a program called simpleHttpClient.py that we call for 
all transitions, only changing the arguments that are passed in. However, there 
is no requirement that each transition invoke the same program; this API allows 
running arbitrary commands in arbitrary directories with arbitrary arguments.
+
+Notice that that for the OFFLINE \-\-\> DROPPED transition, we do not run any 
command (specifically, we specify the NOP command). This just tells Helix that 
the system doesn't care about when things are dropped, and it can consider the 
transition already done.
+
+#### Command Line
+
+It is also possible to configure everything directly from the command line. 
Here's how that would look for cluster-wide configuration:
+
+```
+# Specify the script for OFFLINE --> ONLINE
+/helix-admin.sh --zkSvr localhost:2181 --setConfig CLUSTER clusterName 
OFFLINE-ONLINE.command="simpleHttpClient.py 
OFFLINE-ONLINE",OFFLINE-ONLINE.workingDir="/path/to/script", 
OFFLINE-ONLINE.command.pidfile="/path/to/pidfile"
+
+# Specify the script for ONLINE --> OFFLINE
+/helix-admin.sh --zkSvr localhost:2181 --setConfig CLUSTER clusterName 
ONLINE-OFFLINE.command="simpleHttpClient.py 
ONLINE-OFFLINE",ONLINE-OFFLINE.workingDir="/path/to/script", 
OFFLINE-ONLINE.command.pidfile="/path/to/pidfile"
+
+# Specify NOP for OFFLINE --> DROPPED
+/helix-admin.sh --zkSvr localhost:2181 --setConfig CLUSTER clusterName 
ONLINE-OFFLINE.command="nop"
+```
+
+Like in the Java configuration, it is also possible to specify a resource 
scope instead of a cluster scope:
+
+```
+# Specify the script for OFFLINE --> ONLINE
+/helix-admin.sh --zkSvr localhost:2181 --setConfig RESOURCE 
clusterName,resourceName OFFLINE-ONLINE.command="simpleHttpClient.py 
OFFLINE-ONLINE",OFFLINE-ONLINE.workingDir="/path/to/script", 
OFFLINE-ONLINE.command.pidfile="/path/to/pidfile"
+```
+
+### Starting the Agent
+
+There should be an agent running for every participant you have running. 
Ideally, its lifecycle should match that of the participant. Here, we have a 
simple long-running participant called simpleHttpServer.py. Its only purpose is 
to record state transitions.
+
+Here are some ways that you can start the Helix agent:
+
+#### Java
+
+```
+// Start your application process
+ExternalCommand serverCmd = ExternalCommand.start(workingDir + 
"/simpleHttpServer.py");
+
+// Start the agent
+Thread agentThread = new Thread() {
+  @Override
+  public void run() {
+    while(!isInterrupted()) {
+      try {
+        HelixAgentMain.main(new String[] {
+            "--zkSvr", zkAddr, "--cluster", clusterName, "--instanceName", 
instanceName,
+            "--stateModel", "OnlineOffline"
+        });
+      } catch (InterruptedException e) {
+        LOG.info("Agent thread interrupted", e);
+        interrupt();
+      } catch (Exception e) {
+        LOG.error("Exception start helix-agent", e);
+      }
+    }
+  }
+};
+agentThread.start();
+
+// Wait for the process to terminate (either intentionally or unintentionally)
+serverCmd.waitFor();
+
+// Kill the agent
+agentThread.interrupt();
+```
+
+#### Command Line
+
+```
+# Build Helix and start the agent
+mvn clean install -DskipTests
+chmod +x helix-agent/target/helix-agent-pkg/bin/*
+helix-agent/target/helix-agent-pkg/bin/start-helix-agent.sh --zkSvr 
zkAddr1,zkAddr2 --cluster clusterName --instanceName instanceName --stateModel 
OnlineOffline
+
+# Here, you can define your own logic to terminate this agent when your 
process terminates
+...
+```
+
+### Example
+
+[Here](https://git-wip-us.apache.org/repos/asf?p=helix.git;a=blob;f=helix-agent/src/test/java/org/apache/helix/agent/TestHelixAgent.java;h=ccf64ce5544207c7e48261682ea69945b71da7f1;hb=refs/heads/master)
 is a basic system that uses the Helix agent package.
+
+### Notes
+
+As you may have noticed from the examples, the participant program and the 
state transition program are two different programs. The former is a 
_long-running_ process that is directly tied to the Helix agent. The latter is 
a process that only exists while a state transition is underway. Despite this, 
these two processes should be intertwined. The transition command will need to 
communicate to the participant to actually complete the state transition and 
the participant will need to communicate whether or not this was successful. 
The implementation of this protocol is the responsibility of the system.

Reply via email to