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/05/21 02:41:29 UTC

[1/2] Applying patch by rms(bob) to update the docs

Updated Branches:
  refs/heads/master 96cc67417 -> 93a8770e4


http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/93a8770e/src/site/markdown/tutorial_spectator.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/tutorial_spectator.md b/src/site/markdown/tutorial_spectator.md
new file mode 100644
index 0000000..03fc45a
--- /dev/null
+++ b/src/site/markdown/tutorial_spectator.md
@@ -0,0 +1,78 @@
+<!---
+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.
+-->
+
+# Helix Tutorial: Spectator
+
+Next, we\'ll learn how to implement a SPECTATOR.  Typically, a spectator needs to react to changes within the distributed system.  Examples: a client that needs to know where to send a request, a topic consumer in a consumer group.  The spectator is automatically informed of changes in the _external state_ of the cluster, but it does not have to add any code to keep track of other components in the system.
+
+### Start the Helix agent
+
+Same as for a PARTICIPANT, The Helix agent is the common component that connects each system component with the controller.
+
+It requires the following parameters:
+
+* clusterName: A logical name to represent the group of nodes
+* instanceName: A logical name of the process creating the manager instance. Generally this is host:port.
+* instanceType: Type of the process. This can be one of the following types, in this case, use SPECTATOR:
+    * CONTROLLER: Process that controls the cluster, any number of controllers can be started but only one will be active at any given time.
+    * PARTICIPANT: Process that performs the actual task in the distributed system.
+    * SPECTATOR: Process that observes the changes in the cluster.
+    * ADMIN: To carry out system admin actions.
+* zkConnectString: Connection string to Zookeeper. This is of the form host1:port1,host2:port2,host3:port3.
+
+After the Helix manager instance is created, only thing that needs to be registered is the state model factory.
+The methods of the State Model will be called when controller sends transitions to the Participant.  In this example, we'll use the OnlineOffline factory.  Other options:
+
+_NEED TO ADD OTHER OPTIONS HERE_
+
+### Spectator Code
+
+A spectator observes the cluster and is notified when the state of the system changes. Helix consolidates the state of entire cluster in one Znode called ExternalView.
+Helix provides a default implementation RoutingTableProvider that caches the cluster state and updates it when there is a change in the cluster.
+
+```
+manager = HelixManagerFactory.getZKHelixManager(clusterName,
+                                                          instanceName,
+                                                          InstanceType.PARTICIPANT,
+                                                          zkConnectString);
+manager.connect();
+RoutingTableProvider routingTableProvider = new RoutingTableProvider();
+manager.addExternalViewChangeListener(routingTableProvider);
+```
+
+In the following code snippet, the application sends the request to a valid instance by interrogating the external view.  Suppose the desired resource for this request is in the partition myDB_1.
+
+```
+## instances = routingTableProvider.getInstances(, "PARTITION_NAME", "PARTITION_STATE");
+instances = routingTableProvider.getInstances("myDB", "myDB_1", "ONLINE");
+
+////////////////////////////////////////////////////////////////////////////////////////////////
+// Application-specific code to send a request to one of the instances                        //
+////////////////////////////////////////////////////////////////////////////////////////////////
+
+theInstance = instances.get(0);  // should choose an instance and throw an exception if none are available
+result = theInstance.sendRequest(yourApplicationRequest, responseObject);
+
+```
+
+When the external view changes, the application needs to react by sending requests to a different instance.  
+
+_SHOULD WE ADD AN EXAMPLE OF THIS?  IT WOULD MAKE THINGS MORE COMPLICATED, BECAUSE THE APP SHOULD HAVE A LOCAL COPY OF THE ROUTING TABLE AND UPDATE IT ONLY WHEN THE EXTERNAL VIEW CHANGES, THAT\'S HOW I'D IMPLEMENT IT._
+
+

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/93a8770e/src/site/markdown/tutorial_state.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/tutorial_state.md b/src/site/markdown/tutorial_state.md
new file mode 100644
index 0000000..cb51be9
--- /dev/null
+++ b/src/site/markdown/tutorial_state.md
@@ -0,0 +1,60 @@
+<!---
+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.
+-->
+
+# Helix Tutorial: State Machine Configuration
+
+In this chapter, we\'ll learn about the state models provided by Helix, and how to create your own custom state model.
+
+## State Models
+
+Helix comes with 3 default state models that are commonly used.  It is possible to have multiple state models in a cluster. 
+Every resource that is added should be configured to use a state model that govern its _ideal state_.
+
+### MASTER-SLAVE
+
+* Has 3 states: OFFLINE, SLAVE, MASTER
+* Maximum # of masters: 1
+* Slaves are based on the replication factor. Replication factor can be specified while adding the resource
+
+
+### ONLINE-OFFLINE
+* Has 2 states: OFFLINE and ONLINE.  This simple state model is a good starting point for most applications.
+
+### LEADER-STANDBY
+* 1 Leader and multiple stand-bys.  The idea is that exactly one leader accomplishes a designated task, the stand-bys are ready to take over if the leader fails.
+
+## Constraints
+
+In addition to the state machine configuration, one can specify the constraints of states and transitions.
+
+For example, one can say:
+* MASTER:1
+ Maximum number of replicas in MASTER state at any time is 1
+
+* OFFLINE-SLAVE:5 
+Maximum number of OFFLINE-SLAVE transitions that can happen concurrently in the system is 5 in this example.
+
+### State Priority
+
+Helix uses a greedy approach to satisfy the state constraints. For example, if the state machine configuration says it needs 1 MASTER and 2 SLAVES, but only 1 node is active, Helix must promote it to MASTER. This behavior is achieved by providing the state priority list as MASTER,SLAVE.
+
+### State Transition Priority
+
+Helix tries to fire as many transitions as possible in parallel to reach the stable state without violating constraints. By default, Helix simply sorts the transitions alphabetically and fires as many as it can without violating the constraints. You can control this by overriding the priority order.
+

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/93a8770e/src/site/markdown/tutorial_throttling.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/tutorial_throttling.md b/src/site/markdown/tutorial_throttling.md
new file mode 100644
index 0000000..0fcc9bc
--- /dev/null
+++ b/src/site/markdown/tutorial_throttling.md
@@ -0,0 +1,35 @@
+<!---
+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.
+-->
+
+# Helix Tutorial: Throttling
+
+In this chapter, we\'ll learn how to control the parallel execution of cluster tasks.  Only a centralized cluster manager with global knowledge is capable of coordinating this decision.
+
+### Throttling
+
+Since all state changes in the system are triggered through transitions, Helix can control the number of transitions that can happen in parallel. Some of the transitions may be light weight, but some might involve moving data, which is quite expensive from a network and iops perspective.
+
+Helix allows applications to set a threshold on transitions. The threshold can be set at multiple scopes:
+
+* MessageType e.g STATE_TRANSITION
+* TransitionType e.g SLAVE-MASTER
+* Resource e.g database
+* Node i.e per-node maximum transitions in parallel
+
+_I can\'t find this code._  See HelixManager.getHelixAdmin.addMessageConstraint() 

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/93a8770e/src/site/site.xml
----------------------------------------------------------------------
diff --git a/src/site/site.xml b/src/site/site.xml
index 1bff519..e44a43e 100644
--- a/src/site/site.xml
+++ b/src/site/site.xml
@@ -58,11 +58,10 @@
 
     <menu name="Helix">
       <item name="Introduction" href="./index.html"/>
+      <item name="Core concepts" href="./Concepts.html"/>
+      <item name="Architecture" href="./Architecture.html"/>
       <item name="Quick Start" href="./Quickstart.html"/>
-      <item name="Core concept" href="./Concepts.html"/>
       <item name="Tutorial" href="./Tutorial.html"/>
-      <item name="Architecture" href="./Architecture.html"/>
-      <item name="Features" href="./Features.html"/>
       <item name="release ${currentRelease}" href="releasenotes/release-${currentRelease}.html"/>
       <item name="Download" href="./download.html"/>
     </menu>


[2/2] git commit: Applying patch by rms(bob) to update the docs

Posted by ki...@apache.org.
Applying patch by rms(bob) to update the docs


Project: http://git-wip-us.apache.org/repos/asf/incubator-helix/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-helix/commit/93a8770e
Tree: http://git-wip-us.apache.org/repos/asf/incubator-helix/tree/93a8770e
Diff: http://git-wip-us.apache.org/repos/asf/incubator-helix/diff/93a8770e

Branch: refs/heads/master
Commit: 93a8770e41e61337525234945193afc2280a3c3d
Parents: 96cc674
Author: kishoreg <ki...@apache.org>
Authored: Mon May 20 17:41:18 2013 -0700
Committer: kishoreg <ki...@apache.org>
Committed: Mon May 20 17:41:18 2013 -0700

----------------------------------------------------------------------
 src/site/markdown/Quickstart.md           |  481 ++++++++++++++++++-----
 src/site/markdown/Tutorial.md             |  313 +++++-----------
 src/site/markdown/index.md                |  210 +++--------
 src/site/markdown/tutorial_admin.md       |  167 ++++++++
 src/site/markdown/tutorial_controller.md  |   90 +++++
 src/site/markdown/tutorial_health.md      |   42 ++
 src/site/markdown/tutorial_messaging.md   |   67 ++++
 src/site/markdown/tutorial_participant.md |   98 +++++
 src/site/markdown/tutorial_propstore.md   |   30 ++
 src/site/markdown/tutorial_rebalance.md   |  168 +++++++++
 src/site/markdown/tutorial_spectator.md   |   78 ++++
 src/site/markdown/tutorial_state.md       |   60 +++
 src/site/markdown/tutorial_throttling.md  |   35 ++
 src/site/site.xml                         |    5 +-
 14 files changed, 1353 insertions(+), 491 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/93a8770e/src/site/markdown/Quickstart.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/Quickstart.md b/src/site/markdown/Quickstart.md
index 992f73f..6807c77 100644
--- a/src/site/markdown/Quickstart.md
+++ b/src/site/markdown/Quickstart.md
@@ -17,52 +17,57 @@ specific language governing permissions and limitations
 under the License.
 -->
 
-Build/Download Helix
---------------------
-
-### Build
+Get Helix
+---------
 
+First, let\'s get Helix, either build or download.
 
-Jump to download section to skip building code
+### Build
 
     git clone https://git-wip-us.apache.org/repos/asf/incubator-helix.git
     cd incubator-helix
     mvn install package -DskipTests 
     cd helix-core/target/helix-core-pkg/bin //This folder contains all the scripts used in following sections
-    chmod \+x *
+    chmod +x *
 
-### Download Helix
+### Download
 
-Instead of building the package from the code, you can download the 0.6.0-incubating release package from [here](https://dist.apache.org/repos/dist/dev/incubator/helix/0.6.0-incubating/binaries/helix-core-0.6.0-incubating-pkg.tar) 
+Download the 0.6.1-incubating release package [here](https://dist.apache.org/repos/dist/dev/incubator/helix/0.6.1-incubating/binaries/helix-core-0.6.1-incubating-pkg.tar) 
 
-Short Version
--------------
+Overview
+--------
 
-    cd helix-core/target/helix-core-pkg/bin
-    chmod +x *
-    ./quickstart.sh
+In this Quickstart, we\'ll set up a master-slave replicated, partitioned system.  Then we\'ll demonstrate how to add a node, rebalance the partitions, and show how Helix manages failover.
 
+The steps:
 
-This gives an overview of Helix Apis and how it facilitates Automatic Partition Management, Failure Handling and Cluster Expansion. 
+* Define a cluster
+* Add two nodes to the cluster
+* Add a 6-partition resource with 1 master and 2 slave replicas per partition
+* Verify that the cluster is healthy and inspect the Helix view
+* Expand the cluster: add a few nodes and rebalance the partitions
+* Failover: stop a node and verify the mastership transfer
 
-The quick start does the following
+Let\'s Do It
+------------
 
-* Add a cluster
-* Add two nodes
-* Add a resource with 6 partitions with 1 master and 1 slave replica.
-* Shows the cluster state after all nodes are up.
-* Add a node and see the partitions get rebalanced
-* Shows cluster state after the rebalance
-* Stop a node and see that master ship transfer occurs.
-* Shows cluster state after node is stopped.
+Helix provides command line interfaces to set up the cluster and view the cluster state. The best way to understand how Helix views a cluster is to build a cluster.
 
-Long Version
-------------
+#### First, get to the tools directory.
 
-Helix provides command line interfaces to setup the cluster and also view the cluster state. Follow the steps below to get a better understanding of the steps involved to setup a distributed system.
+If you built the code
 
-### Install/Start zookeeper
+```
+cd helix/helix-core/target/helix-core-pkg/bin
+```
 
+If you downloaded the release package, extract it.
+
+```
+cd helix-core-pkg/bin
+```
+
+### Install/Start zookeeper
 
 Zookeeper can be started in standalone mode or replicated mode.
 
@@ -71,119 +76,385 @@ More info is available at
 * http://zookeeper.apache.org/doc/r3.3.3/zookeeperStarted.html
 * http://zookeeper.apache.org/doc/trunk/zookeeperAdmin.html#sc_zkMulitServerSetup
 
-In this example, we will start zookeeper in local mode.
+In this example, let\'s start zookeeper in local mode.
 
-### Cluster setup
+##### start zookeeper locally on port 2199
 
-helix-admin tool is used for cluster administration tasks. Apart from a command line interface Helix supports a REST interface as well.
+    ./start-standalone-zookeeper.sh 2199 &
+
+
+### Define the Cluster
 
-zookeeper_address is of the format host:port e.g localhost:2199 for standalone or host1:port,host2:port for multi node.
+The helix-admin tool is used for cluster administration tasks. In the Quickstart, we\'ll use the command line interface. Helix supports a REST interface as well.
 
-In the following section we will see how one can set up a mock mycluster cluster with 
+zookeeper_address is of the format host:port e.g localhost:2199 for standalone or host1:port,host2:port for multi-node.
 
-* 3 instances running on localhost at 12913, 12914,12915 
-* One database named MyDB with 6 partitions 
+Next, we\'ll set up a cluster MYCLUSTER cluster with these attributes:
+
+* 3 instances running on localhost at ports 12913,12914,12915 
+* One database named myDB with 6 partitions 
 * Each partition will have 3 replicas with 1 master, 2 slaves
 * zookeeper running locally at localhost:2199
 
-Note that this mock cluster does not have any functionality apart from handling Helix callbacks.
- 
-### Steps
+##### Create the cluster MYCLUSTER
+    ## helix-admin.sh --zkSvr <zk_address> --addCluster <clustername> 
+    ./helix-admin.sh --zkSvr localhost:2199 --addCluster MYCLUSTER 
 
-* If you built the code
+##### Add nodes to the cluster
 
-```
-cd helix/helix-core/target/helix-core-pkg
-```
+In this case we\'ll add three nodes: localhost:12913, localhost:12914, localhost:12915
 
-* If you downloaded the release package, extract it.
+    ## helix-admin.sh --zkSvr <zk_address>  --addNode <clustername> <host:port>
+    ./helix-admin.sh --zkSvr localhost:2199  --addNode MYCLUSTER localhost:12913
+    ./helix-admin.sh --zkSvr localhost:2199  --addNode MYCLUSTER localhost:12914
+    ./helix-admin.sh --zkSvr localhost:2199  --addNode MYCLUSTER localhost:12915
 
-```
-cd helix-core-pkg
-```
-     
-##### start zookeeper locally at port 2199
+#### Define the resource and partitioning
 
-    ./start-standalone-zookeeper.sh 2199 &
+In this example, the resource is a database, partitioned 6 ways.  (In a production system, it\'s common to over-partition for better load balancing.  Helix has been used in production to manage hundreds of databases each with 10s or 100s of partitions running on 10s of physical nodes.)
 
-##### create the cluster mycluster
-    ## helix-admin.sh --zkSvr localhost:2199 --addCluster <clustername> 
-    ./helix-admin.sh --zkSvr localhost:2199 --addCluster mycluster 
+##### Create a database with 6 partitions using the MasterSlave state model. 
 
-##### Create a database with 6 partitions using MasterSlave state model. This ensures there will be one master for each partition 
-    ### helix-admin.sh --zkSvr localhost:2199  --addResource <clustername> <resourceName> <numPartitions> <StateModelName>
-    ./helix-admin.sh --zkSvr localhost:2199  --addResource mycluster myDB 6 MasterSlave
+Helix ensures there will be exactly one master for each partition.
+
+    ## helix-admin.sh --zkSvr <zk_address> --addResource <clustername> <resourceName> <numPartitions> <StateModelName>
+    ./helix-admin.sh --zkSvr localhost:2199 --addResource MYCLUSTER myDB 6 MasterSlave
    
-##### Add nodes to the cluster, in this case we add three nodes, hostname:port is host and port on which the service will start
-    ## helix-admin.sh --zkSvr <zk_address>  --addNode <clustername> <host:port>
-    ./helix-admin.sh --zkSvr localhost:2199  --addNode mycluster localhost:12913
-    ./helix-admin.sh --zkSvr localhost:2199  --addNode mycluster localhost:12914
-    ./helix-admin.sh --zkSvr localhost:2199  --addNode mycluster localhost:12915
+##### Now we can let Helix assign partitions to nodes. 
+
+This command will distribute the partitions amongst all the nodes in the cluster. In this example, each partition has 3 replicas.
+
+    ## helix-admin.sh --zkSvr <zk_address> --rebalance <clustername> <resourceName> <replication factor>
+    ./helix-admin.sh --zkSvr localhost:2199 --rebalance MYCLUSTER myDB 3
+
+Now the cluster is defined in Zookeeper.  The nodes (localhost:12913, localhost:12914, localhost:12915) and resource (myDB, with 6 partitions using the MasterSlave model).  And the _ideal state_ has been calculated, assuming a replication factor of 3.
 
-##### After adding nodes assign partitions to nodes. This command will distribute the partitions amongst all the nodes in the cluster. Each partition will have 3 replicas    
-     helix-admin.sh --rebalance <clustername> <resourceName> <replication factor>
-    ./helix-admin.sh --zkSvr localhost:2199 --rebalance mycluster myDB 3
+##### Start the Helix Controller
 
-##### Start Helix Controller
-    #This will start the cluster manager which will manage <mycluster>
-    ./run-helix-controller --zkSvr localhost:2199 --cluster mycluster 2>&1 > /tmp/controller.log &
+Now that the cluster is defined in Zookeeper, the Helix controller can manage the cluster.
 
-##### Start Example Participant, This is a dummy participant where the transitions are no-ops.    
-    ./start-helix-participant.sh --help
-    # start process 1 process corresponding to every host port added during cluster setup
-    ./start-helix-participant.sh --zkSvr localhost:2199 --cluster mycluster --host localhost --port 12913 --stateModelType MasterSlave 2>&1 > /tmp/participant_12913.log 
-    ./start-helix-participant.sh --zkSvr localhost:2199 --cluster mycluster --host localhost --port 12914 --stateModelType MasterSlave 2>&1 > /tmp/participant_12914.log
-    ./start-helix-participant.sh --zkSvr localhost:2199 --cluster mycluster --host localhost --port 12915 --stateModelType MasterSlave 2>&1 > /tmp/participant_12915.log
+    ## Start the cluster manager, which will manage MYCLUSTER
+    ./run-helix-controller.sh --zkSvr localhost:2199 --cluster MYCLUSTER 2>&1 > /tmp/controller.log &
 
+##### Start up the cluster to be managed
 
-### Inspect Cluster Data
+We\'ve started up Zookeeper, defined the cluster, the resources, the partitioning, and started up the Helix controller.  Next, we\'ll start up the nodes of the system to be managed.  Each node is a Participant, which is an instance of the system component to be managed.  Helix assigns work to Participants, keeps track of their roles and health, and takes action when a node fails.
 
+    # start up each instance.  These are mock implementations that are actively managed by Helix
+    ./start-helix-participant.sh --zkSvr localhost:2199 --cluster MYCLUSTER --host localhost --port 12913 --stateModelType MasterSlave 2>&1 > /tmp/participant_12913.log 
+    ./start-helix-participant.sh --zkSvr localhost:2199 --cluster MYCLUSTER --host localhost --port 12914 --stateModelType MasterSlave 2>&1 > /tmp/participant_12914.log
+    ./start-helix-participant.sh --zkSvr localhost:2199 --cluster MYCLUSTER --host localhost --port 12915 --stateModelType MasterSlave 2>&1 > /tmp/participant_12915.log
 
-At any time, we can get the cluster status on zookeeper and view the partition assignment and current state of each partition.
 
-Command line tool
+#### Inspect the Cluster
 
-##### List existing clusters
-    ./helix-admin.sh --zkSvr localhost:2199 --listClusters        
+Now, let\'s see the Helix view of our cluster.  We\'ll work our way down as follows:
+
+```
+Clusters -> MYCLUSTER -> instances -> instance detail
+                      -> resources -> resource detail
+                      -> partitions
+```
+
+A single Helix controller can manage multiple clusters, though so far, we\'ve only defined one cluster.  Let\'s see:
+
+```
+## List existing clusters
+./helix-admin.sh --zkSvr localhost:2199 --listClusters        
+
+Existing clusters:
+MYCLUSTER
+```
                                        
-#####  Query info of a cluster
+Now, let\'s see the Helix view of MYCLUSTER
+
+```
+## helix-admin.sh --zkSvr <zk_address> --listClusterInfo <clusterName> 
+./helix-admin.sh --zkSvr localhost:2199 --listClusterInfo MYCLUSTER
+
+Existing resources in cluster MYCLUSTER:
+myDB
+Instances in cluster MYCLUSTER:
+localhost_12915
+localhost_12914
+localhost_12913
+```
 
-    #helix-admin.sh --zkSvr localhost:2199 --listClusterInfo <clusterName> 
-    ./helix-admin.sh --zkSvr localhost:2199 --listClusterInfo mycluster
 
-#####  List Instances in a cluster
-    ## helix-admin.sh --zkSvr localhost:2199 --listInstances <clusterName>
-     ./helix-admin.sh --zkSvr localhost:2199 --listInstances mycluster
-    
-##### Query info of a Instance in a cluster
-    #./helix-admin.sh --zkSvr localhost:2199 --listInstanceInfo <clusterName InstanceName>    
-     ./helix-admin.sh --zkSvr localhost:2199 --listInstanceInfo mycluster localhost_12913
-     ./helix-admin.sh --zkSvr localhost:2199 --listInstanceInfo mycluster localhost_12914
-     ./helix-admin.sh --zkSvr localhost:2199 --listInstanceInfo mycluster localhost_12915
-
-##### List resourceGroups hosted in a cluster
-    ## helix-admin.sh --zkSvr localhost:2199 --listResources <clusterName>
-    ./helix-admin.sh --zkSvr localhost:2199 --listResources mycluster
+Let\'s look at the details of an instance
+
+```
+## ./helix-admin.sh --zkSvr <zk_address> --listInstanceInfo <clusterName> <InstanceName>    
+./helix-admin.sh --zkSvr localhost:2199 --listInstanceInfo MYCLUSTER localhost_12913
+
+InstanceConfig: {
+  "id" : "localhost_12913",
+  "mapFields" : {
+  },
+  "listFields" : {
+  },
+  "simpleFields" : {
+    "HELIX_ENABLED" : "true",
+    "HELIX_HOST" : "localhost",
+    "HELIX_PORT" : "12913"
+  }
+}
+```
+
     
 ##### Query info of a resource
-    ## helix-admin.sh --zkSvr localhost:2199 --listResourceInfo <clusterName resourceName>
-    ./helix-admin.sh --zkSvr localhost:2199 --listResourceInfo mycluster myDB
 
-##### Query info about a partition   
-    ## helix-admin.sh --zkSvr localhost:2199 --listResourceInfo <clusterName partition> 
+```
+## helix-admin.sh --zkSvr <zk_address> --listResourceInfo <clusterName> <resourceName>
+./helix-admin.sh --zkSvr localhost:2199 --listResourceInfo MYCLUSTER myDB
+
+IdealState for myDB:
+{
+  "id" : "myDB",
+  "mapFields" : {
+    "myDB_0" : {
+      "localhost_12913" : "SLAVE",
+      "localhost_12914" : "MASTER",
+      "localhost_12915" : "SLAVE"
+    },
+    "myDB_1" : {
+      "localhost_12913" : "SLAVE",
+      "localhost_12914" : "SLAVE",
+      "localhost_12915" : "MASTER"
+    },
+    "myDB_2" : {
+      "localhost_12913" : "MASTER",
+      "localhost_12914" : "SLAVE",
+      "localhost_12915" : "SLAVE"
+    },
+    "myDB_3" : {
+      "localhost_12913" : "SLAVE",
+      "localhost_12914" : "SLAVE",
+      "localhost_12915" : "MASTER"
+    },
+    "myDB_4" : {
+      "localhost_12913" : "MASTER",
+      "localhost_12914" : "SLAVE",
+      "localhost_12915" : "SLAVE"
+    },
+    "myDB_5" : {
+      "localhost_12913" : "SLAVE",
+      "localhost_12914" : "MASTER",
+      "localhost_12915" : "SLAVE"
+    }
+  },
+  "listFields" : {
+    "myDB_0" : [ "localhost_12914", "localhost_12913", "localhost_12915" ],
+    "myDB_1" : [ "localhost_12915", "localhost_12913", "localhost_12914" ],
+    "myDB_2" : [ "localhost_12913", "localhost_12915", "localhost_12914" ],
+    "myDB_3" : [ "localhost_12915", "localhost_12913", "localhost_12914" ],
+    "myDB_4" : [ "localhost_12913", "localhost_12914", "localhost_12915" ],
+    "myDB_5" : [ "localhost_12914", "localhost_12915", "localhost_12913" ]
+  },
+  "simpleFields" : {
+    "IDEAL_STATE_MODE" : "AUTO",
+    "NUM_PARTITIONS" : "6",
+    "REPLICAS" : "3",
+    "STATE_MODEL_DEF_REF" : "MasterSlave",
+    "STATE_MODEL_FACTORY_NAME" : "DEFAULT"
+  }
+}
+
+ExternalView for myDB:
+{
+  "id" : "myDB",
+  "mapFields" : {
+  },
+  "listFields" : {
+  },
+  "simpleFields" : {
+    "BUCKET_SIZE" : "0"
+  }
+}
+```
+
+Now, let\'s look at one of the partitions:
+
+    ## helix-admin.sh --zkSvr <zk_address> --listResourceInfo <clusterName> <partition> 
     ./helix-admin.sh --zkSvr localhost:2199 --listResourceInfo mycluster myDB_0
-   
-##### List all state models in the cluster
-    # helix-admin.sh --zkSvr localhost:2199 --listStateModels <clusterName>
-    ./helix-admin.sh --zkSvr localhost:2199 --listStateModels mycluster
-    
-##### Query info about a state model in a cluster
-    ## helix-admin.sh --zkSvr localhost:2199 --listStateModel <clusterName stateModelName>
-    ./helix-admin.sh --zkSvr localhost:2199 --listStateModel mycluster MasterSlave
+
+#### Expand the Cluster
+
+Next, we\'ll show how Helix does the work that you\'d otherwise have to build into your system.  When you add capacity to your cluster, you want the work to be evenly distributed.  In this example, we started with 3 nodes, with 6 partitions.  The partitions were evenly balanced, 2 masters and 4 slaves per node. Let\'s add 3 more nodes: localhost:12916, localhost:12917, localhost:12918
+
+    ./helix-admin.sh --zkSvr localhost:2199  --addNode MYCLUSTER localhost:12916
+    ./helix-admin.sh --zkSvr localhost:2199  --addNode MYCLUSTER localhost:12917
+    ./helix-admin.sh --zkSvr localhost:2199  --addNode MYCLUSTER localhost:12918
+
+And now, let Helix do the work for you.  To shift the work, simply rebalance.  After the rebalance, each node will have one master and two slaves.
+
+    ./helix-admin.sh --zkSvr localhost:2199 --rebalance MYCLUSTER myDB 3
+
+#### View the cluster
+
+OK, let\'s see how it looks:
+
+
+```
+./helix-admin.sh --zkSvr localhost:2199 --listResourceInfo MYCLUSTER myDB
+
+IdealState for myDB:
+{
+  "id" : "myDB",
+  "mapFields" : {
+    "myDB_0" : {
+      "localhost_12913" : "SLAVE",
+      "localhost_12914" : "SLAVE",
+      "localhost_12917" : "MASTER"
+    },
+    "myDB_1" : {
+      "localhost_12916" : "SLAVE",
+      "localhost_12917" : "SLAVE",
+      "localhost_12918" : "MASTER"
+    },
+    "myDB_2" : {
+      "localhost_12913" : "MASTER",
+      "localhost_12917" : "SLAVE",
+      "localhost_12918" : "SLAVE"
+    },
+    "myDB_3" : {
+      "localhost_12915" : "MASTER",
+      "localhost_12917" : "SLAVE",
+      "localhost_12918" : "SLAVE"
+    },
+    "myDB_4" : {
+      "localhost_12916" : "MASTER",
+      "localhost_12917" : "SLAVE",
+      "localhost_12918" : "SLAVE"
+    },
+    "myDB_5" : {
+      "localhost_12913" : "SLAVE",
+      "localhost_12914" : "MASTER",
+      "localhost_12915" : "SLAVE"
+    }
+  },
+  "listFields" : {
+    "myDB_0" : [ "localhost_12917", "localhost_12913", "localhost_12914" ],
+    "myDB_1" : [ "localhost_12918", "localhost_12917", "localhost_12916" ],
+    "myDB_2" : [ "localhost_12913", "localhost_12917", "localhost_12918" ],
+    "myDB_3" : [ "localhost_12915", "localhost_12917", "localhost_12918" ],
+    "myDB_4" : [ "localhost_12916", "localhost_12917", "localhost_12918" ],
+    "myDB_5" : [ "localhost_12914", "localhost_12915", "localhost_12913" ]
+  },
+  "simpleFields" : {
+    "IDEAL_STATE_MODE" : "AUTO",
+    "NUM_PARTITIONS" : "6",
+    "REPLICAS" : "3",
+    "STATE_MODEL_DEF_REF" : "MasterSlave",
+    "STATE_MODEL_FACTORY_NAME" : "DEFAULT"
+  }
+}
+
+ExternalView for myDB:
+{
+  "id" : "myDB",
+  "mapFields" : {
+  },
+  "listFields" : {
+  },
+  "simpleFields" : {
+    "BUCKET_SIZE" : "0"
+  }
+}
+```
+
+Mission accomplished.  The partitions are nicely balanced.
+
+#### How about Failover?
+
+Building a fault tolerant system isn\'t trivial, but with Helix, it\'s easy.  Helix detects a failed instance, and triggers mastership transfer automatically.
+
+First, let's fail an instance:
+
+_KILL A NODE (need to find the pid via listInstanceInfo)_
+
+We lost localhost:12918, so myDB_1 lost its MASTER.  Helix can fix that, it will transfer mastership to a healthy node that is currently a SLAVE, say localhost:12197.  Helix balances the load as best as it can, given there are 6 partitions on 5 nodes.  Let\'s see:
+
+
+```
+./helix-admin.sh --zkSvr localhost:2199 --listResourceInfo MYCLUSTER myDB
+
+IdealState for myDB:
+{
+  "id" : "myDB",
+  "mapFields" : {
+    "myDB_0" : {
+      "localhost_12913" : "SLAVE",
+      "localhost_12914" : "SLAVE",
+      "localhost_12917" : "MASTER"
+    },
+    "myDB_1" : {
+      "localhost_12916" : "SLAVE",
+      "localhost_12917" : "MASTER"
+      "localhost_12918" : "OFFLINE"
+    },
+    "myDB_2" : {
+      "localhost_12913" : "MASTER",
+      "localhost_12917" : "SLAVE",
+      "localhost_12918" : "OFFLINE"
+    },
+    "myDB_3" : {
+      "localhost_12915" : "MASTER",
+      "localhost_12917" : "SLAVE",
+      "localhost_12918" : "OFFLINE"
+    },
+    "myDB_4" : {
+      "localhost_12916" : "MASTER",
+      "localhost_12917" : "SLAVE",
+      "localhost_12918" : "OFFLINE"
+    },
+    "myDB_5" : {
+      "localhost_12913" : "SLAVE",
+      "localhost_12914" : "MASTER",
+      "localhost_12915" : "SLAVE"
+    }
+  },
+  "listFields" : {
+    "myDB_0" : [ "localhost_12917", "localhost_12913", "localhost_12914" ],
+    "myDB_1" : [ "localhost_12917", "localhost_12916", "localhost_12918" ],
+    "myDB_2" : [ "localhost_12913", "localhost_12917", "localhost_12918" ],
+    "myDB_3" : [ "localhost_12915", "localhost_12917", "localhost_12918" ],
+    "myDB_4" : [ "localhost_12916", "localhost_12917", "localhost_12918" ],
+    "myDB_5" : [ "localhost_12914", "localhost_12915", "localhost_12913" ]
+  },
+  "simpleFields" : {
+    "IDEAL_STATE_MODE" : "AUTO",
+    "NUM_PARTITIONS" : "6",
+    "REPLICAS" : "3",
+    "STATE_MODEL_DEF_REF" : "MasterSlave",
+    "STATE_MODEL_FACTORY_NAME" : "DEFAULT"
+  }
+}
+
+ExternalView for myDB:
+{
+  "id" : "myDB",
+  "mapFields" : {
+  },
+  "listFields" : {
+  },
+  "simpleFields" : {
+    "BUCKET_SIZE" : "0"
+  }
+}
+```
+
+As we\'ve seen in this Quickstart, Helix takes care of partitioning, load balancing, elasticity, failure detection and recovery.
 
 ##### ZOOINSPECTOR
 
-Use ZooInspector that comes with zookeeper to browse the data. This is a java applet ( make sure you have X windows)
+You can view all of the underlying data by going direct to zookeeper.  Use ZooInspector that comes with zookeeper to browse the data. This is a java applet (make sure you have X windows)
+
 To start zooinspector run the following command from <zk_install_directory>/contrib/ZooInspector
       
     java -cp zookeeper-3.3.3-ZooInspector.jar:lib/jtoaster-1.0.4.jar:../../lib/log4j-1.2.15.jar:../../zookeeper-3.3.3.jar org.apache.zookeeper.inspector.ZooInspector
+
+#### Next
+
+Now that you understand the idea of Helix, read the [tutorial](./tutorial.html) to learn how to choose the right state model and constraints for your system, and how to implement it.  In many cases, the built-in features meet your requirements.  And best of all, Helix is a customizable framework, so you can plug in your own behavior, while retaining the automation provided by Helix.
+

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/93a8770e/src/site/markdown/Tutorial.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/Tutorial.md b/src/site/markdown/Tutorial.md
index 8386306..27f9fd9 100644
--- a/src/site/markdown/Tutorial.md
+++ b/src/site/markdown/Tutorial.md
@@ -17,40 +17,74 @@ specific language governing permissions and limitations
 under the License.
 -->
 
-Lets walk through the steps in building a distributed system using Helix.
+# Helix Tutorial
 
-### Start zookeeper
+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.
 
-This starts a zookeeper in standalone mode. For production deployment, see [Apache Zookeeper] page for instructions.
+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. [State Machines](./tutorial_state.html)
+6. [Messaging](./tutorial_messaging.html)
+7. [Customized health check](./tutorial_health.html)
+8. [Throttling](./tutorial_throttling.html)
+9. [Application Property Store](./tutorial_propstore.html)
+10. [Admin Interface](./tutorial_admin.html)
+
+### Preliminaries
+
+First, we need to set up the system.  Let\'s walk through the steps in building a distributed system using Helix.
+
+### 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 cluster
+### Create a cluster
 
-Creating a cluster will create appropriate znodes on zookeeper.   
+Creating a cluster will define the cluster in appropriate znodes on zookeeper.   
+
+Using the java API:
 
 ```
-    //Create setuptool instance
+    // Create setup tool instance
+    // Note: ZK_ADDRESS is the host:port of Zookeeper
+    String ZK_ADDRESS = "localhost:2199";
     admin = new ZKHelixAdmin(ZK_ADDRESS);
+
     String CLUSTER_NAME = "helix-demo";
     //Create cluster namespace in zookeeper
-    admin.addCluster(clusterName);
+    admin.addCluster(CLUSTER_NAME);
 ```
 
 OR
 
+Using the command-line interface:
+
 ```
     ./helix-admin.sh --zkSvr localhost:2199 --addCluster helix-demo 
 ```
 
 
-### Configure nodes
-
-Add new nodes to the cluster, configure new nodes in the cluster. Each node in the cluster must be uniquely identifiable. 
-Most commonly used convention is hostname:port.
+### 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.
 
 ```
     String CLUSTER_NAME = "helix-demo";
@@ -64,40 +98,57 @@ Most commonly used convention is hostname:port.
       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
 
-Resource represents the actual task performed by the nodes. It can be a database, index, topic, queue or any other processing.
-A Resource can be divided into many sub parts called as partitions. 
+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.
 
-#### Define state model and constraints
+The allowed states: 
 
-For scalability and fault tolerance each partition can have one or more replicas. 
-State model allows one to declare the system behavior by first enumerating the various STATES and TRANSITIONS between them.
-A simple model is ONLINE-OFFLINE where ONLINE means the task is active and OFFLINE means its not active.
-You can also specify how of replicas must be in each state. 
-For example In a Search System, one might need more than one node serving the same index. 
-Helix allows one to express this via constraints on each STATE.   
+* MASTER
+* SLAVE
+* OFFLINE
 
-The following snippet shows how to declare the state model and constraints for MASTER-SLAVE model.
+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. Lower the rank higher the
-    // priority
+    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);
 
@@ -108,26 +159,24 @@ The following snippet shows how to declare the state model and constraints for M
     builder.addTransition(MASTER, SLAVE);
 
     // set constraints on states.
-    // static constraint
+
+    // static constraint: upper bound of 1 MASTER
     builder.upperBound(MASTER, 1);
-    // dynamic constraint, R means it should be derived based on the replica,
-    // this allows use different replication factor for each resource without 
-    //having to define a new state model
+
+    // 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();
     admin.addStateModelDef(CLUSTER_NAME, STATE_MODEL_NAME, myStateModel);
-   
 ```
 
-
-
- 
 #### 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 and placing it on a particular node.
-
+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
 
@@ -135,200 +184,16 @@ There are 3 assignment modes Helix can operate on
 * AUTO: Application decides the placement but Helix decides the state of a partition.
 * CUSTOM: Application controls the placement and state of a partition.
 
-For more info on the modes see the *partition placement* section on [Features](./Features.html) page.
+For more info on the assignment modes, see [Rebalancing Algorithms](./tutorial_rebalance.html) of the tutorial.
 
 ```
-    String RESOURCE_NAME="MyDB";
-    int NUM_PARTITIONs=6;
+    String RESOURCE_NAME = "MyDB";
+    int NUM_PARTITIONS = 6;
     STATE_MODEL_NAME = "MasterSlave";
     String MODE = "AUTO";
     int NUM_REPLICAS = 2;
+
     admin.addResource(CLUSTER_NAME, RESOURCE_NAME, NUM_PARTITIONS, STATE_MODEL_NAME, MODE);
     admin.rebalance(CLUSTER_NAME, RESOURCE_NAME, NUM_REPLICAS);
 ```
 
-### Starting a Helix based process
-
-The first step of using the Helix api will be creating a Helix manager instance. 
-It requires the following parameters:
- 
-* clusterName: A logical name to represent the group of nodes
-* instanceName: A logical name of the process creating the manager instance. Generally this is host:port.
-* instanceType: Type of the process. This can be one of the following types:
-    * CONTROLLER: Process that controls the cluster, any number of controllers can be started but only one will be active at any given time.
-    * PARTICIPANT: Process that performs the actual task in the distributed system. 
-    * SPECTATOR: Process that observes the changes in the cluster.
-    * ADMIN: To carry out system admin actions.
-* zkConnectString: Connection string to Zookeeper. This is of the form host1:port1,host2:port2,host3:port3. 
-
-```
-      manager = HelixManagerFactory.getZKHelixManager(clusterName,
-                                                      instanceName,
-                                                      instanceType,
-                                                      zkConnectString);
-```
-                                                      
-
-
-### Participant
-Starting up a participant is pretty straightforward. After the Helix manager instance is created, only thing that needs to be registered is the state model factory. 
-The Methods on the State Model will be called when controller sends transitions to the Participant.
-
-```
-      manager = HelixManagerFactory.getZKHelixManager(clusterName,
-                                                          instanceName,
-                                                          InstanceType.PARTICIPANT,
-                                                          zkConnectString);
-     StateMachineEngine stateMach = manager.getStateMachineEngine();
-     //create a stateModelFactory that returns a statemodel object for each partition. 
-     stateModelFactory = new OnlineOfflineStateModelFactory();     
-     stateMach.registerStateModelFactory(stateModelType, stateModelFactory);
-     manager.connect();
-```
-
-```
-public class OnlineOfflineStateModelFactory extends
-        StateModelFactory<StateModel> {
-    @Override
-    public StateModel createNewStateModel(String stateUnitKey) {
-        OnlineOfflineStateModel stateModel = new OnlineOfflineStateModel();
-        return stateModel;
-    }
-    @StateModelInfo(states = "{'OFFLINE','ONLINE'}", initialState = "OFFINE")
-    public static class OnlineOfflineStateModel extends StateModel {
-        @Transition(from = "OFFLINE", to = "ONLINE")
-        public void onBecomeOnlineFromOffline(Message message,
-                NotificationContext context) {
-            System.out
-                    .println("OnlineOfflineStateModel.onBecomeOnlineFromOffline()");
-            //Application logic to handle transition 
-        }
-        @Transition(from = "ONLINE", to = "OFFLINE")
-        public void onBecomeOfflineFromOnline(Message message,
-                NotificationContext context) {
-            System.out
-                        .println("OnlineOfflineStateModel.onBecomeOfflineFromOnline()");
-            //Application logic to handle transition
-        }
-    }
-}
-```
-
-### Controller Code
-Controller needs to know about all changes in the cluster. Helix comes with default implementation to handle all changes in the cluster. 
-If you have a need to add additional functionality, see GenericHelixController on how to configure the pipeline.
-
-
-```
-      manager = HelixManagerFactory.getZKHelixManager(clusterName,
-                                                          instanceName,
-                                                          InstanceType.CONTROLLER,
-                                                          zkConnectString);
-     manager.connect();
-     GenericHelixController controller = new GenericHelixController();
-     manager.addConfigChangeListener(controller);
-     manager.addLiveInstanceChangeListener(controller);
-     manager.addIdealStateChangeListener(controller);
-     manager.addExternalViewChangeListener(controller);
-     manager.addControllerListener(controller);
-```
-This above snippet shows how the controller is started. You can also start the controller using command line interface.
-  
-```
-cd helix
-mvn clean install -Dmaven.test.skip=true
-cd helix-core/target/helix-core-pkg/bin
-chmod +x *
-./run-helix-controller.sh --zkSvr <ZookeeperServerAddress(Required)>  --cluster <Cluster name (Required)>
-```
-See controller deployment modes section in [Features](./Features.html) page for different ways to deploy the controller.
-
-### Spectator Code
-A spectator simply observes all cluster is notified when the state of the system changes. Helix consolidates the state of entire cluster in one Znode called ExternalView.
-Helix provides a default implementation RoutingTableProvider that caches the cluster state and updates it when there is a change in the cluster
-
-```
-manager = HelixManagerFactory.getZKHelixManager(clusterName,
-                                                          instanceName,
-                                                          InstanceType.PARTICIPANT,
-                                                          zkConnectString);
-manager.connect();
-RoutingTableProvider routingTableProvider = new RoutingTableProvider();
-manager.addExternalViewChangeListener(routingTableProvider);
-
-```
-
-In order to figure out who is serving a partition, here are the apis
-
-```
-instances = routingTableProvider.getInstances("DBNAME", "PARITION_NAME", "PARTITION_STATE");
-```
-
-### Zookeeper znode layout.
-
-See  *Helix znode layout* section in [Architecture](./Architecture.html) page.
-
-
-###  Helix Admin operations
-
-Helix provides multiple ways to administer the cluster. It has a command line interface and also a REST interface.
-
-```
-cd helix
-mvn clean install -Dmaven.test.skip=true
-cd helix-core/target/helix-core-pkg/bin
-chmod +x *
-./helix-admin.sh --help
-Provide zookeeper address. Required for all commands  
-   --zkSvr <ZookeeperServerAddress(Required)>       
-
-Add a new cluster                                                          
-   --addCluster <clusterName>                              
-
-Add a new Instance to a cluster                                    
-   --addNode <clusterName InstanceAddress(host:port)>                                      
-
-Add a State model to a cluster                                     
-   --addStateModelDef <clusterName <filename>>    
-
-Add a resource to a cluster            
-   --addResource <clusterName resourceName partitionNum stateModelRef <mode(AUTO_REBALANCE|AUTO|CUSTOM)>>      
-
-Upload an IdealState(Partition to Node Mapping)                                         
-   --addIdealState <clusterName resourceName <filename>>            
-
-Delete a cluster
-   --dropCluster <clusterName>                                                                         
-
-Delete a resource
-   --dropResource <clusterName resourceName>                                                           Drop an existing resource from a cluster
-
-Drop an existing Instance from a cluster    
-   --dropNode <clusterName InstanceAddress(host:port)>                    
-
-Enable/disable the entire cluster, this will basically pause the controller which means no transitions will be trigger, but the existing node sin the cluster continue to function 
-   --enableCluster <clusterName>
-
-Enable/disable a Instance. Useful to take a faulty node out of the cluster.
-   --enableInstance <clusterName InstanceName true/false>
-
-Enable/disable a partition
-   --enablePartition <clusterName instanceName resourceName partitionName true/false>
-
-
-   --listClusterInfo <clusterName>                                                                     Query info of a cluster
-   --listClusters                                                                                      List existing clusters
-   --listInstanceInfo <clusterName InstanceName>                                                       Query info of a Instance in a cluster
-   --listInstances <clusterName>                                                                       List Instances in a cluster
-   --listPartitionInfo <clusterName resourceName partitionName>                                        Query info of a partition
-   --listResourceInfo <clusterName resourceName>                                                       Query info of a resource
-   --listResources <clusterName>                                                                       List resources hosted in a cluster
-   --listStateModel <clusterName stateModelName>                                                       Query info of a state model in a cluster
-   --listStateModels <clusterName>                                                                     Query info of state models in a cluster
-
-```
-
-
-
-
-

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/93a8770e/src/site/markdown/index.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/index.md b/src/site/markdown/index.md
index 273a9cf..cbcb6ba 100644
--- a/src/site/markdown/index.md
+++ b/src/site/markdown/index.md
@@ -17,189 +17,78 @@ specific language governing permissions and limitations
 under the License.
 -->
 
+Navigating the Documentation
+----------------------------
 
-Pages
----------------
-* [Quickstart](./Quickstart.html)
-* [Core concepts](./Concepts.html)
-* [Architecture](./Architecture.html)
-* [Tutorial](./Tutorial.html)
-* [Features](./Features.html)
-* [ApiUsage](./ApiUsage.html)
-* [Javadocs](./apidocs/index.html)
-* [UseCases](./UseCases.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)
-
-WHAT IS HELIX
---------------
-Helix is a generic cluster management framework used for the automatic management of partitioned, replicated and distributed resources hosted on a cluster of nodes. Helix provides the following features: 
-
-1. Automatic assignment of resource/partition to nodes
-2. Node failure detection and recovery
-3. Dynamic addition of Resources 
-4. Dynamic addition of nodes to the cluster
-5. Pluggable distributed state machine to manage the state of a resource via state transitions
-6. Automatic load balancing and throttling of transitions 
-
------
-
-OVERVIEW
----------
-
-A distributed system comprises of one or more *nodes*. Depending on the purpose, each node performs a specific task. For example, in a search system it can be a index, in a pub sub system it can be a topic/queue, in storage system it can be a database. Helix refers to such tasks as a *resource*. In order to scale the system, each node is responsible for a part of task referred to as *partition*. For scalability and fault tolerance, task associated with each partition can run on multiple nodes. Helix refers to them as *replica*. 
- 
-Helix refers to each of the node in the cluster as a *PARTICIPANT*. As seen in many distributed system, there is a central component called *CONTROLLER* that co-ordinates the *PARTICIPANT*'s  during start up, failures and cluster expansion. In most distributed systems need to provide a service discovery mechanism for external entities like clients, request routers, load balancers to interact with the distributed system. These external entities are referred as SPECTATOR.
-
-Helix is built on top of Zookeeper and uses it store the cluster state and serves as the communication channel between CONTROLLER, PARTICIPANT and spectator. There is no single point of failure in Helix.
-
-Helix managed distributed system architecture.
-
-![Helix Design](images/HELIX-components.png)
-
-
-WHAT MAKES IT GENERIC
----------------------
-
-Even though most distributed systems follow similar mechanism of co-ordinating the nodes through a controller or zookeeper, the implementation is 
-specific to the use case. Helix abstracts out the cluster management of distributed system from its core functionality. 
-
-Helix allows one to express the system behavior via  
-
-#### STATE MACHINE
-State machine allows one to express the different roles a replica can take up and transition from one role to another.
-
-* Set of valid states (S1,S2,S3 etc) for each replica
-* Set of valid transitions that allow replicas to transition from one state to another. 
-   
-#### CONSTRAINTS
-Helix allows one to specify constraints on states and transitions. 
-
-* Minimum and maximum number of replicas that need to be in a given state. For example S3: Max=1 S2: Min=2, Max=3
-* Set a max concurrency limit of each transition type. For example, if S1-S2 involves moving data, one can limit the data movement by setting limit on maximum number of concurrent (S1->S2) transitions per node to 5.  
+### Conceptual Understanding
 
-#### OBJECTIVES
-Objectives are used to control the replica placement strategy across the nodes. For example
+[Concepts / Terminology](./Concepts.html)
 
-* Replicas must be evenly distributed across nodes.  
-* Replicas of one partition must be on different nodes/racks.
-* When a node fails, its load must be evenly distributed among rest of the nodes.
-* When new nodes are added, it must result in minimum number of movements.    
+[Architecture](./Architecture.html)
 
-EXAMPLE
--------
+### Hands-on Helix
 
-Consider the simple use cases where all partitions are actively processing search query request. 
-We can express it using a OnlineOffline state model where a task can be either 
-ONLINE (task is active) or OFFLINE (not active).
+[Quickstart](./Quickstart.html)
 
-Similarly take a slightly more complicated system, where we need three states OFFLINE, SLAVE and MASTER. 
+[Tutorial](./Tutorial.html)
 
-The following state machine table provides transition from start state to End state. For example, if the current state is Offline and the target state is Master,
- the table says that the first transition must be Offline-Slave and then Slave-Master.
+[Javadocs](http://helix.incubator.apache.org/apidocs/index.html)
 
-```
-          OFFLINE  | SLAVE  |  MASTER  
-         _____________________________
-        |          |        |         |
-OFFLINE |   N/A    | SLAVE  | SLAVE   |
-        |__________|________|_________|
-        |          |        |         |
-SLAVE   |  OFFLINE |   N/A  | MASTER  |
-        |__________|________|_________|
-        |          |        |         |
-MASTER  | SLAVE    | SLAVE  |   N/A   |
-        |__________|________|_________|
-
-```
-
-
-Another unique feature of Helix is it allows one to add constraints on each state and transitions. 
-
-For example 
-In a OnlineOffline state model one can enforce a constraint that there should be 3 replicas in ONLINE state per partition.
+### Recipes
 
-    ONLINE:3
+[Distributed lock manager](./recipes/lock_manager.html)
 
-In a MasterSlave state model with a replication factor of 3 one can enforce a single master by specifying constraints on number of Masters and Slaves.
+[Rabbit MQ consumer group](./recipes/rabbitmq_consumer_group.html)
 
-    MASTER:1 
-    SLAVE:2
+[Rsync replicated file store](./recipes/rsync_replicated_file_store.html)
 
-Given these constraints, Helix will ensure that there is 1 Master and 2 Slaves by initiating appropriate state transitions in the cluster.
+[Service discovery](./recipes/service_discovery.html)
 
 
-Apart from Constraints on STATES, Helix supports constraints on transitions as well. For example, consider a OFFLINE-BOOTSTRAP transition where a service download the index over the network. 
-Without any throttling during start up of a cluster, all nodes might start downloading at once which might impact the system stability. 
-Using Helix with out changing any application code, one can simply place a constraint of max 5 transitions OFFLINE-BOOTSTRAP across the entire cluster.
-
-The constraints can be at any scope node, resource, transition type and 
-
-Helix comes with 3 commonly used state models, you can also plugin your custom state model. 
-
-1. Master, Slave
-2. Online, Offline
-3. Leader, Standby.
-
-
-Helix framework can be used to build distributed, scalable, elastic and fault tolerant systems by configuring the distributed state machine and its constraints based on application requirements. The application has to provide the implementation for handling state transitions appropriately. Example 
+What Is Helix
+--------------
+Helix is a generic _cluster management_ framework used for the automatic management of partitioned, replicated and distributed resources hosted on a cluster of nodes. 
 
 
-```
-MasterSlaveStateModel extends HelixStateModel {
-
-  void onOfflineToSlave(Message m, NotificationContext context){
-    print("Transitioning from Offline to Slave for resource:"+ m.getResourceName() + " and partition:"+ m.getPartitionName());
-  }
-  void onSlaveToMaster(Message m, NotificationContext context){
-    print("Transitioning from Slave to Master for resource:"+ m.getResourceName() + " and partition:"+ m.getPartitionName());
-  }
-  void onMasterToSlave(Message m, NotificationContext context){
-    print("Transitioning from Master to Slave for resource:"+ m.getResourceName() + " and partition:"+ m.getPartitionName());
-  }
-  void onSlaveToOffline(Message m, NotificationContext context){
-    print("Transitioning from Slave to Offline for resource:"+ m.getResourceName() + " and partition:"+ m.getPartitionName());
-  }
-}
-```
+What Is Cluster Management
+--------------------------
+To understand Helix, first you need to understand what is _cluster management_.  A distributed system typically runs on multiple nodes for the following reasons:
 
-Each transition results in a partition moving from its CURRENT state to a NEW state. These transitions are triggered on changes in the cluster state like 
+* scalability
+* fault tolerance
+* load balancing
 
-* Node start up
-* Node soft and hard failures 
-* Addition of resources
-* Addition of nodes
+Each node performs one or more of the primary function of the cluster, such as storing/serving data, producing/consuming data streams, etc.  Once configured for your system, Helix acts as the global brain for the system.  It is designed to make decisions that cannot be made in isolation.  Examples of decisions that require global knowledge and coordination:
 
+* scheduling of maintainence tasks, such as backups, garbage collection, file consolidation, index rebuilds
+* repartitioning of data or resources across the cluster
+* informing dependent systems of changes so they can react appropriately to cluster changes
+* throttling system tasks and changes
 
-TERMINOLOGIES
--------------
-Helix uses terms that are commonly used to describe distributed data system concepts. 
+While it is possible to integrate these functions into the distributed system, it complicates the code.  Helix has abstracted common cluster management tasks, enabling the system builder to model the desired behavior in a declarative state model, and let Helix manage the coordination.  The result is less new code to write, and a robust, highly operable system.
 
-1. Cluster: A logical set of Instances that perform a similar set of activities. 
-2. Instance: An Instance is a logical entity in the cluster that can be identified by a unique Id. 
-3. Node: A Node is a physical entity in the cluster. A Node can have one or more logical Instances. 
-4. Resource: A resource represents the logical entity hosted by the distributed system. It can be a database name, index or a task group name 
-5. Partition: A resource is generally split into one or more partitions.
-6. Replica: Each partition can have one or more replicas
-7. State: Each replica can have state associated with it. For example: Master, Slave, Leader, Stand By, Offline, Online etc. 
 
+Key Features of Helix
+---------------------
+1. Automatic assignment of resource/partition to nodes
+2. Node failure detection and recovery
+3. Dynamic addition of Resources 
+4. Dynamic addition of nodes to the cluster
+5. Pluggable distributed state machine to manage the state of a resource via state transitions
+6. Automatic load balancing and throttling of transitions 
 
 
-WHY HELIX
--------------
-Helix approach of using a distributed state machine with constraints on state and transitions has the following benefits
+Why Helix
+---------
+Modeling a distributed system as a state machine with constraints on state and transitions has the following benefits:
 
-* Abstract cluster management from the core functionality.
-* Quick transformation from a single node system to a distributed system.
-* PARTICIPANT is not aware of the global state since they simply have to follow the instructions issued by the CONTROLLER. This design provide clear division of responsibilities and easier to debug issues.
-* Since the controller's goal is to satisfy state machine constraints at all times, use cases like cluster startup, node failure, cluster expansion are solved in a similar way.
+* Separates cluster management from the core functionality.
+* Quick transformation from a single node system to an operable, distributed system.
+* Simplicity: System components do not have to manage global cluster.  This division of labor makes it easier to build, debug, and maintain your system.
 
 
-BUILD INSTRUCTIONS
--------------------------
+Build Instructions
+------------------
 
 Requirements: Jdk 1.6+, Maven 2.0.8+
 
@@ -215,13 +104,13 @@ Maven dependency
     <dependency>
       <groupId>org.apache.helix</groupId>
       <artifactId>helix-core</artifactId>
-      <version>0.6.0-incubating</version>
+      <version>0.6.1-incubating</version>
     </dependency>
 ```
 
 [Download](./download.html) Helix artifacts from here.
    
-PUBLICATIONS
+Publications
 -------------
 
 * Untangling cluster management using Helix at [SOCC Oct 2012](http://www.socc2012.org/home/program)  
@@ -229,5 +118,8 @@ PUBLICATIONS
     - [presentation](http://www.slideshare.net/KishoreGopalakrishna/helix-socc-v10final)
 * Building distributed systems using Helix Apache Con Feb 2013
     - [presentation at ApacheCon](http://www.slideshare.net/KishoreGopalakrishna/apache-con-buildingddsusinghelix)
-    - [presentation at vmware](http://www.slideshare.net/KishoreGopalakrishna/apache-helix-presentation-at-vmware)
+    - [presentation at VMWare](http://www.slideshare.net/KishoreGopalakrishna/apache-helix-presentation-at-vmware)
+* Data driven testing:
+    - [short talk at LSPE meetup](http://www.slideshare.net/KishoreGopalakrishna/data-driven-testing)
+    - [paper DBTest 2013 acm SIGMOD:will be published on Jun 24, 2013](http://dbtest2013.soe.ucsc.edu/Program.htm)
 

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/93a8770e/src/site/markdown/tutorial_admin.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/tutorial_admin.md b/src/site/markdown/tutorial_admin.md
new file mode 100644
index 0000000..57f34fc
--- /dev/null
+++ b/src/site/markdown/tutorial_admin.md
@@ -0,0 +1,167 @@
+<!---
+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.
+-->
+
+# Helix Tutorial: Admin Operations
+
+Helix provides interfaces for the operator to administer the cluster.  For convenience, there is a command line interface as well as a REST interface.
+
+###  Helix Admin operations
+
+First, make sure you get to the command-line tool, or include it in your shell PATH.
+
+```
+cd helix/helix-core/target/helix-core-pkg/bin
+```
+
+Get help
+
+```
+./helix-admin.sh --help
+```
+
+All other commands have this form:
+
+```
+./helix-admin.sh --zkSvr <ZookeeperServerAddress (Required)> <command> <parameters>
+```
+
+Now, here are the admin commands:
+
+Add a new cluster
+
+```
+   --addCluster <clusterName>                              
+```
+
+Add a new Instance to a cluster
+
+```
+   --addNode <clusterName> <InstanceAddress (host:port)>
+```
+
+Add a State model to a cluster
+_WE NEED A SPEC FOR A VALID STATE MODEL_                                    
+
+```
+   --addStateModelDef <clusterName> <filename>>    
+```
+
+Add a resource to a cluster
+
+```
+   --addResource <clusterName> <resourceName> <partitionNum> <stateModelRef> <mode (AUTO_REBALANCE|AUTO|CUSTOM)>
+```
+
+Upload an IdealState (Partition to Node Mapping)
+_WE NEED A SPEC FOR A VALID IDEAL STATE_
+
+```
+   --addIdealState <clusterName> <resourceName> <filename>
+```
+
+Delete a cluster
+
+```
+   --dropCluster <clusterName>                                                                         
+```
+
+Delete a resource (drop an existing resource from a cluster)
+
+```
+   --dropResource <clusterName> <resourceName>
+```
+
+Drop an existing instance from a cluster
+
+```
+   --dropNode <clusterName> <InstanceAddress (host:port)>
+```
+
+Enable/disable the entire cluster. This will pause the controller, which means no transitions will be trigger, but the existing nodes in the cluster continue to function, but without any management by the controller.
+
+```
+   --enableCluster <clusterName> <true/false>
+```
+
+Enable/disable an instance. Useful to take a node out of the cluster for maintenance/upgrade.
+
+```
+   --enableInstance <clusterName> <InstanceName> <true/false>
+```
+
+Enable/disable a partition
+
+```
+   --enablePartition <clusterName> <instanceName> <resourceName> <partitionName> <true/false>
+```
+
+Query info of a cluster
+
+```
+   --listClusterInfo <clusterName>
+```
+
+List existing clusters (remember, Helix can manage multiple clusters)
+
+```
+   --listClusters
+```
+
+Query info of a single Instance in a cluster
+
+```
+   --listInstanceInfo <clusterName> <InstanceName>
+```
+
+List instances in a cluster
+
+```
+   --listInstances <clusterName>
+```
+
+Query info of a partition
+
+```
+   --listPartitionInfo <clusterName> <resourceName> <partitionName>
+```
+
+Query info of a resource
+
+```
+   --listResourceInfo <clusterName> <resourceName>
+```
+
+List resources hosted in a cluster
+
+```
+   --listResources <clusterName>
+```
+
+Query info of a state model in a cluster
+
+```
+   --listStateModel <clusterName> <stateModelName>
+```
+
+Query info of state models in a cluster
+
+```
+   --listStateModels <clusterName>                                                                     
+```
+

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/93a8770e/src/site/markdown/tutorial_controller.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/tutorial_controller.md b/src/site/markdown/tutorial_controller.md
new file mode 100644
index 0000000..17cd532
--- /dev/null
+++ b/src/site/markdown/tutorial_controller.md
@@ -0,0 +1,90 @@
+<!---
+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.
+-->
+
+# Helix Tutorial: Controller
+
+Next, let\'s implement the controller.  This is the brain of the cluster.  Helix makes sure there is exactly one active controller running the cluster.
+
+### Start the Helix agent
+
+
+It requires the following parameters:
+ 
+* clusterName: A logical name to represent the group of nodes
+* instanceName: A logical name of the process creating the manager instance. Generally this is host:port.
+* instanceType: Type of the process. This can be one of the following types, in this case use CONTROLLER:
+    * CONTROLLER: Process that controls the cluster, any number of controllers can be started but only one will be active at any given time.
+    * PARTICIPANT: Process that performs the actual task in the distributed system. 
+    * SPECTATOR: Process that observes the changes in the cluster.
+    * ADMIN: To carry out system admin actions.
+* zkConnectString: Connection string to Zookeeper. This is of the form host1:port1,host2:port2,host3:port3. 
+
+```
+      manager = HelixManagerFactory.getZKHelixManager(clusterName,
+                                                      instanceName,
+                                                      instanceType,
+                                                      zkConnectString);
+```
+
+### Controller Code
+
+The Controller needs to know about all changes in the cluster. Helix takes care of this with the default implementation.
+If you need additional functionality, see GenericHelixController on how to configure the pipeline.
+
+```
+      manager = HelixManagerFactory.getZKHelixManager(clusterName,
+                                                          instanceName,
+                                                          InstanceType.CONTROLLER,
+                                                          zkConnectString);
+     manager.connect();
+     GenericHelixController controller = new GenericHelixController();
+     manager.addConfigChangeListener(controller);
+     manager.addLiveInstanceChangeListener(controller);
+     manager.addIdealStateChangeListener(controller);
+     manager.addExternalViewChangeListener(controller);
+     manager.addControllerListener(controller);
+```
+The snippet above shows how the controller is started. You can also start the controller using command line interface.
+  
+```
+cd helix/helix-core/target/helix-core-pkg/bin
+./run-helix-controller.sh --zkSvr <Zookeeper ServerAddress (Required)>  --cluster <Cluster name (Required)>
+```
+
+### Controller deployment modes
+
+Helix provides multiple options to deploy the controller.
+
+#### STANDALONE
+
+The Controller can be started as a separate process to manage a cluster. This is the recommended approach. However, since one controller can be a single point of failure, multiple controller processes are required for reliability.  Even if multiple controllers are running, only one will be actively managing the cluster at any time and is decided by a leader-election process. If the leader fails, another leader will take over managing the cluster.
+
+Even though we recommend this method of deployment, it has the drawback of having to manage an additional service for each cluster. See Controller As a Service option.
+
+#### EMBEDDED
+
+If setting up a separate controller process is not viable, then it is possible to embed the controller as a library in each of the participants.
+
+#### CONTROLLER AS A SERVICE
+
+One of the cool feature we added in Helix was to use a set of controllers to manage a large number of clusters. 
+
+For example if you have X clusters to be managed, instead of deploying X*3 (3 controllers for fault tolerance) controllers for each cluster, one can deploy just 3 controllers.  Each controller can manage X/3 clusters.  If any controller fails, the remaining two will manage X/2 clusters.
+
+

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/93a8770e/src/site/markdown/tutorial_health.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/tutorial_health.md b/src/site/markdown/tutorial_health.md
new file mode 100644
index 0000000..ae29436
--- /dev/null
+++ b/src/site/markdown/tutorial_health.md
@@ -0,0 +1,42 @@
+<!---
+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.
+-->
+
+# Helix Tutorial: Customizing Health Checks
+
+In this chapter, we\'ll learn how to customize the health check, based on metrics of your distributed system.  
+
+### Health Checks
+
+Note: _this in currently in development mode, not yet ready for production._
+
+Helix provides the ability for each node in the system to report health metrics on a periodic basis. 
+
+Helix supports multiple ways to aggregate these metrics:
+
+* SUM
+* AVG
+* EXPONENTIAL DECAY
+* WINDOW
+
+Helix persists the aggregated value only.
+
+Applications can define a threshold on the aggregate values according to the SLAs, and when the SLA is violated Helix will fire an alert. 
+Currently Helix only fires an alert, but in a future release we plan to use these metrics to either mark the node dead or load balance the partitions.
+This feature will be valuable for distributed systems that support multi-tenancy and have a large variation in work load patterns.  In addition, this can be used to detect skewed partitions (hotspots) and rebalance the cluster.
+

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/93a8770e/src/site/markdown/tutorial_messaging.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/tutorial_messaging.md b/src/site/markdown/tutorial_messaging.md
new file mode 100644
index 0000000..f3fef10
--- /dev/null
+++ b/src/site/markdown/tutorial_messaging.md
@@ -0,0 +1,67 @@
+<!---
+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.
+-->
+
+# Helix Tutorial: Messaging
+
+In this chapter, we\'ll learn about messaging, a convenient feature in Helix for sending messages between nodes of a cluster.  This is an interesting feature which is quite useful in practice. It is common that nodes in a distributed system require a mechanism to interact with each other.  
+
+### Example: Bootstrapping a Replica
+
+Consider a search system  where the index replica starts up and it does not have an index. A typical solution is to get the index from a common location, or to copy the index from another replica.
+
+Helix provides a messaging api for intra-cluster communication between nodes in the system.  Helix provides a mechanism to specify the message recipient in terms of resource, partition, and state rather than specifying hostnames.  Helix ensures that the message is delivered to all of the required recipients. In this particular use case, the instance can specify the recipient criteria as all replicas of the desired partition to bootstrap.
+Since Helix is aware of the global state of the system, it can send the message to appropriate nodes. Once the nodes respond, Helix provides the bootstrapping replica with all the responses.
+
+This is a very generic api and can also be used to schedule various periodic tasks in the cluster, such as data backups, log cleanup, etc.
+System Admins can also perform ad-hoc tasks, such as on-demand backups or a system command (such as rm -rf ;) across all nodes of the cluster
+
+```
+      ClusterMessagingService messagingService = manager.getMessagingService();
+
+      // Construct the Message
+      Message requestBackupUriRequest = new Message(
+          MessageType.USER_DEFINE_MSG, UUID.randomUUID().toString());
+      requestBackupUriRequest
+          .setMsgSubType(BootstrapProcess.REQUEST_BOOTSTRAP_URL);
+      requestBackupUriRequest.setMsgState(MessageState.NEW);
+
+      // Set the Recipient criteria: all nodes that satisfy the criteria will receive the message
+      Criteria recipientCriteria = new Criteria();
+      recipientCriteria.setInstanceName("%");
+      recipientCriteria.setRecipientInstanceType(InstanceType.PARTICIPANT);
+      recipientCriteria.setResource("MyDB");
+      recipientCriteria.setPartition("");
+
+      // Should be processed only by process(es) that are active at the time of sending the message
+      //   This means if the recipient is restarted after message is sent, it will not be processe.
+      recipientCriteria.setSessionSpecific(true);
+
+      // wait for 30 seconds
+      int timeout = 30000;
+
+      // the handler that will be invoked when any recipient responds to the message.
+      BootstrapReplyHandler responseHandler = new BootstrapReplyHandler();
+
+      // this will return only after all recipients respond or after timeout
+      int sentMessageCount = messagingService.sendAndWait(recipientCriteria,
+          requestBackupUriRequest, responseHandler, timeout);
+```
+
+See HelixManager.DefaultMessagingService in [Javadocs](http://helix.incubator.apache.org/apidocs/reference/org/apache/helix/messaging/DefaultMessagingService.html) for more info.
+

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/93a8770e/src/site/markdown/tutorial_participant.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/tutorial_participant.md b/src/site/markdown/tutorial_participant.md
new file mode 100644
index 0000000..49d0701
--- /dev/null
+++ b/src/site/markdown/tutorial_participant.md
@@ -0,0 +1,98 @@
+<!---
+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.
+-->
+
+# Helix Tutorial: Participant
+
+In this chapter, we\'ll learn how to implement a PARTICIPANT, which is a primary functional component of a distributed system.
+
+
+### Start the Helix agent
+
+The Helix agent is a common component that connects each system component with the controller.
+
+It requires the following parameters:
+ 
+* clusterName: A logical name to represent the group of nodes
+* instanceName: A logical name of the process creating the manager instance. Generally this is host:port.
+* instanceType: Type of the process. This can be one of the following types, in this case, use PARTICIPANT
+    * CONTROLLER: Process that controls the cluster, any number of controllers can be started but only one will be active at any given time.
+    * PARTICIPANT: Process that performs the actual task in the distributed system. 
+    * SPECTATOR: Process that observes the changes in the cluster.
+    * ADMIN: To carry out system admin actions.
+* zkConnectString: Connection string to Zookeeper. This is of the form host1:port1,host2:port2,host3:port3. 
+
+After the Helix manager instance is created, only thing that needs to be registered is the state model factory. 
+The methods of the State Model will be called when controller sends transitions to the Participant.  In this example, we'll use the OnlineOffline factory.  Other options include:
+
+* MasterSlaveStateModelFactory
+* _NEED TO COMPLETE THIS LIST_
+
+```
+      manager = HelixManagerFactory.getZKHelixManager(clusterName,
+                                                          instanceName,
+                                                          InstanceType.PARTICIPANT,
+                                                          zkConnectString);
+     StateMachineEngine stateMach = manager.getStateMachineEngine();
+
+     //create a stateModelFactory that returns a statemodel object for each partition. 
+     stateModelFactory = new OnlineOfflineStateModelFactory();     
+     stateMach.registerStateModelFactory(stateModelType, stateModelFactory);
+     manager.connect();
+```
+
+Helix doesn\'t know what it means to change from OFFLIN\-\-\>ONLINE or ONLINE\-\-\>OFFLINE.  The following code snippet shows where you insert your system logic for these two state transitions.
+
+```
+public class OnlineOfflineStateModelFactory extends
+        StateModelFactory<StateModel> {
+    @Override
+    public StateModel createNewStateModel(String stateUnitKey) {
+        OnlineOfflineStateModel stateModel = new OnlineOfflineStateModel();
+        return stateModel;
+    }
+    @StateModelInfo(states = "{'OFFLINE','ONLINE'}", initialState = "OFFINE")
+    public static class OnlineOfflineStateModel extends StateModel {
+
+        @Transition(from = "OFFLINE", to = "ONLINE")
+        public void onBecomeOnlineFromOffline(Message message,
+                NotificationContext context) {
+
+            System.out.println("OnlineOfflineStateModel.onBecomeOnlineFromOffline()");
+
+            ////////////////////////////////////////////////////////////////////////////////////////////////
+            // Application logic to handle transition                                                     //
+            // For example, you might start a service, run initialization, etc                            //
+            ////////////////////////////////////////////////////////////////////////////////////////////////
+        }
+
+        @Transition(from = "ONLINE", to = "OFFLINE")
+        public void onBecomeOfflineFromOnline(Message message,
+                NotificationContext context) {
+
+            System.out.println("OnlineOfflineStateModel.onBecomeOfflineFromOnline()");
+
+            ////////////////////////////////////////////////////////////////////////////////////////////////
+            // Application logic to handle transition                                                     //
+            // For example, you might shutdown a service, log this event, or change monitoring settings   //
+            ////////////////////////////////////////////////////////////////////////////////////////////////
+        }
+    }
+}
+```
+

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/93a8770e/src/site/markdown/tutorial_propstore.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/tutorial_propstore.md b/src/site/markdown/tutorial_propstore.md
new file mode 100644
index 0000000..6af3c28
--- /dev/null
+++ b/src/site/markdown/tutorial_propstore.md
@@ -0,0 +1,30 @@
+<!---
+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.
+-->
+
+# Helix Tutorial: Application Property Store
+
+In this chapter, we\'ll learn how to use the application property store.
+
+### Property Store
+
+It is common that an application needs support for distributed, shared data structures.  Helix uses Zookeeper to store the application data and hence provides notifications when the data changes.
+
+While you could use Zookeeper directly, Helix supports caching the data and a write-through cache. This is far more efficient than reading from Zookeeper for every access.
+
+See [HelixManager.getHelixPropertyStore](http://helix.incubator.apache.org/apidocs/reference/org/apache/helix/store/package-summary.html) for details.

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/93a8770e/src/site/markdown/tutorial_rebalance.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/tutorial_rebalance.md b/src/site/markdown/tutorial_rebalance.md
new file mode 100644
index 0000000..1f5930d
--- /dev/null
+++ b/src/site/markdown/tutorial_rebalance.md
@@ -0,0 +1,168 @@
+<!---
+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.
+-->
+
+# Helix Tutorial: Rebalancing Algorithms
+
+The placement of partitions in a distributed system is essential for the reliability and scalability of the system.  For example, when a node fails, it is important that the partitions hosted on that node are reallocated evenly among the remaining nodes. Consistent hashing is one such algorithm that can satisfy this guarantee.  Helix provides a variant of consistent hashing based on the RUSH algorithm.
+
+This means given a number of partitions, replicas and number of nodes, Helix does the automatic assignment of partition to nodes such that:
+
+* Each node has the same number of partitions
+* Replicas of the same partition do not stay on the same node
+* When a node fails, the partitions will be equally distributed among the remaining nodes
+* When new nodes are added, the number of partitions moved will be minimized along with satisfying the above criteria
+
+Helix employs a rebalancing algorithm to compute the _ideal state_ of the system.  When the _current state_ differs from the _ideal state_, Helix uses it as the target state of the system and computes the appropriate transitions needed to bring it to the _ideal state_.
+
+Helix makes it easy to perform this operation, while giving you control over the algorithm.  In this section, we\'ll see how to implement the desired behavior.
+
+Helix has three options for rebalancing, in increasing order of customization by the system builder:
+
+* AUTO_REBALANCE
+* AUTO
+* CUSTOM
+
+```
+            |AUTO REBALANCE|   AUTO     |   CUSTOM  |       
+            -----------------------------------------
+   LOCATION | HELIX        |  APP       |  APP      |
+            -----------------------------------------
+      STATE | HELIX        |  HELIX     |  APP      |
+            -----------------------------------------
+```
+
+
+### AUTO_REBALANCE
+
+When the idealstate mode is set to AUTO_REBALANCE, Helix controls both the location of the replica along with the state. This option is useful for applications where creation of a replica is not expensive. 
+
+For example, consider this system that uses a MasterSlave state model, with 3 partitions and 2 replicas in the ideal state.
+
+```
+{
+  "id" : "MyResource",
+  "simpleFields" : {
+    "IDEAL_STATE_MODE" : "AUTO_REBALANCE",
+    "NUM_PARTITIONS" : "3",
+    "REPLICAS" : "2",
+    "STATE_MODEL_DEF_REF" : "MasterSlave",
+  }
+  "listFields" : {
+    "MyResource_0" : [],
+    "MyResource_1" : [],
+    "MyResource_2" : []
+  },
+  "mapFields" : {
+  }
+}
+```
+
+If there are 3 nodes in the cluster, then Helix will balance the masters and slaves equally.  The ideal state is therefore:
+
+```
+{
+  "id" : "MyResource",
+  "simpleFields" : {
+    "NUM_PARTITIONS" : "3",
+    "REPLICAS" : "2",
+    "STATE_MODEL_DEF_REF" : "MasterSlave",
+  },
+  "mapFields" : {
+    "MyResource_0" : {
+      "N1" : "MASTER",
+      "N2" : "SLAVE",
+    },
+    "MyResource_1" : {
+      "N2" : "MASTER",
+      "N3" : "SLAVE",
+    },
+    "MyResource_2" : {
+      "N3" : "MASTER",
+      "N1" : "SLAVE",
+    }
+  }
+}
+```
+
+Another typical example is evenly distributing a group of tasks among the currently healthy processes. For example, if there are 60 tasks and 4 nodes, Helix assigns 15 tasks to each node. 
+When one node fails, Helix redistributes its 15 tasks to the remaining 3 nodes, resulting in a balanced 20 tasks per node. Similarly, if a node is added, Helix re-allocates 3 tasks from each of the 4 nodes to the 5th node, resulting in a balanced distribution of 12 tasks per node.. 
+
+#### AUTO
+
+When the application needs to control the placement of the replicas, use the AUTO idealstate mode.
+
+Example: In the ideal state below, the partition \'MyResource_0\' is constrained to be placed only on node1 or node2.  The choice of _state_ is still controlled by Helix.  That means MyResource_0.MASTER could be on node1 and MyResource_0.SLAVE on node2, or vice-versa but neither would be placed on node3.
+
+```
+{
+  "id" : "MyResource",
+  "simpleFields" : {
+    "IDEAL_STATE_MODE" : "AUTO",
+    "NUM_PARTITIONS" : "3",
+    "REPLICAS" : "2",
+    "STATE_MODEL_DEF_REF" : "MasterSlave",
+  }
+  "listFields" : {
+    "MyResource_0" : [node1, node2],
+    "MyResource_1" : [node2, node3],
+    "MyResource_2" : [node3, node1]
+  },
+  "mapFields" : {
+  }
+}
+```
+
+The MasterSlave state model requires that a partition has exactly one MASTER at all times, and the other replicas should be SLAVEs.  In this simple example with 2 replicas per partition, there would be one MASTER and one SLAVE.  Upon failover, a SLAVE has to assume mastership, and a new SLAVE will be generated.
+
+In this mode when node1 fails, unlike in AUTO-REBALANCE mode the partition is _not_ moved from node1 to node3. Instead, Helix will decide to change the state of MyResource_0 on node2 from SLAVE to MASTER, based on the system constraints. 
+
+#### CUSTOM
+
+Finally, Helix offers a third mode called CUSTOM, in which the application controls the placement _and_ state of each replica. The application needs to implement a callback interface that Helix invokes when the cluster state changes. 
+Within this callback, the application can recompute the idealstate. Helix will then issue appropriate transitions such that _Idealstate_ and _Currentstate_ converges.
+
+Here\'s an example, again with 3 partitions, 2 replicas per partition, and the MasterSlave state model:
+
+```
+{
+  "id" : "MyResource",
+  "simpleFields" : {
+      "IDEAL_STATE_MODE" : "CUSTOM",
+    "NUM_PARTITIONS" : "3",
+    "REPLICAS" : "2",
+    "STATE_MODEL_DEF_REF" : "MasterSlave",
+  },
+  "mapFields" : {
+    "MyResource_0" : {
+      "N1" : "MASTER",
+      "N2" : "SLAVE",
+    },
+    "MyResource_1" : {
+      "N2" : "MASTER",
+      "N3" : "SLAVE",
+    },
+    "MyResource_2" : {
+      "N3" : "MASTER",
+      "N1" : "SLAVE",
+    }
+  }
+}
+```
+
+Suppose the current state of the system is 'MyResource_0' -> {N1:MASTER, N2:SLAVE} and the application changes the ideal state to 'MyResource_0' -> {N1:SLAVE,N2:MASTER}. While the application decides which node is MASTER and which is SLAVE, Helix will not blindly issue MASTER-->SLAVE to N1 and SLAVE-->MASTER to N2 in parallel, since that might result in a transient state where both N1 and N2 are masters, which violates the MasterSlave constraint that there is exactly one MASTER at a time.  Helix will first issue MASTER-->SLAVE to N1 and after it is completed, it will issue SLAVE-->MASTER to N2.