You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@helix.apache.org by ka...@apache.org on 2013/10/19 01:39:36 UTC

[2/3] git commit: [HELIX-144] Validate StateModelDefinition before adding it to the cluster, rb=14704

[HELIX-144] Validate StateModelDefinition before adding it to the cluster, rb=14704


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

Branch: refs/heads/helix-0.6.2-release
Commit: 202365d1431d273fa7577710be20273479b14142
Parents: b23f983
Author: Kanak Biscuitwala <ka...@apache.org>
Authored: Wed Oct 16 18:06:58 2013 -0700
Committer: Kanak Biscuitwala <ka...@apache.org>
Committed: Fri Oct 18 16:31:40 2013 -0700

----------------------------------------------------------------------
 .../helix/manager/zk/ZKHelixDataAccessor.java   |  18 +-
 .../helix/model/StateModelDefinition.java       |  23 +-
 .../util/StateModelDefinitionValidator.java     | 290 +++++++++++++++++++
 .../helix/model/TestStateModelValidity.java     | 252 ++++++++++++++++
 src/site/markdown/tutorial_state.md             |  79 ++++-
 5 files changed, 642 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/202365d1/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixDataAccessor.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixDataAccessor.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixDataAccessor.java
index 025402d..94e8feb 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixDataAccessor.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixDataAccessor.java
@@ -76,7 +76,18 @@ public class ZKHelixDataAccessor implements HelixDataAccessor, ControllerChangeL
     PropertyType type = key.getType();
     String path = key.getPath();
     int options = constructOptions(type);
-    return _baseDataAccessor.create(path, value.getRecord(), options);
+    boolean success = false;
+    switch (type) {
+    case STATEMODELDEFS:
+      if (value.isValid()) {
+        success = _baseDataAccessor.create(path, value.getRecord(), options);
+      }
+      break;
+    default:
+      success = _baseDataAccessor.create(path, value.getRecord(), options);
+      break;
+    }
+    return success;
   }
 
   @Override
@@ -431,6 +442,11 @@ public class ZKHelixDataAccessor implements HelixDataAccessor, ControllerChangeL
           bucketizedRecords.set(i, childBucketizedRecords);
         }
         break;
+      case STATEMODELDEFS:
+        if (value.isValid()) {
+          records.add(value.getRecord());
+        }
+        break;
       default:
         records.add(value.getRecord());
         break;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/202365d1/helix-core/src/main/java/org/apache/helix/model/StateModelDefinition.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/model/StateModelDefinition.java b/helix-core/src/main/java/org/apache/helix/model/StateModelDefinition.java
index 7f08b6f..e99e173 100644
--- a/helix-core/src/main/java/org/apache/helix/model/StateModelDefinition.java
+++ b/helix-core/src/main/java/org/apache/helix/model/StateModelDefinition.java
@@ -31,7 +31,7 @@ import org.apache.helix.HelixDefinedState;
 import org.apache.helix.HelixProperty;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.model.builder.StateTransitionTableBuilder;
-import org.apache.log4j.Logger;
+import org.apache.helix.model.util.StateModelDefinitionValidator;
 
 /**
  * Describe the state model
@@ -43,7 +43,6 @@ public class StateModelDefinition extends HelixProperty {
     STATE_PRIORITY_LIST
   }
 
-  private static final Logger _logger = Logger.getLogger(StateModelDefinition.class.getName());
   /**
    * state model's initial state
    */
@@ -188,16 +187,7 @@ public class StateModelDefinition extends HelixProperty {
 
   @Override
   public boolean isValid() {
-    if (getInitialState() == null) {
-      _logger.error("State model does not contain init state, statemodel:" + _record.getId());
-      return false;
-    }
-    if (_record.getListField(StateModelDefinitionProperty.STATE_PRIORITY_LIST.toString()) == null) {
-      _logger.error("CurrentState does not contain StatesPriorityList, state model : "
-          + _record.getId());
-      return false;
-    }
-    return true;
+    return StateModelDefinitionValidator.isStateModelDefinitionValid(this);
   }
 
   // TODO move this to model.builder package, refactor StateModelConfigGenerator to use this
@@ -319,6 +309,8 @@ public class StateModelDefinition extends HelixProperty {
      */
     public StateModelDefinition build() {
       ZNRecord record = new ZNRecord(_statemodelName);
+
+      // get sorted state priorities by specified values
       ArrayList<String> statePriorityList = new ArrayList<String>(statesMap.keySet());
       Comparator<? super String> c1 = new Comparator<String>() {
 
@@ -328,8 +320,9 @@ public class StateModelDefinition extends HelixProperty {
         }
       };
       Collections.sort(statePriorityList, c1);
-      ArrayList<Transition> transitionList = new ArrayList<Transition>(transitionMap.keySet());
 
+      // get sorted transition priorities by specified values
+      ArrayList<Transition> transitionList = new ArrayList<Transition>(transitionMap.keySet());
       Comparator<? super Transition> c2 = new Comparator<Transition>() {
         @Override
         public int compare(Transition o1, Transition o2) {
@@ -347,6 +340,8 @@ public class StateModelDefinition extends HelixProperty {
           statePriorityList);
       record.setListField(StateModelDefinitionProperty.STATE_TRANSITION_PRIORITYLIST.toString(),
           transitionPriorityList);
+
+      // compute full paths for next states
       StateTransitionTableBuilder stateTransitionTableBuilder = new StateTransitionTableBuilder();
       Map<String, Map<String, String>> transitionTable =
           stateTransitionTableBuilder.buildTransitionTable(statePriorityList,
@@ -354,6 +349,8 @@ public class StateModelDefinition extends HelixProperty {
       for (String state : transitionTable.keySet()) {
         record.setMapField(state + ".next", transitionTable.get(state));
       }
+
+      // state counts
       for (String state : statePriorityList) {
         HashMap<String, String> metadata = new HashMap<String, String>();
         if (stateConstraintMap.get(state) != null) {

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/202365d1/helix-core/src/main/java/org/apache/helix/model/util/StateModelDefinitionValidator.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/model/util/StateModelDefinitionValidator.java b/helix-core/src/main/java/org/apache/helix/model/util/StateModelDefinitionValidator.java
new file mode 100644
index 0000000..6ecc800
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/model/util/StateModelDefinitionValidator.java
@@ -0,0 +1,290 @@
+package org.apache.helix.model.util;
+
+/*
+ * 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.
+ */
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.model.StateModelDefinition;
+import org.apache.log4j.Logger;
+
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+
+/**
+ * Validator logic for a StateModelDefinition.<br/>
+ * <br/>
+ * Usage:<br/>
+ * StateModelDefinition stateModelDef = ...;<br/>
+ * StateModelDefinitionValidator.isStateModelDefinitionValid(stateModelDef);
+ */
+public class StateModelDefinitionValidator {
+  private static final Logger _logger = Logger.getLogger(StateModelDefinitionValidator.class);
+  private final StateModelDefinition _stateModelDef;
+  private final List<String> _statePriorityList;
+  private final List<String> _transitionPriorityList;
+  private final Set<String> _stateSet;
+
+  /**
+   * Instantiate a validator instance
+   * @param stateModelDef the state model definition to validate
+   */
+  private StateModelDefinitionValidator(StateModelDefinition stateModelDef) {
+    _stateModelDef = stateModelDef;
+    _statePriorityList = stateModelDef.getStatesPriorityList();
+    _transitionPriorityList = stateModelDef.getStateTransitionPriorityList();
+    _stateSet = Sets.newHashSet(_statePriorityList);
+  }
+
+  /**
+   * Check if the StateModelDefinition passes all validation checks
+   * @return true if state model definition is valid, false otherwise
+   */
+  public boolean isStateModelDefinitionValid() {
+    // has a name
+    if (_stateModelDef.getId() == null || _stateModelDef.getId().isEmpty()) {
+      _logger.error("State model does not have a name");
+      return false;
+    }
+
+    // has an initial state
+    if (_stateModelDef.getInitialState() == null || _stateModelDef.getInitialState().isEmpty()) {
+      _logger
+          .error("State model does not contain init state, statemodel:" + _stateModelDef.getId());
+      return false;
+    }
+
+    // has states
+    if (_statePriorityList == null || _statePriorityList.isEmpty()) {
+      _logger.error("CurrentState does not contain StatesPriorityList, state model : "
+          + _stateModelDef.getId());
+      return false;
+    }
+
+    // initial state is a state
+    if (!_stateSet.contains(_stateModelDef.getInitialState())) {
+      _logger.error("Defined states does not include the initial state, state model: "
+          + _stateModelDef.getId());
+      return false;
+    }
+
+    // has a dropped state
+    if (!_stateSet.contains(HelixDefinedState.DROPPED.toString())) {
+      _logger.error("Defined states does not include the DROPPED state, state model: "
+          + _stateModelDef.getId());
+      return false;
+    }
+
+    // make sure individual checks all pass
+    if (!areStateCountsValid() || !areNextStatesValid() || !isTransitionPriorityListValid()
+        || !arePathsValid()) {
+      return false;
+    }
+
+    return true;
+  }
+
+  /**
+   * Check if state counts are properly defined for each state
+   * @return true if state counts valid, false otherwise
+   */
+  private boolean areStateCountsValid() {
+    for (String state : _statePriorityList) {
+      // all states should have a count
+      String count = _stateModelDef.getNumInstancesPerState(state);
+      if (count == null) {
+        _logger.error("State " + state + " needs an upper bound constraint, state model: "
+            + _stateModelDef.getId());
+        return false;
+      }
+
+      // count should be a number, N, or R
+      try {
+        Integer.parseInt(count);
+      } catch (NumberFormatException e) {
+        if (!count.equals("N") && !count.equals("R")) {
+          _logger.error("State " + state + " has invalid count " + count + ", state model: "
+              + _stateModelDef.getId());
+          return false;
+        }
+      }
+    }
+    return true;
+  }
+
+  /**
+   * Check if the state transition priority list is properly formed
+   * @return true if the transition priority list is valid, false otherwise
+   */
+  private boolean isTransitionPriorityListValid() {
+    if (_transitionPriorityList != null) {
+      for (String transition : _transitionPriorityList) {
+        // ensure that transition is of form FROM-TO
+        int index = transition.indexOf('-');
+        int lastIndex = transition.indexOf('-');
+        if (index <= 0 || index >= transition.length() - 1 || index != lastIndex) {
+          _logger.error("Transition " + transition + " is not of the form SRC-DEST, state model: "
+              + _stateModelDef.getId());
+          return false;
+        }
+
+        // from and to states should be valid states
+        String from = transition.substring(0, index);
+        String to = transition.substring(index + 1);
+        if (!_stateSet.contains(from)) {
+          _logger.error("State " + from + " in " + transition
+              + " is not a defined state, state model" + _stateModelDef.getId());
+          return false;
+        }
+        if (!_stateSet.contains(to)) {
+          _logger.error("State " + to + " in " + transition
+              + " is not a defined state, state model: " + _stateModelDef.getId());
+          return false;
+        }
+
+        // the next state for the transition should be the to state
+        if (!to.equals(_stateModelDef.getNextStateForTransition(from, to))) {
+          _logger.error("Transition " + transition + " must have " + to + " as the next state");
+          return false;
+        }
+      }
+    }
+    return true;
+  }
+
+  /**
+   * Check if the "next" states in the state model definition are valid. These check the next values
+   * at a single level. To check full paths, use {@link #arePathsValid()}.
+   * @return true if next states are properly defined, false otherwise
+   */
+  private boolean areNextStatesValid() {
+    for (String state : _statePriorityList) {
+      // all states can reach DROPPED
+      if (!state.equals(HelixDefinedState.DROPPED.toString())
+          && _stateModelDef.getNextStateForTransition(state, HelixDefinedState.DROPPED.toString()) == null) {
+        _logger.error("State " + state + " cannot reach the DROPPED state, state model: "
+            + _stateModelDef.getId());
+        return false;
+      }
+
+      // initial state should reach all states (other than error)
+      if (!state.equals(_stateModelDef.getInitialState())
+          && !state.equals(HelixDefinedState.ERROR.toString())
+          && _stateModelDef.getNextStateForTransition(_stateModelDef.getInitialState(), state) == null) {
+        _logger.error("Initial state " + _stateModelDef.getInitialState()
+            + " should be able to reach all states, state model: " + _stateModelDef.getId());
+        return false;
+      }
+
+      // validate "next" states
+      for (String destState : _statePriorityList) {
+        if (state.equals(destState)) {
+          continue;
+        }
+        // the next state should exist
+        String intermediate = _stateModelDef.getNextStateForTransition(state, destState);
+        if (intermediate != null && !_stateSet.contains(intermediate)) {
+          _logger.error("Intermediate state " + intermediate + " for transition " + state + "-"
+              + destState + " is not a valid state, state model: " + _stateModelDef.getId());
+          return false;
+        }
+
+        // the next state should not allow a self loop
+        if (intermediate != null && intermediate.equals(state)) {
+          _logger.error("Intermediate state " + intermediate + " for transition " + state + "-"
+              + destState + " should never be the from state, state model: "
+              + _stateModelDef.getId());
+          return false;
+        }
+      }
+    }
+    return true;
+  }
+
+  /**
+   * Check that the state model does not have loops or unreachable states and that next states
+   * actually help make progress
+   * @return true if the transitions are valid, false otherwise
+   */
+  private boolean arePathsValid() {
+    // create a map for memoized path checking
+    Map<String, Set<String>> alreadyChecked = Maps.newHashMap();
+    for (String state : _statePriorityList) {
+      alreadyChecked.put(state, new HashSet<String>());
+    }
+
+    // check all pairs for paths
+    for (String from : _statePriorityList) {
+      for (String to : _statePriorityList) {
+        // ignore self transitions
+        if (from.equals(to)) {
+          continue;
+        }
+
+        // see if a path is claimed to exist
+        Set<String> used = Sets.newHashSet(from);
+        String next = _stateModelDef.getNextStateForTransition(from, to);
+        if (next == null) {
+          if (from.equals(_stateModelDef.getInitialState())
+              && !to.equals(HelixDefinedState.ERROR.toString())) {
+            _logger.error("Initial state " + from + " cannot reach " + to + ", state model: "
+                + _stateModelDef.getId());
+            return false;
+          }
+          continue;
+        }
+        // if a path exists, follow it all the way
+        while (!to.equals(next)) {
+          // no need to proceed if this path has already been traversed
+          if (alreadyChecked.get(next).contains(to)) {
+            break;
+          }
+          if (used.contains(next)) {
+            _logger.error("Path from " + from + " to " + to
+                + " contains an infinite loop, state model: " + _stateModelDef.getId());
+            return false;
+          }
+          alreadyChecked.get(next).add(to);
+          used.add(next);
+          next = _stateModelDef.getNextStateForTransition(next, to);
+          if (next == null) {
+            _logger.error("Path from " + from + " to " + to + " is incomplete, state model: "
+                + _stateModelDef.getId());
+            return false;
+          }
+        }
+        alreadyChecked.get(from).add(to);
+      }
+    }
+    return true;
+  }
+
+  /**
+   * Validate a StateModelDefinition instance
+   * @param stateModelDef the state model definition to validate
+   * @return true if the state model definition is valid, false otherwise
+   */
+  public static boolean isStateModelDefinitionValid(StateModelDefinition stateModelDef) {
+    return new StateModelDefinitionValidator(stateModelDef).isStateModelDefinitionValid();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/202365d1/helix-core/src/test/java/org/apache/helix/model/TestStateModelValidity.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/model/TestStateModelValidity.java b/helix-core/src/test/java/org/apache/helix/model/TestStateModelValidity.java
new file mode 100644
index 0000000..2e4d5f0
--- /dev/null
+++ b/helix-core/src/test/java/org/apache/helix/model/TestStateModelValidity.java
@@ -0,0 +1,252 @@
+package org.apache.helix.model;
+
+/*
+ * 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.
+ */
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.ZNRecord;
+import org.apache.helix.model.StateModelDefinition.StateModelDefinitionProperty;
+import org.apache.helix.tools.StateModelConfigGenerator;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+import com.google.common.collect.Lists;
+
+public class TestStateModelValidity {
+  /**
+   * Ensure that state models that we know to be good pass validation
+   */
+  @Test
+  public void testValidModels() {
+    StateModelDefinition masterSlave =
+        new StateModelDefinition(StateModelConfigGenerator.generateConfigForMasterSlave());
+    Assert.assertTrue(masterSlave.isValid());
+
+    StateModelDefinition leaderStandby =
+        new StateModelDefinition(StateModelConfigGenerator.generateConfigForLeaderStandby());
+    Assert.assertTrue(leaderStandby.isValid());
+
+    StateModelDefinition onlineOffline =
+        new StateModelDefinition(StateModelConfigGenerator.generateConfigForOnlineOffline());
+    Assert.assertTrue(onlineOffline.isValid());
+  }
+
+  /**
+   * Ensure that Helix responds negatively if DROPPED is not specified
+   */
+  @Test
+  public void testNoDroppedState() {
+    StateModelDefinition stateModel =
+        new StateModelDefinition.Builder("stateModel").initialState("OFFLINE").addState("OFFLINE")
+            .addState("MASTER").addState("SLAVE").addTransition("OFFLINE", "SLAVE")
+            .addTransition("SLAVE", "MASTER").addTransition("MASTER", "SLAVE")
+            .addTransition("SLAVE", "OFFLINE").build();
+    Assert.assertFalse(stateModel.isValid());
+  }
+
+  /**
+   * Ensure that Helix can catch when a state doesn't have a path to DROPPED
+   */
+  @Test
+  public void testNoPathToDropped() {
+    StateModelDefinition stateModel =
+        new StateModelDefinition.Builder("stateModel").initialState("OFFLINE").addState("OFFLINE")
+            .addState("MASTER").addState("SLAVE").addState("DROPPED")
+            .addTransition("OFFLINE", "SLAVE").addTransition("SLAVE", "MASTER")
+            .addTransition("SLAVE", "OFFLINE").addTransition("OFFLINE", "DROPPED").build();
+    Assert.assertFalse(stateModel.isValid());
+
+    // now see that adding MASTER-DROPPED fixes the problem
+    stateModel =
+        new StateModelDefinition.Builder("stateModel").initialState("OFFLINE").addState("OFFLINE")
+            .addState("MASTER").addState("SLAVE").addState("DROPPED")
+            .addTransition("OFFLINE", "SLAVE").addTransition("SLAVE", "MASTER")
+            .addTransition("SLAVE", "OFFLINE").addTransition("OFFLINE", "DROPPED")
+            .addTransition("MASTER", "DROPPED").build();
+    Assert.assertTrue(stateModel.isValid());
+  }
+
+  /**
+   * The initial state should be added as a state, otherwise validation check should fail
+   */
+  @Test
+  public void testInitialStateIsNotState() {
+    StateModelDefinition stateModel =
+        new StateModelDefinition.Builder("stateModel").initialState("OFFLINE").addState("MASTER")
+            .addState("SLAVE").addState("DROPPED").addTransition("OFFLINE", "SLAVE")
+            .addTransition("SLAVE", "MASTER").addTransition("SLAVE", "OFFLINE")
+            .addTransition("OFFLINE", "DROPPED").addTransition("MASTER", "SLAVE").build();
+    Assert.assertFalse(stateModel.isValid());
+  }
+
+  /**
+   * There should be an initial state, otherwise instantiation should fail
+   */
+  @Test
+  public void testNoInitialState() {
+    try {
+      new StateModelDefinition.Builder("stateModel").addState("OFFLINE").addState("MASTER")
+          .addState("SLAVE").addState("DROPPED").addTransition("OFFLINE", "SLAVE")
+          .addTransition("SLAVE", "MASTER").addTransition("SLAVE", "OFFLINE")
+          .addTransition("OFFLINE", "DROPPED").addTransition("MASTER", "SLAVE").build();
+      Assert.fail("StateModelDefinition creation should fail if no initial state");
+    } catch (IllegalArgumentException e) {
+    }
+  }
+
+  /**
+   * SRC and DEST in a transition SRC-TEST must be valid states
+   */
+  @Test
+  public void testTransitionsWithInvalidStates() {
+    // invalid to state
+    StateModelDefinition stateModel =
+        new StateModelDefinition.Builder("stateModel").initialState("OFFLINE").addState("OFFLINE")
+            .addState("MASTER").addState("SLAVE").addState("DROPPED")
+            .addTransition("OFFLINE", "SLAVE").addTransition("SLAVE", "MASTER")
+            .addTransition("SLAVE", "OFFLINE").addTransition("OFFLINE", "DROPPED")
+            .addTransition("MASTER", "SLAVE").addTransition("OFFLINE", "INVALID").build();
+    Assert.assertFalse(stateModel.isValid());
+
+    // invalid from state
+    stateModel =
+        new StateModelDefinition.Builder("stateModel").initialState("OFFLINE").addState("OFFLINE")
+            .addState("MASTER").addState("SLAVE").addState("DROPPED")
+            .addTransition("OFFLINE", "SLAVE").addTransition("SLAVE", "MASTER")
+            .addTransition("SLAVE", "OFFLINE").addTransition("OFFLINE", "DROPPED")
+            .addTransition("MASTER", "SLAVE").addTransition("INVALID", "MASTER").build();
+    Assert.assertFalse(stateModel.isValid());
+  }
+
+  /**
+   * The initial state should be able to reach all states, should fail validation otherwise
+   */
+  @Test
+  public void testUnreachableState() {
+    StateModelDefinition stateModel =
+        new StateModelDefinition.Builder("stateModel").initialState("OFFLINE").addState("OFFLINE")
+            .addState("MASTER").addState("SLAVE").addState("DROPPED")
+            .addTransition("OFFLINE", "SLAVE").addTransition("SLAVE", "OFFLINE")
+            .addTransition("OFFLINE", "DROPPED").addTransition("MASTER", "SLAVE")
+            .addTransition("MASTER", "DROPPED").build();
+    Assert.assertFalse(stateModel.isValid());
+  }
+
+  /**
+   * The validator should fail on any detected infinite loops
+   */
+  @Test
+  public void testLoopInStateModel() {
+    // create an infinite loop ONE --> TWO --> ONE
+    ZNRecord record = new ZNRecord("MasterSlave");
+    record.setSimpleField(StateModelDefinitionProperty.INITIAL_STATE.toString(), "OFFLINE");
+    List<String> statePriorityList =
+        Lists.newArrayList("ONE", "TWO", "THREE", "OFFLINE", "DROPPED", "ERROR");
+    record.setListField(StateModelDefinitionProperty.STATE_PRIORITY_LIST.toString(),
+        statePriorityList);
+    for (String state : statePriorityList) {
+      String key = state + ".meta";
+      Map<String, String> metadata = new HashMap<String, String>();
+      metadata.put("count", "-1");
+      record.setMapField(key, metadata);
+    }
+    for (String state : statePriorityList) {
+      String key = state + ".next";
+      if (state.equals("ONE")) {
+        Map<String, String> metadata = new HashMap<String, String>();
+        metadata.put("THREE", "TWO");
+        metadata.put("TWO", "TWO");
+        metadata.put("OFFLINE", "OFFLINE");
+        metadata.put("DROPPED", "DROPPED");
+        record.setMapField(key, metadata);
+      } else if (state.equals("TWO")) {
+        Map<String, String> metadata = new HashMap<String, String>();
+        metadata.put("THREE", "ONE");
+        metadata.put("OFFLINE", "OFFLINE");
+        metadata.put("DROPPED", "OFFLINE");
+        record.setMapField(key, metadata);
+      } else if (state.equals("THREE")) {
+        Map<String, String> metadata = new HashMap<String, String>();
+        metadata.put("OFFLINE", "OFFLINE");
+        metadata.put("DROPPED", "OFFLINE");
+        record.setMapField(key, metadata);
+      } else if (state.equals("OFFLINE")) {
+        Map<String, String> metadata = new HashMap<String, String>();
+        metadata.put("ONE", "ONE");
+        metadata.put("TWO", "TWO");
+        metadata.put("THREE", "THREE");
+        metadata.put("DROPPED", "DROPPED");
+        record.setMapField(key, metadata);
+      } else if (state.equals("ERROR")) {
+        Map<String, String> metadata = new HashMap<String, String>();
+        metadata.put("OFFLINE", "OFFLINE");
+        record.setMapField(key, metadata);
+      }
+    }
+    List<String> stateTransitionPriorityList = new ArrayList<String>();
+    record.setListField(StateModelDefinitionProperty.STATE_TRANSITION_PRIORITYLIST.toString(),
+        stateTransitionPriorityList);
+
+    StateModelDefinition stateModel = new StateModelDefinition(record);
+    Assert.assertFalse(stateModel.isValid());
+  }
+
+  /**
+   * This is the example used on the website, so this must work
+   */
+  @Test
+  public void testBasic() {
+    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();
+
+    Assert.assertTrue(stateModel.isValid());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/202365d1/src/site/markdown/tutorial_state.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/tutorial_state.md b/src/site/markdown/tutorial_state.md
index cb51be9..e9a79d3 100644
--- a/src/site/markdown/tutorial_state.md
+++ b/src/site/markdown/tutorial_state.md
@@ -28,15 +28,17 @@ Every resource that is added should be configured to use a state model that gove
 
 ### MASTER-SLAVE
 
-* Has 3 states: OFFLINE, SLAVE, MASTER
-* Maximum # of masters: 1
-* Slaves are based on the replication factor. Replication factor can be specified while adding the resource
+* 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
@@ -44,17 +46,82 @@ Every resource that is added should be configured to use a state model that gove
 In addition to the state machine configuration, one can specify the constraints of states and transitions.
 
 For example, one can say:
+
 * MASTER:1
- Maximum number of replicas in MASTER state at any time is 1
+<br/>Maximum number of replicas in MASTER state at any time is 1
 
 * OFFLINE-SLAVE:5 
-Maximum number of OFFLINE-SLAVE transitions that can happen concurrently in the system is 5 in this example.
+<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.
+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
+
+### 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());
+```
+
+