You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ch...@apache.org on 2017/08/24 05:06:49 UTC

[8/8] hbase git commit: HBASE-18503 Change ***Util and Master to use TableDescriptor and ColumnFamilyDescriptor

HBASE-18503 Change ***Util and Master to use TableDescriptor and ColumnFamilyDescriptor


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

Branch: refs/heads/master
Commit: 25ff9d0bbf36a68cdac99035c8d5ab1eb889ceb9
Parents: 12f2b02
Author: Chia-Ping Tsai <ch...@gmail.com>
Authored: Thu Aug 24 13:03:38 2017 +0800
Committer: Chia-Ping Tsai <ch...@gmail.com>
Committed: Thu Aug 24 13:03:38 2017 +0800

----------------------------------------------------------------------
 .../hadoop/hbase/backup/util/BackupUtils.java   |   4 +-
 .../hadoop/hbase/backup/util/RestoreTool.java   |  48 ++--
 .../apache/hadoop/hbase/HColumnDescriptor.java  |  11 +-
 .../apache/hadoop/hbase/HTableDescriptor.java   |  11 +-
 .../client/ColumnFamilyDescriptorBuilder.java   |  13 +-
 .../apache/hadoop/hbase/client/HBaseAdmin.java  |  41 +--
 .../hadoop/hbase/client/RawAsyncHBaseAdmin.java |   2 +-
 .../hbase/client/TableDescriptorBuilder.java    |  20 +-
 .../hbase/shaded/protobuf/ProtobufUtil.java     | 101 +-------
 .../hbase/shaded/protobuf/RequestConverter.java |  18 +-
 .../apache/hadoop/hbase/TableDescriptors.java   |  15 +-
 .../hbase/client/ClientSideRegionScanner.java   |   3 +-
 .../hbase/client/TableSnapshotScanner.java      |   3 +-
 .../mapreduce/TableSnapshotInputFormatImpl.java |  18 +-
 .../hadoop/hbase/master/CatalogJanitor.java     |  13 +-
 .../master/ExpiredMobFileCleanerChore.java      |  10 +-
 .../org/apache/hadoop/hbase/master/HMaster.java |  86 +++----
 .../hadoop/hbase/master/MasterFileSystem.java   |  24 +-
 .../hbase/master/MasterMobCompactionThread.java |  10 +-
 .../hadoop/hbase/master/MasterRpcServices.java  |  25 +-
 .../hadoop/hbase/master/MasterServices.java     |  15 +-
 .../hadoop/hbase/master/MobCompactionChore.java |  10 +-
 .../hadoop/hbase/master/TableStateManager.java  |   6 +-
 .../assignment/MergeTableRegionsProcedure.java  |  10 +-
 .../master/assignment/RegionStateStore.java     |  12 +-
 .../master/balancer/RegionLocationFinder.java   |  12 +-
 .../master/cleaner/ReplicationMetaCleaner.java  |  10 +-
 .../procedure/AddColumnFamilyProcedure.java     |  50 ++--
 .../procedure/CloneSnapshotProcedure.java       |  51 ++--
 .../master/procedure/CreateTableProcedure.java  |  66 ++---
 .../procedure/DeleteColumnFamilyProcedure.java  |  37 +--
 .../procedure/ModifyColumnFamilyProcedure.java  |  43 ++--
 .../master/procedure/ModifyTableProcedure.java  |  75 +++---
 .../procedure/RestoreSnapshotProcedure.java     |  34 +--
 .../procedure/TruncateTableProcedure.java       |  22 +-
 .../master/snapshot/MasterSnapshotVerifier.java |   8 +-
 .../hbase/master/snapshot/SnapshotManager.java  |  29 ++-
 .../master/snapshot/TakeSnapshotHandler.java    |  10 +-
 .../hadoop/hbase/mob/ExpiredMobFileCleaner.java |  10 +-
 .../org/apache/hadoop/hbase/mob/MobUtils.java   |  27 +-
 .../hbase/mob/compactions/MobCompactor.java     |   6 +-
 .../compactions/PartitionedMobCompactor.java    |   4 +-
 .../hbase/regionserver/CompactionTool.java      |  16 +-
 .../hbase/regionserver/HRegionFileSystem.java   |   8 +-
 .../hbase/regionserver/HRegionServer.java       |   8 +-
 .../hbase/regionserver/RSRpcServices.java       |   8 +-
 .../regionserver/handler/OpenMetaHandler.java   |   6 +-
 .../handler/OpenPriorityRegionHandler.java      |   5 +-
 .../regionserver/handler/OpenRegionHandler.java |  10 +-
 .../RegionReplicaReplicationEndpoint.java       |   4 +-
 .../hbase/snapshot/RestoreSnapshotHelper.java   |  14 +-
 .../hadoop/hbase/snapshot/SnapshotManifest.java |  18 +-
 .../hadoop/hbase/util/FSTableDescriptors.java   | 258 ++++++++++---------
 .../org/apache/hadoop/hbase/util/HBaseFsck.java |  68 ++---
 .../hadoop/hbase/util/HBaseFsckRepair.java      |   4 +-
 .../hadoop/hbase/util/ModifyRegionUtils.java    |  33 ++-
 .../hadoop/hbase/HBaseTestingUtility.java       |  13 +-
 .../TestFSTableDescriptorForceCreation.java     |  13 +-
 .../TestHColumnDescriptorDefaultVersions.java   |  12 +-
 .../hbase/client/TestAsyncTableAdminApi.java    |   5 +-
 .../hbase/master/MockNoopMasterServices.java    |  13 +-
 .../master/assignment/MockMasterServices.java   |  24 +-
 .../MasterProcedureTestingUtility.java          |  40 +--
 .../procedure/TestCreateTableProcedure.java     |  33 ++-
 .../TestMasterFailoverWithProcedures.java       |   4 +-
 .../procedure/TestMasterProcedureWalLease.java  |   4 +-
 ...stTableDescriptorModificationFromClient.java |   7 +-
 .../TestPartitionedMobCompactor.java            |   3 +-
 .../regionserver/TestGetClosestAtOrBefore.java  |   6 +-
 .../TestRegionMergeTransactionOnCluster.java    |  14 +-
 .../regionserver/TestRegionServerNoMaster.java  |   4 +-
 .../hbase/security/access/SecureTestUtil.java   |  21 +-
 .../hbase/snapshot/MobSnapshotTestingUtils.java |  65 ++---
 .../hbase/snapshot/SnapshotTestingUtils.java    |  50 ++--
 .../snapshot/TestRestoreSnapshotHelper.java     |  16 +-
 .../hbase/snapshot/TestSnapshotManifest.java    |   8 +-
 .../hbase/util/TestFSTableDescriptors.java      |  81 +++---
 77 files changed, 942 insertions(+), 977 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/25ff9d0b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/BackupUtils.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/BackupUtils.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/BackupUtils.java
index ce77645..11a1a3d 100644
--- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/BackupUtils.java
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/BackupUtils.java
@@ -44,7 +44,6 @@ import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hbase.HConstants;
 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;
@@ -56,6 +55,7 @@ import org.apache.hadoop.hbase.backup.impl.BackupManifest;
 import org.apache.hadoop.hbase.backup.impl.BackupManifest.BackupImage;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
@@ -139,7 +139,7 @@ public final class BackupUtils {
         LOG.warn("Table " + table + " does not exists, skipping it.");
         continue;
       }
-      HTableDescriptor orig = FSTableDescriptors.getTableDescriptorFromFs(fs, rootDir, table);
+      TableDescriptor orig = FSTableDescriptors.getTableDescriptorFromFs(fs, rootDir, table);
 
       // write a copy of descriptor to the target directory
       Path target = new Path(backupInfo.getTableBackupDir(table));

http://git-wip-us.apache.org/repos/asf/hbase/blob/25ff9d0b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreTool.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreTool.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreTool.java
index 0cfe099..2e311cf 100644
--- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreTool.java
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreTool.java
@@ -33,16 +33,17 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
-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.backup.BackupRestoreFactory;
 import org.apache.hadoop.hbase.backup.HBackupFileSystem;
 import org.apache.hadoop.hbase.backup.RestoreJob;
 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.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.io.HFileLink;
 import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles;
@@ -122,10 +123,10 @@ public class RestoreTool {
   }
 
 
-  void modifyTableSync(Connection conn, HTableDescriptor desc) throws IOException {
+  void modifyTableSync(Connection conn, TableDescriptor desc) throws IOException {
 
     try (Admin admin = conn.getAdmin();) {
-      admin.modifyTable(desc.getTableName(), desc);
+      admin.modifyTable(desc);
       int attempt = 0;
       int maxAttempts = 600;
       while (!admin.isTableAvailable(desc.getTableName())) {
@@ -172,29 +173,30 @@ public class RestoreTool {
       // adjust table schema
       for (int i = 0; i < tableNames.length; i++) {
         TableName tableName = tableNames[i];
-        HTableDescriptor tableDescriptor = getTableDescriptor(fileSys, tableName, incrBackupId);
+        TableDescriptor tableDescriptor = getTableDescriptor(fileSys, tableName, incrBackupId);
         LOG.debug("Found descriptor " + tableDescriptor + " through " + incrBackupId);
 
         TableName newTableName = newTableNames[i];
-        HTableDescriptor newTableDescriptor = new HTableDescriptor(admin.getTableDescriptor(newTableName));
-        List<HColumnDescriptor> families = Arrays.asList(tableDescriptor.getColumnFamilies());
-        List<HColumnDescriptor> existingFamilies =
+        TableDescriptor newTableDescriptor = admin.listTableDescriptor(newTableName);
+        List<ColumnFamilyDescriptor> families = Arrays.asList(tableDescriptor.getColumnFamilies());
+        List<ColumnFamilyDescriptor> existingFamilies =
             Arrays.asList(newTableDescriptor.getColumnFamilies());
+        TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(newTableDescriptor);
         boolean schemaChangeNeeded = false;
-        for (HColumnDescriptor family : families) {
+        for (ColumnFamilyDescriptor family : families) {
           if (!existingFamilies.contains(family)) {
-            newTableDescriptor.addFamily(family);
+            builder.addColumnFamily(family);
             schemaChangeNeeded = true;
           }
         }
-        for (HColumnDescriptor family : existingFamilies) {
+        for (ColumnFamilyDescriptor family : existingFamilies) {
           if (!families.contains(family)) {
-            newTableDescriptor.removeFamily(family.getName());
+            builder.removeColumnFamily(family.getName());
             schemaChangeNeeded = true;
           }
         }
         if (schemaChangeNeeded) {
-          modifyTableSync(conn, newTableDescriptor);
+          modifyTableSync(conn, builder.build());
           LOG.info("Changed " + newTableDescriptor.getTableName() + " to: " + newTableDescriptor);
         }
       }
@@ -253,24 +255,24 @@ public class RestoreTool {
   /**
    * Get table descriptor
    * @param tableName is the table backed up
-   * @return {@link HTableDescriptor} saved in backup image of the table
+   * @return {@link TableDescriptor} saved in backup image of the table
    */
-  HTableDescriptor getTableDesc(TableName tableName) throws FileNotFoundException, IOException {
+  TableDescriptor getTableDesc(TableName tableName) throws FileNotFoundException, IOException {
     Path tableInfoPath = this.getTableInfoPath(tableName);
     SnapshotDescription desc = SnapshotDescriptionUtils.readSnapshotInfo(fs, tableInfoPath);
     SnapshotManifest manifest = SnapshotManifest.open(conf, fs, tableInfoPath, desc);
-    HTableDescriptor tableDescriptor = manifest.getTableDescriptor();
+    TableDescriptor tableDescriptor = manifest.getTableDescriptor();
     if (!tableDescriptor.getTableName().equals(tableName)) {
       LOG.error("couldn't find Table Desc for table: " + tableName + " under tableInfoPath: "
           + tableInfoPath.toString());
-      LOG.error("tableDescriptor.getNameAsString() = " + tableDescriptor.getNameAsString());
+      LOG.error("tableDescriptor.getNameAsString() = " + tableDescriptor.getTableName().getNameAsString());
       throw new FileNotFoundException("couldn't find Table Desc for table: " + tableName
           + " under tableInfoPath: " + tableInfoPath.toString());
     }
     return tableDescriptor;
   }
 
-  private HTableDescriptor getTableDescriptor(FileSystem fileSys, TableName tableName,
+  private TableDescriptor getTableDescriptor(FileSystem fileSys, TableName tableName,
       String lastIncrBackupId) throws IOException {
     if (lastIncrBackupId != null) {
       String target =
@@ -289,7 +291,7 @@ public class RestoreTool {
     FileSystem fileSys = tableBackupPath.getFileSystem(this.conf);
 
     // get table descriptor first
-    HTableDescriptor tableDescriptor = getTableDescriptor(fileSys, tableName, lastIncrBackupId);
+    TableDescriptor tableDescriptor = getTableDescriptor(fileSys, tableName, lastIncrBackupId);
     if (tableDescriptor != null) {
       LOG.debug("Retrieved descriptor: " + tableDescriptor + " thru " + lastIncrBackupId);
     }
@@ -325,7 +327,7 @@ public class RestoreTool {
           LOG.debug("find table descriptor but no archive dir for table " + tableName
               + ", will only create table");
         }
-        tableDescriptor = new HTableDescriptor(newTableName, tableDescriptor);
+        tableDescriptor = TableDescriptorBuilder.copy(newTableName, tableDescriptor);
         checkAndCreateTable(conn, tableBackupPath, tableName, newTableName, null, tableDescriptor,
           truncateIfExists);
         return;
@@ -336,9 +338,9 @@ public class RestoreTool {
     }
 
     if (tableDescriptor == null) {
-      tableDescriptor = new HTableDescriptor(newTableName);
+      tableDescriptor = TableDescriptorBuilder.newBuilder(newTableName).build();
     } else {
-      tableDescriptor = new HTableDescriptor(newTableName, tableDescriptor);
+      tableDescriptor = TableDescriptorBuilder.copy(newTableName, tableDescriptor);
     }
 
     // record all region dirs:
@@ -470,7 +472,7 @@ public class RestoreTool {
    * @throws IOException exception
    */
   private void checkAndCreateTable(Connection conn, Path tableBackupPath, TableName tableName,
-      TableName targetTableName, ArrayList<Path> regionDirList, HTableDescriptor htd,
+      TableName targetTableName, ArrayList<Path> regionDirList, TableDescriptor htd,
       boolean truncateIfExists) throws IOException {
     try (Admin admin = conn.getAdmin();) {
       boolean createNew = false;

http://git-wip-us.apache.org/repos/asf/hbase/blob/25ff9d0b/hbase-client/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java
index 507bf49..8802553 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java
@@ -639,13 +639,10 @@ public class HColumnDescriptor implements ColumnFamilyDescriptor, Comparable<HCo
     if (this == obj) {
       return true;
     }
-    if (obj == null) {
-      return false;
+    if (obj instanceof HColumnDescriptor) {
+      return delegatee.equals(((HColumnDescriptor) obj).delegatee);
     }
-    if (!(obj instanceof HColumnDescriptor)) {
-      return false;
-    }
-    return compareTo((HColumnDescriptor)obj) == 0;
+    return false;
   }
 
   /**
@@ -658,7 +655,7 @@ public class HColumnDescriptor implements ColumnFamilyDescriptor, Comparable<HCo
 
   @Override
   public int compareTo(HColumnDescriptor other) {
-    return delegatee.compareTo(other.delegatee);
+    return COMPARATOR.compare(this, other);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/25ff9d0b/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 a0f23c1..86ba287 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
@@ -495,13 +495,10 @@ public class HTableDescriptor implements TableDescriptor, Comparable<HTableDescr
     if (this == obj) {
       return true;
     }
-    if (obj == null) {
-      return false;
+    if (obj instanceof HTableDescriptor) {
+      return delegatee.equals(((HTableDescriptor) obj).delegatee);
     }
-    if (!(obj instanceof HTableDescriptor)) {
-      return false;
-    }
-    return compareTo((HTableDescriptor)obj) == 0;
+    return false;
   }
 
   /**
@@ -523,7 +520,7 @@ public class HTableDescriptor implements TableDescriptor, Comparable<HTableDescr
    */
   @Override
   public int compareTo(final HTableDescriptor other) {
-    return delegatee.compareTo(other.delegatee);
+    return TableDescriptor.COMPARATOR.compare(this, other);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/25ff9d0b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ColumnFamilyDescriptorBuilder.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ColumnFamilyDescriptorBuilder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ColumnFamilyDescriptorBuilder.java
index b3abaca..67d2c56 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ColumnFamilyDescriptorBuilder.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ColumnFamilyDescriptorBuilder.java
@@ -1160,13 +1160,10 @@ public class ColumnFamilyDescriptorBuilder {
       if (this == obj) {
         return true;
       }
-      if (obj == null) {
-        return false;
+      if (obj instanceof ModifyableColumnFamilyDescriptor) {
+        return ColumnFamilyDescriptor.COMPARATOR.compare(this, (ModifyableColumnFamilyDescriptor) obj) == 0;
       }
-      if (!(obj instanceof ModifyableColumnFamilyDescriptor)) {
-        return false;
-      }
-      return compareTo((ModifyableColumnFamilyDescriptor) obj) == 0;
+      return false;
     }
 
     @Override
@@ -1188,7 +1185,7 @@ public class ColumnFamilyDescriptorBuilder {
      * @see #parseFrom(byte[])
      */
     private byte[] toByteArray() {
-      return ProtobufUtil.prependPBMagic(ProtobufUtil.convertToColumnFamilySchema(this)
+      return ProtobufUtil.prependPBMagic(ProtobufUtil.toColumnFamilySchema(this)
                       .toByteArray());
     }
 
@@ -1213,7 +1210,7 @@ public class ColumnFamilyDescriptorBuilder {
       } catch (IOException e) {
         throw new DeserializationException(e);
       }
-      return ProtobufUtil.convertToColumnDesc(cfs);
+      return ProtobufUtil.toColumnFamilyDescriptor(cfs);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/25ff9d0b/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 11f3273..a2fa7e0 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
@@ -378,7 +378,7 @@ public class HBaseAdmin implements Admin {
                   .setNamespaceName(Bytes.toString(name)).build())
                 .getTableSchemaList()
                 .stream()
-                .map(ProtobufUtil::convertToTableDesc)
+                .map(ProtobufUtil::toTableDescriptor)
                 .collect(Collectors.toList());
       }
     });
@@ -459,8 +459,8 @@ public class HBaseAdmin implements Admin {
       protected HTableDescriptor[] rpcCall() throws Exception {
         GetTableDescriptorsRequest req =
             RequestConverter.buildGetTableDescriptorsRequest(pattern, includeSysTables);
-        return ProtobufUtil.getHTableDescriptorArray(master.getTableDescriptors(getRpcController(),
-            req));
+        return ProtobufUtil.toTableDescriptorList(master.getTableDescriptors(getRpcController(),
+                req)).stream().map(ImmutableHTableDescriptor::new).toArray(HTableDescriptor[]::new);
       }
     });
   }
@@ -525,7 +525,7 @@ public class HBaseAdmin implements Admin {
             RequestConverter.buildGetTableDescriptorsRequest(tableName);
         GetTableDescriptorsResponse htds = master.getTableDescriptors(getRpcController(), req);
         if (!htds.getTableSchemaList().isEmpty()) {
-          return ProtobufUtil.convertToTableDesc(htds.getTableSchemaList().get(0));
+          return ProtobufUtil.toTableDescriptor(htds.getTableSchemaList().get(0));
         }
         return null;
       }
@@ -554,7 +554,7 @@ public class HBaseAdmin implements Admin {
             RequestConverter.buildGetTableDescriptorsRequest(tableName);
         GetTableDescriptorsResponse htds = master.getTableDescriptors(getRpcController(), req);
         if (!htds.getTableSchemaList().isEmpty()) {
-          return ProtobufUtil.convertToHTableDesc(htds.getTableSchemaList().get(0));
+          return new ImmutableHTableDescriptor(ProtobufUtil.toTableDescriptor(htds.getTableSchemaList().get(0)));
         }
         return null;
       }
@@ -2300,7 +2300,7 @@ public class HBaseAdmin implements Admin {
                 .build()).getTableSchemaList();
         HTableDescriptor[] res = new HTableDescriptor[list.size()];
         for(int i=0; i < list.size(); i++) {
-          res[i] = new ImmutableHTableDescriptor(ProtobufUtil.convertToHTableDesc(list.get(i)));
+          res[i] = new ImmutableHTableDescriptor(ProtobufUtil.toTableDescriptor(list.get(i)));
         }
         return res;
       }
@@ -2419,33 +2419,14 @@ public class HBaseAdmin implements Admin {
       protected HTableDescriptor[] rpcCall() throws Exception {
         GetTableDescriptorsRequest req =
             RequestConverter.buildGetTableDescriptorsRequest(tableNames);
-          return ProtobufUtil.
-              getHTableDescriptorArray(master.getTableDescriptors(getRpcController(), req));
+          return ProtobufUtil.toTableDescriptorList(master.getTableDescriptors(getRpcController(), req))
+                  .stream()
+                  .map(ImmutableHTableDescriptor::new)
+                  .toArray(HTableDescriptor[]::new);
       }
     });
   }
 
-  /**
-   * Get tableDescriptor
-   * @param tableName one table name
-   * @return HTD the HTableDescriptor or null if the table not exists
-   * @throws IOException if a remote or network exception occurs
-   */
-  private HTableDescriptor getTableDescriptorByTableName(TableName tableName)
-      throws IOException {
-    List<TableName> tableNames = new ArrayList<>(1);
-    tableNames.add(tableName);
-
-    HTableDescriptor[] htdl = getTableDescriptorsByTableName(tableNames);
-
-    if (htdl == null || htdl.length == 0) {
-      return null;
-    }
-    else {
-      return htdl[0];
-    }
-  }
-
   @Override
   public HTableDescriptor[] getTableDescriptors(List<String> names)
   throws IOException {
@@ -3709,7 +3690,7 @@ public class HBaseAdmin implements Admin {
      * @return the table descriptor
      */
     protected TableDescriptor getTableDescriptor() throws IOException {
-      return getAdmin().getTableDescriptorByTableName(getTableName());
+      return getAdmin().listTableDescriptor(getTableName());
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/25ff9d0b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
index ba68a96..19bc2f4 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
@@ -453,7 +453,7 @@ public class RawAsyncHBaseAdmin implements AsyncAdmin {
             return;
           }
           if (!tableSchemas.isEmpty()) {
-            future.complete(ProtobufUtil.convertToTableDesc(tableSchemas.get(0)));
+            future.complete(ProtobufUtil.toTableDescriptor(tableSchemas.get(0)));
           } else {
             future.completeExceptionally(new TableNotFoundException(tableName.getNameAsString()));
           }

http://git-wip-us.apache.org/repos/asf/hbase/blob/25ff9d0b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java
index 44d5c99..a710077 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java
@@ -38,6 +38,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Coprocessor;
 import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
@@ -252,10 +253,14 @@ public class TableDescriptorBuilder {
     return new TableDescriptorBuilder(name);
   }
 
-  public static TableDescriptor copy(TableDescriptor desc) throws DeserializationException {
+  public static TableDescriptor copy(TableDescriptor desc) {
     return new ModifyableTableDescriptor(desc);
   }
 
+  public static TableDescriptor copy(TableName name, TableDescriptor desc) {
+    return new ModifyableTableDescriptor(name, desc);
+  }
+
   /**
    * Copy all configuration, values, families, and name from the input.
    * @param desc The desciptor to copy
@@ -1012,13 +1017,10 @@ public class TableDescriptorBuilder {
       if (this == obj) {
         return true;
       }
-      if (obj == null) {
-        return false;
+      if (obj instanceof ModifyableTableDescriptor) {
+        return TableDescriptor.COMPARATOR.compare(this, (ModifyableTableDescriptor) obj) == 0;
       }
-      if (!(obj instanceof ModifyableTableDescriptor)) {
-        return false;
-      }
-      return compareTo((ModifyableTableDescriptor) obj) == 0;
+      return false;
     }
 
     /**
@@ -1395,7 +1397,7 @@ public class TableDescriptorBuilder {
      * @return the bytes in pb format
      */
     private byte[] toByteArray() {
-      return ProtobufUtil.prependPBMagic(ProtobufUtil.convertToTableSchema(this).toByteArray());
+      return ProtobufUtil.prependPBMagic(ProtobufUtil.toTableSchema(this).toByteArray());
     }
 
     /**
@@ -1415,7 +1417,7 @@ public class TableDescriptorBuilder {
       HBaseProtos.TableSchema.Builder builder = HBaseProtos.TableSchema.newBuilder();
       try {
         ProtobufUtil.mergeFrom(builder, bytes, pblen, bytes.length - pblen);
-        return ProtobufUtil.convertToTableDesc(builder.build());
+        return ProtobufUtil.toTableDescriptor(builder.build());
       } catch (IOException e) {
         throw new DeserializationException(e);
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/25ff9d0b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
index a527883..abcc5e2 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
@@ -17,15 +17,12 @@
  */
 package org.apache.hadoop.hbase.shaded.protobuf;
 
-import java.awt.image.BandCombineOp;
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
-import java.io.InterruptedIOException;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.Method;
 import java.nio.ByteBuffer;
-import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
@@ -50,10 +47,8 @@ import org.apache.hadoop.hbase.ClusterStatus;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseIOException;
-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.KeyValue;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.ProcedureInfo;
@@ -74,7 +69,6 @@ import org.apache.hadoop.hbase.client.Cursor;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.ImmutableHTableDescriptor;
 import org.apache.hadoop.hbase.client.Increment;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.PackagePrivateFieldAccessor;
@@ -102,7 +96,6 @@ import org.apache.hadoop.hbase.quotas.SpaceViolationPolicy;
 import org.apache.hadoop.hbase.quotas.ThrottleType;
 import org.apache.hadoop.hbase.replication.ReplicationLoadSink;
 import org.apache.hadoop.hbase.replication.ReplicationLoadSource;
-import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.visibility.Authorizations;
 import org.apache.hadoop.hbase.security.visibility.CellVisibility;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString;
@@ -182,7 +175,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos;
 import org.apache.hadoop.hbase.util.Addressing;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.DynamicClassLoader;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.ExceptionUtil;
 import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
 import org.apache.hadoop.hbase.util.Methods;
@@ -425,24 +417,6 @@ public final class ProtobufUtil {
   }
 
   /**
-   * Get HTableDescriptor[] from GetTableDescriptorsResponse protobuf
-   *
-   * @param proto the GetTableDescriptorsResponse
-   * @return a immutable HTableDescriptor array
-   * @deprecated Use {@link #toTableDescriptorList} after removing the HTableDescriptor
-   */
-  @Deprecated
-  public static HTableDescriptor[] getHTableDescriptorArray(GetTableDescriptorsResponse proto) {
-    if (proto == null) return null;
-
-    HTableDescriptor[] ret = new HTableDescriptor[proto.getTableSchemaCount()];
-    for (int i = 0; i < proto.getTableSchemaCount(); ++i) {
-      ret[i] = new ImmutableHTableDescriptor(convertToHTableDesc(proto.getTableSchema(i)));
-    }
-    return ret;
-  }
-
-  /**
    * Get a list of TableDescriptor from GetTableDescriptorsResponse protobuf
    *
    * @param proto the GetTableDescriptorsResponse
@@ -450,7 +424,7 @@ public final class ProtobufUtil {
    */
   public static List<TableDescriptor> toTableDescriptorList(GetTableDescriptorsResponse proto) {
     if (proto == null) return new ArrayList<>();
-    return proto.getTableSchemaList().stream().map(ProtobufUtil::convertToTableDesc)
+    return proto.getTableSchemaList().stream().map(ProtobufUtil::toTableDescriptor)
         .collect(Collectors.toList());
   }
 
@@ -2841,11 +2815,11 @@ public final class ProtobufUtil {
   }
 
   /**
-   * Converts an HColumnDescriptor to ColumnFamilySchema
-   * @param hcd the HColummnDescriptor
+   * Converts an ColumnFamilyDescriptor to ColumnFamilySchema
+   * @param hcd the ColumnFamilySchema
    * @return Convert this instance to a the pb column family type
    */
-  public static ColumnFamilySchema convertToColumnFamilySchema(ColumnFamilyDescriptor hcd) {
+  public static ColumnFamilySchema toColumnFamilySchema(ColumnFamilyDescriptor hcd) {
     ColumnFamilySchema.Builder builder = ColumnFamilySchema.newBuilder();
     builder.setName(UnsafeByteOperations.unsafeWrap(hcd.getName()));
     for (Map.Entry<Bytes, Bytes> e : hcd.getValues().entrySet()) {
@@ -2864,31 +2838,11 @@ public final class ProtobufUtil {
   }
 
   /**
-   * Converts a ColumnFamilySchema to HColumnDescriptor
-   * @param cfs the ColumnFamilySchema
-   * @return An {@link HColumnDescriptor} made from the passed in <code>cfs</code>
-   */
-  @Deprecated
-  public static HColumnDescriptor convertToHColumnDesc(final ColumnFamilySchema cfs) {
-    // Use the empty constructor so we preserve the initial values set on construction for things
-    // like maxVersion.  Otherwise, we pick up wrong values on deserialization which makes for
-    // unrelated-looking test failures that are hard to trace back to here.
-    HColumnDescriptor hcd = new HColumnDescriptor(cfs.getName().toByteArray());
-    for (BytesBytesPair a: cfs.getAttributesList()) {
-      hcd.setValue(a.getFirst().toByteArray(), a.getSecond().toByteArray());
-    }
-    for (NameStringPair a: cfs.getConfigurationList()) {
-      hcd.setConfiguration(a.getName(), a.getValue());
-    }
-    return hcd;
-  }
-
-  /**
-   * Converts a ColumnFamilySchema to HColumnDescriptor
+   * Converts a ColumnFamilySchema to ColumnFamilyDescriptor
    * @param cfs the ColumnFamilySchema
-   * @return An {@link HColumnDescriptor} made from the passed in <code>cfs</code>
+   * @return An {@link ColumnFamilyDescriptor} made from the passed in <code>cfs</code>
    */
-  public static ColumnFamilyDescriptor convertToColumnDesc(final ColumnFamilySchema cfs) {
+  public static ColumnFamilyDescriptor toColumnFamilyDescriptor(final ColumnFamilySchema cfs) {
     // Use the empty constructor so we preserve the initial values set on construction for things
     // like maxVersion.  Otherwise, we pick up wrong values on deserialization which makes for
     // unrelated-looking test failures that are hard to trace back to here.
@@ -2900,11 +2854,11 @@ public final class ProtobufUtil {
   }
 
   /**
-   * Converts an HTableDescriptor to TableSchema
-   * @param htd the HTableDescriptor
-   * @return Convert the current {@link HTableDescriptor} into a pb TableSchema instance.
+   * Converts an TableDescriptor to TableSchema
+   * @param htd the TableDescriptor
+   * @return Convert the current {@link TableDescriptor} into a pb TableSchema instance.
    */
-  public static TableSchema convertToTableSchema(TableDescriptor htd) {
+  public static TableSchema toTableSchema(TableDescriptor htd) {
     TableSchema.Builder builder = TableSchema.newBuilder();
     builder.setTableName(toProtoTableName(htd.getTableName()));
     for (Map.Entry<Bytes, Bytes> e : htd.getValues().entrySet()) {
@@ -2914,7 +2868,7 @@ public final class ProtobufUtil {
       builder.addAttributes(aBuilder.build());
     }
     for (ColumnFamilyDescriptor hcd : htd.getColumnFamilies()) {
-      builder.addColumnFamilies(convertToColumnFamilySchema(hcd));
+      builder.addColumnFamilies(toColumnFamilySchema(hcd));
     }
     for (Map.Entry<String, String> e : htd.getConfiguration().entrySet()) {
       NameStringPair.Builder aBuilder = NameStringPair.newBuilder();
@@ -2926,43 +2880,16 @@ public final class ProtobufUtil {
   }
 
   /**
-   * Converts a TableSchema to HTableDescriptor
-   * @param ts A pb TableSchema instance.
-   * @return An {@link HTableDescriptor} made from the passed in pb <code>ts</code>.
-   * @deprecated Use {@link #convertToTableDesc} after removing the HTableDescriptor
-   */
-  @Deprecated
-  public static HTableDescriptor convertToHTableDesc(final TableSchema ts) {
-    List<ColumnFamilySchema> list = ts.getColumnFamiliesList();
-    HColumnDescriptor [] hcds = new HColumnDescriptor[list.size()];
-    int index = 0;
-    for (ColumnFamilySchema cfs: list) {
-      hcds[index++] = ProtobufUtil.convertToHColumnDesc(cfs);
-    }
-    HTableDescriptor htd = new HTableDescriptor(ProtobufUtil.toTableName(ts.getTableName()));
-    for (HColumnDescriptor hcd : hcds) {
-      htd.addFamily(hcd);
-    }
-    for (BytesBytesPair a: ts.getAttributesList()) {
-      htd.setValue(a.getFirst().toByteArray(), a.getSecond().toByteArray());
-    }
-    for (NameStringPair a: ts.getConfigurationList()) {
-      htd.setConfiguration(a.getName(), a.getValue());
-    }
-    return htd;
-  }
-
-  /**
    * Converts a TableSchema to TableDescriptor
    * @param ts A pb TableSchema instance.
    * @return An {@link TableDescriptor} made from the passed in pb <code>ts</code>.
    */
-  public static TableDescriptor convertToTableDesc(final TableSchema ts) {
+  public static TableDescriptor toTableDescriptor(final TableSchema ts) {
     TableDescriptorBuilder builder
       = TableDescriptorBuilder.newBuilder(ProtobufUtil.toTableName(ts.getTableName()));
     ts.getColumnFamiliesList()
       .stream()
-      .map(ProtobufUtil::convertToColumnDesc)
+      .map(ProtobufUtil::toColumnFamilyDescriptor)
       .forEach(builder::addColumnFamily);
     ts.getAttributesList()
       .forEach(a -> builder.setValue(a.getFirst().toByteArray(), a.getSecond().toByteArray()));

http://git-wip-us.apache.org/repos/asf/hbase/blob/25ff9d0b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
index 08ed3dc..a8a56c7 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
@@ -1080,7 +1080,7 @@ public final class RequestConverter {
       final long nonce) {
     AddColumnRequest.Builder builder = AddColumnRequest.newBuilder();
     builder.setTableName(ProtobufUtil.toProtoTableName(tableName));
-    builder.setColumnFamilies(ProtobufUtil.convertToColumnFamilySchema(column));
+    builder.setColumnFamilies(ProtobufUtil.toColumnFamilySchema(column));
     builder.setNonceGroup(nonceGroup);
     builder.setNonce(nonce);
     return builder.build();
@@ -1120,7 +1120,7 @@ public final class RequestConverter {
       final long nonce) {
     ModifyColumnRequest.Builder builder = ModifyColumnRequest.newBuilder();
     builder.setTableName(ProtobufUtil.toProtoTableName((tableName)));
-    builder.setColumnFamilies(ProtobufUtil.convertToColumnFamilySchema(column));
+    builder.setColumnFamilies(ProtobufUtil.toColumnFamilySchema(column));
     builder.setNonceGroup(nonceGroup);
     builder.setNonce(nonce);
     return builder.build();
@@ -1306,28 +1306,28 @@ public final class RequestConverter {
   /**
    * Creates a protocol buffer CreateTableRequest
    *
-   * @param hTableDesc
+   * @param tableDescriptor
    * @param splitKeys
    * @return a CreateTableRequest
    */
   public static CreateTableRequest buildCreateTableRequest(
-      final TableDescriptor hTableDesc,
+      final TableDescriptor tableDescriptor,
       final byte [][] splitKeys,
       final long nonceGroup,
       final long nonce) {
-    return buildCreateTableRequest(hTableDesc, Optional.ofNullable(splitKeys), nonceGroup, nonce);
+    return buildCreateTableRequest(tableDescriptor, Optional.ofNullable(splitKeys), nonceGroup, nonce);
   }
 
   /**
    * Creates a protocol buffer CreateTableRequest
-   * @param hTableDesc
+   * @param tableDescriptor
    * @param splitKeys
    * @return a CreateTableRequest
    */
-  public static CreateTableRequest buildCreateTableRequest(TableDescriptor hTableDesc,
+  public static CreateTableRequest buildCreateTableRequest(TableDescriptor tableDescriptor,
       Optional<byte[][]> splitKeys, long nonceGroup, long nonce) {
     CreateTableRequest.Builder builder = CreateTableRequest.newBuilder();
-    builder.setTableSchema(ProtobufUtil.convertToTableSchema(hTableDesc));
+    builder.setTableSchema(ProtobufUtil.toTableSchema(tableDescriptor));
     splitKeys.ifPresent(keys -> Arrays.stream(keys).forEach(
       key -> builder.addSplitKeys(UnsafeByteOperations.unsafeWrap(key))));
     builder.setNonceGroup(nonceGroup);
@@ -1349,7 +1349,7 @@ public final class RequestConverter {
       final long nonce) {
     ModifyTableRequest.Builder builder = ModifyTableRequest.newBuilder();
     builder.setTableName(ProtobufUtil.toProtoTableName((tableName)));
-    builder.setTableSchema(ProtobufUtil.convertToTableSchema(tableDesc));
+    builder.setTableSchema(ProtobufUtil.toTableSchema(tableDesc));
     builder.setNonceGroup(nonceGroup);
     builder.setNonce(nonce);
     return builder.build();

http://git-wip-us.apache.org/repos/asf/hbase/blob/25ff9d0b/hbase-server/src/main/java/org/apache/hadoop/hbase/TableDescriptors.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/TableDescriptors.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/TableDescriptors.java
index 7de2629..58b28e4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/TableDescriptors.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/TableDescriptors.java
@@ -21,6 +21,7 @@ import java.io.IOException;
 import java.util.Map;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 
 /**
  * Get, remove and modify table descriptors.
@@ -33,7 +34,7 @@ public interface TableDescriptors {
    * @return TableDescriptor for tablename
    * @throws IOException
    */
-  HTableDescriptor get(final TableName tableName)
+  TableDescriptor get(final TableName tableName)
       throws IOException;
 
   /**
@@ -41,16 +42,16 @@ public interface TableDescriptors {
    * @return Map of all descriptors.
    * @throws IOException
    */
-  Map<String, HTableDescriptor> getByNamespace(String name)
+  Map<String, TableDescriptor> getByNamespace(String name)
   throws IOException;
 
   /**
-   * Get Map of all HTableDescriptors. Populates the descriptor cache as a
+   * Get Map of all TableDescriptors. Populates the descriptor cache as a
    * side effect.
    * @return Map of all descriptors.
    * @throws IOException
    */
-  Map<String, HTableDescriptor> getAll()
+  Map<String, TableDescriptor> getAll()
   throws IOException;
 
   /**
@@ -59,7 +60,7 @@ public interface TableDescriptors {
    * @return Map of all descriptors.
    * @throws IOException
    */
-  Map<String, HTableDescriptor> getAllDescriptors()
+  Map<String, TableDescriptor> getAllDescriptors()
       throws IOException;
 
   /**
@@ -67,7 +68,7 @@ public interface TableDescriptors {
    * @param htd Descriptor to set into TableDescriptors
    * @throws IOException
    */
-  void add(final HTableDescriptor htd)
+  void add(final TableDescriptor htd)
   throws IOException;
 
   /**
@@ -75,7 +76,7 @@ public interface TableDescriptors {
    * @return Instance of table descriptor or null if none found.
    * @throws IOException
    */
-  HTableDescriptor remove(final TableName tablename)
+  TableDescriptor remove(final TableName tablename)
   throws IOException;
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/25ff9d0b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ClientSideRegionScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ClientSideRegionScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ClientSideRegionScanner.java
index 7ae0537..1d0d57b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ClientSideRegionScanner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ClientSideRegionScanner.java
@@ -30,7 +30,6 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 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.metrics.ScanMetrics;
 import org.apache.hadoop.hbase.regionserver.HRegion;
@@ -50,7 +49,7 @@ public class ClientSideRegionScanner extends AbstractClientScanner {
   List<Cell> values;
 
   public ClientSideRegionScanner(Configuration conf, FileSystem fs,
-      Path rootDir, HTableDescriptor htd, HRegionInfo hri, Scan scan, ScanMetrics scanMetrics)
+      Path rootDir, TableDescriptor htd, HRegionInfo hri, Scan scan, ScanMetrics scanMetrics)
           throws IOException {
     // region is immutable, set isolation level
     scan.setIsolationLevel(IsolationLevel.READ_UNCOMMITTED);

http://git-wip-us.apache.org/repos/asf/hbase/blob/25ff9d0b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/TableSnapshotScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/TableSnapshotScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/TableSnapshotScanner.java
index b861969..bcd433c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/TableSnapshotScanner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/TableSnapshotScanner.java
@@ -31,7 +31,6 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.snapshot.RestoreSnapshotHelper;
 import org.apache.hadoop.hbase.util.FSUtils;
@@ -75,7 +74,7 @@ public class TableSnapshotScanner extends AbstractClientScanner {
   private Path restoreDir;
   private Scan scan;
   private ArrayList<HRegionInfo> regions;
-  private HTableDescriptor htd;
+  private TableDescriptor htd;
 
   private ClientSideRegionScanner currentRegionScanner  = null;
   private int currentRegion = -1;

http://git-wip-us.apache.org/repos/asf/hbase/blob/25ff9d0b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableSnapshotInputFormatImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableSnapshotInputFormatImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableSnapshotInputFormatImpl.java
index 2f6955e..bf11473 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableSnapshotInputFormatImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableSnapshotInputFormatImpl.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.hbase.mapreduce;
 
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -28,7 +29,6 @@ import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HDFSBlocksDistribution;
 import org.apache.hadoop.hbase.HDFSBlocksDistribution.HostAndWeight;
 import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.client.ClientSideRegionScanner;
@@ -81,7 +81,7 @@ public class TableSnapshotInputFormatImpl {
    */
   public static class InputSplit implements Writable {
 
-    private HTableDescriptor htd;
+    private TableDescriptor htd;
     private HRegionInfo regionInfo;
     private String[] locations;
     private String scan;
@@ -90,7 +90,7 @@ public class TableSnapshotInputFormatImpl {
     // constructor for mapreduce framework / Writable
     public InputSplit() {}
 
-    public InputSplit(HTableDescriptor htd, HRegionInfo regionInfo, List<String> locations,
+    public InputSplit(TableDescriptor htd, HRegionInfo regionInfo, List<String> locations,
         Scan scan, Path restoreDir) {
       this.htd = htd;
       this.regionInfo = regionInfo;
@@ -108,7 +108,7 @@ public class TableSnapshotInputFormatImpl {
       this.restoreDir = restoreDir.toString();
     }
 
-    public HTableDescriptor getHtd() {
+    public TableDescriptor getHtd() {
       return htd;
     }
 
@@ -129,7 +129,7 @@ public class TableSnapshotInputFormatImpl {
       return locations;
     }
 
-    public HTableDescriptor getTableDescriptor() {
+    public TableDescriptor getTableDescriptor() {
       return htd;
     }
 
@@ -142,7 +142,7 @@ public class TableSnapshotInputFormatImpl {
     @Override
     public void write(DataOutput out) throws IOException {
       TableSnapshotRegionSplit.Builder builder = TableSnapshotRegionSplit.newBuilder()
-          .setTable(ProtobufUtil.convertToTableSchema(htd))
+          .setTable(ProtobufUtil.toTableSchema(htd))
           .setRegion(HRegionInfo.convert(regionInfo));
 
       for (String location : locations) {
@@ -169,7 +169,7 @@ public class TableSnapshotInputFormatImpl {
       byte[] buf = new byte[len];
       in.readFully(buf);
       TableSnapshotRegionSplit split = TableSnapshotRegionSplit.PARSER.parseFrom(buf);
-      this.htd = ProtobufUtil.convertToHTableDesc(split.getTable());
+      this.htd = ProtobufUtil.toTableDescriptor(split.getTable());
       this.regionInfo = HRegionInfo.convert(split.getRegion());
       List<String> locationsList = split.getLocationsList();
       this.locations = locationsList.toArray(new String[locationsList.size()]);
@@ -196,7 +196,7 @@ public class TableSnapshotInputFormatImpl {
     public void initialize(InputSplit split, Configuration conf) throws IOException {
       this.scan = TableMapReduceUtil.convertStringToScan(split.getScan());
       this.split = split;
-      HTableDescriptor htd = split.htd;
+      TableDescriptor htd = split.htd;
       HRegionInfo hri = this.split.getRegionInfo();
       FileSystem fs = FSUtils.getCurrentFileSystem(conf);
 
@@ -311,7 +311,7 @@ public class TableSnapshotInputFormatImpl {
   public static List<InputSplit> getSplits(Scan scan, SnapshotManifest manifest,
       List<HRegionInfo> regionManifests, Path restoreDir, Configuration conf) throws IOException {
     // load table descriptor
-    HTableDescriptor htd = manifest.getTableDescriptor();
+    TableDescriptor htd = manifest.getTableDescriptor();
 
     Path tableDir = FSUtils.getTableDir(restoreDir, htd.getTableName());
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/25ff9d0b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java
index 8daa7db..bcda145 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java
@@ -31,21 +31,20 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-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.MetaTableAccessor;
 import org.apache.hadoop.hbase.ScheduledChore;
 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.client.Connection;
 import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
 import org.apache.hadoop.hbase.master.assignment.GCMergedRegionsProcedure;
 import org.apache.hadoop.hbase.master.assignment.GCRegionProcedure;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
-import org.apache.hadoop.hbase.procedure2.Procedure;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -206,7 +205,7 @@ public class CatalogJanitor extends ScheduledChore {
     FileSystem fs = this.services.getMasterFileSystem().getFileSystem();
     Path rootdir = this.services.getMasterFileSystem().getRootDir();
     Path tabledir = FSUtils.getTableDir(rootdir, mergedRegion.getTable());
-    HTableDescriptor htd = getTableDescriptor(mergedRegion.getTable());
+    TableDescriptor htd = getTableDescriptor(mergedRegion.getTable());
     HRegionFileSystem regionFs = null;
     try {
       regionFs = HRegionFileSystem.openRegionFromFileSystem(
@@ -414,12 +413,12 @@ public class CatalogJanitor extends ScheduledChore {
     }
 
     boolean references = false;
-    HTableDescriptor parentDescriptor = getTableDescriptor(parent.getTable());
+    TableDescriptor parentDescriptor = getTableDescriptor(parent.getTable());
     try {
       regionFs = HRegionFileSystem.openRegionFromFileSystem(
           this.services.getConfiguration(), fs, tabledir, daughter, true);
 
-      for (HColumnDescriptor family: parentDescriptor.getFamilies()) {
+      for (ColumnFamilyDescriptor family: parentDescriptor.getColumnFamilies()) {
         if ((references = regionFs.hasReferences(family.getNameAsString()))) {
           break;
         }
@@ -432,7 +431,7 @@ public class CatalogJanitor extends ScheduledChore {
     return new Pair<>(Boolean.TRUE, Boolean.valueOf(references));
   }
 
-  private HTableDescriptor getTableDescriptor(final TableName tableName)
+  private TableDescriptor getTableDescriptor(final TableName tableName)
       throws FileNotFoundException, IOException {
     return this.services.getTableDescriptors().get(tableName);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/25ff9d0b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ExpiredMobFileCleanerChore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ExpiredMobFileCleanerChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ExpiredMobFileCleanerChore.java
index faa4f0e..c4438bb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ExpiredMobFileCleanerChore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ExpiredMobFileCleanerChore.java
@@ -23,11 +23,11 @@ import java.util.concurrent.TimeUnit;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.ScheduledChore;
 import org.apache.hadoop.hbase.TableDescriptors;
 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.master.locking.LockManager;
 import org.apache.hadoop.hbase.master.locking.LockProcedure;
 import org.apache.hadoop.hbase.mob.ExpiredMobFileCleaner;
@@ -61,9 +61,9 @@ public class ExpiredMobFileCleanerChore extends ScheduledChore {
   protected void chore() {
     try {
       TableDescriptors htds = master.getTableDescriptors();
-      Map<String, HTableDescriptor> map = htds.getAll();
-      for (HTableDescriptor htd : map.values()) {
-        for (HColumnDescriptor hcd : htd.getColumnFamilies()) {
+      Map<String, TableDescriptor> map = htds.getAll();
+      for (TableDescriptor htd : map.values()) {
+        for (ColumnFamilyDescriptor hcd : htd.getColumnFamilies()) {
           if (hcd.isMobEnabled() && hcd.getMinVersions() == 0) {
             // clean only for mob-enabled column.
             // obtain a read table lock before cleaning, synchronize with MobFileCompactionChore.

http://git-wip-us.apache.org/repos/asf/hbase/blob/25ff9d0b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index 6b4d4e9..93624de 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -41,6 +41,7 @@ import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Function;
 import java.util.regex.Pattern;
 
 import javax.servlet.ServletException;
@@ -60,10 +61,8 @@ import org.apache.hadoop.hbase.CoordinatedStateManager;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HBaseIOException;
 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.HTableDescriptor;
 import org.apache.hadoop.hbase.MasterNotRunningException;
 import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
@@ -77,9 +76,12 @@ import org.apache.hadoop.hbase.TableNotDisabledException;
 import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.UnknownRegionException;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.MasterSwitchType;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.client.TableState;
 import org.apache.hadoop.hbase.coprocessor.BypassCoprocessorException;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
@@ -590,11 +592,9 @@ public class HMaster extends HRegionServer implements MasterServices {
     return connector.getLocalPort();
   }
 
-  @Override
-  protected TableDescriptors getFsTableDescriptors() throws IOException {
-    return super.getFsTableDescriptors();
+  protected Function<TableDescriptorBuilder, TableDescriptorBuilder> getMetaTableObserver() {
+    return builder -> builder.setRegionReplication(conf.getInt(HConstants.META_REPLICAS_NUM, HConstants.DEFAULT_META_REPLICA_NUM));
   }
-
   /**
    * For compatibility, if failed with regionserver credentials, try the master one
    */
@@ -761,9 +761,7 @@ public class HMaster extends HRegionServer implements MasterServices {
 
     // enable table descriptors cache
     this.tableDescriptors.setCacheOn();
-    // set the META's descriptor to the correct replication
-    this.tableDescriptors.get(TableName.META_TABLE_NAME).setRegionReplication(
-        conf.getInt(HConstants.META_REPLICAS_NUM, HConstants.DEFAULT_META_REPLICA_NUM));
+
     // warm-up HTDs cache on master initialization
     if (preLoadTableDescriptors) {
       status.setStatus("Pre-loading table descriptors");
@@ -1501,7 +1499,7 @@ public class HMaster extends HRegionServer implements MasterServices {
           return false;
         }
 
-        HTableDescriptor tblDesc = getTableDescriptors().get(table);
+        TableDescriptor tblDesc = getTableDescriptors().get(table);
         if (table.isSystemTable() || (tblDesc != null &&
             !tblDesc.isNormalizationEnabled())) {
           LOG.debug("Skipping normalization for table: " + table + ", as it's either system"
@@ -1712,34 +1710,34 @@ public class HMaster extends HRegionServer implements MasterServices {
 
   @Override
   public long createTable(
-      final HTableDescriptor hTableDescriptor,
+      final TableDescriptor tableDescriptor,
       final byte [][] splitKeys,
       final long nonceGroup,
       final long nonce) throws IOException {
     checkInitialized();
 
-    String namespace = hTableDescriptor.getTableName().getNamespaceAsString();
+    String namespace = tableDescriptor.getTableName().getNamespaceAsString();
     this.clusterSchemaService.getNamespace(namespace);
 
-    HRegionInfo[] newRegions = ModifyRegionUtils.createHRegionInfos(hTableDescriptor, splitKeys);
-    sanityCheckTableDescriptor(hTableDescriptor);
+    HRegionInfo[] newRegions = ModifyRegionUtils.createHRegionInfos(tableDescriptor, splitKeys);
+    sanityCheckTableDescriptor(tableDescriptor);
 
     return MasterProcedureUtil.submitProcedure(
         new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
       @Override
       protected void run() throws IOException {
-        getMaster().getMasterCoprocessorHost().preCreateTable(hTableDescriptor, newRegions);
+        getMaster().getMasterCoprocessorHost().preCreateTable(tableDescriptor, newRegions);
 
-        LOG.info(getClientIdAuditPrefix() + " create " + hTableDescriptor);
+        LOG.info(getClientIdAuditPrefix() + " create " + tableDescriptor);
 
         // TODO: We can handle/merge duplicate requests, and differentiate the case of
         //       TableExistsException by saying if the schema is the same or not.
         ProcedurePrepareLatch latch = ProcedurePrepareLatch.createLatch();
         submitProcedure(new CreateTableProcedure(
-            procedureExecutor.getEnvironment(), hTableDescriptor, newRegions, latch));
+            procedureExecutor.getEnvironment(), tableDescriptor, newRegions, latch));
         latch.await();
 
-        getMaster().getMasterCoprocessorHost().postCreateTable(hTableDescriptor, newRegions);
+        getMaster().getMasterCoprocessorHost().postCreateTable(tableDescriptor, newRegions);
       }
 
       @Override
@@ -1750,25 +1748,25 @@ public class HMaster extends HRegionServer implements MasterServices {
   }
 
   @Override
-  public long createSystemTable(final HTableDescriptor hTableDescriptor) throws IOException {
+  public long createSystemTable(final TableDescriptor tableDescriptor) throws IOException {
     if (isStopped()) {
       throw new MasterNotRunningException();
     }
 
-    TableName tableName = hTableDescriptor.getTableName();
+    TableName tableName = tableDescriptor.getTableName();
     if (!(tableName.isSystemTable())) {
       throw new IllegalArgumentException(
         "Only system table creation can use this createSystemTable API");
     }
 
-    HRegionInfo[] newRegions = ModifyRegionUtils.createHRegionInfos(hTableDescriptor, null);
+    HRegionInfo[] newRegions = ModifyRegionUtils.createHRegionInfos(tableDescriptor, null);
 
-    LOG.info(getClientIdAuditPrefix() + " create " + hTableDescriptor);
+    LOG.info(getClientIdAuditPrefix() + " create " + tableDescriptor);
 
     // This special create table is called locally to master.  Therefore, no RPC means no need
     // to use nonce to detect duplicated RPC call.
     long procId = this.procedureExecutor.submitProcedure(
-      new CreateTableProcedure(procedureExecutor.getEnvironment(), hTableDescriptor, newRegions));
+      new CreateTableProcedure(procedureExecutor.getEnvironment(), tableDescriptor, newRegions));
 
     return procId;
   }
@@ -1778,7 +1776,7 @@ public class HMaster extends HRegionServer implements MasterServices {
    * values (compression, etc) work. Throws an exception if something is wrong.
    * @throws IOException
    */
-  private void sanityCheckTableDescriptor(final HTableDescriptor htd) throws IOException {
+  private void sanityCheckTableDescriptor(final TableDescriptor htd) throws IOException {
     final String CONF_KEY = "hbase.table.sanity.checks";
     boolean logWarn = false;
     if (!conf.getBoolean(CONF_KEY, true)) {
@@ -1848,7 +1846,7 @@ public class HMaster extends HRegionServer implements MasterServices {
       warnOrThrowExceptionForFailure(logWarn, CONF_KEY, message, null);
     }
 
-    for (HColumnDescriptor hcd : htd.getColumnFamilies()) {
+    for (ColumnFamilyDescriptor hcd : htd.getColumnFamilies()) {
       if (hcd.getTimeToLive() <= 0) {
         String message = "TTL for column family " + hcd.getNameAsString() + " must be positive.";
         warnOrThrowExceptionForFailure(logWarn, CONF_KEY, message, null);
@@ -1869,7 +1867,7 @@ public class HMaster extends HRegionServer implements MasterServices {
       }
       // max versions already being checked
 
-      // HBASE-13776 Setting illegal versions for HColumnDescriptor
+      // HBASE-13776 Setting illegal versions for ColumnFamilyDescriptor
       //  does not throw IllegalArgumentException
       // check minVersions <= maxVerions
       if (hcd.getMinVersions() > hcd.getMaxVersions()) {
@@ -1893,7 +1891,7 @@ public class HMaster extends HRegionServer implements MasterServices {
     }
   }
 
-  private void checkReplicationScope(HColumnDescriptor hcd) throws IOException{
+  private void checkReplicationScope(ColumnFamilyDescriptor hcd) throws IOException{
     // check replication scope
     WALProtos.ScopeType scop = WALProtos.ScopeType.valueOf(hcd.getScope());
     if (scop == null) {
@@ -1905,7 +1903,7 @@ public class HMaster extends HRegionServer implements MasterServices {
     }
   }
 
-  private void checkCompactionPolicy(Configuration conf, HTableDescriptor htd)
+  private void checkCompactionPolicy(Configuration conf, TableDescriptor htd)
       throws IOException {
     // FIFO compaction has some requirements
     // Actually FCP ignores periodic major compactions
@@ -1925,7 +1923,7 @@ public class HMaster extends HRegionServer implements MasterServices {
       blockingFileCount = conf.getInt(HStore.BLOCKING_STOREFILES_KEY, blockingFileCount);
     }
 
-    for (HColumnDescriptor hcd : htd.getColumnFamilies()) {
+    for (ColumnFamilyDescriptor hcd : htd.getColumnFamilies()) {
       String compactionPolicy =
           hcd.getConfigurationValue(DefaultStoreEngine.DEFAULT_COMPACTION_POLICY_CLASS_KEY);
       if (compactionPolicy == null) {
@@ -1938,7 +1936,7 @@ public class HMaster extends HRegionServer implements MasterServices {
       String message = null;
 
       // 1. Check TTL
-      if (hcd.getTimeToLive() == HColumnDescriptor.DEFAULT_TTL) {
+      if (hcd.getTimeToLive() == ColumnFamilyDescriptorBuilder.DEFAULT_TTL) {
         message = "Default TTL is not supported for FIFO compaction";
         throw new IOException(message);
       }
@@ -2040,36 +2038,36 @@ public class HMaster extends HRegionServer implements MasterServices {
     }, getServerName().toShortString() + ".masterManager"));
   }
 
-  private void checkCompression(final HTableDescriptor htd)
+  private void checkCompression(final TableDescriptor htd)
   throws IOException {
     if (!this.masterCheckCompression) return;
-    for (HColumnDescriptor hcd : htd.getColumnFamilies()) {
+    for (ColumnFamilyDescriptor hcd : htd.getColumnFamilies()) {
       checkCompression(hcd);
     }
   }
 
-  private void checkCompression(final HColumnDescriptor hcd)
+  private void checkCompression(final ColumnFamilyDescriptor hcd)
   throws IOException {
     if (!this.masterCheckCompression) return;
     CompressionTest.testCompression(hcd.getCompressionType());
     CompressionTest.testCompression(hcd.getCompactionCompressionType());
   }
 
-  private void checkEncryption(final Configuration conf, final HTableDescriptor htd)
+  private void checkEncryption(final Configuration conf, final TableDescriptor htd)
   throws IOException {
     if (!this.masterCheckEncryption) return;
-    for (HColumnDescriptor hcd : htd.getColumnFamilies()) {
+    for (ColumnFamilyDescriptor hcd : htd.getColumnFamilies()) {
       checkEncryption(conf, hcd);
     }
   }
 
-  private void checkEncryption(final Configuration conf, final HColumnDescriptor hcd)
+  private void checkEncryption(final Configuration conf, final ColumnFamilyDescriptor hcd)
   throws IOException {
     if (!this.masterCheckEncryption) return;
     EncryptionTest.testEncryption(conf, hcd.getEncryptionType(), hcd.getEncryptionKey());
   }
 
-  private void checkClassLoading(final Configuration conf, final HTableDescriptor htd)
+  private void checkClassLoading(final Configuration conf, final TableDescriptor htd)
   throws IOException {
     RegionSplitPolicy.getSplitPolicyClass(htd, conf);
     RegionCoprocessorHost.testTableCoprocessorAttrs(conf, htd);
@@ -2143,7 +2141,7 @@ public class HMaster extends HRegionServer implements MasterServices {
   @Override
   public long addColumn(
       final TableName tableName,
-      final HColumnDescriptor columnDescriptor,
+      final ColumnFamilyDescriptor columnDescriptor,
       final long nonceGroup,
       final long nonce)
       throws IOException {
@@ -2179,7 +2177,7 @@ public class HMaster extends HRegionServer implements MasterServices {
   @Override
   public long modifyColumn(
       final TableName tableName,
-      final HColumnDescriptor descriptor,
+      final ColumnFamilyDescriptor descriptor,
       final long nonceGroup,
       final long nonce)
       throws IOException {
@@ -2373,7 +2371,7 @@ public class HMaster extends HRegionServer implements MasterServices {
   }
 
   @Override
-  public long modifyTable(final TableName tableName, final HTableDescriptor descriptor,
+  public long modifyTable(final TableName tableName, final TableDescriptor descriptor,
       final long nonceGroup, final long nonce) throws IOException {
     checkInitialized();
     sanityCheckTableDescriptor(descriptor);
@@ -3127,7 +3125,7 @@ public class HMaster extends HRegionServer implements MasterServices {
   throws IOException {
     if (tableNameList == null || tableNameList.isEmpty()) {
       // request for all TableDescriptors
-      Collection<HTableDescriptor> allHtds;
+      Collection<TableDescriptor> allHtds;
       if (namespace != null && namespace.length() > 0) {
         // Do a check on the namespace existence. Will fail if does not exist.
         this.clusterSchemaService.getNamespace(namespace);
@@ -3135,7 +3133,7 @@ public class HMaster extends HRegionServer implements MasterServices {
       } else {
         allHtds = tableDescriptors.getAll().values();
       }
-      for (HTableDescriptor desc: allHtds) {
+      for (TableDescriptor desc: allHtds) {
         if (tableStateManager.isTablePresent(desc.getTableName())
             && (includeSysTables || !desc.getTableName().isSystemTable())) {
           htds.add(desc);
@@ -3144,7 +3142,7 @@ public class HMaster extends HRegionServer implements MasterServices {
     } else {
       for (TableName s: tableNameList) {
         if (tableStateManager.isTablePresent(s)) {
-          HTableDescriptor desc = tableDescriptors.get(s);
+          TableDescriptor desc = tableDescriptors.get(s);
           if (desc != null) {
             htds.add(desc);
           }
@@ -3249,7 +3247,7 @@ public class HMaster extends HRegionServer implements MasterServices {
    * @param allFiles Whether add all mob files into the compaction.
    */
   public void requestMobCompaction(TableName tableName,
-    List<HColumnDescriptor> columns, boolean allFiles) throws IOException {
+                                   List<ColumnFamilyDescriptor> columns, boolean allFiles) throws IOException {
     mobCompactThread.requestMobCompaction(conf, fs, tableName, columns, allFiles);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/25ff9d0b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java
index ee195cc..f9d47e0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java
@@ -28,13 +28,15 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hbase.ClusterId;
-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.backup.HFileArchiver;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+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.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.fs.HFileSystem;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureConstants;
@@ -387,10 +389,8 @@ public class MasterFileSystem {
       // not make it in first place.  Turn off block caching for bootstrap.
       // Enable after.
       HRegionInfo metaHRI = new HRegionInfo(HRegionInfo.FIRST_META_REGIONINFO);
-      HTableDescriptor metaDescriptor = new FSTableDescriptors(c).get(TableName.META_TABLE_NAME);
-      setInfoFamilyCachingForMeta(metaDescriptor, false);
-      HRegion meta = HRegion.createHRegion(metaHRI, rd, c, metaDescriptor, null);
-      setInfoFamilyCachingForMeta(metaDescriptor, true);
+      TableDescriptor metaDescriptor = new FSTableDescriptors(c).get(TableName.META_TABLE_NAME);
+      HRegion meta = HRegion.createHRegion(metaHRI, rd, c, setInfoFamilyCachingForMeta(metaDescriptor, false), null);
       meta.close();
     } catch (IOException e) {
         e = e instanceof RemoteException ?
@@ -403,13 +403,17 @@ public class MasterFileSystem {
   /**
    * Enable in memory caching for hbase:meta
    */
-  public static void setInfoFamilyCachingForMeta(HTableDescriptor metaDescriptor, final boolean b) {
-    for (HColumnDescriptor hcd: metaDescriptor.getColumnFamilies()) {
+  public static TableDescriptor setInfoFamilyCachingForMeta(TableDescriptor metaDescriptor, final boolean b) {
+    TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(metaDescriptor);
+    for (ColumnFamilyDescriptor hcd: metaDescriptor.getColumnFamilies()) {
       if (Bytes.equals(hcd.getName(), HConstants.CATALOG_FAMILY)) {
-        hcd.setBlockCacheEnabled(b);
-        hcd.setInMemory(b);
+        builder.modifyColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(hcd)
+                .setBlockCacheEnabled(b)
+                .setInMemory(b)
+                .build());
       }
     }
+    return builder.build();
   }
 
   public void deleteFamilyFromFS(HRegionInfo region, byte[] familyName)

http://git-wip-us.apache.org/repos/asf/hbase/blob/25ff9d0b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMobCompactionThread.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMobCompactionThread.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMobCompactionThread.java
index 2b1232a..d092efe 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMobCompactionThread.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMobCompactionThread.java
@@ -31,9 +31,9 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
-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.master.locking.LockManager;
 import org.apache.hadoop.hbase.master.locking.LockProcedure;
 import org.apache.hadoop.hbase.mob.MobUtils;
@@ -79,7 +79,7 @@ public class MasterMobCompactionThread {
    * @param allFiles Whether add all mob files into the compaction.
    */
   public void requestMobCompaction(Configuration conf, FileSystem fs, TableName tableName,
-      List<HColumnDescriptor> columns, boolean allFiles) throws IOException {
+                                   List<ColumnFamilyDescriptor> columns, boolean allFiles) throws IOException {
     master.reportMobCompactionStart(tableName);
     try {
       masterMobPool.execute(new CompactionRunner(fs, tableName, columns,
@@ -102,11 +102,11 @@ public class MasterMobCompactionThread {
   private class CompactionRunner implements Runnable {
     private FileSystem fs;
     private TableName tableName;
-    private List<HColumnDescriptor> hcds;
+    private List<ColumnFamilyDescriptor> hcds;
     private boolean allFiles;
     private ExecutorService pool;
 
-    public CompactionRunner(FileSystem fs, TableName tableName, List<HColumnDescriptor> hcds,
+    public CompactionRunner(FileSystem fs, TableName tableName, List<ColumnFamilyDescriptor> hcds,
       boolean allFiles, ExecutorService pool) {
       super();
       this.fs = fs;
@@ -123,7 +123,7 @@ public class MasterMobCompactionThread {
           MobUtils.getTableLockName(tableName), LockProcedure.LockType.EXCLUSIVE,
           this.getClass().getName() + ": mob compaction");
       try {
-        for (HColumnDescriptor hcd : hcds) {
+        for (ColumnFamilyDescriptor hcd : hcds) {
           MobUtils.doMobCompaction(conf, fs, tableName, hcd, pool, allFiles, lock);
         }
       } catch (IOException e) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/25ff9d0b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
index 6e9b1e2..3ec2c45 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
@@ -31,10 +31,8 @@ import java.util.Set;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
-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.MetaTableAccessor;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.ProcedureInfo;
@@ -43,6 +41,7 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.UnknownRegionException;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.client.MasterSwitchType;
 import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableState;
@@ -363,7 +362,7 @@ public class MasterRpcServices extends RSRpcServices
     try {
       long procId = master.addColumn(
           ProtobufUtil.toTableName(req.getTableName()),
-          ProtobufUtil.convertToHColumnDesc(req.getColumnFamilies()),
+          ProtobufUtil.toColumnFamilyDescriptor(req.getColumnFamilies()),
           req.getNonceGroup(),
           req.getNonce());
       if (procId == -1) {
@@ -439,11 +438,11 @@ public class MasterRpcServices extends RSRpcServices
   @Override
   public CreateTableResponse createTable(RpcController controller, CreateTableRequest req)
   throws ServiceException {
-    HTableDescriptor hTableDescriptor = ProtobufUtil.convertToHTableDesc(req.getTableSchema());
+    TableDescriptor tableDescriptor = ProtobufUtil.toTableDescriptor(req.getTableSchema());
     byte [][] splitKeys = ProtobufUtil.getSplitKeysArray(req);
     try {
       long procId =
-          master.createTable(hTableDescriptor, splitKeys, req.getNonceGroup(), req.getNonce());
+          master.createTable(tableDescriptor, splitKeys, req.getNonceGroup(), req.getNonce());
       return CreateTableResponse.newBuilder().setProcId(procId).build();
     } catch (IOException ioe) {
       throw new ServiceException(ioe);
@@ -865,7 +864,7 @@ public class MasterRpcServices extends RSRpcServices
       if (descriptors != null && descriptors.size() > 0) {
         // Add the table descriptors to the response
         for (TableDescriptor htd: descriptors) {
-          builder.addTableSchema(ProtobufUtil.convertToTableSchema(htd));
+          builder.addTableSchema(ProtobufUtil.toTableSchema(htd));
         }
       }
       return builder.build();
@@ -1118,7 +1117,7 @@ public class MasterRpcServices extends RSRpcServices
           ListTableDescriptorsByNamespaceResponse.newBuilder();
       for (TableDescriptor htd : master
           .listTableDescriptorsByNamespace(request.getNamespaceName())) {
-        b.addTableSchema(ProtobufUtil.convertToTableSchema(htd));
+        b.addTableSchema(ProtobufUtil.toTableSchema(htd));
       }
       return b.build();
     } catch (IOException e) {
@@ -1147,7 +1146,7 @@ public class MasterRpcServices extends RSRpcServices
     try {
       long procId = master.modifyColumn(
         ProtobufUtil.toTableName(req.getTableName()),
-        ProtobufUtil.convertToHColumnDesc(req.getColumnFamilies()),
+        ProtobufUtil.toColumnFamilyDescriptor(req.getColumnFamilies()),
         req.getNonceGroup(),
         req.getNonce());
       if (procId == -1) {
@@ -1181,7 +1180,7 @@ public class MasterRpcServices extends RSRpcServices
     try {
       long procId = master.modifyTable(
         ProtobufUtil.toTableName(req.getTableName()),
-        ProtobufUtil.convertToHTableDesc(req.getTableSchema()),
+        ProtobufUtil.toTableDescriptor(req.getTableSchema()),
         req.getNonceGroup(),
         req.getNonce());
       return ModifyTableResponse.newBuilder().setProcId(procId).build();
@@ -1532,12 +1531,12 @@ public class MasterRpcServices extends RSRpcServices
       throw new DoNotRetryIOException("Table " + tableName + " is not enabled");
     }
     boolean allFiles = false;
-    List<HColumnDescriptor> compactedColumns = new ArrayList<>();
-    HColumnDescriptor[] hcds = master.getTableDescriptors().get(tableName).getColumnFamilies();
+    List<ColumnFamilyDescriptor> compactedColumns = new ArrayList<>();
+    ColumnFamilyDescriptor[] hcds = master.getTableDescriptors().get(tableName).getColumnFamilies();
     byte[] family = null;
     if (request.hasFamily()) {
       family = request.getFamily().toByteArray();
-      for (HColumnDescriptor hcd : hcds) {
+      for (ColumnFamilyDescriptor hcd : hcds) {
         if (Bytes.equals(family, hcd.getName())) {
           if (!hcd.isMobEnabled()) {
             LOG.error("Column family " + hcd.getNameAsString() + " is not a mob column family");
@@ -1548,7 +1547,7 @@ public class MasterRpcServices extends RSRpcServices
         }
       }
     } else {
-      for (HColumnDescriptor hcd : hcds) {
+      for (ColumnFamilyDescriptor hcd : hcds) {
         if (hcd.isMobEnabled()) {
           compactedColumns.add(hcd);
         }

http://git-wip-us.apache.org/repos/asf/hbase/blob/25ff9d0b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
index 6e97bf4..cde9e34 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
@@ -21,9 +21,7 @@ package org.apache.hadoop.hbase.master;
 import java.io.IOException;
 import java.util.List;
 
-import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.ProcedureInfo;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
@@ -32,6 +30,7 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotDisabledException;
 import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.client.MasterSwitchType;
 import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.executor.ExecutorService;
@@ -159,17 +158,17 @@ public interface MasterServices extends Server {
    *     a single region is created.
    */
   long createTable(
-      final HTableDescriptor desc,
+      final TableDescriptor desc,
       final byte[][] splitKeys,
       final long nonceGroup,
       final long nonce) throws IOException;
 
   /**
    * Create a system table using the given table definition.
-   * @param hTableDescriptor The system table definition
+   * @param tableDescriptor The system table definition
    *     a single region is created.
    */
-  long createSystemTable(final HTableDescriptor hTableDescriptor) throws IOException;
+  long createSystemTable(final TableDescriptor tableDescriptor) throws IOException;
 
   /**
    * Delete a table
@@ -207,7 +206,7 @@ public interface MasterServices extends Server {
    */
   long modifyTable(
       final TableName tableName,
-      final HTableDescriptor descriptor,
+      final TableDescriptor descriptor,
       final long nonceGroup,
       final long nonce)
       throws IOException;
@@ -247,7 +246,7 @@ public interface MasterServices extends Server {
    */
   long addColumn(
       final TableName tableName,
-      final HColumnDescriptor column,
+      final ColumnFamilyDescriptor column,
       final long nonceGroup,
       final long nonce)
       throws IOException;
@@ -262,7 +261,7 @@ public interface MasterServices extends Server {
    */
   long modifyColumn(
       final TableName tableName,
-      final HColumnDescriptor descriptor,
+      final ColumnFamilyDescriptor descriptor,
       final long nonceGroup,
       final long nonce)
       throws IOException;