You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@helix.apache.org by jx...@apache.org on 2017/06/22 23:15:52 UTC

[09/28] helix git commit: Move latest website back to master branch

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

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

http://git-wip-us.apache.org/repos/asf/helix/blob/39e0d3fb/website/0.6.8/src/site/markdown/tutorial_controller.md
----------------------------------------------------------------------
diff --git a/website/0.6.8/src/site/markdown/tutorial_controller.md b/website/0.6.8/src/site/markdown/tutorial_controller.md
new file mode 100644
index 0000000..d3c5526
--- /dev/null
+++ b/website/0.6.8/src/site/markdown/tutorial_controller.md
@@ -0,0 +1,153 @@
+<!---
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License.
+-->
+
+<head>
+  <title>Tutorial - Controller</title>
+</head>
+
+## [Helix Tutorial](./Tutorial.html): 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 a Connection
+
+The Helix manager 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();
+```
+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 the 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 features 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.
+
+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 features we added in Helix is 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/helix/blob/39e0d3fb/website/0.6.8/src/site/markdown/tutorial_messaging.md
----------------------------------------------------------------------
diff --git a/website/0.6.8/src/site/markdown/tutorial_messaging.md b/website/0.6.8/src/site/markdown/tutorial_messaging.md
new file mode 100644
index 0000000..c0c1987
--- /dev/null
+++ b/website/0.6.8/src/site/markdown/tutorial_messaging.md
@@ -0,0 +1,70 @@
+<!---
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License.
+-->
+
+<head>
+  <title>Tutorial - Messaging</title>
+</head>
+
+## [Helix Tutorial](./Tutorial.html): 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 that 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.  This API 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 the 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 the [Javadocs](http://helix.apache.org/javadocs/0.6.8/reference/org/apache/helix/messaging/DefaultMessagingService.html) for more information.

http://git-wip-us.apache.org/repos/asf/helix/blob/39e0d3fb/website/0.6.8/src/site/markdown/tutorial_participant.md
----------------------------------------------------------------------
diff --git a/website/0.6.8/src/site/markdown/tutorial_participant.md b/website/0.6.8/src/site/markdown/tutorial_participant.md
new file mode 100644
index 0000000..cb38e45
--- /dev/null
+++ b/website/0.6.8/src/site/markdown/tutorial_participant.md
@@ -0,0 +1,102 @@
+<!---
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License.
+-->
+
+<head>
+  <title>Tutorial - Participant</title>
+</head>
+
+## [Helix Tutorial](./Tutorial.html): Participant
+
+In this chapter, we\'ll learn how to implement a __Participant__, which is a primary functional component of a distributed system.
+
+
+### Start a Connection
+
+The Helix manager 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, the 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
+* LeaderStandbyStateModelFactory
+* BootstrapHandler
+
+
+```
+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();
+```
+
+### Example State Model Factory
+
+Helix doesn\'t know what it means to change from OFFLINE\-\-\>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/helix/blob/39e0d3fb/website/0.6.8/src/site/markdown/tutorial_propstore.md
----------------------------------------------------------------------
diff --git a/website/0.6.8/src/site/markdown/tutorial_propstore.md b/website/0.6.8/src/site/markdown/tutorial_propstore.md
new file mode 100644
index 0000000..12dea6b
--- /dev/null
+++ b/website/0.6.8/src/site/markdown/tutorial_propstore.md
@@ -0,0 +1,34 @@
+<!---
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License.
+-->
+
+<head>
+  <title>Tutorial - Application Property Store</title>
+</head>
+
+## [Helix Tutorial](./Tutorial.html): 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 with a write-through cache. This is far more efficient than reading from ZooKeeper for every access.
+
+See [HelixManager.getHelixPropertyStore](http://helix.apache.org/javadocs/0.6.8/reference/org/apache/helix/store/package-summary.html) for details.

http://git-wip-us.apache.org/repos/asf/helix/blob/39e0d3fb/website/0.6.8/src/site/markdown/tutorial_rebalance.md
----------------------------------------------------------------------
diff --git a/website/0.6.8/src/site/markdown/tutorial_rebalance.md b/website/0.6.8/src/site/markdown/tutorial_rebalance.md
new file mode 100644
index 0000000..2e1a79b
--- /dev/null
+++ b/website/0.6.8/src/site/markdown/tutorial_rebalance.md
@@ -0,0 +1,181 @@
+<!---
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License.
+-->
+
+<head>
+  <title>Tutorial - Rebalancing Algorithms</title>
+</head>
+
+## [Helix Tutorial](./Tutorial.html): 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, among others.
+
+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 four options for rebalancing, in increasing order of customization by the system builder:
+
+* FULL_AUTO
+* SEMI_AUTO
+* CUSTOMIZED
+* USER_DEFINED
+
+```
+            |FULL_AUTO     |  SEMI_AUTO | CUSTOMIZED|  USER_DEFINED  |
+            ---------------------------------------------------------|
+   LOCATION | HELIX        |  APP       |  APP      |      APP       |
+            ---------------------------------------------------------|
+      STATE | HELIX        |  HELIX     |  APP      |      APP       |
+            ----------------------------------------------------------
+```
+
+
+### FULL_AUTO
+
+When the rebalance mode is set to FULL_AUTO, 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" : {
+    "REBALANCE_MODE" : "FULL_AUTO",
+    "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..
+
+### SEMI_AUTO
+
+When the application needs to control the placement of the replicas, use the SEMI_AUTO rebalance 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" : {
+    "REBALANCE_MODE" : "SEMI_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 FULL_AUTO 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.
+
+### CUSTOMIZED
+
+Helix offers a third mode called CUSTOMIZED, 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" : {
+    "REBALANCE_MODE" : "CUSTOMIZED",
+    "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.
+
+### USER_DEFINED
+
+For maximum flexibility, Helix exposes an interface that can allow applications to plug in custom rebalancing logic. By providing the name of a class that implements the Rebalancer interface, Helix will automatically call the contained method whenever there is a change to the live participants in the cluster. For more, see [User-Defined Rebalancer](./tutorial_user_def_rebalancer.html).
+
+### Backwards Compatibility
+
+In previous versions, FULL_AUTO was called AUTO_REBALANCE and SEMI_AUTO was called AUTO. Furthermore, they were presented as the IDEAL_STATE_MODE. Helix supports both IDEAL_STATE_MODE and REBALANCE_MODE, but IDEAL_STATE_MODE is now deprecated and may be phased out in future versions.

http://git-wip-us.apache.org/repos/asf/helix/blob/39e0d3fb/website/0.6.8/src/site/markdown/tutorial_spectator.md
----------------------------------------------------------------------
diff --git a/website/0.6.8/src/site/markdown/tutorial_spectator.md b/website/0.6.8/src/site/markdown/tutorial_spectator.md
new file mode 100644
index 0000000..e43cd6b
--- /dev/null
+++ b/website/0.6.8/src/site/markdown/tutorial_spectator.md
@@ -0,0 +1,75 @@
+<!---
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License.
+-->
+
+<head>
+  <title>Tutorial - Spectator</title>
+</head>
+
+## [Helix Tutorial](./Tutorial.html): 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 a Connection
+
+Same as for a participant, The Helix manager is the common component that connects each system component with the cluster.
+
+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, the only thing that needs to be registered is the listener.  When the ExternalView changes, the listener is notified.
+
+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.SPECTATOR,
+                                                zkConnectString);
+manager.connect();
+RoutingTableProvider routingTableProvider = new RoutingTableProvider();
+manager.addExternalViewChangeListener(routingTableProvider);
+```
+
+### Spectator Code
+
+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.

http://git-wip-us.apache.org/repos/asf/helix/blob/39e0d3fb/website/0.6.8/src/site/markdown/tutorial_state.md
----------------------------------------------------------------------
diff --git a/website/0.6.8/src/site/markdown/tutorial_state.md b/website/0.6.8/src/site/markdown/tutorial_state.md
new file mode 100644
index 0000000..856b8b3
--- /dev/null
+++ b/website/0.6.8/src/site/markdown/tutorial_state.md
@@ -0,0 +1,131 @@
+<!---
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License.
+-->
+
+<head>
+  <title>Tutorial - State Machine Configuration</title>
+</head>
+
+## [Helix Tutorial](./Tutorial.html): 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
+
+* 3 states: OFFLINE, SLAVE, MASTER
+* Maximum number of masters: 1
+* Slaves are based on the replication factor. The 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
+<br/>Maximum number of replicas in MASTER state at any time is 1
+
+* OFFLINE-SLAVE:5
+<br/>Maximum number of OFFLINE-SLAVE transitions that can happen concurrently in the system is 5 in this example.
+
+#### Dynamic State Constraints
+
+We also support two dynamic upper bounds for the number of replicas in each state:
+
+* N: The number of replicas in the state is at most the number of live participants in the cluster
+* R: The number of replicas in the state is at most the specified replica count for the partition
+
+#### 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.
+
+### Special States
+
+There are a few Helix-defined states that are important to be aware of.
+
+#### DROPPED
+
+The DROPPED state is used to signify a replica that was served by a given participant, but is no longer served. This allows Helix and its participants to effectively clean up. There are two requirements that every new state model should follow with respect to the DROPPED state:
+
+* The DROPPED state must be defined
+* There must be a path to DROPPED for every state in the model
+
+#### ERROR
+
+The ERROR state is used whenever the participant serving a partition encountered an error and cannot continue to serve the partition. HelixAdmin has \"reset\" functionality to allow for participants to recover from the ERROR state.
+
+### Annotated Example
+
+Below is a complete definition of a Master-Slave state model. Notice the fields marked REQUIRED; these are essential for any state model definition.
+
+```
+StateModelDefinition stateModel = new StateModelDefinition.Builder("MasterSlave")
+  // OFFLINE is the state that the system starts in (initial state is REQUIRED)
+  .initialState("OFFLINE")
+
+  // Lowest number here indicates highest priority, no value indicates lowest priority
+  .addState("MASTER", 1)
+  .addState("SLAVE", 2)
+  .addState("OFFLINE")
+
+  // Note the special inclusion of the DROPPED state (REQUIRED)
+  .addState(HelixDefinedState.DROPPED.toString())
+
+  // No more than one master allowed
+  .upperBound("MASTER", 1)
+
+  // R indicates an upper bound of number of replicas for each partition
+  .dynamicUpperBound("SLAVE", "R")
+
+  // Add some high-priority transitions
+  .addTransition("SLAVE", "MASTER", 1)
+  .addTransition("OFFLINE", "SLAVE", 2)
+
+  // Using the same priority value indicates that these transitions can fire in any order
+  .addTransition("MASTER", "SLAVE", 3)
+  .addTransition("SLAVE", "OFFLINE", 3)
+
+  // Not specifying a value defaults to lowest priority
+  // Notice the inclusion of the OFFLINE to DROPPED transition
+  // Since every state has a path to OFFLINE, they each now have a path to DROPPED (REQUIRED)
+  .addTransition("OFFLINE", HelixDefinedState.DROPPED.toString())
+
+  // Create the StateModelDefinition instance
+  .build();
+
+  // Use the isValid() function to make sure the StateModelDefinition will work without issues
+  Assert.assertTrue(stateModel.isValid());
+```

http://git-wip-us.apache.org/repos/asf/helix/blob/39e0d3fb/website/0.6.8/src/site/markdown/tutorial_task_framework.md
----------------------------------------------------------------------
diff --git a/website/0.6.8/src/site/markdown/tutorial_task_framework.md b/website/0.6.8/src/site/markdown/tutorial_task_framework.md
new file mode 100644
index 0000000..2415a95
--- /dev/null
+++ b/website/0.6.8/src/site/markdown/tutorial_task_framework.md
@@ -0,0 +1,359 @@
+<!---
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License.
+-->
+
+<head>
+  <title>Tutorial - Task Framework</title>
+</head>
+
+## [Helix Tutorial](./Tutorial.html): Task Framework
+
+Task framework, in Helix, provides executable task scheduling and workflow management. In Helix, three layers of task abstraction have been offered to user for defining their logics of dependencies. The graph shows the relationships between three layers. Workflow can contain multiple jobs. One job can depend on other one. Multiple tasks, including same task different partition and different task different partition, can be added in one job.
+Task framework not only can abstract three layers task logics but also helps doing task assignment and rebalancing. User can create a workflow (or a job queue) at first beginning. Then jobs can be added into workflow. Those jobs contain the executable tasks implemented by user. Once workflow is completed, Helix will schedule the works based on the condition user provided.
+
+![Task Framework flow chart](./images/TaskFrameworkLayers.png)
+
+### Key Concepts
+* Task is the basic unit in Helix task framework. It can represents the a single runnable logics that user prefer to execute for each partition (distributed units).
+* Job defines one time operation across all the partitions. It contains multiple Tasks and configuration of tasks, such as how many tasks, timeout per task and so on.
+* Workflow is directed acyclic graph represents the relationships and running orders of Jobs. In addition, a workflow can also provide customized configuration, for example, Job dependencies.
+* JobQueue is another type of Workflow. Different from normal one, JobQueue is not terminated until user kill it. Also JobQueue can keep accepting newly coming jobs.
+
+### Implement Your Task
+
+#### [Task Interface](https://github.com/apache/helix/blob/helix-0.6.x/helix-core/src/main/java/org/apache/helix/task/Task.java)
+
+The task interface contains two methods: run and cancel. User can implement his or her own logic in run function and cancel / roll back logic in cancel function.
+
+```
+public class MyTask implements Task {
+  @Override
+  TaskResult run() {
+    // Task logic
+  }
+ 
+  @Override
+  void cancel() {
+    // Cancel logic
+  }
+}
+```
+
+#### [TaskConfig](https://github.com/apache/helix/blob/helix-0.6.x/helix-core/src/main/java/org/apache/helix/task/TaskConfig.java)
+
+In helix, usually an object config represents the abstraction of that object, such as TaskConfig, JobConfig and WorkflowConfig. TaskConfig contains configurable task conditions. TaskConfig does not require to have any input to create a new object:
+
+```
+TaskConfig taskConfig = new TaskConfig(null, null, null, null);
+```
+
+For these four fields:
+* Command: The task command, will use Job command if this is null
+* ID: Task unique id, will generate a new ID for this task if input is null
+* TaskTargetPartition: Target partition of a target. Could be null
+* ConfigMap: Task property key-value map containing all other property stated above, such as command, ID.
+
+#### Share Content Across Tasks and Jobs
+
+Task framework also provides a feature that user can store the key-value data per task, job and workflow. The content stored at workflow layer can shared by different jobs belong to this workflow. Similarly content persisted at job layer can shared by different tasks nested in this job. Currently, user can extend the abstract class [UserContentStore](https://github.com/apache/helix/blob/helix-0.6.x/helix-core/src/main/java/org/apache/helix/task/UserContentStore.java) and use two methods putUserContent and getUserContent. It will similar to hash map put and get method except a Scope.  The Scope will define which layer this key-value pair to be persisted.
+
+```
+public class MyTask extends UserContentStore implements Task {
+  @Override
+  TaskResult run() {
+    putUserContent("KEY", "WORKFLOWVALUE", SCOPE.WORKFLOW);
+    putUserContent("KEY", "JOBVALUE", SCOPE.JOB);
+    putUserContent("KEY", "TASKVALUE", SCOPE.TASK);
+    String taskValue = getUserContent("KEY", SCOPE.TASK);
+  }
+ ...
+}
+```
+
+#### Return [Task Results](https://github.com/apache/helix/blob/helix-0.6.x/helix-core/src/main/java/org/apache/helix/task/TaskResult.java)
+
+User can define the TaskResult for a task once it is at final stage (complete or failed). The TaskResult contains two fields: status and info. Status is current Task Status including COMPLETED, CANCELLED, FAILED and FATAL_FAILED. The difference between FAILED and FATAL_FAILED is that once the task defined as FATAL_FAILED, helix will not do the retry for this task and abort it. The other field is information, which is a String type. User can pass any information including error message, description and so on.
+
+```
+TaskResult run() {
+    ....
+    return new TaskResult(TaskResult.Status.FAILED, "ERROR MESSAGE OR OTHER INFORMATION");
+}
+```
+
+#### Task Retry and Abort
+
+Helix provides retry logics to users. User can specify the how many times allowed to tolerant failure of tasks under a job. It is a method will be introduced in Following Job Section. Another choice offered to user that if user thinks a task is very critical and do not want to do the retry once it is failed, user can return a TaskResult stated above with FATAL_FAILED status. Then Helix will not do the retry for that task.
+
+```
+return new TaskResult(TaskResult.Status.FATAL_FAILED, "DO NOT WANT TO RETRY, ERROR MESSAGE");
+```
+
+#### [TaskDriver](https://github.com/apache/helix/blob/helix-0.6.x/helix-core/src/main/java/org/apache/helix/task/TaskDriver.java)
+
+All the control operation related to workflow and job are based on TaskDriver object. TaskDriver offers several APIs to controller, modify and track the tasks. Those APIs will be introduced in each section when they are necessary. TaskDriver object can be created either by [HelixManager](https://github.com/apache/helix/blob/helix-0.6.x/helix-core/src/main/java/org/apache/helix/HelixManager.java) or [ZkClient](https://github.com/apache/helix/blob/helix-0.6.x/helix-core/src/main/java/org/apache/helix/manager/zk/ZkClient.java) with cluster name:
+
+```
+HelixManager manager = new ZKHelixManager(CLUSTER_NAME, INSTANCE_NAME, InstanceType.PARTICIPANT, ZK_ADDRESS);
+TaskDriver taskDriver1 = new TaskDriver(manager);
+ 
+TaskDriver taskDriver2 = new TaskDriver(zkclient, CLUSTER_NAME);
+```
+
+#### Propagate Task Error Message to Helix
+
+When task encounter an error, it could be returned by TaskResult. Unfortunately, user can not get this TaskResult object directly. But Helix provides error messages persistent. Thus user can fetch the error messages from Helix via TaskDriver, which introduced above. The error messages will be stored in Info field per Job. Thus user have to get JobContext, which is the job status and result object.
+
+```
+taskDriver.getJobContext("JOBNAME").getInfo();
+```
+
+### Creating a Workflow
+
+#### One-time Workflow
+
+As common use, one-time workflow will be the default workflow as user created. The first step is to create a WorkflowConfig.Builder object with workflow name. Then all configs can be set in WorkflowConfig.Builder. Once the configuration is done, [WorkflowConfig](https://github.com/apache/helix/blob/helix-0.6.x/helix-core/src/main/java/org/apache/helix/task/WorkflowConfig.java) object can be got from WorkflowConfig.Builder object.
+We have two rules to validate the Workflow configuration:
+* Expiry time should not be less than 0
+* Schedule config should be valid either one-time or a positive interval magnitude (Recurrent workflow)
+Example:
+
+```
+Workflow.Builder myWorkflowBuilder = new Workflow.Builder("MyWorkflow");
+myWorkflowBuilder.setExpiry(5000L);
+Workflow myWorkflow = myWorkflowBuilder.build();
+```
+
+#### Recurrent Workflow
+
+Recurrent workflow is the workflow scheduled periodically. The only config different from One-time workflow is to set a recurrent [ScheduleConfig](https://github.com/apache/helix/blob/helix-0.6.x/helix-core/src/main/java/org/apache/helix/task/ScheduleConfig.java). There two methods in ScheduleConfig can help you to create a ScheduleConfig object: recurringFromNow and recurringFromDate. Both of them needs recurUnit (time unit for recurrent) and recurInteval (magnitude of recurrent interval). Here's the example:
+
+```
+ScheduleConfig myConfig1 = ScheduleConfig.recurringFFromNow(TimeUnit.MINUTES, 5L);
+ScheduleConfig myConfig2 = ScheduleConfig.recurringFFromDate(Calendar.getInstance.getTime, TimeUnit.HOURS, 10L);
+```
+
+Once this schedule config is created. It could be set in the workflow config:
+
+```
+Workflow.Builder myWorkflowBuilder = new Workflow.Builder("MyWorkflow");
+myWorkflowBuilder.setExpiry(2000L)
+                 .setScheduleConfig(ScheduleConfig.recurringFromNow(TimeUnit.DAYS, 5));
+Workflow myWorkflow = myWorkflowBuilder.build();
+```
+
+#### Start a Workflow
+
+Start a workflow is just using taskdrive to start it. Since this is an async call, after start the workflow, user can keep doing actions.
+
+```
+taskDriver.start(myWorkflow);
+```
+
+#### Stop a Workflow
+
+Stop workflow can be executed via TaskDriver:
+
+```
+taskDriver.stop(myWorkflow);
+```
+
+#### Resume a Workflow
+
+Once the workflow is stopped, it does not mean the workflow is gone. Thus user can resume the workflow that has been stopped. Using TaskDriver resume the workflow:
+
+```
+taskDriver.resume(myWorkflow);
+```
+
+#### Delete a Workflow
+
+Simliar to start, stop and resume, delete operation is supported by TaskDriver.
+
+```
+taskDriver.delete(myWorkflow);
+```
+
+#### Add a Job
+
+WARNING: Job can only be added to WorkflowConfig.Builder. Once WorkflowConfig built, no job can be added! For creating a Job, please refering following section (Create a Job)
+
+```
+myWorkflowBuilder.addJob("JobName", jobConfigBuilder);
+```
+
+#### Add a Job dependency
+
+Jobs can have dependencies. If one job2 depends job1, job2 will not be scheduled until job1 finished.
+
+```
+myWorkflowBuilder.addParentChildDependency(ParentJobName, ChildJobName);
+```
+
+#### Additional Workflow Options
+
+| Additional Config Options | Detail |
+| ------------------------- | ------ |
+| _setJobDag(JobDag v)_ | If user already defined the job DAG, it could be set with this method. |
+| _setExpiry(long v, TimeUnit unit)_ | Set the expiration time for this workflow. |
+| _setFailureThreshold(int failureThreshold)_ | Set the failure threshold for this workflow, once job failures reach this number, the workflow will be failed. |
+| _setWorkflowType(String workflowType)_ | Set the user defined workflowType for this workflow. |
+| _setTerminable(boolean isTerminable)_ | Set the whether this workflow is terminable or not. |
+| _setCapacity(int capacity)_ | Set the number of jobs that workflow can hold before reject further jobs. Only used when workflow is not terminable. |
+| _setTargetState(TargetState v)_ | Set the final state of this workflow. |
+
+### Creating a Queue
+
+[Job queue](https://github.com/apache/helix/blob/helix-0.6.x/helix-core/src/main/java/org/apache/helix/task/JobQueue.java) is another shape of workflow. Here listed different between a job queue and workflow:
+
+| Property | Workflow | Job Queue |
+| -------- | -------- | --------- |
+| Existing time | Workflow will be deleted after it is done. | Job queue will be there until user delete it. |
+| Add jobs | Once workflow is build, no job can be added. | Job queue can keep accepting jobs. |
+| Parallel run | Allows parallel run for jobs without dependencies | No parallel run allowed except setting _ParallelJobs_ |
+
+For creating a job queue, user have to provide queue name and workflow config (please refer above Create a Workflow). Similar to other task object, create a JobQueue.Builder first. Then JobQueue can be validated and generated via build function.
+
+```
+WorkflowConfig.Builder myWorkflowCfgBuilder = new WorkflowConfig.Builder().setWorkFlowType("MyType");
+JobQueue jobQueue = new JobQueue.Builder("MyQueueName").setWorkflowConfig(myWorkflowCfgBuilder.build()).build();
+```
+
+####Append Job to Queue
+
+WARNING:Different from normal workflow, job for JobQueue can be append even in anytime. Similar to workflow add a job, job can be appended via enqueueJob function via TaskDriver.
+
+```
+jobQueueBuilder.enqueueJob("JobName", jobConfigBuilder);
+```
+
+####Delete Job from Queue
+
+Helix allowed user to delete a job from existing queue. We offers delete API in TaskDriver to do this. Delete job from queue and this queue has to be stopped. Then user can resume the job once delete success.
+
+```
+taskDriver.stop("QueueName");
+taskDriver.deleteJob("QueueName", "JobName");
+taskDriver.resume("QueueName");
+```
+
+####Additional Option for JobQueue
+
+_setParallelJobs(int parallelJobs)_ : Set the how many jobs can parallel running, except there is any dependencies.
+
+###Create a Job
+
+Before generate a [JobConfig](https://github.com/apache/helix/blob/helix-0.6.x/helix-core/src/main/java/org/apache/helix/task/JobConfig.java) object, user still have to use JobConfig.Builder to build JobConfig.
+
+```
+JobConfig.Builder myJobCfgBuilder = new JobConfig.Builder();
+JobConfig myJobCfg = myJobCfgBuilder.build();
+```
+
+Helix has couple rules to validate a job:
+* Each job must at least have one task to execute. For adding tasks and task rules please refer following section Add Tasks.
+* Task timeout should not less than zero.
+* Number of concurrent tasks per instances should not less than one.
+* Maximum attempts per task should not less than one
+* There must be a workflow name
+
+#### Add Tasks
+
+There are two ways of adding tasks:
+* Add by TaskConfig. Tasks can be added via adding TaskConfigs. User can create a List of TaskConfigs or add TaskConfigMap, which is a task id to TaskConfig mapping.
+
+```
+TaskConfig taskCfg = new TaskConfig(null, null, null, null);
+List<TaskConfig> taskCfgs = new ArrayList<TaskConfig>();
+myJobCfg.addTaskConfigs(taskCfgs);
+ 
+Map<String, TaskConfig> taskCfgMap = new HashMap<String, TaskConfig>();
+taskCfgMap.put(taskCfg.getId(), taskCfg);
+myJobCfg.addTaskConfigMap(taskCfgMap);
+```
+
+* Add by Job command. If user does not want to specify each TaskConfig, we can create identical tasks based on Job command with number of tasks.
+
+```
+myJobCfg.setCommand("JobCommand").setNumberOfTasks(10);
+```
+WARNING: Either user provides TaskConfigs / TaskConfigMap or both of Job command and number tasks (except Targeted Job, refer following section) . Otherwise, validation will be failed.
+
+#### Generic Job
+
+Generic Job is the default job created. It does not have targeted resource. Thus this generic job could be assigned to one of eligble instances.
+
+#### Targeted Job
+
+Targeted Job has set up the target resource. For this kind of job, Job command is necessary, but number of tasks is not. The tasks will depends on the partion number of targeted resource. To set target resource, just put target resource name to JobConfig.Builder.
+
+```
+myJobCfgBuilder.setTargetResource("TargetResourceName");
+```
+
+In addition, user can specify the instance target state. For example, if user want to run the Task on "Master" state instance, setTargetPartitionState method can help to set the partition to assign to specific instance.
+
+```
+myJobCfgBuilder.setTargetPartitionState(Arrays.asList(new String[]{"Master", "Slave"}));
+```
+
+#### Instance Group
+
+Grouping jobs with targeted group of instances feature has been supported. User firstly have to define the instance group tag for instances, which means label some instances with specific tag. Then user can put those tags to a job that only would like to assigned to those instances. For example, customer data only available on instance 1, 2, 3. These three instances can be tagged as "CUSTOMER" and  customer data related jobs can set  the instance group tag "CUSTOMER". Thus customer data related jobs will only assign to instance 1, 2, 3. 
+To add instance group tag, just set it in JobConfig.Builder:
+
+```
+jobCfg.setInstanceGroupTag("INSTANCEGROUPTAG");
+```
+
+#### Additional Job Options
+
+| Operation | Detail |
+| --------- | ------ |
+| _setWorkflow(String workflowName)_ | Set the workflow that this job belongs to |
+| _setTargetPartions(List\<String\> targetPartionNames)_ | Set list of partition names |
+| _setTargetPartionStates(Set\<String\>)_ | Set the partition states |
+| _setCommand(String command)_ | Set the job command |
+| _setJobCommandConfigMap(Map\<String, String\> v)_ | Set the job command config maps |
+| _setTimeoutPerTask(long v)_ | Set the timeout for each task |
+| _setNumConcurrentTasksPerInstance(int v)_ | Set number of tasks can concurrent run on same instance |
+| _setMaxAttemptsPerTask(int v)_ | Set times of retry for a task |
+| _setFailureThreshold(int v)_ | Set failure tolerance of tasks for this job |
+| _setTaskRetryDelay(long v)_ | Set the delay time before a task retry |
+| _setIgnoreDependentJobFailure(boolean ignoreDependentJobFailure)_ | Set whether ignore the job failure of parent job of this job |
+| _setJobType(String jobType)_ | Set the job type of this job |
+
+### Monitor the status of your job
+As we introduced the excellent util TaskDriver in Workflow Section, we have extra more functionality that provided to user. The user can synchronized wait Job or Workflow until it reaches certain STATES. The function Helix have API pollForJobState and pollForWorkflowState. For pollForJobState, it accepts arguments:
+* Workflow name, required
+* Job name, required
+* Timeout, not required, will be three minutes if user choose function without timeout argument. Time unit is milisecond.
+* TaskStates, at least one state. This function can accept multiple TaskState, will end function until one of those TaskState reaches.
+For example:
+
+```
+taskDriver.pollForJobState("MyWorkflowName", "MyJobName", 180000L, TaskState.FAILED, TaskState.FATAL_FAILED);
+taskDriver.pollForJobState("MyWorkflowName", "MyJobName", TaskState.COMPLETED);
+```
+
+For pollForWorkflowState, it accepts similar arguments except Job name. For example:
+
+```
+taskDriver.pollForWorkflowState("MyWorkflowName", 180000L, TaskState.FAILED, TaskState.FATAL_FAILED);
+taskDriver.pollForWorkflowState("MyWorkflowName", TaskState.COMPLETED);
+```

http://git-wip-us.apache.org/repos/asf/helix/blob/39e0d3fb/website/0.6.8/src/site/markdown/tutorial_throttling.md
----------------------------------------------------------------------
diff --git a/website/0.6.8/src/site/markdown/tutorial_throttling.md b/website/0.6.8/src/site/markdown/tutorial_throttling.md
new file mode 100644
index 0000000..16a6f81
--- /dev/null
+++ b/website/0.6.8/src/site/markdown/tutorial_throttling.md
@@ -0,0 +1,39 @@
+<!---
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License.
+-->
+
+<head>
+  <title>Tutorial - Throttling</title>
+</head>
+
+## [Helix Tutorial](./Tutorial.html): Throttling
+
+In this chapter, we\'ll learn how to control the parallel execution of cluster tasks.  Only a centralized cluster manager with global knowledge (i.e. Helix) 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 lightweight, 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
+
+