You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by zh...@apache.org on 2020/07/28 01:25:35 UTC

[hbase] branch master updated: HBASE-24770 Reimplement the Constraints API and revisit the IA annotations on related classes (#2140)

This is an automated email from the ASF dual-hosted git repository.

zhangduo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hbase.git


The following commit(s) were added to refs/heads/master by this push:
     new 7e6e7a7  HBASE-24770 Reimplement the Constraints API and revisit the IA annotations on related classes (#2140)
7e6e7a7 is described below

commit 7e6e7a7051f71c4f554c53d756538cb753cdfc66
Author: Duo Zhang <zh...@apache.org>
AuthorDate: Tue Jul 28 09:24:55 2020 +0800

    HBASE-24770 Reimplement the Constraints API and revisit the IA annotations on related classes (#2140)
    
    Signed-off-by: stack <st...@apache.org>
---
 .../hbase/client/TableDescriptorBuilder.java       |  20 +-
 .../apache/hadoop/hbase/constraint/Constraint.java |  78 ++-
 .../hadoop/hbase/constraint/Constraints.java       | 604 +++++++--------------
 .../hadoop/hbase/constraint/package-info.java      | 465 ++++++++--------
 .../hadoop/hbase/constraint/TestConstraint.java    |  95 ++--
 .../hadoop/hbase/constraint/TestConstraints.java   | 136 ++---
 6 files changed, 609 insertions(+), 789 deletions(-)

diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java
index e929811..bf591a1 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java
@@ -32,11 +32,11 @@ import java.util.Optional;
 import java.util.Set;
 import java.util.TreeMap;
 import java.util.TreeSet;
+import java.util.function.BiPredicate;
 import java.util.function.Function;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 import java.util.stream.Collectors;
-
 import org.apache.hadoop.hbase.Coprocessor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
@@ -380,6 +380,10 @@ public class TableDescriptorBuilder {
     return this;
   }
 
+  public boolean hasCoprocessor(String classNameToMatch) {
+    return desc.hasCoprocessor(classNameToMatch);
+  }
+
   public TableDescriptorBuilder setColumnFamily(final ColumnFamilyDescriptor family) {
     desc.setColumnFamily(Objects.requireNonNull(family));
     return this;
@@ -411,6 +415,16 @@ public class TableDescriptorBuilder {
     return this;
   }
 
+  public TableDescriptorBuilder removeValue(BiPredicate<Bytes, Bytes> predicate) {
+    List<Bytes> toRemove =
+      desc.getValues().entrySet().stream().filter(e -> predicate.test(e.getKey(), e.getValue()))
+        .map(Map.Entry::getKey).collect(Collectors.toList());
+    for (Bytes key : toRemove) {
+      removeValue(key);
+    }
+    return this;
+  }
+
   public TableDescriptorBuilder removeColumnFamily(final byte[] name) {
     desc.removeColumnFamily(name);
     return this;
@@ -531,6 +545,10 @@ public class TableDescriptorBuilder {
     return this;
   }
 
+  public String getValue(String key) {
+    return desc.getValue(key);
+  }
+
   /**
    * Sets replication scope all & only the columns already in the builder. Columns added later won't
    * be backfilled with replication scope.
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/constraint/Constraint.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/constraint/Constraint.java
index 4a63ec1..c0c4b60 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/constraint/Constraint.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/constraint/Constraint.java
@@ -22,45 +22,39 @@ import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.hbase.client.Put;
 
 /**
- * Apply a {@link Constraint} (in traditional database terminology) to a HTable.
- * Any number of {@link Constraint Constraints} can be added to the table, in
- * any order.
- * <p>
+ * Apply a {@link Constraint} (in traditional database terminology) to a Table. Any number of
+ * {@link Constraint Constraints} can be added to the table, in any order.
+ * <p/>
  * A {@link Constraint} must be added to a table before the table is loaded via
- * {@link Constraints#add(org.apache.hadoop.hbase.HTableDescriptor, Class[])} or
- * {@link Constraints#add(org.apache.hadoop.hbase.HTableDescriptor,
- * org.apache.hadoop.hbase.util.Pair...)}
- * (if you want to add a configuration with the {@link Constraint}). Constraints
- * will be run in the order that they are added. Further, a Constraint will be
- * configured before it is run (on load).
- * <p>
- * See {@link Constraints#enableConstraint(org.apache.hadoop.hbase.HTableDescriptor, Class)} and
- * {@link Constraints#disableConstraint(org.apache.hadoop.hbase.HTableDescriptor, Class)} for
- * enabling/disabling of a given {@link Constraint} after it has been added.
- * <p>
+ * {@link Constraints#add(org.apache.hadoop.hbase.client.TableDescriptorBuilder, Class...)} or
+ * {@link Constraints#add(org.apache.hadoop.hbase.client.TableDescriptorBuilder, org.apache.hadoop.hbase.util.Pair...)}
+ * (if you want to add a configuration with the {@link Constraint}). Constraints will be run in the
+ * order that they are added. Further, a Constraint will be configured before it is run (on load).
+ * <p/>
+ * See
+ * {@link Constraints#enableConstraint(org.apache.hadoop.hbase.client.TableDescriptorBuilder, Class)}
+ * and
+ * {@link Constraints#disableConstraint(org.apache.hadoop.hbase.client.TableDescriptorBuilder, Class)}
+ * for enabling/disabling of a given {@link Constraint} after it has been added.
+ * <p/>
  * If a {@link Put} is invalid, the Constraint should throw some sort of
- * {@link org.apache.hadoop.hbase.constraint.ConstraintException}, indicating
- * that the {@link Put} has failed. When
- * this exception is thrown, not further retries of the {@link Put} are
- * attempted nor are any other {@link Constraint Constraints} attempted (the
- * {@link Put} is clearly not valid). Therefore, there are performance
- * implications in the order in which {@link BaseConstraint Constraints} are
- * specified.
- * <p>
+ * {@link org.apache.hadoop.hbase.constraint.ConstraintException}, indicating that the {@link Put}
+ * has failed. When this exception is thrown, not further retries of the {@link Put} are attempted
+ * nor are any other {@link Constraint Constraints} attempted (the {@link Put} is clearly not
+ * valid). Therefore, there are performance implications in the order in which {@link BaseConstraint
+ * Constraints} are specified.
+ * <p/>
  * If a {@link Constraint} fails to fail the {@link Put} via a
- * {@link org.apache.hadoop.hbase.constraint.ConstraintException}, but instead
- * throws a {@link RuntimeException},
- * the entire constraint processing mechanism ({@link ConstraintProcessor}) will
- * be unloaded from the table. This ensures that the region server is still
- * functional, but no more {@link Put Puts} will be checked via
- * {@link Constraint Constraints}.
- * <p>
- * Further, {@link Constraint Constraints} should probably not be used to
- * enforce cross-table references as it will cause tremendous write slowdowns,
- * but it is possible.
- * <p>
+ * {@link org.apache.hadoop.hbase.constraint.ConstraintException}, but instead throws a
+ * {@link RuntimeException}, the entire constraint processing mechanism
+ * ({@link ConstraintProcessor}) will be unloaded from the table. This ensures that the region
+ * server is still functional, but no more {@link Put Puts} will be checked via {@link Constraint
+ * Constraints}.
+ * <p/>
+ * Further, {@link Constraint Constraints} should probably not be used to enforce cross-table
+ * references as it will cause tremendous write slowdowns, but it is possible.
+ * <p/>
  * NOTE: Implementing classes must have a nullary (no-args) constructor
- *
  * @see BaseConstraint
  * @see Constraints
  */
@@ -68,15 +62,13 @@ import org.apache.hadoop.hbase.client.Put;
 public interface Constraint extends Configurable {
 
   /**
-   * Check a {@link Put} to ensure it is valid for the table. If the {@link Put}
-   * is valid, then just return from the method. Otherwise, throw an
-   * {@link Exception} specifying what happened. This {@link Exception} is
-   * propagated back to the client so you can see what caused the {@link Put} to
-   * fail.
+   * Check a {@link Put} to ensure it is valid for the table. If the {@link Put} is valid, then just
+   * return from the method. Otherwise, throw an {@link Exception} specifying what happened. This
+   * {@link Exception} is propagated back to the client so you can see what caused the {@link Put}
+   * to fail.
    * @param p {@link Put} to check
-   * @throws org.apache.hadoop.hbase.constraint.ConstraintException when the
-   * {@link Put} does not match the
-   *         constraint.
+   * @throws org.apache.hadoop.hbase.constraint.ConstraintException when the {@link Put} does not
+   *           match the constraint.
    */
   void check(Put p) throws ConstraintException;
 
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/constraint/Constraints.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/constraint/Constraints.java
index 1381136..ee942db 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/constraint/Constraints.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/constraint/Constraints.java
@@ -29,26 +29,24 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.regex.Pattern;
-
-import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Utilities for adding/removing constraints from a table.
- * <p>
- * Constraints can be added on table load time, via the {@link HTableDescriptor}.
- * <p>
- * NOTE: this class is NOT thread safe. Concurrent setting/enabling/disabling of
- * constraints can cause constraints to be run at incorrect times or not at all.
+ * <p/>
+ * Since {@link TableDescriptor} is immutable now, you should use {@link TableDescriptorBuilder}.
+ * And when disabling or removing constraints, you could use
+ * {@link TableDescriptorBuilder#newBuilder(TableDescriptor)} to clone the old
+ * {@link TableDescriptor} and then pass it the below methods.
  */
-@InterfaceAudience.Private
+@InterfaceAudience.Public
 public final class Constraints {
   private static final int DEFAULT_PRIORITY = -1;
 
@@ -57,8 +55,8 @@ public final class Constraints {
 
   private static final Logger LOG = LoggerFactory.getLogger(Constraints.class);
   private static final String CONSTRAINT_HTD_KEY_PREFIX = "constraint $";
-  private static final Pattern CONSTRAINT_HTD_ATTR_KEY_PATTERN = Pattern
-      .compile(CONSTRAINT_HTD_KEY_PREFIX, Pattern.LITERAL);
+  private static final Pattern CONSTRAINT_HTD_ATTR_KEY_PATTERN =
+    Pattern.compile(CONSTRAINT_HTD_KEY_PREFIX, Pattern.LITERAL);
 
   // configuration key for if the constraint is enabled
   private static final String ENABLED_KEY = "_ENABLED";
@@ -74,112 +72,57 @@ public final class Constraints {
 
   /**
    * Enable constraints on a table.
-   * <p>
-   * Currently, if you attempt to add a constraint to the table, then
-   * Constraints will automatically be turned on.
-   * 
-   * @param desc
-   *          table description to add the processor
-   * @throws IOException
-   *           If the {@link ConstraintProcessor} CP couldn't be added to the
-   *           table.
+   * <p/>
+   * Currently, if you attempt to add a constraint to the table, then Constraints will automatically
+   * be turned on.
    */
-  public static void enable(HTableDescriptor desc) throws IOException {
-    // if the CP has already been loaded, do nothing
-    String clazz = ConstraintProcessor.class.getName();
-    if (desc.hasCoprocessor(clazz)) {
-      return;
+  public static TableDescriptorBuilder enable(TableDescriptorBuilder builder) throws IOException {
+    if (!builder.hasCoprocessor(ConstraintProcessor.class.getName())) {
+      builder.setCoprocessor(ConstraintProcessor.class.getName());
     }
-
-    // add the constrain processor CP to the table
-    desc.addCoprocessor(clazz);
-  }
-
-  public static void enable(TableDescriptorBuilder.ModifyableTableDescriptor desc)
-      throws IOException {
-    // if the CP has already been loaded, do nothing
-    String clazz = ConstraintProcessor.class.getName();
-    if (desc.hasCoprocessor(clazz)) {
-      return;
-    }
-
-    // add the constrain processor CP to the table
-    desc.setCoprocessor(clazz);
+    return builder;
   }
 
   /**
-   * Turn off processing constraints for a given table, even if constraints have
-   * been turned on or added.
-   * 
-   * @param desc
-   *          {@link HTableDescriptor} where to disable {@link Constraint
-   *          Constraints}.
+   * Turn off processing constraints for a given table, even if constraints have been turned on or
+   * added.
    */
-  public static void disable(HTableDescriptor desc) {
-    desc.removeCoprocessor(ConstraintProcessor.class.getName());
-  }
-
-  public static void disable(TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor) {
-    tableDescriptor.removeCoprocessor(ConstraintProcessor.class.getName());
+  public static TableDescriptorBuilder disable(TableDescriptorBuilder builder) throws IOException {
+    return builder.removeCoprocessor(ConstraintProcessor.class.getName());
   }
 
   /**
-   * Remove all {@link Constraint Constraints} that have been added to the table
-   * and turn off the constraint processing.
-   * <p>
-   * All {@link Configuration Configurations} and their associated
-   * {@link Constraint} are removed.
-   * 
-   * @param desc
-   *          {@link HTableDescriptor} to remove {@link Constraint Constraints}
-   *          from.
+   * Remove all {@link Constraint Constraints} that have been added to the table and turn off the
+   * constraint processing.
+   * <p/>
+   * All {@link Configuration Configurations} and their associated {@link Constraint} are removed.
    */
-  public static void remove(HTableDescriptor desc) {
-    // disable constraints
-    disable(desc);
-
-    // remove all the constraint settings
-    List<Bytes> keys = new ArrayList<>();
-    // loop through all the key, values looking for constraints
-    for (Map.Entry<Bytes, Bytes> e : desc
-        .getValues().entrySet()) {
-      String key = Bytes.toString((e.getKey().get()));
-      String[] className = CONSTRAINT_HTD_ATTR_KEY_PATTERN.split(key);
-      if (className.length == 2) {
-        keys.add(e.getKey());
-      }
-    }
-    // now remove all the keys we found
-    for (Bytes key : keys) {
-      desc.remove(key);
-    }
+  public static TableDescriptorBuilder remove(TableDescriptorBuilder builder) throws IOException {
+    disable(builder);
+    return builder
+      .removeValue((k, v) -> CONSTRAINT_HTD_ATTR_KEY_PATTERN.split(k.toString()).length == 2);
   }
 
   /**
    * Check to see if the Constraint is currently set.
-   * 
-   * @param desc
-   *          {@link HTableDescriptor} to check
-   * @param clazz
-   *          {@link Constraint} class to check for.
-   * @return <tt>true</tt> if the {@link Constraint} is present, even if it is
-   *         disabled. <tt>false</tt> otherwise.
+   * @param desc {@link TableDescriptor} to check
+   * @param clazz {@link Constraint} class to check for.
+   * @return <tt>true</tt> if the {@link Constraint} is present, even if it is disabled.
+   *         <tt>false</tt> otherwise.
    */
-  public static boolean has(HTableDescriptor desc,
-      Class<? extends Constraint> clazz) {
+  public static boolean has(TableDescriptor desc, Class<? extends Constraint> clazz) {
     return getKeyValueForClass(desc, clazz) != null;
   }
 
   /**
    * Get the kv {@link Entry} in the descriptor for the specified class
-   * 
-   * @param desc {@link HTableDescriptor} to read
+   * @param desc {@link TableDescriptor} to read
    * @param clazz To search for
-   * @return The {@link Pair} of {@literal <key, value>} in the table, if that class is
-   *         present. {@code NULL} otherwise.
+   * @return The {@link Pair} of {@literal <key, value>} in the table, if that class is present.
+   *         {@code null} otherwise.
    */
-  private static Pair<String, String> getKeyValueForClass(
-      HTableDescriptor desc, Class<? extends Constraint> clazz) {
+  private static Pair<String, String> getKeyValueForClass(TableDescriptor desc,
+    Class<? extends Constraint> clazz) {
     // get the serialized version of the constraint
     String key = serializeConstraintClass(clazz);
     String value = desc.getValue(key);
@@ -187,179 +130,126 @@ public final class Constraints {
     return value == null ? null : new Pair<>(key, value);
   }
 
-  private static Pair<String, String> getKeyValueForClass(
-      TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor,
-      Class<? extends Constraint> clazz) {
+  /**
+   * Get the kv {@link Entry} in the descriptor builder for the specified class
+   * @param builder {@link TableDescriptorBuilder} to read
+   * @param clazz To search for
+   * @return The {@link Pair} of {@literal <key, value>} in the table, if that class is present.
+   *         {@code null} otherwise.
+   */
+  private static Pair<String, String> getKeyValueForClass(TableDescriptorBuilder builder,
+    Class<? extends Constraint> clazz) {
     // get the serialized version of the constraint
     String key = serializeConstraintClass(clazz);
-    String value = tableDescriptor.getValue(key);
+    String value = builder.getValue(key);
 
     return value == null ? null : new Pair<>(key, value);
   }
 
   /**
    * Add configuration-less constraints to the table.
-   * <p>
-   * This will overwrite any configuration associated with the previous
-   * constraint of the same class.
-   * <p>
-   * Each constraint, when added to the table, will have a specific priority,
-   * dictating the order in which the {@link Constraint} will be run. A
-   * {@link Constraint} earlier in the list will be run before those later in
-   * the list. The same logic applies between two Constraints over time (earlier
-   * added is run first on the regionserver).
-   * 
-   * @param desc
-   *          {@link HTableDescriptor} to add {@link Constraint Constraints}
-   * @param constraints
-   *          {@link Constraint Constraints} to add. All constraints are
-   *          considered automatically enabled on add
-   * @throws IOException
-   *           If constraint could not be serialized/added to table
-   */
-  public static void add(HTableDescriptor desc,
-      Class<? extends Constraint>... constraints) throws IOException {
-    // make sure constraints are enabled
-    enable(desc);
-    long priority = getNextPriority(desc);
-
-    // store each constraint
-    for (Class<? extends Constraint> clazz : constraints) {
-      addConstraint(desc, clazz, null, priority++);
-    }
-    updateLatestPriority(desc, priority);
-  }
-
-  /**
-   * Add configuration-less constraints to the table.
-   * <p>
-   * This will overwrite any configuration associated with the previous
-   * constraint of the same class.
-   * <p>
-   * Each constraint, when added to the table, will have a specific priority,
-   * dictating the order in which the {@link Constraint} will be run. A
-   * {@link Constraint} earlier in the list will be run before those later in
-   * the list. The same logic applies between two Constraints over time (earlier
-   * added is run first on the regionserver).
-   *
-   * @param tableDescriptor TableDescriptorBuilder.ModifyableTableDescriptor
-   *   to add {@link Constraint}
-   * @param constraints {@link Constraint} to add. All constraints are
-   *   considered automatically enabled on add
+   * <p/>
+   * This will overwrite any configuration associated with the previous constraint of the same
+   * class.
+   * <p/>
+   * Each constraint, when added to the table, will have a specific priority, dictating the order in
+   * which the {@link Constraint} will be run. A {@link Constraint} earlier in the list will be run
+   * before those later in the list. The same logic applies between two Constraints over time
+   * (earlier added is run first on the regionserver).
+   * @param builder {@link TableDescriptorBuilder} to add a {@link Constraint}
+   * @param constraints {@link Constraint Constraints} to add. All constraints are considered
+   *          automatically enabled on add
    * @throws IOException If constraint could not be serialized/added to table
    */
-  public static void add(TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor,
+  @SafeVarargs
+  public static TableDescriptorBuilder add(TableDescriptorBuilder builder,
     Class<? extends Constraint>... constraints) throws IOException {
     // make sure constraints are enabled
-    enable(tableDescriptor);
-    long priority = getNextPriority(tableDescriptor);
+    enable(builder);
+    long priority = getNextPriority(builder);
 
     // store each constraint
     for (Class<? extends Constraint> clazz : constraints) {
-      writeConstraint(tableDescriptor, serializeConstraintClass(clazz),
-        configure(null, true, priority));
+      addConstraint(builder, clazz, null, priority++);
     }
-    updateLatestPriority(tableDescriptor, priority);
+    return updateLatestPriority(builder, priority);
   }
 
   /**
    * Add constraints and their associated configurations to the table.
    * <p>
-   * Adding the same constraint class twice will overwrite the first
-   * constraint's configuration
+   * Adding the same constraint class twice will overwrite the first constraint's configuration
    * <p>
-   * Each constraint, when added to the table, will have a specific priority,
-   * dictating the order in which the {@link Constraint} will be run. A
-   * {@link Constraint} earlier in the list will be run before those later in
-   * the list. The same logic applies between two Constraints over time (earlier
-   * added is run first on the regionserver).
-   * 
-   * @param desc
-   *          {@link HTableDescriptor} to add a {@link Constraint}
-   * @param constraints
-   *          {@link Pair} of a {@link Constraint} and its associated
-   *          {@link Configuration}. The Constraint will be configured on load
-   *          with the specified configuration.All constraints are considered
-   *          automatically enabled on add
-   * @throws IOException
-   *           if any constraint could not be deserialized. Assumes if 1
-   *           constraint is not loaded properly, something has gone terribly
-   *           wrong and that all constraints need to be enforced.
+   * Each constraint, when added to the table, will have a specific priority, dictating the order in
+   * which the {@link Constraint} will be run. A {@link Constraint} earlier in the list will be run
+   * before those later in the list. The same logic applies between two Constraints over time
+   * (earlier added is run first on the regionserver).
+   * @param builder {@link TableDescriptorBuilder} to add a {@link Constraint}
+   * @param constraints {@link Pair} of a {@link Constraint} and its associated
+   *          {@link Configuration}. The Constraint will be configured on load with the specified
+   *          configuration.All constraints are considered automatically enabled on add
+   * @throws IOException if any constraint could not be deserialized. Assumes if 1 constraint is not
+   *           loaded properly, something has gone terribly wrong and that all constraints need to
+   *           be enforced.
    */
-  public static void add(HTableDescriptor desc,
-      Pair<Class<? extends Constraint>, Configuration>... constraints)
-      throws IOException {
-    enable(desc);
-    long priority = getNextPriority(desc);
+  @SafeVarargs
+  public static TableDescriptorBuilder add(TableDescriptorBuilder builder,
+    Pair<Class<? extends Constraint>, Configuration>... constraints) throws IOException {
+    enable(builder);
+    long priority = getNextPriority(builder);
     for (Pair<Class<? extends Constraint>, Configuration> pair : constraints) {
-      addConstraint(desc, pair.getFirst(), pair.getSecond(), priority++);
+      addConstraint(builder, pair.getFirst(), pair.getSecond(), priority++);
     }
-    updateLatestPriority(desc, priority);
+    return updateLatestPriority(builder, priority);
   }
 
   /**
    * Add a {@link Constraint} to the table with the given configuration
-   * <p>
-   * Each constraint, when added to the table, will have a specific priority,
-   * dictating the order in which the {@link Constraint} will be run. A
-   * {@link Constraint} added will run on the regionserver before those added to
-   * the {@link HTableDescriptor} later.
-   * 
-   * @param desc
-   *          table descriptor to the constraint to
-   * @param constraint
-   *          to be added
-   * @param conf
-   *          configuration associated with the constraint
-   * @throws IOException
-   *           if any constraint could not be deserialized. Assumes if 1
-   *           constraint is not loaded properly, something has gone terribly
-   *           wrong and that all constraints need to be enforced.
+   * <p/>
+   * Each constraint, when added to the table, will have a specific priority, dictating the order in
+   * which the {@link Constraint} will be run. A {@link Constraint} added will run on the
+   * regionserver before those added to the {@link TableDescriptorBuilder} later.
+   * @param builder {@link TableDescriptorBuilder} to add a {@link Constraint}
+   * @param constraint to be added
+   * @param conf configuration associated with the constraint
+   * @throws IOException if any constraint could not be deserialized. Assumes if 1 constraint is not
+   *           loaded properly, something has gone terribly wrong and that all constraints need to
+   *           be enforced.
    */
-  public static void add(HTableDescriptor desc,
-      Class<? extends Constraint> constraint, Configuration conf)
-      throws IOException {
-    enable(desc);
-    long priority = getNextPriority(desc);
-    addConstraint(desc, constraint, conf, priority++);
-
-    updateLatestPriority(desc, priority);
+  public static TableDescriptorBuilder add(TableDescriptorBuilder builder,
+    Class<? extends Constraint> constraint, Configuration conf) throws IOException {
+    enable(builder);
+    long priority = getNextPriority(builder);
+    addConstraint(builder, constraint, conf, priority++);
+
+    return updateLatestPriority(builder, priority);
   }
 
   /**
    * Write the raw constraint and configuration to the descriptor.
-   * <p>
-   * This method takes care of creating a new configuration based on the passed
-   * in configuration and then updating that with enabled and priority of the
-   * constraint.
-   * <p>
+   * <p/>
+   * This method takes care of creating a new configuration based on the passed in configuration and
+   * then updating that with enabled and priority of the constraint.
+   * <p/>
    * When a constraint is added, it is automatically enabled.
    */
-  private static void addConstraint(HTableDescriptor desc,
-      Class<? extends Constraint> clazz, Configuration conf, long priority)
-      throws IOException {
-    writeConstraint(desc, serializeConstraintClass(clazz),
-        configure(conf, true, priority));
+  private static TableDescriptorBuilder addConstraint(TableDescriptorBuilder builder,
+    Class<? extends Constraint> clazz, Configuration conf, long priority) throws IOException {
+    return writeConstraint(builder, serializeConstraintClass(clazz),
+      configure(conf, true, priority));
   }
 
   /**
-   * Setup the configuration for a constraint as to whether it is enabled and
-   * its priority
-   * 
-   * @param conf
-   *          on which to base the new configuration
-   * @param enabled
-   *          <tt>true</tt> if it should be run
-   * @param priority
-   *          relative to other constraints
-   * @return a new configuration, storable in the {@link HTableDescriptor}
+   * Setup the configuration for a constraint as to whether it is enabled and its priority
+   * @param conf on which to base the new configuration
+   * @param enabled <tt>true</tt> if it should be run
+   * @param priority relative to other constraints
+   * @return a new configuration, storable in the {@link TableDescriptor}
    */
-  private static Configuration configure(Configuration conf, boolean enabled,
-      long priority) {
+  private static Configuration configure(Configuration conf, boolean enabled, long priority) {
     // create the configuration to actually be stored
     // clone if possible, but otherwise just create an empty configuration
-    Configuration toWrite = conf == null ? new Configuration()
-        : new Configuration(conf);
+    Configuration toWrite = conf == null ? new Configuration() : new Configuration(conf);
 
     // update internal properties
     toWrite.setBooleanIfUnset(ENABLED_KEY, enabled);
@@ -373,46 +263,32 @@ public final class Constraints {
   }
 
   /**
-   * Just write the class to a String representation of the class as a key for
-   * the {@link HTableDescriptor}
-   * 
-   * @param clazz
-   *          Constraint class to convert to a {@link HTableDescriptor} key
-   * @return key to store in the {@link HTableDescriptor}
+   * Just write the class to a String representation of the class as a key for the
+   * {@link TableDescriptor}
+   * @param clazz Constraint class to convert to a {@link TableDescriptor} key
+   * @return key to store in the {@link TableDescriptor}
    */
-  private static String serializeConstraintClass(
-      Class<? extends Constraint> clazz) {
+  private static String serializeConstraintClass(Class<? extends Constraint> clazz) {
     String constraintClazz = clazz.getName();
     return CONSTRAINT_HTD_KEY_PREFIX + constraintClazz;
   }
 
   /**
-   * Write the given key and associated configuration to the
-   * {@link HTableDescriptor}
+   * Write the given key and associated configuration to the {@link TableDescriptorBuilder}.
    */
-  private static void writeConstraint(HTableDescriptor desc, String key,
-      Configuration conf) throws IOException {
+  private static TableDescriptorBuilder writeConstraint(TableDescriptorBuilder builder, String key,
+    Configuration conf) throws IOException {
     // store the key and conf in the descriptor
-    desc.setValue(key, serializeConfiguration(conf));
-  }
-
-  private static void writeConstraint(
-      TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor, String key,
-      Configuration conf) throws IOException {
-    // store the key and conf in the descriptor
-    tableDescriptor.setValue(key, serializeConfiguration(conf));
+    return builder.setValue(key, serializeConfiguration(conf));
   }
 
   /**
    * Write the configuration to a String
-   * 
-   * @param conf
-   *          to write
+   * @param conf to write
    * @return String representation of that configuration
    * @throws IOException
    */
-  private static String serializeConfiguration(Configuration conf)
-      throws IOException {
+  private static String serializeConfiguration(Configuration conf) throws IOException {
     // write the configuration out to the data stream
     ByteArrayOutputStream bos = new ByteArrayOutputStream();
     DataOutputStream dos = new DataOutputStream(bos);
@@ -424,13 +300,10 @@ public final class Constraints {
 
   /**
    * Read the {@link Configuration} stored in the byte stream.
-   * 
-   * @param bytes
-   *          to read from
+   * @param bytes to read from
    * @return A valid configuration
    */
-  private static Configuration readConfiguration(byte[] bytes)
-      throws IOException {
+  private static Configuration readConfiguration(byte[] bytes) throws IOException {
     ByteArrayInputStream is = new ByteArrayInputStream(bytes);
     Configuration conf = new Configuration(false);
     conf.addResource(is);
@@ -439,20 +312,16 @@ public final class Constraints {
 
   /**
    * Read in the configuration from the String encoded configuration
-   * 
-   * @param bytes
-   *          to read from
+   * @param bytes to read from
    * @return A valid configuration
-   * @throws IOException
-   *           if the configuration could not be read
+   * @throws IOException if the configuration could not be read
    */
-  private static Configuration readConfiguration(String bytes)
-      throws IOException {
+  private static Configuration readConfiguration(String bytes) throws IOException {
     return readConfiguration(Bytes.toBytes(bytes));
   }
 
-  private static long getNextPriority(TableDescriptor desc) {
-    String value = desc.getValue(COUNTER_KEY);
+  private static long getNextPriority(TableDescriptorBuilder builder) {
+    String value = builder.getValue(COUNTER_KEY);
 
     long priority;
     // get the current priority
@@ -465,41 +334,30 @@ public final class Constraints {
     return priority;
   }
 
-  private static void updateLatestPriority(HTableDescriptor desc, long priority) {
+  private static TableDescriptorBuilder updateLatestPriority(TableDescriptorBuilder builder,
+    long priority) {
     // update the max priority
-    desc.setValue(COUNTER_KEY, Long.toString(priority));
-  }
-
-  private static void updateLatestPriority(
-      TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor, long priority) {
-    // update the max priority
-    tableDescriptor.setValue(COUNTER_KEY, Long.toString(priority));
+    return builder.setValue(COUNTER_KEY, Long.toString(priority));
   }
 
   /**
-   * Update the configuration for the {@link Constraint}; does not change the
-   * order in which the constraint is run.
-   * 
-   * @param desc
-   *          {@link HTableDescriptor} to update
-   * @param clazz
-   *          {@link Constraint} to update
-   * @param configuration
-   *          to update the {@link Constraint} with.
-   * @throws IOException
-   *           if the Constraint was not stored correctly
-   * @throws IllegalArgumentException
-   *           if the Constraint was not present on this table.
+   * Update the configuration for the {@link Constraint}; does not change the order in which the
+   * constraint is run.
+   * @param builder {@link TableDescriptorBuilder} to update
+   * @param clazz {@link Constraint} to update
+   * @param configuration to update the {@link Constraint} with.
+   * @throws IOException if the Constraint was not stored correctly
+   * @throws IllegalArgumentException if the Constraint was not present on this table.
    */
-  public static void setConfiguration(HTableDescriptor desc,
-      Class<? extends Constraint> clazz, Configuration configuration)
-      throws IOException, IllegalArgumentException {
+  public static TableDescriptorBuilder setConfiguration(TableDescriptorBuilder builder,
+    Class<? extends Constraint> clazz, Configuration configuration)
+    throws IOException, IllegalArgumentException {
     // get the entry for this class
-    Pair<String, String> e = getKeyValueForClass(desc, clazz);
+    Pair<String, String> e = getKeyValueForClass(builder, clazz);
 
     if (e == null) {
-      throw new IllegalArgumentException("Constraint: " + clazz.getName()
-          + " is not associated with this table.");
+      throw new IllegalArgumentException(
+        "Constraint: " + clazz.getName() + " is not associated with this table.");
     }
 
     // clone over the configuration elements
@@ -513,95 +371,54 @@ public final class Constraints {
     conf.setIfUnset(PRIORITY_KEY, internal.get(PRIORITY_KEY));
 
     // update the current value
-    writeConstraint(desc, e.getFirst(), conf);
+    return writeConstraint(builder, e.getFirst(), conf);
   }
 
   /**
-   * Remove the constraint (and associated information) for the table
-   * descriptor.
-   * 
-   * @param desc
-   *          {@link HTableDescriptor} to modify
-   * @param clazz
-   *          {@link Constraint} class to remove
+   * Remove the constraint (and associated information) for the table descriptor.
+   * @param builder {@link TableDescriptorBuilder} to modify
+   * @param clazz {@link Constraint} class to remove
    */
-  public static void remove(HTableDescriptor desc,
-      Class<? extends Constraint> clazz) {
+  public static TableDescriptorBuilder remove(TableDescriptorBuilder builder,
+    Class<? extends Constraint> clazz) {
     String key = serializeConstraintClass(clazz);
-    desc.remove(key);
+    return builder.removeValue(key);
   }
 
   /**
-   * Enable the given {@link Constraint}. Retains all the information (e.g.
-   * Configuration) for the {@link Constraint}, but makes sure that it gets
-   * loaded on the table.
-   * 
-   * @param desc
-   *          {@link HTableDescriptor} to modify
-   * @param clazz
-   *          {@link Constraint} to enable
-   * @throws IOException
-   *           If the constraint cannot be properly deserialized
+   * Enable the given {@link Constraint}. Retains all the information (e.g. Configuration) for the
+   * {@link Constraint}, but makes sure that it gets loaded on the table.
+   * @param builder {@link TableDescriptorBuilder} to modify
+   * @param clazz {@link Constraint} to enable
+   * @throws IOException If the constraint cannot be properly deserialized
    */
-  public static void enableConstraint(HTableDescriptor desc,
-      Class<? extends Constraint> clazz) throws IOException {
-    changeConstraintEnabled(desc, clazz, true);
+  public static void enableConstraint(TableDescriptorBuilder builder,
+    Class<? extends Constraint> clazz) throws IOException {
+    changeConstraintEnabled(builder, clazz, true);
   }
 
   /**
-   * Disable the given {@link Constraint}. Retains all the information (e.g.
-   * Configuration) for the {@link Constraint}, but it just doesn't load the
-   * {@link Constraint} on the table.
-   * 
-   * @param desc
-   *          {@link HTableDescriptor} to modify
-   * @param clazz
-   *          {@link Constraint} to disable.
-   * @throws IOException
-   *           if the constraint cannot be found
+   * Disable the given {@link Constraint}. Retains all the information (e.g. Configuration) for the
+   * {@link Constraint}, but it just doesn't load the {@link Constraint} on the table.
+   * @param builder {@link TableDescriptorBuilder} to modify
+   * @param clazz {@link Constraint} to disable.
+   * @throws IOException if the constraint cannot be found
    */
-  public static void disableConstraint(HTableDescriptor desc,
-      Class<? extends Constraint> clazz) throws IOException {
-    changeConstraintEnabled(desc, clazz, false);
-  }
-
-  public static void disableConstraint(
-      TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor,
-      Class<? extends Constraint> clazz) throws IOException {
-    changeConstraintEnabled(tableDescriptor, clazz, false);
+  public static void disableConstraint(TableDescriptorBuilder builder,
+    Class<? extends Constraint> clazz) throws IOException {
+    changeConstraintEnabled(builder, clazz, false);
   }
 
   /**
-   * Change the whether the constraint (if it is already present) is enabled or
-   * disabled.
+   * Change the whether the constraint (if it is already present) is enabled or disabled.
    */
-  private static void changeConstraintEnabled(HTableDescriptor desc,
-      Class<? extends Constraint> clazz, boolean enabled) throws IOException {
-    // get the original constraint
-    Pair<String, String> entry = getKeyValueForClass(desc, clazz);
-    if (entry == null) {
-      throw new IllegalArgumentException("Constraint: " + clazz.getName()
-          + " is not associated with this table. You can't enable it!");
-    }
-
-    // create a new configuration from that conf
-    Configuration conf = readConfiguration(entry.getSecond());
-
-    // set that it is enabled
-    conf.setBoolean(ENABLED_KEY, enabled);
-
-    // write it back out
-    writeConstraint(desc, entry.getFirst(), conf);
-  }
-
-  private static void changeConstraintEnabled(
-      TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor,
+  private static TableDescriptorBuilder changeConstraintEnabled(TableDescriptorBuilder builder,
     Class<? extends Constraint> clazz, boolean enabled) throws IOException {
     // get the original constraint
-    Pair<String, String> entry = getKeyValueForClass(tableDescriptor, clazz);
+    Pair<String, String> entry = getKeyValueForClass(builder, clazz);
     if (entry == null) {
-      throw new IllegalArgumentException("Constraint: " + clazz.getName()
-        + " is not associated with this table. You can't enable it!");
+      throw new IllegalArgumentException("Constraint: " + clazz.getName() +
+        " is not associated with this table. You can't enable it!");
     }
 
     // create a new configuration from that conf
@@ -611,23 +428,19 @@ public final class Constraints {
     conf.setBoolean(ENABLED_KEY, enabled);
 
     // write it back out
-    writeConstraint(tableDescriptor, entry.getFirst(), conf);
+    return writeConstraint(builder, entry.getFirst(), conf);
   }
 
   /**
    * Check to see if the given constraint is enabled.
-   * 
-   * @param desc
-   *          {@link HTableDescriptor} to check.
-   * @param clazz
-   *          {@link Constraint} to check for
-   * @return <tt>true</tt> if the {@link Constraint} is present and enabled.
-   *         <tt>false</tt> otherwise.
-   * @throws IOException
-   *           If the constraint has improperly stored in the table
+   * @param desc {@link TableDescriptor} to check.
+   * @param clazz {@link Constraint} to check for
+   * @return <tt>true</tt> if the {@link Constraint} is present and enabled. <tt>false</tt>
+   *         otherwise.
+   * @throws IOException If the constraint has improperly stored in the table
    */
-  public static boolean enabled(HTableDescriptor desc,
-      Class<? extends Constraint> clazz) throws IOException {
+  public static boolean enabled(TableDescriptor desc, Class<? extends Constraint> clazz)
+    throws IOException {
     // get the kv
     Pair<String, String> entry = getKeyValueForClass(desc, clazz);
     // its not enabled so just return false. In fact, its not even present!
@@ -643,24 +456,19 @@ public final class Constraints {
 
   /**
    * Get the constraints stored in the table descriptor
-   * 
-   * @param desc
-   *          To read from
-   * @param classloader
-   *          To use when loading classes. If a special classloader is used on a
-   *          region, for instance, then that should be the classloader used to
-   *          load the constraints. This could also apply to unit-testing
-   *          situation, where want to ensure that class is reloaded or not.
+   * @param desc To read from
+   * @param classloader To use when loading classes. If a special classloader is used on a region,
+   *          for instance, then that should be the classloader used to load the constraints. This
+   *          could also apply to unit-testing situation, where want to ensure that class is
+   *          reloaded or not.
    * @return List of configured {@link Constraint Constraints}
-   * @throws IOException
-   *           if any part of reading/arguments fails
+   * @throws IOException if any part of reading/arguments fails
    */
-  static List<? extends Constraint> getConstraints(TableDescriptor desc,
-      ClassLoader classloader) throws IOException {
+  static List<? extends Constraint> getConstraints(TableDescriptor desc, ClassLoader classloader)
+    throws IOException {
     List<Constraint> constraints = new ArrayList<>();
     // loop through all the key, values looking for constraints
-    for (Map.Entry<Bytes, Bytes> e : desc
-        .getValues().entrySet()) {
+    for (Map.Entry<Bytes, Bytes> e : desc.getValues().entrySet()) {
       // read out the constraint
       String key = Bytes.toString(e.getKey().get()).trim();
       String[] className = CONSTRAINT_HTD_ATTR_KEY_PATTERN.split(key);
@@ -676,27 +484,25 @@ public final class Constraints {
           conf = readConfiguration(e.getValue().get());
         } catch (IOException e1) {
           // long that we don't have a valid configuration stored, and move on.
-          LOG.warn("Corrupted configuration found for key:" + key
-              + ",  skipping it.");
+          LOG.warn("Corrupted configuration found for key:" + key + ",  skipping it.");
           continue;
         }
         // if it is not enabled, skip it
         if (!conf.getBoolean(ENABLED_KEY, false)) {
-          if (LOG.isDebugEnabled())
-            LOG.debug("Constraint: " + key + " is DISABLED - skipping it");
+          LOG.debug("Constraint: {} is DISABLED - skipping it", key);
           // go to the next constraint
           continue;
         }
 
         try {
           // add the constraint, now that we expect it to be valid.
-          Class<? extends Constraint> clazz = classloader.loadClass(key)
-              .asSubclass(Constraint.class);
+          Class<? extends Constraint> clazz =
+            classloader.loadClass(key).asSubclass(Constraint.class);
           Constraint constraint = clazz.getDeclaredConstructor().newInstance();
           constraint.setConf(conf);
           constraints.add(constraint);
-        } catch (InvocationTargetException | NoSuchMethodException | ClassNotFoundException |
-            InstantiationException | IllegalAccessException e1) {
+        } catch (InvocationTargetException | NoSuchMethodException | ClassNotFoundException
+          | InstantiationException | IllegalAccessException e1) {
           throw new IOException(e1);
         }
       }
@@ -711,7 +517,7 @@ public final class Constraints {
     public int compare(Constraint c1, Constraint c2) {
       // compare the priorities of the constraints stored in their configuration
       return Long.compare(c1.getConf().getLong(PRIORITY_KEY, DEFAULT_PRIORITY),
-          c2.getConf().getLong(PRIORITY_KEY, DEFAULT_PRIORITY));
+        c2.getConf().getLong(PRIORITY_KEY, DEFAULT_PRIORITY));
     }
   };
 
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/constraint/package-info.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/constraint/package-info.java
index 6729f7c..0696fc8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/constraint/package-info.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/constraint/package-info.java
@@ -18,110 +18,121 @@
 
 /**
  * Restrict the domain of a data attribute, often times to fulfill business rules/requirements.
+ * <h2>Table of Contents</h2>
+ * <ul>
+ * <li><a href="#overview">Overview</a></li>
+ * <li><a href="#concurrency">Concurrency and Atomicity</a></li>
+ * <li><a href="#caveats">Caveats</a></li>
+ * <li><a href="#usage">Example Usage</a></li>
+ * </ul>
+ * <h2><a name="overview">Overview</a></h2> Constraints are used to enforce business rules in a
+ * database. By checking all {@link org.apache.hadoop.hbase.client.Put Puts} on a given table, you
+ * can enforce very specific data policies. For instance, you can ensure that a certain column
+ * family-column qualifier pair always has a value between 1 and 10. Otherwise, the
+ * {@link org.apache.hadoop.hbase.client.Put} is rejected and the data integrity is maintained.
+ * <p/>
+ * Constraints are designed to be configurable, so a constraint can be used across different tables,
+ * but implement different behavior depending on the specific configuration given to that
+ * constraint.
+ * <p/>
+ * By adding a constraint to a table (see <a href="#usage">Example Usage</a>), constraints will
+ * automatically be enabled. You also then have the option of to disable (just 'turn off') or remove
+ * (delete all associated information) all constraints on a table. If you remove all constraints
+ * (see
+ * {@link org.apache.hadoop.hbase.constraint.Constraints#remove(org.apache.hadoop.hbase.client.TableDescriptorBuilder)},
+ * you must re-add any {@link org.apache.hadoop.hbase.constraint.Constraint} you want on that table.
+ * However, if they are just disabled (see
+ * {@link org.apache.hadoop.hbase.constraint.Constraints#disable(org.apache.hadoop.hbase.client.TableDescriptorBuilder)},
+ * all you need to do is enable constraints again, and everything will be turned back on as it was
+ * configured. Individual constraints can also be individually enabled, disabled or removed without
+ * affecting other constraints.
+ * <p/>
+ * By default, constraints are disabled on a table. This means you will not see <i>any</i> slow down
+ * on a table if constraints are not enabled.
+ * <p/>
+ * <h2><a name="concurrency">Concurrency and Atomicity</a></h2> Currently, no attempts at enforcing
+ * correctness in a multi-threaded scenario when modifying a constraint, via
+ * {@link org.apache.hadoop.hbase.constraint.Constraints}, to the the
+ * {@link org.apache.hadoop.hbase.client.TableDescriptorBuilder}. This is particularly important
+ * when adding a constraint(s) to the {@link org.apache.hadoop.hbase.client.TableDescriptorBuilder}
+ * as it first retrieves the next priority from a custom value set in the descriptor, adds each
+ * constraint (with increasing priority) to the descriptor, and then the next available priority is
+ * re-stored back in the {@link org.apache.hadoop.hbase.client.TableDescriptorBuilder}.
+ * <p/>
+ * Locking is recommended around each of Constraints add methods:
+ * {@link org.apache.hadoop.hbase.constraint.Constraints#add(org.apache.hadoop.hbase.client.TableDescriptorBuilder, Class...)},
+ * {@link org.apache.hadoop.hbase.constraint.Constraints#add(org.apache.hadoop.hbase.client.TableDescriptorBuilder, org.apache.hadoop.hbase.util.Pair...)},
+ * and
+ * {@link org.apache.hadoop.hbase.constraint.Constraints#add(org.apache.hadoop.hbase.client.TableDescriptorBuilder, Class, org.apache.hadoop.conf.Configuration)}.
+ * Any changes on <i>a single TableDescriptor</i> should be serialized, either within a single
+ * thread or via external mechanisms.
+ * <p/>
+ * Note that having a higher priority means that a constraint will run later; e.g. a constraint with
+ * priority 1 will run before a constraint with priority 2.
+ * <p/>
+ * Since Constraints currently are designed to just implement simple checks (e.g. is the value in
+ * the right range), there will be no atomicity conflicts. Even if one of the puts finishes the
+ * constraint first, the single row will not be corrupted and the 'fastest' write will win; the
+ * underlying region takes care of breaking the tie and ensuring that writes get serialized to the
+ * table. So yes, this doesn't ensure that we are going to get specific ordering or even a fully
+ * consistent view of the underlying data.
+ * <p/>
+ * Each constraint should only use local/instance variables, unless doing more advanced usage.
+ * Static variables could cause difficulties when checking concurrent writes to the same region,
+ * leading to either highly locked situations (decreasing through-put) or higher probability of
+ * errors. However, as long as each constraint just uses local variables, each thread interacting
+ * with the constraint will execute correctly and efficiently.
+ * <h2><a name="caveats">Caveats</a></h2> In traditional (SQL) databases, Constraints are often used
+ * to enforce <a href="http://en.wikipedia.org/wiki/Relational_database#Constraints">referential
+ * integrity</a>. However, in HBase, this will likely cause significant overhead and dramatically
+ * decrease the number of {@link org.apache.hadoop.hbase.client.Put Puts}/second possible on a
+ * table. This is because to check the referential integrity when making a
+ * {@link org.apache.hadoop.hbase.client.Put}, one must block on a scan for the 'remote' table,
+ * checking for the valid reference. For millions of {@link org.apache.hadoop.hbase.client.Put Puts}
+ * a second, this will breakdown very quickly. There are several options around the blocking
+ * behavior including, but not limited to:
+ * <ul>
+ * <li>Create a 'pre-join' table where the keys are already denormalized</li>
+ * <li>Designing for 'incorrect' references</li>
+ * <li>Using an external enforcement mechanism</li>
+ * </ul>
+ * There are also several general considerations that must be taken into account, when using
+ * Constraints:
+ * <ol>
+ * <li>All changes made via {@link org.apache.hadoop.hbase.constraint.Constraints} will make
+ * modifications to the {@link org.apache.hadoop.hbase.client.TableDescriptor} for a given table. As
+ * such, the usual renabling of tables should be used for propagating changes to the table. When at
+ * all possible, Constraints should be added to the table before the table is created.</li>
+ * <li>Constraints are run in the order that they are added to a table. This has implications for
+ * what order constraints should be added to a table.</li>
+ * <li>Whenever new Constraint jars are added to a region server, those region servers need to go
+ * through a rolling restart to make sure that they pick up the new jars and can enable the new
+ * constraints.</li>
+ * <li>There are certain keys that are reserved for the Configuration namespace:
+ * <ul>
+ * <li>_ENABLED - used server-side to determine if a constraint should be run</li>
+ * <li>_PRIORITY - used server-side to determine what order a constraint should be run</li>
+ * </ul>
+ * If these items are set, they will be respected in the constraint configuration, but they are
+ * taken care of by default in when adding constraints to an
+ * {@link org.apache.hadoop.hbase.client.TableDescriptorBuilder} via the usual method.</li>
+ * </ol>
+ * <p/>
+ * Under the hood, constraints are implemented as a Coprocessor (see
+ * {@link org.apache.hadoop.hbase.constraint.ConstraintProcessor} if you are interested).
+ * <h2><a name="usage">Example usage</a></h2> First, you must define a
+ * {@link org.apache.hadoop.hbase.constraint.Constraint}. The best way to do this is to extend
+ * {@link org.apache.hadoop.hbase.constraint.BaseConstraint}, which takes care of some of the more
+ * mundane details of using a {@link org.apache.hadoop.hbase.constraint.Constraint}.
+ * <p/>
+ * Let's look at one possible implementation of a constraint - an IntegerConstraint(there are also
+ * several simple examples in the tests). The IntegerConstraint checks to make sure that the value
+ * is a String-encoded <code>int</code>. It is really simple to implement this kind of constraint,
+ * the only method needs to be implemented is
+ * {@link org.apache.hadoop.hbase.constraint.Constraint#check(org.apache.hadoop.hbase.client.Put)}:
+ * <div style="background-color: #cccccc; padding: 2px"> <blockquote>
  *
- <h2> Table of Contents</h2>
- <ul>
- <li><a href="#overview">Overview</a></li>
- <li><a href="#concurrency">Concurrency and Atomicity</a></li>
- <li><a href="#caveats">Caveats</a></li>
- <li><a href="#usage">Example Usage</a></li>
- </ul>
-
- <h2><a name="overview">Overview</a></h2>
- Constraints are used to enforce business rules in a database.
- By checking all {@link org.apache.hadoop.hbase.client.Put Puts} on a given table, you can enforce very specific data policies.
- For instance, you can ensure that a certain column family-column qualifier pair always has a value between 1 and 10.
- Otherwise, the {@link org.apache.hadoop.hbase.client.Put} is rejected and the data integrity is maintained.
- <p>
- Constraints are designed to be configurable, so a constraint can be used across different tables, but implement different
- behavior depending on the specific configuration given to that constraint.
- <p>
- By adding a constraint to a table (see <a href="#usage">Example Usage</a>), constraints will automatically enabled.
- You also then have the option of to disable (just 'turn off') or remove (delete all associated information) all constraints on a table.
- If you remove all constraints
- (see {@link org.apache.hadoop.hbase.constraint.Constraints#remove(org.apache.hadoop.hbase.HTableDescriptor)},
- you must re-add any {@link org.apache.hadoop.hbase.constraint.Constraint} you want on that table.
- However, if they are just disabled (see {@link org.apache.hadoop.hbase.constraint.Constraints#disable(org.apache.hadoop.hbase.HTableDescriptor)},
- all you need to do is enable constraints again, and everything will be turned back on as it was configured.
- Individual constraints can also be individually enabled, disabled or removed without affecting other constraints.
- <p>
- By default, constraints are disabled on a table.
- This means you will not see <i>any</i> slow down on a table if constraints are not enabled.
- <p>
-
- <h2><a name="concurrency">Concurrency and Atomicity</a></h2>
- Currently, no attempts at enforcing correctness in a multi-threaded scenario when modifying a constraint, via
- {@link org.apache.hadoop.hbase.constraint.Constraints}, to the the {@link org.apache.hadoop.hbase.HTableDescriptor}.
- This is particularly important when adding a constraint(s) to the {@link org.apache.hadoop.hbase.HTableDescriptor}
- as it first retrieves the next priority from a custom value set in the descriptor,
- adds each constraint (with increasing priority) to the descriptor, and then the next available priority is re-stored
- back in the {@link org.apache.hadoop.hbase.HTableDescriptor}.
- <p>
- Locking is recommended around each of Constraints add methods:
- {@link org.apache.hadoop.hbase.constraint.Constraints#add(org.apache.hadoop.hbase.HTableDescriptor, Class...)},
- {@link org.apache.hadoop.hbase.constraint.Constraints#add(org.apache.hadoop.hbase.HTableDescriptor, org.apache.hadoop.hbase.util.Pair...)},
- and {@link org.apache.hadoop.hbase.constraint.Constraints#add(org.apache.hadoop.hbase.HTableDescriptor, Class, org.apache.hadoop.conf.Configuration)}.
- Any changes on <i>a single HTableDescriptor</i> should be serialized, either within a single thread or via external mechanisms.
- <p>
- Note that having a higher priority means that a constraint will run later; e.g. a constraint with priority 1 will run before a
- constraint with priority 2.
- <p>
- Since Constraints currently are designed to just implement simple checks (e.g. is the value in the right range), there will
- be no atomicity conflicts.
- Even if one of the puts finishes the constraint first, the single row will not be corrupted and the 'fastest' write will win;
- the underlying region takes care of breaking the tie and ensuring that writes get serialized to the table.
- So yes, this doesn't ensure that we are going to get specific ordering or even a fully consistent view of the underlying data.
- <p>
- Each constraint should only use local/instance variables, unless doing more advanced usage. Static variables could cause difficulties
- when checking concurrent writes to the same region, leading to either highly locked situations (decreasing through-put) or higher probability of errors.
- However, as long as each constraint just uses local variables, each thread interacting with the constraint will execute correctly and efficiently.
-
- <h2><a name="caveats">Caveats</a></h2>
- In traditional (SQL) databases, Constraints are often used to enforce <a href="http://en.wikipedia.org/wiki/Relational_database#Constraints">referential integrity</a>.
- However, in HBase, this will likely cause significant overhead and dramatically decrease the number of
- {@link org.apache.hadoop.hbase.client.Put Puts}/second possible on a table. This is because to check the referential integrity
- when making a {@link org.apache.hadoop.hbase.client.Put}, one must block on a scan for the 'remote' table, checking for the valid reference.
- For millions of {@link org.apache.hadoop.hbase.client.Put Puts} a second, this will breakdown very quickly.
- There are several options around the blocking behavior including, but not limited to:
- <ul>
- <li>Create a 'pre-join' table where the keys are already denormalized</li>
- <li>Designing for 'incorrect' references</li>
- <li>Using an external enforcement mechanism</li>
- </ul>
- There are also several general considerations that must be taken into account, when using Constraints:
- <ol>
- <li>All changes made via {@link org.apache.hadoop.hbase.constraint.Constraints} will make modifications to the
- {@link org.apache.hadoop.hbase.HTableDescriptor} for a given table. As such, the usual renabling of tables should be used for
- propagating changes to the table. When at all possible, Constraints should be added to the table before the table is created.</li>
- <li>Constraints are run in the order that they are added to a table. This has implications for what order constraints should
- be added to a table.</li>
- <li>Whenever new Constraint jars are added to a region server, those region servers need to go through a rolling restart to
- make sure that they pick up the new jars and can enable the new constraints.</li>
- <li>There are certain keys that are reserved for the Configuration namespace:
- <ul>
- <li>_ENABLED - used server-side to determine if a constraint should be run</li>
- <li>_PRIORITY - used server-side to determine what order a constraint should be run</li>
- </ul>
- If these items are set, they will be respected in the constraint configuration, but they are taken care of by default in when
- adding constraints to an {@link org.apache.hadoop.hbase.HTableDescriptor} via the usual method.</li>
- </ol>
- <p>
- Under the hood, constraints are implemented as a Coprocessor (see {@link org.apache.hadoop.hbase.constraint.ConstraintProcessor}
- if you are interested).
-
-
- <h2><a name="usage">Example usage</a></h2>
- First, you must define a {@link org.apache.hadoop.hbase.constraint.Constraint}.
- The best way to do this is to extend {@link org.apache.hadoop.hbase.constraint.BaseConstraint}, which takes care of some of the more
- mundane details of using a {@link org.apache.hadoop.hbase.constraint.Constraint}.
- <p>
- Let's look at one possible implementation of a constraint - an IntegerConstraint(there are also several simple examples in the tests).
- The IntegerConstraint checks to make sure that the value is a String-encoded <code>int</code>.
- It is really simple to implement this kind of constraint, the only method needs to be implemented is
- {@link org.apache.hadoop.hbase.constraint.Constraint#check(org.apache.hadoop.hbase.client.Put)}:
-
- <div style="background-color: #cccccc; padding: 2px">
- <blockquote><pre>
+ * <pre>
  public class IntegerConstraint extends BaseConstraint {
  public void check(Put p) throws ConstraintException {
 
@@ -140,123 +151,153 @@
  throw new ConstraintException("Value in Put (" + p
  + ") was not a String-encoded integer", e);
  } } }
- </pre></blockquote>
- </div>
- <p>
- Note that all exceptions that you expect to be thrown must be caught and then rethrown as a
- {@link org.apache.hadoop.hbase.constraint.ConstraintException}. This way, you can be sure that a
- {@link org.apache.hadoop.hbase.client.Put} fails for an expected reason, rather than for any reason.
- For example, an {@link java.lang.OutOfMemoryError} is probably indicative of an inherent problem in
- the {@link org.apache.hadoop.hbase.constraint.Constraint}, rather than a failed {@link org.apache.hadoop.hbase.client.Put}.
- <p>
- If an unexpected exception is thrown (for example, any kind of uncaught {@link java.lang.RuntimeException}),
- constraint-checking will be 'unloaded' from the regionserver where that error occurred.
- This means no further {@link org.apache.hadoop.hbase.constraint.Constraint Constraints} will be checked on that server
- until it is reloaded. This is done to ensure the system remains as available as possible.
- Therefore, be careful when writing your own Constraint.
- <p>
- So now that we have a Constraint, we want to add it to a table. It's as easy as:
-
- <div style="background-color: #cccccc; padding: 2px">
- <blockquote><pre>
- HTableDescriptor desc = new HTableDescriptor(TABLE_NAME);
+ * </pre>
+ *
+ * </blockquote> </div>
+ * <p/>
+ * Note that all exceptions that you expect to be thrown must be caught and then rethrown as a
+ * {@link org.apache.hadoop.hbase.constraint.ConstraintException}. This way, you can be sure that a
+ * {@link org.apache.hadoop.hbase.client.Put} fails for an expected reason, rather than for any
+ * reason. For example, an {@link java.lang.OutOfMemoryError} is probably indicative of an inherent
+ * problem in the {@link org.apache.hadoop.hbase.constraint.Constraint}, rather than a failed
+ * {@link org.apache.hadoop.hbase.client.Put}.
+ * <p/>
+ * If an unexpected exception is thrown (for example, any kind of uncaught
+ * {@link java.lang.RuntimeException}), constraint-checking will be 'unloaded' from the regionserver
+ * where that error occurred. This means no further
+ * {@link org.apache.hadoop.hbase.constraint.Constraint Constraints} will be checked on that server
+ * until it is reloaded. This is done to ensure the system remains as available as possible.
+ * Therefore, be careful when writing your own Constraint.
+ * <p/>
+ * So now that we have a Constraint, we want to add it to a table. It's as easy as:
+ * <div style="background-color: #cccccc; padding: 2px"> <blockquote>
+ *
+ * <pre>
+ TableDescriptor builder = TableDescriptorBuilder.newBuilder(TABLE_NAME);
  ...
- Constraints.add(desc, IntegerConstraint.class);
- </pre></blockquote></div>
- <p>
- Once we added the IntegerConstraint, constraints will be enabled on the table (once it is created) and
- we will always check to make sure that the value is an String-encoded integer.
- <p>
- However, suppose we also write our own constraint, <code>MyConstraint.java</code>.
- First, you need to make sure this class-files are in the classpath (in a jar) on the regionserver where
- that constraint will be run (this could require a rolling restart on the region server - see <a href="#caveats">Caveats</a> above)
- <p>
- Suppose that MyConstraint also uses a Configuration (see {@link org.apache.hadoop.hbase.constraint.Constraint#getConf()}).
- Then adding MyConstraint looks like this:
-
- <div style="background-color: #cccccc; padding: 2px">
- <blockquote><pre>
- HTableDescriptor desc = new HTableDescriptor(TABLE_NAME);
+ Constraints.add(builder, IntegerConstraint.class);
+ * </pre>
+ *
+ * </blockquote></div>
+ * <p/>
+ * Once we added the IntegerConstraint, constraints will be enabled on the table (once it is
+ * created) and we will always check to make sure that the value is an String-encoded integer.
+ * <p/>
+ * However, suppose we also write our own constraint, <code>MyConstraint.java</code>. First, you
+ * need to make sure this class-files are in the classpath (in a jar) on the regionserver where that
+ * constraint will be run (this could require a rolling restart on the region server - see
+ * <a href="#caveats">Caveats</a> above)
+ * <p/>
+ * Suppose that MyConstraint also uses a Configuration (see
+ * {@link org.apache.hadoop.hbase.constraint.Constraint#getConf()}). Then adding MyConstraint looks
+ * like this: <div style="background-color: #cccccc; padding: 2px"> <blockquote>
+ *
+ * <pre>
+ TableDescriptor builder = TableDescriptorBuilder.newBuilder(TABLE_NAME);
  Configuration conf = new Configuration(false);
  ...
  (add values to the conf)
  (modify the table descriptor)
  ...
- Constraints.add(desc, new Pair(MyConstraint.class, conf));
- </pre></blockquote></div>
- <p>
- At this point we added both the IntegerConstraint and MyConstraint to the table, the IntegerConstraint
- <i>will be run first</i>, followed by MyConstraint.
- <p>
- Suppose we realize that the {@link org.apache.hadoop.conf.Configuration} for MyConstraint is actually wrong
- when it was added to the table. Note, when it is added to the table, it is <i>not</i> added by reference,
- but is instead copied into the {@link org.apache.hadoop.hbase.HTableDescriptor}.
- Thus, to change the {@link org.apache.hadoop.conf.Configuration} we are using for MyConstraint, we need to do this:
-
- <div style="background-color: #cccccc; padding: 2px">
- <blockquote><pre>
+ Constraints.add(builder, new Pair(MyConstraint.class, conf));
+ * </pre>
+ *
+ * </blockquote></div>
+ * <p/>
+ * At this point we added both the IntegerConstraint and MyConstraint to the table, the
+ * IntegerConstraint <i>will be run first</i>, followed by MyConstraint.
+ * <p/>
+ * Suppose we realize that the {@link org.apache.hadoop.conf.Configuration} for MyConstraint is
+ * actually wrong when it was added to the table. Note, when it is added to the table, it is
+ * <i>not</i> added by reference, but is instead copied into the
+ * {@link org.apache.hadoop.hbase.client.TableDescriptor}. Thus, to change the
+ * {@link org.apache.hadoop.conf.Configuration} we are using for MyConstraint, we need to do this:
+ * <div style="background-color: #cccccc; padding: 2px"> <blockquote>
+ *
+ * <pre>
  (add/modify the conf)
  ...
  Constraints.setConfiguration(desc, MyConstraint.class, conf);
- </pre></blockquote></div>
- <p>
- This will overwrite the previous configuration for MyConstraint, but <i>not</i> change the order of the
- constraint nor if it is enabled/disabled.
- <p>
- Note that the same constraint class can be added multiple times to a table without repercussion.
- A use case for this is the same constraint working differently based on its configuration.
-
- <p>
- Suppose then we want to disable <i>just</i> MyConstraint. Its as easy as:
- <div style="background-color: #cccccc">
- <blockquote><pre>
- Constraints.disable(desc, MyConstraint.class);
- </pre></blockquote></div>
- <p>
- This just turns off MyConstraint, but retains the position and the configuration associated with MyConstraint.
- Now, if we want to re-enable the constraint, its just another one-liner:
- <div style="background-color: #cccccc">
- <blockquote><pre>
- Constraints.enable(desc, MyConstraint.class);
- </pre></blockquote></div>
- <p>
- Similarly, constraints on the entire table are disabled via:
- <div style="background-color: #cccccc">
- <blockquote><pre>
- Constraints.disable(desc);
- </pre></blockquote></div>
- <p>
- Or enabled via:
-
- <div style="background-color: #cccccc">
- <blockquote><pre>
- Constraints.enable(desc);
- </pre></blockquote></div>
- <p>
- Lastly, suppose you want to remove MyConstraint from the table, including with position it should be run at and its configuration.
- This is similarly simple:
- <div style="background-color: #cccccc">
- <blockquote><pre>
- Constraints.remove(desc, MyConstraint.class);
- </pre></blockquote></div>
- <p>
- Also, removing <i>all</i> constraints from a table is similarly simple:
- <div style="background-color: #cccccc">
- <blockquote><pre>
- Constraints.remove(desc);
- </pre></blockquote></div>
- This will remove all constraints (and associated information) from the table and turn off the constraint processing.
- <p><b>NOTE</b><p>
- It is important to note the use above of
- <div style="background-color: #cccccc">
- <blockquote><pre>
- Configuration conf = new Configuration(false);
- </pre></blockquote></div>
- If you just use <code> new Configuration()</code>, then the Configuration will be loaded with the default
- properties. While in the simple case, this is not going to be an issue, it will cause pain down the road.
- First, these extra properties are going to cause serious bloat in your {@link org.apache.hadoop.hbase.HTableDescriptor},
- meaning you are keeping around a ton of redundant information. Second, it is going to make examining
- your table in the shell, via <code>describe 'table'</code>, a huge pain as you will have to dig through
- a ton of irrelevant config values to find the ones you set. In short, just do it the right way.
+ * </pre>
+ *
+ * </blockquote></div>
+ * <p/>
+ * This will overwrite the previous configuration for MyConstraint, but <i>not</i> change the order
+ * of the constraint nor if it is enabled/disabled.
+ * <p/>
+ * Note that the same constraint class can be added multiple times to a table without repercussion.
+ * A use case for this is the same constraint working differently based on its configuration.
+ * <p/>
+ * Suppose then we want to disable <i>just</i> MyConstraint. Its as easy as:
+ * <div style="background-color: #cccccc"> <blockquote>
+ *
+ * <pre>
+ * Constraints.disable(desc, MyConstraint.class);
+ * </pre>
+ *
+ * </blockquote></div>
+ * <p/>
+ * This just turns off MyConstraint, but retains the position and the configuration associated with
+ * MyConstraint. Now, if we want to re-enable the constraint, its just another one-liner:
+ * <div style="background-color: #cccccc"> <blockquote>
+ *
+ * <pre>
+ * Constraints.enable(desc, MyConstraint.class);
+ * </pre>
+ *
+ * </blockquote></div>
+ * <p/>
+ * Similarly, constraints on the entire table are disabled via:
+ * <div style="background-color: #cccccc"> <blockquote>
+ *
+ * <pre>
+ * Constraints.disable(desc);
+ * </pre>
+ *
+ * </blockquote></div>
+ * <p/>
+ * Or enabled via: <div style="background-color: #cccccc"> <blockquote>
+ *
+ * <pre>
+ * Constraints.enable(desc);
+ * </pre>
+ *
+ * </blockquote></div>
+ * <p/>
+ * Lastly, suppose you want to remove MyConstraint from the table, including with position it should
+ * be run at and its configuration. This is similarly simple:
+ * <div style="background-color: #cccccc"> <blockquote>
+ *
+ * <pre>
+ * Constraints.remove(desc, MyConstraint.class);
+ * </pre>
+ *
+ * </blockquote></div>
+ * <p/>
+ * Also, removing <i>all</i> constraints from a table is similarly simple:
+ * <div style="background-color: #cccccc"> <blockquote>
+ *
+ * <pre>
+ * Constraints.remove(desc);
+ * </pre>
+ *
+ * </blockquote></div> This will remove all constraints (and associated information) from the table
+ * and turn off the constraint processing.
+ * <p/>
+ * <b>NOTE</b>
+ * <p/>
+ * It is important to note the use above of <div style="background-color: #cccccc"> <blockquote>
+ *
+ * <pre>
+ * Configuration conf = new Configuration(false);
+ * </pre>
+ *
+ * </blockquote></div> If you just use <code> new Configuration()</code>, then the Configuration
+ * will be loaded with the default properties. While in the simple case, this is not going to be an
+ * issue, it will cause pain down the road. First, these extra properties are going to cause serious
+ * bloat in your {@link org.apache.hadoop.hbase.client.TableDescriptor}, meaning you are keeping
+ * around a ton of redundant information. Second, it is going to make examining your table in the
+ * shell, via <code>describe 'table'</code>, a huge pain as you will have to dig through a ton of
+ * irrelevant config values to find the ones you set. In short, just do it the right way.
  */
 package org.apache.hadoop.hbase.constraint;
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/constraint/TestConstraint.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/constraint/TestConstraint.java
index 40952a3..182dda5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/constraint/TestConstraint.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/constraint/TestConstraint.java
@@ -44,15 +44,14 @@ import org.slf4j.LoggerFactory;
 /**
  * Do the complex testing of constraints against a minicluster
  */
-@Category({MiscTests.class, MediumTests.class})
+@Category({ MiscTests.class, MediumTests.class })
 public class TestConstraint {
 
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
-      HBaseClassTestRule.forClass(TestConstraint.class);
+    HBaseClassTestRule.forClass(TestConstraint.class);
 
-  private static final Logger LOG = LoggerFactory
-      .getLogger(TestConstraint.class);
+  private static final Logger LOG = LoggerFactory.getLogger(TestConstraint.class);
 
   private static HBaseTestingUtility util;
   private static final TableName tableName = TableName.valueOf("test");
@@ -69,24 +68,20 @@ public class TestConstraint {
 
   /**
    * Test that we run a passing constraint
-   * @throws Exception
    */
-  @SuppressWarnings("unchecked")
   @Test
   public void testConstraintPasses() throws Exception {
     // create the table
     // it would be nice if this was also a method on the util
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
+    TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableName);
 
-    for (byte[] family : new byte[][]{dummy, test}) {
-      tableDescriptor.setColumnFamily(
-        new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family));
+    for (byte[] family : new byte[][] { dummy, test }) {
+      builder.setColumnFamily(ColumnFamilyDescriptorBuilder.of(family));
     }
     // add a constraint
-    Constraints.add(tableDescriptor, CheckWasRunConstraint.class);
+    Constraints.add(builder, CheckWasRunConstraint.class);
 
-    util.getAdmin().createTable(tableDescriptor);
+    util.getAdmin().createTable(builder.build());
     Table table = util.getConnection().getTable(tableName);
     try {
       // test that we don't fail on a valid put
@@ -103,25 +98,20 @@ public class TestConstraint {
 
   /**
    * Test that constraints will fail properly
-   * @throws Exception
    */
-  @SuppressWarnings("unchecked")
   @Test
   public void testConstraintFails() throws Exception {
-
     // create the table
     // it would be nice if this was also a method on the util
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
-    for (byte[] family : new byte[][]{dummy, test}) {
-      tableDescriptor.setColumnFamily(
-        new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family));
+    TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableName);
+    for (byte[] family : new byte[][] { dummy, test }) {
+      builder.setColumnFamily(ColumnFamilyDescriptorBuilder.of(family));
     }
 
     // add a constraint that is sure to fail
-    Constraints.add(tableDescriptor, AllFailConstraint.class);
+    Constraints.add(builder, AllFailConstraint.class);
 
-    util.getAdmin().createTable(tableDescriptor);
+    util.getAdmin().createTable(builder.build());
     Table table = util.getConnection().getTable(tableName);
 
     // test that we do fail on violation
@@ -140,29 +130,25 @@ public class TestConstraint {
 
   /**
    * Check that if we just disable one constraint, then
-   * @throws Throwable
    */
-  @SuppressWarnings("unchecked")
   @Test
-  public void testDisableConstraint() throws Throwable {
+  public void testDisableConstraint() throws Exception {
     // create the table
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
+    TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableName);
     // add a family to the table
-    for (byte[] family : new byte[][]{dummy, test}) {
-      tableDescriptor.setColumnFamily(
-        new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family));
+    for (byte[] family : new byte[][] { dummy, test }) {
+      builder.setColumnFamily(ColumnFamilyDescriptorBuilder.of(family));
     }
     // add a constraint to make sure it others get run
-    Constraints.add(tableDescriptor, CheckWasRunConstraint.class);
+    Constraints.add(builder, CheckWasRunConstraint.class);
 
     // Add Constraint to check
-    Constraints.add(tableDescriptor, AllFailConstraint.class);
+    Constraints.add(builder, AllFailConstraint.class);
 
     // and then disable the failing constraint
-    Constraints.disableConstraint(tableDescriptor, AllFailConstraint.class);
+    Constraints.disableConstraint(builder, AllFailConstraint.class);
 
-    util.getAdmin().createTable(tableDescriptor);
+    util.getAdmin().createTable(builder.build());
     Table table = util.getConnection().getTable(tableName);
     try {
       // test that we don't fail because its disabled
@@ -178,27 +164,23 @@ public class TestConstraint {
 
   /**
    * Test that if we disable all constraints, then nothing gets run
-   * @throws Throwable
    */
-  @SuppressWarnings("unchecked")
   @Test
-  public void testDisableConstraints() throws Throwable {
+  public void testDisableConstraints() throws Exception {
     // create the table
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
+    TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableName);
 
     // add a family to the table
-    for (byte[] family : new byte[][]{dummy, test}) {
-      tableDescriptor.setColumnFamily(
-        new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family));
+    for (byte[] family : new byte[][] { dummy, test }) {
+      builder.setColumnFamily(ColumnFamilyDescriptorBuilder.of(family));
     }
     // add a constraint to check to see if is run
-    Constraints.add(tableDescriptor, CheckWasRunConstraint.class);
+    Constraints.add(builder, CheckWasRunConstraint.class);
 
     // then disable all the constraints
-    Constraints.disable(tableDescriptor);
+    Constraints.disable(builder);
 
-    util.getAdmin().createTable(tableDescriptor);
+    util.getAdmin().createTable(builder.build());
     Table table = util.getConnection().getTable(tableName);
     try {
       // test that we do fail on violation
@@ -215,26 +197,23 @@ public class TestConstraint {
 
   /**
    * Check to make sure a constraint is unloaded when it fails
-   * @throws Exception
    */
   @Test
   public void testIsUnloaded() throws Exception {
     // create the table
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
+    TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableName);
 
     // add a family to the table
-    for (byte[] family : new byte[][]{dummy, test}) {
-      tableDescriptor.setColumnFamily(
-        new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family));
+    for (byte[] family : new byte[][] { dummy, test }) {
+      builder.setColumnFamily(ColumnFamilyDescriptorBuilder.of(family));
     }
     // make sure that constraints are unloaded
-    Constraints.add(tableDescriptor, RuntimeFailConstraint.class);
+    Constraints.add(builder, RuntimeFailConstraint.class);
     // add a constraint to check to see if is run
-    Constraints.add(tableDescriptor, CheckWasRunConstraint.class);
+    Constraints.add(builder, CheckWasRunConstraint.class);
     CheckWasRunConstraint.wasRun = false;
 
-    util.getAdmin().createTable(tableDescriptor);
+    util.getAdmin().createTable(builder.build());
     Table table = util.getConnection().getTable(tableName);
 
     // test that we do fail on violation
@@ -242,9 +221,9 @@ public class TestConstraint {
     byte[] qualifier = new byte[0];
     put.addColumn(dummy, qualifier, Bytes.toBytes("pass"));
 
-    try{
-    table.put(put);
-    fail("RuntimeFailConstraint wasn't triggered - this put shouldn't work!");
+    try {
+      table.put(put);
+      fail("RuntimeFailConstraint wasn't triggered - this put shouldn't work!");
     } catch (Exception e) {// NOOP
     }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/constraint/TestConstraints.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/constraint/TestConstraints.java
index 5a65447..2087a98 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/constraint/TestConstraints.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/constraint/TestConstraints.java
@@ -25,9 +25,9 @@ import static org.junit.Assert.fail;
 import java.util.List;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.TableNameTestRule;
 import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.constraint.TestConstraint.CheckWasRunConstraint;
 import org.apache.hadoop.hbase.constraint.WorksConstraint.NameConstraint;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
@@ -37,38 +37,35 @@ import org.junit.ClassRule;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
-import org.junit.rules.TestName;
 
 /**
- * Test reading/writing the constraints into the {@link HTableDescriptor}
+ * Test reading/writing the constraints into the {@link TableDescriptorBuilder}.
  */
-@Category({MiscTests.class, SmallTests.class})
+@Category({ MiscTests.class, SmallTests.class })
 public class TestConstraints {
 
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
-      HBaseClassTestRule.forClass(TestConstraints.class);
+    HBaseClassTestRule.forClass(TestConstraints.class);
 
   @Rule
-  public TestName name = new TestName();
+  public TableNameTestRule name = new TableNameTestRule();
 
-  @SuppressWarnings("unchecked")
   @Test
-  public void testSimpleReadWrite() throws Throwable {
-    HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(name.getMethodName()));
-    Constraints.add(desc, WorksConstraint.class);
+  public void testSimpleReadWrite() throws Exception {
+    TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(name.getTableName());
+    Constraints.add(builder, WorksConstraint.class);
 
-    List<? extends Constraint> constraints = Constraints.getConstraints(desc,
-        this.getClass().getClassLoader());
+    List<? extends Constraint> constraints =
+      Constraints.getConstraints(builder.build(), this.getClass().getClassLoader());
     assertEquals(1, constraints.size());
 
     assertEquals(WorksConstraint.class, constraints.get(0).getClass());
 
     // Check that we can add more than 1 constraint and that ordering is
     // preserved
-    Constraints.add(desc, AlsoWorks.class, NameConstraint.class);
-    constraints = Constraints.getConstraints(desc, this.getClass()
-        .getClassLoader());
+    Constraints.add(builder, AlsoWorks.class, NameConstraint.class);
+    constraints = Constraints.getConstraints(builder.build(), this.getClass().getClassLoader());
     assertEquals(3, constraints.size());
 
     assertEquals(WorksConstraint.class, constraints.get(0).getClass());
@@ -77,26 +74,24 @@ public class TestConstraints {
 
   }
 
-  @SuppressWarnings("unchecked")
   @Test
-  public void testReadWriteWithConf() throws Throwable {
-    HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(name.getMethodName()));
-    Constraints.add(desc,
-      new Pair<>(CheckConfigurationConstraint.class,
-        CheckConfigurationConstraint.getConfiguration()));
-
-    List<? extends Constraint> c = Constraints.getConstraints(desc, this
-        .getClass().getClassLoader());
+  public void testReadWriteWithConf() throws Exception {
+    TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(name.getTableName());
+    Constraints.add(builder, new Pair<>(CheckConfigurationConstraint.class,
+      CheckConfigurationConstraint.getConfiguration()));
+
+    List<? extends Constraint> c =
+      Constraints.getConstraints(builder.build(), this.getClass().getClassLoader());
     assertEquals(1, c.size());
 
     assertEquals(CheckConfigurationConstraint.class, c.get(0).getClass());
 
     // check to make sure that we overwrite configurations
-    Constraints.add(desc, new Pair<>(
-        CheckConfigurationConstraint.class, new Configuration(false)));
+    Constraints.add(builder,
+      new Pair<>(CheckConfigurationConstraint.class, new Configuration(false)));
 
     try {
-      Constraints.getConstraints(desc, this.getClass().getClassLoader());
+      Constraints.getConstraints(builder.build(), this.getClass().getClassLoader());
       fail("No exception thrown  - configuration not overwritten");
     } catch (IllegalArgumentException e) {
       // expect to have the exception, so don't do anything
@@ -105,100 +100,89 @@ public class TestConstraints {
 
   /**
    * Test that Constraints are properly enabled, disabled, and removed
-   *
-   * @throws Exception
    */
-  @SuppressWarnings("unchecked")
   @Test
   public void testEnableDisableRemove() throws Exception {
-    HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(name.getMethodName()));
+    TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(name.getTableName());
     // check general enabling/disabling of constraints
     // first add a constraint
-    Constraints.add(desc, AllPassConstraint.class);
+    Constraints.add(builder, AllPassConstraint.class);
     // make sure everything is enabled
-    assertTrue(Constraints.enabled(desc, AllPassConstraint.class));
-    assertTrue(desc.hasCoprocessor(ConstraintProcessor.class.getName()));
+    assertTrue(Constraints.enabled(builder.build(), AllPassConstraint.class));
+    assertTrue(builder.hasCoprocessor(ConstraintProcessor.class.getName()));
 
     // check disabling
-    Constraints.disable(desc);
-    assertFalse(desc.hasCoprocessor(ConstraintProcessor.class.getName()));
+    Constraints.disable(builder);
+    assertFalse(builder.hasCoprocessor(ConstraintProcessor.class.getName()));
     // make sure the added constraints are still present
-    assertTrue(Constraints.enabled(desc, AllPassConstraint.class));
+    assertTrue(Constraints.enabled(builder.build(), AllPassConstraint.class));
 
     // check just removing the single constraint
-    Constraints.remove(desc, AllPassConstraint.class);
-    assertFalse(Constraints.has(desc, AllPassConstraint.class));
+    Constraints.remove(builder, AllPassConstraint.class);
+    assertFalse(Constraints.has(builder.build(), AllPassConstraint.class));
 
     // Add back the single constraint
-    Constraints.add(desc, AllPassConstraint.class);
+    Constraints.add(builder, AllPassConstraint.class);
 
     // and now check that when we remove constraints, all are gone
-    Constraints.remove(desc);
-    assertFalse(desc.hasCoprocessor(ConstraintProcessor.class.getName()));
-    assertFalse(Constraints.has(desc, AllPassConstraint.class));
+    Constraints.remove(builder);
+    assertFalse(builder.hasCoprocessor(ConstraintProcessor.class.getName()));
+    assertFalse(Constraints.has(builder.build(), AllPassConstraint.class));
 
   }
 
   /**
    * Test that when we update a constraint the ordering is not modified.
-   *
-   * @throws Exception
    */
-  @SuppressWarnings("unchecked")
   @Test
   public void testUpdateConstraint() throws Exception {
-    HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(name.getMethodName()));
-    Constraints.add(desc, CheckConfigurationConstraint.class,
-        CheckWasRunConstraint.class);
-    Constraints.setConfiguration(desc, CheckConfigurationConstraint.class,
-        CheckConfigurationConstraint.getConfiguration());
+    TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(name.getTableName());
+    Constraints.add(builder, CheckConfigurationConstraint.class, CheckWasRunConstraint.class);
+    Constraints.setConfiguration(builder, CheckConfigurationConstraint.class,
+      CheckConfigurationConstraint.getConfiguration());
 
-    List<? extends Constraint> constraints = Constraints.getConstraints(desc,
-        this.getClass().getClassLoader());
+    List<? extends Constraint> constraints =
+      Constraints.getConstraints(builder.build(), this.getClass().getClassLoader());
 
     assertEquals(2, constraints.size());
 
     // check to make sure the order didn't change
-    assertEquals(CheckConfigurationConstraint.class, constraints.get(0)
-        .getClass());
+    assertEquals(CheckConfigurationConstraint.class, constraints.get(0).getClass());
     assertEquals(CheckWasRunConstraint.class, constraints.get(1).getClass());
   }
 
   /**
-   * Test that if a constraint hasn't been set that there are no problems with
-   * attempting to remove it.
-   *
-   * @throws Throwable
-   *           on failure.
+   * Test that if a constraint hasn't been set that there are no problems with attempting to remove
+   * it.
    */
   @Test
-  public void testRemoveUnsetConstraint() throws Throwable {
-    HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(name.getMethodName()));
-    Constraints.remove(desc);
-    Constraints.remove(desc, AlsoWorks.class);
+  public void testRemoveUnsetConstraint() throws Exception {
+    TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(name.getTableName());
+    Constraints.remove(builder);
+    Constraints.remove(builder, AlsoWorks.class);
   }
 
   @Test
-  public void testConfigurationPreserved() throws Throwable {
+  public void testConfigurationPreserved() throws Exception {
     Configuration conf = new Configuration();
     conf.setBoolean("_ENABLED", false);
     conf.setLong("_PRIORITY", 10);
-    HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(name.getMethodName()));
-    Constraints.add(desc, AlsoWorks.class, conf);
-    Constraints.add(desc, WorksConstraint.class);
-    assertFalse(Constraints.enabled(desc, AlsoWorks.class));
-    List<? extends Constraint> constraints = Constraints.getConstraints(desc,
-        this.getClass().getClassLoader());
+    TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(name.getTableName());
+    Constraints.add(builder, AlsoWorks.class, conf);
+    Constraints.add(builder, WorksConstraint.class);
+    assertFalse(Constraints.enabled(builder.build(), AlsoWorks.class));
+    List<? extends Constraint> constraints =
+      Constraints.getConstraints(builder.build(), this.getClass().getClassLoader());
     for (Constraint c : constraints) {
       Configuration storedConf = c.getConf();
-      if (c instanceof AlsoWorks)
+      if (c instanceof AlsoWorks) {
         assertEquals(10, storedConf.getLong("_PRIORITY", -1));
+      }
       // its just a worksconstraint
-      else
+      else {
         assertEquals(2, storedConf.getLong("_PRIORITY", -1));
-
+      }
     }
-
   }
 
   // ---------- Constraints just used for testing