You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@helix.apache.org by ol...@apache.org on 2012/10/23 22:48:18 UTC

[3/3] git commit: import content from github wiki as it

import content from github wiki as it


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

Branch: refs/heads/master
Commit: e98143873fd0c4638511033637a3737afef4bd53
Parents: fb44bb2
Author: olivier lamy <ol...@apache.org>
Authored: Tue Oct 23 22:31:49 2012 +0200
Committer: olivier lamy <ol...@apache.org>
Committed: Tue Oct 23 22:31:49 2012 +0200

----------------------------------------------------------------------
 pom.xml                                            |   20 +-
 src/site/markdown/ApiUsage.md                      |  238 +++++++++++++++
 src/site/markdown/Architecture.md                  |  225 ++++++++++++++
 src/site/markdown/Features.md                      |  161 ++++++++++
 src/site/markdown/Home.md                          |  135 ++++++++
 src/site/markdown/Quickstart.md                    |  130 ++++++++
 src/site/markdown/Sample_App.md                    |  206 +++++++++++++
 src/site/markdown/UseCases.md                      |   89 ++++++
 src/site/markdown/index.md                         |  135 ++++++++
 src/site/resources/images/bootstrap_statemodel.gif |  Bin 0 -> 24919 bytes
 src/site/resources/images/helix-architecture.png   |  Bin 0 -> 282390 bytes
 src/site/resources/images/helix-znode-layout.png   |  Bin 0 -> 53074 bytes
 src/site/resources/images/statemachine.png         |  Bin 0 -> 41641 bytes
 src/site/resources/images/system.png               |  Bin 0 -> 79791 bytes
 src/site/site.xml                                  |   57 ++++
 15 files changed, 1393 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/e9814387/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index b6ae767..8ae4382 100644
--- a/pom.xml
+++ b/pom.xml
@@ -17,7 +17,8 @@ KIND, either express or implied.  See the License for the
 specific language governing permissions and limitations
 under the License.
 -->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
   <modelVersion>4.0.0</modelVersion>
 
   <parent>
@@ -199,6 +200,18 @@ under the License.
           </execution>
         </executions>
       </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-site-plugin</artifactId>
+        <version>3.2</version>
+        <dependencies>
+          <dependency>
+            <groupId>org.apache.maven.doxia</groupId>
+            <artifactId>doxia-module-markdown</artifactId>
+            <version>1.3</version>
+          </dependency>
+        </dependencies>
+      </plugin>
 
     </plugins>
   </build>
@@ -208,11 +221,12 @@ under the License.
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-checkstyle-plugin</artifactId>
         <version>2.8</version>
-        <configuration>
+        <!-- disabled currently -->
+        <!--configuration>
           <configLocation>file:///${project.parent.basedir}/li_checkstyle.xml</configLocation>
           <failOnError>true</failOnError>
           <violationSeverity>error</violationSeverity>
-        </configuration>
+        </configuration-->
       </plugin>
     </plugins>
   </reporting>

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/e9814387/src/site/markdown/ApiUsage.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/ApiUsage.md b/src/site/markdown/ApiUsage.md
new file mode 100644
index 0000000..b161233
--- /dev/null
+++ b/src/site/markdown/ApiUsage.md
@@ -0,0 +1,238 @@
+
+# Create an instance of Manager
+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);
+```
+                                                      
+#Setting up a cluster
+Initial setup of a cluster, involves creating appropriate znodes in the cluster. 
+```
+    //Create setuptool instance
+    ClusterSetupTool setupTool = new ClusterSetupTool(zkConnectString);
+    //Create cluster namespace in zookeeper
+    setupTool.addCluster(clusterName, true);
+    //Add six Participant instances, each instance must have a unique id. host:port is the standard convention
+    String instances[] = new String[6];
+    for (int i = 0; i < storageInstanceInfoArray.length; i++)
+    {
+      instance[i] = "localhost:" + (8900 + i);
+    }
+    setupTool.addInstancesToCluster(clusterName, instances);
+    //add the resource with 10 partitions to the cluster. Using MasterSlave state model. 
+    //See the section on how to configure a application specific state model
+    setupTool.addResourceToCluster(clusterName, "TestDB", 10, "MasterSlave");
+    //This will do the assignment of partitions to instances. Assignment algorithm is based on consistent hashing and RUSH. 
+    //See how to do custom partition assignment
+    setupTool.rebalanceResource(clusterName, "TestDB", 3);
+```
+
+## 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. Helix comes with commands to start the controller.  
+```
+cd helix
+mvn clean install -Dmaven.test.skip=true
+cd helix-core/target/helix-core-pkg/bin
+chmod +x *
+./run-helix-controller --zkSvr <ZookeeperServerAddress(Required)>  --cluster <Cluster name (Required)>
+```
+
+## 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");
+```
+##  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 --help
+    --activateCluster <clusterName grandCluster true/false>                                             Enable/disable a cluster in distributed controller mode
+    --addAlert <clusterName alertName>                                                                  Add an alert
+    --addCluster <clusterName>                                                                          Add a new cluster
+    --addIdealState <clusterName reourceName <filename>>                                                Add a State model to a cluster
+    --addNode <clusterName InstanceAddress(host:port)>                                                  Add a new Instance to a cluster
+    --addResource <clusterName resourceName partitionNum stateModelRef <-mode modeValue>>               Add a resource to a cluster
+    --addResourceProperty <clusterName resourceName propertyName propertyValue>                         Add a resource property
+    --addStat <clusterName statName>                                                                    Add a persistent stat
+    --addStateModelDef <clusterName <filename>>                                                         Add a State model to a cluster
+    --bucketSize <Size of a bucket for a resource>                                                      Specify size of a bucket, used with addResourceGroup command
+    --dropAlert <clusterName alertName>                                                                 Drop an alert
+    --dropCluster <clusterName>                                                                         Delete a cluster
+    --dropNode <clusterName InstanceAddress(host:port)>                                                 Drop an existing Instance from a cluster
+    --dropResource <clusterName resourceName>                                                           Drop an existing resource from a cluster
+    --dropStat <clusterName statName>                                                                   Drop a persistent stat
+    --enableCluster <clusterName true/false>                                                            pause/resume the controller of a cluster
+    --enableInstance <clusterName InstanceName true/false>                                              Enable/disable a Instance
+    --enablePartition <clusterName instanceName resourceName partitionName true/false>                  Enable/disable a partition
+    --expandCluster <clusterName>                                                                       Expand a cluster and all the resources
+    --expandResource <clusterName resourceName>                                                         Expand resource to additional nodes
+    --getConfig <ConfigScope(e.g. CLUSTER=cluster,RESOURCE=rc,...) KeySet(e.g. k1,k2,...)>              Get a config
+    --help                                                                                              Prints command-line options info
+    --key <Resource key prefix>                                                                         Specify resource key prefix, used with rebalance command
+    --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
+    --mode <IdealState mode>                                                                            Specify resource mode, used with addResourceGroup command
+    --rebalance <clusterName resourceName replicas>                                                     Rebalance a resource in a cluster
+    --removeResourceProperty <clusterName resourceName propertyName>                                    Remove a resource property
+    --resetPartition <clusterName instanceName resourceName partitionName>                              Reset a partition in error state
+    --setConfig <ConfigScope(e.g. CLUSTER=cluster,RESOURCE=rc,...) KeyValueMap(e.g. k1=v1,k2=v2,...)>   Set a config
+    --swapInstance <clusterName oldInstance newInstance>                                                Swap an old instance from a cluster with a new instance
+    --zkSvr <ZookeeperServerAddress(Required)>                                                          Provide zookeeper address
+```
+
+## Idealstate modes and configuration
+
+
+ * AUTO mode: ideal state is pre-generated using consistent hashing 
+ `setupTool.addResourceToCluster(clusterName, resourceName, partitionNumber, "MasterSlave")`
+ `setupTool.rebalanceStorageCluster(clusterName, resourceName, replicas)`
+ * AUTO_REBALANCE mode: ideal state is generated dynamically by cluster manager
+ `setupTool.addResourceToCluster(clusterName, resourceName, partitionNumber, "MasterSlave", "AUTO_REBALANCE)`
+ `setupTool.rebalanceStorageCluster(clusterName, resourceName, replicas)`
+ * CUSTOMIZED mode: ideal state is pre-generated from a JSON format file
+ `setupTool.addIdealState(clusterName, resourceName, idealStateJsonFile)`
+
+## Configuring state model
+
+```
+StateModelConfigGenerator generator = new StateModelConfigGenerator();
+ZnRecord stateModelConfig = generator.generateConfigForOnlineOffline();
+StateModelDefinition stateModelDef = new StateModelDefinition(stateModelConfig);
+ClusterSetup setupTool = new ClusterSetup(zkConnectString);
+setupTool.addStateModelDef(cluster,stateModelName,stateModelDef);
+```
+See StateModelConfigGenerator to get more info on creating custom state model.
+
+## Messaging Api usage
+
+See BootstrapProcess.java in examples package to see how Participants can exchange messages with each other.
+
+```
+      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 the process that is active at the time of sending the message. 
+      //This means if the recipient is restarted after message is sent, it will not be processed.
+      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);
+
+```
+For more details on MessagingService see ClusterMessagingService
+
+
+

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/e9814387/src/site/markdown/Architecture.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/Architecture.md b/src/site/markdown/Architecture.md
new file mode 100644
index 0000000..4eaa9c8
--- /dev/null
+++ b/src/site/markdown/Architecture.md
@@ -0,0 +1,225 @@
+Helix aims to provide the following abilities to a distributed system
+
+* Auto management of a cluster hosting partitioned, replicated resources
+* Soft and hard failure detection and handling.
+* Automatic load balancing via smart placement of resources on servers(nodes) based on server capacity and resource profile (size of partition, access patterns, etc)
+* Centralized config management and self discovery. Eliminates the need to modify config on each node.
+* Fault tolerance and optimized rebalancing during cluster expansion.
+* Manages entire operational lifecycle of a node. Addition, start, stop, enable/disable without downtime.
+* Monitor cluster health and provide alerts on SLA violation.
+* Service discovery mechanism to route requests.
+
+To build such a system, we need a mechanism to co-ordinate between different nodes/components in the system. This mechanism can be achieved with a software that reacts to any change in the cluster and comes up with a set of tasks needed to bring the cluster to a stable state. The set of tasks will be assigned to one or more nodes in the cluster. Helix serves this purpose of managing the various components in the cluster.
+
+![Helix Design](images/system.png)
+
+Distributed System Components
+
+In general any distributed system cluster will have the following
+
+* Set of nodes also referred to as an instance.
+* Set of resources which can be a database, lucene index or a task.
+* Each resource is also partitioned into one or more Partitions. 
+* Each partition may have one or more copies called replicas.
+* Each replica can have a state associated with it. For example Master, Slave, Leader, Standby, Online, Offline etc
+
+Roles
+-----
+
+Not all nodes in a distributed system will perform similar functionality. For e.g, a few nodes might be serving requests, few nodes might be sending the request and some nodes might be controlling the nodes in the cluster. Based on functionality we have grouped them into
+
+We have divided Helix in 3 logical components based on their responsibility 
+
+1. PARTICIPANT: The nodes that actually host the distributed resources.
+2. SPECTATOR: The nodes that simply observe the PARTICIPANT State and route the request accordingly. Routers, for example, need to know the Instance on which a partition is hosted and its state in order to route the request to the appropriate end point.
+3. CONTROLLER: The controller observes and controls the PARTICIPANT nodes. It is responsible for coordinating all transitions in the cluster and ensuring that state constraints are satisfied and cluster stability is maintained. 
+
+
+These are simply logical components and can be deployed as per the system requirements. For example.
+
+1. Controller can be deployed as a separate service. 
+2. Controller can be deployed along with Participant but only one Controller will be active at any given time.
+
+Both have pros and cons, which will be discussed later and one can chose the mode of deployment as per system needs.
+
+
+## Cluster state/metadata store
+
+We need a distributed store to maintain the state of the cluster and a notification system to notify if there is any change in the cluster state. Helix uses Zookeeper to achieve this functionality.
+
+Zookeeper provides
+
+* A way to represent PERSISTENT state which basically remains until its deleted.
+* A way to represent TRANSIENT/EPHEMERAL state which vanishes when the process that created the STATE dies.
+* Notification mechanism when there is a change in PERSISTENT/EPHEMERAL STATE
+
+The name space provided by ZooKeeper is much like that of a standard file system. A name is a sequence of path elements separated by a slash (/). Every node[ZNODE] in ZooKeeper's name space is identified by a path.
+
+More info on Zookeeper can be found here http://zookeeper.apache.org
+
+## Statemachine and constraints
+
+Even though the concept of Resource, Partition, Replicas is common to most distributed systems, one thing that differentiates one distributed system from another is the way each partition is assigned a state and the constraints on each state.
+
+For example, 
+
+1. If a system is serving READ ONLY data then all partitions replicas are equal and they can either be ONLINE or OFFLINE.
+2. If a system takes BOTH READ and WRITES but ensure that WRITES go through only one partition then the states will be MASTER,SLAVE and OFFLINE. Writes go through the MASTER and is replicated to the SLAVES. Optionally, READS can go through SLAVES  
+
+Apart from defining State for each partition, the transition path to each STATE can be application specific. For example, in order to become master it might be a requirement to first become a SLAVE. This ensures that if the SLAVE does not have the data as part of OFFLINE-SLAVE transition it can bootstrap data from other nodes in the system.
+
+Helix provides a way to configure an application specific state machine along with constraints on each state. Along with constraints on State, Helix also provides a way to specify constraints on transitions.(More on this later)
+
+
+<pre><code>
+          OFFLINE  | SLAVE  |  MASTER  
+         _____________________________
+        |          |        |         |
+OFFLINE |   N/A    | SLAVE  | SLAVE   |
+        |__________|________|_________|
+        |          |        |         |
+SLAVE   |  OFFLINE |   N/A  | MASTER  |
+        |__________|________|_________|
+        |          |        |         |
+MASTER  | SLAVE    | SLAVE  |   N/A   |
+        |__________|________|_________|
+
+</code></pre>
+
+![Helix Design](images/statemachine.png)
+
+## Concepts
+
+The following terminologies are used in Helix to model a state machine.
+
+* IDEALSTATE:The state in which we need the cluster to be in if all nodes are up and running. In other words, all state constraints are satisfied.
+* CURRENSTATE:Represents the current state of each node in the cluster 
+* EXTERNALVIEW:Represents the combined view of CURRENTSTATE of all nodes.  
+
+Goal of Helix is always to make the CURRENTSTATE of the system same as the IDEALSTATE. Some scenarios where this may not be true are
+
+* When all nodes are down
+* When one or more nodes fail
+* New nodes are added and the partitions need to be reassigned.
+
+### IDEALSTATE
+
+Helix lets the application define the IdealState on a resource basis which basically consists of
+
+* List of partitions. Example 64
+* Number of replicas for each partition. Example 3
+* Node and State for each replica.
+
+Example
+
+* Partition-1, replica-1, Master, Node-1
+* Partition-1, replica-2, Slave, Node-2
+* Partition-1, replica-3, Slave, Node-3
+* .....
+* .....
+* Partition-p, replica-3, Slave, Node-n
+
+Helix comes with various algorithms to automatically assign the partitions to nodes. The default algorithm minimizes the number of shuffles that happen when new nodes are added to the system
+
+### CURRENTSTATE
+
+Every Instance in the cluster hosts one or more partitions of a resource. Each of the partitions has a State associated with it.
+
+Example Node-1
+
+* Partition-1, Master
+* Partition-2, Slave
+* ....
+* ....
+* Partition-p, Slave
+
+### EXTERNALVIEW
+
+External clients needs to know the state of each partition in the cluster and the Node hosting that partition. Helix provides one view of the system to SPECTATORS as EXTERNAL VIEW. EXTERNAL VIEW is simply an aggregate of all CURRENTSTATE
+
+* Partition-1, replica-1, Master, Node-1
+* Partition-1, replica-2, Slave, Node-2
+* Partition-1, replica-3, Slave, Node-3
+* .....
+* .....
+* Partition-p, replica-3, Slave, Node-n
+
+## Process Workflow
+
+Mode of operation in a cluster
+
+A node process can be one of the following
+
+* PARTICIPANT: The process registers itself in the cluster and acts on the messages received in its queue and updates the current state.  Example:Storage Node
+* SPECTATOR: The process is simply interested in the changes in the externalView. Router is a spectator of Storage cluster.
+* CONTROLLER:This process actively controls the cluster by reacting to changes in Cluster State and sending messages to PARTICIPANTS
+
+
+### Participant Node Process
+
+
+* When Node starts up, it registers itself under LIVEINSTANCES
+* After registering, it waits for new Messages in the message queue
+* When it receives a message, it will perform the required task as indicated in the message.
+* After the task is completed, depending on the task outcome it updates the CURRENTSTATE.
+
+### Controller Process
+
+* Watches IDEALSTATE
+* Node goes down/comes up or Node is added/removed. Watches LIVEINSTANCES and CURRENTSTATE of each Node in the cluster
+* Triggers appropriate state transition by sending message to PARTICIPANT
+
+### Spectator Process
+
+* When the process starts it asks cluster manager agent to be notified of changes in ExternalView
+* When ever it receives a notification it reads the externalView and performs required duties. For Router, it updates its routing table.
+
+h4. Interaction between controller, participant and spectator
+
+The following picture shows how controllers, participants and spectators interact with each other.
+
+![Helix Architecture](images/helix-architecture.png)
+
+## Core algorithm
+
+
+* Controller gets the Ideal State and the Current State of active storage nodes from ZK
+* Compute the delta between Ideal State and Current State for each partition across all participant nodes
+* For each partition compute tasks based on State Machine Table. Its possible to configure priority on the state Transition. For example in case of Master Slave:
+    * Attempt Mastership transfer if possible without violating constraint.
+    * Partition Addition
+    * Drop Partition 
+* Add the tasks in parallel if possible to respective queue for each storage node keeping in mind
+* The tasks added are mutually independent.
+* If a task is dependent on another task being completed do not add that task.
+* After any task is completed by Participant, Controllers gets notified of the change and State Transition algorithm is re-run until the current state is same as Ideal State.
+
+## Helix znode layout
+Helix organizes znodes under clusterName in multiple levels. 
+The top level (under clusterName) znodes are all Helix defined and in upper case
+* PROPERTYSTORE: application property store
+* STATEMODELDEFES: state model definitions
+* INSTANCES: instance runtime information including current state and messages
+* CONFIGS: configurations
+* IDEALSTATES: ideal states
+* EXTERNALVIEW: external views
+* LIVEINSTANCES: live instances
+* CONTROLLER: cluster controller runtime information
+
+Under INSTANCES, there are runtime znodes for each instance. An instance organizes znodes as follows:
+* CURRENTSTATES
+ * sessionId
+  * resourceName
+* ERRORS
+* STATUSUPDATES
+* MESSAGES
+* HEALTHREPORT
+
+Under CONFIGS, there are different scopes of configurations:
+* RESOURCE: contains resource scope configurations
+* CLUSTER: contains cluster scope configurations
+* PARTICIPANT: contains participant scope configurations
+
+The following image shows an example of Helix znodes layout for a cluster named "test-cluster":
+
+![Helix znode layout](images/helix-znode-layout.png)

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/e9814387/src/site/markdown/Features.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/Features.md b/src/site/markdown/Features.md
new file mode 100644
index 0000000..1f8f8be
--- /dev/null
+++ b/src/site/markdown/Features.md
@@ -0,0 +1,161 @@
+As we started using Helix in production we found various things that were needed as part of most distributed data systems.
+
+These features have been implemented in a way that other systems can benefit.
+
+Partition Placement
+-------------------
+The placement of partitions in a DDS is very critical for 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 guarantee this.
+Helix by default comes with a variant of consistent hashing based of 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 and 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 two criteria.
+
+In simple terms, partition assignment can be defined as the mapping of Replica,State to a Node in the cluster. For example, lets say the system as 2 partitions(P1,P2) and each partition has 2 replicas and there are 2 nodes(N1,N2) in the system and two possible states Master and Slave 
+
+The partition assignment table can look like 
+
+    P1 -> {N1:M, N2:S}
+    P2 -> {N1:S, N2:M}
+This means Partition P1 must be a Master at N1 and Slave at N2 and vice versa for P2
+
+Helix provides multiple ways to control the partition placement. See Execution modes section for more info on this.
+
+IdealState execution modes 
+--------------------------
+Idealstate is defined as the state of the DDS when all nodes are up and running and healthy. 
+Helix uses this as the target state of the system and computes the appropriate transitions needed in the system to bring it to a stable state. 
+
+Helix supports 3 different execution modes which allows application to explicitly control the placement and state of the replica.
+
+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. 
+A typical example is evenly distributing a group of tasks among the currently alive 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. Similarly, if a node is added, Helix re-allocates 3 tasks from each of the 4 nodes to the 5th node. 
+
+AUTO
+When the idealstate mode is set to AUTO, Helix only controls STATE of the replicas where as the location of the partition is controlled by application. 
+For example the application can say P1->{N1,N2,N3} which means P1 should only exist N1,N2,N3. In this mode when N1 fails, unlike in AUTO-REBALANCE mode the partition is not moved from N1 to others nodes in the cluster. 
+But Helix might decide to change the state of P1 in N2 and N3 based on the system constraints. For example, if a system constraint specified that there should be 1 Master and if the Master failed, then N2 will be made the master.
+
+CUSTOM
+Helix offers a third mode called CUSTOM, in which application can completely control the placement and state of each replica. Applications will have to implement an interface that Helix will invoke when the cluster state changes. 
+Within this callback, the application can recompute the partition assignment mapping. Helix will then issue transitions to get the system to the final state. Note that Helix will ensure that system constraints are not violated at any time.
+For example, the current state of the system might be P1 -> {N1:M,N2:S} and the application changes the ideal state to P2 -> {N1:S,N2:M}. Helix will not blindly issue M-S to N1 and S-M to N2 in parallel since it might result in a transient state where both N1 and N2 are masters.
+Helix will issue S-M to N2 only when N1 has changed its state to S.
+ 
+
+State Machine Configuration
+---------------------------
+Helix comes with 3 default state models that are most commonly used. Its possible to have multiple state models in a cluster. 
+Every resource that is added should have a reference to the state model. 
+
+* MASTER-SLAVE: Has 3 states OFFLINE,SLAVE,MASTER. Max masters is 1. Slaves will be based on the replication factor. Replication factor can be specified while adding the resource
+* ONLINE-OFFLINE: Has 2 states OFFLINE and ONLINE. Very simple state model and most applications start off with this state model.
+* LEADER-STANDBY:1 Leader and many stand bys. In general the standby's are idle.
+
+Apart from providing the state machine configuration, one can specify the constraints of states and transitions.
+
+For example one can say
+Master:1. Max number of replicas in Master state at any time is 1.
+OFFLINE-SLAVE:5 Max number of Offline-Slave transitions that can happen concurrently in the system
+
+STATE PRIORITY
+Helix uses 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. 
+One can control this by overriding the priority order.
+ 
+Config management
+-----------------
+Helix allows applications to store application specific properties. The configuration can have different scopes.
+
+* Cluster
+* Node specific
+* Resource specific
+* Partition specific
+
+Helix also provides notifications when any configs are changed. This allows applications to support dynamic configuration changes.
+
+See HelixManager.getConfigAccessor for more info
+
+Intra cluster messaging api
+---------------------------
+This is an interesting feature which is quite useful in practice. Often times, nodes in DDS requires a mechanism to interact with each other. One such requirement is a process of bootstrapping a replica.
+
+Consider a search system use case where the index replica starts up and it does not have an index. One of the commonly used solutions is to get the index from a common location or to copy the index from another replica.
+Helix provides a messaging api, that can be used to talk to other nodes in the system. The value added that Helix provides here is, message recipient can be specified in terms of resource, 
+partition, state and 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 P1. 
+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 like data backups etc. 
+System Admins can also perform adhoc tasks like on demand backup or execute a system command(like rm -rf ;-)) across all nodes.
+
+See HelixManager.getMessagingService for more info.
+
+
+Application specific property storage
+-------------------------------------
+There are several usecases where applications needs support for distributed data structures. Helix uses Zookeeper to store the application data and hence provides notifications when the data changes. 
+One value add Helix provides is the ability to specify cache the data and also write through cache. This is more efficient than reading from ZK every time.
+
+See HelixManager.getHelixPropertyStore
+
+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 around which is quite expensive.
+Helix allows applications to set threshold on transitions. The threshold can be set at the multiple scopes.
+
+* MessageType e.g STATE_TRANSITION
+* TransitionType e.g SLAVE-MASTER
+* Resource e.g database
+* Node i.e per node max transitions in parallel.
+
+See HelixManager.getHelixAdmin.addMessageConstraint() 
+
+Health monitoring and alerting
+------------------------------
+This in currently in development mode, not yet productionized.
+
+Helix provides ability for each node in the system to report health metrics on a periodic basis. 
+Helix supports multiple ways to aggregate these metrics like simple SUM, AVG, EXPONENTIAL DECAY, WINDOW. Helix will only persist the aggregated value.
+Applications can define threshold on the aggregate values according to the SLA's and when the SLA is violated Helix will fire an alert. 
+Currently Helix only fires an alert but eventually we plan to use this metrics to either mark the node dead or load balance the partitions. 
+This feature will be valuable in for distributed systems that support multi-tenancy and have huge variation in work load patterns. Another place this can be used is to detect skewed partitions and rebalance the cluster.
+
+This feature is not yet stable and do not recommend to be used in production.
+
+
+
+Controller deployment modes
+---------------------------
+Read Architecture wiki for more details on the Role of a controller. In simple words, it basically controls the participants in the cluster by issuing transitions.
+
+Helix provides multiple options to deploy the controller.
+
+STANDALONE
+
+Controller can be started as a separate process to manage a cluster. This is the recommended approach. How ever 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 resume 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 participant. 
+
+CONTROLLER AS A SERVICE
+
+One of the cool feature we added in helix was 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 only 3 controllers. Each controller can manage X/3 clusters. 
+If any controller fails the remaining two will manage X/2 clusters. At LinkedIn, we always deploy controllers in this mode. 
+
+
+
+
+
+
+
+ 

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/e9814387/src/site/markdown/Home.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/Home.md b/src/site/markdown/Home.md
new file mode 100644
index 0000000..0381cdb
--- /dev/null
+++ b/src/site/markdown/Home.md
@@ -0,0 +1,135 @@
+Pages
+---------------
+* [Home][]
+* [SampleApp][]
+* [Quickstart][]
+* [Architecture][]
+* [Features][]
+* [ApiUsage][]
+* [Javadocs][]
+* [UseCases][]
+
+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
+-------------
+Helix uses terms that are commonly used to describe distributed data system concepts. 
+
+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. 
+
+To summarize, a resource (database, index or any task) in general is partitioned, replicated and distributed among the Instance/nodes in the cluster and each partition has a state associated with it. 
+
+Helix manages the state of a resource by supporting a pluggable distributed state machine. One can define the state machine table along with the constraints for each state. 
+
+Here are some common state models used
+
+1. Master, Slave
+2. Online, Offline
+3. Leader, Standby.
+
+For example in the case of a MasterSlave state model one can specify the state machine as follows. The table says given a start state and an end state what should be the next state. 
+For example, if the current state is Offline and the target state is Master, the table says that the next state is Slave.  So in this case, Helix issues an Offline-Slave transition
+
+<pre><code>
+          OFFLINE  | SLAVE  |  MASTER  
+         _____________________________
+        |          |        |         |
+OFFLINE |   N/A    | SLAVE  | SLAVE   |
+        |__________|________|_________|
+        |          |        |         |
+SLAVE   |  OFFLINE |   N/A  | MASTER  |
+        |__________|________|_________|
+        |          |        |         |
+MASTER  | SLAVE    | SLAVE  |   N/A   |
+        |__________|________|_________|
+
+</code></pre>
+
+Helix also supports the ability to provide constraints on each state. For example in a MasterSlave state model with a replication factor of 3 one can say 
+
+    MASTER:1 
+    SLAVE:2
+
+Helix will automatically maintain 1 Master and 2 Slaves by initiating appropriate state transitions on each instance in the cluster. 
+
+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 
+
+* Node start up
+* Node soft and hard failures 
+* Addition of resources
+* Addition of nodes
+
+In simple words, Helix is a distributed state machine with support for constraints on each state.
+
+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 
+
+Once the state machine and constraints are configured through Helix, application will have the provide implementation to handle the transitions appropriately.  
+
+<pre><code>
+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());
+  }
+}
+</code></pre>
+
+Once the state machine is configured, the framework allows one to 
+
+* Dynamically add nodes to the cluster
+* Automatically modify the topology(rebalance partitions) of the cluster  
+* Dynamically add resources to the cluster
+* Enable/disable partition/instances for software upgrade without impacting availability.
+
+Helix uses Zookeeper for maintaining the cluster state and change notifications.
+
+WHY HELIX
+-------------
+Helix approach of using a distributed state machine with constraints on state and transitions has benefited us in multiple ways.
+
+* Abstract cluster management aspects from the core functionality of DDS.
+* Each node in DDS is not aware of the global state since they simply have to follow . This proved quite useful since we could deploy the same system in different topologies.
+* 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.
+
+At LinkedIn, we have been able to use this to manage 3 different distributed systems that look very different on paper.  
+
+----------------
+[Home]:wiki/Home "Introduction"
+[SampleApp]:wiki/Sample_App "Sample Application"
+[Quickstart]: wiki/Quickstart "Quick Start Guide"
+[Architecture]: wiki/Architecture "Helix Architecture"
+[Features]: wiki/Features "Features"
+[ApiUsage]: wiki/ApiUsage "Api usage"
+[Javadocs]: http://linkedin.github.com/helix/apidocs/
+[UseCases]: wiki/UseCases "Some usecases at LinkedIn"
+
+
+
+
+   
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/e9814387/src/site/markdown/Quickstart.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/Quickstart.md b/src/site/markdown/Quickstart.md
new file mode 100644
index 0000000..61b99eb
--- /dev/null
+++ b/src/site/markdown/Quickstart.md
@@ -0,0 +1,130 @@
+Install/Start zookeeper
+-----------------------
+
+Zookeeper can be started in standalone mode or replicated mode.
+
+More info is available at http://zookeeper.apache.org/doc/r3.3.3/zookeeperStarted.html
+and http://zookeeper.apache.org/doc/trunk/zookeeperAdmin.html#sc_zkMulitServerSetup
+
+In this example, we will start zookeeper in local mode.
+
+BUILD Helix
+-----------
+Jump to Download section to skip this test.
+
+    git clone git@github.com:linkedin/helix.git
+    cd helix-core
+    mvn install package appassembler:assemble -Dmaven.test.skip=true 
+    cd target/helix-core-pkg/bin //This folder contains all the scripts used in following sections
+    chmod \+x *
+
+Download Helix
+--------------
+Instead of building the package from the code, you can download the 0.5.28 release package from [here](http://linkedin.github.com/helix/download/release-0.5.28/helix-core-pkg-0.5.28.tar.gz) 
+
+Cluster setup
+-------------
+cluster-admin tool is used for cluster administration tasks. Apart from a command line interface Helix supports a REST interface as well.
+
+zookeeper_address is of the format host:port e.g localhost:2199 for standalone or host1:port,host2:port for multi node.
+
+In the following section we will see how one can set up a mock mycluster cluster with 
+
+* 3 node instances running on localhost at 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
+-----
+If you build the code
+cd helix/helix-core/target/helix-core-pkg
+If you download the release package, extract it.
+cd helix-core-pkg
+     
+#### start zookeeper locally at port 2199
+
+    ./start-standalone-zookeeper 2199 &
+
+#### create the cluster mycluster
+    ## helix-admin --zkSvr localhost:2199 --addCluster <clustername> 
+    ./helix-admin --zkSvr localhost:2199 --addCluster mycluster 
+
+#### Create a database with 6 partitions using MasterSlave state model. This ensures there will be one master for each partition 
+    ### helix-admin --zkSvr localhost:2199  --addResource <clustername> <resourceName> <numPartitions> <StateModelName>
+    ./helix-admin --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 --zkSvr <zk_address>  --addNode <clustername> <host:port>
+    ./helix-admin --zkSvr localhost:2199  --addNode mycluster localhost:12913
+    ./helix-admin --zkSvr localhost:2199  --addNode mycluster localhost:12914
+    ./helix-admin --zkSvr localhost:2199  --addNode mycluster localhost:12915
+
+#### 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 --rebalance <clustername> <resourceName> <replication factor>
+    ./helix-admin --zkSvr localhost:2199 --rebalance mycluster myDB 3
+
+#### 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 &
+
+#### Start Example Participant, This is a dummy participant where the transitions are no-ops.    
+    ./start-helix-participant --help
+    # start process 1 process corresponding to every host port added during cluster setup
+    ./start-helix-participant --zkSvr localhost:2199 --cluster mycluster --host localhost --port 12913 --stateModelType MasterSlave 2>&1 > /tmp/participant_12913.log 
+    ./start-helix-participant --zkSvr localhost:2199 --cluster mycluster --host localhost --port 12914 --stateModelType MasterSlave 2>&1 > /tmp/participant_12914.log
+    ./start-helix-participant --zkSvr localhost:2199 --cluster mycluster --host localhost --port 12915 --stateModelType MasterSlave 2>&1 > /tmp/participant_12915.log
+
+
+Inspect Cluster Data
+--------------------
+
+We can see the cluster state on zookeeper and know the partition assignment and current state of each partition.
+
+Command line tool
+#### List existing clusters
+    ./helix-admin --zkSvr localhost:2199 --listClusters        
+                                       
+####  Query info of a cluster
+
+    #helix-admin --zkSvr localhost:2199 --listClusterInfo <clusterName> 
+    ./helix-admin --zkSvr localhost:2199 --listClusterInfo mycluster
+
+####  List Instances in a cluster
+    ## helix-admin --zkSvr localhost:2199 --listInstances <clusterName>
+     ./helix-admin --zkSvr localhost:2199 --listInstances mycluster
+    
+#### Query info of a Instance in a cluster
+    #./helix-admin --zkSvr localhost:2199 --listInstanceInfo <clusterName InstanceName>    
+     ./helix-admin --zkSvr localhost:2199 --listInstanceInfo mycluster localhost_12913
+     ./helix-admin --zkSvr localhost:2199 --listInstanceInfo mycluster localhost_12914
+     ./helix-admin --zkSvr localhost:2199 --listInstanceInfo mycluster localhost_12915
+
+#### List resourceGroups hosted in a cluster
+    ## helix-admin --zkSvr localhost:2199 --listResources <clusterName>
+    ./helix-admin --zkSvr localhost:2199 --listResources mycluster
+    
+#### Query info of a resource
+    ## helix-admin --zkSvr localhost:2199 --listResourceInfo <clusterName resourceName>
+    ./helix-admin --zkSvr localhost:2199 --listResourceInfo mycluster myDB
+
+#### Query info about a partition   
+    ## helix-admin --zkSvr localhost:2199 --listResourceInfo <clusterName partition> 
+    ./helix-admin --zkSvr localhost:2199 --listResourceInfo mycluster myDB_0
+   
+#### List all state models in the cluster
+    # helix-admin --zkSvr localhost:2199 --listStateModels <clusterName>
+    ./helix-admin --zkSvr localhost:2199 --listStateModels mycluster
+    
+#### Query info about a state model in a cluster
+    ## helix-admin --zkSvr localhost:2199 --listStateModel <clusterName stateModelName>
+    ./helix-admin --zkSvr localhost:2199 --listStateModel mycluster MasterSlave
+
+#### ZOOINSPECTOR
+
+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
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/e9814387/src/site/markdown/Sample_App.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/Sample_App.md b/src/site/markdown/Sample_App.md
new file mode 100644
index 0000000..873a898
--- /dev/null
+++ b/src/site/markdown/Sample_App.md
@@ -0,0 +1,206 @@
+RabbitMQ Consumer Group
+=======================
+
+[RabbitMQ](http://www.rabbitmq.com/) is a well known Open source software the provides robust messaging for applications.
+
+One of the commonly implemented recipes using this software is a work queue.  http://www.rabbitmq.com/tutorials/tutorial-four-java.html describes the use case where
+
+* A producer sends a message with a routing key. 
+* The message goes to the queues whose binding key exactly matches the routing key of the message.	
+* There are multiple consumers and each consumer is interested in processing only a subset of the messages by binding to the interested keys
+
+The example provided [here](http://www.rabbitmq.com/tutorials/tutorial-four-java.html) describes how multiple consumers can be started to process all the tasks.
+
+While this works, in production systems one needs the following 
+* Ability to handle failures: when a consumers fails another consumer must be started or the other consumers must start processing these messages that should have been processed by the failed consumer.
+* When the existing consumers cannot keep up with the task generation rate, new consumers will be added. The tasks must be redistributed among all the consumers. 
+
+In this sample app, we explain how these set of consumers can be grouped together and handle consumer failures and expansion automatically.
+
+Mapping this usecase to Helix is pretty easy as the binding key/routing key is equivalent to a partition. 
+
+Lets take a real example. Lets say a topic has 6 partitions, and we have 2 consumers to process all the queues. 
+What we want is all 6 queues to be evenly divided among 2 consumers. 
+Eventually when the system scales, we add more consumers to keep up. This will make each consumer process tasks from 2 queues.
+Now let's say that a consumer fails and that the number of active consumers is now reduced to 2. This means each consumer must process 3 queues.
+
+We showcase how such a dynamic App can be developed using Helix. 
+
+Try it
+======
+Before getting into the details on how to develop such an App using Helix, you can try the following steps to get a feel of it.
+
+```
+git clone git@github.com:linkedin/helix.git
+cd helix
+./build
+export HELIX_PKG_ROOT=`pwd`/helix-core/target/helix-core-pkg
+```
+OR
+Download the latest 0.5.28 release tar ball from [here](http://linkedin.github.com/helix/download/release-0.5.28//helix-core-pkg-0.5.28.tar.gz)
+```
+tar -xzvf helix-core-pkg-0.5.28.tar.gz
+export HELIX_PKG_ROOT=`pwd`/helix-core-pkg
+```
+
+Download the rabbitmq-consumer-group recipe from [here](http://linkedin.github.com/helix/download/release-0.5.28/rabbitmq-consumer-group-0.5.28.tar.gz)
+```
+tar -xzvf rabbitmq-consumer-group-0.5.28.tar.gz
+export HELIX_RABBITMQ_ROOT=`pwd`/rabbitmq-consumer-group/
+```
+```
+chmod +x $HELIX_PKG_ROOT/bin/*
+chmod +x $HELIX_RABBITMQ_ROOT/bin/*
+```
+
+Install Rabbit MQ
+----------------
+
+Setting up RabbitMQ on a local box is straightforward. You can find the instructions here
+http://www.rabbitmq.com/download.html
+
+Start ZK
+--------
+Start zookeeper at port 2199
+```
+$HELIX_PKG_ROOT/bin/start-standalone-zookeeper 2199
+```
+
+Setup the consumer group cluster
+--------------------------------
+This will setup the cluster by creating a "rabbitmq-consumer-group" cluster and adds a "topic" resource with "6" queues. 
+```
+$HELIX_RABBITMQ_ROOT/bin/setup-cluster.sh localhost:2199 
+```
+
+Add consumers
+-------------
+Start 2 consumers in 2 different terminals. Each consumer is given a unique id.
+```
+//start-consumer.sh zookeeperAddress (e.g. localhost:2181) consumerId , rabbitmqServer (e.g. localhost)
+$HELIX_RABBITMQ_ROOT/bin/start-consumer.sh localhost:2199 0 localhost 
+$HELIX_RABBITMQ_ROOT/bin/start-consumer.sh localhost:2199 1 localhost 
+
+```
+
+Start HelixController
+--------------------
+Now start a Helix controller that starts managing the "rabbitmq-consumer-group" cluster.
+```
+$HELIX_RABBITMQ_ROOT/bin/start-cluster-manager.sh localhost:2199
+```
+Send messages to the Topic
+--------------------------
+
+Start sending messages to the topic. This script randomly selects a routing key (1-6) and sends the message to topic. 
+Based on the key, messages gets routed to the appropriate queue.
+
+```
+$HELIX_RABBITMQ_ROOT/bin/send-message.sh localhost 20
+```
+After running this, you should see all 20 messages being processed by 2 consumers. 
+
+Add another consumer
+--------------------
+Once a new consumer is started, helix detects it. In order to balance the load between 3 consumers, it deallocates 1 partition from the existing consumers and allocates it to the new consumer. We see that
+each consumer is now processing only 2 queues.
+Helix makes sure that old nodes are asked to stop consuming before the new consumer is asked to start consuming for a given partition. But the transitions for each partition can happen in parallel.
+```
+$HELIX_RABBITMQ_ROOT/bin/start-consumer.sh localhost:2199 2 localhost
+```
+
+Send messages again to the topic.
+```
+$HELIX_RABBITMQ_ROOT/bin/send-message.sh localhost 100
+```
+You should see that messages are now received by all 3 consumers.
+
+Stop a consumer
+---------------
+In any terminal press CTRL^C and notice that Helix detects the consumer failure and distributes the 2 partitions that were processed by failed consumer to the remaining 2 active consumers.
+
+
+How does it work
+================
+
+Find the entire code [here](https://github.com/linkedin/helix/tree/master/recipes/rabbitmq-consumer-group/src/main/java/com/linkedin/helix/recipes/rabbitmq). 
+ 
+Cluster setup
+-------------
+This step creates znode on zookeeper for the cluster and adds the state model. We use online offline state model since there is no need for other states. The consumer is either processing a queue or it is not.
+
+It creates a resource called "rabbitmq-consumer-group" with 6 partitions. The execution mode is set to AUTO_REBALANCE. This means that the Helix controls the assignment of partition to consumers and automatically distributes the partitions evenly among the active consumers. When a consumer is added or removed, it ensures that a minimum number of partitions are shuffled.
+
+```
+      zkclient = new ZkClient(zkAddr, ZkClient.DEFAULT_SESSION_TIMEOUT,
+          ZkClient.DEFAULT_CONNECTION_TIMEOUT, new ZNRecordSerializer());
+      ZKHelixAdmin admin = new ZKHelixAdmin(zkclient);
+      
+      // add cluster
+      admin.addCluster(clusterName, true);
+
+      // add state model definition
+      StateModelConfigGenerator generator = new StateModelConfigGenerator();
+      admin.addStateModelDef(clusterName, "OnlineOffline",
+          new StateModelDefinition(generator.generateConfigForOnlineOffline()));
+
+      // add resource "topic" which has 6 partitions
+      String resourceName = "rabbitmq-consumer-group";
+      admin.addResource(clusterName, resourceName, 6, "OnlineOffline", "AUTO_REBALANCE");
+```
+Starting the consumers
+----------------------
+The only thing consumers need to know is the zkaddress, cluster name and consumer id. It does not need to know anything else.
+
+```
+   _manager =
+          HelixManagerFactory.getZKHelixManager(_clusterName,
+                                                _consumerId,
+                                                InstanceType.PARTICIPANT,
+                                                _zkAddr);
+
+      StateMachineEngine stateMach = _manager.getStateMachineEngine();
+      ConsumerStateModelFactory modelFactory =
+          new ConsumerStateModelFactory(_consumerId, _mqServer);
+      stateMach.registerStateModelFactory("OnlineOffline", modelFactory);
+
+      _manager.connect();
+
+```
+Once the consumer has registered the statemodel and the controller is started, the consumer starts getting callbacks (onBecomeOnlineFromOffline) for the partition it needs to host. All it needs to do as part of the callback is to start consuming messages from the appropriate queue. Similarly, when the controller deallocates a partitions from a consumer, it fires onBecomeOfflineFromOnline for the same partition. 
+As a part of this transition, the consumer will stop consuming from a that queue.
+
+```
+ @Transition(to = "ONLINE", from = "OFFLINE")
+  public void onBecomeOnlineFromOffline(Message message, NotificationContext context)
+  {
+    LOG.debug(_consumerId + " becomes ONLINE from OFFLINE for " + _partition);
+
+    if (_thread == null)
+    {
+      LOG.debug("Starting ConsumerThread for " + _partition + "...");
+      _thread = new ConsumerThread(_partition, _mqServer, _consumerId);
+      _thread.start();
+      LOG.debug("Starting ConsumerThread for " + _partition + " done");
+
+    }
+  }
+
+  @Transition(to = "OFFLINE", from = "ONLINE")
+  public void onBecomeOfflineFromOnline(Message message, NotificationContext context)
+      throws InterruptedException
+  {
+    LOG.debug(_consumerId + " becomes OFFLINE from ONLINE for " + _partition);
+
+    if (_thread != null)
+    {
+      LOG.debug("Stopping " + _consumerId + " for " + _partition + "...");
+
+      _thread.interrupt();
+      _thread.join(2000);
+      _thread = null;
+      LOG.debug("Stopping " +  _consumerId + " for " + _partition + " done");
+
+    }
+  }
+```
\ No newline at end of file

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

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/e9814387/src/site/markdown/index.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/index.md b/src/site/markdown/index.md
new file mode 100644
index 0000000..0381cdb
--- /dev/null
+++ b/src/site/markdown/index.md
@@ -0,0 +1,135 @@
+Pages
+---------------
+* [Home][]
+* [SampleApp][]
+* [Quickstart][]
+* [Architecture][]
+* [Features][]
+* [ApiUsage][]
+* [Javadocs][]
+* [UseCases][]
+
+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
+-------------
+Helix uses terms that are commonly used to describe distributed data system concepts. 
+
+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. 
+
+To summarize, a resource (database, index or any task) in general is partitioned, replicated and distributed among the Instance/nodes in the cluster and each partition has a state associated with it. 
+
+Helix manages the state of a resource by supporting a pluggable distributed state machine. One can define the state machine table along with the constraints for each state. 
+
+Here are some common state models used
+
+1. Master, Slave
+2. Online, Offline
+3. Leader, Standby.
+
+For example in the case of a MasterSlave state model one can specify the state machine as follows. The table says given a start state and an end state what should be the next state. 
+For example, if the current state is Offline and the target state is Master, the table says that the next state is Slave.  So in this case, Helix issues an Offline-Slave transition
+
+<pre><code>
+          OFFLINE  | SLAVE  |  MASTER  
+         _____________________________
+        |          |        |         |
+OFFLINE |   N/A    | SLAVE  | SLAVE   |
+        |__________|________|_________|
+        |          |        |         |
+SLAVE   |  OFFLINE |   N/A  | MASTER  |
+        |__________|________|_________|
+        |          |        |         |
+MASTER  | SLAVE    | SLAVE  |   N/A   |
+        |__________|________|_________|
+
+</code></pre>
+
+Helix also supports the ability to provide constraints on each state. For example in a MasterSlave state model with a replication factor of 3 one can say 
+
+    MASTER:1 
+    SLAVE:2
+
+Helix will automatically maintain 1 Master and 2 Slaves by initiating appropriate state transitions on each instance in the cluster. 
+
+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 
+
+* Node start up
+* Node soft and hard failures 
+* Addition of resources
+* Addition of nodes
+
+In simple words, Helix is a distributed state machine with support for constraints on each state.
+
+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 
+
+Once the state machine and constraints are configured through Helix, application will have the provide implementation to handle the transitions appropriately.  
+
+<pre><code>
+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());
+  }
+}
+</code></pre>
+
+Once the state machine is configured, the framework allows one to 
+
+* Dynamically add nodes to the cluster
+* Automatically modify the topology(rebalance partitions) of the cluster  
+* Dynamically add resources to the cluster
+* Enable/disable partition/instances for software upgrade without impacting availability.
+
+Helix uses Zookeeper for maintaining the cluster state and change notifications.
+
+WHY HELIX
+-------------
+Helix approach of using a distributed state machine with constraints on state and transitions has benefited us in multiple ways.
+
+* Abstract cluster management aspects from the core functionality of DDS.
+* Each node in DDS is not aware of the global state since they simply have to follow . This proved quite useful since we could deploy the same system in different topologies.
+* 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.
+
+At LinkedIn, we have been able to use this to manage 3 different distributed systems that look very different on paper.  
+
+----------------
+[Home]:wiki/Home "Introduction"
+[SampleApp]:wiki/Sample_App "Sample Application"
+[Quickstart]: wiki/Quickstart "Quick Start Guide"
+[Architecture]: wiki/Architecture "Helix Architecture"
+[Features]: wiki/Features "Features"
+[ApiUsage]: wiki/ApiUsage "Api usage"
+[Javadocs]: http://linkedin.github.com/helix/apidocs/
+[UseCases]: wiki/UseCases "Some usecases at LinkedIn"
+
+
+
+
+   
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/e9814387/src/site/resources/images/bootstrap_statemodel.gif
----------------------------------------------------------------------
diff --git a/src/site/resources/images/bootstrap_statemodel.gif b/src/site/resources/images/bootstrap_statemodel.gif
new file mode 100644
index 0000000..b8f8a42
Binary files /dev/null and b/src/site/resources/images/bootstrap_statemodel.gif differ

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/e9814387/src/site/resources/images/helix-architecture.png
----------------------------------------------------------------------
diff --git a/src/site/resources/images/helix-architecture.png b/src/site/resources/images/helix-architecture.png
new file mode 100644
index 0000000..6f69a2d
Binary files /dev/null and b/src/site/resources/images/helix-architecture.png differ

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/e9814387/src/site/resources/images/helix-znode-layout.png
----------------------------------------------------------------------
diff --git a/src/site/resources/images/helix-znode-layout.png b/src/site/resources/images/helix-znode-layout.png
new file mode 100644
index 0000000..5bafc45
Binary files /dev/null and b/src/site/resources/images/helix-znode-layout.png differ

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/e9814387/src/site/resources/images/statemachine.png
----------------------------------------------------------------------
diff --git a/src/site/resources/images/statemachine.png b/src/site/resources/images/statemachine.png
new file mode 100644
index 0000000..43d27ec
Binary files /dev/null and b/src/site/resources/images/statemachine.png differ

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/e9814387/src/site/resources/images/system.png
----------------------------------------------------------------------
diff --git a/src/site/resources/images/system.png b/src/site/resources/images/system.png
new file mode 100644
index 0000000..f8a05c8
Binary files /dev/null and b/src/site/resources/images/system.png differ

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/e9814387/src/site/site.xml
----------------------------------------------------------------------
diff --git a/src/site/site.xml b/src/site/site.xml
new file mode 100644
index 0000000..61162c6
--- /dev/null
+++ b/src/site/site.xml
@@ -0,0 +1,57 @@
+<?xml version="1.0" encoding="ISO-8859-1"?>
+<!--
+  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.
+-->
+<project name="Apache Helix">
+
+  <publishDate position="right"/>
+  <version position="right"/>
+
+  <skin>
+    <groupId>org.apache.maven.skins</groupId>
+    <artifactId>maven-fluido-skin</artifactId>
+    <version>1.3.0</version>
+  </skin>
+
+  <body>
+
+    <breadcrumbs position="left">
+      <item name="Apache Helix" href="http://helix.incubator.apache.org/"/>
+    </breadcrumbs>
+
+
+    <menu ref="reports" inherit="bottom"/>
+    <menu ref="modules" inherit="bottom"/>
+
+    <menu name="ASF">
+      <item name="How Apache Works" href="http://www.apache.org/foundation/how-it-works.html"/>
+      <item name="Foundation" href="http://www.apache.org/foundation/"/>
+      <item name="Sponsoring Apache" href="http://www.apache.org/foundation/sponsorship.html"/>
+      <item name="Thanks" href="http://www.apache.org/foundation/thanks.html"/>
+    </menu>
+
+
+  </body>
+
+  <custom>
+    <fluidoSkin>
+      <topBarEnabled>true</topBarEnabled>
+      <sideBarEnabled>false</sideBarEnabled>
+      <googleSearch></googleSearch>
+    </fluidoSkin>
+  </custom>
+
+</project>
\ No newline at end of file