http://git-wip-us.apache.org/repos/asf/hbase/blob/b0334863/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/TakeSnapshotHandler.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/TakeSnapshotHandler.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/TakeSnapshotHandler.java
index fce4eaa..979a351 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/TakeSnapshotHandler.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/TakeSnapshotHandler.java
@@ -33,9 +33,9 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.MetaTableAccessor;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.errorhandling.ForeignException;
 import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
 import org.apache.hadoop.hbase.errorhandling.ForeignExceptionSnare;
@@ -89,7 +89,7 @@ public abstract class TakeSnapshotHandler extends 
EventHandler implements Snapsh
   protected final SnapshotManifest snapshotManifest;
   protected final SnapshotManager snapshotManager;
 
-  protected HTableDescriptor htd;
+  protected TableDescriptor htd;
 
   /**
    * @param snapshot descriptor of the snapshot to take
@@ -124,12 +124,12 @@ public abstract class TakeSnapshotHandler extends 
EventHandler implements Snapsh
       "Taking " + snapshot.getType() + " snapshot on table: " + snapshotTable);
   }
 
-  private HTableDescriptor loadTableDescriptor()
+  private TableDescriptor loadTableDescriptor()
       throws FileNotFoundException, IOException {
-    HTableDescriptor htd =
+    TableDescriptor htd =
       this.master.getTableDescriptors().get(snapshotTable);
     if (htd == null) {
-      throw new IOException("HTableDescriptor missing for " + snapshotTable);
+      throw new IOException("TableDescriptor missing for " + snapshotTable);
     }
     return htd;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/b0334863/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/ExpiredMobFileCleaner.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/ExpiredMobFileCleaner.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/ExpiredMobFileCleaner.java
index d4a54bb..b1d1415 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/ExpiredMobFileCleaner.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/ExpiredMobFileCleaner.java
@@ -26,15 +26,15 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hbase.HBaseConfiguration;
-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.client.Admin;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
@@ -59,7 +59,7 @@ public class ExpiredMobFileCleaner extends Configured 
implements Tool {
    * @param tableName The current table name.
    * @param family The current family.
    */
-  public void cleanExpiredMobFiles(String tableName, HColumnDescriptor family) 
throws IOException {
+  public void cleanExpiredMobFiles(String tableName, ColumnFamilyDescriptor 
family) throws IOException {
     Configuration conf = getConf();
     TableName tn = TableName.valueOf(tableName);
     FileSystem fs = FileSystem.get(conf);
@@ -98,8 +98,8 @@ public class ExpiredMobFileCleaner extends Configured 
implements Tool {
     Connection connection = ConnectionFactory.createConnection(getConf());
     Admin admin = connection.getAdmin();
     try {
-      HTableDescriptor htd = admin.getTableDescriptor(tn);
-      HColumnDescriptor family = htd.getFamily(Bytes.toBytes(familyName));
+      TableDescriptor htd = admin.listTableDescriptor(tn);
+      ColumnFamilyDescriptor family = 
htd.getColumnFamily(Bytes.toBytes(familyName));
       if (family == null || !family.isMobEnabled()) {
         throw new IOException("Column family " + familyName + " is not a MOB 
column family");
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/b0334863/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java
index 80bda28..4273098 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java
@@ -44,10 +44,8 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Tag;
 import org.apache.hadoop.hbase.TagType;
@@ -57,6 +55,7 @@ import 
org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.client.MobCompactPartitionPolicy;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.io.HFileLink;
 import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.crypto.Encryption;
@@ -286,7 +285,7 @@ public final class MobUtils {
    * @throws IOException
    */
   public static void cleanExpiredMobFiles(FileSystem fs, Configuration conf, 
TableName tableName,
-      HColumnDescriptor columnDescriptor, CacheConfig cacheConfig, long 
current)
+      ColumnFamilyDescriptor columnDescriptor, CacheConfig cacheConfig, long 
current)
       throws IOException {
     long timeToLive = columnDescriptor.getTimeToLive();
     if (Integer.MAX_VALUE == timeToLive) {
@@ -519,7 +518,7 @@ public final class MobUtils {
    * @throws IOException
    */
   public static StoreFileWriter createWriter(Configuration conf, FileSystem fs,
-      HColumnDescriptor family, String date, Path basePath, long maxKeyCount,
+      ColumnFamilyDescriptor family, String date, Path basePath, long 
maxKeyCount,
       Compression.Algorithm compression, String startKey, CacheConfig 
cacheConfig,
       Encryption.Context cryptoContext, boolean isCompaction)
       throws IOException {
@@ -543,7 +542,7 @@ public final class MobUtils {
    * @throws IOException
    */
   public static StoreFileWriter createRefFileWriter(Configuration conf, 
FileSystem fs,
-    HColumnDescriptor family, Path basePath, long maxKeyCount, CacheConfig 
cacheConfig,
+    ColumnFamilyDescriptor family, Path basePath, long maxKeyCount, 
CacheConfig cacheConfig,
     Encryption.Context cryptoContext, boolean isCompaction)
     throws IOException {
     return createWriter(conf, fs, family,
@@ -570,7 +569,7 @@ public final class MobUtils {
    * @throws IOException
    */
   public static StoreFileWriter createWriter(Configuration conf, FileSystem fs,
-      HColumnDescriptor family, String date, Path basePath, long maxKeyCount,
+      ColumnFamilyDescriptor family, String date, Path basePath, long 
maxKeyCount,
       Compression.Algorithm compression, byte[] startKey, CacheConfig 
cacheConfig,
       Encryption.Context cryptoContext, boolean isCompaction)
       throws IOException {
@@ -596,7 +595,7 @@ public final class MobUtils {
    * @throws IOException
    */
   public static StoreFileWriter createDelFileWriter(Configuration conf, 
FileSystem fs,
-      HColumnDescriptor family, String date, Path basePath, long maxKeyCount,
+      ColumnFamilyDescriptor family, String date, Path basePath, long 
maxKeyCount,
       Compression.Algorithm compression, byte[] startKey, CacheConfig 
cacheConfig,
       Encryption.Context cryptoContext)
       throws IOException {
@@ -623,7 +622,7 @@ public final class MobUtils {
    * @throws IOException
    */
   public static StoreFileWriter createWriter(Configuration conf, FileSystem fs,
-      HColumnDescriptor family, MobFileName mobFileName, Path basePath, long 
maxKeyCount,
+                                             ColumnFamilyDescriptor family, 
MobFileName mobFileName, Path basePath, long maxKeyCount,
       Compression.Algorithm compression, CacheConfig cacheConfig, 
Encryption.Context cryptoContext,
       boolean isCompaction)
       throws IOException {
@@ -797,7 +796,7 @@ public final class MobUtils {
    * @param allFiles Whether add all mob files into the compaction.
    */
   public static void doMobCompaction(Configuration conf, FileSystem fs, 
TableName tableName,
-    HColumnDescriptor hcd, ExecutorService pool, boolean allFiles, 
LockManager.MasterLock lock)
+                                     ColumnFamilyDescriptor hcd, 
ExecutorService pool, boolean allFiles, LockManager.MasterLock lock)
       throws IOException {
     String className = conf.get(MobConstants.MOB_COMPACTOR_CLASS_KEY,
         PartitionedMobCompactor.class.getName());
@@ -805,7 +804,7 @@ public final class MobUtils {
     MobCompactor compactor = null;
     try {
       compactor = ReflectionUtils.instantiateWithCustomCtor(className, new 
Class[] {
-        Configuration.class, FileSystem.class, TableName.class, 
HColumnDescriptor.class,
+        Configuration.class, FileSystem.class, TableName.class, 
ColumnFamilyDescriptor.class,
         ExecutorService.class }, new Object[] { conf, fs, tableName, hcd, pool 
});
     } catch (Exception e) {
       throw new IOException("Unable to load configured mob file compactor '" + 
className + "'", e);
@@ -857,9 +856,9 @@ public final class MobUtils {
    * @param htd The current table descriptor.
    * @return Whether this table has mob-enabled columns.
    */
-  public static boolean hasMobColumns(HTableDescriptor htd) {
-    HColumnDescriptor[] hcds = htd.getColumnFamilies();
-    for (HColumnDescriptor hcd : hcds) {
+  public static boolean hasMobColumns(TableDescriptor htd) {
+    ColumnFamilyDescriptor[] hcds = htd.getColumnFamilies();
+    for (ColumnFamilyDescriptor hcd : hcds) {
       if (hcd.isMobEnabled()) {
         return true;
       }
@@ -899,7 +898,7 @@ public final class MobUtils {
    * @param fileDate The date string parsed from the mob file name.
    * @return True if the mob file is expired.
    */
-  public static boolean isMobFileExpired(HColumnDescriptor column, long 
current, String fileDate) {
+  public static boolean isMobFileExpired(ColumnFamilyDescriptor column, long 
current, String fileDate) {
     if (column.getMinVersions() > 0) {
       return false;
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/b0334863/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/MobCompactor.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/MobCompactor.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/MobCompactor.java
index 77de0cd..7ebdbc7 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/MobCompactor.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/MobCompactor.java
@@ -27,9 +27,9 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.mob.MobUtils;
 import org.apache.hadoop.hbase.util.FSUtils;
 
@@ -42,14 +42,14 @@ public abstract class MobCompactor {
   protected FileSystem fs;
   protected Configuration conf;
   protected TableName tableName;
-  protected HColumnDescriptor column;
+  protected ColumnFamilyDescriptor column;
 
   protected Path mobTableDir;
   protected Path mobFamilyDir;
   protected ExecutorService pool;
 
   public MobCompactor(Configuration conf, FileSystem fs, TableName tableName,
-    HColumnDescriptor column, ExecutorService pool) {
+                      ColumnFamilyDescriptor column, ExecutorService pool) {
     this.conf = conf;
     this.fs = fs;
     this.tableName = tableName;

http://git-wip-us.apache.org/repos/asf/hbase/blob/b0334863/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactor.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactor.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactor.java
index d37292c..da664cd 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactor.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactor.java
@@ -45,13 +45,13 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
-import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Tag;
 import org.apache.hadoop.hbase.TagType;
 import org.apache.hadoop.hbase.TagUtil;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.MobCompactPartitionPolicy;
@@ -109,7 +109,7 @@ public class PartitionedMobCompactor extends MobCompactor {
   private Encryption.Context cryptoContext = Encryption.Context.NONE;
 
   public PartitionedMobCompactor(Configuration conf, FileSystem fs, TableName 
tableName,
-    HColumnDescriptor column, ExecutorService pool) throws IOException {
+                                 ColumnFamilyDescriptor column, 
ExecutorService pool) throws IOException {
     super(conf, fs, tableName, column, pool);
     mergeableSize = 
conf.getLong(MobConstants.MOB_COMPACTION_MERGEABLE_THRESHOLD,
       MobConstants.DEFAULT_MOB_COMPACTION_MERGEABLE_THRESHOLD);

http://git-wip-us.apache.org/repos/asf/hbase/blob/b0334863/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionTool.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionTool.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionTool.java
index e1d2ea1..eb9a5f7 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionTool.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionTool.java
@@ -37,8 +37,8 @@ import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.HDFSBlocksDistribution;
 import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.mapreduce.JobUtil;
 import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
@@ -109,13 +109,13 @@ public class CompactionTool extends Configured implements 
Tool {
       if (isFamilyDir(fs, path)) {
         Path regionDir = path.getParent();
         Path tableDir = regionDir.getParent();
-        HTableDescriptor htd = FSTableDescriptors.getTableDescriptorFromFs(fs, 
tableDir);
+        TableDescriptor htd = FSTableDescriptors.getTableDescriptorFromFs(fs, 
tableDir);
         HRegionInfo hri = HRegionFileSystem.loadRegionInfoFileContent(fs, 
regionDir);
         compactStoreFiles(tableDir, htd, hri,
             path.getName(), compactOnce, major);
       } else if (isRegionDir(fs, path)) {
         Path tableDir = path.getParent();
-        HTableDescriptor htd = FSTableDescriptors.getTableDescriptorFromFs(fs, 
tableDir);
+        TableDescriptor htd = FSTableDescriptors.getTableDescriptorFromFs(fs, 
tableDir);
         compactRegion(tableDir, htd, path, compactOnce, major);
       } else if (isTableDir(fs, path)) {
         compactTable(path, compactOnce, major);
@@ -127,13 +127,13 @@ public class CompactionTool extends Configured implements 
Tool {
 
     private void compactTable(final Path tableDir, final boolean compactOnce, 
final boolean major)
         throws IOException {
-      HTableDescriptor htd = FSTableDescriptors.getTableDescriptorFromFs(fs, 
tableDir);
+      TableDescriptor htd = FSTableDescriptors.getTableDescriptorFromFs(fs, 
tableDir);
       for (Path regionDir: FSUtils.getRegionDirs(fs, tableDir)) {
         compactRegion(tableDir, htd, regionDir, compactOnce, major);
       }
     }
 
-    private void compactRegion(final Path tableDir, final HTableDescriptor htd,
+    private void compactRegion(final Path tableDir, final TableDescriptor htd,
         final Path regionDir, final boolean compactOnce, final boolean major)
         throws IOException {
       HRegionInfo hri = HRegionFileSystem.loadRegionInfoFileContent(fs, 
regionDir);
@@ -147,7 +147,7 @@ public class CompactionTool extends Configured implements 
Tool {
      * If the compact once flag is not specified, execute the compaction until
      * no more compactions are needed. Uses the Configuration settings 
provided.
      */
-    private void compactStoreFiles(final Path tableDir, final HTableDescriptor 
htd,
+    private void compactStoreFiles(final Path tableDir, final TableDescriptor 
htd,
         final HRegionInfo hri, final String familyName, final boolean 
compactOnce,
         final boolean major) throws IOException {
       HStore store = getStore(conf, fs, tableDir, htd, hri, familyName, 
tmpDir);
@@ -177,7 +177,7 @@ public class CompactionTool extends Configured implements 
Tool {
      * the store dir to compact as source.
      */
     private static HStore getStore(final Configuration conf, final FileSystem 
fs,
-        final Path tableDir, final HTableDescriptor htd, final HRegionInfo hri,
+        final Path tableDir, final TableDescriptor htd, final HRegionInfo hri,
         final String familyName, final Path tempDir) throws IOException {
       HRegionFileSystem regionFs = new HRegionFileSystem(conf, fs, tableDir, 
hri) {
         @Override
@@ -186,7 +186,7 @@ public class CompactionTool extends Configured implements 
Tool {
         }
       };
       HRegion region = new HRegion(regionFs, null, conf, htd, null);
-      return new HStore(region, htd.getFamily(Bytes.toBytes(familyName)), 
conf);
+      return new HStore(region, 
htd.getColumnFamily(Bytes.toBytes(familyName)), conf);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b0334863/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
index 9cb1316..74a2998 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
@@ -41,12 +41,12 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.backup.HFileArchiver;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.fs.HFileSystem;
 import org.apache.hadoop.hbase.io.Reference;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -340,8 +340,8 @@ public class HRegionFileSystem {
    * @return true if region has reference file
    * @throws IOException
    */
-  public boolean hasReferences(final HTableDescriptor htd) throws IOException {
-    for (HColumnDescriptor family : htd.getFamilies()) {
+  public boolean hasReferences(final TableDescriptor htd) throws IOException {
+    for (ColumnFamilyDescriptor family : htd.getColumnFamilies()) {
       if (hasReferences(family.getNameAsString())) {
         return true;
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/b0334863/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index 28b7a43..ae2f7dd 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -50,6 +50,7 @@ import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Function;
 
 import javax.management.MalformedObjectNameException;
 import javax.management.ObjectName;
@@ -89,6 +90,7 @@ import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionUtils;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.client.locking.EntityLock;
 import org.apache.hadoop.hbase.client.locking.LockServiceClient;
 import org.apache.hadoop.hbase.conf.ConfigurationManager;
@@ -700,7 +702,11 @@ public class HRegionServer extends HasThread implements
 
   protected TableDescriptors getFsTableDescriptors() throws IOException {
     return new FSTableDescriptors(this.conf,
-      this.fs, this.rootDir, !canUpdateTableDescriptor(), false);
+      this.fs, this.rootDir, !canUpdateTableDescriptor(), false, 
getMetaTableObserver());
+  }
+
+  protected Function<TableDescriptorBuilder, TableDescriptorBuilder> 
getMetaTableObserver() {
+    return null;
   }
 
   protected void setInitLatch(CountDownLatch latch) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/b0334863/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
index 9a25275..7fc025a 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
@@ -58,7 +58,6 @@ import org.apache.hadoop.hbase.DroppedSnapshotException;
 import org.apache.hadoop.hbase.HBaseIOException;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.MultiActionResultTooLarge;
 import org.apache.hadoop.hbase.NotServingRegionException;
 import org.apache.hadoop.hbase.ServerName;
@@ -77,6 +76,7 @@ import org.apache.hadoop.hbase.client.RegionReplicaUtil;
 import org.apache.hadoop.hbase.client.Result;
 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.VersionInfoUtil;
 import org.apache.hadoop.hbase.conf.ConfigurationObserver;
 import org.apache.hadoop.hbase.exceptions.FailedSanityCheckException;
@@ -1859,7 +1859,7 @@ public class RSRpcServices implements 
HBaseRPCErrorHandler,
 
     OpenRegionResponse.Builder builder = OpenRegionResponse.newBuilder();
     final int regionCount = request.getOpenInfoCount();
-    final Map<TableName, HTableDescriptor> htds = new HashMap<>(regionCount);
+    final Map<TableName, TableDescriptor> htds = new HashMap<>(regionCount);
     final boolean isBulkAssign = regionCount > 1;
     try {
       checkOpen();
@@ -1898,7 +1898,7 @@ public class RSRpcServices implements 
HBaseRPCErrorHandler,
 
     for (RegionOpenInfo regionOpenInfo : request.getOpenInfoList()) {
       final HRegionInfo region = 
HRegionInfo.convert(regionOpenInfo.getRegion());
-      HTableDescriptor htd;
+      TableDescriptor htd;
       try {
         String encodedName = region.getEncodedName();
         byte[] encodedNameBytes = region.getEncodedNameAsBytes();
@@ -2020,7 +2020,7 @@ public class RSRpcServices implements 
HBaseRPCErrorHandler,
 
     RegionInfo regionInfo = request.getRegionInfo();
     final HRegionInfo region = HRegionInfo.convert(regionInfo);
-    HTableDescriptor htd;
+    TableDescriptor htd;
     WarmupRegionResponse response = WarmupRegionResponse.getDefaultInstance();
 
     try {

http://git-wip-us.apache.org/repos/asf/hbase/blob/b0334863/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenMetaHandler.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenMetaHandler.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenMetaHandler.java
index e49b164..6913ecd 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenMetaHandler.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenMetaHandler.java
@@ -20,8 +20,8 @@ package org.apache.hadoop.hbase.regionserver.handler;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.Server;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.executor.EventType;
 import org.apache.hadoop.hbase.regionserver.RegionServerServices;
 
@@ -33,8 +33,8 @@ import 
org.apache.hadoop.hbase.regionserver.RegionServerServices;
 @InterfaceAudience.Private
 public class OpenMetaHandler extends OpenRegionHandler {
   public OpenMetaHandler(final Server server,
-      final RegionServerServices rsServices, HRegionInfo regionInfo,
-      final HTableDescriptor htd, long masterSystemTime) {
+                         final RegionServerServices rsServices, HRegionInfo 
regionInfo,
+                         final TableDescriptor htd, long masterSystemTime) {
     super(server, rsServices, regionInfo, htd, masterSystemTime, 
EventType.M_RS_OPEN_META);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/b0334863/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenPriorityRegionHandler.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenPriorityRegionHandler.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenPriorityRegionHandler.java
index 83d4d3f..ced9ef2 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenPriorityRegionHandler.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenPriorityRegionHandler.java
@@ -19,11 +19,10 @@
 package org.apache.hadoop.hbase.regionserver.handler;
 
 import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.executor.EventType;
-import org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler;
 import org.apache.hadoop.hbase.regionserver.RegionServerServices;
 
 /**
@@ -34,7 +33,7 @@ import 
org.apache.hadoop.hbase.regionserver.RegionServerServices;
 @InterfaceAudience.Private
 public class OpenPriorityRegionHandler extends OpenRegionHandler {
   public OpenPriorityRegionHandler(Server server, RegionServerServices 
rsServices,
-      HRegionInfo regionInfo, HTableDescriptor htd, long masterSystemTime) {
+                                   HRegionInfo regionInfo, TableDescriptor 
htd, long masterSystemTime) {
     super(server, rsServices, regionInfo, htd, masterSystemTime,
        EventType.M_RS_OPEN_PRIORITY_REGION);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/b0334863/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java
index 8369100..bbb084c 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java
@@ -25,8 +25,8 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.Server;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.executor.EventHandler;
 import org.apache.hadoop.hbase.executor.EventType;
 import 
org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
@@ -48,18 +48,18 @@ public class OpenRegionHandler extends EventHandler {
   protected final RegionServerServices rsServices;
 
   private final HRegionInfo regionInfo;
-  private final HTableDescriptor htd;
+  private final TableDescriptor htd;
   private final long masterSystemTime;
 
   public OpenRegionHandler(final Server server,
       final RegionServerServices rsServices, HRegionInfo regionInfo,
-      HTableDescriptor htd, long masterSystemTime) {
+      TableDescriptor htd, long masterSystemTime) {
     this(server, rsServices, regionInfo, htd, masterSystemTime, 
EventType.M_RS_OPEN_REGION);
   }
 
   protected OpenRegionHandler(final Server server,
-      final RegionServerServices rsServices, final HRegionInfo regionInfo,
-      final HTableDescriptor htd, long masterSystemTime, EventType eventType) {
+                              final RegionServerServices rsServices, final 
HRegionInfo regionInfo,
+                              final TableDescriptor htd, long 
masterSystemTime, EventType eventType) {
     super(server, eventType);
     this.rsServices = rsServices;
     this.regionInfo = regionInfo;

http://git-wip-us.apache.org/repos/asf/hbase/blob/b0334863/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RegionReplicaReplicationEndpoint.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RegionReplicaReplicationEndpoint.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RegionReplicaReplicationEndpoint.java
index 9f600da..4ea0434 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RegionReplicaReplicationEndpoint.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RegionReplicaReplicationEndpoint.java
@@ -43,7 +43,6 @@ import org.apache.hadoop.hbase.HBaseIOException;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HRegionLocation;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.RegionLocations;
 import org.apache.hadoop.hbase.TableDescriptors;
 import org.apache.hadoop.hbase.TableName;
@@ -55,6 +54,7 @@ import 
org.apache.hadoop.hbase.client.RegionAdminServiceCallable;
 import org.apache.hadoop.hbase.client.RegionReplicaUtil;
 import org.apache.hadoop.hbase.client.RetryingCallable;
 import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.ipc.HBaseRpcController;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.protobuf.ReplicationProtbufUtil;
@@ -399,7 +399,7 @@ public class RegionReplicaReplicationEndpoint extends 
HBaseReplicationEndpoint {
       if (requiresReplication == null) {
         // check if the table requires memstore replication
         // some unit-test drop the table, so we should do a bypass check and 
always replicate.
-        HTableDescriptor htd = tableDescriptors.get(tableName);
+        TableDescriptor htd = tableDescriptors.get(tableName);
         requiresReplication = htd == null || 
htd.hasRegionMemstoreReplication();
         memstoreReplicationEnabled.put(tableName, requiresReplication);
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/b0334863/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotHelper.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotHelper.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotHelper.java
index 9875ac0..cae4c7e 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotHelper.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotHelper.java
@@ -34,6 +34,7 @@ import java.util.Set;
 import java.util.TreeMap;
 import java.util.concurrent.ThreadPoolExecutor;
 
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.ListMultimap;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -43,7 +44,6 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.backup.HFileArchiver;
 import org.apache.hadoop.hbase.MetaTableAccessor;
@@ -125,7 +125,7 @@ public class RestoreSnapshotHelper {
   private final SnapshotDescription snapshotDesc;
   private final TableName snapshotTable;
 
-  private final HTableDescriptor tableDesc;
+  private final TableDescriptor tableDesc;
   private final Path rootDir;
   private final Path tableDir;
 
@@ -136,7 +136,7 @@ public class RestoreSnapshotHelper {
   public RestoreSnapshotHelper(final Configuration conf,
       final FileSystem fs,
       final SnapshotManifest manifest,
-      final HTableDescriptor tableDescriptor,
+      final TableDescriptor tableDescriptor,
       final Path rootDir,
       final ForeignExceptionDispatcher monitor,
       final MonitoredTask status) {
@@ -146,7 +146,7 @@ public class RestoreSnapshotHelper {
   public RestoreSnapshotHelper(final Configuration conf,
       final FileSystem fs,
       final SnapshotManifest manifest,
-      final HTableDescriptor tableDescriptor,
+      final TableDescriptor tableDescriptor,
       final Path rootDir,
       final ForeignExceptionDispatcher monitor,
       final MonitoredTask status,
@@ -265,18 +265,18 @@ public class RestoreSnapshotHelper {
    */
   public static class RestoreMetaChanges {
     private final Map<String, Pair<String, String> > parentsMap;
-    private final HTableDescriptor htd;
+    private final TableDescriptor htd;
 
     private List<HRegionInfo> regionsToRestore = null;
     private List<HRegionInfo> regionsToRemove = null;
     private List<HRegionInfo> regionsToAdd = null;
 
-    public RestoreMetaChanges(HTableDescriptor htd, Map<String, Pair<String, 
String> > parentsMap) {
+    public RestoreMetaChanges(TableDescriptor htd, Map<String, Pair<String, 
String> > parentsMap) {
       this.parentsMap = parentsMap;
       this.htd = htd;
     }
 
-    public HTableDescriptor getTableDescriptor() {
+    public TableDescriptor getTableDescriptor() {
       return htd;
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/b0334863/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifest.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifest.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifest.java
index f70fe9e..32cdabf 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifest.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifest.java
@@ -36,10 +36,10 @@ import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.errorhandling.ForeignExceptionSnare;
 import org.apache.hadoop.hbase.mob.MobUtils;
 import org.apache.hadoop.hbase.regionserver.HRegion;
@@ -76,7 +76,7 @@ public final class SnapshotManifest {
 
   private List<SnapshotRegionManifest> regionManifests;
   private SnapshotDescription desc;
-  private HTableDescriptor htd;
+  private TableDescriptor htd;
 
   private final ForeignExceptionSnare monitor;
   private final Configuration conf;
@@ -119,7 +119,7 @@ public final class SnapshotManifest {
   /**
    * Return a SnapshotManifest instance with the information already loaded 
in-memory.
    *    SnapshotManifest manifest = SnapshotManifest.open(...)
-   *    HTableDescriptor htd = manifest.getTableDescriptor()
+   *    TableDescriptor htd = manifest.getTableDescriptor()
    *    for (SnapshotRegionManifest regionManifest: 
manifest.getRegionManifests())
    *      hri = regionManifest.getRegionInfo()
    *      for (regionManifest.getFamilyFiles())
@@ -136,7 +136,7 @@ public final class SnapshotManifest {
   /**
    * Add the table descriptor to the snapshot manifest
    */
-  public void addTableDescriptor(final HTableDescriptor htd) throws 
IOException {
+  public void addTableDescriptor(final TableDescriptor htd) throws IOException 
{
     this.htd = htd;
   }
 
@@ -182,7 +182,7 @@ public final class SnapshotManifest {
     LOG.debug("Creating references for mob files");
 
     Path mobRegionPath = MobUtils.getMobRegionPath(conf, 
regionInfo.getTable());
-    for (HColumnDescriptor hcd : htd.getColumnFamilies()) {
+    for (ColumnFamilyDescriptor hcd : htd.getColumnFamilies()) {
       // 2.1. build the snapshot reference for the store if it's a mob store
       if (!hcd.isMobEnabled()) {
         continue;
@@ -377,7 +377,7 @@ public final class SnapshotManifest {
       case SnapshotManifestV2.DESCRIPTOR_VERSION: {
         SnapshotDataManifest dataManifest = readDataManifest();
         if (dataManifest != null) {
-          htd = 
ProtobufUtil.convertToHTableDesc(dataManifest.getTableSchema());
+          htd = ProtobufUtil.toTableDescriptor(dataManifest.getTableSchema());
           regionManifests = dataManifest.getRegionManifestsList();
         } else {
           // Compatibility, load the v1 regions
@@ -429,7 +429,7 @@ public final class SnapshotManifest {
   /**
    * Get the table descriptor from the Snapshot
    */
-  public HTableDescriptor getTableDescriptor() {
+  public TableDescriptor getTableDescriptor() {
     return this.htd;
   }
 
@@ -485,7 +485,7 @@ public final class SnapshotManifest {
     }
 
     SnapshotDataManifest.Builder dataManifestBuilder = 
SnapshotDataManifest.newBuilder();
-    dataManifestBuilder.setTableSchema(ProtobufUtil.convertToTableSchema(htd));
+    dataManifestBuilder.setTableSchema(ProtobufUtil.toTableSchema(htd));
 
     if (v1Regions != null && v1Regions.size() > 0) {
       dataManifestBuilder.addAllRegionManifests(v1Regions);

http://git-wip-us.apache.org/repos/asf/hbase/blob/b0334863/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java
----------------------------------------------------------------------
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 eb6b766..a73883b 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
@@ -24,11 +24,10 @@ import java.util.List;
 import java.util.Map;
 import java.util.TreeMap;
 import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.Function;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
-import 
org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
-import org.apache.hadoop.hbase.shaded.com.google.common.primitives.Ints;
 import edu.umd.cs.findbugs.annotations.Nullable;
 import org.apache.commons.lang.NotImplementedException;
 import org.apache.commons.logging.Log;
@@ -40,17 +39,19 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathFilter;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.Coprocessor;
-import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.regionserver.BloomType;
+import 
org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hbase.shaded.com.google.common.primitives.Ints;
 import org.apache.hadoop.hbase.TableDescriptors;
 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;
 
 /**
  * Implementation of {@link TableDescriptors} that reads descriptors from the
@@ -79,10 +80,14 @@ public class FSTableDescriptors implements TableDescriptors 
{
   private volatile boolean usecache;
   private volatile boolean fsvisited;
 
-  @VisibleForTesting long cachehits = 0;
-  @VisibleForTesting long invocations = 0;
+  @VisibleForTesting
+  long cachehits = 0;
+  @VisibleForTesting
+  long invocations = 0;
 
-  /** The file name prefix used to store HTD in HDFS  */
+  /**
+   * The file name prefix used to store HTD in HDFS
+   */
   static final String TABLEINFO_FILE_PREFIX = ".tableinfo";
   static final String TABLEINFO_DIR = ".tabledesc";
   static final String TMP_DIR = ".tmp";
@@ -90,12 +95,12 @@ public class FSTableDescriptors implements TableDescriptors 
{
   // This cache does not age out the old stuff.  Thinking is that the amount
   // of data we keep up in here is so small, no need to do occasional purge.
   // TODO.
-  private final Map<TableName, HTableDescriptor> cache = new 
ConcurrentHashMap<>();
+  private final Map<TableName, TableDescriptor> cache = new 
ConcurrentHashMap<>();
 
   /**
    * Table descriptor for <code>hbase:meta</code> catalog table
    */
-  private final HTableDescriptor metaTableDescriptor;
+  private final TableDescriptor metaTableDescriptor;
 
   /**
    * Construct a FSTableDescriptors instance using the hbase root dir of the 
given
@@ -107,91 +112,112 @@ public class FSTableDescriptors implements 
TableDescriptors {
   }
 
   public FSTableDescriptors(final Configuration conf, final FileSystem fs, 
final Path rootdir)
-  throws IOException {
+          throws IOException {
     this(conf, fs, rootdir, false, true);
   }
 
   /**
    * @param fsreadonly True if we are read-only when it comes to filesystem
-   * operations; i.e. on remove, we do not do delete in fs.
+   *                   operations; i.e. on remove, we do not do delete in fs.
+   */
+  public FSTableDescriptors(final Configuration conf, final FileSystem fs,
+                            final Path rootdir, final boolean fsreadonly, 
final boolean usecache) throws IOException {
+    this(conf, fs, rootdir, fsreadonly, usecache, null);
+  }
+
+  /**
+   * @param fsreadonly True if we are read-only when it comes to filesystem
+   *                   operations; i.e. on remove, we do not do delete in fs.
+   * @param metaObserver Used by HMaster. It need to modify the 
META_REPLICAS_NUM for meta table descriptor.
+   *                     see HMaster#finishActiveMasterInitialization
+   *                     TODO: This is a workaround. Should remove this ugly 
code...
    */
   public FSTableDescriptors(final Configuration conf, final FileSystem fs,
-    final Path rootdir, final boolean fsreadonly, final boolean usecache) 
throws IOException {
-    super();
+                            final Path rootdir, final boolean fsreadonly, 
final boolean usecache,
+                            Function<TableDescriptorBuilder, 
TableDescriptorBuilder> metaObserver) throws IOException {
     this.fs = fs;
     this.rootdir = rootdir;
     this.fsreadonly = fsreadonly;
     this.usecache = usecache;
+    this.metaTableDescriptor = metaObserver == null ? 
createMetaTableDescriptor(conf)
+          : metaObserver.apply(createMetaTableDescriptorBuilder(conf)).build();
+  }
 
-    this.metaTableDescriptor = createMetaTableDescriptor(conf);
+  @VisibleForTesting
+  public static TableDescriptorBuilder createMetaTableDescriptorBuilder(final 
Configuration conf) throws IOException {
+    return TableDescriptorBuilder.newBuilder(TableName.META_TABLE_NAME)
+            
.addColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(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,
+                            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)
+                    .build())
+            
.addColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(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,
+                            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)
+                    .build())
+            
.addColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(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,
+                            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)
+                    .build())
+            
.addColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(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,
+                            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)
+                    .build())
+            
.addColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(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)
+                    .build())
+            
.addCoprocessor("org.apache.hadoop.hbase.coprocessor.MultiRowMutationEndpoint",
+                    null, Coprocessor.PRIORITY_SYSTEM, null);
   }
 
   @VisibleForTesting
-  public static HTableDescriptor createMetaTableDescriptor(final Configuration 
conf)
+  public static TableDescriptor createMetaTableDescriptor(final Configuration 
conf)
       throws IOException {
-    return new 
HTableDescriptor(TableDescriptorBuilder.newBuilder(TableName.META_TABLE_NAME)
-          .addColumnFamily(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,
-                    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))
-          .addColumnFamily(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,
-                    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))
-          .addColumnFamily(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,
-                    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))
-          .addColumnFamily(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,
-                    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))
-          .addColumnFamily(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());
+    return createMetaTableDescriptorBuilder(conf).build();
   }
 
   @Override
@@ -219,7 +245,7 @@ public class FSTableDescriptors implements TableDescriptors 
{
    */
   @Override
   @Nullable
-  public HTableDescriptor get(final TableName tablename)
+  public TableDescriptor get(final TableName tablename)
   throws IOException {
     invocations++;
     if (TableName.META_TABLE_NAME.equals(tablename)) {
@@ -234,13 +260,13 @@ public class FSTableDescriptors implements 
TableDescriptors {
 
     if (usecache) {
       // Look in cache of descriptors.
-      HTableDescriptor cachedtdm = this.cache.get(tablename);
+      TableDescriptor cachedtdm = this.cache.get(tablename);
       if (cachedtdm != null) {
         cachehits++;
         return cachedtdm;
       }
     }
-    HTableDescriptor tdmt = null;
+    TableDescriptor tdmt = null;
     try {
       tdmt = getTableDescriptorFromFs(fs, rootdir, tablename);
     } catch (NullPointerException e) {
@@ -264,21 +290,21 @@ public class FSTableDescriptors implements 
TableDescriptors {
    * Returns a map from table name to table descriptor for all tables.
    */
   @Override
-  public Map<String, HTableDescriptor> getAllDescriptors()
+  public Map<String, TableDescriptor> getAllDescriptors()
   throws IOException {
-    Map<String, HTableDescriptor> tds = new TreeMap<>();
+    Map<String, TableDescriptor> tds = new TreeMap<>();
 
     if (fsvisited && usecache) {
-      for (Map.Entry<TableName, HTableDescriptor> entry: 
this.cache.entrySet()) {
+      for (Map.Entry<TableName, TableDescriptor> entry: this.cache.entrySet()) 
{
         tds.put(entry.getKey().toString(), entry.getValue());
       }
       // add hbase:meta to the response
-      tds.put(this.metaTableDescriptor.getNameAsString(), metaTableDescriptor);
+      tds.put(this.metaTableDescriptor.getTableName().getNameAsString(), 
metaTableDescriptor);
     } else {
       LOG.debug("Fetching table descriptors from the filesystem.");
       boolean allvisited = true;
       for (Path d : FSUtils.getTableDirs(fs, rootdir)) {
-        HTableDescriptor htd = null;
+        TableDescriptor htd = null;
         try {
           htd = get(FSUtils.getTableName(d));
         } catch (FileNotFoundException fnfe) {
@@ -301,10 +327,10 @@ public class FSTableDescriptors implements 
TableDescriptors {
    * Returns a map from table name to table descriptor for all tables.
    */
   @Override
-  public Map<String, HTableDescriptor> getAll() throws IOException {
-    Map<String, HTableDescriptor> htds = new TreeMap<>();
-    Map<String, HTableDescriptor> allDescriptors = getAllDescriptors();
-    for (Map.Entry<String, HTableDescriptor> entry : allDescriptors
+  public Map<String, TableDescriptor> getAll() throws IOException {
+    Map<String, TableDescriptor> htds = new TreeMap<>();
+    Map<String, TableDescriptor> allDescriptors = getAllDescriptors();
+    for (Map.Entry<String, TableDescriptor> entry : allDescriptors
         .entrySet()) {
       htds.put(entry.getKey(), entry.getValue());
     }
@@ -316,13 +342,13 @@ public class FSTableDescriptors implements 
TableDescriptors {
     * @see #get(org.apache.hadoop.hbase.TableName)
     */
   @Override
-  public Map<String, HTableDescriptor> getByNamespace(String name)
+  public Map<String, TableDescriptor> getByNamespace(String name)
   throws IOException {
-    Map<String, HTableDescriptor> htds = new TreeMap<>();
+    Map<String, TableDescriptor> htds = new TreeMap<>();
     List<Path> tableDirs =
         FSUtils.getLocalTableDirs(fs, FSUtils.getNamespaceDir(rootdir, name));
     for (Path d: tableDirs) {
-      HTableDescriptor htd = null;
+      TableDescriptor htd = null;
       try {
         htd = get(FSUtils.getTableName(d));
       } catch (FileNotFoundException fnfe) {
@@ -340,7 +366,7 @@ public class FSTableDescriptors implements TableDescriptors 
{
    * and updates the local cache with it.
    */
   @Override
-  public void add(HTableDescriptor htd) throws IOException {
+  public void add(TableDescriptor htd) throws IOException {
     if (fsreadonly) {
       throw new NotImplementedException("Cannot add a table descriptor - in 
read only mode");
     }
@@ -351,7 +377,7 @@ public class FSTableDescriptors implements TableDescriptors 
{
     if 
(HConstants.HBASE_NON_USER_TABLE_DIRS.contains(tableName.getNameAsString())) {
       throw new NotImplementedException(
           "Cannot add a table descriptor for a reserved subdirectory name: "
-              + htd.getNameAsString());
+              + htd.getTableName().getNameAsString());
     }
     updateTableDescriptor(htd);
   }
@@ -362,7 +388,7 @@ public class FSTableDescriptors implements TableDescriptors 
{
    * from the FileSystem.
    */
   @Override
-  public HTableDescriptor remove(final TableName tablename)
+  public TableDescriptor remove(final TableName tablename)
   throws IOException {
     if (fsreadonly) {
       throw new NotImplementedException("Cannot remove a table descriptor - in 
read only mode");
@@ -373,7 +399,7 @@ public class FSTableDescriptors implements TableDescriptors 
{
         throw new IOException("Failed delete of " + tabledir.toString());
       }
     }
-    HTableDescriptor descriptor = this.cache.remove(tablename);
+    TableDescriptor descriptor = this.cache.remove(tablename);
     return descriptor;
   }
 
@@ -557,7 +583,7 @@ public class FSTableDescriptors implements TableDescriptors 
{
    * if it exists, bypassing the local cache.
    * Returns null if it's not found.
    */
-  public static HTableDescriptor getTableDescriptorFromFs(FileSystem fs,
+  public static TableDescriptor getTableDescriptorFromFs(FileSystem fs,
       Path hbaseRootDir, TableName tableName) throws IOException {
     Path tableDir = FSUtils.getTableDir(hbaseRootDir, tableName);
     return getTableDescriptorFromFs(fs, tableDir);
@@ -568,7 +594,7 @@ public class FSTableDescriptors implements TableDescriptors 
{
    * directly from the file system if it exists.
    * @throws TableInfoMissingException if there is no descriptor
    */
-  public static HTableDescriptor getTableDescriptorFromFs(FileSystem fs, Path 
tableDir)
+  public static TableDescriptor getTableDescriptorFromFs(FileSystem fs, Path 
tableDir)
   throws IOException {
     FileStatus status = getTableInfoPath(fs, tableDir, false);
     if (status == null) {
@@ -577,7 +603,7 @@ public class FSTableDescriptors implements TableDescriptors 
{
     return readTableDescriptor(fs, status);
   }
 
-  private static HTableDescriptor readTableDescriptor(FileSystem fs, 
FileStatus status)
+  private static TableDescriptor readTableDescriptor(FileSystem fs, FileStatus 
status)
       throws IOException {
     int len = Ints.checkedCast(status.getLen());
     byte [] content = new byte[len];
@@ -587,9 +613,9 @@ public class FSTableDescriptors implements TableDescriptors 
{
     } finally {
       fsDataInputStream.close();
     }
-    HTableDescriptor htd = null;
+    TableDescriptor htd = null;
     try {
-      htd = HTableDescriptor.parseFrom(content);
+      htd = TableDescriptorBuilder.parseFrom(content);
     } catch (DeserializationException e) {
       throw new IOException("content=" + Bytes.toShort(content), e);
     }
@@ -601,7 +627,7 @@ public class FSTableDescriptors implements TableDescriptors 
{
    * @throws IOException Thrown if failed update.
    * @throws NotImplementedException if in read only mode
    */
-  @VisibleForTesting Path updateTableDescriptor(HTableDescriptor td)
+  @VisibleForTesting Path updateTableDescriptor(TableDescriptor td)
   throws IOException {
     if (fsreadonly) {
       throw new NotImplementedException("Cannot update a table descriptor - in 
read only mode");
@@ -663,7 +689,7 @@ public class FSTableDescriptors implements TableDescriptors 
{
    * @return Descriptor file or null if we failed write.
    */
   private static Path writeTableDescriptor(final FileSystem fs,
-    final HTableDescriptor htd, final Path tableDir,
+    final TableDescriptor htd, final Path tableDir,
     final FileStatus currentDescriptorFile)
   throws IOException {
     // Get temporary dir into which we'll first write a file to avoid 
half-written file phenomenon.
@@ -718,42 +744,42 @@ public class FSTableDescriptors implements 
TableDescriptors {
     return tableInfoDirPath;
   }
 
-  private static void writeTD(final FileSystem fs, final Path p, final 
HTableDescriptor htd)
+  private static void writeTD(final FileSystem fs, final Path p, final 
TableDescriptor htd)
   throws IOException {
     FSDataOutputStream out = fs.create(p, false);
     try {
       // We used to write this file out as a serialized HTD Writable followed 
by two '\n's and then
       // the toString version of HTD.  Now we just write out the pb 
serialization.
-      out.write(htd.toByteArray());
+      out.write(TableDescriptorBuilder.toByteArray(htd));
     } finally {
       out.close();
     }
   }
 
   /**
-   * Create new HTableDescriptor in HDFS. Happens when we are creating table.
+   * Create new TableDescriptor in HDFS. Happens when we are creating table.
    * Used by tests.
    * @return True if we successfully created file.
    */
-  public boolean createTableDescriptor(HTableDescriptor htd) throws 
IOException {
+  public boolean createTableDescriptor(TableDescriptor htd) throws IOException 
{
     return createTableDescriptor(htd, false);
   }
 
   /**
-   * Create new HTableDescriptor in HDFS. Happens when we are creating table. 
If
+   * Create new TableDescriptor in HDFS. Happens when we are creating table. If
    * forceCreation is true then even if previous table descriptor is present it
    * will be overwritten
    *
    * @return True if we successfully created file.
    */
-  public boolean createTableDescriptor(HTableDescriptor htd, boolean 
forceCreation)
+  public boolean createTableDescriptor(TableDescriptor htd, boolean 
forceCreation)
   throws IOException {
     Path tableDir = getTableDir(htd.getTableName());
     return createTableDescriptorForTableDirectory(tableDir, htd, 
forceCreation);
   }
 
   /**
-   * Create a new HTableDescriptor in HDFS in the specified table directory. 
Happens when we create
+   * Create a new TableDescriptor in HDFS in the specified table directory. 
Happens when we create
    * a new table or snapshot a table.
    * @param tableDir table directory under which we should write the file
    * @param htd description of the table to write
@@ -764,7 +790,7 @@ public class FSTableDescriptors implements TableDescriptors 
{
    * @throws IOException if a filesystem error occurs
    */
   public boolean createTableDescriptorForTableDirectory(Path tableDir,
-      HTableDescriptor htd, boolean forceCreation) throws IOException {
+      TableDescriptor htd, boolean forceCreation) throws IOException {
     if (fsreadonly) {
       throw new NotImplementedException("Cannot create a table descriptor - in 
read only mode");
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/b0334863/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
index ff5d482..199ed7d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
@@ -17,6 +17,10 @@
  */
 package org.apache.hadoop.hbase.util;
 
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.shaded.com.google.common.base.Joiner;
 import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableList;
@@ -84,11 +88,9 @@ import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.ClusterStatus;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HRegionLocation;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.MasterNotRunningException;
 import org.apache.hadoop.hbase.MetaTableAccessor;
@@ -939,7 +941,7 @@ public class HBaseFsck extends Configured implements 
Closeable {
     TableName tableName = hi.getTableName();
     TableInfo tableInfo = tablesInfo.get(tableName);
     Preconditions.checkNotNull(tableInfo, "Table '" + tableName + "' not 
present!");
-    HTableDescriptor template = tableInfo.getHTD();
+    TableDescriptor template = tableInfo.getHTD();
 
     // find min and max key values
     Pair<byte[],byte[]> orphanRegionRange = null;
@@ -1200,17 +1202,17 @@ public class HBaseFsck extends Configured implements 
Closeable {
    */
   private void reportTablesInFlux() {
     AtomicInteger numSkipped = new AtomicInteger(0);
-    HTableDescriptor[] allTables = getTables(numSkipped);
+    TableDescriptor[] allTables = getTables(numSkipped);
     errors.print("Number of Tables: " + allTables.length);
     if (details) {
       if (numSkipped.get() > 0) {
         errors.detail("Number of Tables in flux: " + numSkipped.get());
       }
-      for (HTableDescriptor td : allTables) {
+      for (TableDescriptor td : allTables) {
         errors.detail("  Table: " + td.getTableName() + "\t" +
                            (td.isReadOnly() ? "ro" : "rw") + "\t" +
                             (td.isMetaRegion() ? "META" : "    ") + "\t" +
-                           " families: " + td.getFamilies().size());
+                           " families: " + td.getColumnFamilyCount());
       }
     }
   }
@@ -1314,7 +1316,7 @@ public class HBaseFsck extends Configured implements 
Closeable {
         modTInfo = new TableInfo(tableName);
         tablesInfo.put(tableName, modTInfo);
         try {
-          HTableDescriptor htd =
+          TableDescriptor htd =
               FSTableDescriptors.getTableDescriptorFromFs(fs, hbaseRoot, 
tableName);
           modTInfo.htds.add(htd);
         } catch (IOException ioe) {
@@ -1361,17 +1363,17 @@ public class HBaseFsck extends Configured implements 
Closeable {
    * To fabricate a .tableinfo file with following contents<br>
    * 1. the correct tablename <br>
    * 2. the correct colfamily list<br>
-   * 3. the default properties for both {@link HTableDescriptor} and {@link 
HColumnDescriptor}<br>
+   * 3. the default properties for both {@link TableDescriptor} and {@link 
ColumnFamilyDescriptor}<br>
    * @throws IOException
    */
   private boolean fabricateTableInfo(FSTableDescriptors fstd, TableName 
tableName,
       Set<String> columns) throws IOException {
     if (columns ==null || columns.isEmpty()) return false;
-    HTableDescriptor htd = new HTableDescriptor(tableName);
+    TableDescriptorBuilder builder = 
TableDescriptorBuilder.newBuilder(tableName);
     for (String columnfamimly : columns) {
-      htd.addFamily(new HColumnDescriptor(columnfamimly));
+      builder.addColumnFamily(ColumnFamilyDescriptorBuilder.of(columnfamimly));
     }
-    fstd.createTableDescriptor(htd, true);
+    fstd.createTableDescriptor(builder.build(), true);
     return true;
   }
 
@@ -1396,7 +1398,7 @@ public class HBaseFsck extends Configured implements 
Closeable {
    * 2. else create a default .tableinfo file with following items<br>
    * &nbsp;2.1 the correct tablename <br>
    * &nbsp;2.2 the correct colfamily list<br>
-   * &nbsp;2.3 the default properties for both {@link HTableDescriptor} and 
{@link HColumnDescriptor}<br>
+   * &nbsp;2.3 the default properties for both {@link TableDescriptor} and 
{@link ColumnFamilyDescriptor}<br>
    * @throws IOException
    */
   public void fixOrphanTables() throws IOException {
@@ -1404,7 +1406,7 @@ public class HBaseFsck extends Configured implements 
Closeable {
 
       List<TableName> tmpList = new 
ArrayList<>(orphanTableDirs.keySet().size());
       tmpList.addAll(orphanTableDirs.keySet());
-      HTableDescriptor[] htds = getHTableDescriptors(tmpList);
+      TableDescriptor[] htds = getTableDescriptors(tmpList);
       Iterator<Entry<TableName, Set<String>>> iter =
           orphanTableDirs.entrySet().iterator();
       int j = 0;
@@ -1417,7 +1419,7 @@ public class HBaseFsck extends Configured implements 
Closeable {
         LOG.info("Trying to fix orphan table error: " + tableName);
         if (j < htds.length) {
           if (tableName.equals(htds[j].getTableName())) {
-            HTableDescriptor htd = htds[j];
+            TableDescriptor htd = htds[j];
             LOG.info("fixing orphan table: " + tableName + " from cache");
             fstd.createTableDescriptor(htd, true);
             j++;
@@ -1426,7 +1428,7 @@ public class HBaseFsck extends Configured implements 
Closeable {
         } else {
           if (fabricateTableInfo(fstd, tableName, entry.getValue())) {
             LOG.warn("fixing orphan table: " + tableName + " with a default 
.tableinfo file");
-            LOG.warn("Strongly recommend to modify the HTableDescriptor if 
necessary for: " + tableName);
+            LOG.warn("Strongly recommend to modify the TableDescriptor if 
necessary for: " + tableName);
             iter.remove();
           } else {
             LOG.error("Unable to create default .tableinfo for " + tableName + 
" while missing column family information");
@@ -1463,7 +1465,7 @@ public class HBaseFsck extends Configured implements 
Closeable {
     Path rootdir = FSUtils.getRootDir(getConf());
     Configuration c = getConf();
     HRegionInfo metaHRI = new HRegionInfo(HRegionInfo.FIRST_META_REGIONINFO);
-    HTableDescriptor metaDescriptor = new 
FSTableDescriptors(c).get(TableName.META_TABLE_NAME);
+    TableDescriptor metaDescriptor = new 
FSTableDescriptors(c).get(TableName.META_TABLE_NAME);
     MasterFileSystem.setInfoFamilyCachingForMeta(metaDescriptor, false);
     // The WAL subsystem will use the default rootDir rather than the passed 
in rootDir
     // unless I pass along via the conf.
@@ -2646,8 +2648,8 @@ public class HBaseFsck extends Configured implements 
Closeable {
    * regions reported for the table, but table dir is there in hdfs
    */
   private void loadTableInfosForTablesWithNoRegion() throws IOException {
-    Map<String, HTableDescriptor> allTables = new 
FSTableDescriptors(getConf()).getAll();
-    for (HTableDescriptor htd : allTables.values()) {
+    Map<String, TableDescriptor> allTables = new 
FSTableDescriptors(getConf()).getAll();
+    for (TableDescriptor htd : allTables.values()) {
       if (checkMetaOnly && !htd.isMetaTable()) {
         continue;
       }
@@ -2770,8 +2772,8 @@ public class HBaseFsck extends Configured implements 
Closeable {
     // region split calculator
     final RegionSplitCalculator<HbckInfo> sc = new 
RegionSplitCalculator<>(cmp);
 
-    // Histogram of different HTableDescriptors found.  Ideally there is only 
one!
-    final Set<HTableDescriptor> htds = new HashSet<>();
+    // Histogram of different TableDescriptors found.  Ideally there is only 
one!
+    final Set<TableDescriptor> htds = new HashSet<>();
 
     // key = start split, values = set of splits in problem group
     final Multimap<byte[], HbckInfo> overlapGroups =
@@ -2788,9 +2790,9 @@ public class HBaseFsck extends Configured implements 
Closeable {
     /**
      * @return descriptor common to all regions.  null if are none or multiple!
      */
-    private HTableDescriptor getHTD() {
+    private TableDescriptor getHTD() {
       if (htds.size() == 1) {
-        return (HTableDescriptor)htds.toArray()[0];
+        return (TableDescriptor)htds.toArray()[0];
       } else {
         LOG.error("None/Multiple table descriptors found for table '"
           + tableName + "' regions: " + htds);
@@ -2960,7 +2962,7 @@ public class HBaseFsck extends Configured implements 
Closeable {
             "First region should start with an empty key.  Creating a new " +
             "region and regioninfo in HDFS to plug the hole.",
             getTableInfo(), next);
-        HTableDescriptor htd = getTableInfo().getHTD();
+        TableDescriptor htd = getTableInfo().getHTD();
         // from special EMPTY_START_ROW to next region's startKey
         HRegionInfo newRegion = new HRegionInfo(htd.getTableName(),
             HConstants.EMPTY_START_ROW, next.getStartKey());
@@ -2977,7 +2979,7 @@ public class HBaseFsck extends Configured implements 
Closeable {
         errors.reportError(ERROR_CODE.LAST_REGION_ENDKEY_NOT_EMPTY,
             "Last region should end with an empty key.  Creating a new "
                 + "region and regioninfo in HDFS to plug the hole.", 
getTableInfo());
-        HTableDescriptor htd = getTableInfo().getHTD();
+        TableDescriptor htd = getTableInfo().getHTD();
         // from curEndKey to EMPTY_START_ROW
         HRegionInfo newRegion = new HRegionInfo(htd.getTableName(), curEndKey,
             HConstants.EMPTY_START_ROW);
@@ -3001,7 +3003,7 @@ public class HBaseFsck extends Configured implements 
Closeable {
                 + Bytes.toStringBinary(holeStopKey)
                 + ".  Creating a new regioninfo and region "
                 + "dir in hdfs to plug the hole.");
-        HTableDescriptor htd = getTableInfo().getHTD();
+        TableDescriptor htd = getTableInfo().getHTD();
         HRegionInfo newRegion = new HRegionInfo(htd.getTableName(), 
holeStartKey, holeStopKey);
         HRegion region = HBaseFsckRepair.createHDFSRegionDir(conf, newRegion, 
htd);
         LOG.info("Plugged hole by creating new empty region: "+ newRegion + " 
" +region);
@@ -3202,7 +3204,7 @@ public class HBaseFsck extends Configured implements 
Closeable {
         }
 
         // create new empty container region.
-        HTableDescriptor htd = getTableInfo().getHTD();
+        TableDescriptor htd = getTableInfo().getHTD();
         // from start key to end Key
         HRegionInfo newRegion = new HRegionInfo(htd.getTableName(), 
range.getFirst(),
             range.getSecond());
@@ -3503,7 +3505,7 @@ public class HBaseFsck extends Configured implements 
Closeable {
    * @return tables that have not been modified recently
    * @throws IOException if an error is encountered
    */
-  HTableDescriptor[] getTables(AtomicInteger numSkipped) {
+  TableDescriptor[] getTables(AtomicInteger numSkipped) {
     List<TableName> tableNames = new ArrayList<>();
     long now = EnvironmentEdgeManager.currentTime();
 
@@ -3520,19 +3522,19 @@ public class HBaseFsck extends Configured implements 
Closeable {
         }
       }
     }
-    return getHTableDescriptors(tableNames);
+    return getTableDescriptors(tableNames);
   }
 
-  HTableDescriptor[] getHTableDescriptors(List<TableName> tableNames) {
-    HTableDescriptor[] htd = new HTableDescriptor[0];
-      LOG.info("getHTableDescriptors == tableNames => " + tableNames);
+  TableDescriptor[] getTableDescriptors(List<TableName> tableNames) {
+      LOG.info("getTableDescriptors == tableNames => " + tableNames);
     try (Connection conn = ConnectionFactory.createConnection(getConf());
         Admin admin = conn.getAdmin()) {
-      htd = admin.getTableDescriptorsByTableName(tableNames);
+      List<TableDescriptor> tds = admin.listTableDescriptors(tableNames);
+      return tds.toArray(new TableDescriptor[tds.size()]);
     } catch (IOException e) {
       LOG.debug("Exception getting table descriptors", e);
     }
-    return htd;
+    return new TableDescriptor[0];
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/b0334863/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java
index 961e8a0..6552b32 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java
@@ -28,7 +28,6 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
@@ -40,6 +39,7 @@ import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.master.RegionState;
 import org.apache.hadoop.hbase.master.ServerManager;
 import org.apache.hadoop.hbase.regionserver.HRegion;
@@ -188,7 +188,7 @@ public class HBaseFsckRepair {
    * Creates, flushes, and closes a new region.
    */
   public static HRegion createHDFSRegionDir(Configuration conf,
-      HRegionInfo hri, HTableDescriptor htd) throws IOException {
+      HRegionInfo hri, TableDescriptor htd) throws IOException {
     // Create HRegion
     Path root = FSUtils.getRootDir(conf);
     HRegion region = HRegion.createHRegion(hri, root, conf, htd, null);

http://git-wip-us.apache.org/repos/asf/hbase/blob/b0334863/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ModifyRegionUtils.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ModifyRegionUtils.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ModifyRegionUtils.java
index 8ea7012..cc478a5 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ModifyRegionUtils.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ModifyRegionUtils.java
@@ -39,9 +39,8 @@ import 
org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.regionserver.HRegion;
-import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
 
 /**
  * Utility methods for interacting with the regions.
@@ -61,13 +60,13 @@ public abstract class ModifyRegionUtils {
     void editRegion(final HRegionInfo region) throws IOException;
   }
 
-  public static HRegionInfo[] createHRegionInfos(HTableDescriptor 
hTableDescriptor,
+  public static HRegionInfo[] createHRegionInfos(TableDescriptor 
tableDescriptor,
       byte[][] splitKeys) {
     long regionId = System.currentTimeMillis();
     HRegionInfo[] hRegionInfos = null;
     if (splitKeys == null || splitKeys.length == 0) {
       hRegionInfos = new HRegionInfo[]{
-        new HRegionInfo(hTableDescriptor.getTableName(), null, null, false, 
regionId)
+        new HRegionInfo(tableDescriptor.getTableName(), null, null, false, 
regionId)
       };
     } else {
       int numRegions = splitKeys.length + 1;
@@ -77,7 +76,7 @@ public abstract class ModifyRegionUtils {
       for (int i = 0; i < numRegions; i++) {
         endKey = (i == splitKeys.length) ? null : splitKeys[i];
         hRegionInfos[i] =
-             new HRegionInfo(hTableDescriptor.getTableName(), startKey, endKey,
+             new HRegionInfo(tableDescriptor.getTableName(), startKey, endKey,
                  false, regionId);
         startKey = endKey;
       }
@@ -91,20 +90,20 @@ public abstract class ModifyRegionUtils {
    *
    * @param conf {@link Configuration}
    * @param rootDir Root directory for HBase instance
-   * @param hTableDescriptor description of the table
+   * @param tableDescriptor description of the table
    * @param newRegions {@link HRegionInfo} that describes the regions to create
    * @param task {@link RegionFillTask} custom code to populate region after 
creation
    * @throws IOException
    */
   public static List<HRegionInfo> createRegions(final Configuration conf, 
final Path rootDir,
-      final HTableDescriptor hTableDescriptor, final HRegionInfo[] newRegions,
+      final TableDescriptor tableDescriptor, final HRegionInfo[] newRegions,
       final RegionFillTask task) throws IOException {
     if (newRegions == null) return null;
     int regionNumber = newRegions.length;
     ThreadPoolExecutor exec = getRegionOpenAndInitThreadPool(conf,
-        "RegionOpenAndInitThread-" + hTableDescriptor.getTableName(), 
regionNumber);
+        "RegionOpenAndInitThread-" + tableDescriptor.getTableName(), 
regionNumber);
     try {
-      return createRegions(exec, conf, rootDir, hTableDescriptor, newRegions, 
task);
+      return createRegions(exec, conf, rootDir, tableDescriptor, newRegions, 
task);
     } finally {
       exec.shutdownNow();
     }
@@ -117,15 +116,15 @@ public abstract class ModifyRegionUtils {
    * @param exec Thread Pool Executor
    * @param conf {@link Configuration}
    * @param rootDir Root directory for HBase instance
-   * @param hTableDescriptor description of the table
+   * @param tableDescriptor description of the table
    * @param newRegions {@link HRegionInfo} that describes the regions to create
    * @param task {@link RegionFillTask} custom code to populate region after 
creation
    * @throws IOException
    */
   public static List<HRegionInfo> createRegions(final ThreadPoolExecutor exec,
-      final Configuration conf, final Path rootDir,
-      final HTableDescriptor hTableDescriptor, final HRegionInfo[] newRegions,
-      final RegionFillTask task) throws IOException {
+                                                final Configuration conf, 
final Path rootDir,
+                                                final TableDescriptor 
tableDescriptor, final HRegionInfo[] newRegions,
+                                                final RegionFillTask task) 
throws IOException {
     if (newRegions == null) return null;
     int regionNumber = newRegions.length;
     CompletionService<HRegionInfo> completionService = new 
ExecutorCompletionService<>(exec);
@@ -134,7 +133,7 @@ public abstract class ModifyRegionUtils {
       completionService.submit(new Callable<HRegionInfo>() {
         @Override
         public HRegionInfo call() throws IOException {
-          return createRegion(conf, rootDir, hTableDescriptor, newRegion, 
task);
+          return createRegion(conf, rootDir, tableDescriptor, newRegion, task);
         }
       });
     }
@@ -156,20 +155,20 @@ public abstract class ModifyRegionUtils {
    * Create new set of regions on the specified file-system.
    * @param conf {@link Configuration}
    * @param rootDir Root directory for HBase instance
-   * @param hTableDescriptor description of the table
+   * @param tableDescriptor description of the table
    * @param newRegion {@link HRegionInfo} that describes the region to create
    * @param task {@link RegionFillTask} custom code to populate region after 
creation
    * @throws IOException
    */
   public static HRegionInfo createRegion(final Configuration conf, final Path 
rootDir,
-      final HTableDescriptor hTableDescriptor, final HRegionInfo newRegion,
+      final TableDescriptor tableDescriptor, final HRegionInfo newRegion,
       final RegionFillTask task) throws IOException {
     // 1. Create HRegion
     // The WAL subsystem will use the default rootDir rather than the passed 
in rootDir
     // unless I pass along via the conf.
     Configuration confForWAL = new Configuration(conf);
     confForWAL.set(HConstants.HBASE_DIR, rootDir.toString());
-    HRegion region = HRegion.createHRegion(newRegion, rootDir, conf, 
hTableDescriptor, null, false);
+    HRegion region = HRegion.createHRegion(newRegion, rootDir, conf, 
tableDescriptor, null, false);
     try {
       // 2. Custom user code to interact with the created region
       if (task != null) {

Reply via email to