You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@helix.apache.org by ki...@apache.org on 2013/11/20 22:12:51 UTC

[36/52] [abbrv] [HELIX-270] Include documentation for previous version on the website

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/150ce693/site-releases/trunk/src/site/markdown/Tutorial.md
----------------------------------------------------------------------
diff --git a/site-releases/trunk/src/site/markdown/Tutorial.md b/site-releases/trunk/src/site/markdown/Tutorial.md
new file mode 100644
index 0000000..ee5a393
--- /dev/null
+++ b/site-releases/trunk/src/site/markdown/Tutorial.md
@@ -0,0 +1,284 @@
+<!---
+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</title>
+</head>
+
+# Helix Tutorial
+
+In this tutorial, we will cover the roles of a Helix-managed cluster, and show the code you need to write to integrate with it.  In many cases, there is a simple default behavior that is often appropriate, but you can also customize the behavior.
+
+Convention: we first cover the _basic_ approach, which is the easiest to implement.  Then, we'll describe _advanced_ options, which give you more control over the system behavior, but require you to write more code.
+
+
+### Prerequisites
+
+1. Read [Concepts/Terminology](./Concepts.html) and [Architecture](./Architecture.html)
+2. Read the [Quickstart guide](./Quickstart.html) to learn how Helix models and manages a cluster
+3. Install Helix source.  See: [Quickstart](./Quickstart.html) for the steps.
+
+### Tutorial Outline
+
+1. [Participant](./tutorial_participant.html)
+2. [Spectator](./tutorial_spectator.html)
+3. [Controller](./tutorial_controller.html)
+4. [Rebalancing Algorithms](./tutorial_rebalance.html)
+5. [User-Defined Rebalancing](./tutorial_user_def_rebalancer.html)
+6. [State Machines](./tutorial_state.html)
+7. [Messaging](./tutorial_messaging.html)
+8. [Customized health check](./tutorial_health.html)
+9. [Throttling](./tutorial_throttling.html)
+10. [Application Property Store](./tutorial_propstore.html)
+11. [Logical Accessors](./tutorial_accessors.html)
+12. [Admin Interface](./tutorial_admin.html)
+13. [YAML Cluster Setup](./tutorial_yaml.html)
+
+### Preliminaries
+
+First, we need to set up the system.  Let\'s walk through the steps in building a distributed system using Helix. We will show how to do this using both the Java admin interface, as well as the [cluster accessor](./tutorial_accessors.html) interface. You can choose either interface depending on which most closely matches your needs.
+
+### Start Zookeeper
+
+This starts a zookeeper in standalone mode. For production deployment, see [Apache Zookeeper](http://zookeeper.apache.org) for instructions.
+
+```
+    ./start-standalone-zookeeper.sh 2199 &
+```
+
+### Create a cluster
+
+Creating a cluster will define the cluster in appropriate znodes on zookeeper.   
+
+Using the Java accessor API:
+
+```
+// Note: ZK_ADDRESS is the host:port of Zookeeper
+String ZK_ADDRESS = "localhost:2199";
+HelixConnection connection = new ZKHelixConnection(ZK_ADDRESS);
+
+ClusterId clusterId = ClusterId.from("helix-demo");
+ClusterAccessor clusterAccessor = connection.createClusterAccessor(clusterId);
+ClusterConfig clusterConfig = new ClusterConfig.Builder(clusterId).build();
+clusterAccessor.createCluster(clusterConfig);
+```
+
+OR
+
+Using the HelixAdmin Java interface:
+
+```
+// Create setup tool instance
+// Note: ZK_ADDRESS is the host:port of Zookeeper
+String ZK_ADDRESS = "localhost:2199";
+HelixAdmin admin = new ZKHelixAdmin(ZK_ADDRESS);
+
+String CLUSTER_NAME = "helix-demo";
+//Create cluster namespace in zookeeper
+admin.addCluster(CLUSTER_NAME);
+```
+
+OR
+
+Using the command-line interface:
+
+```
+    ./helix-admin.sh --zkSvr localhost:2199 --addCluster helix-demo 
+```
+
+
+### Configure the nodes of the cluster
+
+First we\'ll add new nodes to the cluster, then configure the nodes in the cluster. Each node in the cluster must be uniquely identifiable. 
+The most commonly used convention is hostname_port.
+
+```
+int NUM_NODES = 2;
+String hosts[] = new String[]{"localhost","localhost"};
+int ports[] = new int[]{7000,7001};
+for (int i = 0; i < NUM_NODES; i++)
+{
+  ParticipantId participantId = ParticipantId.from(hosts[i] + "_" + ports[i]);
+
+  // set additional configuration for the participant; these can be accessed during node start up
+  UserConfig userConfig = new UserConfig(Scope.participant(participantId));
+  userConfig.setSimpleField("key", "value");
+
+  // configure and add the participant
+  ParticipantConfig participantConfig = new ParticipantConfig.Builder(participantId)
+      .hostName(hosts[i]).port(ports[i]).enabled(true).userConfig(userConfig).build();
+  clusterAccessor.addParticipantToCluster(participantConfig);
+}
+```
+
+OR
+
+Using the HelixAdmin Java interface:
+
+```
+String CLUSTER_NAME = "helix-demo";
+int NUM_NODES = 2;
+String hosts[] = new String[]{"localhost","localhost"};
+String ports[] = new String[]{7000,7001};
+for (int i = 0; i < NUM_NODES; i++)
+{
+  InstanceConfig instanceConfig = new InstanceConfig(hosts[i] + "_" + ports[i]);
+  instanceConfig.setHostName(hosts[i]);
+  instanceConfig.setPort(ports[i]);
+  instanceConfig.setInstanceEnabled(true);
+
+  //Add additional system specific configuration if needed. These can be accessed during the node start up.
+  instanceConfig.getRecord().setSimpleField("key", "value");
+  admin.addInstance(CLUSTER_NAME, instanceConfig);
+}
+```
+
+### Configure the resource
+
+A _resource_ represents the actual task performed by the nodes. It can be a database, index, topic, queue or any other processing entity.
+A _resource_ can be divided into many sub-parts known as _partitions_.
+
+
+#### Define the _state model_ and _constraints_
+
+For scalability and fault tolerance, each partition can have one or more replicas. 
+The _state model_ allows one to declare the system behavior by first enumerating the various STATES, and the TRANSITIONS between them.
+A simple model is ONLINE-OFFLINE where ONLINE means the task is active and OFFLINE means it\'s not active.
+You can also specify how many replicas must be in each state, these are known as _constraints_.
+For example, in a search system, one might need more than one node serving the same index to handle the load.
+
+The allowed states: 
+
+* MASTER
+* SLAVE
+* OFFLINE
+
+The allowed transitions: 
+
+* OFFLINE to SLAVE
+* SLAVE to OFFLINE
+* SLAVE to MASTER
+* MASTER to SLAVE
+
+The constraints:
+
+* no more than 1 MASTER per partition
+* the rest of the replicas should be slaves
+
+The following snippet shows how to declare the _state model_ and _constraints_ for the MASTER-SLAVE model.
+
+```
+StateModelDefinition.Builder builder = new StateModelDefinition.Builder(STATE_MODEL_NAME);
+
+// Add states and their rank to indicate priority. A lower rank corresponds to a higher priority
+builder.addState(MASTER, 1);
+builder.addState(SLAVE, 2);
+builder.addState(OFFLINE);
+
+// Set the initial state when the node starts
+builder.initialState(OFFLINE);
+
+// Add transitions between the states.
+builder.addTransition(OFFLINE, SLAVE);
+builder.addTransition(SLAVE, OFFLINE);
+builder.addTransition(SLAVE, MASTER);
+builder.addTransition(MASTER, SLAVE);
+
+// set constraints on states.
+
+// static constraint: upper bound of 1 MASTER
+builder.upperBound(MASTER, 1);
+
+// dynamic constraint: R means it should be derived based on the replication factor for the cluster
+// this allows a different replication factor for each resource without 
+// having to define a new state model
+//
+builder.dynamicUpperBound(SLAVE, "R");
+StateModelDefinition statemodelDefinition = builder.build();
+```
+
+Then, add the state model definition:
+
+```
+clusterAccessor.addStateModelDefinitionToCluster(stateModelDefinition);
+```
+
+OR
+
+```
+admin.addStateModelDef(CLUSTER_NAME, STATE_MODEL_NAME, stateModelDefinition);
+```
+
+#### Assigning partitions to nodes
+
+The final goal of Helix is to ensure that the constraints on the state model are satisfied. 
+Helix does this by assigning a STATE to a partition (such as MASTER, SLAVE), and placing it on a particular node.
+
+There are 3 assignment modes Helix can operate on
+
+* FULL_AUTO: Helix decides the placement and state of a partition.
+* SEMI_AUTO: Application decides the placement but Helix decides the state of a partition.
+* CUSTOMIZED: Application controls the placement and state of a partition.
+
+For more info on the assignment modes, see [Rebalancing Algorithms](./tutorial_rebalance.html) section of the tutorial.
+
+Here is an example of adding the resource in SEMI_AUTO mode (i.e. locations of partitions are specified a priori):
+
+```
+int NUM_PARTITIONS = 6;
+int NUM_REPLICAS = 2;
+ResourceId resourceId = resourceId.from("MyDB");
+
+SemiAutoRebalancerContext context = new SemiAutoRebalancerContext.Builder(resourceId)
+  .replicaCount(NUM_REPLICAS).addPartitions(NUM_PARTITIONS)
+  .stateModelDefId(stateModelDefinition.getStateModelDefId())
+  .addPreferenceList(partition1Id, preferenceList) // preferred locations of each partition
+  // add other preference lists per partition
+  .build();
+
+// or add all preference lists at once if desired (map of PartitionId to List of ParticipantId)
+context.setPreferenceLists(preferenceLists);
+
+// or generate a default set of preference lists given the set of all participants
+context.generateDefaultConfiguration(stateModelDefinition, participantIdSet);
+```
+
+OR
+
+```
+String RESOURCE_NAME = "MyDB";
+int NUM_PARTITIONS = 6;
+String MODE = "SEMI_AUTO";
+int NUM_REPLICAS = 2;
+
+admin.addResource(CLUSTER_NAME, RESOURCE_NAME, NUM_PARTITIONS, STATE_MODEL_NAME, MODE);
+
+// specify the preference lists yourself
+IdealState idealState = admin.getResourceIdealState(CLUSTER_NAME, RESOURCE_NAME);
+idealState.setPreferenceList(partitionId, preferenceList); // preferred locations of each partition
+// add other preference lists per partition
+
+// or add all preference lists at once if desired
+idealState.getRecord().setListFields(preferenceLists);
+admin.setResourceIdealState(CLUSTER_NAME, RESOURCE_NAME, idealState);
+
+// or generate a default set of preference lists 
+admin.rebalance(CLUSTER_NAME, RESOURCE_NAME, NUM_REPLICAS);
+```
+

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/150ce693/site-releases/trunk/src/site/markdown/UseCases.md
----------------------------------------------------------------------
diff --git a/site-releases/trunk/src/site/markdown/UseCases.md b/site-releases/trunk/src/site/markdown/UseCases.md
new file mode 100644
index 0000000..001b012
--- /dev/null
+++ b/site-releases/trunk/src/site/markdown/UseCases.md
@@ -0,0 +1,113 @@
+<!---
+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>Use Cases</title>
+</head>
+
+
+# Use cases at LinkedIn
+
+At LinkedIn Helix framework is used to manage 3 distributed data systems which are quite different from each other.
+
+* Espresso
+* Databus
+* Search As A Service
+
+## Espresso
+
+Espresso is a distributed, timeline consistent, scal- able, document store that supports local secondary indexing and local transactions. 
+Espresso databases are horizontally partitioned into a number of partitions, with each partition having a certain number of replicas 
+distributed across the storage nodes.
+Espresso designates one replica of each partition as master and the rest as slaves; only one master may exist for each partition at any time.
+Espresso enforces timeline consistency where only the master of a partition can accept writes to its records, and all slaves receive and 
+apply the same writes through a replication stream. 
+For load balancing, both master and slave partitions are assigned evenly across all storage nodes. 
+For fault tolerance, it adds the constraint that no two replicas of the same partition may be located on the same node.
+
+### State model
+Espresso follows a Master-Slave state model. A replica can be in Offline,Slave or Master state. 
+The state machine table describes the next state given the Current State, Final State
+
+```
+          OFFLINE  | SLAVE  |  MASTER  
+         _____________________________
+        |          |        |         |
+OFFLINE |   N/A    | SLAVE  | SLAVE   |
+        |__________|________|_________|
+        |          |        |         |
+SLAVE   |  OFFLINE |   N/A  | MASTER  |
+        |__________|________|_________|
+        |          |        |         |
+MASTER  | SLAVE    | SLAVE  |   N/A   |
+        |__________|________|_________|
+
+```
+
+### Constraints
+* Max number of replicas in Master state:1
+* Execution mode AUTO. i.e on node failure no new replicas will be created. Only the State of remaining replicas will be changed.
+* Number of mastered partitions on each node must be approximately same.
+* The above constraint must be satisfied when a node fails or a new node is added.
+* When new nodes are added the number of partitions moved must be minimized.
+* When new nodes are added the max number of OFFLINE-SLAVE transitions that can happen concurrently on new node is X.
+
+## Databus
+
+Databus is a change data capture (CDC) system that provides a common pipeline for transporting events 
+from LinkedIn primary databases to caches within various applications.
+Databus deploys a cluster of relays that pull the change log from multiple databases and 
+let consumers subscribe to the change log stream. Each Databus relay connects to one or more database servers and 
+hosts a certain subset of databases (and partitions) from those database servers. 
+
+For a large partitioned database (e.g. Espresso), the change log is consumed by a bank of consumers. 
+Each databus partition is assigned to a consumer such that partitions are evenly distributed across consumers and each partition is
+assigned to exactly one consumer at a time. The set of consumers may grow over time, and consumers may leave the group due to planned or unplanned 
+outages. In these cases, partitions must be reassigned, while maintaining balance and the single consumer-per-partition invariant.
+
+### State model
+Databus consumers follow a simple Offline-Online state model.
+The state machine table describes the next state given the Current State, Final State
+
+<pre><code>
+          OFFLINE  | ONLINE |   
+         ___________________|
+        |          |        |
+OFFLINE |   N/A    | ONLINE |
+        |__________|________|
+        |          |        |
+ONLINE  |  OFFLINE |   N/A  |
+        |__________|________|
+
+
+</code></pre>
+
+
+## Search As A Service
+
+LinkedIn�s Search-as-a-service lets internal customers define custom indexes on a chosen dataset 
+and then makes those indexes searchable via a service API. The index service runs on a cluster of machines. 
+The index is broken into partitions and each partition has a configured number of replicas.
+Each cluster server runs an instance of the Sensei system (an online index store) and hosts index partitions. 
+Each new indexing service gets assigned to a set of servers, and the partition replicas must be evenly distributed across those servers.
+
+### State model
+![Helix Design](images/bootstrap_statemodel.gif) 
+
+

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/150ce693/site-releases/trunk/src/site/markdown/index.md
----------------------------------------------------------------------
diff --git a/site-releases/trunk/src/site/markdown/index.md b/site-releases/trunk/src/site/markdown/index.md
new file mode 100644
index 0000000..2eae374
--- /dev/null
+++ b/site-releases/trunk/src/site/markdown/index.md
@@ -0,0 +1,56 @@
+<!---
+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>Home</title>
+</head>
+
+Navigating the Documentation
+----------------------------
+
+### Conceptual Understanding
+
+[Concepts / Terminology](./Concepts.html)
+
+[Architecture](./Architecture.html)
+
+### Hands-on Helix
+
+[Getting Helix](./Building.html)
+
+[Quickstart](./Quickstart.html)
+
+[Tutorial](./Tutorial.html)
+
+[Javadocs](http://helix.incubator.apache.org/apidocs)
+
+### 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)
+
+[User-Defined Rebalancer Example](./recipes/user_def_rebalancer.html)
+

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/150ce693/site-releases/trunk/src/site/markdown/recipes/lock_manager.md
----------------------------------------------------------------------
diff --git a/site-releases/trunk/src/site/markdown/recipes/lock_manager.md b/site-releases/trunk/src/site/markdown/recipes/lock_manager.md
new file mode 100644
index 0000000..252ace7
--- /dev/null
+++ b/site-releases/trunk/src/site/markdown/recipes/lock_manager.md
@@ -0,0 +1,253 @@
+<!---
+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 use Zookeeper to model the 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 can successfully create it then, it acquires the lock
+* Else it will watch on 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 lock. 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 dont 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 with in 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/incubator-helix.git
+cd incubator-helix
+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 FULL_AUTO
+```
+
+##### 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 );
+  }
+
+}
+
+```
+
+LockFactory that creates the lock
+ 
+```
+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 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
+
+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/incubator-helix/blob/150ce693/site-releases/trunk/src/site/markdown/recipes/rabbitmq_consumer_group.md
----------------------------------------------------------------------
diff --git a/site-releases/trunk/src/site/markdown/recipes/rabbitmq_consumer_group.md b/site-releases/trunk/src/site/markdown/recipes/rabbitmq_consumer_group.md
new file mode 100644
index 0000000..9edc2cb
--- /dev/null
+++ b/site-releases/trunk/src/site/markdown/recipes/rabbitmq_consumer_group.md
@@ -0,0 +1,227 @@
+<!---
+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 a 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 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 consumers 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 which reduces the number of active consumers to 2. This means each consumer must process 3 queues.
+
+We showcase how such a dynamic App 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/incubator-helix.git
+cd incubator-helix
+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 Rabbit MQ
+----------------
+
+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 HelixController
+--------------------
+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 it work
+================
+
+Find the entire code [here](https://git-wip-us.apache.org/repos/asf?p=incubator-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 FULL_AUTO. 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", "FULL_AUTO");
+```
+
+Starting the consumers
+----------------------
+The only thing consumers need to know is the zkaddress, 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 statemodel 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");
+
+    }
+  }
+```
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/150ce693/site-releases/trunk/src/site/markdown/recipes/rsync_replicated_file_store.md
----------------------------------------------------------------------
diff --git a/site-releases/trunk/src/site/markdown/recipes/rsync_replicated_file_store.md b/site-releases/trunk/src/site/markdown/recipes/rsync_replicated_file_store.md
new file mode 100644
index 0000000..f8a74a0
--- /dev/null
+++ b/site-releases/trunk/src/site/markdown/recipes/rsync_replicated_file_store.md
@@ -0,0 +1,165 @@
+<!---
+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 real time rsync replicated file system
+===========================================
+
+Quickdemo
+---------
+
+* 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 master are replicated to the slaves automatically. In this demo, a.txt and b.txt are written to ```/tmp/localhost_12001/filestore``` and it gets 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 appears in ```/tmp/localhost_12003/filestore```
+* Ignore the interrupted exceptions on the console :-).
+
+
+```
+git clone https://git-wip-us.apache.org/repos/asf/incubator-helix.git
+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 usecases is also that files are typically only added or deleted, rarely updated. When there are updates, they are rare and do not have any concurrency requirements.
+
+These are much simpler requirements than what general purpose distributed file system have to satisfy including concurrent access to files, random access for reads and updates, posix compliance etc. 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 the 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, 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.
+Sequence gets incremented on every transaction and and generation is increment when a new master is elected. 
+
+### Replication
+
+Replication is required to slave to keep up with the 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.
+
+
+### Fail over
+
+When a master fails, a new slave will be promoted to master. If the prev master node is reachable, then the new master will flush all the 
+changes from previous master before taking up mastership. The new master will record the end transaction id of the current generation and then starts 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 demonstrate a file store that uses rsync as the replication mechanism. One can envision a similar system where instead of using rsync, 
+can implement a custom solution to notify the slave of the changes and also provide an api to pull the change files.
+#### Concept
+* 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 support file uploads and downloads and writes the files to ```file_store_dir```. This is not included in this application. Idea is that most applications have different ways of implementing this component and has some business logic associated with it. 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 File System 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 notification 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/incubator-helix/blob/150ce693/site-releases/trunk/src/site/markdown/recipes/service_discovery.md
----------------------------------------------------------------------
diff --git a/site-releases/trunk/src/site/markdown/recipes/service_discovery.md b/site-releases/trunk/src/site/markdown/recipes/service_discovery.md
new file mode 100644
index 0000000..8e06ead
--- /dev/null
+++ b/site-releases/trunk/src/site/markdown/recipes/service_discovery.md
@@ -0,0 +1,191 @@
+<!---
+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 enable service discovery. 
+The basic idea is that when a server starts up it advertises its configuration/metadata such as host name port etc 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 watchevent 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 from zookeeper.
+
+
+#### 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 worst especially when servers are starting up. 
+But good thing about setting watch is that clients are immediately notified of a change which is not true in case of polling. 
+In some cases, having both WATCH and POLL 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 important scenarios to take care of
+* What happens when zookeeper session expires. All the watches/ephemeral nodes previously added/created by this server are lost. 
+One needs to add the watches again , recreate the ephemeral nodes etc.
+* Due to network issues or java GC pauses session expiry might happen again and again also known as flapping. Its important for the server to detect this and deregister itself.
+
+##### Other operational things to consider
+* What if the node is behaving badly, one might kill the server but 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. In common, service discovery pattern means that servers start up with some configuration and then simply puts its configuration/metadata in 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 of 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.
+
+h3. How to use Helix to achieve this
+
+Even though Helix has higher level abstraction in terms of statemachine, constraints and objectives, 
+service discovery is one of things that existed since we started. 
+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 ZK watch to track the liveness of a server.
+
+This recipe, simply demonstrate how one can re-use that part for implementing service discovery. This demonstrates multiple MODE's 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 else where
+
+* It has the concept of disabling a node which means that a badly behaving node, can be disabled using 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 admin api at various granulaties like cluster, instance, resource, partition 
+    * Configuration can be dynamically changed.
+    * Notifies the server when configuration changes.
+
+
+##### checkout and build
+
+```
+git clone https://git-wip-us.apache.org/repos/asf/incubator-helix.git
+cd incubator-helix
+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/incubator-helix/blob/150ce693/site-releases/trunk/src/site/markdown/recipes/task_dag_execution.md
----------------------------------------------------------------------
diff --git a/site-releases/trunk/src/site/markdown/recipes/task_dag_execution.md b/site-releases/trunk/src/site/markdown/recipes/task_dag_execution.md
new file mode 100644
index 0000000..f0474e4
--- /dev/null
+++ b/site-releases/trunk/src/site/markdown/recipes/task_dag_execution.md
@@ -0,0 +1,204 @@
+<!---
+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 desired parallelism and will start only when up-stream 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 DAG
+
+Each stage is represented as a Node along with the upstream dependency and desired parallelism.  Each stage is modelled as a resource in Helix using OnlineOffline state model. As part of Offline to Online transition, we watch the external view of upstream resources and wait for them to transition to 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/incubator-helix.git
+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 
+
+```
+
+```
+
+
+
+
+
+                       +-----------------+       +----------------+
+                       |   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/incubator-helix/blob/150ce693/site-releases/trunk/src/site/markdown/recipes/user_def_rebalancer.md
----------------------------------------------------------------------
diff --git a/site-releases/trunk/src/site/markdown/recipes/user_def_rebalancer.md b/site-releases/trunk/src/site/markdown/recipes/user_def_rebalancer.md
new file mode 100644
index 0000000..68fd954
--- /dev/null
+++ b/site-releases/trunk/src/site/markdown/recipes/user_def_rebalancer.md
@@ -0,0 +1,285 @@
+<!---
+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.
+-->
+Lock Manager with a User-Defined Rebalancer
+-------------------------------------------
+Helix is able to compute node preferences and state assignments automatically using general-purpose algorithms. In many cases, a distributed system implementer may choose to instead define a customized approach to computing the location of replicas, the state mapping, or both in response to the addition or removal of participants. The following is an implementation of the [Distributed Lock Manager](./lock_manager.html) that includes a user-defined rebalancer.
+
+### Define the cluster and locks
+
+The YAML file below fully defines the cluster and the locks. A lock can be in one of two states: locked and unlocked. Transitions can happen in either direction, and the locked is preferred. A resource in this example is the entire collection of locks to distribute. A partition is mapped to a lock; in this case that means there are 12 locks. These 12 locks will be distributed across 3 nodes. The constraints indicate that only one replica of a lock can be in the locked state at any given time. These locks can each only have a single holder, defined by a replica count of 1.
+
+Notice the rebalancer section of the definition. The mode is set to USER_DEFINED and the class name refers to the plugged-in rebalancer implementation that inherits from [HelixRebalancer](http://helix.incubator.apache.org/javadocs/0.7.0-incubating/reference/org/apache/helix/controller/rebalancer/HelixRebalancer.html). This implementation is called whenever the state of the cluster changes, as is the case when participants are added or removed from the system.
+
+Location: incubator-helix/recipes/user-rebalanced-lock-manager/src/main/resources/lock-manager-config.yaml
+
+```
+clusterName: lock-manager-custom-rebalancer # unique name for the cluster
+resources:
+  - name: lock-group # unique resource name
+    rebalancer: # we will provide our own rebalancer
+      mode: USER_DEFINED
+      class: org.apache.helix.userdefinedrebalancer.LockManagerRebalancer
+    partitions:
+      count: 12 # number of locks
+      replicas: 1 # number of simultaneous holders for each lock
+    stateModel:
+      name: lock-unlock # unique model name
+      states: [LOCKED, RELEASED, DROPPED] # the list of possible states
+      transitions: # the list of possible transitions
+        - name: Unlock
+          from: LOCKED
+          to: RELEASED
+        - name: Lock
+          from: RELEASED
+          to: LOCKED
+        - name: DropLock
+          from: LOCKED
+          to: DROPPED
+        - name: DropUnlock
+          from: RELEASED
+          to: DROPPED
+        - name: Undrop
+          from: DROPPED
+          to: RELEASED
+      initialState: RELEASED
+    constraints:
+      state:
+        counts: # maximum number of replicas of a partition that can be in each state
+          - name: LOCKED
+            count: "1"
+          - name: RELEASED
+            count: "-1"
+          - name: DROPPED
+            count: "-1"
+        priorityList: [LOCKED, RELEASED, DROPPED] # states in order of priority
+      transition: # transitions priority to enforce order that transitions occur
+        priorityList: [Unlock, Lock, Undrop, DropUnlock, DropLock]
+participants: # list of nodes that can acquire locks
+  - name: localhost_12001
+    host: localhost
+    port: 12001
+  - name: localhost_12002
+    host: localhost
+    port: 12002
+  - name: localhost_12003
+    host: localhost
+    port: 12003
+```
+
+Then, Helix\'s YAMLClusterSetup tool can read in the configuration and bootstrap the cluster immediately:
+
+```
+YAMLClusterSetup setup = new YAMLClusterSetup(zkAddress);
+InputStream input =
+    Thread.currentThread().getContextClassLoader()
+        .getResourceAsStream("lock-manager-config.yaml");
+YAMLClusterSetup.YAMLClusterConfig config = setup.setupCluster(input);
+```
+
+### Write a rebalancer
+Below is a full implementation of a rebalancer that extends [HelixRebalancer](http://helix.incubator.apache.org/javadocs/0.7.0-incubating/reference/org/apache/helix/controller/rebalancer/HelixRebalancer.html). In this case, it simply throws out the previous resource assignment, computes the target node for as many partition replicas as can hold a lock in the LOCKED state (in this example, one), and assigns them the LOCKED state (which is at the head of the state preference list). Clearly a more robust implementation would likely examine the current ideal state to maintain current assignments, and the full state list to handle models more complicated than this one. However, for a simple lock holder implementation, this is sufficient.
+
+Location: incubator-helix/recipes/user-rebalanced-lock-manager/src/main/java/org/apache/helix/userdefinedrebalancer/LockManagerRebalancer.java
+
+```
+@Override
+public ResourceAssignment computeResourceMapping(RebalancerConfig rebalancerConfig, Cluster cluster,
+    ResourceCurrentState currentState) {
+  // Get the rebalcancer context (a basic partitioned one)
+  PartitionedRebalancerContext context = rebalancerConfig.getRebalancerContext(
+      PartitionedRebalancerContext.class);
+
+  // Initialize an empty mapping of locks to participants
+  ResourceAssignment assignment = new ResourceAssignment(context.getResourceId());
+
+  // Get the list of live participants in the cluster
+  List<ParticipantId> liveParticipants = new ArrayList<ParticipantId>(
+      cluster.getLiveParticipantMap().keySet());
+
+  // Get the state model (should be a simple lock/unlock model) and the highest-priority state
+  StateModelDefId stateModelDefId = context.getStateModelDefId();
+  StateModelDefinition stateModelDef = cluster.getStateModelMap().get(stateModelDefId);
+  if (stateModelDef.getStatesPriorityList().size() < 1) {
+    LOG.error("Invalid state model definition. There should be at least one state.");
+    return assignment;
+  }
+  State lockState = stateModelDef.getTypedStatesPriorityList().get(0);
+
+  // Count the number of participants allowed to lock each lock
+  String stateCount = stateModelDef.getNumParticipantsPerState(lockState);
+  int lockHolders = 0;
+  try {
+    // a numeric value is a custom-specified number of participants allowed to lock the lock
+    lockHolders = Integer.parseInt(stateCount);
+  } catch (NumberFormatException e) {
+    LOG.error("Invalid state model definition. The lock state does not have a valid count");
+    return assignment;
+  }
+
+  // Fairly assign the lock state to the participants using a simple mod-based sequential
+  // assignment. For instance, if each lock can be held by 3 participants, lock 0 would be held
+  // by participants (0, 1, 2), lock 1 would be held by (1, 2, 3), and so on, wrapping around the
+  // number of participants as necessary.
+  // This assumes a simple lock-unlock model where the only state of interest is which nodes have
+  // acquired each lock.
+  int i = 0;
+  for (PartitionId partition : context.getPartitionSet()) {
+    Map<ParticipantId, State> replicaMap = new HashMap<ParticipantId, State>();
+    for (int j = i; j < i + lockHolders; j++) {
+      int participantIndex = j % liveParticipants.size();
+      ParticipantId participant = liveParticipants.get(participantIndex);
+      // enforce that a participant can only have one instance of a given lock
+      if (!replicaMap.containsKey(participant)) {
+        replicaMap.put(participant, lockState);
+      }
+    }
+    assignment.addReplicaMap(partition, replicaMap);
+    i++;
+  }
+  return assignment;
+}
+```
+
+### Start up the participants
+Here is a lock class based on the newly defined lock-unlock state model so that the participant can receive callbacks on state transitions.
+
+Location: incubator-helix/recipes/user-rebalanced-lock-manager/src/main/java/org/apache/helix/userdefinedrebalancer/Lock.java
+
+```
+public class Lock extends StateModel {
+  private String lockName;
+
+  public Lock(String lockName) {
+    this.lockName = lockName;
+  }
+
+  @Transition(from = "RELEASED", to = "LOCKED")
+  public void lock(Message m, NotificationContext context) {
+    System.out.println(context.getManager().getInstanceName() + " acquired lock:" + lockName);
+  }
+
+  @Transition(from = "LOCKED", to = "RELEASED")
+  public void release(Message m, NotificationContext context) {
+    System.out.println(context.getManager().getInstanceName() + " releasing lock:" + lockName);
+  }
+}
+```
+
+Here is the factory to make the Lock class accessible.
+
+Location: incubator-helix/recipes/user-rebalanced-lock-manager/src/main/java/org/apache/helix/userdefinedrebalancer/LockFactory.java
+
+```
+public class LockFactory extends StateModelFactory<Lock> {
+  @Override
+  public Lock createNewStateModel(String lockName) {
+    return new Lock(lockName);
+  }
+}
+```
+
+Finally, here is the factory registration and the start of the participant:
+
+```
+participantManager =
+    HelixManagerFactory.getZKHelixManager(clusterName, participantName, InstanceType.PARTICIPANT,
+        zkAddress);
+participantManager.getStateMachineEngine().registerStateModelFactory(stateModelName,
+    new LockFactory());
+participantManager.connect();
+```
+
+### Start up the controller
+
+```
+controllerManager =
+    HelixControllerMain.startHelixController(zkAddress, config.clusterName, "controller",
+        HelixControllerMain.STANDALONE);
+```
+
+### Try it out
+#### Building 
+```
+git clone https://git-wip-us.apache.org/repos/asf/incubator-helix.git
+cd incubator-helix
+mvn clean install package -DskipTests
+cd recipes/user-rebalanced-lock-manager/target/user-rebalanced-lock-manager-pkg/bin
+chmod +x *
+./lock-manager-demo.sh
+```
+
+#### Output
+
+```
+./lock-manager-demo 
+STARTING localhost_12002
+STARTING localhost_12001
+STARTING localhost_12003
+STARTED localhost_12001
+STARTED localhost_12003
+STARTED localhost_12002
+localhost_12003 acquired lock:lock-group_4
+localhost_12002 acquired lock:lock-group_8
+localhost_12001 acquired lock:lock-group_10
+localhost_12001 acquired lock:lock-group_3
+localhost_12001 acquired lock:lock-group_6
+localhost_12003 acquired lock:lock-group_0
+localhost_12002 acquired lock:lock-group_5
+localhost_12001 acquired lock:lock-group_9
+localhost_12002 acquired lock:lock-group_2
+localhost_12003 acquired lock:lock-group_7
+localhost_12003 acquired lock:lock-group_11
+localhost_12002 acquired lock:lock-group_1
+lockName  acquired By
+======================================
+lock-group_0  localhost_12003
+lock-group_1  localhost_12002
+lock-group_10 localhost_12001
+lock-group_11 localhost_12003
+lock-group_2  localhost_12002
+lock-group_3  localhost_12001
+lock-group_4  localhost_12003
+lock-group_5  localhost_12002
+lock-group_6  localhost_12001
+lock-group_7  localhost_12003
+lock-group_8  localhost_12002
+lock-group_9  localhost_12001
+Stopping the first participant
+localhost_12001 Interrupted
+localhost_12002 acquired lock:lock-group_3
+localhost_12003 acquired lock:lock-group_6
+localhost_12003 acquired lock:lock-group_10
+localhost_12002 acquired lock:lock-group_9
+lockName  acquired By
+======================================
+lock-group_0  localhost_12003
+lock-group_1  localhost_12002
+lock-group_10 localhost_12003
+lock-group_11 localhost_12003
+lock-group_2  localhost_12002
+lock-group_3  localhost_12002
+lock-group_4  localhost_12003
+lock-group_5  localhost_12002
+lock-group_6  localhost_12003
+lock-group_7  localhost_12003
+lock-group_8  localhost_12002
+lock-group_9  localhost_12002
+```
+
+Notice that the lock assignment directly follows the assignment generated by the user-defined rebalancer both initially and after a participant is removed from the system.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/150ce693/site-releases/trunk/src/site/markdown/tutorial_accessors.md
----------------------------------------------------------------------
diff --git a/site-releases/trunk/src/site/markdown/tutorial_accessors.md b/site-releases/trunk/src/site/markdown/tutorial_accessors.md
new file mode 100644
index 0000000..bde50d2
--- /dev/null
+++ b/site-releases/trunk/src/site/markdown/tutorial_accessors.md
@@ -0,0 +1,125 @@
+<!---
+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 - Logical Accessors</title>
+</head>
+
+# [Helix Tutorial](./Tutorial.html): Logical Accessors
+
+Helix constructs follow a logical hierarchy. A cluster contains participants, and serve logical resources. Each resource can be divided into partitions, which themselves can be replicated. Helix now supports configuring and modifying clusters programmatically in a hierarchical way using logical accessors.
+
+[Click here](http://helix.incubator.apache.org/apidocs/reference/org/apache/helix/api/accessor/package-summary.html) for the Javadocs of the accessors.
+
+### An Example
+
+#### Configure a Participant
+
+A participant is a combination of a host, port, and a UserConfig. A UserConfig is an arbitrary set of properties a Helix user can attach to any participant.
+
+```
+ParticipantId participantId = ParticipantId.from("localhost_12345");
+ParticipantConfig participantConfig = new ParticipantConfig.Builder(participantId)
+    .hostName("localhost").port(12345).build();
+```
+
+#### Configure a Resource
+
+##### RebalancerContext
+A Resource is essentially a combination of a RebalancerContext and a UserConfig. A [RebalancerContext](http://helix.incubator.apache.org/apidocs/reference/org/apache/helix/controller/rebalancer/context/RebalancerContext.html) consists of all the key properties required to rebalance a resource, including how it is partitioned and replicated, and what state model it follows. Most Helix resources will make use of a [PartitionedRebalancerContext](http://helix.incubator.apache.org/apidocs/reference/org/apache/helix/controller/rebalancer/context/PartitionedRebalancerContext.html), which is a RebalancerContext for resources that are partitioned.
+
+Recall that there are four [rebalancing modes](./tutorial_rebalance.html) that Helix provides, and so Helix also provides the following subclasses for PartitionedRebalancerContext:
+
+* [FullAutoRebalancerContext](http://helix.incubator.apache.org/apidocs/reference/org/apache/helix/controller/rebalancer/context/FullAutoRebalancerContext.html) for FULL_AUTO mode.
+* [SemiAutoRebalancerContext](http://helix.incubator.apache.org/apidocs/reference/org/apache/helix/controller/rebalancer/context/SemiAutoRebalancerContext.html) for SEMI_AUTO mode. This class allows a user to specify "preference lists" to indicate where each partition should ideally be served
+* [CustomRebalancerContext](http://helix.incubator.apache.org/apidocs/reference/org/apache/helix/controller/rebalancer/context/CustomRebalancerContext.html) for CUSTOMIZED mode. This class allows a user tp specify "preference maps" to indicate the location and state for each partition replica.
+
+Helix also supports arbitrary subclasses of PartitionedRebalancerContext and even arbitrary implementations of RebalancerContext for applications that need a user-defined approach for rebalancing. For more, see [User-Defined Rebalancing](./tutorial_user_def_rebalancer.html)
+
+##### In Action
+
+Here is an example of a configured resource with a rebalancer context for FULL_AUTO mode and two partitions:
+
+```
+ResourceId resourceId = ResourceId.from("sampleResource");
+StateModelDefinition stateModelDef = getStateModelDef();
+Partition partition1 = new Partition(PartitionId.from(resourceId, "1"));
+Partition partition2 = new Partition(PartitionId.from(resourceId, "2"));
+FullAutoRebalancerContext rebalanceContext =
+    new FullAutoRebalancerContext.Builder(resourceId).replicaCount(1).addPartition(partition1)
+        .addPartition(partition2).stateModelDefId(stateModelDef.getStateModelDefId()).build();
+ResourceConfig resourceConfig =
+    new ResourceConfig.Builder(resourceId).rebalancerContext(rebalanceContext).build();
+```
+
+#### Add the Cluster
+
+Now we can take the participant and resource configured above, add them to a cluster configuration, and then persist the entire cluster at once using a ClusterAccessor:
+
+```
+// configure the cluster
+ClusterId clusterId = ClusterId.from("sampleCluster");
+ClusterConfig clusterConfig = new ClusterConfig.Builder(clusterId).addParticipant(participantConfig)
+    .addResource(resourceConfig).addStateModelDefinition(stateModelDef).build();
+
+// create the cluster using a ClusterAccessor
+HelixConnection connection = new ZkHelixConnection(zkAddr);
+connection.connect();
+ClusterAccessor clusterAccessor = connection.createClusterAccessor(clusterId);
+clusterAccessor.createCluster(clusterConfig);
+```
+
+### Create, Read, Update, and Delete
+
+Note that you don't have to specify the entire cluster beforehand! Helix provides a ClusterAccessor, ParticipantAccessor, and ResourceAccessor to allow changing as much or as little of the cluster as needed on the fly. You can add a resource or participant to a cluster, reconfigure a resource, participant, or cluster, remove components from the cluster, and more. See the [Javadocs](http://helix.incubator.apache.org/apidocs/reference/org/apache/helix/api/accessor/package-summary.html) to see all that the accessor classes can do.
+
+#### Delta Classes
+
+Updating a cluster, participant, or resource should involve selecting the element to change, and then letting Helix change only that component. To do this, Helix has included Delta classes for ClusterConfig, ParticipantConfig, and ResourceConfig.
+
+#### Example: Updating a Participant
+
+Tags are used for Helix depolyments where only certain participants can be allowed to serve certain resources. To do this, Helix only assigns resource replicas to participants who have a tag that the resource specifies. In this example, we will use ParticipantConfig.Delta to remove a participant tag and add another as part of a reconfiguration.
+
+```
+// specify the change to the participant
+ParticipantConfig.Delta delta = new ParticipantConfig.Delta(participantId).addTag("newTag").removeTag("oldTag");
+
+// update the participant configuration
+ParticipantAccessor participantAccessor = connection.createParticipantAccessor(clusterId);
+participantAccessor.updateParticipant(participantId, delta);
+```
+
+#### Example: Dropping a Resource
+Removing a resource from the cluster is quite simple:
+
+```
+clusterAccessor.dropResourceFromCluster(resourceId);
+```
+
+#### Example: Reading the Cluster
+Reading a full snapshot of the cluster is also a one-liner:
+
+```
+Cluster cluster = clusterAccessor.readCluster();
+```
+
+### Atomic Accessors
+
+Helix also includes versions of ClusterAccessor, ParticipantAccessor, and ResourceAccessor that can complete operations atomically relative to one another. The specific semantics of the atomic operations are included in the Javadocs. These atomic classes should be used sparingly and only in cases where contention can adversely affect the correctness of a Helix-based cluster. For most deployments, this is not the case, and using these classes will cause a degradation in performance. However, the interface for all atomic accessors mirrors that of the non-atomic accessors.
\ No newline at end of file