You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by sy...@apache.org on 2017/04/26 22:52:46 UTC

[39/40] hbase git commit: HBASE-15583 Any HTableDescriptor we give out should be immutable

HBASE-15583 Any HTableDescriptor we give out should be immutable


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/053e6154
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/053e6154
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/053e6154

Branch: refs/heads/hbase-12439
Commit: 053e61541e6f45bbd2866faf4fe6c766a3f0c245
Parents: 8973582
Author: Chia-Ping Tsai <ch...@gmail.com>
Authored: Thu Apr 27 03:22:29 2017 +0800
Committer: Chia-Ping Tsai <ch...@gmail.com>
Committed: Thu Apr 27 03:22:29 2017 +0800

----------------------------------------------------------------------
 bin/region_status.rb                            |    2 +-
 .../apache/hadoop/hbase/HTableDescriptor.java   | 1055 ++---------
 .../org/apache/hadoop/hbase/client/Admin.java   |   26 +-
 .../apache/hadoop/hbase/client/AsyncAdmin.java  |   47 +-
 .../hadoop/hbase/client/AsyncHBaseAdmin.java    |   45 +-
 .../apache/hadoop/hbase/client/HBaseAdmin.java  |    7 +-
 .../org/apache/hadoop/hbase/client/HTable.java  |    2 +-
 .../hbase/client/ImmutableHTableDescriptor.java |   79 +
 .../hadoop/hbase/client/TableDescriptor.java    |  256 +++
 .../hbase/client/TableDescriptorBuilder.java    | 1639 ++++++++++++++++++
 .../client/UnmodifyableHTableDescriptor.java    |  127 --
 .../hbase/shaded/protobuf/ProtobufUtil.java     |   46 +-
 .../hbase/shaded/protobuf/RequestConverter.java |    3 +-
 .../client/TestImmutableHTableDescriptor.java   |  102 ++
 .../client/TestTableDescriptorBuilder.java      |  376 ++++
 .../TestUnmodifyableHTableDescriptor.java       |   47 -
 .../hadoop/hbase/rest/RowResourceBase.java      |    2 +-
 .../rest/client/TestRemoteAdminRetries.java     |    3 +-
 .../hbase/rsgroup/RSGroupInfoManager.java       |    1 -
 .../hbase/rsgroup/RSGroupInfoManagerImpl.java   |    2 +-
 .../hadoop/hbase/rsgroup/TestRSGroups.java      |    2 +-
 .../hadoop/hbase/backup/util/RestoreTool.java   |    6 +-
 .../hadoop/hbase/util/FSTableDescriptors.java   |  116 +-
 .../hadoop/hbase/HBaseTestingUtility.java       |    2 +-
 .../TestFSTableDescriptorForceCreation.java     |    2 +-
 .../hbase/client/TestAsyncTableAdminApi.java    |   12 +-
 .../hbase/client/TestAsyncTableBatch.java       |    2 +-
 .../TestReplicationAdminWithClusters.java       |    4 +-
 .../TestSimpleRegionNormalizerOnCluster.java    |    4 +-
 .../regionserver/TestEncryptionKeyRotation.java |    4 +-
 .../TestEncryptionRandomKeying.java             |    2 +-
 .../hadoop/hbase/regionserver/TestHRegion.java  |    6 +-
 .../TestCoprocessorWhitelistMasterObserver.java |    4 +-
 .../hbase/snapshot/MobSnapshotTestingUtils.java |    2 +-
 .../hbase/snapshot/SnapshotTestingUtils.java    |    2 +-
 .../hbase/util/TestFSTableDescriptors.java      |    6 +-
 .../hbase/util/TestHBaseFsckEncryption.java     |    2 +-
 .../hadoop/hbase/util/TestHBaseFsckOneRS.java   |    2 +-
 hbase-shell/src/main/ruby/hbase/admin.rb        |   11 +-
 .../src/main/ruby/shell/commands/alter_async.rb |    4 +-
 40 files changed, 2820 insertions(+), 1242 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/053e6154/bin/region_status.rb
----------------------------------------------------------------------
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 @@ end
 
 # 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?

http://git-wip-us.apache.org/repos/asf/hbase/blob/053e6154/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
----------------------------------------------------------------------
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 @@ import org.apache.hadoop.hbase.util.Bytes;
  * 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 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    * @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 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    * 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 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    *
    * @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 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    * @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 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    *
    * @return true if table is <code> hbase:meta </code> region.
    */
+  @Override
   public boolean isMetaTable() {
-    return isMetaRegion() && !isRootRegion();
+    return delegatee.isMetaTable();
   }
 
   /**
@@ -471,17 +152,10 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    *
    * @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 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    *
    * @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));
-    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)));
+    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.
+   * @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 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    *
    * @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 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    * only.
    */
   public HTableDescriptor setReadOnly(final boolean readOnly) {
-    return setValue(READONLY_KEY, readOnly? TRUE: FALSE);
+    delegatee.setReadOnly(readOnly);
+    return this;
   }
 
   /**
@@ -623,8 +269,9 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    *
    * @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 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    * @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 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    *
    * @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 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    * @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 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    * @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 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    * 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 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    *
    * @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 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    * @return name of table as a String
    */
   public String getNameAsString() {
-    return name.getNameAsString();
+    return delegatee.getTableName().getNameAsString();
   }
 
   /**
@@ -725,7 +349,7 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    * @param clazz the class name
    */
   public HTableDescriptor setRegionSplitPolicyClassName(String clazz) {
-    setValue(SPLIT_POLICY, clazz);
+    delegatee.setRegionSplitPolicyClassName(clazz);
     return this;
   }
 
@@ -737,26 +361,9 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    * @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 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    *
    * @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 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    * 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 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    *
    * @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 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    * @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 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    * @param clazz the class name
    */
   public HTableDescriptor setFlushPolicyClassName(String clazz) {
-    setValue(FLUSH_POLICY, clazz);
+    delegatee.setFlushPolicyClassName(clazz);
     return this;
   }
 
@@ -840,8 +441,9 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    * @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 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    * @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 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    * @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 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    * @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 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    */
   @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 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    * 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 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    */
   @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 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    */
   @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 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    * @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 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    * @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 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    *                                  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 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    *
    * @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 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    *
    * @return Count of column families of the table
    */
+  @Override
   public int getColumnFamilyCount() {
-    return families.size();
+    return delegatee.getColumnFamilyCount();
   }
 
   /**
@@ -1209,9 +646,9 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    *
    * @see #getFamilies()
    */
+  @Override
   public HColumnDescriptor[] getColumnFamilies() {
-    Collection<HColumnDescriptor> hColumnDescriptors = getFamilies();
-    return hColumnDescriptors.toArray(new HColumnDescriptor[hColumnDescriptors.size()]);
+    return delegatee.getColumnFamilies();
   }
 
 
@@ -1223,8 +660,9 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    * @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 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    * passed in column.
    */
   public HColumnDescriptor removeFamily(final byte [] column) {
-    return this.families.remove(column);
+    return delegatee.removeFamily(column);
   }
 
   /**
@@ -1251,7 +689,7 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    * @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 class HTableDescriptor implements Comparable<HTableDescriptor> {
   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 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    * @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 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    * @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 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    *
    * @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 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    * @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 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    * @see #parseFrom(byte[])
    */
   public byte[] toByteArray() {
-    return ProtobufUtil.prependPBMagic(ProtobufUtil.convertToTableSchema(this).toByteArray());
+    return delegatee.toByteArray();
   }
 
   /**
@@ -1521,54 +803,39 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
    */
   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);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/053e6154/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
----------------------------------------------------------------------
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 @@ public interface Admin extends Abortable, Closeable {
   /**
    * 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 @@ public interface Admin extends Abortable, Closeable {
    * 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 @@ public interface Admin extends Abortable, Closeable {
    *
    * @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 @@ public interface Admin extends Abortable, Closeable {
    *
    * @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 @@ public interface Admin extends Abortable, Closeable {
    * 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 @@ public interface Admin extends Abortable, Closeable {
    * #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 @@ public interface Admin extends Abortable, Closeable {
    *
    * @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 @@ public interface Admin extends Abortable, Closeable {
    *
    * @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 @@ public interface Admin extends Abortable, Closeable {
    *
    * @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 @@ public interface Admin extends Abortable, Closeable {
    *
    * @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 @@ public interface Admin extends Abortable, Closeable {
    *
    * @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 @@ public interface Admin extends Abortable, Closeable {
    * 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 @@ public interface Admin extends Abortable, Closeable {
    * 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 @@ public interface Admin extends Abortable, Closeable {
    * 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)

http://git-wip-us.apache.org/repos/asf/hbase/blob/053e6154/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
----------------------------------------------------------------------
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 java.util.regex.Pattern;
 
 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 @@ public interface AsyncAdmin {
 
   /**
    * 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 @@ public interface AsyncAdmin {
   /**
    * 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 @@ public interface AsyncAdmin {
    * @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 @@ public interface AsyncAdmin {
    * @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 @@ public interface AsyncAdmin {
    * {@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 @@ public interface AsyncAdmin {
    * {@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 @@ public interface AsyncAdmin {
    * {@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 @@ public interface AsyncAdmin {
    * {@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 @@ public interface AsyncAdmin {
    * {@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 @@ public interface AsyncAdmin {
    * {@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

http://git-wip-us.apache.org/repos/asf/hbase/blob/053e6154/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
----------------------------------------------------------------------
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.commons.logging.LogFactory;
 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 @@ public class AsyncHBaseAdmin implements AsyncAdmin {
     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 @@ public class AsyncHBaseAdmin implements AsyncAdmin {
               }
             })).<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 @@ public class AsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @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 @@ public class AsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @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 @@ public class AsyncHBaseAdmin implements AsyncAdmin {
             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 @@ public class AsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @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 @@ public class AsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @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 @@ public class AsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @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 @@ public class AsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @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 @@ public class AsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @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");
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/053e6154/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
----------------------------------------------------------------------
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 @@ public class HBaseAdmin implements Admin {
       }
     }, rpcCallerFactory, operationTimeout, rpcTimeout);
     if (htd != null) {
-      return htd;
+      return new ImmutableHTableDescriptor(htd);
     }
     throw new TableNotFoundException(tableName.getNameAsString());
   }
@@ -532,7 +532,7 @@ public class HBaseAdmin implements Admin {
       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 @@ public class HBaseAdmin implements Admin {
                 .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;
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/053e6154/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
----------------------------------------------------------------------
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 @@ public class HTable implements Table {
     HTableDescriptor htd = HBaseAdmin.getTableDescriptor(tableName, connection, rpcCallerFactory,
       rpcControllerFactory, operationTimeout, readRpcTimeout);
     if (htd != null) {
-      return new UnmodifyableHTableDescriptor(htd);
+      return new ImmutableHTableDescriptor(htd);
     }
     return null;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/053e6154/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ImmutableHTableDescriptor.java
----------------------------------------------------------------------
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");
+    }
+  }
+}