HBASE-15583 Any HTableDescriptor we give out should be immutable
diff --git a/bin/region_status.rb b/bin/region_status.rb
index 91873cb..f889de9 100644
--- a/bin/region_status.rb
+++ b/bin/region_status.rb
@@ -133,7 +133,7 @@
 
 # query the master to see how many regions are on region servers
 if not $tablename.nil?
-  $TableName = HTableDescriptor.new($tablename.to_java_bytes).getTableName()
+  $TableName = TableName.valueOf($tablename.to_java_bytes)
 end
 while true
   if $tablename.nil?
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
index ed0659c..e3cf2ec 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
@@ -19,29 +19,20 @@
 package org.apache.hadoop.hbase;
 
 import java.io.IOException;
-import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-import java.util.TreeMap;
-import java.util.TreeSet;
-import java.util.regex.Matcher;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Durability;
-import org.apache.hadoop.hbase.client.RegionReplicaUtil;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder.ModifyableTableDescriptor;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.security.User;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.TableSchema;
 import org.apache.hadoop.hbase.util.Bytes;
 
 /**
@@ -49,271 +40,34 @@
  * all the column families, is the table a catalog table, <code> -ROOT- </code> or
  * <code> hbase:meta </code>, if the table is read only, the maximum size of the memstore,
  * when the region split should occur, coprocessors associated with it etc...
+ * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
+ *             use {@link TableDescriptorBuilder} to build {@link HTableDescriptor}.
  */
+@Deprecated
 @InterfaceAudience.Public
-public class HTableDescriptor implements Comparable<HTableDescriptor> {
-
-  private static final Log LOG = LogFactory.getLog(HTableDescriptor.class);
-
-  private TableName name = null;
-
-  /**
-   * A map which holds the metadata information of the table. This metadata
-   * includes values like IS_ROOT, IS_META, DEFERRED_LOG_FLUSH, SPLIT_POLICY,
-   * MAX_FILE_SIZE, READONLY, MEMSTORE_FLUSHSIZE etc...
-   */
-  private final Map<Bytes, Bytes> values = new HashMap<>();
-
-  /**
-   * A map which holds the configuration specific to the table.
-   * The keys of the map have the same names as config keys and override the defaults with
-   * table-specific settings. Example usage may be for compactions, etc.
-   */
-  private final Map<String, String> configuration = new HashMap<>();
-
-  public static final String SPLIT_POLICY = "SPLIT_POLICY";
-
-  /**
-   * <em>INTERNAL</em> Used by HBase Shell interface to access this metadata
-   * attribute which denotes the maximum size of the store file after which
-   * a region split occurs
-   *
-   * @see #getMaxFileSize()
-   */
-  public static final String MAX_FILESIZE = "MAX_FILESIZE";
-  private static final Bytes MAX_FILESIZE_KEY =
-      new Bytes(Bytes.toBytes(MAX_FILESIZE));
-
-  public static final String OWNER = "OWNER";
-  public static final Bytes OWNER_KEY =
-      new Bytes(Bytes.toBytes(OWNER));
-
-  /**
-   * <em>INTERNAL</em> Used by rest interface to access this metadata
-   * attribute which denotes if the table is Read Only
-   *
-   * @see #isReadOnly()
-   */
-  public static final String READONLY = "READONLY";
-  private static final Bytes READONLY_KEY =
-      new Bytes(Bytes.toBytes(READONLY));
-
-  /**
-   * <em>INTERNAL</em> Used by HBase Shell interface to access this metadata
-   * attribute which denotes if the table is compaction enabled
-   *
-   * @see #isCompactionEnabled()
-   */
-  public static final String COMPACTION_ENABLED = "COMPACTION_ENABLED";
-  private static final Bytes COMPACTION_ENABLED_KEY =
-      new Bytes(Bytes.toBytes(COMPACTION_ENABLED));
-
-  /**
-   * <em>INTERNAL</em> Used by HBase Shell interface to access this metadata
-   * attribute which represents the maximum size of the memstore after which
-   * its contents are flushed onto the disk
-   *
-   * @see #getMemStoreFlushSize()
-   */
-  public static final String MEMSTORE_FLUSHSIZE = "MEMSTORE_FLUSHSIZE";
-  private static final Bytes MEMSTORE_FLUSHSIZE_KEY =
-      new Bytes(Bytes.toBytes(MEMSTORE_FLUSHSIZE));
-
-  public static final String FLUSH_POLICY = "FLUSH_POLICY";
-
-  /**
-   * <em>INTERNAL</em> Used by rest interface to access this metadata
-   * attribute which denotes if the table is a -ROOT- region or not
-   *
-   * @see #isRootRegion()
-   */
-  public static final String IS_ROOT = "IS_ROOT";
-  private static final Bytes IS_ROOT_KEY =
-      new Bytes(Bytes.toBytes(IS_ROOT));
-
-  /**
-   * <em>INTERNAL</em> Used by rest interface to access this metadata
-   * attribute which denotes if it is a catalog table, either
-   * <code> hbase:meta </code> or <code> -ROOT- </code>
-   *
-   * @see #isMetaRegion()
-   */
-  public static final String IS_META = "IS_META";
-  private static final Bytes IS_META_KEY =
-      new Bytes(Bytes.toBytes(IS_META));
-
-  /**
-   * <em>INTERNAL</em> Used by HBase Shell interface to access this metadata
-   * attribute which denotes if the deferred log flush option is enabled.
-   * @deprecated Use {@link #DURABILITY} instead.
-   */
-  @Deprecated
-  public static final String DEFERRED_LOG_FLUSH = "DEFERRED_LOG_FLUSH";
-  @Deprecated
-  private static final Bytes DEFERRED_LOG_FLUSH_KEY =
-      new Bytes(Bytes.toBytes(DEFERRED_LOG_FLUSH));
-
-  /**
-   * <em>INTERNAL</em> {@link Durability} setting for the table.
-   */
-  public static final String DURABILITY = "DURABILITY";
-  private static final Bytes DURABILITY_KEY =
-      new Bytes(Bytes.toBytes("DURABILITY"));
-
-  /**
-   * <em>INTERNAL</em> number of region replicas for the table.
-   */
-  public static final String REGION_REPLICATION = "REGION_REPLICATION";
-  private static final Bytes REGION_REPLICATION_KEY =
-      new Bytes(Bytes.toBytes(REGION_REPLICATION));
-
-  /**
-   * <em>INTERNAL</em> flag to indicate whether or not the memstore should be replicated
-   * for read-replicas (CONSISTENCY =&gt; TIMELINE).
-   */
-  public static final String REGION_MEMSTORE_REPLICATION = "REGION_MEMSTORE_REPLICATION";
-  private static final Bytes REGION_MEMSTORE_REPLICATION_KEY =
-      new Bytes(Bytes.toBytes(REGION_MEMSTORE_REPLICATION));
-
-  /**
-   * <em>INTERNAL</em> Used by shell/rest interface to access this metadata
-   * attribute which denotes if the table should be treated by region normalizer.
-   *
-   * @see #isNormalizationEnabled()
-   */
-  public static final String NORMALIZATION_ENABLED = "NORMALIZATION_ENABLED";
-  private static final Bytes NORMALIZATION_ENABLED_KEY =
-    new Bytes(Bytes.toBytes(NORMALIZATION_ENABLED));
-
-  /** Default durability for HTD is USE_DEFAULT, which defaults to HBase-global default value */
-  private static final Durability DEFAULT_DURABLITY = Durability.USE_DEFAULT;
-
-  public static final String PRIORITY = "PRIORITY";
-  private static final Bytes PRIORITY_KEY =
-    new Bytes(Bytes.toBytes(PRIORITY));
-
-  /** Relative priority of the table used for rpc scheduling */
-  private static final int DEFAULT_PRIORITY = HConstants.NORMAL_QOS;
-
-  /*
-   *  The below are ugly but better than creating them each time till we
-   *  replace booleans being saved as Strings with plain booleans.  Need a
-   *  migration script to do this.  TODO.
-   */
-  private static final Bytes FALSE =
-      new Bytes(Bytes.toBytes(Boolean.FALSE.toString()));
-
-  private static final Bytes TRUE =
-      new Bytes(Bytes.toBytes(Boolean.TRUE.toString()));
-
-  private static final boolean DEFAULT_DEFERRED_LOG_FLUSH = false;
-
-  /**
-   * Constant that denotes whether the table is READONLY by default and is false
-   */
-  public static final boolean DEFAULT_READONLY = false;
-
-  /**
-   * Constant that denotes whether the table is compaction enabled by default
-   */
-  public static final boolean DEFAULT_COMPACTION_ENABLED = true;
-
-  /**
-   * Constant that denotes whether the table is normalized by default.
-   */
-  public static final boolean DEFAULT_NORMALIZATION_ENABLED = false;
-
-  /**
-   * Constant that denotes the maximum default size of the memstore after which
-   * the contents are flushed to the store files
-   */
-  public static final long DEFAULT_MEMSTORE_FLUSH_SIZE = 1024*1024*128L;
-
-  public static final int DEFAULT_REGION_REPLICATION = 1;
-
-  public static final boolean DEFAULT_REGION_MEMSTORE_REPLICATION = true;
-
-  private final static Map<String, String> DEFAULT_VALUES = new HashMap<>();
-  private final static Set<Bytes> RESERVED_KEYWORDS = new HashSet<>();
-
-  static {
-    DEFAULT_VALUES.put(MAX_FILESIZE,
-        String.valueOf(HConstants.DEFAULT_MAX_FILE_SIZE));
-    DEFAULT_VALUES.put(READONLY, String.valueOf(DEFAULT_READONLY));
-    DEFAULT_VALUES.put(MEMSTORE_FLUSHSIZE,
-        String.valueOf(DEFAULT_MEMSTORE_FLUSH_SIZE));
-    DEFAULT_VALUES.put(DEFERRED_LOG_FLUSH,
-        String.valueOf(DEFAULT_DEFERRED_LOG_FLUSH));
-    DEFAULT_VALUES.put(DURABILITY, DEFAULT_DURABLITY.name()); //use the enum name
-    DEFAULT_VALUES.put(REGION_REPLICATION, String.valueOf(DEFAULT_REGION_REPLICATION));
-    DEFAULT_VALUES.put(NORMALIZATION_ENABLED, String.valueOf(DEFAULT_NORMALIZATION_ENABLED));
-    DEFAULT_VALUES.put(PRIORITY, String.valueOf(DEFAULT_PRIORITY));
-    for (String s : DEFAULT_VALUES.keySet()) {
-      RESERVED_KEYWORDS.add(new Bytes(Bytes.toBytes(s)));
-    }
-    RESERVED_KEYWORDS.add(IS_ROOT_KEY);
-    RESERVED_KEYWORDS.add(IS_META_KEY);
-  }
-
-  /**
-   * Cache of whether this is a meta table or not.
-   */
-  private volatile Boolean meta = null;
-  /**
-   * Cache of whether this is root table or not.
-   */
-  private volatile Boolean root = null;
-
-  /**
-   * Durability setting for the table
-   */
-  private Durability durability = null;
-
-  /**
-   * Maps column family name to the respective HColumnDescriptors
-   */
-  private final Map<byte [], HColumnDescriptor> families =
-    new TreeMap<>(Bytes.BYTES_RAWCOMPARATOR);
-
-  /**
-   * <em> INTERNAL </em> Private constructor used internally creating table descriptors for
-   * catalog tables, <code>hbase:meta</code> and <code>-ROOT-</code>.
-   */
-  @InterfaceAudience.Private
-  protected HTableDescriptor(final TableName name, HColumnDescriptor[] families) {
-    setName(name);
-    for(HColumnDescriptor descriptor : families) {
-      this.families.put(descriptor.getName(), descriptor);
-    }
-  }
-
-  /**
-   * <em> INTERNAL </em>Private constructor used internally creating table descriptors for
-   * catalog tables, <code>hbase:meta</code> and <code>-ROOT-</code>.
-   */
-  protected HTableDescriptor(final TableName name, HColumnDescriptor[] families,
-      Map<Bytes, Bytes> values) {
-    setName(name);
-    for(HColumnDescriptor descriptor : families) {
-      this.families.put(descriptor.getName(), descriptor);
-    }
-    for (Map.Entry<Bytes, Bytes> entry :
-        values.entrySet()) {
-      setValue(entry.getKey(), entry.getValue());
-    }
-  }
-
-  /**
-   * Default constructor which constructs an empty object.
-   * For deserializing an HTableDescriptor instance only.
-   * @deprecated As of release 0.96 (<a href="https://issues.apache.org/jira/browse/HBASE-5453">HBASE-5453</a>).
-   *             This was made protected in 2.0.0 and will be removed in HBase 3.0.0.
-   *             Used by Writables and Writables are going away.
-   */
-  @Deprecated
-  protected HTableDescriptor() {
-    super();
-  }
+public class HTableDescriptor implements TableDescriptor, Comparable<HTableDescriptor> {
+  public static final String SPLIT_POLICY = TableDescriptorBuilder.SPLIT_POLICY;
+  public static final String MAX_FILESIZE = TableDescriptorBuilder.MAX_FILESIZE;
+  public static final String OWNER = TableDescriptorBuilder.OWNER;
+  public static final Bytes OWNER_KEY = TableDescriptorBuilder.OWNER_KEY;
+  public static final String READONLY = TableDescriptorBuilder.READONLY;
+  public static final String COMPACTION_ENABLED = TableDescriptorBuilder.COMPACTION_ENABLED;
+  public static final String MEMSTORE_FLUSHSIZE = TableDescriptorBuilder.MEMSTORE_FLUSHSIZE;
+  public static final String FLUSH_POLICY = TableDescriptorBuilder.FLUSH_POLICY;
+  public static final String IS_ROOT = TableDescriptorBuilder.IS_ROOT;
+  public static final String IS_META = TableDescriptorBuilder.IS_META;
+  public static final String DURABILITY = TableDescriptorBuilder.DURABILITY;
+  public static final String REGION_REPLICATION = TableDescriptorBuilder.REGION_REPLICATION;
+  public static final String REGION_MEMSTORE_REPLICATION = TableDescriptorBuilder.REGION_MEMSTORE_REPLICATION;
+  public static final String NORMALIZATION_ENABLED = TableDescriptorBuilder.NORMALIZATION_ENABLED;
+  public static final String PRIORITY = TableDescriptorBuilder.PRIORITY;
+  public static final boolean DEFAULT_READONLY = TableDescriptorBuilder.DEFAULT_READONLY;
+  public static final boolean DEFAULT_COMPACTION_ENABLED = TableDescriptorBuilder.DEFAULT_COMPACTION_ENABLED;
+  public static final boolean DEFAULT_NORMALIZATION_ENABLED = TableDescriptorBuilder.DEFAULT_NORMALIZATION_ENABLED;
+  public static final long DEFAULT_MEMSTORE_FLUSH_SIZE = TableDescriptorBuilder.DEFAULT_MEMSTORE_FLUSH_SIZE;
+  public static final int DEFAULT_REGION_REPLICATION = TableDescriptorBuilder.DEFAULT_REGION_REPLICATION;
+  public static final boolean DEFAULT_REGION_MEMSTORE_REPLICATION = TableDescriptorBuilder.DEFAULT_REGION_MEMSTORE_REPLICATION;
+  private final ModifyableTableDescriptor delegatee;
 
   /**
    * Construct a table descriptor specifying a TableName object
@@ -321,39 +75,18 @@
    * @see <a href="https://issues.apache.org/jira/browse/HBASE-174">HADOOP-1581 HBASE: (HBASE-174) Un-openable tablename bug</a>
    */
   public HTableDescriptor(final TableName name) {
-    super();
-    setName(name);
-  }
-
-  /**
-   * Construct a table descriptor specifying a byte array table name
-   * @param name Table name.
-   * @see <a href="https://issues.apache.org/jira/browse/HBASE-174">HADOOP-1581 (HBASE-174) HBASE: Un-openable tablename bug</a>
-   */
-  @Deprecated
-  public HTableDescriptor(final byte[] name) {
-    this(TableName.valueOf(name));
-  }
-
-  /**
-   * Construct a table descriptor specifying a String table name
-   * @param name Table name.
-   * @see <a href="https://issues.apache.org/jira/browse/HBASE-174">HADOOP-1581 (HBASE-174) HBASE: Un-openable tablename bug</a>
-   */
-  @Deprecated
-  public HTableDescriptor(final String name) {
-    this(TableName.valueOf(name));
+    this(name, Collections.EMPTY_LIST, Collections.EMPTY_MAP, Collections.EMPTY_MAP);
   }
 
   /**
    * Construct a table descriptor by cloning the descriptor passed as a parameter.
    * <p>
    * Makes a deep copy of the supplied descriptor.
-   * Can make a modifiable descriptor from an UnmodifyableHTableDescriptor.
+   * Can make a modifiable descriptor from an ImmutableHTableDescriptor.
    * @param desc The descriptor.
    */
   public HTableDescriptor(final HTableDescriptor desc) {
-    this(desc.name, desc);
+    this(desc.getTableName(), desc.getFamilies(), desc.getValues(), desc.getConfiguration());
   }
 
   /**
@@ -361,36 +94,25 @@
    * but using a different table name.
    * <p>
    * Makes a deep copy of the supplied descriptor.
-   * Can make a modifiable descriptor from an UnmodifyableHTableDescriptor.
+   * Can make a modifiable descriptor from an ImmutableHTableDescriptor.
    * @param name Table name.
    * @param desc The descriptor.
    */
   public HTableDescriptor(final TableName name, final HTableDescriptor desc) {
-    super();
-    setName(name);
-    setMetaFlags(this.name);
-    for (HColumnDescriptor c: desc.families.values()) {
-      this.families.put(c.getName(), new HColumnDescriptor(c));
-    }
-    for (Map.Entry<Bytes, Bytes> e :
-        desc.values.entrySet()) {
-      setValue(e.getKey(), e.getValue());
-    }
-    for (Map.Entry<String, String> e : desc.configuration.entrySet()) {
-      this.configuration.put(e.getKey(), e.getValue());
-    }
+    this(name, desc.getFamilies(), desc.getValues(), desc.getConfiguration());
   }
 
-  /*
-   * Set meta flags on this table.
-   * IS_ROOT_KEY is set if its a -ROOT- table
-   * IS_META_KEY is set either if its a -ROOT- or a hbase:meta table
-   * Called by constructors.
-   * @param name
-   */
-  private void setMetaFlags(final TableName name) {
-    setMetaRegion(isRootRegion() ||
-        name.equals(TableName.META_TABLE_NAME));
+  public HTableDescriptor(final TableDescriptor desc) {
+    this(desc.getTableName(), desc.getFamilies(), desc.getValues(), desc.getConfiguration());
+  }
+
+  private HTableDescriptor(final TableName name, final Collection<HColumnDescriptor> families,
+      Map<Bytes, Bytes> values, Map<String, String> configuration) {
+    this(new ModifyableTableDescriptor(name, families, values, configuration));
+  }
+
+  protected HTableDescriptor(ModifyableTableDescriptor delegatee) {
+    this.delegatee = delegatee;
   }
 
   /**
@@ -398,23 +120,9 @@
    *
    * @return true if this is a <code> -ROOT- </code> region
    */
+  @Override
   public boolean isRootRegion() {
-    if (this.root == null) {
-      this.root = isSomething(IS_ROOT_KEY, false)? Boolean.TRUE: Boolean.FALSE;
-    }
-    return this.root.booleanValue();
-  }
-
-  /**
-   * <em> INTERNAL </em> Used to denote if the current table represents
-   * <code> -ROOT- </code> region. This is used internally by the
-   * HTableDescriptor constructors
-   *
-   * @param isRoot true if this is the <code> -ROOT- </code> region
-   */
-  protected void setRootRegion(boolean isRoot) {
-    // TODO: Make the value a boolean rather than String of boolean.
-    setValue(IS_ROOT_KEY, isRoot? TRUE: FALSE);
+    return delegatee.isRootRegion();
   }
 
   /**
@@ -424,37 +132,9 @@
    * @return true if this table is <code> hbase:meta </code>
    * region
    */
+  @Override
   public boolean isMetaRegion() {
-    if (this.meta == null) {
-      this.meta = calculateIsMetaRegion();
-    }
-    return this.meta.booleanValue();
-  }
-
-  private synchronized Boolean calculateIsMetaRegion() {
-    byte [] value = getValue(IS_META_KEY);
-    return (value != null)? Boolean.valueOf(Bytes.toString(value)): Boolean.FALSE;
-  }
-
-  private boolean isSomething(final Bytes key,
-      final boolean valueIfNull) {
-    byte [] value = getValue(key);
-    if (value != null) {
-      return Boolean.valueOf(Bytes.toString(value));
-    }
-    return valueIfNull;
-  }
-
-  /**
-   * <em> INTERNAL </em> Used to denote if the current table represents
-   * <code> -ROOT- </code> or <code> hbase:meta </code> region. This is used
-   * internally by the HTableDescriptor constructors
-   *
-   * @param isMeta true if its either <code> -ROOT- </code> or
-   * <code> hbase:meta </code> region
-   */
-  protected void setMetaRegion(boolean isMeta) {
-    setValue(IS_META_KEY, isMeta? TRUE: FALSE);
+    return delegatee.isMetaRegion();
   }
 
   /**
@@ -462,8 +142,9 @@
    *
    * @return true if table is <code> hbase:meta </code> region.
    */
+  @Override
   public boolean isMetaTable() {
-    return isMetaRegion() && !isRootRegion();
+    return delegatee.isMetaTable();
   }
 
   /**
@@ -471,17 +152,10 @@
    *
    * @param key The key.
    * @return The value.
-   * @see #values
    */
+  @Override
   public byte[] getValue(byte[] key) {
-    return getValue(new Bytes(key));
-  }
-
-  private byte[] getValue(final Bytes key) {
-    Bytes ibw = values.get(key);
-    if (ibw == null)
-      return null;
-    return ibw.get();
+    return delegatee.getValue(key);
   }
 
   /**
@@ -489,110 +163,80 @@
    *
    * @param key The key.
    * @return The value.
-   * @see #values
    */
   public String getValue(String key) {
-    byte[] value = getValue(Bytes.toBytes(key));
-    if (value == null)
-      return null;
-    return Bytes.toString(value);
+    return delegatee.getValue(key);
   }
 
   /**
-   * Getter for fetching an unmodifiable {@link #values} map.
-   *
-   * @return unmodifiable map {@link #values}.
-   * @see #values
+   * @return Getter for fetching an unmodifiable map.
    */
+  @Override
   public Map<Bytes, Bytes> getValues() {
-    // shallow pointer copy
-    return Collections.unmodifiableMap(values);
+    return delegatee.getValues();
   }
 
   /**
-   * Setter for storing metadata as a (key, value) pair in {@link #values} map
+   * Setter for storing metadata as a (key, value) pair in map
    *
    * @param key The key.
-   * @param value The value.
-   * @see #values
+   * @param value The value. If null, removes the setting.
    */
   public HTableDescriptor setValue(byte[] key, byte[] value) {
-    setValue(new Bytes(key), new Bytes(value));
+    delegatee.setValue(key, value);
     return this;
   }
 
   /*
-   * @param key The key.
-   * @param value The value.
-   */
-  private HTableDescriptor setValue(final Bytes key,
-      final String value) {
-    setValue(key, new Bytes(Bytes.toBytes(value)));
-    return this;
-  }
-
-  /*
-   * Setter for storing metadata as a (key, value) pair in {@link #values} map
+   * Setter for storing metadata as a (key, value) pair in map
    *
    * @param key The key.
-   * @param value The value.
+   * @param value The value. If null, removes the setting.
    */
   public HTableDescriptor setValue(final Bytes key, final Bytes value) {
-    if (key.compareTo(DEFERRED_LOG_FLUSH_KEY) == 0) {
-      boolean isDeferredFlush = Boolean.valueOf(Bytes.toString(value.get()));
-      LOG.warn("HTableDescriptor property:" + DEFERRED_LOG_FLUSH + " is deprecated, " +
-          "use " + DURABILITY + " instead");
-      setDurability(isDeferredFlush ? Durability.ASYNC_WAL : DEFAULT_DURABLITY);
-      return this;
-    }
-    values.put(key, value);
+    delegatee.setValue(key, value);
     return this;
   }
 
   /**
-   * Setter for storing metadata as a (key, value) pair in {@link #values} map
+   * Setter for storing metadata as a (key, value) pair in map
    *
    * @param key The key.
-   * @param value The value.
-   * @see #values
+   * @param value The value. If null, removes the setting.
    */
   public HTableDescriptor setValue(String key, String value) {
-    if (value == null) {
-      remove(key);
-    } else {
-      setValue(Bytes.toBytes(key), Bytes.toBytes(value));
-    }
+    delegatee.setValue(key, value);
     return this;
   }
 
   /**
-   * Remove metadata represented by the key from the {@link #values} map
+   * Remove metadata represented by the key from the map
    *
    * @param key Key whose key and value we're to remove from HTableDescriptor
    * parameters.
    */
   public void remove(final String key) {
-    remove(new Bytes(Bytes.toBytes(key)));
+    delegatee.remove(key);
   }
 
   /**
-   * Remove metadata represented by the key from the {@link #values} map
+   * Remove metadata represented by the key from the map
    *
    * @param key Key whose key and value we're to remove from HTableDescriptor
    * parameters.
    */
   public void remove(Bytes key) {
-    values.remove(key);
+    delegatee.remove(key);
   }
 
   /**
-   * Remove metadata represented by the key from the {@link #values} map
+   * Remove metadata represented by the key from the map
    *
    * @param key Key whose key and value we're to remove from HTableDescriptor
    * parameters.
    */
   public void remove(final byte [] key) {
-    remove(new Bytes(key));
+    delegatee.remove(key);
   }
 
   /**
@@ -601,8 +245,9 @@
    *
    * @return true if all columns in the table should be read only
    */
+  @Override
   public boolean isReadOnly() {
-    return isSomething(READONLY_KEY, DEFAULT_READONLY);
+    return delegatee.isReadOnly();
   }
 
   /**
@@ -614,7 +259,8 @@
    * only.
    */
   public HTableDescriptor setReadOnly(final boolean readOnly) {
-    return setValue(READONLY_KEY, readOnly? TRUE: FALSE);
+    delegatee.setReadOnly(readOnly);
+    return this;
   }
 
   /**
@@ -623,8 +269,9 @@
    *
    * @return true if table compaction enabled
    */
+  @Override
   public boolean isCompactionEnabled() {
-    return isSomething(COMPACTION_ENABLED_KEY, DEFAULT_COMPACTION_ENABLED);
+    return delegatee.isCompactionEnabled();
   }
 
   /**
@@ -633,7 +280,7 @@
    * @param isEnable True if enable compaction.
    */
   public HTableDescriptor setCompactionEnabled(final boolean isEnable) {
-    setValue(COMPACTION_ENABLED_KEY, isEnable ? TRUE : FALSE);
+    delegatee.setCompactionEnabled(isEnable);
     return this;
   }
 
@@ -643,8 +290,9 @@
    *
    * @return true if region normalization is enabled for this table
    */
+  @Override
   public boolean isNormalizationEnabled() {
-    return isSomething(NORMALIZATION_ENABLED_KEY, DEFAULT_NORMALIZATION_ENABLED);
+    return delegatee.isNormalizationEnabled();
   }
 
   /**
@@ -653,7 +301,7 @@
    * @param isEnable True if enable normalization.
    */
   public HTableDescriptor setNormalizationEnabled(final boolean isEnable) {
-    setValue(NORMALIZATION_ENABLED_KEY, isEnable ? TRUE : FALSE);
+    delegatee.setNormalizationEnabled(isEnable);
     return this;
   }
 
@@ -662,8 +310,7 @@
    * @param durability enum value
    */
   public HTableDescriptor setDurability(Durability durability) {
-    this.durability = durability;
-    setValue(DURABILITY_KEY, durability.name());
+    delegatee.setDurability(durability);
     return this;
   }
 
@@ -671,22 +318,9 @@
    * Returns the durability setting for the table.
    * @return durability setting for the table.
    */
+  @Override
   public Durability getDurability() {
-    if (this.durability == null) {
-      byte[] durabilityValue = getValue(DURABILITY_KEY);
-      if (durabilityValue == null) {
-        this.durability = DEFAULT_DURABLITY;
-      } else {
-        try {
-          this.durability = Durability.valueOf(Bytes.toString(durabilityValue));
-        } catch (IllegalArgumentException ex) {
-          LOG.warn("Received " + ex + " because Durability value for HTableDescriptor"
-            + " is not known. Durability:" + Bytes.toString(durabilityValue));
-          this.durability = DEFAULT_DURABLITY;
-        }
-      }
-    }
-    return this.durability;
+    return delegatee.getDurability();
   }
 
   /**
@@ -694,19 +328,9 @@
    *
    * @return TableName
    */
+  @Override
   public TableName getTableName() {
-    return name;
-  }
-
-  /**
-   * Get the name of the table as a byte array.
-   *
-   * @return name of table
-   * @deprecated Use {@link #getTableName()} instead
-   */
-  @Deprecated
-  public byte[] getName() {
-    return name.getName();
+    return delegatee.getTableName();
   }
 
   /**
@@ -715,7 +339,7 @@
    * @return name of table as a String
    */
   public String getNameAsString() {
-    return name.getNameAsString();
+    return delegatee.getTableName().getNameAsString();
   }
 
   /**
@@ -725,7 +349,7 @@
    * @param clazz the class name
    */
   public HTableDescriptor setRegionSplitPolicyClassName(String clazz) {
-    setValue(SPLIT_POLICY, clazz);
+    delegatee.setRegionSplitPolicyClassName(clazz);
     return this;
   }
 
@@ -737,26 +361,9 @@
    * @return the class name of the region split policy for this table.
    * If this returns null, the default split policy is used.
    */
+  @Override
    public String getRegionSplitPolicyClassName() {
-    return getValue(SPLIT_POLICY);
-  }
-
-  /**
-   * Set the name of the table.
-   *
-   * @param name name of table
-   */
-  @Deprecated
-  public HTableDescriptor setName(byte[] name) {
-    setName(TableName.valueOf(name));
-    return this;
-  }
-
-  @Deprecated
-  public HTableDescriptor setName(TableName name) {
-    this.name = name;
-    setMetaFlags(this.name);
-    return this;
+    return delegatee.getRegionSplitPolicyClassName();
   }
 
   /**
@@ -768,12 +375,9 @@
    *
    * @see #setMaxFileSize(long)
    */
+   @Override
   public long getMaxFileSize() {
-    byte [] value = getValue(MAX_FILESIZE_KEY);
-    if (value != null) {
-      return Long.parseLong(Bytes.toString(value));
-    }
-    return -1;
+    return delegatee.getMaxFileSize();
   }
 
   /**
@@ -792,7 +396,7 @@
    * before a split is triggered.
    */
   public HTableDescriptor setMaxFileSize(long maxFileSize) {
-    setValue(MAX_FILESIZE_KEY, Long.toString(maxFileSize));
+    delegatee.setMaxFileSize(maxFileSize);
     return this;
   }
 
@@ -803,12 +407,9 @@
    *
    * @see #setMemStoreFlushSize(long)
    */
+  @Override
   public long getMemStoreFlushSize() {
-    byte [] value = getValue(MEMSTORE_FLUSHSIZE_KEY);
-    if (value != null) {
-      return Long.parseLong(Bytes.toString(value));
-    }
-    return -1;
+    return delegatee.getMemStoreFlushSize();
   }
 
   /**
@@ -818,7 +419,7 @@
    * @param memstoreFlushSize memory cache flush size for each hregion
    */
   public HTableDescriptor setMemStoreFlushSize(long memstoreFlushSize) {
-    setValue(MEMSTORE_FLUSHSIZE_KEY, Long.toString(memstoreFlushSize));
+    delegatee.setMemStoreFlushSize(memstoreFlushSize);
     return this;
   }
 
@@ -829,7 +430,7 @@
    * @param clazz the class name
    */
   public HTableDescriptor setFlushPolicyClassName(String clazz) {
-    setValue(FLUSH_POLICY, clazz);
+    delegatee.setFlushPolicyClassName(clazz);
     return this;
   }
 
@@ -840,8 +441,9 @@
    * @return the class name of the flush policy for this table. If this returns null, the default
    *         flush policy is used.
    */
+  @Override
   public String getFlushPolicyClassName() {
-    return getValue(FLUSH_POLICY);
+    return delegatee.getFlushPolicyClassName();
   }
 
   /**
@@ -850,14 +452,7 @@
    * @param family HColumnDescriptor of family to add.
    */
   public HTableDescriptor addFamily(final HColumnDescriptor family) {
-    if (family.getName() == null || family.getName().length <= 0) {
-      throw new IllegalArgumentException("Family name cannot be null or empty");
-    }
-    if (hasFamily(family.getName())) {
-      throw new IllegalArgumentException("Family '" +
-        family.getNameAsString() + "' already exists so cannot be added");
-    }
-    this.families.put(family.getName(), family);
+    delegatee.addFamily(family);
     return this;
   }
 
@@ -867,14 +462,7 @@
    * @return this (for chained invocation)
    */
   public HTableDescriptor modifyFamily(final HColumnDescriptor family) {
-    if (family.getName() == null || family.getName().length <= 0) {
-      throw new IllegalArgumentException("Family name cannot be null or empty");
-    }
-    if (!hasFamily(family.getName())) {
-      throw new IllegalArgumentException("Column family '" + family.getNameAsString()
-        + "' does not exist");
-    }
-    this.families.put(family.getName(), family);
+    delegatee.modifyFamily(family);
     return this;
   }
 
@@ -883,8 +471,9 @@
    * @param familyName Family name or column name.
    * @return true if the table contains the specified family name
    */
+  @Override
   public boolean hasFamily(final byte [] familyName) {
-    return families.containsKey(familyName);
+    return delegatee.hasFamily(familyName);
   }
 
   /**
@@ -894,13 +483,7 @@
    */
   @Override
   public String toString() {
-    StringBuilder s = new StringBuilder();
-    s.append('\'').append(Bytes.toString(name.getName())).append('\'');
-    s.append(getValues(true));
-    for (HColumnDescriptor f : families.values()) {
-      s.append(", ").append(f);
-    }
-    return s.toString();
+    return delegatee.toString();
   }
 
   /**
@@ -908,109 +491,14 @@
    * descriptors (with only the non-default column family attributes)
    */
   public String toStringCustomizedValues() {
-    StringBuilder s = new StringBuilder();
-    s.append('\'').append(Bytes.toString(name.getName())).append('\'');
-    s.append(getValues(false));
-    for(HColumnDescriptor hcd : families.values()) {
-      s.append(", ").append(hcd.toStringCustomizedValues());
-    }
-    return s.toString();
+    return delegatee.toStringCustomizedValues();
   }
 
   /**
    * @return map of all table attributes formatted into string.
    */
   public String toStringTableAttributes() {
-   return getValues(true).toString();
-  }
-
-  private StringBuilder getValues(boolean printDefaults) {
-    StringBuilder s = new StringBuilder();
-
-    // step 1: set partitioning and pruning
-    Set<Bytes> reservedKeys = new TreeSet<>();
-    Set<Bytes> userKeys = new TreeSet<>();
-    for (Map.Entry<Bytes, Bytes> entry : values.entrySet()) {
-      if (entry.getKey() == null || entry.getKey().get() == null) continue;
-      String key = Bytes.toString(entry.getKey().get());
-      // in this section, print out reserved keywords + coprocessor info
-      if (!RESERVED_KEYWORDS.contains(entry.getKey()) && !key.startsWith("coprocessor$")) {
-        userKeys.add(entry.getKey());
-        continue;
-      }
-      // only print out IS_ROOT/IS_META if true
-      String value = Bytes.toString(entry.getValue().get());
-      if (key.equalsIgnoreCase(IS_ROOT) || key.equalsIgnoreCase(IS_META)) {
-        if (Boolean.valueOf(value) == false) continue;
-      }
-      // see if a reserved key is a default value. may not want to print it out
-      if (printDefaults
-          || !DEFAULT_VALUES.containsKey(key)
-          || !DEFAULT_VALUES.get(key).equalsIgnoreCase(value)) {
-        reservedKeys.add(entry.getKey());
-      }
-    }
-
-    // early exit optimization
-    boolean hasAttributes = !reservedKeys.isEmpty() || !userKeys.isEmpty();
-    if (!hasAttributes && configuration.isEmpty()) return s;
-
-    s.append(", {");
-    // step 2: printing attributes
-    if (hasAttributes) {
-      s.append("TABLE_ATTRIBUTES => {");
-
-      // print all reserved keys first
-      boolean printCommaForAttr = false;
-      for (Bytes k : reservedKeys) {
-        String key = Bytes.toString(k.get());
-        String value = Bytes.toStringBinary(values.get(k).get());
-        if (printCommaForAttr) s.append(", ");
-        printCommaForAttr = true;
-        s.append(key);
-        s.append(" => ");
-        s.append('\'').append(value).append('\'');
-      }
-
-      if (!userKeys.isEmpty()) {
-        // print all non-reserved, advanced config keys as a separate subset
-        if (printCommaForAttr) s.append(", ");
-        printCommaForAttr = true;
-        s.append(HConstants.METADATA).append(" => ");
-        s.append("{");
-        boolean printCommaForCfg = false;
-        for (Bytes k : userKeys) {
-          String key = Bytes.toString(k.get());
-          String value = Bytes.toStringBinary(values.get(k).get());
-          if (printCommaForCfg) s.append(", ");
-          printCommaForCfg = true;
-          s.append('\'').append(key).append('\'');
-          s.append(" => ");
-          s.append('\'').append(value).append('\'');
-        }
-        s.append("}");
-      }
-    }
-
-    // step 3: printing all configuration:
-    if (!configuration.isEmpty()) {
-      if (hasAttributes) {
-        s.append(", ");
-      }
-      s.append(HConstants.CONFIGURATION).append(" => ");
-      s.append('{');
-      boolean printCommaForConfig = false;
-      for (Map.Entry<String, String> e : configuration.entrySet()) {
-        if (printCommaForConfig) s.append(", ");
-        printCommaForConfig = true;
-        s.append('\'').append(e.getKey()).append('\'');
-        s.append(" => ");
-        s.append('\'').append(e.getValue()).append('\'');
-      }
-      s.append("}");
-    }
-    s.append("}"); // end METHOD
-    return s;
+   return delegatee.toStringTableAttributes();
   }
 
   /**
@@ -1041,15 +529,7 @@
    */
   @Override
   public int hashCode() {
-    int result = this.name.hashCode();
-    if (this.families.size() > 0) {
-      for (HColumnDescriptor e: this.families.values()) {
-        result ^= e.hashCode();
-      }
-    }
-    result ^= values.hashCode();
-    result ^= configuration.hashCode();
-    return result;
+    return delegatee.hashCode();
   }
 
   // Comparable
@@ -1063,39 +543,7 @@
    */
   @Override
   public int compareTo(final HTableDescriptor other) {
-    int result = this.name.compareTo(other.name);
-    if (result == 0) {
-      result = families.size() - other.families.size();
-    }
-    if (result == 0 && families.size() != other.families.size()) {
-      result = Integer.valueOf(families.size()).compareTo(
-          Integer.valueOf(other.families.size()));
-    }
-    if (result == 0) {
-      for (Iterator<HColumnDescriptor> it = families.values().iterator(),
-          it2 = other.families.values().iterator(); it.hasNext(); ) {
-        result = it.next().compareTo(it2.next());
-        if (result != 0) {
-          break;
-        }
-      }
-    }
-    if (result == 0) {
-      // punt on comparison for ordering, just calculate difference
-      result = this.values.hashCode() - other.values.hashCode();
-      if (result < 0)
-        result = -1;
-      else if (result > 0)
-        result = 1;
-    }
-    if (result == 0) {
-      result = this.configuration.hashCode() - other.configuration.hashCode();
-      if (result < 0)
-        result = -1;
-      else if (result > 0)
-        result = 1;
-    }
-    return result;
+    return delegatee.compareTo(other.delegatee);
   }
 
   /**
@@ -1105,35 +553,25 @@
    * @return Immutable collection of {@link HColumnDescriptor} of all the
    * column families.
    */
+  @Override
   public Collection<HColumnDescriptor> getFamilies() {
-    return Collections.unmodifiableCollection(this.families.values());
+    return delegatee.getFamilies();
   }
 
   /**
    * Return true if there are at least one cf whose replication scope is serial.
    */
+  @Override
   public boolean hasSerialReplicationScope() {
-    for (HColumnDescriptor column: getFamilies()){
-      if (column.getScope() == HConstants.REPLICATION_SCOPE_SERIAL){
-        return true;
-      }
-    }
-    return false;
+    return delegatee.hasSerialReplicationScope();
   }
 
   /**
    * Returns the configured replicas per region
    */
+  @Override
   public int getRegionReplication() {
-    return getIntValue(REGION_REPLICATION_KEY, DEFAULT_REGION_REPLICATION);
-  }
-
-  private int getIntValue(Bytes key, int defaultVal) {
-    byte[] val = getValue(key);
-    if (val == null || val.length == 0) {
-      return defaultVal;
-    }
-    return Integer.parseInt(Bytes.toString(val));
+    return delegatee.getRegionReplication();
   }
 
   /**
@@ -1141,16 +579,16 @@
    * @param regionReplication the replication factor per region
    */
   public HTableDescriptor setRegionReplication(int regionReplication) {
-    setValue(REGION_REPLICATION_KEY,
-        new Bytes(Bytes.toBytes(Integer.toString(regionReplication))));
+    delegatee.setRegionReplication(regionReplication);
     return this;
   }
 
   /**
    * @return true if the read-replicas memstore replication is enabled.
    */
+  @Override
   public boolean hasRegionMemstoreReplication() {
-    return isSomething(REGION_MEMSTORE_REPLICATION_KEY, DEFAULT_REGION_MEMSTORE_REPLICATION);
+    return delegatee.hasRegionMemstoreReplication();
   }
 
   /**
@@ -1163,21 +601,18 @@
    *                                  data only when the primary flushes the memstore.
    */
   public HTableDescriptor setRegionMemstoreReplication(boolean memstoreReplication) {
-    setValue(REGION_MEMSTORE_REPLICATION_KEY, memstoreReplication ? TRUE : FALSE);
-    // If the memstore replication is setup, we do not have to wait for observing a flush event
-    // from primary before starting to serve reads, because gaps from replication is not applicable
-    setConfiguration(RegionReplicaUtil.REGION_REPLICA_WAIT_FOR_PRIMARY_FLUSH_CONF_KEY,
-      Boolean.toString(memstoreReplication));
+    delegatee.setRegionMemstoreReplication(memstoreReplication);
     return this;
   }
 
   public HTableDescriptor setPriority(int priority) {
-    setValue(PRIORITY_KEY, Integer.toString(priority));
+    delegatee.setPriority(priority);
     return this;
   }
 
+  @Override
   public int getPriority() {
-    return getIntValue(PRIORITY_KEY, DEFAULT_PRIORITY);
+    return delegatee.getPriority();
   }
 
   /**
@@ -1188,8 +623,9 @@
    *
    * @return Immutable sorted set of the keys of the families.
    */
+  @Override
   public Set<byte[]> getFamiliesKeys() {
-    return Collections.unmodifiableSet(this.families.keySet());
+    return delegatee.getFamiliesKeys();
   }
 
   /**
@@ -1197,8 +633,9 @@
    *
    * @return Count of column families of the table
    */
+  @Override
   public int getColumnFamilyCount() {
-    return families.size();
+    return delegatee.getColumnFamilyCount();
   }
 
   /**
@@ -1209,9 +646,9 @@
    *
    * @see #getFamilies()
    */
+  @Override
   public HColumnDescriptor[] getColumnFamilies() {
-    Collection<HColumnDescriptor> hColumnDescriptors = getFamilies();
-    return hColumnDescriptors.toArray(new HColumnDescriptor[hColumnDescriptors.size()]);
+    return delegatee.getColumnFamilies();
   }
 
 
@@ -1223,8 +660,9 @@
    * @return Column descriptor for the passed family name or the family on
    * passed in column.
    */
+  @Override
   public HColumnDescriptor getFamily(final byte [] column) {
-    return this.families.get(column);
+    return delegatee.getFamily(column);
   }
 
 
@@ -1237,7 +675,7 @@
    * passed in column.
    */
   public HColumnDescriptor removeFamily(final byte [] column) {
-    return this.families.remove(column);
+    return delegatee.removeFamily(column);
   }
 
   /**
@@ -1251,7 +689,7 @@
    * @throws IOException
    */
   public HTableDescriptor addCoprocessor(String className) throws IOException {
-    addCoprocessor(className, null, Coprocessor.PRIORITY_USER, null);
+    delegatee.addCoprocessor(className);
     return this;
   }
 
@@ -1272,32 +710,8 @@
   public HTableDescriptor addCoprocessor(String className, Path jarFilePath,
                              int priority, final Map<String, String> kvs)
   throws IOException {
-    checkHasCoprocessor(className);
-
-    // Validate parameter kvs and then add key/values to kvString.
-    StringBuilder kvString = new StringBuilder();
-    if (kvs != null) {
-      for (Map.Entry<String, String> e: kvs.entrySet()) {
-        if (!e.getKey().matches(HConstants.CP_HTD_ATTR_VALUE_PARAM_KEY_PATTERN)) {
-          throw new IOException("Illegal parameter key = " + e.getKey());
-        }
-        if (!e.getValue().matches(HConstants.CP_HTD_ATTR_VALUE_PARAM_VALUE_PATTERN)) {
-          throw new IOException("Illegal parameter (" + e.getKey() +
-              ") value = " + e.getValue());
-        }
-        if (kvString.length() != 0) {
-          kvString.append(',');
-        }
-        kvString.append(e.getKey());
-        kvString.append('=');
-        kvString.append(e.getValue());
-      }
-    }
-
-    String value = ((jarFilePath == null)? "" : jarFilePath.toString()) +
-        "|" + className + "|" + Integer.toString(priority) + "|" +
-        kvString.toString();
-    return addCoprocessorToMap(value);
+    delegatee.addCoprocessor(className, jarFilePath, priority, kvs);
+    return this;
   }
 
   /**
@@ -1312,42 +726,7 @@
    * @throws IOException
    */
   public HTableDescriptor addCoprocessorWithSpec(final String specStr) throws IOException {
-    String className = getCoprocessorClassNameFromSpecStr(specStr);
-    if (className == null) {
-      throw new IllegalArgumentException("Format does not match " +
-        HConstants.CP_HTD_ATTR_VALUE_PATTERN + ": " + specStr);
-    }
-    checkHasCoprocessor(className);
-    return addCoprocessorToMap(specStr);
-  }
-
-  private void checkHasCoprocessor(final String className) throws IOException {
-    if (hasCoprocessor(className)) {
-      throw new IOException("Coprocessor " + className + " already exists.");
-    }
-  }
-
-  /**
-   * Add coprocessor to values Map
-   * @param specStr The Coprocessor specification all in in one String formatted so matches
-   * {@link HConstants#CP_HTD_ATTR_VALUE_PATTERN}
-   * @return Returns <code>this</code>
-   */
-  private HTableDescriptor addCoprocessorToMap(final String specStr) {
-    if (specStr == null) return this;
-    // generate a coprocessor key
-    int maxCoprocessorNumber = 0;
-    Matcher keyMatcher;
-    for (Map.Entry<Bytes, Bytes> e: this.values.entrySet()) {
-      keyMatcher = HConstants.CP_HTD_ATTR_KEY_PATTERN.matcher(Bytes.toString(e.getKey().get()));
-      if (!keyMatcher.matches()) {
-        continue;
-      }
-      maxCoprocessorNumber = Math.max(Integer.parseInt(keyMatcher.group(1)), maxCoprocessorNumber);
-    }
-    maxCoprocessorNumber++;
-    String key = "coprocessor$" + Integer.toString(maxCoprocessorNumber);
-    this.values.put(new Bytes(Bytes.toBytes(key)), new Bytes(Bytes.toBytes(specStr)));
+    delegatee.addCoprocessorWithSpec(specStr);
     return this;
   }
 
@@ -1357,23 +736,9 @@
    * @param classNameToMatch - Class name of the co-processor
    * @return true of the table has a co-processor className
    */
+  @Override
   public boolean hasCoprocessor(String classNameToMatch) {
-    Matcher keyMatcher;
-    for (Map.Entry<Bytes, Bytes> e :
-        this.values.entrySet()) {
-      keyMatcher =
-          HConstants.CP_HTD_ATTR_KEY_PATTERN.matcher(
-              Bytes.toString(e.getKey().get()));
-      if (!keyMatcher.matches()) {
-        continue;
-      }
-      String className = getCoprocessorClassNameFromSpecStr(Bytes.toString(e.getValue().get()));
-      if (className == null) continue;
-      if (className.equals(classNameToMatch.trim())) {
-        return true;
-      }
-    }
-    return false;
+    return delegatee.hasCoprocessor(classNameToMatch);
   }
 
   /**
@@ -1381,29 +746,9 @@
    *
    * @return The list of co-processors classNames
    */
+  @Override
   public List<String> getCoprocessors() {
-    List<String> result = new ArrayList<>(this.values.entrySet().size());
-    Matcher keyMatcher;
-    for (Map.Entry<Bytes, Bytes> e : this.values.entrySet()) {
-      keyMatcher = HConstants.CP_HTD_ATTR_KEY_PATTERN.matcher(Bytes.toString(e.getKey().get()));
-      if (!keyMatcher.matches()) {
-        continue;
-      }
-      String className = getCoprocessorClassNameFromSpecStr(Bytes.toString(e.getValue().get()));
-      if (className == null) continue;
-      result.add(className); // classname is the 2nd field
-    }
-    return result;
-  }
-
-  /**
-   * @param spec String formatted as per {@link HConstants#CP_HTD_ATTR_VALUE_PATTERN}
-   * @return Class parsed from passed in <code>spec</code> or null if no match or classpath found
-   */
-  private static String getCoprocessorClassNameFromSpecStr(final String spec) {
-    Matcher matcher = HConstants.CP_HTD_ATTR_VALUE_PATTERN.matcher(spec);
-    // Classname is the 2nd field
-    return matcher != null && matcher.matches()? matcher.group(2).trim(): null;
+    return delegatee.getCoprocessors();
   }
 
   /**
@@ -1411,97 +756,34 @@
    * @param className Class name of the co-processor
    */
   public void removeCoprocessor(String className) {
-    Bytes match = null;
-    Matcher keyMatcher;
-    Matcher valueMatcher;
-    for (Map.Entry<Bytes, Bytes> e : this.values
-        .entrySet()) {
-      keyMatcher = HConstants.CP_HTD_ATTR_KEY_PATTERN.matcher(Bytes.toString(e
-          .getKey().get()));
-      if (!keyMatcher.matches()) {
-        continue;
-      }
-      valueMatcher = HConstants.CP_HTD_ATTR_VALUE_PATTERN.matcher(Bytes
-          .toString(e.getValue().get()));
-      if (!valueMatcher.matches()) {
-        continue;
-      }
-      // get className and compare
-      String clazz = valueMatcher.group(2).trim(); // classname is the 2nd field
-      // remove the CP if it is present
-      if (clazz.equals(className.trim())) {
-        match = e.getKey();
-        break;
-      }
-    }
-    // if we found a match, remove it
-    if (match != null)
-      remove(match);
+    delegatee.removeCoprocessor(className);
   }
 
-  /**
-   * Returns the {@link Path} object representing the table directory under
-   * path rootdir
-   *
-   * Deprecated use FSUtils.getTableDir() instead.
-   *
-   * @param rootdir qualified path of HBase root directory
-   * @param tableName name of table
-   * @return {@link Path} for table
-   */
-  @Deprecated
-  public static Path getTableDir(Path rootdir, final byte [] tableName) {
-    //This is bad I had to mirror code from FSUTils.getTableDir since
-    //there is no module dependency between hbase-client and hbase-server
-    TableName name = TableName.valueOf(tableName);
-    return new Path(rootdir, new Path(HConstants.BASE_NAMESPACE_DIR,
-              new Path(name.getNamespaceAsString(), new Path(name.getQualifierAsString()))));
-  }
-
-  public final static String NAMESPACE_FAMILY_INFO = "info";
-  public final static byte[] NAMESPACE_FAMILY_INFO_BYTES = Bytes.toBytes(NAMESPACE_FAMILY_INFO);
-  public final static byte[] NAMESPACE_COL_DESC_BYTES = Bytes.toBytes("d");
+  public final static String NAMESPACE_FAMILY_INFO = TableDescriptorBuilder.NAMESPACE_FAMILY_INFO;
+  public final static byte[] NAMESPACE_FAMILY_INFO_BYTES = TableDescriptorBuilder.NAMESPACE_FAMILY_INFO_BYTES;
+  public final static byte[] NAMESPACE_COL_DESC_BYTES = TableDescriptorBuilder.NAMESPACE_COL_DESC_BYTES;
 
   /** Table descriptor for namespace table */
-  public static final HTableDescriptor NAMESPACE_TABLEDESC = new HTableDescriptor(
-      TableName.NAMESPACE_TABLE_NAME,
-      new HColumnDescriptor[] {
-          new HColumnDescriptor(NAMESPACE_FAMILY_INFO)
-              // Ten is arbitrary number.  Keep versions to help debugging.
-              .setMaxVersions(10)
-              .setInMemory(true)
-              .setBlocksize(8 * 1024)
-              .setScope(HConstants.REPLICATION_SCOPE_LOCAL)
-              // Enable cache of data blocks in L1 if more than one caching tier deployed:
-              // e.g. if using CombinedBlockCache (BucketCache).
-              .setCacheDataInL1(true)
-      });
+  public static final HTableDescriptor NAMESPACE_TABLEDESC
+    = new HTableDescriptor(TableDescriptorBuilder.NAMESPACE_TABLEDESC);
 
   @Deprecated
   public HTableDescriptor setOwner(User owner) {
-    return setOwnerString(owner != null ? owner.getShortName() : null);
+    delegatee.setOwner(owner);
+    return this;
   }
 
   // used by admin.rb:alter(table_name,*args) to update owner.
   @Deprecated
   public HTableDescriptor setOwnerString(String ownerString) {
-    if (ownerString != null) {
-      setValue(OWNER_KEY, ownerString);
-    } else {
-      remove(OWNER_KEY);
-    }
+    delegatee.setOwnerString(ownerString);
     return this;
   }
 
+  @Override
   @Deprecated
   public String getOwnerString() {
-    if (getValue(OWNER_KEY) != null) {
-      return Bytes.toString(getValue(OWNER_KEY));
-    }
-    // Note that every table should have an owner (i.e. should have OWNER_KEY set).
-    // hbase:meta and -ROOT- should return system user as owner, not null (see
-    // MasterFileSystem.java:bootstrap()).
-    return null;
+    return delegatee.getOwnerString();
   }
 
   /**
@@ -1509,7 +791,7 @@
    * @see #parseFrom(byte[])
    */
   public byte[] toByteArray() {
-    return ProtobufUtil.prependPBMagic(ProtobufUtil.convertToTableSchema(this).toByteArray());
+    return delegatee.toByteArray();
   }
 
   /**
@@ -1521,54 +803,39 @@
    */
   public static HTableDescriptor parseFrom(final byte [] bytes)
   throws DeserializationException, IOException {
-    if (!ProtobufUtil.isPBMagicPrefix(bytes)) {
-      throw new DeserializationException("Expected PB encoded HTableDescriptor");
-    }
-    int pblen = ProtobufUtil.lengthOfPBMagic();
-    TableSchema.Builder builder = TableSchema.newBuilder();
-    TableSchema ts;
-    try {
-      ProtobufUtil.mergeFrom(builder, bytes, pblen, bytes.length - pblen);
-      ts = builder.build();
-    } catch (IOException e) {
-      throw new DeserializationException(e);
-    }
-    return ProtobufUtil.convertToHTableDesc(ts);
+    return new HTableDescriptor(ModifyableTableDescriptor.parseFrom(bytes));
   }
 
   /**
    * Getter for accessing the configuration value by key
    */
+  @Override
   public String getConfigurationValue(String key) {
-    return configuration.get(key);
+    return delegatee.getConfigurationValue(key);
   }
 
   /**
-   * Getter for fetching an unmodifiable {@link #configuration} map.
+   * Getter for fetching an unmodifiable map.
    */
+  @Override
   public Map<String, String> getConfiguration() {
-    // shallow pointer copy
-    return Collections.unmodifiableMap(configuration);
+    return delegatee.getConfiguration();
   }
 
   /**
-   * Setter for storing a configuration setting in {@link #configuration} map.
+   * Setter for storing a configuration setting in map.
    * @param key Config key. Same as XML config key e.g. hbase.something.or.other.
    * @param value String value. If null, removes the setting.
    */
   public HTableDescriptor setConfiguration(String key, String value) {
-    if (value == null) {
-      removeConfiguration(key);
-    } else {
-      configuration.put(key, value);
-    }
+    delegatee.setConfiguration(key, value);
     return this;
   }
 
   /**
-   * Remove a config setting represented by the key from the {@link #configuration} map
+   * Remove a config setting represented by the key from the map
    */
   public void removeConfiguration(final String key) {
-    configuration.remove(key);
+    delegatee.removeConfiguration(key);
   }
 }
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
index 3e767d2..decf81f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
@@ -95,7 +95,7 @@
   /**
    * List all the userspace tables.
    *
-   * @return - returns an array of HTableDescriptors
+   * @return - returns an array of read-only HTableDescriptors
    * @throws IOException if a remote or network exception occurs
    */
   HTableDescriptor[] listTables() throws IOException;
@@ -104,7 +104,7 @@
    * List all the userspace tables matching the given pattern.
    *
    * @param pattern The compiled regular expression to match against
-   * @return - returns an array of HTableDescriptors
+   * @return - returns an array of read-only HTableDescriptors
    * @throws IOException if a remote or network exception occurs
    * @see #listTables()
    */
@@ -125,7 +125,7 @@
    *
    * @param pattern The compiled regular expression to match against
    * @param includeSysTables False to match only against userspace tables
-   * @return - returns an array of HTableDescriptors
+   * @return - returns an array of read-only HTableDescriptors
    * @throws IOException if a remote or network exception occurs
    * @see #listTables()
    */
@@ -137,7 +137,7 @@
    *
    * @param regex The regular expression to match against
    * @param includeSysTables False to match only against userspace tables
-   * @return - returns an array of HTableDescriptors
+   * @return - returns an array of read-only HTableDescriptors
    * @throws IOException if a remote or network exception occurs
    * @see #listTables(java.util.regex.Pattern, boolean)
    */
@@ -192,7 +192,7 @@
    * Method for getting the tableDescriptor
    *
    * @param tableName as a {@link TableName}
-   * @return the tableDescriptor
+   * @return the read-only tableDescriptor
    * @throws org.apache.hadoop.hbase.TableNotFoundException
    * @throws IOException if a remote or network exception occurs
    */
@@ -293,7 +293,8 @@
    * #listTables(java.lang.String)} and {@link #deleteTable(org.apache.hadoop.hbase.TableName)}
    *
    * @param regex The regular expression to match table names against
-   * @return Table descriptors for tables that couldn't be deleted
+   * @return Table descriptors for tables that couldn't be deleted.
+   *         The return htds are read-only
    * @throws IOException
    * @see #deleteTables(java.util.regex.Pattern)
    * @see #deleteTable(org.apache.hadoop.hbase.TableName)
@@ -308,6 +309,7 @@
    *
    * @param pattern The pattern to match table names against
    * @return Table descriptors for tables that couldn't be deleted
+   *         The return htds are read-only
    * @throws IOException
    */
   HTableDescriptor[] deleteTables(Pattern pattern) throws IOException;
@@ -373,6 +375,8 @@
    *
    * @param regex The regular expression to match table names against
    * @throws IOException
+   * @return Table descriptors for tables that couldn't be enabled.
+   *         The return HTDs are read-only.
    * @see #enableTables(java.util.regex.Pattern)
    * @see #enableTable(org.apache.hadoop.hbase.TableName)
    */
@@ -386,6 +390,8 @@
    *
    * @param pattern The pattern to match table names against
    * @throws IOException
+   * @return Table descriptors for tables that couldn't be enabled.
+   *         The return HTDs are read-only.
    */
   HTableDescriptor[] enableTables(Pattern pattern) throws IOException;
 
@@ -422,6 +428,7 @@
    *
    * @param regex The regular expression to match table names against
    * @return Table descriptors for tables that couldn't be disabled
+   *         The return htds are read-only
    * @throws IOException
    * @see #disableTables(java.util.regex.Pattern)
    * @see #disableTable(org.apache.hadoop.hbase.TableName)
@@ -436,6 +443,7 @@
    *
    * @param pattern The pattern to match table names against
    * @return Table descriptors for tables that couldn't be disabled
+   *         The return htds are read-only
    * @throws IOException
    */
   HTableDescriptor[] disableTables(Pattern pattern) throws IOException;
@@ -1166,7 +1174,7 @@
    * Get list of table descriptors by namespace
    *
    * @param name namespace name
-   * @return A descriptor
+   * @return HTD[] the read-only tableDescriptors
    * @throws IOException
    */
   HTableDescriptor[] listTableDescriptorsByNamespace(final String name)
@@ -1199,7 +1207,7 @@
    * Get tableDescriptors
    *
    * @param tableNames List of table names
-   * @return HTD[] the tableDescriptor
+   * @return HTD[] the read-only tableDescriptors
    * @throws IOException if a remote or network exception occurs
    */
   HTableDescriptor[] getTableDescriptorsByTableName(List<TableName> tableNames)
@@ -1209,7 +1217,7 @@
    * Get tableDescriptors
    *
    * @param names List of table names
-   * @return HTD[] the tableDescriptor
+   * @return HTD[] the read-only tableDescriptors
    * @throws IOException if a remote or network exception occurs
    */
   HTableDescriptor[] getTableDescriptors(List<String> names)
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
index b7c60dd..3343c7a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
@@ -25,7 +25,6 @@
 
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.TableName;
@@ -57,27 +56,27 @@
 
   /**
    * List all the userspace tables.
-   * @return - returns an array of HTableDescriptors wrapped by a {@link CompletableFuture}.
+   * @return - returns an array of TableDescriptors wrapped by a {@link CompletableFuture}.
    * @see #listTables(Pattern, boolean)
    */
-  CompletableFuture<HTableDescriptor[]> listTables();
+  CompletableFuture<TableDescriptor[]> listTables();
 
   /**
    * List all the tables matching the given pattern.
    * @param regex The regular expression to match against
    * @param includeSysTables False to match only against userspace tables
-   * @return - returns an array of HTableDescriptors wrapped by a {@link CompletableFuture}.
+   * @return - returns an array of TableDescriptors wrapped by a {@link CompletableFuture}.
    * @see #listTables(Pattern, boolean)
    */
-  CompletableFuture<HTableDescriptor[]> listTables(String regex, boolean includeSysTables);
+  CompletableFuture<TableDescriptor[]> listTables(String regex, boolean includeSysTables);
 
   /**
    * List all the tables matching the given pattern.
    * @param pattern The compiled regular expression to match against
    * @param includeSysTables False to match only against userspace tables
-   * @return - returns an array of HTableDescriptors wrapped by a {@link CompletableFuture}.
+   * @return - returns an array of TableDescriptors wrapped by a {@link CompletableFuture}.
    */
-  CompletableFuture<HTableDescriptor[]> listTables(Pattern pattern, boolean includeSysTables);
+  CompletableFuture<TableDescriptor[]> listTables(Pattern pattern, boolean includeSysTables);
 
   /**
    * List all of the names of userspace tables.
@@ -107,15 +106,15 @@
   /**
    * Method for getting the tableDescriptor
    * @param tableName as a {@link TableName}
-   * @return the tableDescriptor wrapped by a {@link CompletableFuture}.
+   * @return the read-only tableDescriptor wrapped by a {@link CompletableFuture}.
    */
-  CompletableFuture<HTableDescriptor> getTableDescriptor(final TableName tableName);
+  CompletableFuture<TableDescriptor> getTableDescriptor(final TableName tableName);
 
   /**
    * Creates a new table.
    * @param desc table descriptor for table
    */
-  CompletableFuture<Void> createTable(HTableDescriptor desc);
+  CompletableFuture<Void> createTable(TableDescriptor desc);
 
   /**
    * Creates a new table with the specified number of regions. The start key specified will become
@@ -128,7 +127,7 @@
    * @param endKey end of key range
    * @param numRegions the total number of regions to create
    */
-  CompletableFuture<Void> createTable(HTableDescriptor desc, byte[] startKey, byte[] endKey,
+  CompletableFuture<Void> createTable(TableDescriptor desc, byte[] startKey, byte[] endKey,
       int numRegions);
 
   /**
@@ -138,7 +137,7 @@
    * @param desc table descriptor for table
    * @param splitKeys array of split keys for the initial regions of the table
    */
-  CompletableFuture<Void> createTable(final HTableDescriptor desc, byte[][] splitKeys);
+  CompletableFuture<Void> createTable(final TableDescriptor desc, byte[][] splitKeys);
 
   /**
    * Deletes a table.
@@ -153,9 +152,9 @@
    * {@link #deleteTable(org.apache.hadoop.hbase.TableName)}
    * @param regex The regular expression to match table names against
    * @return Table descriptors for tables that couldn't be deleted. The return value will be wrapped
-   *         by a {@link CompletableFuture}.
+   *         by a {@link CompletableFuture}. The return HTDs are read-only.
    */
-  CompletableFuture<HTableDescriptor[]> deleteTables(String regex);
+  CompletableFuture<TableDescriptor[]> deleteTables(String regex);
 
   /**
    * Delete tables matching the passed in pattern and wait on completion. Warning: Use this method
@@ -164,9 +163,9 @@
    * {@link #deleteTable(org.apache.hadoop.hbase.TableName)}
    * @param pattern The pattern to match table names against
    * @return Table descriptors for tables that couldn't be deleted. The return value will be wrapped
-   *         by a {@link CompletableFuture}.
+   *         by a {@link CompletableFuture}. The return HTDs are read-only.
    */
-  CompletableFuture<HTableDescriptor[]> deleteTables(Pattern pattern);
+  CompletableFuture<TableDescriptor[]> deleteTables(Pattern pattern);
 
   /**
    * Truncate a table.
@@ -187,9 +186,9 @@
    * {@link #enableTable(TableName)}
    * @param regex The regular expression to match table names against
    * @return Table descriptors for tables that couldn't be enabled. The return value will be wrapped
-   *         by a {@link CompletableFuture}.
+   *         by a {@link CompletableFuture}. The return HTDs are read-only.
    */
-  CompletableFuture<HTableDescriptor[]> enableTables(String regex);
+  CompletableFuture<TableDescriptor[]> enableTables(String regex);
 
   /**
    * Enable tables matching the passed in pattern. Warning: Use this method carefully, there is no
@@ -197,9 +196,9 @@
    * {@link #enableTable(TableName)}
    * @param pattern The pattern to match table names against
    * @return Table descriptors for tables that couldn't be enabled. The return value will be wrapped
-   *         by a {@link CompletableFuture}.
+   *         by a {@link CompletableFuture}. The return HTDs are read-only.
    */
-  CompletableFuture<HTableDescriptor[]> enableTables(Pattern pattern);
+  CompletableFuture<TableDescriptor[]> enableTables(Pattern pattern);
 
   /**
    * Disable a table. The table has to be in enabled state for it to be disabled.
@@ -213,9 +212,9 @@
    * {@link #disableTable(TableName)}
    * @param regex The regular expression to match table names against
    * @return Table descriptors for tables that couldn't be disabled. The return value will be wrapped by a
-   *         {@link CompletableFuture}.
+   *         {@link CompletableFuture}. The return HTDs are read-only.
    */
-  CompletableFuture<HTableDescriptor[]> disableTables(String regex);
+  CompletableFuture<TableDescriptor[]> disableTables(String regex);
 
   /**
    * Disable tables matching the passed in pattern. Warning: Use this method carefully, there is no
@@ -223,9 +222,9 @@
    * {@link #disableTable(TableName)}
    * @param pattern The pattern to match table names against
    * @return Table descriptors for tables that couldn't be disabled. The return value will be wrapped by a
-   *         {@link CompletableFuture}.
+   *         {@link CompletableFuture}. The return HTDs are read-only.
    */
-  CompletableFuture<HTableDescriptor[]> disableTables(Pattern pattern);
+  CompletableFuture<TableDescriptor[]> disableTables(Pattern pattern);
 
   /**
    * @param tableName name of table to check
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
index 54e1e8b..a54cc7a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
@@ -44,7 +44,6 @@
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HRegionLocation;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.MetaTableAccessor.QueryType;
 import org.apache.hadoop.hbase.NotServingRegionException;
@@ -293,10 +292,10 @@
     CompletableFuture<Void> operate(TableName table);
   }
 
-  private CompletableFuture<HTableDescriptor[]> batchTableOperations(Pattern pattern,
+  private CompletableFuture<TableDescriptor[]> batchTableOperations(Pattern pattern,
       TableOperator operator, String operationType) {
-    CompletableFuture<HTableDescriptor[]> future = new CompletableFuture<>();
-    List<HTableDescriptor> failed = new LinkedList<>();
+    CompletableFuture<TableDescriptor[]> future = new CompletableFuture<>();
+    List<TableDescriptor> failed = new LinkedList<>();
     listTables(pattern, false).whenComplete(
       (tables, error) -> {
         if (error != null) {
@@ -311,7 +310,7 @@
               }
             })).<CompletableFuture> toArray(size -> new CompletableFuture[size]);
         CompletableFuture.allOf(futures).thenAccept((v) -> {
-          future.complete(failed.toArray(new HTableDescriptor[failed.size()]));
+          future.complete(failed.toArray(new TableDescriptor[failed.size()]));
         });
       });
     return future;
@@ -328,25 +327,25 @@
   }
 
   @Override
-  public CompletableFuture<HTableDescriptor[]> listTables() {
+  public CompletableFuture<TableDescriptor[]> listTables() {
     return listTables((Pattern) null, false);
   }
 
   @Override
-  public CompletableFuture<HTableDescriptor[]> listTables(String regex, boolean includeSysTables) {
+  public CompletableFuture<TableDescriptor[]> listTables(String regex, boolean includeSysTables) {
     return listTables(Pattern.compile(regex), false);
   }
 
   @Override
-  public CompletableFuture<HTableDescriptor[]> listTables(Pattern pattern, boolean includeSysTables) {
+  public CompletableFuture<TableDescriptor[]> listTables(Pattern pattern, boolean includeSysTables) {
     return this
-        .<HTableDescriptor[]>newMasterCaller()
+        .<TableDescriptor[]>newMasterCaller()
         .action(
           (controller, stub) -> this
-              .<GetTableDescriptorsRequest, GetTableDescriptorsResponse, HTableDescriptor[]> call(
+              .<GetTableDescriptorsRequest, GetTableDescriptorsResponse, TableDescriptor[]> call(
                 controller, stub, RequestConverter.buildGetTableDescriptorsRequest(pattern,
                   includeSysTables), (s, c, req, done) -> s.getTableDescriptors(c, req, done), (
-                    resp) -> ProtobufUtil.getHTableDescriptorArray(resp))).call();
+                    resp) -> ProtobufUtil.getTableDescriptorArray(resp))).call();
   }
 
   @Override
@@ -372,8 +371,8 @@
   }
 
   @Override
-  public CompletableFuture<HTableDescriptor> getTableDescriptor(TableName tableName) {
-    CompletableFuture<HTableDescriptor> future = new CompletableFuture<>();
+  public CompletableFuture<TableDescriptor> getTableDescriptor(TableName tableName) {
+    CompletableFuture<TableDescriptor> future = new CompletableFuture<>();
     this.<List<TableSchema>> newMasterCaller()
         .action(
           (controller, stub) -> this
@@ -386,7 +385,7 @@
             return;
           }
           if (!tableSchemas.isEmpty()) {
-            future.complete(ProtobufUtil.convertToHTableDesc(tableSchemas.get(0)));
+            future.complete(ProtobufUtil.convertToTableDesc(tableSchemas.get(0)));
           } else {
             future.completeExceptionally(new TableNotFoundException(tableName.getNameAsString()));
           }
@@ -395,12 +394,12 @@
   }
 
   @Override
-  public CompletableFuture<Void> createTable(HTableDescriptor desc) {
+  public CompletableFuture<Void> createTable(TableDescriptor desc) {
     return createTable(desc, null);
   }
 
   @Override
-  public CompletableFuture<Void> createTable(HTableDescriptor desc, byte[] startKey, byte[] endKey,
+  public CompletableFuture<Void> createTable(TableDescriptor desc, byte[] startKey, byte[] endKey,
       int numRegions) {
     try {
       return createTable(desc, getSplitKeys(startKey, endKey, numRegions));
@@ -410,7 +409,7 @@
   }
 
   @Override
-  public CompletableFuture<Void> createTable(HTableDescriptor desc, byte[][] splitKeys) {
+  public CompletableFuture<Void> createTable(TableDescriptor desc, byte[][] splitKeys) {
     if (desc.getTableName() == null) {
       return failedFuture(new IllegalArgumentException("TableName cannot be null"));
     }
@@ -447,12 +446,12 @@
   }
 
   @Override
-  public CompletableFuture<HTableDescriptor[]> deleteTables(String regex) {
+  public CompletableFuture<TableDescriptor[]> deleteTables(String regex) {
     return deleteTables(Pattern.compile(regex));
   }
 
   @Override
-  public CompletableFuture<HTableDescriptor[]> deleteTables(Pattern pattern) {
+  public CompletableFuture<TableDescriptor[]> deleteTables(Pattern pattern) {
     return batchTableOperations(pattern, (table) -> deleteTable(table), "DELETE");
   }
 
@@ -473,12 +472,12 @@
   }
 
   @Override
-  public CompletableFuture<HTableDescriptor[]> enableTables(String regex) {
+  public CompletableFuture<TableDescriptor[]> enableTables(String regex) {
     return enableTables(Pattern.compile(regex));
   }
 
   @Override
-  public CompletableFuture<HTableDescriptor[]> enableTables(Pattern pattern) {
+  public CompletableFuture<TableDescriptor[]> enableTables(Pattern pattern) {
     return batchTableOperations(pattern, (table) -> enableTable(table), "ENABLE");
   }
 
@@ -491,12 +490,12 @@
   }
 
   @Override
-  public CompletableFuture<HTableDescriptor[]> disableTables(String regex) {
+  public CompletableFuture<TableDescriptor[]> disableTables(String regex) {
     return disableTables(Pattern.compile(regex));
   }
 
   @Override
-  public CompletableFuture<HTableDescriptor[]> disableTables(Pattern pattern) {
+  public CompletableFuture<TableDescriptor[]> disableTables(Pattern pattern) {
     return batchTableOperations(pattern, (table) -> disableTable(table), "DISABLE");
   }
 
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
index 7e79c20..14af586 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
@@ -441,7 +441,7 @@
       }
     }, rpcCallerFactory, operationTimeout, rpcTimeout);
     if (htd != null) {
-      return htd;
+      return new ImmutableHTableDescriptor(htd);
     }
     throw new TableNotFoundException(tableName.getNameAsString());
   }
@@ -532,7 +532,7 @@
       super(admin, desc.getTableName(),
               (response != null && response.hasProcId()) ? response.getProcId() : null);
       this.splitKeys = splitKeys;
-      this.desc = desc;
+      this.desc = new ImmutableHTableDescriptor(desc);
     }
 
     @Override
@@ -2138,8 +2138,7 @@
                 .build()).getTableSchemaList();
         HTableDescriptor[] res = new HTableDescriptor[list.size()];
         for(int i=0; i < list.size(); i++) {
-
-          res[i] = ProtobufUtil.convertToHTableDesc(list.get(i));
+          res[i] = new ImmutableHTableDescriptor(ProtobufUtil.convertToHTableDesc(list.get(i)));
         }
         return res;
       }
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
index 3bdbed5..e89d4ed 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
@@ -266,7 +266,7 @@
     HTableDescriptor htd = HBaseAdmin.getTableDescriptor(tableName, connection, rpcCallerFactory,
       rpcControllerFactory, operationTimeout, readRpcTimeout);
     if (htd != null) {
-      return new UnmodifyableHTableDescriptor(htd);
+      return new ImmutableHTableDescriptor(htd);
     }
     return null;
   }
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ImmutableHTableDescriptor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ImmutableHTableDescriptor.java
new file mode 100644
index 0000000..89d1291
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ImmutableHTableDescriptor.java
@@ -0,0 +1,79 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.client;
+
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder.ModifyableTableDescriptor;
+import org.apache.hadoop.hbase.util.Bytes;
+
+/**
+ * Read-only table descriptor.
+ */
+@Deprecated // deprecated for hbase 2.0, remove for hbase 3.0. see HTableDescriptor.
+@InterfaceAudience.Public
+public class ImmutableHTableDescriptor extends HTableDescriptor {
+
+  /*
+   * Create an unmodifyable copy of an HTableDescriptor
+   * @param desc
+   */
+  public ImmutableHTableDescriptor(final HTableDescriptor desc) {
+    super(new UnmodifyableTableDescriptor(desc));
+  }
+
+  @Deprecated // deprecated for hbase 2.0, remove for hbase 3.0. see HTableDescriptor.
+  private static class UnmodifyableTableDescriptor extends ModifyableTableDescriptor {
+
+    UnmodifyableTableDescriptor(final TableDescriptor desc) {
+      super(desc);
+    }
+
+    @Override
+    protected ModifyableTableDescriptor setFamily(HColumnDescriptor family) {
+      throw new UnsupportedOperationException("HTableDescriptor is read-only");
+    }
+
+    @Override
+    public HColumnDescriptor removeFamily(final byte[] column) {
+      throw new UnsupportedOperationException("HTableDescriptor is read-only");
+    }
+
+    @Override
+    public ModifyableTableDescriptor setValue(final Bytes key, final Bytes value) {
+      throw new UnsupportedOperationException("HTableDescriptor is read-only");
+    }
+
+    @Override
+    public void remove(Bytes key) {
+      throw new UnsupportedOperationException("HTableDescriptor is read-only");
+    }
+
+    @Override
+    public ModifyableTableDescriptor setConfiguration(String key, String value) {
+      throw new UnsupportedOperationException("HTableDescriptor is read-only");
+    }
+
+    @Override
+    public void removeConfiguration(final String key) {
+      throw new UnsupportedOperationException("HTableDescriptor is read-only");
+    }
+  }
+}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptor.java
new file mode 100644
index 0000000..58a18ec
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptor.java
@@ -0,0 +1,256 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.client;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.Set;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.util.Bytes;
+
+
+/**
+ * TableDescriptor contains the details about an HBase table such as the descriptors of
+ * all the column families, is the table a catalog table, <code> -ROOT- </code> or
+ * <code> hbase:meta </code>, if the table is read only, the maximum size of the memstore,
+ * when the region split should occur, coprocessors associated with it etc...
+ */
+@InterfaceAudience.Public
+public interface TableDescriptor {
+
+  /**
+   * Returns an array all the {@link HColumnDescriptor} of the column families
+   * of the table.
+   *
+   * @return Array of all the HColumnDescriptors of the current table
+   *
+   * @see #getFamilies()
+   */
+  HColumnDescriptor[] getColumnFamilies();
+
+  /**
+   * Returns the count of the column families of the table.
+   *
+   * @return Count of column families of the table
+   */
+  int getColumnFamilyCount();
+
+  /**
+   * Getter for fetching an unmodifiable map.
+   *
+   * @return an unmodifiable map
+   */
+  Map<String, String> getConfiguration();
+
+  /**
+   * Getter for accessing the configuration value by key
+   *
+   * @param key the key whose associated value is to be returned
+   * @return the value to which the specified key is mapped, or {@code null} if
+   * this map contains no mapping for the key
+   */
+  String getConfigurationValue(String key);
+
+  /**
+   * Return the list of attached co-processor represented by their name
+   * className
+   *
+   * @return The list of co-processors classNames
+   */
+  Collection<String> getCoprocessors();
+
+  /**
+   * Returns the durability setting for the table.
+   *
+   * @return durability setting for the table.
+   */
+  Durability getDurability();
+
+  /**
+   * Returns an unmodifiable collection of all the {@link HColumnDescriptor} of
+   * all the column families of the table.
+   *
+   * @return Immutable collection of {@link HColumnDescriptor} of all the column
+   * families.
+   */
+  Collection<HColumnDescriptor> getFamilies();
+
+  /**
+   * Returns all the column family names of the current table. The map of
+   * TableDescriptor contains mapping of family name to HColumnDescriptors.
+   * This returns all the keys of the family map which represents the column
+   * family names of the table.
+   *
+   * @return Immutable sorted set of the keys of the families.
+   */
+  Set<byte[]> getFamiliesKeys();
+
+  /**
+   * Returns the HColumnDescriptor for a specific column family with name as
+   * specified by the parameter column.
+   *
+   * @param column Column family name
+   * @return Column descriptor for the passed family name or the family on
+   * passed in column.
+   */
+  HColumnDescriptor getFamily(final byte[] column);
+
+  /**
+   * This gets the class associated with the flush policy which determines the
+   * stores need to be flushed when flushing a region. The class used by default
+   * is defined in org.apache.hadoop.hbase.regionserver.FlushPolicy.
+   *
+   * @return the class name of the flush policy for this table. If this returns
+   * null, the default flush policy is used.
+   */
+  String getFlushPolicyClassName();
+
+  /**
+   * Returns the maximum size upto which a region can grow to after which a
+   * region split is triggered. The region size is represented by the size of
+   * the biggest store file in that region.
+   *
+   * @return max hregion size for table, -1 if not set.
+   */
+  long getMaxFileSize();
+
+  /**
+   * Returns the size of the memstore after which a flush to filesystem is
+   * triggered.
+   *
+   * @return memory cache flush size for each hregion, -1 if not set.
+   */
+  long getMemStoreFlushSize();
+
+  int getPriority();
+
+  /**
+   * @return Returns the configured replicas per region
+   */
+  int getRegionReplication();
+
+  /**
+   * This gets the class associated with the region split policy which
+   * determines when a region split should occur. The class used by default is
+   * defined in org.apache.hadoop.hbase.regionserver.RegionSplitPolicy
+   *
+   * @return the class name of the region split policy for this table. If this
+   * returns null, the default split policy is used.
+   */
+  String getRegionSplitPolicyClassName();
+
+  /**
+   * Get the name of the table
+   *
+   * @return TableName
+   */
+  TableName getTableName();
+
+  @Deprecated
+  String getOwnerString();
+
+  /**
+   * Getter for accessing the metadata associated with the key
+   *
+   * @param key The key.
+   * @return The value.
+   */
+  byte[] getValue(byte[] key);
+
+  /**
+   * @return Getter for fetching an unmodifiable map.
+   */
+  Map<Bytes, Bytes> getValues();
+
+  /**
+   * Check if the table has an attached co-processor represented by the name
+   * className
+   *
+   * @param classNameToMatch - Class name of the co-processor
+   * @return true of the table has a co-processor className
+   */
+  boolean hasCoprocessor(String classNameToMatch);
+
+  /**
+   * Checks to see if this table contains the given column family
+   *
+   * @param familyName Family name or column name.
+   * @return true if the table contains the specified family name
+   */
+  boolean hasFamily(final byte[] familyName);
+
+  /**
+   * @return true if the read-replicas memstore replication is enabled.
+   */
+  boolean hasRegionMemstoreReplication();
+
+  /**
+   * @return true if there are at least one cf whose replication scope is
+   * serial.
+   */
+  boolean hasSerialReplicationScope();
+
+  /**
+   * Check if the compaction enable flag of the table is true. If flag is false
+   * then no minor/major compactions will be done in real.
+   *
+   * @return true if table compaction enabled
+   */
+  boolean isCompactionEnabled();
+
+  /**
+   * Checks if this table is <code> hbase:meta </code> region.
+   *
+   * @return true if this table is <code> hbase:meta </code> region
+   */
+  boolean isMetaRegion();
+
+  /**
+   * Checks if the table is a <code>hbase:meta</code> table
+   *
+   * @return true if table is <code> hbase:meta </code> region.
+   */
+  boolean isMetaTable();
+
+  /**
+   * Check if normalization enable flag of the table is true. If flag is false
+   * then no region normalizer won't attempt to normalize this table.
+   *
+   * @return true if region normalization is enabled for this table
+   */
+  boolean isNormalizationEnabled();
+
+  /**
+   * Check if the readOnly flag of the table is set. If the readOnly flag is set
+   * then the contents of the table can only be read from but not modified.
+   *
+   * @return true if all columns in the table should be read only
+   */
+  boolean isReadOnly();
+
+  /**
+   * Check if the descriptor represents a <code> -ROOT- </code> region.
+   *
+   * @return true if this is a <code> -ROOT- </code> region
+   */
+  boolean isRootRegion();
+
+}
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
new file mode 100644
index 0000000..a372ced
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java
@@ -0,0 +1,1639 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.client;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.TreeSet;
+import java.util.function.Function;
+import java.util.regex.Matcher;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.Coprocessor;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
+import org.apache.hadoop.hbase.util.Bytes;
+
+@InterfaceAudience.Public
+public class TableDescriptorBuilder {
+
+  private static final Log LOG = LogFactory.getLog(TableDescriptorBuilder.class);
+
+  public static final String SPLIT_POLICY = "SPLIT_POLICY";
+
+  /**
+   * <em>INTERNAL</em> Used by HBase Shell interface to access this metadata
+   * attribute which denotes the maximum size of the store file after which a
+   * region split occurs.
+   */
+  public static final String MAX_FILESIZE = "MAX_FILESIZE";
+  private static final Bytes MAX_FILESIZE_KEY
+          = new Bytes(Bytes.toBytes(MAX_FILESIZE));
+
+  public static final String OWNER = "OWNER";
+  public static final Bytes OWNER_KEY
+          = new Bytes(Bytes.toBytes(OWNER));
+
+  /**
+   * <em>INTERNAL</em> Used by rest interface to access this metadata attribute
+   * which denotes if the table is Read Only.
+   */
+  public static final String READONLY = "READONLY";
+  private static final Bytes READONLY_KEY
+          = new Bytes(Bytes.toBytes(READONLY));
+
+  /**
+   * <em>INTERNAL</em> Used by HBase Shell interface to access this metadata
+   * attribute which denotes if the table is compaction enabled.
+   */
+  public static final String COMPACTION_ENABLED = "COMPACTION_ENABLED";
+  private static final Bytes COMPACTION_ENABLED_KEY
+          = new Bytes(Bytes.toBytes(COMPACTION_ENABLED));
+
+  /**
+   * <em>INTERNAL</em> Used by HBase Shell interface to access this metadata
+   * attribute which represents the maximum size of the memstore after which its
+   * contents are flushed onto the disk.
+   */
+  public static final String MEMSTORE_FLUSHSIZE = "MEMSTORE_FLUSHSIZE";
+  private static final Bytes MEMSTORE_FLUSHSIZE_KEY
+          = new Bytes(Bytes.toBytes(MEMSTORE_FLUSHSIZE));
+
+  public static final String FLUSH_POLICY = "FLUSH_POLICY";
+
+  /**
+   * <em>INTERNAL</em> Used by rest interface to access this metadata attribute
+   * which denotes if the table is a -ROOT- region or not.
+   */
+  public static final String IS_ROOT = "IS_ROOT";
+  private static final Bytes IS_ROOT_KEY
+          = new Bytes(Bytes.toBytes(IS_ROOT));
+
+  /**
+   * <em>INTERNAL</em> Used by rest interface to access this metadata attribute
+   * which denotes if it is a catalog table, either <code> hbase:meta </code> or <code> -ROOT-
+   * </code>.
+   */
+  public static final String IS_META = "IS_META";
+  private static final Bytes IS_META_KEY
+          = new Bytes(Bytes.toBytes(IS_META));
+
+  /**
+   * <em>INTERNAL</em> {@link Durability} setting for the table.
+   */
+  public static final String DURABILITY = "DURABILITY";
+  private static final Bytes DURABILITY_KEY
+          = new Bytes(Bytes.toBytes("DURABILITY"));
+
+  /**
+   * <em>INTERNAL</em> number of region replicas for the table.
+   */
+  public static final String REGION_REPLICATION = "REGION_REPLICATION";
+  private static final Bytes REGION_REPLICATION_KEY
+          = new Bytes(Bytes.toBytes(REGION_REPLICATION));
+
+  /**
+   * <em>INTERNAL</em> flag to indicate whether or not the memstore should be
+   * replicated for read-replicas (CONSISTENCY =&gt; TIMELINE).
+   */
+  public static final String REGION_MEMSTORE_REPLICATION = "REGION_MEMSTORE_REPLICATION";
+  private static final Bytes REGION_MEMSTORE_REPLICATION_KEY
+          = new Bytes(Bytes.toBytes(REGION_MEMSTORE_REPLICATION));
+
+  /**
+   * <em>INTERNAL</em> Used by shell/rest interface to access this metadata
+   * attribute which denotes if the table should be treated by region
+   * normalizer.
+   */
+  public static final String NORMALIZATION_ENABLED = "NORMALIZATION_ENABLED";
+  private static final Bytes NORMALIZATION_ENABLED_KEY
+          = new Bytes(Bytes.toBytes(NORMALIZATION_ENABLED));
+
+  /**
+   * Default durability for HTD is USE_DEFAULT, which defaults to HBase-global
+   * default value
+   */
+  private static final Durability DEFAULT_DURABLITY = Durability.USE_DEFAULT;
+
+  public static final String PRIORITY = "PRIORITY";
+  private static final Bytes PRIORITY_KEY
+          = new Bytes(Bytes.toBytes(PRIORITY));
+
+  /**
+   * Relative priority of the table used for rpc scheduling
+   */
+  private static final int DEFAULT_PRIORITY = HConstants.NORMAL_QOS;
+
+  /*
+     *  The below are ugly but better than creating them each time till we
+     *  replace booleans being saved as Strings with plain booleans.  Need a
+     *  migration script to do this.  TODO.
+   */
+  private static final Bytes FALSE
+          = new Bytes(Bytes.toBytes(Boolean.FALSE.toString()));
+
+  private static final Bytes TRUE
+          = new Bytes(Bytes.toBytes(Boolean.TRUE.toString()));
+
+  /**
+   * Constant that denotes whether the table is READONLY by default and is false
+   */
+  public static final boolean DEFAULT_READONLY = false;
+
+  /**
+   * Constant that denotes whether the table is compaction enabled by default
+   */
+  public static final boolean DEFAULT_COMPACTION_ENABLED = true;
+
+  /**
+   * Constant that denotes whether the table is normalized by default.
+   */
+  public static final boolean DEFAULT_NORMALIZATION_ENABLED = false;
+
+  /**
+   * Constant that denotes the maximum default size of the memstore after which
+   * the contents are flushed to the store files
+   */
+  public static final long DEFAULT_MEMSTORE_FLUSH_SIZE = 1024 * 1024 * 128L;
+
+  public static final int DEFAULT_REGION_REPLICATION = 1;
+
+  public static final boolean DEFAULT_REGION_MEMSTORE_REPLICATION = true;
+
+  private final static Map<String, String> DEFAULT_VALUES = new HashMap<>();
+  private final static Set<Bytes> RESERVED_KEYWORDS = new HashSet<>();
+
+  static {
+    DEFAULT_VALUES.put(MAX_FILESIZE,
+            String.valueOf(HConstants.DEFAULT_MAX_FILE_SIZE));
+    DEFAULT_VALUES.put(READONLY, String.valueOf(DEFAULT_READONLY));
+    DEFAULT_VALUES.put(MEMSTORE_FLUSHSIZE,
+            String.valueOf(DEFAULT_MEMSTORE_FLUSH_SIZE));
+    DEFAULT_VALUES.put(DURABILITY, DEFAULT_DURABLITY.name()); //use the enum name
+    DEFAULT_VALUES.put(REGION_REPLICATION, String.valueOf(DEFAULT_REGION_REPLICATION));
+    DEFAULT_VALUES.put(NORMALIZATION_ENABLED, String.valueOf(DEFAULT_NORMALIZATION_ENABLED));
+    DEFAULT_VALUES.put(PRIORITY, String.valueOf(DEFAULT_PRIORITY));
+    DEFAULT_VALUES.keySet().stream()
+            .map(s -> new Bytes(Bytes.toBytes(s))).forEach(RESERVED_KEYWORDS::add);
+    RESERVED_KEYWORDS.add(IS_ROOT_KEY);
+    RESERVED_KEYWORDS.add(IS_META_KEY);
+  }
+
+  public final static String NAMESPACE_FAMILY_INFO = "info";
+  public final static byte[] NAMESPACE_FAMILY_INFO_BYTES = Bytes.toBytes(NAMESPACE_FAMILY_INFO);
+  public final static byte[] NAMESPACE_COL_DESC_BYTES = Bytes.toBytes("d");
+
+  /**
+   * Table descriptor for namespace table
+   */
+  public static final TableDescriptor NAMESPACE_TABLEDESC
+    = TableDescriptorBuilder.newBuilder(TableName.NAMESPACE_TABLE_NAME)
+                            .addFamily(new HColumnDescriptor(NAMESPACE_FAMILY_INFO)
+                              // Ten is arbitrary number.  Keep versions to help debugging.
+                              .setMaxVersions(10)
+                              .setInMemory(true)
+                              .setBlocksize(8 * 1024)
+                              .setScope(HConstants.REPLICATION_SCOPE_LOCAL)
+                              // Enable cache of data blocks in L1 if more than one caching tier deployed:
+                              // e.g. if using CombinedBlockCache (BucketCache).
+                              .setCacheDataInL1(true))
+                            .doBuild();
+  private final ModifyableTableDescriptor desc;
+
+  /**
+   * @param desc The table descriptor to serialize
+   * @return This instance serialized with pb with pb magic prefix
+   */
+  public static byte[] toByteArray(TableDescriptor desc) {
+    if (desc instanceof ModifyableTableDescriptor) {
+      return ((ModifyableTableDescriptor) desc).toByteArray();
+    }
+    // TODO: remove this if the HTableDescriptor is removed
+    if (desc instanceof HTableDescriptor) {
+      return ((HTableDescriptor) desc).toByteArray();
+    }
+    return new ModifyableTableDescriptor(desc).toByteArray();
+  }
+
+  /**
+   * The input should be created by {@link #toByteArray}.
+   * @param pbBytes A pb serialized TableDescriptor instance with pb magic prefix
+   * @return This instance serialized with pb with pb magic prefix
+   * @throws org.apache.hadoop.hbase.exceptions.DeserializationException
+   */
+  public static TableDescriptorBuilder newBuilder(byte[] pbBytes) throws DeserializationException {
+    return new TableDescriptorBuilder(ModifyableTableDescriptor.parseFrom(pbBytes));
+  }
+
+  public static TableDescriptorBuilder newBuilder(final TableName name) {
+    return new TableDescriptorBuilder(name);
+  }
+
+  /**
+   * Copy all configuration, values, families, and name from the input.
+   * @param desc The desciptor to copy
+   * @return A clone of input
+   */
+  public static TableDescriptorBuilder newBuilder(final TableDescriptor desc) {
+    return new TableDescriptorBuilder(desc);
+  }
+
+  private TableDescriptorBuilder(final TableName name) {
+    this.desc = new ModifyableTableDescriptor(name);
+  }
+
+  private TableDescriptorBuilder(final TableDescriptor desc) {
+    this.desc = new ModifyableTableDescriptor(desc);
+  }
+
+  public TableDescriptorBuilder addCoprocessor(String className) throws IOException {
+    return addCoprocessor(className, null, Coprocessor.PRIORITY_USER, null);
+  }
+
+  public TableDescriptorBuilder addCoprocessor(String className, Path jarFilePath,
+          int priority, final Map<String, String> kvs) throws IOException {
+    desc.addCoprocessor(className, jarFilePath, priority, kvs);
+    return this;
+  }
+
+  public TableDescriptorBuilder addCoprocessorWithSpec(final String specStr) throws IOException {
+    desc.addCoprocessorWithSpec(specStr);
+    return this;
+  }
+
+  public TableDescriptorBuilder addFamily(final HColumnDescriptor family) {
+    desc.addFamily(family);
+    return this;
+  }
+
+  public TableDescriptorBuilder modifyFamily(final HColumnDescriptor family) {
+    desc.modifyFamily(family);
+    return this;
+  }
+
+  public TableDescriptorBuilder remove(Bytes key) {
+    desc.remove(key);
+    return this;
+  }
+
+  public TableDescriptorBuilder remove(byte[] key) {
+    desc.remove(key);
+    return this;
+  }
+
+  public TableDescriptorBuilder removeConfiguration(final String key) {
+    desc.removeConfiguration(key);
+    return this;
+  }
+
+  public TableDescriptorBuilder removeFamily(final byte[] column) {
+    desc.removeFamily(column);
+    return this;
+  }
+
+  public TableDescriptorBuilder removeCoprocessor(String className) {
+    desc.removeCoprocessor(className);
+    return this;
+  }
+
+  public TableDescriptorBuilder setCompactionEnabled(final boolean isEnable) {
+    desc.setCompactionEnabled(isEnable);
+    return this;
+  }
+
+  public TableDescriptorBuilder setConfiguration(String key, String value) {
+    desc.setConfiguration(key, value);
+    return this;
+  }
+
+  public TableDescriptorBuilder setDurability(Durability durability) {
+    desc.setDurability(durability);
+    return this;
+  }
+
+  public TableDescriptorBuilder setFlushPolicyClassName(String clazz) {
+    desc.setFlushPolicyClassName(clazz);
+    return this;
+  }
+
+  public TableDescriptorBuilder setMaxFileSize(long maxFileSize) {
+    desc.setMaxFileSize(maxFileSize);
+    return this;
+  }
+
+  public TableDescriptorBuilder setMemStoreFlushSize(long memstoreFlushSize) {
+    desc.setMemStoreFlushSize(memstoreFlushSize);
+    return this;
+  }
+
+  public TableDescriptorBuilder setNormalizationEnabled(final boolean isEnable) {
+    desc.setNormalizationEnabled(isEnable);
+    return this;
+  }
+
+  @Deprecated
+  public TableDescriptorBuilder setOwner(User owner) {
+    desc.setOwner(owner);
+    return this;
+  }
+
+  @Deprecated
+  public TableDescriptorBuilder setOwnerString(String ownerString) {
+    desc.setOwnerString(ownerString);
+    return this;
+  }
+
+  public TableDescriptorBuilder setPriority(int priority) {
+    desc.setPriority(priority);
+    return this;
+  }
+
+  public TableDescriptorBuilder setReadOnly(final boolean readOnly) {
+    desc.setReadOnly(readOnly);
+    return this;
+  }
+
+  public TableDescriptorBuilder setRegionMemstoreReplication(boolean memstoreReplication) {
+    desc.setRegionMemstoreReplication(memstoreReplication);
+    return this;
+  }
+
+  public TableDescriptorBuilder setRegionReplication(int regionReplication) {
+    desc.setRegionReplication(regionReplication);
+    return this;
+  }
+
+  public TableDescriptorBuilder setRegionSplitPolicyClassName(String clazz) {
+    desc.setRegionSplitPolicyClassName(clazz);
+    return this;
+  }
+
+  public TableDescriptorBuilder setValue(final Bytes key, final Bytes value) {
+    desc.setValue(key, value);
+    return this;
+  }
+
+  public TableDescriptorBuilder setValue(final byte[] key, final byte[] value) {
+    desc.setValue(key, value);
+    return this;
+  }
+
+  // TODO: replaced the HTableDescriptor by TableDescriptor
+  public HTableDescriptor build() {
+    return new HTableDescriptor(desc);
+  }
+
+  // TODO: remove this in HBase 3.0.0.
+  private TableDescriptor doBuild() {
+    return new ModifyableTableDescriptor(desc);
+  }
+
+  /**
+   * ModifyableTableDescriptor contains the details about an HBase table such as the
+   * descriptors of all the column families, is the table a catalog table, <code> -ROOT-
+   * </code> or <code> hbase:meta </code>, if the table is read only, the
+   * maximum size of the memstore, when the region split should occur,
+   * coprocessors associated with it etc... TODO: make this private after
+   * removing the HTableDescriptor
+   */
+  @InterfaceAudience.Private
+  public static class ModifyableTableDescriptor
+          implements TableDescriptor, Comparable<ModifyableTableDescriptor> {
+
+    private final TableName name;
+
+    /**
+     * A map which holds the metadata information of the table. This metadata
+     * includes values like IS_ROOT, IS_META, SPLIT_POLICY, MAX_FILE_SIZE,
+     * READONLY, MEMSTORE_FLUSHSIZE etc...
+     */
+    private final Map<Bytes, Bytes> values = new HashMap<>();
+
+    /**
+     * A map which holds the configuration specific to the table. The keys of
+     * the map have the same names as config keys and override the defaults with
+     * table-specific settings. Example usage may be for compactions, etc.
+     */
+    private final Map<String, String> configuration = new HashMap<>();
+
+    /**
+     * Maps column family name to the respective HColumnDescriptors
+     */
+    private final Map<byte[], HColumnDescriptor> families
+            = new TreeMap<>(Bytes.BYTES_RAWCOMPARATOR);
+
+    /**
+     * Construct a table descriptor specifying a TableName object
+     *
+     * @param name Table name.
+     * @see
+     * <a href="https://issues.apache.org/jira/browse/HBASE-174">HADOOP-1581
+     * HBASE: (HBASE-174) Un-openable tablename bug</a>
+     */
+    private ModifyableTableDescriptor(final TableName name) {
+      this(name, Collections.EMPTY_LIST, Collections.EMPTY_MAP, Collections.EMPTY_MAP);
+    }
+
+    /**
+     * Construct a table descriptor by cloning the descriptor passed as a
+     * parameter.
+     * <p>
+     * Makes a deep copy of the supplied descriptor.
+     * TODO: make this private after removing the HTableDescriptor
+     * @param desc The descriptor.
+     */
+    @InterfaceAudience.Private
+    protected ModifyableTableDescriptor(final TableDescriptor desc) {
+      this(desc.getTableName(), desc.getFamilies(), desc.getValues(), desc.getConfiguration());
+    }
+
+    // TODO: make this private after removing the HTableDescriptor
+    @InterfaceAudience.Private
+    public ModifyableTableDescriptor(final TableName name, final Collection<HColumnDescriptor> families,
+            Map<Bytes, Bytes> values, Map<String, String> configuration) {
+      this.name = name;
+      families.forEach(c -> this.families.put(c.getName(), new HColumnDescriptor(c)));
+      values.forEach(this.values::put);
+      configuration.forEach(this.configuration::put);
+      setMetaFlags(name);
+    }
+
+    /*
+     * Set meta flags on this table.
+     * IS_ROOT_KEY is set if its a -ROOT- table
+     * IS_META_KEY is set either if its a -ROOT- or a hbase:meta table
+     * Called by constructors.
+     * @param name
+     */
+    private void setMetaFlags(final TableName name) {
+      values.put(IS_META_KEY, isRootRegion()
+              || name.equals(TableName.META_TABLE_NAME) ? TRUE : FALSE);
+    }
+
+    /**
+     * Check if the descriptor represents a <code> -ROOT- </code> region.
+     *
+     * @return true if this is a <code> -ROOT- </code> region
+     */
+    @Override
+    public boolean isRootRegion() {
+      return isSomething(IS_ROOT_KEY, false);
+    }
+
+    /**
+     * Checks if this table is <code> hbase:meta </code> region.
+     *
+     * @return true if this table is <code> hbase:meta </code> region
+     */
+    @Override
+    public boolean isMetaRegion() {
+      return isSomething(IS_META_KEY, false);
+    }
+
+    private boolean isSomething(final Bytes key,
+            final boolean valueIfNull) {
+      byte[] value = getValue(key);
+      if (value != null) {
+        return Boolean.valueOf(Bytes.toString(value));
+      }
+      return valueIfNull;
+    }
+
+    /**
+     * Checks if the table is a <code>hbase:meta</code> table
+     *
+     * @return true if table is <code> hbase:meta </code> region.
+     */
+    @Override
+    public boolean isMetaTable() {
+      return isMetaRegion() && !isRootRegion();
+    }
+
+    /**
+     * Getter for accessing the metadata associated with the key
+     *
+     * @param key The key.
+     * @return The value.
+     * @see #values
+     */
+    @Override
+    public byte[] getValue(byte[] key) {
+      return getValue(new Bytes(key));
+    }
+
+    private byte[] getValue(final Bytes key) {
+      Bytes ibw = values.get(key);
+      if (ibw == null) {
+        return null;
+      }
+      return ibw.get();
+    }
+
+    /**
+     * Getter for accessing the metadata associated with the key
+     *
+     * @param key The key.
+     * @return The value.
+     * @see #values
+     */
+    public String getValue(String key) {
+      byte[] value = getValue(Bytes.toBytes(key));
+      if (value == null) {
+        return null;
+      }
+      return Bytes.toString(value);
+    }
+
+    /**
+     * Getter for fetching an unmodifiable {@link #values} map.
+     *
+     * @return unmodifiable map {@link #values}.
+     * @see #values
+     */
+    @Override
+    public Map<Bytes, Bytes> getValues() {
+      // shallow pointer copy
+      return Collections.unmodifiableMap(values);
+    }
+
+    /**
+     * Setter for storing metadata as a (key, value) pair in {@link #values} map
+     *
+     * @param key The key.
+     * @param value The value. If null, removes the setting.
+     * @return the modifyable TD
+     * @see #values
+     */
+    public ModifyableTableDescriptor setValue(byte[] key, byte[] value) {
+      return setValue(toBytesOrNull(key, v -> v),
+              toBytesOrNull(value, v -> v));
+    }
+
+    /*
+     * @param key The key.
+     * @param value The value. If null, removes the setting.
+     */
+    private ModifyableTableDescriptor setValue(final Bytes key,
+            final String value) {
+      return setValue(key, toBytesOrNull(value, Bytes::toBytes));
+    }
+
+    /*
+     * Setter for storing metadata as a (key, value) pair in {@link #values} map
+     *
+     * @param key The key.
+     * @param value The value. If null, removes the setting.
+     */
+    public ModifyableTableDescriptor setValue(final Bytes key, final Bytes value) {
+      if (value == null) {
+        remove(key);
+      } else {
+        values.put(key, value);
+      }
+      return this;
+    }
+
+    /**
+     * Setter for storing metadata as a (key, value) pair in {@link #values} map
+     *
+     * @param key The key.
+     * @param value The value. If null, removes the setting.
+     * @return the modifyable TD
+     * @see #values
+     */
+    public ModifyableTableDescriptor setValue(String key, String value) {
+      return setValue(toBytesOrNull(key, Bytes::toBytes),
+              toBytesOrNull(value, Bytes::toBytes));
+    }
+
+    private static <T> Bytes toBytesOrNull(T t, Function<T, byte[]> f) {
+      if (t == null) {
+        return null;
+      } else {
+        return new Bytes(f.apply(t));
+      }
+    }
+
+    /**
+     * Remove metadata represented by the key from the {@link #values} map
+     *
+     * @param key Key whose key and value we're to remove from TableDescriptor
+     * parameters.
+     */
+    public void remove(final String key) {
+      remove(new Bytes(Bytes.toBytes(key)));
+    }
+
+    /**
+     * Remove metadata represented by the key from the {@link #values} map
+     *
+     * @param key Key whose key and value we're to remove from TableDescriptor
+     * parameters.
+     */
+    public void remove(Bytes key) {
+      values.remove(key);
+    }
+
+    /**
+     * Remove metadata represented by the key from the {@link #values} map
+     *
+     * @param key Key whose key and value we're to remove from TableDescriptor
+     * parameters.
+     */
+    public void remove(final byte[] key) {
+      remove(new Bytes(key));
+    }
+
+    /**
+     * Check if the readOnly flag of the table is set. If the readOnly flag is
+     * set then the contents of the table can only be read from but not
+     * modified.
+     *
+     * @return true if all columns in the table should be read only
+     */
+    @Override
+    public boolean isReadOnly() {
+      return isSomething(READONLY_KEY, DEFAULT_READONLY);
+    }
+
+    /**
+     * Setting the table as read only sets all the columns in the table as read
+     * only. By default all tables are modifiable, but if the readOnly flag is
+     * set to true then the contents of the table can only be read but not
+     * modified.
+     *
+     * @param readOnly True if all of the columns in the table should be read
+     * only.
+     * @return the modifyable TD
+     */
+    public ModifyableTableDescriptor setReadOnly(final boolean readOnly) {
+      return setValue(READONLY_KEY, readOnly ? TRUE : FALSE);
+    }
+
+    /**
+     * Check if the compaction enable flag of the table is true. If flag is
+     * false then no minor/major compactions will be done in real.
+     *
+     * @return true if table compaction enabled
+     */
+    @Override
+    public boolean isCompactionEnabled() {
+      return isSomething(COMPACTION_ENABLED_KEY, DEFAULT_COMPACTION_ENABLED);
+    }
+
+    /**
+     * Setting the table compaction enable flag.
+     *
+     * @param isEnable True if enable compaction.
+     * @return the modifyable TD
+     */
+    public ModifyableTableDescriptor setCompactionEnabled(final boolean isEnable) {
+      return setValue(COMPACTION_ENABLED_KEY, isEnable ? TRUE : FALSE);
+    }
+
+    /**
+     * Check if normalization enable flag of the table is true. If flag is false
+     * then no region normalizer won't attempt to normalize this table.
+     *
+     * @return true if region normalization is enabled for this table
+     */
+    @Override
+    public boolean isNormalizationEnabled() {
+      return isSomething(NORMALIZATION_ENABLED_KEY, DEFAULT_NORMALIZATION_ENABLED);
+    }
+
+    /**
+     * Setting the table normalization enable flag.
+     *
+     * @param isEnable True if enable normalization.
+     * @return the modifyable TD
+     */
+    public ModifyableTableDescriptor setNormalizationEnabled(final boolean isEnable) {
+      return setValue(NORMALIZATION_ENABLED_KEY, isEnable ? TRUE : FALSE);
+    }
+
+    /**
+     * Sets the {@link Durability} setting for the table. This defaults to
+     * Durability.USE_DEFAULT.
+     *
+     * @param durability enum value
+     * @return the modifyable TD
+     */
+    public ModifyableTableDescriptor setDurability(Durability durability) {
+      return setValue(DURABILITY_KEY, durability.name());
+    }
+
+    /**
+     * Returns the durability setting for the table.
+     *
+     * @return durability setting for the table.
+     */
+    @Override
+    public Durability getDurability() {
+      byte[] durabilityValue = getValue(DURABILITY_KEY);
+      if (durabilityValue == null) {
+        return DEFAULT_DURABLITY;
+      } else {
+        try {
+          return Durability.valueOf(Bytes.toString(durabilityValue));
+        } catch (IllegalArgumentException ex) {
+          LOG.warn("Received " + ex + " because Durability value for TableDescriptor"
+                  + " is not known. Durability:" + Bytes.toString(durabilityValue));
+          return DEFAULT_DURABLITY;
+        }
+      }
+    }
+
+    /**
+     * Get the name of the table
+     *
+     * @return TableName
+     */
+    @Override
+    public TableName getTableName() {
+      return name;
+    }
+
+    /**
+     * This sets the class associated with the region split policy which
+     * determines when a region split should occur. The class used by default is
+     * defined in org.apache.hadoop.hbase.regionserver.RegionSplitPolicy
+     *
+     * @param clazz the class name
+     * @return the modifyable TD
+     */
+    public ModifyableTableDescriptor setRegionSplitPolicyClassName(String clazz) {
+      return setValue(SPLIT_POLICY, clazz);
+    }
+
+    /**
+     * This gets the class associated with the region split policy which
+     * determines when a region split should occur. The class used by default is
+     * defined in org.apache.hadoop.hbase.regionserver.RegionSplitPolicy
+     *
+     * @return the class name of the region split policy for this table. If this
+     * returns null, the default split policy is used.
+     */
+    @Override
+    public String getRegionSplitPolicyClassName() {
+      return getValue(SPLIT_POLICY);
+    }
+
+    /**
+     * Returns the maximum size upto which a region can grow to after which a
+     * region split is triggered. The region size is represented by the size of
+     * the biggest store file in that region.
+     *
+     * @return max hregion size for table, -1 if not set.
+     *
+     * @see #setMaxFileSize(long)
+     */
+    @Override
+    public long getMaxFileSize() {
+      byte[] value = getValue(MAX_FILESIZE_KEY);
+      if (value != null) {
+        return Long.parseLong(Bytes.toString(value));
+      }
+      return -1;
+    }
+
+    /**
+     * Sets the maximum size upto which a region can grow to after which a
+     * region split is triggered. The region size is represented by the size of
+     * the biggest store file in that region, i.e. If the biggest store file
+     * grows beyond the maxFileSize, then the region split is triggered. This
+     * defaults to a value of 256 MB.
+     * <p>
+     * This is not an absolute value and might vary. Assume that a single row
+     * exceeds the maxFileSize then the storeFileSize will be greater than
+     * maxFileSize since a single row cannot be split across multiple regions
+     * </p>
+     *
+     * @param maxFileSize The maximum file size that a store file can grow to
+     * before a split is triggered.
+     * @return the modifyable TD
+     */
+    public ModifyableTableDescriptor setMaxFileSize(long maxFileSize) {
+      return setValue(MAX_FILESIZE_KEY, Long.toString(maxFileSize));
+    }
+
+    /**
+     * Returns the size of the memstore after which a flush to filesystem is
+     * triggered.
+     *
+     * @return memory cache flush size for each hregion, -1 if not set.
+     *
+     * @see #setMemStoreFlushSize(long)
+     */
+    @Override
+    public long getMemStoreFlushSize() {
+      byte[] value = getValue(MEMSTORE_FLUSHSIZE_KEY);
+      if (value != null) {
+        return Long.parseLong(Bytes.toString(value));
+      }
+      return -1;
+    }
+
+    /**
+     * Represents the maximum size of the memstore after which the contents of
+     * the memstore are flushed to the filesystem. This defaults to a size of 64
+     * MB.
+     *
+     * @param memstoreFlushSize memory cache flush size for each hregion
+     * @return the modifyable TD
+     */
+    public ModifyableTableDescriptor setMemStoreFlushSize(long memstoreFlushSize) {
+      return setValue(MEMSTORE_FLUSHSIZE_KEY, Long.toString(memstoreFlushSize));
+    }
+
+    /**
+     * This sets the class associated with the flush policy which determines
+     * determines the stores need to be flushed when flushing a region. The
+     * class used by default is defined in
+     * org.apache.hadoop.hbase.regionserver.FlushPolicy.
+     *
+     * @param clazz the class name
+     * @return the modifyable TD
+     */
+    public ModifyableTableDescriptor setFlushPolicyClassName(String clazz) {
+      return setValue(FLUSH_POLICY, clazz);
+    }
+
+    /**
+     * This gets the class associated with the flush policy which determines the
+     * stores need to be flushed when flushing a region. The class used by
+     * default is defined in org.apache.hadoop.hbase.regionserver.FlushPolicy.
+     *
+     * @return the class name of the flush policy for this table. If this
+     * returns null, the default flush policy is used.
+     */
+    @Override
+    public String getFlushPolicyClassName() {
+      return getValue(FLUSH_POLICY);
+    }
+
+    /**
+     * Adds a column family. For the updating purpose please use
+     * {@link #modifyFamily(HColumnDescriptor)} instead.
+     *
+     * @param family HColumnDescriptor of family to add.
+     * @return the modifyable TD
+     */
+    public ModifyableTableDescriptor addFamily(final HColumnDescriptor family) {
+      if (family.getName() == null || family.getName().length <= 0) {
+        throw new IllegalArgumentException("Family name cannot be null or empty");
+      }
+      if (hasFamily(family.getName())) {
+        throw new IllegalArgumentException("Family '"
+                + family.getNameAsString() + "' already exists so cannot be added");
+      }
+      return setFamily(family);
+    }
+
+    /**
+     * Modifies the existing column family.
+     *
+     * @param family HColumnDescriptor of family to update
+     * @return this (for chained invocation)
+     */
+    public ModifyableTableDescriptor modifyFamily(final HColumnDescriptor family) {
+      if (family.getName() == null || family.getName().length <= 0) {
+        throw new IllegalArgumentException("Family name cannot be null or empty");
+      }
+      if (!hasFamily(family.getName())) {
+        throw new IllegalArgumentException("Column family '" + family.getNameAsString()
+                + "' does not exist");
+      }
+      return setFamily(family);
+    }
+
+    // TODO: make this private after removing the UnmodifyableTableDescriptor
+    protected ModifyableTableDescriptor setFamily(HColumnDescriptor family) {
+      families.put(family.getName(), family);
+      return this;
+    }
+
+    /**
+     * Checks to see if this table contains the given column family
+     *
+     * @param familyName Family name or column name.
+     * @return true if the table contains the specified family name
+     */
+    @Override
+    public boolean hasFamily(final byte[] familyName) {
+      return families.containsKey(familyName);
+    }
+
+    /**
+     * @return Name of this table and then a map of all of the column family descriptors.
+     */
+    @Override
+    public String toString() {
+      StringBuilder s = new StringBuilder();
+      s.append('\'').append(Bytes.toString(name.getName())).append('\'');
+      s.append(getValues(true));
+      families.values().forEach(f -> s.append(", ").append(f));
+      return s.toString();
+    }
+
+    /**
+     * @return Name of this table and then a map of all of the column family
+     * descriptors (with only the non-default column family attributes)
+     */
+    public String toStringCustomizedValues() {
+      StringBuilder s = new StringBuilder();
+      s.append('\'').append(Bytes.toString(name.getName())).append('\'');
+      s.append(getValues(false));
+      families.values().forEach(hcd -> s.append(", ").append(hcd.toStringCustomizedValues()));
+      return s.toString();
+    }
+
+    /**
+     * @return map of all table attributes formatted into string.
+     */
+    public String toStringTableAttributes() {
+      return getValues(true).toString();
+    }
+
+    private StringBuilder getValues(boolean printDefaults) {
+      StringBuilder s = new StringBuilder();
+
+      // step 1: set partitioning and pruning
+      Set<Bytes> reservedKeys = new TreeSet<>();
+      Set<Bytes> userKeys = new TreeSet<>();
+      for (Map.Entry<Bytes, Bytes> entry : values.entrySet()) {
+        if (entry.getKey() == null || entry.getKey().get() == null) {
+          continue;
+        }
+        String key = Bytes.toString(entry.getKey().get());
+        // in this section, print out reserved keywords + coprocessor info
+        if (!RESERVED_KEYWORDS.contains(entry.getKey()) && !key.startsWith("coprocessor$")) {
+          userKeys.add(entry.getKey());
+          continue;
+        }
+        // only print out IS_ROOT/IS_META if true
+        String value = Bytes.toString(entry.getValue().get());
+        if (key.equalsIgnoreCase(IS_ROOT) || key.equalsIgnoreCase(IS_META)) {
+          if (Boolean.valueOf(value) == false) {
+            continue;
+          }
+        }
+        // see if a reserved key is a default value. may not want to print it out
+        if (printDefaults
+                || !DEFAULT_VALUES.containsKey(key)
+                || !DEFAULT_VALUES.get(key).equalsIgnoreCase(value)) {
+          reservedKeys.add(entry.getKey());
+        }
+      }
+
+      // early exit optimization
+      boolean hasAttributes = !reservedKeys.isEmpty() || !userKeys.isEmpty();
+      if (!hasAttributes && configuration.isEmpty()) {
+        return s;
+      }
+
+      s.append(", {");
+      // step 2: printing attributes
+      if (hasAttributes) {
+        s.append("TABLE_ATTRIBUTES => {");
+
+        // print all reserved keys first
+        boolean printCommaForAttr = false;
+        for (Bytes k : reservedKeys) {
+          String key = Bytes.toString(k.get());
+          String value = Bytes.toStringBinary(values.get(k).get());
+          if (printCommaForAttr) {
+            s.append(", ");
+          }
+          printCommaForAttr = true;
+          s.append(key);
+          s.append(" => ");
+          s.append('\'').append(value).append('\'');
+        }
+
+        if (!userKeys.isEmpty()) {
+          // print all non-reserved, advanced config keys as a separate subset
+          if (printCommaForAttr) {
+            s.append(", ");
+          }
+          s.append(HConstants.METADATA).append(" => ");
+          s.append("{");
+          boolean printCommaForCfg = false;
+          for (Bytes k : userKeys) {
+            String key = Bytes.toString(k.get());
+            String value = Bytes.toStringBinary(values.get(k).get());
+            if (printCommaForCfg) {
+              s.append(", ");
+            }
+            printCommaForCfg = true;
+            s.append('\'').append(key).append('\'');
+            s.append(" => ");
+            s.append('\'').append(value).append('\'');
+          }
+          s.append("}");
+        }
+      }
+
+      // step 3: printing all configuration:
+      if (!configuration.isEmpty()) {
+        if (hasAttributes) {
+          s.append(", ");
+        }
+        s.append(HConstants.CONFIGURATION).append(" => ");
+        s.append('{');
+        boolean printCommaForConfig = false;
+        for (Map.Entry<String, String> e : configuration.entrySet()) {
+          if (printCommaForConfig) {
+            s.append(", ");
+          }
+          printCommaForConfig = true;
+          s.append('\'').append(e.getKey()).append('\'');
+          s.append(" => ");
+          s.append('\'').append(e.getValue()).append('\'');
+        }
+        s.append("}");
+      }
+      s.append("}"); // end METHOD
+      return s;
+    }
+
+    /**
+     * Compare the contents of the descriptor with another one passed as a
+     * parameter. Checks if the obj passed is an instance of ModifyableTableDescriptor,
+     * if yes then the contents of the descriptors are compared.
+     *
+     * @return true if the contents of the the two descriptors exactly match
+     *
+     * @see java.lang.Object#equals(java.lang.Object)
+     */
+    @Override
+    public boolean equals(Object obj) {
+      if (this == obj) {
+        return true;
+      }
+      if (obj == null) {
+        return false;
+      }
+      if (!(obj instanceof ModifyableTableDescriptor)) {
+        return false;
+      }
+      return compareTo((ModifyableTableDescriptor) obj) == 0;
+    }
+
+    /**
+     * @see java.lang.Object#hashCode()
+     */
+    @Override
+    public int hashCode() {
+      int result = this.name.hashCode();
+      if (this.families.size() > 0) {
+        for (HColumnDescriptor e : this.families.values()) {
+          result ^= e.hashCode();
+        }
+      }
+      result ^= values.hashCode();
+      result ^= configuration.hashCode();
+      return result;
+    }
+
+    // Comparable
+    /**
+     * Compares the descriptor with another descriptor which is passed as a
+     * parameter. This compares the content of the two descriptors and not the
+     * reference.
+     *
+     * @param other The MTD to compare
+     * @return 0 if the contents of the descriptors are exactly matching, 1 if
+     * there is a mismatch in the contents
+     */
+    @Override
+    public int compareTo(final ModifyableTableDescriptor other) {
+      int result = this.name.compareTo(other.name);
+      if (result == 0) {
+        result = families.size() - other.families.size();
+      }
+      if (result == 0 && families.size() != other.families.size()) {
+        result = Integer.valueOf(families.size()).compareTo(other.families.size());
+      }
+      if (result == 0) {
+        for (Iterator<HColumnDescriptor> it = families.values().iterator(),
+                it2 = other.families.values().iterator(); it.hasNext();) {
+          result = it.next().compareTo(it2.next());
+          if (result != 0) {
+            break;
+          }
+        }
+      }
+      if (result == 0) {
+        // punt on comparison for ordering, just calculate difference
+        result = this.values.hashCode() - other.values.hashCode();
+        if (result < 0) {
+          result = -1;
+        } else if (result > 0) {
+          result = 1;
+        }
+      }
+      if (result == 0) {
+        result = this.configuration.hashCode() - other.configuration.hashCode();
+        if (result < 0) {
+          result = -1;
+        } else if (result > 0) {
+          result = 1;
+        }
+      }
+      return result;
+    }
+
+    /**
+     * Returns an unmodifiable collection of all the {@link HColumnDescriptor}
+     * of all the column families of the table.
+     *
+     * @return Immutable collection of {@link HColumnDescriptor} of all the
+     * column families.
+     */
+    @Override
+    public Collection<HColumnDescriptor> getFamilies() {
+      return Collections.unmodifiableCollection(this.families.values());
+    }
+
+    /**
+     * Return true if there are at least one cf whose replication scope is
+     * serial.
+     */
+    @Override
+    public boolean hasSerialReplicationScope() {
+      return getFamilies()
+              .stream()
+              .anyMatch(column -> column.getScope() == HConstants.REPLICATION_SCOPE_SERIAL);
+    }
+
+    /**
+     * Returns the configured replicas per region
+     */
+    @Override
+    public int getRegionReplication() {
+      return getIntValue(REGION_REPLICATION_KEY, DEFAULT_REGION_REPLICATION);
+    }
+
+    private int getIntValue(Bytes key, int defaultVal) {
+      byte[] val = getValue(key);
+      if (val == null || val.length == 0) {
+        return defaultVal;
+      }
+      return Integer.parseInt(Bytes.toString(val));
+    }
+
+    /**
+     * Sets the number of replicas per region.
+     *
+     * @param regionReplication the replication factor per region
+     * @return the modifyable TD
+     */
+    public ModifyableTableDescriptor setRegionReplication(int regionReplication) {
+      return setValue(REGION_REPLICATION_KEY,
+              new Bytes(Bytes.toBytes(Integer.toString(regionReplication))));
+    }
+
+    /**
+     * @return true if the read-replicas memstore replication is enabled.
+     */
+    @Override
+    public boolean hasRegionMemstoreReplication() {
+      return isSomething(REGION_MEMSTORE_REPLICATION_KEY, DEFAULT_REGION_MEMSTORE_REPLICATION);
+    }
+
+    /**
+     * Enable or Disable the memstore replication from the primary region to the
+     * replicas. The replication will be used only for meta operations (e.g.
+     * flush, compaction, ...)
+     *
+     * @param memstoreReplication true if the new data written to the primary
+     * region should be replicated. false if the secondaries can tollerate to
+     * have new data only when the primary flushes the memstore.
+     * @return the modifyable TD
+     */
+    public ModifyableTableDescriptor setRegionMemstoreReplication(boolean memstoreReplication) {
+      setValue(REGION_MEMSTORE_REPLICATION_KEY, memstoreReplication ? TRUE : FALSE);
+      // If the memstore replication is setup, we do not have to wait for observing a flush event
+      // from primary before starting to serve reads, because gaps from replication is not applicable
+      return setConfiguration(RegionReplicaUtil.REGION_REPLICA_WAIT_FOR_PRIMARY_FLUSH_CONF_KEY,
+              Boolean.toString(memstoreReplication));
+    }
+
+    public ModifyableTableDescriptor setPriority(int priority) {
+      return setValue(PRIORITY_KEY, Integer.toString(priority));
+    }
+
+    @Override
+    public int getPriority() {
+      return getIntValue(PRIORITY_KEY, DEFAULT_PRIORITY);
+    }
+
+    /**
+     * Returns all the column family names of the current table. The map of
+     * TableDescriptor contains mapping of family name to HColumnDescriptors.
+     * This returns all the keys of the family map which represents the column
+     * family names of the table.
+     *
+     * @return Immutable sorted set of the keys of the families.
+     */
+    @Override
+    public Set<byte[]> getFamiliesKeys() {
+      return Collections.unmodifiableSet(this.families.keySet());
+    }
+
+    /**
+     * Returns the count of the column families of the table.
+     *
+     * @return Count of column families of the table
+     */
+    @Override
+    public int getColumnFamilyCount() {
+      return families.size();
+    }
+
+    /**
+     * Returns an array all the {@link HColumnDescriptor} of the column families
+     * of the table.
+     *
+     * @return Array of all the HColumnDescriptors of the current table
+     *
+     * @see #getFamilies()
+     */
+    @Override
+    public HColumnDescriptor[] getColumnFamilies() {
+      Collection<HColumnDescriptor> hColumnDescriptors = getFamilies();
+      return hColumnDescriptors.toArray(new HColumnDescriptor[hColumnDescriptors.size()]);
+    }
+
+    /**
+     * Returns the HColumnDescriptor for a specific column family with name as
+     * specified by the parameter column.
+     *
+     * @param column Column family name
+     * @return Column descriptor for the passed family name or the family on
+     * passed in column.
+     */
+    @Override
+    public HColumnDescriptor getFamily(final byte[] column) {
+      return this.families.get(column);
+    }
+
+    /**
+     * Removes the HColumnDescriptor with name specified by the parameter column
+     * from the table descriptor
+     *
+     * @param column Name of the column family to be removed.
+     * @return Column descriptor for the passed family name or the family on
+     * passed in column.
+     */
+    public HColumnDescriptor removeFamily(final byte[] column) {
+      return this.families.remove(column);
+    }
+
+    /**
+     * Add a table coprocessor to this table. The coprocessor type must be
+     * org.apache.hadoop.hbase.coprocessor.RegionObserver or Endpoint. It won't
+     * check if the class can be loaded or not. Whether a coprocessor is
+     * loadable or not will be determined when a region is opened.
+     *
+     * @param className Full class name.
+     * @throws IOException
+     * @return the modifyable TD
+     */
+    public ModifyableTableDescriptor addCoprocessor(String className) throws IOException {
+      return addCoprocessor(className, null, Coprocessor.PRIORITY_USER, null);
+    }
+
+    /**
+     * Add a table coprocessor to this table. The coprocessor type must be
+     * org.apache.hadoop.hbase.coprocessor.RegionObserver or Endpoint. It won't
+     * check if the class can be loaded or not. Whether a coprocessor is
+     * loadable or not will be determined when a region is opened.
+     *
+     * @param jarFilePath Path of the jar file. If it's null, the class will be
+     * loaded from default classloader.
+     * @param className Full class name.
+     * @param priority Priority
+     * @param kvs Arbitrary key-value parameter pairs passed into the
+     * coprocessor.
+     * @throws IOException
+     * @return the modifyable TD
+     */
+    public ModifyableTableDescriptor addCoprocessor(String className, Path jarFilePath,
+            int priority, final Map<String, String> kvs)
+            throws IOException {
+      checkHasCoprocessor(className);
+
+      // Validate parameter kvs and then add key/values to kvString.
+      StringBuilder kvString = new StringBuilder();
+      if (kvs != null) {
+        for (Map.Entry<String, String> e : kvs.entrySet()) {
+          if (!e.getKey().matches(HConstants.CP_HTD_ATTR_VALUE_PARAM_KEY_PATTERN)) {
+            throw new IOException("Illegal parameter key = " + e.getKey());
+          }
+          if (!e.getValue().matches(HConstants.CP_HTD_ATTR_VALUE_PARAM_VALUE_PATTERN)) {
+            throw new IOException("Illegal parameter (" + e.getKey()
+                    + ") value = " + e.getValue());
+          }
+          if (kvString.length() != 0) {
+            kvString.append(',');
+          }
+          kvString.append(e.getKey());
+          kvString.append('=');
+          kvString.append(e.getValue());
+        }
+      }
+
+      String value = ((jarFilePath == null) ? "" : jarFilePath.toString())
+              + "|" + className + "|" + Integer.toString(priority) + "|"
+              + kvString.toString();
+      return addCoprocessorToMap(value);
+    }
+
+    /**
+     * Add a table coprocessor to this table. The coprocessor type must be
+     * org.apache.hadoop.hbase.coprocessor.RegionObserver or Endpoint. It won't
+     * check if the class can be loaded or not. Whether a coprocessor is
+     * loadable or not will be determined when a region is opened.
+     *
+     * @param specStr The Coprocessor specification all in in one String
+     * formatted so matches {@link HConstants#CP_HTD_ATTR_VALUE_PATTERN}
+     * @throws IOException
+     * @return the modifyable TD
+     */
+    public ModifyableTableDescriptor addCoprocessorWithSpec(final String specStr) throws IOException {
+      String className = getCoprocessorClassNameFromSpecStr(specStr);
+      if (className == null) {
+        throw new IllegalArgumentException("Format does not match "
+                + HConstants.CP_HTD_ATTR_VALUE_PATTERN + ": " + specStr);
+      }
+      checkHasCoprocessor(className);
+      return addCoprocessorToMap(specStr);
+    }
+
+    private void checkHasCoprocessor(final String className) throws IOException {
+      if (hasCoprocessor(className)) {
+        throw new IOException("Coprocessor " + className + " already exists.");
+      }
+    }
+
+    /**
+     * Add coprocessor to values Map
+     *
+     * @param specStr The Coprocessor specification all in in one String
+     * formatted so matches {@link HConstants#CP_HTD_ATTR_VALUE_PATTERN}
+     * @return Returns <code>this</code>
+     */
+    private ModifyableTableDescriptor addCoprocessorToMap(final String specStr) {
+      if (specStr == null) {
+        return this;
+      }
+      // generate a coprocessor key
+      int maxCoprocessorNumber = 0;
+      Matcher keyMatcher;
+      for (Map.Entry<Bytes, Bytes> e : this.values.entrySet()) {
+        keyMatcher = HConstants.CP_HTD_ATTR_KEY_PATTERN.matcher(Bytes.toString(e.getKey().get()));
+        if (!keyMatcher.matches()) {
+          continue;
+        }
+        maxCoprocessorNumber = Math.max(Integer.parseInt(keyMatcher.group(1)), maxCoprocessorNumber);
+      }
+      maxCoprocessorNumber++;
+      String key = "coprocessor$" + Integer.toString(maxCoprocessorNumber);
+      return setValue(new Bytes(Bytes.toBytes(key)), new Bytes(Bytes.toBytes(specStr)));
+    }
+
+    /**
+     * Check if the table has an attached co-processor represented by the name
+     * className
+     *
+     * @param classNameToMatch - Class name of the co-processor
+     * @return true of the table has a co-processor className
+     */
+    @Override
+    public boolean hasCoprocessor(String classNameToMatch) {
+      Matcher keyMatcher;
+      for (Map.Entry<Bytes, Bytes> e
+              : this.values.entrySet()) {
+        keyMatcher
+                = HConstants.CP_HTD_ATTR_KEY_PATTERN.matcher(
+                        Bytes.toString(e.getKey().get()));
+        if (!keyMatcher.matches()) {
+          continue;
+        }
+        String className = getCoprocessorClassNameFromSpecStr(Bytes.toString(e.getValue().get()));
+        if (className == null) {
+          continue;
+        }
+        if (className.equals(classNameToMatch.trim())) {
+          return true;
+        }
+      }
+      return false;
+    }
+
+    /**
+     * Return the list of attached co-processor represented by their name
+     * className
+     *
+     * @return The list of co-processors classNames
+     */
+    @Override
+    public List<String> getCoprocessors() {
+      List<String> result = new ArrayList<>(this.values.entrySet().size());
+      Matcher keyMatcher;
+      for (Map.Entry<Bytes, Bytes> e : this.values.entrySet()) {
+        keyMatcher = HConstants.CP_HTD_ATTR_KEY_PATTERN.matcher(Bytes.toString(e.getKey().get()));
+        if (!keyMatcher.matches()) {
+          continue;
+        }
+        String className = getCoprocessorClassNameFromSpecStr(Bytes.toString(e.getValue().get()));
+        if (className == null) {
+          continue;
+        }
+        result.add(className); // classname is the 2nd field
+      }
+      return result;
+    }
+
+    /**
+     * @param spec String formatted as per
+     * {@link HConstants#CP_HTD_ATTR_VALUE_PATTERN}
+     * @return Class parsed from passed in <code>spec</code> or null if no match
+     * or classpath found
+     */
+    private static String getCoprocessorClassNameFromSpecStr(final String spec) {
+      Matcher matcher = HConstants.CP_HTD_ATTR_VALUE_PATTERN.matcher(spec);
+      // Classname is the 2nd field
+      return matcher != null && matcher.matches() ? matcher.group(2).trim() : null;
+    }
+
+    /**
+     * Remove a coprocessor from those set on the table
+     *
+     * @param className Class name of the co-processor
+     */
+    public void removeCoprocessor(String className) {
+      Bytes match = null;
+      Matcher keyMatcher;
+      Matcher valueMatcher;
+      for (Map.Entry<Bytes, Bytes> e : this.values
+              .entrySet()) {
+        keyMatcher = HConstants.CP_HTD_ATTR_KEY_PATTERN.matcher(Bytes.toString(e
+                .getKey().get()));
+        if (!keyMatcher.matches()) {
+          continue;
+        }
+        valueMatcher = HConstants.CP_HTD_ATTR_VALUE_PATTERN.matcher(Bytes
+                .toString(e.getValue().get()));
+        if (!valueMatcher.matches()) {
+          continue;
+        }
+        // get className and compare
+        String clazz = valueMatcher.group(2).trim(); // classname is the 2nd field
+        // remove the CP if it is present
+        if (clazz.equals(className.trim())) {
+          match = e.getKey();
+          break;
+        }
+      }
+      // if we found a match, remove it
+      if (match != null) {
+        remove(match);
+      }
+    }
+
+    @Deprecated
+    public ModifyableTableDescriptor setOwner(User owner) {
+      return setOwnerString(owner != null ? owner.getShortName() : null);
+    }
+
+    // used by admin.rb:alter(table_name,*args) to update owner.
+    @Deprecated
+    public ModifyableTableDescriptor setOwnerString(String ownerString) {
+      if (ownerString != null) {
+        setValue(OWNER_KEY, ownerString);
+      } else {
+        remove(OWNER_KEY);
+      }
+      return this;
+    }
+
+    @Override
+    @Deprecated
+    public String getOwnerString() {
+      if (getValue(OWNER_KEY) != null) {
+        return Bytes.toString(getValue(OWNER_KEY));
+      }
+      // Note that every table should have an owner (i.e. should have OWNER_KEY set).
+      // hbase:meta and -ROOT- should return system user as owner, not null (see
+      // MasterFileSystem.java:bootstrap()).
+      return null;
+    }
+
+    public byte[] toByteArray() {
+      return ProtobufUtil.prependPBMagic(ProtobufUtil.convertToTableSchema(this).toByteArray());
+    }
+
+    /**
+     * @param bytes A pb serialized {@link ModifyableTableDescriptor} instance
+     * with pb magic prefix
+     * @return An instance of {@link ModifyableTableDescriptor} made from
+     * <code>bytes</code>
+     * @throws DeserializationException
+     * @see #toByteArray()
+     */
+    public static TableDescriptor parseFrom(final byte[] bytes)
+            throws DeserializationException {
+      if (!ProtobufUtil.isPBMagicPrefix(bytes)) {
+        throw new DeserializationException("Expected PB encoded ModifyableTableDescriptor");
+      }
+      int pblen = ProtobufUtil.lengthOfPBMagic();
+      HBaseProtos.TableSchema.Builder builder = HBaseProtos.TableSchema.newBuilder();
+      try {
+        ProtobufUtil.mergeFrom(builder, bytes, pblen, bytes.length - pblen);
+        return ProtobufUtil.convertToTableDesc(builder.build());
+      } catch (IOException e) {
+        throw new DeserializationException(e);
+      }
+    }
+
+    /**
+     * Getter for accessing the configuration value by key
+     */
+    @Override
+    public String getConfigurationValue(String key) {
+      return configuration.get(key);
+    }
+
+    /**
+     * Getter for fetching an unmodifiable {@link #configuration} map.
+     */
+    @Override
+    public Map<String, String> getConfiguration() {
+      // shallow pointer copy
+      return Collections.unmodifiableMap(configuration);
+    }
+
+    /**
+     * Setter for storing a configuration setting in {@link #configuration} map.
+     *
+     * @param key Config key. Same as XML config key e.g.
+     * hbase.something.or.other.
+     * @param value String value. If null, removes the setting.
+     * @return the modifyable TD
+     */
+    public ModifyableTableDescriptor setConfiguration(String key, String value) {
+      if (value == null) {
+        removeConfiguration(key);
+      } else {
+        configuration.put(key, value);
+      }
+      return this;
+    }
+
+    /**
+     * Remove a config setting represented by the key from the
+     * {@link #configuration} map
+     * @param key Config key.
+     */
+    public void removeConfiguration(final String key) {
+      configuration.remove(key);
+    }
+  }
+
+}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/UnmodifyableHTableDescriptor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/UnmodifyableHTableDescriptor.java
deleted file mode 100644
index b5f5ae9..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/UnmodifyableHTableDescriptor.java
+++ /dev/null
@@ -1,127 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.client;
-
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-
-/**
- * Read-only table descriptor.
- */
-@InterfaceAudience.Public
-public class UnmodifyableHTableDescriptor extends HTableDescriptor {
-  /**
-   * Default constructor.
-   * @deprecated  As of release 2.0.0. This will be removed in HBase 3.0.0.
-   *              Use {@link #UnmodifyableHTableDescriptor(HTableDescriptor)}.
-   */
-  @Deprecated
-  public UnmodifyableHTableDescriptor() {
-    super();
-  }
-
-  /*
-   * Create an unmodifyable copy of an HTableDescriptor
-   * @param desc
-   */
-  UnmodifyableHTableDescriptor(final HTableDescriptor desc) {
-    super(desc.getTableName(), getUnmodifyableFamilies(desc), desc.getValues());
-  }
-
-
-  /*
-   * @param desc
-   * @return Families as unmodifiable array.
-   */
-  private static HColumnDescriptor[] getUnmodifyableFamilies(
-      final HTableDescriptor desc) {
-    HColumnDescriptor [] f = new HColumnDescriptor[desc.getFamilies().size()];
-    int i = 0;
-    for (HColumnDescriptor c: desc.getFamilies()) {
-      f[i++] = c;
-    }
-    return f;
-  }
-
-  /**
-   * Does NOT add a column family. This object is immutable
-   * @param family HColumnDescriptor of familyto add.
-   */
-  @Override
-  public UnmodifyableHTableDescriptor addFamily(final HColumnDescriptor family) {
-    throw new UnsupportedOperationException("HTableDescriptor is read-only");
-  }
-
-  @Override
-  public UnmodifyableHTableDescriptor modifyFamily(HColumnDescriptor family) {
-    throw new UnsupportedOperationException("HTableDescriptor is read-only");
-  }
-
-  /**
-   * @param column
-   * @return Column descriptor for the passed family name or the family on
-   * passed in column.
-   */
-  @Override
-  public HColumnDescriptor removeFamily(final byte [] column) {
-    throw new UnsupportedOperationException("HTableDescriptor is read-only");
-  }
-
-  /**
-   * @see org.apache.hadoop.hbase.HTableDescriptor#setReadOnly(boolean)
-   */
-  @Override
-  public UnmodifyableHTableDescriptor setReadOnly(boolean readOnly) {
-    throw new UnsupportedOperationException("HTableDescriptor is read-only");
-  }
-
-  /**
-   * @see org.apache.hadoop.hbase.HTableDescriptor#setValue(byte[], byte[])
-   */
-  @Override
-  public UnmodifyableHTableDescriptor setValue(byte[] key, byte[] value) {
-    throw new UnsupportedOperationException("HTableDescriptor is read-only");
-  }
-
-  /**
-   * @see org.apache.hadoop.hbase.HTableDescriptor#setValue(java.lang.String, java.lang.String)
-   */
-  @Override
-  public UnmodifyableHTableDescriptor setValue(String key, String value) {
-    throw new UnsupportedOperationException("HTableDescriptor is read-only");
-  }
-
-  /**
-   * @see org.apache.hadoop.hbase.HTableDescriptor#setMaxFileSize(long)
-   */
-  @Override
-  public UnmodifyableHTableDescriptor setMaxFileSize(long maxFileSize) {
-    throw new UnsupportedOperationException("HTableDescriptor is read-only");
-  }
-
-  /**
-   * @see org.apache.hadoop.hbase.HTableDescriptor#setMemStoreFlushSize(long)
-   */
-  @Override
-  public UnmodifyableHTableDescriptor setMemStoreFlushSize(long memstoreFlushSize) {
-    throw new UnsupportedOperationException("HTableDescriptor is read-only");
-  }
-}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
index 04ce040..d4c4231 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
@@ -35,6 +35,7 @@
 import java.util.Set;
 import java.util.concurrent.Callable;
 import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
@@ -67,6 +68,7 @@
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.ImmutableHTableDescriptor;
 import org.apache.hadoop.hbase.client.Increment;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.PackagePrivateFieldAccessor;
@@ -76,6 +78,8 @@
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.SnapshotDescription;
 import org.apache.hadoop.hbase.client.SnapshotType;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
 import org.apache.hadoop.hbase.client.security.SecurityCapability;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
@@ -420,19 +424,34 @@
    * Get HTableDescriptor[] from GetTableDescriptorsResponse protobuf
    *
    * @param proto the GetTableDescriptorsResponse
-   * @return HTableDescriptor[]
+   * @return a immutable HTableDescriptor array
+   * @deprecated Use {@link #getTableDescriptorArray} after removing the HTableDescriptor
    */
+  @Deprecated
   public static HTableDescriptor[] getHTableDescriptorArray(GetTableDescriptorsResponse proto) {
     if (proto == null) return null;
 
     HTableDescriptor[] ret = new HTableDescriptor[proto.getTableSchemaCount()];
     for (int i = 0; i < proto.getTableSchemaCount(); ++i) {
-      ret[i] = convertToHTableDesc(proto.getTableSchema(i));
+      ret[i] = new ImmutableHTableDescriptor(convertToHTableDesc(proto.getTableSchema(i)));
     }
     return ret;
   }
 
   /**
+   * Get TableDescriptor[] from GetTableDescriptorsResponse protobuf
+   *
+   * @param proto the GetTableDescriptorsResponse
+   * @return TableDescriptor[]
+   */
+  public static TableDescriptor[] getTableDescriptorArray(GetTableDescriptorsResponse proto) {
+    if (proto == null) return new TableDescriptor[0];
+    return proto.getTableSchemaList()
+                .stream()
+                .map(ProtobufUtil::convertToTableDesc)
+                .toArray(size -> new TableDescriptor[size]);
+  }
+  /**
    * get the split keys in form "byte [][]" from a CreateTableRequest proto
    *
    * @param proto the CreateTableRequest
@@ -2850,7 +2869,7 @@
    * @param htd the HTableDescriptor
    * @return Convert the current {@link HTableDescriptor} into a pb TableSchema instance.
    */
-  public static TableSchema convertToTableSchema(HTableDescriptor htd) {
+  public static TableSchema convertToTableSchema(TableDescriptor htd) {
     TableSchema.Builder builder = TableSchema.newBuilder();
     builder.setTableName(toProtoTableName(htd.getTableName()));
     for (Map.Entry<Bytes, Bytes> e : htd.getValues().entrySet()) {
@@ -2875,7 +2894,9 @@
    * Converts a TableSchema to HTableDescriptor
    * @param ts A pb TableSchema instance.
    * @return An {@link HTableDescriptor} made from the passed in pb <code>ts</code>.
+   * @deprecated Use {@link #convertToTableDesc} after removing the HTableDescriptor
    */
+  @Deprecated
   public static HTableDescriptor convertToHTableDesc(final TableSchema ts) {
     List<ColumnFamilySchema> list = ts.getColumnFamiliesList();
     HColumnDescriptor [] hcds = new HColumnDescriptor[list.size()];
@@ -2897,6 +2918,25 @@
   }
 
   /**
+   * Converts a TableSchema to TableDescriptor
+   * @param ts A pb TableSchema instance.
+   * @return An {@link TableDescriptor} made from the passed in pb <code>ts</code>.
+   */
+  public static TableDescriptor convertToTableDesc(final TableSchema ts) {
+    TableDescriptorBuilder builder
+      = TableDescriptorBuilder.newBuilder(ProtobufUtil.toTableName(ts.getTableName()));
+    ts.getColumnFamiliesList()
+      .stream()
+      .map(ProtobufUtil::convertToHColumnDesc)
+      .forEach(builder::addFamily);
+    ts.getAttributesList()
+      .forEach(a -> builder.setValue(a.getFirst().toByteArray(), a.getSecond().toByteArray()));
+    ts.getConfigurationList()
+      .forEach(a -> builder.setConfiguration(a.getName(), a.getValue()));
+    return builder.build();
+  }
+
+  /**
    * Creates {@link CompactionState} from
    * {@link org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState}
    * state
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
index a513d66..366e050 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
@@ -45,6 +45,7 @@
 import org.apache.hadoop.hbase.client.Row;
 import org.apache.hadoop.hbase.client.RowMutations;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.replication.ReplicationSerDeHelper;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.filter.ByteArrayComparable;
@@ -1244,7 +1245,7 @@
    * @return a CreateTableRequest
    */
   public static CreateTableRequest buildCreateTableRequest(
-      final HTableDescriptor hTableDesc,
+      final TableDescriptor hTableDesc,
       final byte [][] splitKeys,
       final long nonceGroup,
       final long nonce) {
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestImmutableHTableDescriptor.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestImmutableHTableDescriptor.java
new file mode 100644
index 0000000..91ef72a
--- /dev/null
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestImmutableHTableDescriptor.java
@@ -0,0 +1,102 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.client;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.function.Consumer;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.BuilderStyleTest;
+import org.apache.hadoop.hbase.util.Bytes;
+import static org.junit.Assert.fail;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+
+@Category({ClientTests.class, SmallTests.class})
+public class TestImmutableHTableDescriptor {
+  @Rule
+  public TestName name = new TestName();
+  private static final List<Consumer<ImmutableHTableDescriptor>> TEST_FUNCTION = Arrays.asList(
+    htd -> htd.setValue("a", "a"),
+    htd -> htd.setValue(Bytes.toBytes("a"), Bytes.toBytes("a")),
+    htd -> htd.setValue(new Bytes(Bytes.toBytes("a")), new Bytes(Bytes.toBytes("a"))),
+    htd -> htd.setCompactionEnabled(false),
+    htd -> htd.setConfiguration("aaa", "ccc"),
+    htd -> htd.setDurability(Durability.USE_DEFAULT),
+    htd -> htd.setFlushPolicyClassName("class"),
+    htd -> htd.setMaxFileSize(123),
+    htd -> htd.setMemStoreFlushSize(123123123),
+    htd -> htd.setNormalizationEnabled(false),
+    htd -> htd.setPriority(123),
+    htd -> htd.setReadOnly(true),
+    htd -> htd.setRegionMemstoreReplication(true),
+    htd -> htd.setRegionReplication(123),
+    htd -> htd.setRegionSplitPolicyClassName("class"),
+    htd -> htd.addFamily(new HColumnDescriptor(Bytes.toBytes("fm"))),
+    htd -> htd.remove(new Bytes(Bytes.toBytes("aaa"))),
+    htd -> htd.remove("aaa"),
+    htd -> htd.remove(Bytes.toBytes("aaa")),
+    htd -> htd.removeConfiguration("xxx"),
+    htd -> htd.removeFamily(Bytes.toBytes("fm")),
+    htd -> {
+      try {
+        htd.addCoprocessor("xxx");
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      }
+    }
+  );
+
+  @Test
+  public void testImmutable() {
+    ImmutableHTableDescriptor htd = new ImmutableHTableDescriptor(
+      new HTableDescriptor(TableName.valueOf(name.getMethodName())));
+    TEST_FUNCTION.forEach(f -> {
+      try {
+        f.accept(htd);
+        fail("ImmutableHTableDescriptor can't be modified!!!");
+      } catch (UnsupportedOperationException e) {
+      }
+    });
+  }
+
+  @Test
+  public void testClassMethodsAreBuilderStyle() {
+  /* ImmutableHTableDescriptor should have a builder style setup where setXXX/addXXX methods
+   * can be chainable together:
+   * . For example:
+   * ImmutableHTableDescriptor d
+   *   = new ImmutableHTableDescriptor()
+   *     .setFoo(foo)
+   *     .setBar(bar)
+   *     .setBuz(buz)
+   *
+   * This test ensures that all methods starting with "set" returns the declaring object
+   */
+
+      BuilderStyleTest.assertClassesAreBuilderStyle(ImmutableHTableDescriptor.class);
+  }
+}
\ No newline at end of file
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestTableDescriptorBuilder.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestTableDescriptorBuilder.java
new file mode 100644
index 0000000..c4ecacf
--- /dev/null
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestTableDescriptorBuilder.java
@@ -0,0 +1,376 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.client;
+
+import org.apache.hadoop.hbase.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+import java.util.regex.Pattern;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.testclassification.MiscTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.BuilderStyleTest;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+
+/**
+ * Test setting values in the descriptor
+ */
+@Category({MiscTests.class, SmallTests.class})
+public class TestTableDescriptorBuilder {
+  private static final Log LOG = LogFactory.getLog(TestTableDescriptorBuilder.class);
+
+  @Rule
+  public TestName name = new TestName();
+
+  @Test (expected=IOException.class)
+  public void testAddCoprocessorTwice() throws IOException {
+    String cpName = "a.b.c.d";
+    TableDescriptor htd
+      = TableDescriptorBuilder.newBuilder(TableName.META_TABLE_NAME)
+            .addCoprocessor(cpName)
+            .addCoprocessor(cpName)
+            .build();
+  }
+
+  @Test
+  public void testAddCoprocessorWithSpecStr() throws IOException {
+    String cpName = "a.b.c.d";
+    TableDescriptorBuilder builder
+      = TableDescriptorBuilder.newBuilder(TableName.META_TABLE_NAME);
+
+    try {
+      builder.addCoprocessorWithSpec(cpName);
+      fail();
+    } catch (IllegalArgumentException iae) {
+      // Expected as cpName is invalid
+    }
+
+    // Try minimal spec.
+    try {
+      builder.addCoprocessorWithSpec("file:///some/path" + "|" + cpName);
+      fail();
+    } catch (IllegalArgumentException iae) {
+      // Expected to be invalid
+    }
+
+    // Try more spec.
+    String spec = "hdfs:///foo.jar|com.foo.FooRegionObserver|1001|arg1=1,arg2=2";
+    try {
+      builder.addCoprocessorWithSpec(spec);
+    } catch (IllegalArgumentException iae) {
+      fail();
+    }
+
+    // Try double add of same coprocessor
+    try {
+      builder.addCoprocessorWithSpec(spec);
+      fail();
+    } catch (IOException ioe) {
+      // Expect that the coprocessor already exists
+    }
+  }
+
+  @Test
+  public void testPb() throws DeserializationException, IOException {
+    final int v = 123;
+    TableDescriptor htd
+      = TableDescriptorBuilder.newBuilder(TableName.META_TABLE_NAME)
+          .setMaxFileSize(v)
+          .setDurability(Durability.ASYNC_WAL)
+          .setReadOnly(true)
+          .setRegionReplication(2)
+          .build();
+
+    byte [] bytes = TableDescriptorBuilder.toByteArray(htd);
+    TableDescriptor deserializedHtd = TableDescriptorBuilder.newBuilder(bytes).build();
+    assertEquals(htd, deserializedHtd);
+    assertEquals(v, deserializedHtd.getMaxFileSize());
+    assertTrue(deserializedHtd.isReadOnly());
+    assertEquals(Durability.ASYNC_WAL, deserializedHtd.getDurability());
+    assertEquals(deserializedHtd.getRegionReplication(), 2);
+  }
+
+  /**
+   * Test cps in the table description
+   * @throws Exception
+   */
+  @Test
+  public void testGetSetRemoveCP() throws Exception {
+    // simple CP
+    String className = "org.apache.hadoop.hbase.coprocessor.SimpleRegionObserver";
+    TableDescriptor desc
+      = TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+         .addCoprocessor(className) // add and check that it is present
+        .build();
+    assertTrue(desc.hasCoprocessor(className));
+    desc = TableDescriptorBuilder.newBuilder(desc)
+         .removeCoprocessor(className) // remove it and check that it is gone
+        .build();
+    assertFalse(desc.hasCoprocessor(className));
+  }
+
+  /**
+   * Test cps in the table description
+   * @throws Exception
+   */
+  @Test
+  public void testSetListRemoveCP() throws Exception {
+    TableDescriptor desc
+      = TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName())).build();
+    // Check that any coprocessor is present.
+    assertTrue(desc.getCoprocessors().isEmpty());
+
+    // simple CP
+    String className1 = "org.apache.hadoop.hbase.coprocessor.SimpleRegionObserver";
+    String className2 = "org.apache.hadoop.hbase.coprocessor.SampleRegionWALObserver";
+    desc = TableDescriptorBuilder.newBuilder(desc)
+            .addCoprocessor(className1) // Add the 1 coprocessor and check if present.
+            .build();
+    assertTrue(desc.getCoprocessors().size() == 1);
+    assertTrue(desc.getCoprocessors().contains(className1));
+
+    desc = TableDescriptorBuilder.newBuilder(desc)
+            // Add the 2nd coprocessor and check if present.
+            // remove it and check that it is gone
+            .addCoprocessor(className2)
+            .build();
+    assertTrue(desc.getCoprocessors().size() == 2);
+    assertTrue(desc.getCoprocessors().contains(className2));
+
+    desc = TableDescriptorBuilder.newBuilder(desc)
+            // Remove one and check
+            .removeCoprocessor(className1)
+            .build();
+    assertTrue(desc.getCoprocessors().size() == 1);
+    assertFalse(desc.getCoprocessors().contains(className1));
+    assertTrue(desc.getCoprocessors().contains(className2));
+
+    desc = TableDescriptorBuilder.newBuilder(desc)
+            // Remove the last and check
+            .removeCoprocessor(className2)
+            .build();
+    assertTrue(desc.getCoprocessors().isEmpty());
+    assertFalse(desc.getCoprocessors().contains(className1));
+    assertFalse(desc.getCoprocessors().contains(className2));
+  }
+
+  /**
+   * Test that we add and remove strings from settings properly.
+   * @throws Exception
+   */
+  @Test
+  public void testRemoveString() throws Exception {
+    byte[] key = Bytes.toBytes("Some");
+    byte[] value = Bytes.toBytes("value");
+    TableDescriptor desc
+      = TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+            .setValue(key, value)
+            .build();
+    assertTrue(Bytes.equals(value, desc.getValue(key)));
+    desc = TableDescriptorBuilder.newBuilder(desc)
+            .remove(key)
+            .build();
+    assertTrue(desc.getValue(key) == null);
+  }
+
+  String legalTableNames[] = { "foo", "with-dash_under.dot", "_under_start_ok",
+      "with-dash.with_underscore", "02-01-2012.my_table_01-02", "xyz._mytable_", "9_9_0.table_02"
+      , "dot1.dot2.table", "new.-mytable", "with-dash.with.dot", "legal..t2", "legal..legal.t2",
+      "trailingdots..", "trailing.dots...", "ns:mytable", "ns:_mytable_", "ns:my_table_01-02"};
+  String illegalTableNames[] = { ".dot_start_illegal", "-dash_start_illegal", "spaces not ok",
+      "-dash-.start_illegal", "new.table with space", "01 .table", "ns:-illegaldash",
+      "new:.illegaldot", "new:illegalcolon1:", "new:illegalcolon1:2"};
+
+  @Test
+  public void testLegalTableNames() {
+    for (String tn : legalTableNames) {
+      TableName.isLegalFullyQualifiedTableName(Bytes.toBytes(tn));
+    }
+  }
+
+  @Test
+  public void testIllegalTableNames() {
+    for (String tn : illegalTableNames) {
+      try {
+        TableName.isLegalFullyQualifiedTableName(Bytes.toBytes(tn));
+        fail("invalid tablename " + tn + " should have failed");
+      } catch (Exception e) {
+        // expected
+      }
+    }
+  }
+
+  @Test
+  public void testLegalTableNamesRegex() {
+    for (String tn : legalTableNames) {
+      TableName tName = TableName.valueOf(tn);
+      assertTrue("Testing: '" + tn + "'", Pattern.matches(TableName.VALID_USER_TABLE_REGEX,
+          tName.getNameAsString()));
+    }
+  }
+
+  @Test
+  public void testIllegalTableNamesRegex() {
+    for (String tn : illegalTableNames) {
+      LOG.info("Testing: '" + tn + "'");
+      assertFalse(Pattern.matches(TableName.VALID_USER_TABLE_REGEX, tn));
+    }
+  }
+
+    /**
+   * Test default value handling for maxFileSize
+   */
+  @Test
+  public void testGetMaxFileSize() {
+    TableDescriptor desc = TableDescriptorBuilder
+            .newBuilder(TableName.valueOf(name.getMethodName())).build();
+    assertEquals(-1, desc.getMaxFileSize());
+    desc = TableDescriptorBuilder
+            .newBuilder(TableName.valueOf(name.getMethodName()))
+            .setMaxFileSize(1111L).build();
+    assertEquals(1111L, desc.getMaxFileSize());
+  }
+
+  /**
+   * Test default value handling for memStoreFlushSize
+   */
+  @Test
+  public void testGetMemStoreFlushSize() {
+    TableDescriptor desc = TableDescriptorBuilder
+            .newBuilder(TableName.valueOf(name.getMethodName())).build();
+    assertEquals(-1, desc.getMemStoreFlushSize());
+    desc = TableDescriptorBuilder
+            .newBuilder(TableName.valueOf(name.getMethodName()))
+            .setMemStoreFlushSize(1111L).build();
+    assertEquals(1111L, desc.getMemStoreFlushSize());
+  }
+
+  /**
+   * Test that we add and remove strings from configuration properly.
+   */
+  @Test
+  public void testAddGetRemoveConfiguration() {
+    String key = "Some";
+    String value = "value";
+    TableDescriptor desc = TableDescriptorBuilder
+            .newBuilder(TableName.valueOf(name.getMethodName()))
+            .setConfiguration(key, value)
+            .build();
+    assertEquals(value, desc.getConfigurationValue(key));
+    desc = TableDescriptorBuilder
+            .newBuilder(desc)
+            .removeConfiguration(key)
+            .build();
+    assertEquals(null, desc.getConfigurationValue(key));
+  }
+
+  @Test
+  public void testClassMethodsAreBuilderStyle() {
+    BuilderStyleTest.assertClassesAreBuilderStyle(TableDescriptorBuilder.class);
+  }
+
+  @Test
+  public void testModifyFamily() {
+    byte[] familyName = Bytes.toBytes("cf");
+    HColumnDescriptor hcd = new HColumnDescriptor(familyName);
+    hcd.setBlocksize(1000);
+    hcd.setDFSReplication((short) 3);
+    TableDescriptor htd
+      = TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+              .addFamily(hcd)
+              .build();
+
+    assertEquals(1000, htd.getFamily(familyName).getBlocksize());
+    assertEquals(3, htd.getFamily(familyName).getDFSReplication());
+    hcd = new HColumnDescriptor(familyName);
+    hcd.setBlocksize(2000);
+    hcd.setDFSReplication((short) 1);
+    htd = TableDescriptorBuilder.newBuilder(htd)
+              .modifyFamily(hcd)
+              .build();
+    assertEquals(2000, htd.getFamily(familyName).getBlocksize());
+    assertEquals(1, htd.getFamily(familyName).getDFSReplication());
+  }
+
+  @Test(expected=IllegalArgumentException.class)
+  public void testModifyInexistentFamily() {
+    byte[] familyName = Bytes.toBytes("cf");
+    HColumnDescriptor hcd = new HColumnDescriptor(familyName);
+    TableDescriptor htd = TableDescriptorBuilder
+            .newBuilder(TableName.valueOf(name.getMethodName()))
+            .modifyFamily(hcd)
+            .build();
+  }
+
+  @Test(expected=IllegalArgumentException.class)
+  public void testAddDuplicateFamilies() {
+    byte[] familyName = Bytes.toBytes("cf");
+    HColumnDescriptor hcd = new HColumnDescriptor(familyName);
+    hcd.setBlocksize(1000);
+    TableDescriptor htd = TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+            .addFamily(hcd)
+            .build();
+    assertEquals(1000, htd.getFamily(familyName).getBlocksize());
+    hcd = new HColumnDescriptor(familyName);
+    hcd.setBlocksize(2000);
+    // add duplicate column
+    TableDescriptorBuilder.newBuilder(htd).addFamily(hcd).build();
+  }
+
+  @Test
+  public void testPriority() {
+    TableDescriptor htd = TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+            .setPriority(42)
+            .build();
+    assertEquals(42, htd.getPriority());
+  }
+
+  @Test
+  public void testSerialReplicationScope() {
+    HColumnDescriptor hcdWithScope = new HColumnDescriptor(Bytes.toBytes("cf0"));
+    hcdWithScope.setScope(HConstants.REPLICATION_SCOPE_SERIAL);
+    HColumnDescriptor hcdWithoutScope = new HColumnDescriptor(Bytes.toBytes("cf1"));
+    TableDescriptor htd = TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+            .addFamily(hcdWithoutScope)
+            .build();
+    assertFalse(htd.hasSerialReplicationScope());
+
+    htd = TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+            .addFamily(hcdWithScope)
+            .build();
+    assertTrue(htd.hasSerialReplicationScope());
+
+    htd = TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+            .addFamily(hcdWithScope)
+            .addFamily(hcdWithoutScope)
+            .build();
+    assertTrue(htd.hasSerialReplicationScope());
+  }
+}
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestUnmodifyableHTableDescriptor.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestUnmodifyableHTableDescriptor.java
deleted file mode 100644
index dca0c1f..0000000
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestUnmodifyableHTableDescriptor.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.client;
-
-import org.apache.hadoop.hbase.testclassification.ClientTests;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.apache.hadoop.hbase.util.BuilderStyleTest;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-@Category({ClientTests.class, SmallTests.class})
-public class TestUnmodifyableHTableDescriptor {
-
-    @Test
-    public void testClassMethodsAreBuilderStyle() {
-    /* UnmodifyableHTableDescriptor should have a builder style setup where setXXX/addXXX methods
-     * can be chainable together:
-     * . For example:
-     * UnmodifyableHTableDescriptor d
-     *   = new UnmodifyableHTableDescriptor()
-     *     .setFoo(foo)
-     *     .setBar(bar)
-     *     .setBuz(buz)
-     *
-     * This test ensures that all methods starting with "set" returns the declaring object
-     */
-
-        BuilderStyleTest.assertClassesAreBuilderStyle(UnmodifyableHTableDescriptor.class);
-    }
-
-}
\ No newline at end of file
diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/RowResourceBase.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/RowResourceBase.java
index b4b00a9..406b4e5 100644
--- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/RowResourceBase.java
+++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/RowResourceBase.java
@@ -111,7 +111,7 @@
     if (admin.tableExists(TABLE_NAME)) {
       TEST_UTIL.deleteTable(TABLE_NAME);
     }
-    HTableDescriptor htd = new HTableDescriptor(TABLE);
+    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(TABLE));
     htd.addFamily(new HColumnDescriptor(CFA));
     htd.addFamily(new HColumnDescriptor(CFB));
     admin.createTable(htd);
diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteAdminRetries.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteAdminRetries.java
index 7c888e0..b926d82 100644
--- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteAdminRetries.java
+++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteAdminRetries.java
@@ -32,6 +32,7 @@
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.testclassification.RestTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -116,7 +117,7 @@
     testTimedOutCall(new CallExecutor() {
       @Override
       public void run() throws Exception {
-        remoteAdmin.createTable(new HTableDescriptor(Bytes.toBytes("TestTable")));
+        remoteAdmin.createTable(new HTableDescriptor(TableName.valueOf("TestTable")));
       }
     });
     verify(client, times(RETRIES)).put(anyString(), anyString(), any(byte[].class));
diff --git a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManager.java b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManager.java
index ab5c09f..e1fd82c 100644
--- a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManager.java
+++ b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManager.java
@@ -39,7 +39,6 @@
   //Assigned before user tables
   TableName RSGROUP_TABLE_NAME =
       TableName.valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "rsgroup");
-  byte[] RSGROUP_TABLE_NAME_BYTES = RSGROUP_TABLE_NAME.toBytes();
   String rsGroupZNode = "rsgroup";
   byte[] META_FAMILY_BYTES = Bytes.toBytes("m");
   byte[] META_QUALIFIER_BYTES = Bytes.toBytes("i");
diff --git a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java
index 9f77c77..f2c6118 100644
--- a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java
+++ b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java
@@ -117,7 +117,7 @@
   /** Table descriptor for <code>hbase:rsgroup</code> catalog table */
   private final static HTableDescriptor RSGROUP_TABLE_DESC;
   static {
-    RSGROUP_TABLE_DESC = new HTableDescriptor(RSGROUP_TABLE_NAME_BYTES);
+    RSGROUP_TABLE_DESC = new HTableDescriptor(RSGROUP_TABLE_NAME);
     RSGROUP_TABLE_DESC.addFamily(new HColumnDescriptor(META_FAMILY_BYTES));
     RSGROUP_TABLE_DESC.setRegionSplitPolicyClassName(DisabledRegionSplitPolicy.class.getName());
     try {
diff --git a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroups.java b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroups.java
index d6bd43b..6ef162b 100644
--- a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroups.java
+++ b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroups.java
@@ -189,7 +189,7 @@
     final byte[] tableName = Bytes.toBytes(tablePrefix + "_testCreateAndAssign");
     admin.modifyNamespace(NamespaceDescriptor.create("default")
         .addConfiguration(RSGroupInfo.NAMESPACE_DESC_PROP_GROUP, "default").build());
-    final HTableDescriptor desc = new HTableDescriptor(tableName);
+    final HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
     desc.addFamily(new HColumnDescriptor("f"));
     admin.createTable(desc);
     //wait for created table to be assigned
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreTool.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreTool.java
index d34701f..f744ecb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreTool.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreTool.java
@@ -176,7 +176,7 @@
         LOG.debug("Found descriptor " + tableDescriptor + " through " + incrBackupId);
 
         TableName newTableName = newTableNames[i];
-        HTableDescriptor newTableDescriptor = admin.getTableDescriptor(newTableName);
+        HTableDescriptor newTableDescriptor = new HTableDescriptor(admin.getTableDescriptor(newTableName));
         List<HColumnDescriptor> families = Arrays.asList(tableDescriptor.getColumnFamilies());
         List<HColumnDescriptor> existingFamilies =
             Arrays.asList(newTableDescriptor.getColumnFamilies());
@@ -325,7 +325,7 @@
           LOG.debug("find table descriptor but no archive dir for table " + tableName
               + ", will only create table");
         }
-        tableDescriptor.setName(newTableName);
+        tableDescriptor = new HTableDescriptor(newTableName, tableDescriptor);
         checkAndCreateTable(conn, tableBackupPath, tableName, newTableName, null, tableDescriptor,
           truncateIfExists);
         return;
@@ -338,7 +338,7 @@
     if (tableDescriptor == null) {
       tableDescriptor = new HTableDescriptor(newTableName);
     } else {
-      tableDescriptor.setName(newTableName);
+      tableDescriptor = new HTableDescriptor(newTableName, tableDescriptor);
     }
 
     // record all region dirs:
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java
index c2ca3eb..70fe5c5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java
@@ -48,6 +48,7 @@
 import org.apache.hadoop.hbase.TableInfoMissingException;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.regionserver.BloomType;
 
@@ -128,74 +129,69 @@
   @VisibleForTesting
   public static HTableDescriptor createMetaTableDescriptor(final Configuration conf)
       throws IOException {
-    HTableDescriptor metaDescriptor = new HTableDescriptor(
-        TableName.META_TABLE_NAME,
-        new HColumnDescriptor[] {
-            new HColumnDescriptor(HConstants.CATALOG_FAMILY)
-                .setMaxVersions(conf.getInt(HConstants.HBASE_META_VERSIONS,
+    return new HTableDescriptor(TableDescriptorBuilder.newBuilder(TableName.META_TABLE_NAME)
+          .addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)
+            .setMaxVersions(conf.getInt(HConstants.HBASE_META_VERSIONS,
                     HConstants.DEFAULT_HBASE_META_VERSIONS))
-                .setInMemory(true)
-                .setBlocksize(conf.getInt(HConstants.HBASE_META_BLOCK_SIZE,
+            .setInMemory(true)
+            .setBlocksize(conf.getInt(HConstants.HBASE_META_BLOCK_SIZE,
                     HConstants.DEFAULT_HBASE_META_BLOCK_SIZE))
-                .setScope(HConstants.REPLICATION_SCOPE_LOCAL)
-                    // Disable blooms for meta.  Needs work.  Seems to mess w/ getClosestOrBefore.
-                .setBloomFilterType(BloomType.NONE)
-                    // Enable cache of data blocks in L1 if more than one caching tier deployed:
-                    // e.g. if using CombinedBlockCache (BucketCache).
-                .setCacheDataInL1(true),
-            new HColumnDescriptor(HConstants.REPLICATION_BARRIER_FAMILY)
-                .setMaxVersions(conf.getInt(HConstants.HBASE_META_VERSIONS,
+            .setScope(HConstants.REPLICATION_SCOPE_LOCAL)
+            // Disable blooms for meta.  Needs work.  Seems to mess w/ getClosestOrBefore.
+            .setBloomFilterType(BloomType.NONE)
+            // Enable cache of data blocks in L1 if more than one caching tier deployed:
+            // e.g. if using CombinedBlockCache (BucketCache).
+            .setCacheDataInL1(true))
+          .addFamily(new HColumnDescriptor(HConstants.REPLICATION_BARRIER_FAMILY)
+            .setMaxVersions(conf.getInt(HConstants.HBASE_META_VERSIONS,
                     HConstants.DEFAULT_HBASE_META_VERSIONS))
-                .setInMemory(true)
-                .setBlocksize(conf.getInt(HConstants.HBASE_META_BLOCK_SIZE,
+            .setInMemory(true)
+            .setBlocksize(conf.getInt(HConstants.HBASE_META_BLOCK_SIZE,
                     HConstants.DEFAULT_HBASE_META_BLOCK_SIZE))
-                .setScope(HConstants.REPLICATION_SCOPE_LOCAL)
-                // Disable blooms for meta.  Needs work.  Seems to mess w/ getClosestOrBefore.
-                .setBloomFilterType(BloomType.NONE)
-                // Enable cache of data blocks in L1 if more than one caching tier deployed:
-                // e.g. if using CombinedBlockCache (BucketCache).
-                .setCacheDataInL1(true),
-            new HColumnDescriptor(HConstants.REPLICATION_POSITION_FAMILY)
-                .setMaxVersions(conf.getInt(HConstants.HBASE_META_VERSIONS,
+            .setScope(HConstants.REPLICATION_SCOPE_LOCAL)
+            // Disable blooms for meta.  Needs work.  Seems to mess w/ getClosestOrBefore.
+            .setBloomFilterType(BloomType.NONE)
+            // Enable cache of data blocks in L1 if more than one caching tier deployed:
+            // e.g. if using CombinedBlockCache (BucketCache).
+            .setCacheDataInL1(true))
+          .addFamily(new HColumnDescriptor(HConstants.REPLICATION_POSITION_FAMILY)
+            .setMaxVersions(conf.getInt(HConstants.HBASE_META_VERSIONS,
                     HConstants.DEFAULT_HBASE_META_VERSIONS))
-                .setInMemory(true)
-                .setBlocksize(conf.getInt(HConstants.HBASE_META_BLOCK_SIZE,
+            .setInMemory(true)
+            .setBlocksize(conf.getInt(HConstants.HBASE_META_BLOCK_SIZE,
                     HConstants.DEFAULT_HBASE_META_BLOCK_SIZE))
-                .setScope(HConstants.REPLICATION_SCOPE_LOCAL)
-                // Disable blooms for meta.  Needs work.  Seems to mess w/ getClosestOrBefore.
-                .setBloomFilterType(BloomType.NONE)
-                // Enable cache of data blocks in L1 if more than one caching tier deployed:
-                // e.g. if using CombinedBlockCache (BucketCache).
-                .setCacheDataInL1(true),
-            new HColumnDescriptor(HConstants.REPLICATION_META_FAMILY)
-                .setMaxVersions(conf.getInt(HConstants.HBASE_META_VERSIONS,
+            .setScope(HConstants.REPLICATION_SCOPE_LOCAL)
+            // Disable blooms for meta.  Needs work.  Seems to mess w/ getClosestOrBefore.
+            .setBloomFilterType(BloomType.NONE)
+            // Enable cache of data blocks in L1 if more than one caching tier deployed:
+            // e.g. if using CombinedBlockCache (BucketCache).
+            .setCacheDataInL1(true))
+          .addFamily(new HColumnDescriptor(HConstants.REPLICATION_META_FAMILY)
+            .setMaxVersions(conf.getInt(HConstants.HBASE_META_VERSIONS,
                     HConstants.DEFAULT_HBASE_META_VERSIONS))
-                .setInMemory(true)
-                .setBlocksize(conf.getInt(HConstants.HBASE_META_BLOCK_SIZE,
+            .setInMemory(true)
+            .setBlocksize(conf.getInt(HConstants.HBASE_META_BLOCK_SIZE,
                     HConstants.DEFAULT_HBASE_META_BLOCK_SIZE))
-                .setScope(HConstants.REPLICATION_SCOPE_LOCAL)
-                // Disable blooms for meta.  Needs work.  Seems to mess w/ getClosestOrBefore.
-                .setBloomFilterType(BloomType.NONE)
-                // Enable cache of data blocks in L1 if more than one caching tier deployed:
-                // e.g. if using CombinedBlockCache (BucketCache).
-                .setCacheDataInL1(true),
-            new HColumnDescriptor(HConstants.TABLE_FAMILY)
-                // Ten is arbitrary number.  Keep versions to help debugging.
-                .setMaxVersions(10)
-                .setInMemory(true)
-                .setBlocksize(8 * 1024)
-                .setScope(HConstants.REPLICATION_SCOPE_LOCAL)
-                    // Disable blooms for meta.  Needs work.  Seems to mess w/ getClosestOrBefore.
-                .setBloomFilterType(BloomType.NONE)
-                    // Enable cache of data blocks in L1 if more than one caching tier deployed:
-                    // e.g. if using CombinedBlockCache (BucketCache).
-                .setCacheDataInL1(true)
-        }) {
-    };
-    metaDescriptor.addCoprocessor(
-        "org.apache.hadoop.hbase.coprocessor.MultiRowMutationEndpoint",
-        null, Coprocessor.PRIORITY_SYSTEM, null);
-    return metaDescriptor;
+            .setScope(HConstants.REPLICATION_SCOPE_LOCAL)
+            // Disable blooms for meta.  Needs work.  Seems to mess w/ getClosestOrBefore.
+            .setBloomFilterType(BloomType.NONE)
+            // Enable cache of data blocks in L1 if more than one caching tier deployed:
+            // e.g. if using CombinedBlockCache (BucketCache).
+            .setCacheDataInL1(true))
+          .addFamily(new HColumnDescriptor(HConstants.TABLE_FAMILY)
+            // Ten is arbitrary number.  Keep versions to help debugging.
+            .setMaxVersions(10)
+            .setInMemory(true)
+            .setBlocksize(8 * 1024)
+            .setScope(HConstants.REPLICATION_SCOPE_LOCAL)
+            // Disable blooms for meta.  Needs work.  Seems to mess w/ getClosestOrBefore.
+            .setBloomFilterType(BloomType.NONE)
+            // Enable cache of data blocks in L1 if more than one caching tier deployed:
+            // e.g. if using CombinedBlockCache (BucketCache).
+            .setCacheDataInL1(true))
+          .addCoprocessor("org.apache.hadoop.hbase.coprocessor.MultiRowMutationEndpoint",
+            null, Coprocessor.PRIORITY_SYSTEM, null)
+          .build());
   }
 
   @Override
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
index c0ddbfc..e0edfa3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
@@ -1726,7 +1726,7 @@
   public static void setReplicas(Admin admin, TableName table, int replicaCount)
       throws IOException, InterruptedException {
     admin.disableTable(table);
-    HTableDescriptor desc = admin.getTableDescriptor(table);
+    HTableDescriptor desc = new HTableDescriptor(admin.getTableDescriptor(table));
     desc.setRegionReplication(replicaCount);
     admin.modifyTable(desc.getTableName(), desc);
     admin.enableTable(table);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestFSTableDescriptorForceCreation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestFSTableDescriptorForceCreation.java
index 7744631..7457f43 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestFSTableDescriptorForceCreation.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestFSTableDescriptorForceCreation.java
@@ -59,7 +59,7 @@
     // Cleanup old tests if any detritus laying around.
     Path rootdir = new Path(UTIL.getDataTestDir(), name);
     FSTableDescriptors fstd = new FSTableDescriptors(UTIL.getConfiguration(), fs, rootdir);
-    HTableDescriptor htd = new HTableDescriptor(name);
+    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name));
     fstd.add(htd);
     assertFalse("Should not create new table descriptor", fstd.createTableDescriptor(htd, false));
   }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java
index b7430fe..f6dbb41 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java
@@ -92,7 +92,7 @@
       TEST_UTIL.createTable(tables[i], FAMILY);
     }
 
-    HTableDescriptor[] tableDescs = admin.listTables().get();
+    TableDescriptor[] tableDescs = admin.listTables().get();
     int size = tableDescs.length;
     assertTrue(size >= tables.length);
     for (int i = 0; i < tables.length && i < size; i++) {
@@ -140,13 +140,13 @@
     htd.addFamily(fam2);
     htd.addFamily(fam3);
     admin.createTable(htd).join();
-    HTableDescriptor confirmedHtd = admin.getTableDescriptor(htd.getTableName()).get();
-    assertEquals(htd.compareTo(confirmedHtd), 0);
+    TableDescriptor confirmedHtd = admin.getTableDescriptor(htd.getTableName()).get();
+    assertEquals(htd.compareTo(new HTableDescriptor(confirmedHtd)), 0);
   }
 
   @Test(timeout = 300000)
   public void testCreateTable() throws Exception {
-    HTableDescriptor[] tables = admin.listTables().get();
+    TableDescriptor[] tables = admin.listTables().get();
     int numTables = tables.length;
     final  TableName tableName = TableName.valueOf(name.getMethodName());
     admin.createTable(new HTableDescriptor(tableName).addFamily(new HColumnDescriptor(FAMILY)))
@@ -452,7 +452,7 @@
           } catch (Exception e) {
           }
         });
-    HTableDescriptor[] failed = admin.deleteTables(Pattern.compile("testDeleteTables.*")).get();
+    TableDescriptor[] failed = admin.deleteTables(Pattern.compile("testDeleteTables.*")).get();
     assertEquals(0, failed.length);
     Arrays.stream(tables).forEach((table) -> {
       admin.tableExists(table).thenAccept((exist) -> assertFalse(exist)).join();
@@ -727,7 +727,7 @@
       // Modify colymn family
       admin.modifyColumnFamily(tableName, cfDescriptor).join();
 
-      HTableDescriptor htd = admin.getTableDescriptor(tableName).get();
+      TableDescriptor htd = admin.getTableDescriptor(tableName).get();
       HColumnDescriptor hcfd = htd.getFamily(FAMILY_0);
       assertTrue(hcfd.getBlocksize() == newBlockSize);
     } finally {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableBatch.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableBatch.java
index 30b3b5b..61ff2be 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableBatch.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableBatch.java
@@ -216,7 +216,7 @@
   @Test
   public void testPartialSuccess() throws IOException, InterruptedException, ExecutionException {
     Admin admin = TEST_UTIL.getAdmin();
-    HTableDescriptor htd = admin.getTableDescriptor(TABLE_NAME);
+    HTableDescriptor htd = new HTableDescriptor(admin.getTableDescriptor(TABLE_NAME));
     htd.addCoprocessor(ErrorInjectObserver.class.getName());
     admin.modifyTable(TABLE_NAME, htd);
     AsyncTableBase table = tableGetter.apply(TABLE_NAME);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdminWithClusters.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdminWithClusters.java
index 56f4141..f46354c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdminWithClusters.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdminWithClusters.java
@@ -77,7 +77,7 @@
 
   @Test(timeout = 300000)
   public void disableNotFullReplication() throws Exception {
-    HTableDescriptor table = admin2.getTableDescriptor(tableName);
+    HTableDescriptor table = new HTableDescriptor(admin2.getTableDescriptor(tableName));
     HColumnDescriptor f = new HColumnDescriptor("notReplicatedFamily");
     table.addFamily(f);
     admin1.disableTable(tableName);
@@ -125,7 +125,7 @@
 
   @Test(timeout = 300000)
   public void testEnableReplicationWhenTableDescriptorIsNotSameInClusters() throws Exception {
-    HTableDescriptor table = admin2.getTableDescriptor(tableName);
+    HTableDescriptor table = new HTableDescriptor(admin2.getTableDescriptor(tableName));
     HColumnDescriptor f = new HColumnDescriptor("newFamily");
     table.addFamily(f);
     admin2.disableTable(tableName);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizerOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizerOnCluster.java
index 19df42a..70a78de 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizerOnCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizerOnCluster.java
@@ -140,7 +140,7 @@
       region.flush(true);
     }
 
-    HTableDescriptor htd = admin.getTableDescriptor(TABLENAME);
+    HTableDescriptor htd = new HTableDescriptor(admin.getTableDescriptor(TABLENAME));
     htd.setNormalizationEnabled(true);
     admin.modifyTable(TABLENAME, htd);
 
@@ -217,7 +217,7 @@
       region.flush(true);
     }
 
-    HTableDescriptor htd = admin.getTableDescriptor(tableName);
+    HTableDescriptor htd = new HTableDescriptor(admin.getTableDescriptor(tableName));
     htd.setNormalizationEnabled(true);
     admin.modifyTable(tableName, htd);
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionKeyRotation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionKeyRotation.java
index 3e7477d..4f6d4c2 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionKeyRotation.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionKeyRotation.java
@@ -195,7 +195,7 @@
     // Start the cluster back up
     TEST_UTIL.startMiniHBaseCluster(1, 1);
     // Verify the table can still be loaded
-    TEST_UTIL.waitTableAvailable(htd.getName(), 5000);
+    TEST_UTIL.waitTableAvailable(htd.getTableName(), 5000);
     // Double check that the store file keys can be unwrapped
     storeFilePaths = findStorefilePaths(htd.getTableName());
     assertTrue(storeFilePaths.size() > 0);
@@ -266,7 +266,7 @@
     HColumnDescriptor hcd = htd.getFamilies().iterator().next();
     // Create the test table
     TEST_UTIL.getAdmin().createTable(htd);
-    TEST_UTIL.waitTableAvailable(htd.getName(), 5000);
+    TEST_UTIL.waitTableAvailable(htd.getTableName(), 5000);
     // Create a store file
     Table table = TEST_UTIL.getConnection().getTable(htd.getTableName());
     try {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionRandomKeying.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionRandomKeying.java
index 3d8eeed..cd1f1e7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionRandomKeying.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionRandomKeying.java
@@ -101,7 +101,7 @@
 
     // Create the test table
     TEST_UTIL.getAdmin().createTable(htd);
-    TEST_UTIL.waitTableAvailable(htd.getName(), 5000);
+    TEST_UTIL.waitTableAvailable(htd.getTableName(), 5000);
 
     // Create a store file
     Table table = TEST_UTIL.getConnection().getTable(htd.getTableName());
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
index 095f4bd..53f0892 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
@@ -5734,7 +5734,7 @@
       LOG.info("RegionEventDescriptor from WAL: " + desc);
 
       assertEquals(RegionEventDescriptor.EventType.REGION_OPEN, desc.getEventType());
-      assertTrue(Bytes.equals(desc.getTableName().toByteArray(), htd.getName()));
+      assertTrue(Bytes.equals(desc.getTableName().toByteArray(), htd.getTableName().toBytes()));
       assertTrue(Bytes.equals(desc.getEncodedRegionName().toByteArray(),
         hri.getEncodedNameAsBytes()));
       assertTrue(desc.getLogSequenceNumber() > 0);
@@ -5854,7 +5854,7 @@
       LOG.info("RegionEventDescriptor from WAL: " + desc);
 
       assertEquals(RegionEventDescriptor.EventType.REGION_OPEN, desc.getEventType());
-      assertTrue(Bytes.equals(desc.getTableName().toByteArray(), htd.getName()));
+      assertTrue(Bytes.equals(desc.getTableName().toByteArray(), htd.getTableName().toBytes()));
       assertTrue(Bytes.equals(desc.getEncodedRegionName().toByteArray(),
         hri.getEncodedNameAsBytes()));
       assertTrue(desc.getLogSequenceNumber() > 0);
@@ -5940,7 +5940,7 @@
     LOG.info("RegionEventDescriptor from WAL: " + desc);
 
     assertEquals(RegionEventDescriptor.EventType.REGION_CLOSE, desc.getEventType());
-    assertTrue(Bytes.equals(desc.getTableName().toByteArray(), htd.getName()));
+    assertTrue(Bytes.equals(desc.getTableName().toByteArray(), htd.getTableName().toBytes()));
     assertTrue(Bytes.equals(desc.getEncodedRegionName().toByteArray(),
       hri.getEncodedNameAsBytes()));
     assertTrue(desc.getLogSequenceNumber() > 0);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCoprocessorWhitelistMasterObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCoprocessorWhitelistMasterObserver.java
index aeb909e..bc74f4f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCoprocessorWhitelistMasterObserver.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCoprocessorWhitelistMasterObserver.java
@@ -107,7 +107,7 @@
     UTIL.waitUntilAllRegionsAssigned(TEST_TABLE);
     Connection connection = ConnectionFactory.createConnection(conf);
     Table t = connection.getTable(TEST_TABLE);
-    HTableDescriptor htd = t.getTableDescriptor();
+    HTableDescriptor htd = new HTableDescriptor(t.getTableDescriptor());
     htd.addCoprocessor("net.clayb.hbase.coprocessor.NotWhitelisted",
       new Path(coprocessorPath),
       Coprocessor.PRIORITY_USER, null);
@@ -153,7 +153,7 @@
     // coprocessor file
     admin.disableTable(TEST_TABLE);
     Table t = connection.getTable(TEST_TABLE);
-    HTableDescriptor htd = t.getTableDescriptor();
+    HTableDescriptor htd = new HTableDescriptor(t.getTableDescriptor());
     htd.addCoprocessor("net.clayb.hbase.coprocessor.Whitelisted",
       new Path(coprocessorPath),
       Coprocessor.PRIORITY_USER, null);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/MobSnapshotTestingUtils.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/MobSnapshotTestingUtils.java
index d301214..3e1abb9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/MobSnapshotTestingUtils.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/MobSnapshotTestingUtils.java
@@ -147,7 +147,7 @@
 
     @Override
     public HTableDescriptor createHtd(final String tableName) {
-      HTableDescriptor htd = new HTableDescriptor(tableName);
+      HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName));
       HColumnDescriptor hcd = new HColumnDescriptor(TEST_FAMILY);
       hcd.setMobEnabled(true);
       hcd.setMobThreshold(0L);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java
index ccad85b..3c10dee 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java
@@ -680,7 +680,7 @@
     }
 
     public HTableDescriptor createHtd(final String tableName) {
-      HTableDescriptor htd = new HTableDescriptor(tableName);
+      HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName));
       htd.addFamily(new HColumnDescriptor(TEST_FAMILY));
       return htd;
     }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSTableDescriptors.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSTableDescriptors.java
index ee048b7..8337eb0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSTableDescriptors.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSTableDescriptors.java
@@ -261,7 +261,7 @@
     final int count = 10;
     // Write out table infos.
     for (int i = 0; i < count; i++) {
-      HTableDescriptor htd = new HTableDescriptor(name + i);
+      HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name + i));
       htds.createTableDescriptor(htd);
     }
 
@@ -294,7 +294,7 @@
     final int count = 4;
     // Write out table infos.
     for (int i = 0; i < count; i++) {
-      HTableDescriptor htd = new HTableDescriptor(name + i);
+      HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name + i));
       htds.createTableDescriptor(htd);
     }
     // add hbase:meta
@@ -321,7 +321,7 @@
     final int count = 10;
     // Write out table infos via non-cached FSTableDescriptors
     for (int i = 0; i < count; i++) {
-      HTableDescriptor htd = new HTableDescriptor(name + i);
+      HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name + i));
       nonchtds.createTableDescriptor(htd);
     }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckEncryption.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckEncryption.java
index 17ab004..c1d5778 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckEncryption.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckEncryption.java
@@ -92,7 +92,7 @@
       cfKey));
     htd.addFamily(hcd);
     TEST_UTIL.getAdmin().createTable(htd);
-    TEST_UTIL.waitTableAvailable(htd.getName(), 5000);
+    TEST_UTIL.waitTableAvailable(htd.getTableName(), 5000);
   }
 
   @After
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckOneRS.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckOneRS.java
index 1d09dfa..74ef414 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckOneRS.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckOneRS.java
@@ -248,7 +248,7 @@
       status = FSTableDescriptors.getTableInfoPath(fs, hbaseTableDir);
       assertNotNull(status);
 
-      HTableDescriptor htd = admin.getTableDescriptor(tableName);
+      HTableDescriptor htd = new HTableDescriptor(admin.getTableDescriptor(tableName));
       htd.setValue("NOT_DEFAULT", "true");
       admin.disableTable(tableName);
       admin.modifyTable(tableName, htd);
diff --git a/hbase-shell/src/main/ruby/hbase/admin.rb b/hbase-shell/src/main/ruby/hbase/admin.rb
index 41904be..2db89b7 100644
--- a/hbase-shell/src/main/ruby/hbase/admin.rb
+++ b/hbase-shell/src/main/ruby/hbase/admin.rb
@@ -589,7 +589,7 @@
       table_name = TableName.valueOf(table_name_str)
 
       # Get table descriptor
-      htd = @admin.getTableDescriptor(table_name)
+      htd = org.apache.hadoop.hbase.HTableDescriptor.new(@admin.getTableDescriptor(table_name))
       hasTableUpdate = false
 
       # Process all args
@@ -1206,15 +1206,6 @@
       htd.setNormalizationEnabled(
         JBoolean.valueOf(arg.delete(NORMALIZATION_ENABLED))) if arg[NORMALIZATION_ENABLED]
       htd.setMemStoreFlushSize(JLong.valueOf(arg.delete(MEMSTORE_FLUSHSIZE))) if arg[MEMSTORE_FLUSHSIZE]
-      # DEFERRED_LOG_FLUSH is deprecated and was replaced by DURABILITY.  To keep backward compatible, it still exists.
-      # However, it has to be set before DURABILITY so that DURABILITY could overwrite if both args are set
-      if arg.include?(DEFERRED_LOG_FLUSH)
-        if arg.delete(DEFERRED_LOG_FLUSH).to_s.upcase == "TRUE"
-          htd.setDurability(org.apache.hadoop.hbase.client.Durability.valueOf("ASYNC_WAL"))
-        else
-          htd.setDurability(org.apache.hadoop.hbase.client.Durability.valueOf("SYNC_WAL"))
-        end
-      end
       htd.setDurability(org.apache.hadoop.hbase.client.Durability.valueOf(arg.delete(DURABILITY))) if arg[DURABILITY]
       htd.setPriority(JInteger.valueOf(arg.delete(org.apache.hadoop.hbase.HTableDescriptor::PRIORITY))) if arg[org.apache.hadoop.hbase.HTableDescriptor::PRIORITY]
       htd.setFlushPolicyClassName(arg.delete(org.apache.hadoop.hbase.HTableDescriptor::FLUSH_POLICY)) if arg[org.apache.hadoop.hbase.HTableDescriptor::FLUSH_POLICY]
diff --git a/hbase-shell/src/main/ruby/shell/commands/alter_async.rb b/hbase-shell/src/main/ruby/shell/commands/alter_async.rb
index e0f6deb..05335fb 100644
--- a/hbase-shell/src/main/ruby/shell/commands/alter_async.rb
+++ b/hbase-shell/src/main/ruby/shell/commands/alter_async.rb
@@ -40,8 +40,8 @@
 
   hbase> alter_async 'ns1:t1', 'delete' => 'f1'
 
-You can also change table-scope attributes like MAX_FILESIZE
-MEMSTORE_FLUSHSIZE, READONLY, and DEFERRED_LOG_FLUSH.
+You can also change table-scope attributes like MAX_FILESIZE,
+MEMSTORE_FLUSHSIZE, and READONLY.
 
 For example, to change the max size of a family to 128MB, do: