http://git-wip-us.apache.org/repos/asf/helix/blob/e9428536/website/0.8.0/src/site/markdown/tutorial_task_framework.md
----------------------------------------------------------------------
diff --git a/website/0.8.0/src/site/markdown/tutorial_task_framework.md 
b/website/0.8.0/src/site/markdown/tutorial_task_framework.md
new file mode 100644
index 0000000..2415a95
--- /dev/null
+++ b/website/0.8.0/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/e9428536/website/0.8.0/src/site/markdown/tutorial_throttling.md
----------------------------------------------------------------------
diff --git a/website/0.8.0/src/site/markdown/tutorial_throttling.md 
b/website/0.8.0/src/site/markdown/tutorial_throttling.md
new file mode 100644
index 0000000..16a6f81
--- /dev/null
+++ b/website/0.8.0/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
+
+

http://git-wip-us.apache.org/repos/asf/helix/blob/e9428536/website/0.8.0/src/site/markdown/tutorial_user_def_rebalancer.md
----------------------------------------------------------------------
diff --git a/website/0.8.0/src/site/markdown/tutorial_user_def_rebalancer.md 
b/website/0.8.0/src/site/markdown/tutorial_user_def_rebalancer.md
new file mode 100644
index 0000000..2149739
--- /dev/null
+++ b/website/0.8.0/src/site/markdown/tutorial_user_def_rebalancer.md
@@ -0,0 +1,172 @@
+<!---
+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 - User-Defined Rebalancing</title>
+</head>
+
+## [Helix Tutorial](./Tutorial.html): User-Defined Rebalancing
+
+Even though Helix can compute both the location and the state of replicas 
internally using a default fully-automatic rebalancer, specific applications 
may require rebalancing strategies that optimize for different requirements. 
Thus, Helix allows applications to plug in arbitrary rebalancer algorithms that 
implement a provided interface. One of the main design goals of Helix is to 
provide maximum flexibility to any distributed application. Thus, it allows 
applications to fully implement the rebalancer, which is the core constraint 
solver in the system, if the application developer so chooses.
+
+Whenever the state of the cluster changes, as is the case when participants 
join or leave the cluster, Helix automatically calls the rebalancer to compute 
a new mapping of all the replicas in the resource. When using a pluggable 
rebalancer, the only required step is to register it with Helix. Subsequently, 
no additional bootstrapping steps are necessary. Helix uses reflection to look 
up and load the class dynamically at runtime. As a result, it is also 
technically possible to change the rebalancing strategy used at any time.
+
+The Rebalancer interface is as follows:
+
+```
+void init(HelixManager manager);
+
+IdealState computeNewIdealState(String resourceName, IdealState 
currentIdealState,
+    final CurrentStateOutput currentStateOutput, final ClusterDataCache 
clusterData);
+```
+The first parameter is the resource to rebalance, the second is pre-existing 
ideal mappings, the third is a snapshot of the actual placements and state 
assignments, and the fourth is a full cache of all of the cluster data 
available to Helix. Internally, Helix implements the same interface for its own 
rebalancing routines, so a user-defined rebalancer will be cognizant of the 
same information about the cluster as an internal implementation. Helix strives 
to provide applications the ability to implement algorithms that may require a 
large portion of the entire state of the cluster to make the best placement and 
state assignment decisions possible.
+
+An IdealState is a full representation of the location of each replica of each 
partition of a given resource. This is a simple representation of the placement 
that the algorithm believes is the best possible. If the placement meets all 
defined constraints, this is what will become the actual state of the 
distributed system.
+
+### Specifying a Rebalancer
+For implementations that set up the cluster through existing code, the 
following HelixAdmin calls will update the Rebalancer class:
+
+```
+IdealState idealState = helixAdmin.getResourceIdealState(clusterName, 
resourceName);
+idealState.setRebalanceMode(RebalanceMode.USER_DEFINED);
+idealState.setRebalancerClassName(className);
+helixAdmin.setResourceIdealState(clusterName, resourceName, idealState);
+```
+
+There are two key fields to set to specify that a pluggable rebalancer should 
be used. First, the rebalance mode should be set to USER_DEFINED, and second 
the rebalancer class name should be set to a class that implements Rebalancer 
and is within the scope of the project. The class name is a fully-qualified 
class name consisting of its package and its name. Without specification of the 
USER_DEFINED mode, the user-defined rebalancer class will not be used even if 
specified. Furthermore, Helix will not attempt to rebalance the resources 
through its standard routines if its mode is USER_DEFINED, regardless of 
whether or not a rebalancer class is registered.
+
+### Example
+
+In the next release (0.7.0), we will provide a full recipe of a user-defined 
rebalancer in action.
+
+Consider the case where partitions are locks in a lock manager and 6 locks are 
to be distributed evenly to a set of participants, and only one participant can 
hold each lock. We can define a rebalancing algorithm that simply takes the 
modulus of the lock number and the number of participants to evenly distribute 
the locks across participants. Helix allows capping the number of partitions a 
participant can accept, but since locks are lightweight, we do not need to 
define a restriction in this case. The following is a succinct implementation 
of this algorithm.
+
+```
+@Override
+IdealState computeNewIdealState(String resourceName, IdealState 
currentIdealState,
+    final CurrentStateOutput currentStateOutput, final ClusterDataCache 
clusterData) {
+  // Get the list of live participants in the cluster
+  List<String> liveParticipants = new 
ArrayList<String>(clusterData.getLiveInstances().keySet());
+
+  // Count the number of participants allowed to lock each lock (in this 
example, this is 1)
+  int lockHolders = Integer.parseInt(currentIdealState.getReplicas());
+
+  // Fairly assign the lock state to the participants using a simple mod-based 
sequential
+  // assignment. For instance, if each lock can be held by 3 participants, 
lock 0 would be held
+  // by participants (0, 1, 2), lock 1 would be held by (1, 2, 3), and so on, 
wrapping around the
+  // number of participants as necessary.
+  int i = 0;
+  for (String partition : currentIdealState.getPartitionSet()) {
+    List<String> preferenceList = new ArrayList<String>();
+    for (int j = i; j < i + lockHolders; j++) {
+      int participantIndex = j % liveParticipants.size();
+      String participant = liveParticipants.get(participantIndex);
+      // enforce that a participant can only have one instance of a given lock
+      if (!preferenceList.contains(participant)) {
+        preferenceList.add(participant);
+      }
+    }
+    currentIdealState.setPreferenceList(partition, preferenceList);
+    i++;
+  }
+  return assignment;
+}
+```
+
+Here are the IdealState preference lists emitted by the user-defined 
rebalancer for a 3-participant system whenever there is a change to the set of 
participants.
+
+* Participant_A joins
+
+```
+{
+  "lock_0": ["Participant_A"],
+  "lock_1": ["Participant_A"],
+  "lock_2": ["Participant_A"],
+  "lock_3": ["Participant_A"],
+  "lock_4": ["Participant_A"],
+  "lock_5": ["Participant_A"],
+}
+```
+
+A preference list is a mapping for each resource of partition to the 
participants serving each replica. The state model is a simple LOCKED/RELEASED 
model, so participant A holds all lock partitions in the LOCKED state.
+
+* Participant_B joins
+
+```
+{
+  "lock_0": ["Participant_A"],
+  "lock_1": ["Participant_B"],
+  "lock_2": ["Participant_A"],
+  "lock_3": ["Participant_B"],
+  "lock_4": ["Participant_A"],
+  "lock_5": ["Participant_B"],
+}
+```
+
+Now that there are two participants, the simple mod-based function assigns 
every other lock to the second participant. On any system change, the 
rebalancer is invoked so that the application can define how to redistribute 
its resources.
+
+* Participant_C joins (steady state)
+
+```
+{
+  "lock_0": ["Participant_A"],
+  "lock_1": ["Participant_B"],
+  "lock_2": ["Participant_C"],
+  "lock_3": ["Participant_A"],
+  "lock_4": ["Participant_B"],
+  "lock_5": ["Participant_C"],
+}
+```
+
+This is the steady state of the system. Notice that four of the six locks now 
have a different owner. That is because of the naïve modulus-based assignmemt 
approach used by the user-defined rebalancer. However, the interface is 
flexible enough to allow you to employ consistent hashing or any other scheme 
if minimal movement is a system requirement.
+
+* Participant_B fails
+
+```
+{
+  "lock_0": ["Participant_A"],
+  "lock_1": ["Participant_C"],
+  "lock_2": ["Participant_A"],
+  "lock_3": ["Participant_C"],
+  "lock_4": ["Participant_A"],
+  "lock_5": ["Participant_C"],
+}
+```
+
+On any node failure, as in the case of node addition, the rebalancer is 
invoked automatically so that it can generate a new mapping as a response to 
the change. Helix ensures that the Rebalancer has the opportunity to reassign 
locks as required by the application.
+
+* Participant_B (or the replacement for the original Participant_B) rejoins
+
+```
+{
+  "lock_0": ["Participant_A"],
+  "lock_1": ["Participant_B"],
+  "lock_2": ["Participant_C"],
+  "lock_3": ["Participant_A"],
+  "lock_4": ["Participant_B"],
+  "lock_5": ["Participant_C"],
+}
+```
+
+The rebalancer was invoked once again and the resulting IdealState preference 
lists reflect the steady state.
+
+### Caveats
+- The rebalancer class must be available at runtime, or else Helix will not 
attempt to rebalance at all
+- The Helix controller will only take into account the preference lists in the 
new IdealState for this release. In 0.7.0, Helix rebalancers will be able to 
compute the full resource assignment, including the states.
+- Helix does not currently persist the new IdealState computed by the 
user-defined rebalancer. However, the Helix property store is available for 
saving any computed state. In 0.7.0, Helix will persist the result of running 
the rebalancer.

http://git-wip-us.apache.org/repos/asf/helix/blob/e9428536/website/0.8.0/src/site/markdown/tutorial_yaml.md
----------------------------------------------------------------------
diff --git a/website/0.8.0/src/site/markdown/tutorial_yaml.md 
b/website/0.8.0/src/site/markdown/tutorial_yaml.md
new file mode 100644
index 0000000..1e4772e
--- /dev/null
+++ b/website/0.8.0/src/site/markdown/tutorial_yaml.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 - YAML Cluster Setup</title>
+</head>
+
+## [Helix Tutorial](./Tutorial.html): YAML Cluster Setup
+
+As an alternative to using Helix Admin to set up the cluster, its resources, 
constraints, and the state model, Helix supports bootstrapping a cluster 
configuration based on a YAML file. Below is an annotated example of such a 
file for a simple distributed lock manager where a lock can only be LOCKED or 
RELEASED, and each lock only allows a single participant to hold it in the 
LOCKED state.
+
+```
+clusterName: lock-manager-custom-rebalancer # unique name for the cluster 
(required)
+resources:
+  - name: lock-group # unique resource name (required)
+    rebalancer: # required
+      mode: USER_DEFINED # required - USER_DEFINED means we will provide our 
own rebalancer
+      class: org.apache.helix.userdefinedrebalancer.LockManagerRebalancer # 
required for USER_DEFINED
+    partitions:
+      count: 12 # number of partitions for the resource (default is 1)
+      replicas: 1 # number of replicas per partition (default is 1)
+    stateModel:
+      name: lock-unlock # model name (required)
+      states: [LOCKED, RELEASED, DROPPED] # the list of possible states 
(required if model not built-in)
+      transitions: # the list of possible transitions (required if model not 
built-in)
+        - name: Unlock
+          from: LOCKED
+          to: RELEASED
+        - name: Lock
+          from: RELEASED
+          to: LOCKED
+        - name: DropLock
+          from: LOCKED
+          to: DROPPED
+        - name: DropUnlock
+          from: RELEASED
+          to: DROPPED
+        - name: Undrop
+          from: DROPPED
+          to: RELEASED
+      initialState: RELEASED # (required if model not built-in)
+    constraints:
+      state:
+        counts: # maximum number of replicas of a partition that can be in 
each state (required if model not built-in)
+          - name: LOCKED
+            count: "1"
+          - name: RELEASED
+            count: "-1"
+          - name: DROPPED
+            count: "-1"
+        priorityList: [LOCKED, RELEASED, DROPPED] # states in order of 
priority (all priorities equal if not specified)
+      transition: # transitions priority to enforce order that transitions 
occur
+        priorityList: [Unlock, Lock, Undrop, DropUnlock, DropLock] # all 
priorities equal if not specified
+participants: # list of nodes that can serve replicas (optional if dynamic 
joining is active, required otherwise)
+  - name: localhost_12001
+    host: localhost
+    port: 12001
+  - name: localhost_12002
+    host: localhost
+    port: 12002
+  - name: localhost_12003
+    host: localhost
+    port: 12003
+```
+
+Using a file like the one above, the cluster can be set up either with the 
command line:
+
+```
+helix/helix-core/target/helix-core/pkg/bin/YAMLClusterSetup.sh localhost:2199 
lock-manager-config.yaml
+```
+
+or with code:
+
+```
+YAMLClusterSetup setup = new YAMLClusterSetup(zkAddress);
+InputStream input =
+    Thread.currentThread().getContextClassLoader()
+        .getResourceAsStream("lock-manager-config.yaml");
+YAMLClusterSetup.YAMLClusterConfig config = setup.setupCluster(input);
+```
+
+Some notes:
+
+- A rebalancer class is only required for the USER_DEFINED mode. It is ignored 
otherwise.
+
+- Built-in state models, like OnlineOffline, LeaderStandby, and MasterSlave, 
or state models that have already been added only require a name for 
stateModel. If partition and/or replica counts are not provided, a value of 1 
is assumed.

http://git-wip-us.apache.org/repos/asf/helix/blob/e9428536/website/0.8.0/src/site/resources/.htaccess
----------------------------------------------------------------------
diff --git a/website/0.8.0/src/site/resources/.htaccess 
b/website/0.8.0/src/site/resources/.htaccess
new file mode 100644
index 0000000..d5c7bf3
--- /dev/null
+++ b/website/0.8.0/src/site/resources/.htaccess
@@ -0,0 +1,20 @@
+#
+# 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.
+#
+
+Redirect /download.html /download.cgi

http://git-wip-us.apache.org/repos/asf/helix/blob/e9428536/website/0.8.0/src/site/resources/css/bootstrap-responsive.min.css
----------------------------------------------------------------------
diff --git a/website/0.8.0/src/site/resources/css/bootstrap-responsive.min.css 
b/website/0.8.0/src/site/resources/css/bootstrap-responsive.min.css
new file mode 100644
index 0000000..5cb833f
--- /dev/null
+++ b/website/0.8.0/src/site/resources/css/bootstrap-responsive.min.css
@@ -0,0 +1,9 @@
+/*!
+ * Bootstrap Responsive v2.2.2
+ *
+ * Copyright 2012 Twitter, Inc
+ * Licensed under the Apache License v2.0
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Designed and built with all the love in the world @twitter by @mdo and @fat.
+ 
*/@-ms-viewport{width:device-width}.clearfix{*zoom:1}.clearfix:before,.clearfix:after{display:table;line-height:0;content:""}.clearfix:after{clear:both}.hide-text{font:0/0
 
a;color:transparent;text-shadow:none;background-color:transparent;border:0}.input-block-level{display:block;width:100%;min-height:30px;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}.hidden{display:none;visibility:hidden}.visible-phone{display:none!important}.visible-tablet{display:none!important}.hidden-desktop{display:none!important}.visible-desktop{display:inherit!important}@media(min-width:768px)
 and 
(max-width:979px){.hidden-desktop{display:inherit!important}.visible-desktop{display:none!important}.visible-tablet{display:inherit!important}.hidden-tablet{display:none!important}}@media(max-width:767px){.hidden-desktop{display:inherit!important}.visible-desktop{display:none!important}.visible-phone{display:inherit!important}.hidden-phone{display:none!important}}@media(min-width:1
 
200px){.row{margin-left:-30px;*zoom:1}.row:before,.row:after{display:table;line-height:0;content:""}.row:after{clear:both}[class*="span"]{float:left;min-height:1px;margin-left:30px}.container,.navbar-static-top
 .container,.navbar-fixed-top .container,.navbar-fixed-bottom 
.container{width:1170px}.span12{width:1170px}.span11{width:1070px}.span10{width:970px}.span9{width:870px}.span8{width:770px}.span7{width:670px}.span6{width:570px}.span5{width:470px}.span4{width:370px}.span3{width:270px}.span2{width:170px}.span1{width:70px}.offset12{margin-left:1230px}.offset11{margin-left:1130px}.offset10{margin-left:1030px}.offset9{margin-left:930px}.offset8{margin-left:830px}.offset7{margin-left:730px}.offset6{margin-left:630px}.offset5{margin-left:530px}.offset4{margin-left:430px}.offset3{margin-left:330px}.offset2{margin-left:230px}.offset1{margin-left:130px}.row-fluid{width:100%;*zoom:1}.row-fluid:before,.row-fluid:after{display:table;line-height:0;content:""}.row-fluid:after{clear:both}.row-fl
 uid 
[class*="span"]{display:block;float:left;width:100%;min-height:30px;margin-left:2.564102564102564%;*margin-left:2.5109110747408616%;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}.row-fluid
 [class*="span"]:first-child{margin-left:0}.row-fluid .controls-row 
[class*="span"]+[class*="span"]{margin-left:2.564102564102564%}.row-fluid 
.span12{width:100%;*width:99.94680851063829%}.row-fluid 
.span11{width:91.45299145299145%;*width:91.39979996362975%}.row-fluid 
.span10{width:82.90598290598291%;*width:82.8527914166212%}.row-fluid 
.span9{width:74.35897435897436%;*width:74.30578286961266%}.row-fluid 
.span8{width:65.81196581196582%;*width:65.75877432260411%}.row-fluid 
.span7{width:57.26495726495726%;*width:57.21176577559556%}.row-fluid 
.span6{width:48.717948717948715%;*width:48.664757228587014%}.row-fluid 
.span5{width:40.17094017094017%;*width:40.11774868157847%}.row-fluid 
.span4{width:31.623931623931625%;*width:31.570740134569924%}.row-fluid 
.span3{width:23.0
 76923076923077%;*width:23.023731587561375%}.row-fluid 
.span2{width:14.52991452991453%;*width:14.476723040552828%}.row-fluid 
.span1{width:5.982905982905983%;*width:5.929714493544281%}.row-fluid 
.offset12{margin-left:105.12820512820512%;*margin-left:105.02182214948171%}.row-fluid
 
.offset12:first-child{margin-left:102.56410256410257%;*margin-left:102.45771958537915%}.row-fluid
 
.offset11{margin-left:96.58119658119658%;*margin-left:96.47481360247316%}.row-fluid
 
.offset11:first-child{margin-left:94.01709401709402%;*margin-left:93.91071103837061%}.row-fluid
 
.offset10{margin-left:88.03418803418803%;*margin-left:87.92780505546462%}.row-fluid
 
.offset10:first-child{margin-left:85.47008547008548%;*margin-left:85.36370249136206%}.row-fluid
 
.offset9{margin-left:79.48717948717949%;*margin-left:79.38079650845607%}.row-fluid
 
.offset9:first-child{margin-left:76.92307692307693%;*margin-left:76.81669394435352%}.row-fluid
 
.offset8{margin-left:70.94017094017094%;*margin-left:70.83378796144753%}.row-fluid
  
.offset8:first-child{margin-left:68.37606837606839%;*margin-left:68.26968539734497%}.row-fluid
 
.offset7{margin-left:62.393162393162385%;*margin-left:62.28677941443899%}.row-fluid
 
.offset7:first-child{margin-left:59.82905982905982%;*margin-left:59.72267685033642%}.row-fluid
 
.offset6{margin-left:53.84615384615384%;*margin-left:53.739770867430444%}.row-fluid
 
.offset6:first-child{margin-left:51.28205128205128%;*margin-left:51.175668303327875%}.row-fluid
 
.offset5{margin-left:45.299145299145295%;*margin-left:45.1927623204219%}.row-fluid
 
.offset5:first-child{margin-left:42.73504273504273%;*margin-left:42.62865975631933%}.row-fluid
 
.offset4{margin-left:36.75213675213675%;*margin-left:36.645753773413354%}.row-fluid
 
.offset4:first-child{margin-left:34.18803418803419%;*margin-left:34.081651209310785%}.row-fluid
 
.offset3{margin-left:28.205128205128204%;*margin-left:28.0987452264048%}.row-fluid
 
.offset3:first-child{margin-left:25.641025641025642%;*margin-left:25.53464266230224%}.row-fluid
 .offs
 et2{margin-left:19.65811965811966%;*margin-left:19.551736679396257%}.row-fluid 
.offset2:first-child{margin-left:17.094017094017094%;*margin-left:16.98763411529369%}.row-fluid
 
.offset1{margin-left:11.11111111111111%;*margin-left:11.004728132387708%}.row-fluid
 
.offset1:first-child{margin-left:8.547008547008547%;*margin-left:8.440625568285142%}input,textarea,.uneditable-input{margin-left:0}.controls-row
 
[class*="span"]+[class*="span"]{margin-left:30px}input.span12,textarea.span12,.uneditable-input.span12{width:1156px}input.span11,textarea.span11,.uneditable-input.span11{width:1056px}input.span10,textarea.span10,.uneditable-input.span10{width:956px}input.span9,textarea.span9,.uneditable-input.span9{width:856px}input.span8,textarea.span8,.uneditable-input.span8{width:756px}input.span7,textarea.span7,.uneditable-input.span7{width:656px}input.span6,textarea.span6,.uneditable-input.span6{width:556px}input.span5,textarea.span5,.uneditable-input.span5{width:456px}input.span4,textarea.span4,.u
 
neditable-input.span4{width:356px}input.span3,textarea.span3,.uneditable-input.span3{width:256px}input.span2,textarea.span2,.uneditable-input.span2{width:156px}input.span1,textarea.span1,.uneditable-input.span1{width:56px}.thumbnails{margin-left:-30px}.thumbnails>li{margin-left:30px}.row-fluid
 .thumbnails{margin-left:0}}@media(min-width:768px) and 
(max-width:979px){.row{margin-left:-20px;*zoom:1}.row:before,.row:after{display:table;line-height:0;content:""}.row:after{clear:both}[class*="span"]{float:left;min-height:1px;margin-left:20px}.container,.navbar-static-top
 .container,.navbar-fixed-top .container,.navbar-fixed-bottom 
.container{width:724px}.span12{width:724px}.span11{width:662px}.span10{width:600px}.span9{width:538px}.span8{width:476px}.span7{width:414px}.span6{width:352px}.span5{width:290px}.span4{width:228px}.span3{width:166px}.span2{width:104px}.span1{width:42px}.offset12{margin-left:764px}.offset11{margin-left:702px}.offset10{margin-left:640px}.offset9{margin-left:578px}
 
.offset8{margin-left:516px}.offset7{margin-left:454px}.offset6{margin-left:392px}.offset5{margin-left:330px}.offset4{margin-left:268px}.offset3{margin-left:206px}.offset2{margin-left:144px}.offset1{margin-left:82px}.row-fluid{width:100%;*zoom:1}.row-fluid:before,.row-fluid:after{display:table;line-height:0;content:""}.row-fluid:after{clear:both}.row-fluid
 
[class*="span"]{display:block;float:left;width:100%;min-height:30px;margin-left:2.7624309392265194%;*margin-left:2.709239449864817%;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}.row-fluid
 [class*="span"]:first-child{margin-left:0}.row-fluid .controls-row 
[class*="span"]+[class*="span"]{margin-left:2.7624309392265194%}.row-fluid 
.span12{width:100%;*width:99.94680851063829%}.row-fluid 
.span11{width:91.43646408839778%;*width:91.38327259903608%}.row-fluid 
.span10{width:82.87292817679558%;*width:82.81973668743387%}.row-fluid 
.span9{width:74.30939226519337%;*width:74.25620077583166%}.row-fluid .span8{wid
 th:65.74585635359117%;*width:65.69266486422946%}.row-fluid 
.span7{width:57.18232044198895%;*width:57.12912895262725%}.row-fluid 
.span6{width:48.61878453038674%;*width:48.56559304102504%}.row-fluid 
.span5{width:40.05524861878453%;*width:40.00205712942283%}.row-fluid 
.span4{width:31.491712707182323%;*width:31.43852121782062%}.row-fluid 
.span3{width:22.92817679558011%;*width:22.87498530621841%}.row-fluid 
.span2{width:14.3646408839779%;*width:14.311449394616199%}.row-fluid 
.span1{width:5.801104972375691%;*width:5.747913483013988%}.row-fluid 
.offset12{margin-left:105.52486187845304%;*margin-left:105.41847889972962%}.row-fluid
 
.offset12:first-child{margin-left:102.76243093922652%;*margin-left:102.6560479605031%}.row-fluid
 
.offset11{margin-left:96.96132596685082%;*margin-left:96.8549429881274%}.row-fluid
 
.offset11:first-child{margin-left:94.1988950276243%;*margin-left:94.09251204890089%}.row-fluid
 
.offset10{margin-left:88.39779005524862%;*margin-left:88.2914070765252%}.row-fluid
 .offset10:
 
first-child{margin-left:85.6353591160221%;*margin-left:85.52897613729868%}.row-fluid
 
.offset9{margin-left:79.8342541436464%;*margin-left:79.72787116492299%}.row-fluid
 
.offset9:first-child{margin-left:77.07182320441989%;*margin-left:76.96544022569647%}.row-fluid
 
.offset8{margin-left:71.2707182320442%;*margin-left:71.16433525332079%}.row-fluid
 
.offset8:first-child{margin-left:68.50828729281768%;*margin-left:68.40190431409427%}.row-fluid
 
.offset7{margin-left:62.70718232044199%;*margin-left:62.600799341718584%}.row-fluid
 
.offset7:first-child{margin-left:59.94475138121547%;*margin-left:59.838368402492065%}.row-fluid
 
.offset6{margin-left:54.14364640883978%;*margin-left:54.037263430116376%}.row-fluid
 
.offset6:first-child{margin-left:51.38121546961326%;*margin-left:51.27483249088986%}.row-fluid
 
.offset5{margin-left:45.58011049723757%;*margin-left:45.47372751851417%}.row-fluid
 
.offset5:first-child{margin-left:42.81767955801105%;*margin-left:42.71129657928765%}.row-fluid
 .offset4{margin-left:
 37.01657458563536%;*margin-left:36.91019160691196%}.row-fluid 
.offset4:first-child{margin-left:34.25414364640884%;*margin-left:34.14776066768544%}.row-fluid
 
.offset3{margin-left:28.45303867403315%;*margin-left:28.346655695309746%}.row-fluid
 
.offset3:first-child{margin-left:25.69060773480663%;*margin-left:25.584224756083227%}.row-fluid
 
.offset2{margin-left:19.88950276243094%;*margin-left:19.783119783707537%}.row-fluid
 
.offset2:first-child{margin-left:17.12707182320442%;*margin-left:17.02068884448102%}.row-fluid
 
.offset1{margin-left:11.32596685082873%;*margin-left:11.219583872105325%}.row-fluid
 
.offset1:first-child{margin-left:8.56353591160221%;*margin-left:8.457152932878806%}input,textarea,.uneditable-input{margin-left:0}.controls-row
 
[class*="span"]+[class*="span"]{margin-left:20px}input.span12,textarea.span12,.uneditable-input.span12{width:710px}input.span11,textarea.span11,.uneditable-input.span11{width:648px}input.span10,textarea.span10,.uneditable-input.span10{width:586px}input.
 
span9,textarea.span9,.uneditable-input.span9{width:524px}input.span8,textarea.span8,.uneditable-input.span8{width:462px}input.span7,textarea.span7,.uneditable-input.span7{width:400px}input.span6,textarea.span6,.uneditable-input.span6{width:338px}input.span5,textarea.span5,.uneditable-input.span5{width:276px}input.span4,textarea.span4,.uneditable-input.span4{width:214px}input.span3,textarea.span3,.uneditable-input.span3{width:152px}input.span2,textarea.span2,.uneditable-input.span2{width:90px}input.span1,textarea.span1,.uneditable-input.span1{width:28px}}@media(max-width:767px){body{padding-right:20px;padding-left:20px}.navbar-fixed-top,.navbar-fixed-bottom,.navbar-static-top{margin-right:-20px;margin-left:-20px}.container-fluid{padding:0}.dl-horizontal
 dt{float:none;width:auto;clear:none;text-align:left}.dl-horizontal 
dd{margin-left:0}.container{width:auto}.row-fluid{width:100%}.row,.thumbnails{margin-left:0}.thumbnails>li{float:none;margin-left:0}[class*="span"],.uneditable-input[c
 lass*="span"],.row-fluid 
[class*="span"]{display:block;float:none;width:100%;margin-left:0;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}.span12,.row-fluid
 
.span12{width:100%;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}.row-fluid
 
[class*="offset"]:first-child{margin-left:0}.input-large,.input-xlarge,.input-xxlarge,input[class*="span"],select[class*="span"],textarea[class*="span"],.uneditable-input{display:block;width:100%;min-height:30px;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}.input-prepend
 input,.input-append input,.input-prepend input[class*="span"],.input-append 
input[class*="span"]{display:inline-block;width:auto}.controls-row 
[class*="span"]+[class*="span"]{margin-left:0}.modal{position:fixed;top:20px;right:20px;left:20px;width:auto;margin:0}.modal.fade{top:-100px}.modal.fade.in{top:20px}}@media(max-width:480px){.nav-collapse{-webkit-transform:translate3d(0,0,0)}.page-hea
 der h1 
small{display:block;line-height:20px}input[type="checkbox"],input[type="radio"]{border:1px
 solid #ccc}.form-horizontal 
.control-label{float:none;width:auto;padding-top:0;text-align:left}.form-horizontal
 .controls{margin-left:0}.form-horizontal 
.control-list{padding-top:0}.form-horizontal 
.form-actions{padding-right:10px;padding-left:10px}.media .pull-left,.media 
.pull-right{display:block;float:none;margin-bottom:10px}.media-object{margin-right:0;margin-left:0}.modal{top:10px;right:10px;left:10px}.modal-header
 
.close{padding:10px;margin:-10px}.carousel-caption{position:static}}@media(max-width:979px){body{padding-top:0}.navbar-fixed-top,.navbar-fixed-bottom{position:static}.navbar-fixed-top{margin-bottom:20px}.navbar-fixed-bottom{margin-top:20px}.navbar-fixed-top
 .navbar-inner,.navbar-fixed-bottom .navbar-inner{padding:5px}.navbar 
.container{width:auto;padding:0}.navbar 
.brand{padding-right:10px;padding-left:10px;margin:0 0 0 
-5px}.nav-collapse{clear:both}.nav-collapse .nav{fl
 oat:none;margin:0 0 10px}.nav-collapse .nav>li{float:none}.nav-collapse 
.nav>li>a{margin-bottom:2px}.nav-collapse 
.nav>.divider-vertical{display:none}.nav-collapse .nav 
.nav-header{color:#777;text-shadow:none}.nav-collapse .nav>li>a,.nav-collapse 
.dropdown-menu a{padding:9px 
15px;font-weight:bold;color:#777;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}.nav-collapse
 .btn{padding:4px 10px 
4px;font-weight:normal;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.nav-collapse
 .dropdown-menu li+li a{margin-bottom:2px}.nav-collapse 
.nav>li>a:hover,.nav-collapse .dropdown-menu 
a:hover{background-color:#f2f2f2}.navbar-inverse .nav-collapse 
.nav>li>a,.navbar-inverse .nav-collapse .dropdown-menu 
a{color:#999}.navbar-inverse .nav-collapse .nav>li>a:hover,.navbar-inverse 
.nav-collapse .dropdown-menu a:hover{background-color:#111}.nav-collapse.in 
.btn-group{padding:0;margin-top:5px}.nav-collapse 
.dropdown-menu{position:static;top:auto;left:auto;display:none
 ;float:none;max-width:none;padding:0;margin:0 
15px;background-color:transparent;border:0;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0;-webkit-box-shadow:none;-moz-box-shadow:none;box-shadow:none}.nav-collapse
 .open>.dropdown-menu{display:block}.nav-collapse 
.dropdown-menu:before,.nav-collapse 
.dropdown-menu:after{display:none}.nav-collapse .dropdown-menu 
.divider{display:none}.nav-collapse .nav>li>.dropdown-menu:before,.nav-collapse 
.nav>li>.dropdown-menu:after{display:none}.nav-collapse 
.navbar-form,.nav-collapse .navbar-search{float:none;padding:10px 
15px;margin:10px 0;border-top:1px solid #f2f2f2;border-bottom:1px solid 
#f2f2f2;-webkit-box-shadow:inset 0 1px 0 rgba(255,255,255,0.1),0 1px 0 
rgba(255,255,255,0.1);-moz-box-shadow:inset 0 1px 0 rgba(255,255,255,0.1),0 1px 
0 rgba(255,255,255,0.1);box-shadow:inset 0 1px 0 rgba(255,255,255,0.1),0 1px 0 
rgba(255,255,255,0.1)}.navbar-inverse .nav-collapse 
.navbar-form,.navbar-inverse .nav-collapse .navbar-search{border-top
 -color:#111;border-bottom-color:#111}.navbar .nav-collapse 
.nav.pull-right{float:none;margin-left:0}.nav-collapse,.nav-collapse.collapse{height:0;overflow:hidden}.navbar
 .btn-navbar{display:block}.navbar-static 
.navbar-inner{padding-right:10px;padding-left:10px}}@media(min-width:980px){.nav-collapse.collapse{height:auto!important;overflow:visible!important}}

Reply via email to