You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by zh...@apache.org on 2020/08/04 02:33:05 UTC

[hbase] branch master updated: HBAE-24507 Remove HTableDescriptor and HColumnDescriptor (#2186)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new d2f5a5f  HBAE-24507 Remove HTableDescriptor and HColumnDescriptor (#2186)
d2f5a5f is described below

commit d2f5a5f27bed9753d5dbb2dbf30e1b1aa1aba8ca
Author: Duo Zhang <zh...@apache.org>
AuthorDate: Tue Aug 4 10:31:42 2020 +0800

    HBAE-24507 Remove HTableDescriptor and HColumnDescriptor (#2186)
    
    Signed-off-by: stack <st...@apache.org>
    Signed-off-by: Viraj Jasani <vj...@apache.org>
    Signed-off-by: tedyu <yu...@gmail.com>
---
 .../exemplars/shaded_client/HelloHBase.java        |   6 +-
 .../apache/hadoop/hbase/backup/TestBackupBase.java |  15 +-
 .../hadoop/hbase/backup/TestIncrementalBackup.java |  26 +-
 .../backup/TestIncrementalBackupWithFailures.java  |  10 +-
 .../hadoop/hbase/backup/TestRemoteBackup.java      |  21 +-
 .../org/apache/hadoop/hbase/HColumnDescriptor.java | 792 ----------------
 .../org/apache/hadoop/hbase/HTableDescriptor.java  | 992 ---------------------
 .../client/ColumnFamilyDescriptorBuilder.java      |  13 +-
 .../hbase/client/TableDescriptorBuilder.java       |  32 +-
 .../apache/hadoop/hbase/TestHColumnDescriptor.java | 148 ---
 .../apache/hadoop/hbase/TestHTableDescriptor.java  | 369 --------
 .../client/TestColumnFamilyDescriptorBuilder.java  |  64 +-
 .../hbase/client/TestTableDescriptorBuilder.java   | 170 ++--
 .../coprocessor/TestBatchCoprocessorEndpoint.java  |   9 +-
 .../hbase/coprocessor/TestCoprocessorEndpoint.java |   9 +-
 .../coprocessor/TestCoprocessorTableEndpoint.java  |  32 +-
 .../IntegrationTestIngestStripeCompactions.java    |  16 +-
 .../hadoop/hbase/IntegrationTestIngestWithMOB.java |   8 +-
 .../hadoop/hbase/IntegrationTestLazyCfLoading.java |  13 +-
 .../hadoop/hbase/IntegrationTestMobCompaction.java |  20 +-
 .../StripeCompactionsPerformanceEvaluation.java    |  38 +-
 .../hbase/test/IntegrationTestBigLinkedList.java   |  42 +-
 ...IntegrationTestBigLinkedListWithVisibility.java |  12 +-
 .../hbase/test/IntegrationTestLoadAndVerify.java   |  14 +-
 ...grationTestWithCellVisibilityLoadAndVerify.java |   6 +-
 .../hadoop/hbase/mapred/TableRecordReader.java     |   2 +-
 .../hadoop/hbase/mapred/TableRecordReaderImpl.java |   6 +-
 .../apache/hadoop/hbase/mapreduce/ImportTsv.java   |  25 +-
 .../hbase/mapreduce/TableRecordReaderImpl.java     |   6 +-
 .../apache/hadoop/hbase/PerformanceEvaluation.java |  30 +-
 .../hbase/mapreduce/TestTimeRangeMapRed.java       |  12 +-
 .../apache/hadoop/hbase/rest/SchemaResource.java   |   5 +-
 .../hadoop/hbase/rest/model/ColumnSchemaModel.java |  42 +-
 .../hadoop/hbase/rest/model/TableSchemaModel.java  |  30 +-
 .../hadoop/hbase/rest/PerformanceEvaluation.java   |  20 +-
 .../hadoop/hbase/rest/TestScannersWithFilters.java |  10 +-
 .../hbase/rest/client/TestRemoteAdminRetries.java  |  10 +-
 .../hadoop/hbase/rest/client/TestRemoteTable.java  |  16 +-
 .../hbase/tmpl/master/MasterStatusTmpl.jamon       |   6 +-
 .../org/apache/hadoop/hbase/LocalHBaseCluster.java |  31 +-
 .../hadoop/hbase/constraint/Constraints.java       |   1 -
 .../hadoop/hbase/io/hfile/LruBlockCache.java       |  73 +-
 .../org/apache/hadoop/hbase/quotas/QuotaUtil.java  |  27 +-
 .../security/visibility/VisibilityController.java  |  22 +-
 .../resources/hbase-webapps/master/rsgroup.jsp     |  12 +-
 .../main/resources/hbase-webapps/master/table.jsp  |   2 +-
 .../apache/hadoop/hbase/HBaseTestingUtility.java   | 319 +++----
 ...TestColumnFamilyDescriptorDefaultVersions.java} |  57 +-
 .../org/apache/hadoop/hbase/TestMultiVersions.java |  43 +-
 .../apache/hadoop/hbase/TestRegionRebalancing.java |   9 +-
 .../org/apache/hadoop/hbase/TestSerialization.java |  31 +-
 .../org/apache/hadoop/hbase/client/TestAdmin2.java |  87 +-
 .../hbase/client/TestAsyncTableAdminApi3.java      |   6 +-
 .../hbase/client/TestBlockEvictionFromClient.java  |   9 +-
 .../hadoop/hbase/client/TestEnableTable.java       |  24 +-
 .../hadoop/hbase/client/TestFromClientSide.java    |  25 +-
 .../hadoop/hbase/client/TestFromClientSide3.java   |  58 +-
 .../hadoop/hbase/client/TestFromClientSide4.java   |  19 +-
 .../hadoop/hbase/client/TestFromClientSide5.java   |  19 +-
 .../hbase/client/TestIllegalTableDescriptor.java   | 142 ++-
 .../hbase/client/TestIncrementsFromClientSide.java |  12 +-
 .../hbase/client/TestIntraRowPagination.java       |  11 +-
 .../hbase/client/TestReplicaWithCluster.java       |  89 +-
 .../hadoop/hbase/client/TestReplicasClient.java    |  11 +-
 .../hbase/client/TestScannersFromClientSide.java   |   3 +-
 .../hbase/client/TestServerBusyException.java      |   6 +-
 .../hadoop/hbase/client/TestSizeFailures.java      |  10 +-
 .../client/TestSnapshotCloneIndependence.java      |  18 +-
 .../hbase/client/TestSnapshotFromClient.java       |  11 +-
 .../hadoop/hbase/client/TestSnapshotMetadata.java  |  59 +-
 .../client/TestSnapshotTemporaryDirectory.java     |   7 +-
 .../coprocessor/TestCoprocessorConfiguration.java  |   8 +-
 .../coprocessor/TestCoprocessorInterface.java      |  13 +-
 .../hbase/coprocessor/TestCoprocessorMetrics.java  |  73 +-
 .../TestMasterCoprocessorExceptionWithAbort.java   |   9 +-
 .../TestMasterCoprocessorExceptionWithRemove.java  |  15 +-
 .../hbase/coprocessor/TestMasterObserver.java      |  30 +-
 .../coprocessor/TestOpenTableInCoprocessor.java    |  20 +-
 ...ObserverForAddingMutationsFromCoprocessors.java |  10 +-
 .../coprocessor/TestRegionObserverInterface.java   |   9 +-
 .../TestRegionObserverScannerOpenHook.java         |  37 +-
 .../coprocessor/TestRegionObserverStacking.java    |  15 +-
 .../hadoop/hbase/filter/FilterTestingCluster.java  |  10 +-
 .../hbase/filter/TestDependentColumnFilter.java    |  20 +-
 .../org/apache/hadoop/hbase/filter/TestFilter.java |  45 +-
 .../hbase/filter/TestFilterFromRegionSide.java     |  11 +-
 .../hadoop/hbase/filter/TestFilterWrapper.java     |  16 +-
 .../hbase/filter/TestInvocationRecordFilter.java   |  10 +-
 .../io/encoding/TestSeekBeforeWithReverseScan.java |  10 +-
 .../hfile/TestScannerSelectionUsingKeyRange.java   |  12 +-
 .../org/apache/hadoop/hbase/master/TestMaster.java |   8 +-
 .../hbase/master/TestMasterMetricsWrapper.java     |   8 +-
 .../hadoop/hbase/master/TestRegionPlacement.java   |   8 +-
 .../hbase/master/TestRegionPlansWithThrottle.java  |   6 +-
 .../balancer/TestFavoredNodeTableImport.java       |  13 +-
 .../TestFavoredStochasticLoadBalancer.java         |  80 +-
 .../TestDeleteColumnFamilyProcedureFromClient.java |  16 +-
 .../master/procedure/TestModifyTableProcedure.java |  83 +-
 .../procedure/TestRestoreSnapshotProcedure.java    |  55 +-
 .../TestTableDescriptorModificationFromClient.java |  94 +-
 .../hadoop/hbase/mob/MobStressToolRunner.java      |  21 +-
 .../hbase/mob/TestDefaultMobStoreFlusher.java      |  83 +-
 .../hadoop/hbase/mob/TestMobCompactionOptMode.java |   7 +-
 .../mob/TestMobCompactionOptRegionBatchMode.java   |   1 -
 .../TestMobCompactionRegularRegionBatchMode.java   |   1 -
 .../hbase/mob/TestMobCompactionWithDefaults.java   |  17 +-
 .../hadoop/hbase/mob/TestMobDataBlockEncoding.java |  20 +-
 .../hadoop/hbase/mob/TestMobFileCleanerChore.java  |  19 +-
 .../hadoop/hbase/mob/TestMobStoreCompaction.java   |  15 +-
 .../hadoop/hbase/mob/TestMobStoreScanner.java      |  19 +-
 .../hbase/namespace/TestNamespaceAuditor.java      |  13 +-
 .../hbase/regionserver/TestAtomicOperation.java    |  23 +-
 .../hbase/regionserver/TestBlocksScanned.java      |  37 +-
 .../hadoop/hbase/regionserver/TestBulkLoad.java    |   9 +-
 .../hbase/regionserver/TestColumnSeeking.java      |  13 +-
 .../hbase/regionserver/TestCompactSplitThread.java |  14 +-
 .../hbase/regionserver/TestCompactingMemStore.java |  11 +-
 .../hadoop/hbase/regionserver/TestCompaction.java  |  19 +-
 .../hbase/regionserver/TestCompactionPolicy.java   |  13 +-
 .../hbase/regionserver/TestDeleteMobTable.java     |  23 +-
 .../hbase/regionserver/TestFSErrorsExposed.java    |  11 +-
 .../hadoop/hbase/regionserver/TestHRegion.java     | 102 +--
 .../hbase/regionserver/TestHRegionOnCluster.java   |   7 +-
 .../hbase/regionserver/TestIsDeleteFailure.java    |   6 +-
 .../hbase/regionserver/TestJoinedScanners.java     |  15 +-
 .../hadoop/hbase/regionserver/TestKeepDeletes.java |  29 +-
 .../hbase/regionserver/TestMajorCompaction.java    |  10 +-
 .../hbase/regionserver/TestMinorCompaction.java    |  13 +-
 .../hbase/regionserver/TestMutateRowsRecovery.java |   7 +-
 .../TestNewVersionBehaviorFromClientSide.java      |  11 +-
 .../hadoop/hbase/regionserver/TestParallelPut.java |  10 +-
 .../regionserver/TestPerColumnFamilyFlush.java     |  40 +-
 .../hbase/regionserver/TestRSStatusServlet.java    |  15 +-
 .../hadoop/hbase/regionserver/TestRegionOpen.java  |  16 +-
 .../regionserver/TestRegionReplicaFailover.java    |  30 +-
 .../TestRegionReplicasAreDistributed.java          |   7 +-
 .../TestRegionReplicasWithModifyTable.java         |   8 +-
 .../hbase/regionserver/TestResettingCounters.java  |  11 +-
 .../hbase/regionserver/TestReversibleScanners.java |  10 +-
 .../hadoop/hbase/regionserver/TestRowTooBig.java   |  27 +-
 .../regionserver/TestSCVFWithMiniCluster.java      |  16 +-
 .../hadoop/hbase/regionserver/TestScanner.java     |  23 +-
 .../regionserver/TestScannerWithCorruptHFile.java  |   9 +-
 .../TestSettingTimeoutOnBlockingPoint.java         |  11 +-
 .../TestSplitTransactionOnCluster.java             |  12 +-
 .../regionserver/TestStoreScannerClosure.java      |   7 +-
 .../apache/hadoop/hbase/regionserver/TestTags.java |  42 +-
 .../TestWalAndCompactingMemStoreFlush.java         |  18 +-
 .../compactions/TestCompactedHFilesDischarger.java |   7 +-
 .../compactions/TestDateTieredCompactor.java       |   4 +-
 .../compactions/TestStripeCompactionPolicy.java    |   5 +-
 .../compactions/TestStripeCompactor.java           |   4 +-
 .../regionserver/wal/AbstractTestWALReplay.java    | 355 +++-----
 .../replication/TestMultiSlaveReplication.java     |  15 +-
 .../replication/TestPerTableCFReplication.java     |  83 +-
 .../hbase/replication/TestReplicationWithTags.java |  14 +-
 .../TestGlobalReplicationThrottler.java            |  10 +-
 .../TestRegionReplicaReplicationEndpoint.java      |  73 +-
 ...stRegionReplicaReplicationEndpointNoMaster.java |  10 +-
 .../security/access/TestAccessController.java      |  84 +-
 .../security/access/TestAccessController2.java     |  22 +-
 .../security/access/TestAccessController3.java     |  22 +-
 .../access/TestCellACLWithMultipleVersions.java    |  20 +-
 .../hadoop/hbase/security/access/TestCellACLs.java |  14 +-
 .../TestCoprocessorWhitelistMasterObserver.java    |  48 +-
 .../security/access/TestNamespaceCommands.java     |   8 +-
 .../security/access/TestScanEarlyTermination.java  |  26 +-
 .../access/TestWithDisabledAuthorization.java      |  38 +-
 .../token/TestDelegationTokenWithEncryption.java   |   8 +-
 ...tVisibilityLabelReplicationWithExpAsString.java |  10 +-
 .../security/visibility/TestVisibilityLabels.java  |  35 +-
 .../TestVisibilityLabelsReplication.java           |  10 +-
 .../visibility/TestVisibilityWithCheckAuths.java   |  10 +-
 .../hadoop/hbase/util/BaseTestHBaseFsck.java       |   5 +-
 .../apache/hadoop/hbase/util/HFileTestUtil.java    |   5 +-
 .../hbase/util/TestMiniClusterLoadSequential.java  |   9 +-
 .../hbase/thrift/ThriftHBaseServiceHandler.java    |  21 +-
 .../hadoop/hbase/thrift/ThriftUtilities.java       |  30 +-
 .../hadoop/hbase/thrift2/ThriftUtilities.java      |  18 +-
 .../thrift2/TestThriftHBaseServiceHandler.java     |  36 +-
 .../TestThriftHBaseServiceHandlerWithLabels.java   |  19 +-
 .../TestThriftHBaseServiceHandlerWithReadOnly.java |  19 +-
 182 files changed, 1960 insertions(+), 5132 deletions(-)

diff --git a/hbase-archetypes/hbase-shaded-client-project/src/main/java/org/apache/hbase/archetypes/exemplars/shaded_client/HelloHBase.java b/hbase-archetypes/hbase-shaded-client-project/src/main/java/org/apache/hbase/archetypes/exemplars/shaded_client/HelloHBase.java
index 4179955..053275a 100644
--- a/hbase-archetypes/hbase-shaded-client-project/src/main/java/org/apache/hbase/archetypes/exemplars/shaded_client/HelloHBase.java
+++ b/hbase-archetypes/hbase-shaded-client-project/src/main/java/org/apache/hbase/archetypes/exemplars/shaded_client/HelloHBase.java
@@ -110,10 +110,8 @@ public final class HelloHBase {
               + "], with one Column Family ["
               + Bytes.toString(MY_COLUMN_FAMILY_NAME) + "].");
 
-      admin.createTable(new TableDescriptorBuilder.ModifyableTableDescriptor(MY_TABLE_NAME)
-        .setColumnFamily(
-          new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(
-            MY_COLUMN_FAMILY_NAME)));
+      admin.createTable(TableDescriptorBuilder.newBuilder(MY_TABLE_NAME)
+        .setColumnFamily(ColumnFamilyDescriptorBuilder.of(MY_COLUMN_FAMILY_NAME)).build());
     }
   }
 
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java
index 94a8bde..12060dc 100644
--- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java
+++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestBackupBase.java
@@ -47,13 +47,13 @@ import org.apache.hadoop.hbase.backup.impl.IncrementalTableBackupClient;
 import org.apache.hadoop.hbase.backup.master.LogRollMasterProcedureManager;
 import org.apache.hadoop.hbase.backup.util.BackupUtils;
 import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.Durability;
 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.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.master.cleaner.LogCleaner;
 import org.apache.hadoop.hbase.master.cleaner.TimeToLiveLogCleaner;
@@ -84,7 +84,7 @@ public class TestBackupBase {
   protected static Configuration conf2;
 
   protected static TableName table1 = TableName.valueOf("table1");
-  protected static TableDescriptorBuilder.ModifyableTableDescriptor table1Desc;
+  protected static TableDescriptor table1Desc;
   protected static TableName table2 = TableName.valueOf("table2");
   protected static TableName table3 = TableName.valueOf("table3");
   protected static TableName table4 = TableName.valueOf("table4");
@@ -428,11 +428,8 @@ public class TestBackupBase {
     ha.createNamespace(desc3);
     ha.createNamespace(desc4);
 
-    TableDescriptorBuilder.ModifyableTableDescriptor desc =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(table1);
-    ColumnFamilyDescriptor familyDescriptor =
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(famName);
-    desc.setColumnFamily(familyDescriptor);
+    TableDescriptor desc = TableDescriptorBuilder.newBuilder(table1)
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(famName)).build();
     ha.createTable(desc);
     table1Desc = desc;
     Connection conn = ConnectionFactory.createConnection(conf1);
@@ -440,8 +437,8 @@ public class TestBackupBase {
     loadTable(table);
     table.close();
     table2 = TableName.valueOf("ns2:test-" + tid + 1);
-    desc = new TableDescriptorBuilder.ModifyableTableDescriptor(table2);
-    desc.setColumnFamily(familyDescriptor);
+    desc = TableDescriptorBuilder.newBuilder(table2)
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(famName)).build();
     ha.createTable(desc);
     table = conn.getTable(table2);
     loadTable(table);
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackup.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackup.java
index b53c3e6..5ed4025 100644
--- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackup.java
+++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackup.java
@@ -34,6 +34,8 @@ 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.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -82,14 +84,11 @@ public class TestIncrementalBackup extends TestBackupBase {
     final byte[] fam3Name = Bytes.toBytes("f3");
     final byte[] mobName = Bytes.toBytes("mob");
 
-    table1Desc.setColumnFamily(
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(fam3Name));
-    ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor mobHcd =
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(mobName);
-    mobHcd.setMobEnabled(true);
-    mobHcd.setMobThreshold(5L);
-    table1Desc.setColumnFamily(mobHcd);
-    HBaseTestingUtility.modifyTableSync(TEST_UTIL.getAdmin(), table1Desc);
+    TableDescriptor newTable1Desc = TableDescriptorBuilder.newBuilder(table1Desc)
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(fam3Name))
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(mobName).setMobEnabled(true)
+        .setMobThreshold(5L).build()).build();
+    TEST_UTIL.getAdmin().modifyTable(newTable1Desc);
 
     try (Connection conn = ConnectionFactory.createConnection(conf1)) {
       int NB_ROWS_FAM3 = 6;
@@ -150,13 +149,12 @@ public class TestIncrementalBackup extends TestBackupBase {
       assertTrue(checkSucceeded(backupIdIncMultiple));
 
       // add column family f2 to table1
-      final byte[] fam2Name = Bytes.toBytes("f2");
-      table1Desc.setColumnFamily(
-        new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(fam2Name));
-
       // drop column family f3
-      table1Desc.removeColumnFamily(fam3Name);
-      HBaseTestingUtility.modifyTableSync(TEST_UTIL.getAdmin(), table1Desc);
+      final byte[] fam2Name = Bytes.toBytes("f2");
+      newTable1Desc = TableDescriptorBuilder.newBuilder(newTable1Desc)
+        .setColumnFamily(ColumnFamilyDescriptorBuilder.of(fam2Name)).removeColumnFamily(fam3Name)
+        .build();
+      TEST_UTIL.getAdmin().modifyTable(newTable1Desc);
 
       int NB_ROWS_FAM2 = 7;
       Table t3 = insertIntoTable(conn, table1, fam2Name, 2, NB_ROWS_FAM2);
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithFailures.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithFailures.java
index aeb1edb..00b13ba 100644
--- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithFailures.java
+++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupWithFailures.java
@@ -24,7 +24,6 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.backup.BackupInfo.BackupState;
 import org.apache.hadoop.hbase.backup.impl.BackupAdminImpl;
@@ -37,6 +36,8 @@ 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.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.util.ToolRunner;
@@ -76,16 +77,15 @@ public class TestIncrementalBackupWithFailures extends TestBackupBase {
   // implement all test cases in 1 test since incremental backup/restore has dependencies
   @Test
   public void testIncBackupRestore() throws Exception {
-
     int ADD_ROWS = 99;
     // #1 - create full backup for all tables
     LOG.info("create full backup image for all tables");
 
     List<TableName> tables = Lists.newArrayList(table1, table2);
     final byte[] fam3Name = Bytes.toBytes("f3");
-    table1Desc.setColumnFamily(
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(fam3Name));
-    HBaseTestingUtility.modifyTableSync(TEST_UTIL.getAdmin(), table1Desc);
+    TableDescriptor newTable1Desc = TableDescriptorBuilder.newBuilder(table1Desc)
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(fam3Name)).build();
+    TEST_UTIL.getAdmin().modifyTable(newTable1Desc);
 
     Connection conn = ConnectionFactory.createConnection(conf1);
     int NB_ROWS_FAM3 = 6;
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteBackup.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteBackup.java
index 3ef37c4..136c4b2 100644
--- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteBackup.java
+++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestRemoteBackup.java
@@ -21,7 +21,6 @@ import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
 import java.util.concurrent.CountDownLatch;
-
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
@@ -32,11 +31,12 @@ 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.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.snapshot.MobSnapshotTestingUtils;
 import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
 import org.junit.Assert;
 import org.junit.ClassRule;
 import org.junit.Test;
@@ -44,6 +44,8 @@ import org.junit.experimental.categories.Category;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
+
 @Category(LargeTests.class)
 public class TestRemoteBackup extends TestBackupBase {
 
@@ -93,16 +95,13 @@ public class TestRemoteBackup extends TestBackupBase {
       }
     });
     t.start();
-
-    table1Desc.setColumnFamily(
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(fam3Name));
     // family 2 is MOB enabled
-    ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(fam2Name);
-    familyDescriptor.setMobEnabled(true);
-    familyDescriptor.setMobThreshold(0L);
-    table1Desc.setColumnFamily(familyDescriptor);
-    TEST_UTIL.getAdmin().modifyTable(table1Desc);
+    TableDescriptor newTable1Desc = TableDescriptorBuilder.newBuilder(table1Desc)
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(fam3Name))
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(fam2Name).setMobEnabled(true)
+        .setMobThreshold(0L).build())
+      .build();
+    TEST_UTIL.getAdmin().modifyTable(newTable1Desc);
 
     SnapshotTestingUtils.loadData(TEST_UTIL, table1, 50, fam2Name);
     Table t1 = conn.getTable(table1);
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
deleted file mode 100644
index 4b94929..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java
+++ /dev/null
@@ -1,792 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase;
-
-import java.util.Map;
-
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
-import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
-import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor;
-import org.apache.hadoop.hbase.client.MobCompactPartitionPolicy;
-import org.apache.hadoop.hbase.exceptions.DeserializationException;
-import org.apache.hadoop.hbase.exceptions.HBaseException;
-import org.apache.hadoop.hbase.io.compress.Compression;
-import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
-import org.apache.hadoop.hbase.regionserver.BloomType;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.PrettyPrinter.Unit;
-
-/**
- * An HColumnDescriptor contains information about a column family such as the
- * number of versions, compression settings, etc.
- *
- * It is used as input when creating a table or adding a column.
- */
-@InterfaceAudience.Public
-@Deprecated // remove it in 3.0
-public class HColumnDescriptor implements ColumnFamilyDescriptor, Comparable<HColumnDescriptor> {
-  public static final String IN_MEMORY_COMPACTION = ColumnFamilyDescriptorBuilder.IN_MEMORY_COMPACTION;
-  public static final String COMPRESSION = ColumnFamilyDescriptorBuilder.COMPRESSION;
-  public static final String COMPRESSION_COMPACT = ColumnFamilyDescriptorBuilder.COMPRESSION_COMPACT;
-  public static final String ENCODE_ON_DISK = "ENCODE_ON_DISK";
-  public static final String DATA_BLOCK_ENCODING = ColumnFamilyDescriptorBuilder.DATA_BLOCK_ENCODING;
-  public static final String BLOCKCACHE = ColumnFamilyDescriptorBuilder.BLOCKCACHE;
-  public static final String CACHE_DATA_ON_WRITE = ColumnFamilyDescriptorBuilder.CACHE_DATA_ON_WRITE;
-  public static final String CACHE_INDEX_ON_WRITE = ColumnFamilyDescriptorBuilder.CACHE_INDEX_ON_WRITE;
-  public static final String CACHE_BLOOMS_ON_WRITE = ColumnFamilyDescriptorBuilder.CACHE_BLOOMS_ON_WRITE;
-  public static final String EVICT_BLOCKS_ON_CLOSE = ColumnFamilyDescriptorBuilder.EVICT_BLOCKS_ON_CLOSE;
-  public static final String CACHE_DATA_IN_L1 = "CACHE_DATA_IN_L1";
-  public static final String PREFETCH_BLOCKS_ON_OPEN = ColumnFamilyDescriptorBuilder.PREFETCH_BLOCKS_ON_OPEN;
-  public static final String BLOCKSIZE = ColumnFamilyDescriptorBuilder.BLOCKSIZE;
-  public static final String LENGTH = "LENGTH";
-  public static final String TTL = ColumnFamilyDescriptorBuilder.TTL;
-  public static final String BLOOMFILTER = ColumnFamilyDescriptorBuilder.BLOOMFILTER;
-  public static final String FOREVER = "FOREVER";
-  public static final String REPLICATION_SCOPE = ColumnFamilyDescriptorBuilder.REPLICATION_SCOPE;
-  public static final byte[] REPLICATION_SCOPE_BYTES = Bytes.toBytes(REPLICATION_SCOPE);
-  public static final String MIN_VERSIONS = ColumnFamilyDescriptorBuilder.MIN_VERSIONS;
-  public static final String KEEP_DELETED_CELLS = ColumnFamilyDescriptorBuilder.KEEP_DELETED_CELLS;
-  public static final String COMPRESS_TAGS = ColumnFamilyDescriptorBuilder.COMPRESS_TAGS;
-  public static final String ENCRYPTION = ColumnFamilyDescriptorBuilder.ENCRYPTION;
-  public static final String ENCRYPTION_KEY = ColumnFamilyDescriptorBuilder.ENCRYPTION_KEY;
-  public static final String IS_MOB = ColumnFamilyDescriptorBuilder.IS_MOB;
-  public static final byte[] IS_MOB_BYTES = Bytes.toBytes(IS_MOB);
-  public static final String MOB_THRESHOLD = ColumnFamilyDescriptorBuilder.MOB_THRESHOLD;
-  public static final byte[] MOB_THRESHOLD_BYTES = Bytes.toBytes(MOB_THRESHOLD);
-  public static final long DEFAULT_MOB_THRESHOLD = ColumnFamilyDescriptorBuilder.DEFAULT_MOB_THRESHOLD;
-  public static final String MOB_COMPACT_PARTITION_POLICY = ColumnFamilyDescriptorBuilder.MOB_COMPACT_PARTITION_POLICY;
-  public static final byte[] MOB_COMPACT_PARTITION_POLICY_BYTES = Bytes.toBytes(MOB_COMPACT_PARTITION_POLICY);
-  public static final MobCompactPartitionPolicy DEFAULT_MOB_COMPACT_PARTITION_POLICY
-        = ColumnFamilyDescriptorBuilder.DEFAULT_MOB_COMPACT_PARTITION_POLICY;
-  public static final String DFS_REPLICATION = ColumnFamilyDescriptorBuilder.DFS_REPLICATION;
-  public static final short DEFAULT_DFS_REPLICATION = ColumnFamilyDescriptorBuilder.DEFAULT_DFS_REPLICATION;
-  public static final String STORAGE_POLICY = ColumnFamilyDescriptorBuilder.STORAGE_POLICY;
-  public static final String DEFAULT_COMPRESSION = ColumnFamilyDescriptorBuilder.DEFAULT_COMPRESSION.name();
-  public static final boolean DEFAULT_ENCODE_ON_DISK = true;
-  public static final String DEFAULT_DATA_BLOCK_ENCODING = ColumnFamilyDescriptorBuilder.DEFAULT_DATA_BLOCK_ENCODING.name();
-  public static final int DEFAULT_VERSIONS = ColumnFamilyDescriptorBuilder.DEFAULT_MAX_VERSIONS;
-  public static final int DEFAULT_MIN_VERSIONS = ColumnFamilyDescriptorBuilder.DEFAULT_MIN_VERSIONS;
-  public static final boolean DEFAULT_IN_MEMORY = ColumnFamilyDescriptorBuilder.DEFAULT_IN_MEMORY;
-  public static final KeepDeletedCells DEFAULT_KEEP_DELETED = ColumnFamilyDescriptorBuilder.DEFAULT_KEEP_DELETED;
-  public static final boolean DEFAULT_BLOCKCACHE = ColumnFamilyDescriptorBuilder.DEFAULT_BLOCKCACHE;
-  public static final boolean DEFAULT_CACHE_DATA_ON_WRITE = ColumnFamilyDescriptorBuilder.DEFAULT_CACHE_DATA_ON_WRITE;
-  public static final boolean DEFAULT_CACHE_DATA_IN_L1 = false;
-  public static final boolean DEFAULT_CACHE_INDEX_ON_WRITE = ColumnFamilyDescriptorBuilder.DEFAULT_CACHE_INDEX_ON_WRITE;
-  public static final int DEFAULT_BLOCKSIZE = ColumnFamilyDescriptorBuilder.DEFAULT_BLOCKSIZE;
-  public static final String DEFAULT_BLOOMFILTER =  ColumnFamilyDescriptorBuilder.DEFAULT_BLOOMFILTER.name();
-  public static final boolean DEFAULT_CACHE_BLOOMS_ON_WRITE = ColumnFamilyDescriptorBuilder.DEFAULT_CACHE_BLOOMS_ON_WRITE;
-  public static final int DEFAULT_TTL = ColumnFamilyDescriptorBuilder.DEFAULT_TTL;
-  public static final int DEFAULT_REPLICATION_SCOPE = ColumnFamilyDescriptorBuilder.DEFAULT_REPLICATION_SCOPE;
-  public static final boolean DEFAULT_EVICT_BLOCKS_ON_CLOSE = ColumnFamilyDescriptorBuilder.DEFAULT_EVICT_BLOCKS_ON_CLOSE;
-  public static final boolean DEFAULT_COMPRESS_TAGS = ColumnFamilyDescriptorBuilder.DEFAULT_COMPRESS_TAGS;
-  public static final boolean DEFAULT_PREFETCH_BLOCKS_ON_OPEN = ColumnFamilyDescriptorBuilder.DEFAULT_PREFETCH_BLOCKS_ON_OPEN;
-  public static final String NEW_VERSION_BEHAVIOR = ColumnFamilyDescriptorBuilder.NEW_VERSION_BEHAVIOR;
-  public static final boolean DEFAULT_NEW_VERSION_BEHAVIOR = ColumnFamilyDescriptorBuilder.DEFAULT_NEW_VERSION_BEHAVIOR;
-  protected final ModifyableColumnFamilyDescriptor delegatee;
-
-  /**
-   * Construct a column descriptor specifying only the family name
-   * The other attributes are defaulted.
-   *
-   * @param familyName Column family name. Must be 'printable' -- digit or
-   *   letter -- and may not contain a <code>:</code>
-   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
-   *             (<a href="https://issues.apache.org/jira/browse/HBASE-18433">HBASE-18433</a>).
-   *             Use {@link ColumnFamilyDescriptorBuilder#of(String)}.
-   */
-  @Deprecated
-  public HColumnDescriptor(final String familyName) {
-    this(new ModifyableColumnFamilyDescriptor(Bytes.toBytes(familyName)));
-  }
-
-  /**
-   * Construct a column descriptor specifying only the family name
-   * The other attributes are defaulted.
-   *
-   * @param familyName Column family name. Must be 'printable' -- digit or
-   * letter -- and may not contain a <code>:</code>
-   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
-   *             (<a href="https://issues.apache.org/jira/browse/HBASE-18433">HBASE-18433</a>).
-   *             Use {@link ColumnFamilyDescriptorBuilder#of(byte[])}.
-   */
-  @Deprecated
-  public HColumnDescriptor(final byte [] familyName) {
-    this(new ModifyableColumnFamilyDescriptor(familyName));
-  }
-
-  /**
-   * Constructor.
-   * Makes a deep copy of the supplied descriptor.
-   * Can make a modifiable descriptor from an UnmodifyableHColumnDescriptor.
-   *
-   * @param desc The descriptor.
-   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
-   *             (<a href="https://issues.apache.org/jira/browse/HBASE-18433">HBASE-18433</a>).
-   *             Use {@link ColumnFamilyDescriptorBuilder#copy(ColumnFamilyDescriptor)}.
-   */
-  @Deprecated
-  public HColumnDescriptor(HColumnDescriptor desc) {
-    this(desc, true);
-  }
-
-  protected HColumnDescriptor(HColumnDescriptor desc, boolean deepClone) {
-    this(deepClone ? new ModifyableColumnFamilyDescriptor(desc)
-            : desc.delegatee);
-  }
-
-  protected HColumnDescriptor(ModifyableColumnFamilyDescriptor delegate) {
-    this.delegatee = delegate;
-  }
-
-  /**
-   * @return Name of this column family
-   */
-  @Override
-  public byte [] getName() {
-    return delegatee.getName();
-  }
-
-  /**
-   * @return The name string of this column family
-   */
-  @Override
-  public String getNameAsString() {
-    return delegatee.getNameAsString();
-  }
-
-  /**
-   * @param key The key.
-   * @return The value.
-   */
-  @Override
-  public byte[] getValue(byte[] key) {
-    return delegatee.getValue(key);
-  }
-
-  /**
-   * @param key The key.
-   * @return The value as a string.
-   */
-  public String getValue(String key) {
-    byte[] value = getValue(Bytes.toBytes(key));
-    return value == null ? null : Bytes.toString(value);
-  }
-
-  @Override
-  public Map<Bytes, Bytes> getValues() {
-    return delegatee.getValues();
-  }
-
-  /**
-   * @param key The key.
-   * @param value The value.
-   * @return this (for chained invocation)
-   */
-  public HColumnDescriptor setValue(byte[] key, byte[] value) {
-    getDelegateeForModification().setValue(key, value);
-    return this;
-  }
-
-  /**
-   * @param key Key whose key and value we're to remove from HCD parameters.
-   */
-  public void remove(final byte [] key) {
-    getDelegateeForModification().removeValue(new Bytes(key));
-  }
-
-  /**
-   * @param key The key.
-   * @param value The value.
-   * @return this (for chained invocation)
-   */
-  public HColumnDescriptor setValue(String key, String value) {
-    getDelegateeForModification().setValue(key, value);
-    return this;
-  }
-
-  /**
-   * @return compression type being used for the column family
-   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
-   *             (<a href="https://issues.apache.org/jira/browse/HBASE-13655">HBASE-13655</a>).
-   *             Use {@link #getCompressionType()}.
-   */
-  @Deprecated
-  public Compression.Algorithm getCompression() {
-    return getCompressionType();
-  }
-
-  /**
-   *  @return compression type being used for the column family for major compaction
-   *  @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
-   *             (<a href="https://issues.apache.org/jira/browse/HBASE-13655">HBASE-13655</a>).
-   *             Use {@link #getCompactionCompressionType()}.
-   */
-  @Deprecated
-  public Compression.Algorithm getCompactionCompression() {
-    return getCompactionCompressionType();
-  }
-
-  @Override
-  public int getMaxVersions() {
-    return delegatee.getMaxVersions();
-  }
-
-  /**
-   * @param value maximum number of versions
-   * @return this (for chained invocation)
-   */
-  public HColumnDescriptor setMaxVersions(int value) {
-    getDelegateeForModification().setMaxVersions(value);
-    return this;
-  }
-
-  /**
-   * Set minimum and maximum versions to keep
-   *
-   * @param minVersions minimal number of versions
-   * @param maxVersions maximum number of versions
-   * @return this (for chained invocation)
-   */
-  public HColumnDescriptor setVersions(int minVersions, int maxVersions) {
-    if (minVersions <= 0) {
-      // TODO: Allow minVersion and maxVersion of 0 to be the way you say "Keep all versions".
-      // Until there is support, consider 0 or < 0 -- a configuration error.
-      throw new IllegalArgumentException("Minimum versions must be positive");
-    }
-
-    if (maxVersions < minVersions) {
-      throw new IllegalArgumentException("Unable to set MaxVersion to " + maxVersions
-        + " and set MinVersion to " + minVersions
-        + ", as maximum versions must be >= minimum versions.");
-    }
-    setMinVersions(minVersions);
-    setMaxVersions(maxVersions);
-    return this;
-  }
-
-  @Override
-  public int getBlocksize() {
-    return delegatee.getBlocksize();
-  }
-
-  /**
-   * @param value Blocksize to use when writing out storefiles/hfiles on this
-   * column family.
-   * @return this (for chained invocation)
-   */
-  public HColumnDescriptor setBlocksize(int value) {
-    getDelegateeForModification().setBlocksize(value);
-    return this;
-  }
-
-  @Override
-  public Compression.Algorithm getCompressionType() {
-    return delegatee.getCompressionType();
-  }
-
-  /**
-   * Compression types supported in hbase.
-   * LZO is not bundled as part of the hbase distribution.
-   * See <a href="http://hbase.apache.org/book.html#lzo.compression">LZO Compression</a>
-   * for how to enable it.
-   * @param value Compression type setting.
-   * @return this (for chained invocation)
-   */
-  public HColumnDescriptor setCompressionType(Compression.Algorithm value) {
-    getDelegateeForModification().setCompressionType(value);
-    return this;
-  }
-
-  @Override
-  public DataBlockEncoding getDataBlockEncoding() {
-    return delegatee.getDataBlockEncoding();
-  }
-
-  /**
-   * Set data block encoding algorithm used in block cache.
-   * @param value What kind of data block encoding will be used.
-   * @return this (for chained invocation)
-   */
-  public HColumnDescriptor setDataBlockEncoding(DataBlockEncoding value) {
-    getDelegateeForModification().setDataBlockEncoding(value);
-    return this;
-  }
-
-  /**
-   * Set whether the tags should be compressed along with DataBlockEncoding. When no
-   * DataBlockEncoding is been used, this is having no effect.
-   *
-   * @param value
-   * @return this (for chained invocation)
-   */
-  public HColumnDescriptor setCompressTags(boolean value) {
-    getDelegateeForModification().setCompressTags(value);
-    return this;
-  }
-
-  @Override
-  public boolean isCompressTags() {
-    return delegatee.isCompressTags();
-  }
-
-  @Override
-  public Compression.Algorithm getCompactionCompressionType() {
-    return delegatee.getCompactionCompressionType();
-  }
-
-  /**
-   * Compression types supported in hbase.
-   * LZO is not bundled as part of the hbase distribution.
-   * See <a href="http://hbase.apache.org/book.html#lzo.compression">LZO Compression</a>
-   * for how to enable it.
-   * @param value Compression type setting.
-   * @return this (for chained invocation)
-   */
-  public HColumnDescriptor setCompactionCompressionType(Compression.Algorithm value) {
-    getDelegateeForModification().setCompactionCompressionType(value);
-    return this;
-  }
-
-  @Override
-  public boolean isInMemory() {
-    return delegatee.isInMemory();
-  }
-
-  /**
-   * @param value True if we are to favor keeping all values for this column family in the
-   * HRegionServer cache
-   * @return this (for chained invocation)
-   */
-  public HColumnDescriptor setInMemory(boolean value) {
-    getDelegateeForModification().setInMemory(value);
-    return this;
-  }
-
-  @Override
-  public MemoryCompactionPolicy getInMemoryCompaction() {
-    return delegatee.getInMemoryCompaction();
-  }
-
-  /**
-   * @param value the prefered in-memory compaction policy
-   *                  for this column family
-   * @return this (for chained invocation)
-   */
-  public HColumnDescriptor setInMemoryCompaction(MemoryCompactionPolicy value) {
-    getDelegateeForModification().setInMemoryCompaction(value);
-    return this;
-  }
-
-  @Override
-  public KeepDeletedCells getKeepDeletedCells() {
-    return delegatee.getKeepDeletedCells();
-  }
-
-  /**
-   * @param value True if deleted rows should not be collected
-   * immediately.
-   * @return this (for chained invocation)
-   */
-  public HColumnDescriptor setKeepDeletedCells(KeepDeletedCells value) {
-    getDelegateeForModification().setKeepDeletedCells(value);
-    return this;
-  }
-
-  /**
-   * By default, HBase only consider timestamp in versions. So a previous Delete with higher ts
-   * will mask a later Put with lower ts. Set this to true to enable new semantics of versions.
-   * We will also consider mvcc in versions. See HBASE-15968 for details.
-   */
-  @Override
-  public boolean isNewVersionBehavior() {
-    return delegatee.isNewVersionBehavior();
-  }
-
-  public HColumnDescriptor setNewVersionBehavior(boolean newVersionBehavior) {
-    getDelegateeForModification().setNewVersionBehavior(newVersionBehavior);
-    return this;
-  }
-
-
-  @Override
-  public int getTimeToLive() {
-    return delegatee.getTimeToLive();
-  }
-
-  /**
-   * @param value Time-to-live of cell contents, in seconds.
-   * @return this (for chained invocation)
-   */
-  public HColumnDescriptor setTimeToLive(int value) {
-    getDelegateeForModification().setTimeToLive(value);
-    return this;
-  }
-
-  /**
-   * @param value Time to live of cell contents, in human readable format
-   *                   @see org.apache.hadoop.hbase.util.PrettyPrinter#format(String, Unit)
-   * @return this (for chained invocation)
-   */
-  public HColumnDescriptor setTimeToLive(String value) throws HBaseException {
-    getDelegateeForModification().setTimeToLive(value);
-    return this;
-  }
-
-  @Override
-  public int getMinVersions() {
-    return delegatee.getMinVersions();
-  }
-
-  /**
-   * @param value The minimum number of versions to keep.
-   * (used when timeToLive is set)
-   * @return this (for chained invocation)
-   */
-  public HColumnDescriptor setMinVersions(int value) {
-    getDelegateeForModification().setMinVersions(value);
-    return this;
-  }
-
-  @Override
-  public boolean isBlockCacheEnabled() {
-    return delegatee.isBlockCacheEnabled();
-  }
-
-  /**
-   * @param value True if hfile DATA type blocks should be cached (We always cache
-   * INDEX and BLOOM blocks; you cannot turn this off).
-   * @return this (for chained invocation)
-   */
-  public HColumnDescriptor setBlockCacheEnabled(boolean value) {
-    getDelegateeForModification().setBlockCacheEnabled(value);
-    return this;
-  }
-
-  @Override
-  public BloomType getBloomFilterType() {
-    return delegatee.getBloomFilterType();
-  }
-
-  /**
-   * @param value bloom filter type
-   * @return this (for chained invocation)
-   */
-  public HColumnDescriptor setBloomFilterType(final BloomType value) {
-    getDelegateeForModification().setBloomFilterType(value);
-    return this;
-  }
-
-  @Override
-  public int getScope() {
-    return delegatee.getScope();
-  }
-
- /**
-  * @param value the scope tag
-  * @return this (for chained invocation)
-  */
-  public HColumnDescriptor setScope(int value) {
-    getDelegateeForModification().setScope(value);
-    return this;
-  }
-
-  @Override
-  public boolean isCacheDataOnWrite() {
-    return delegatee.isCacheDataOnWrite();
-  }
-
-  /**
-   * @param value true if we should cache data blocks on write
-   * @return this (for chained invocation)
-   */
-  public HColumnDescriptor setCacheDataOnWrite(boolean value) {
-    getDelegateeForModification().setCacheDataOnWrite(value);
-    return this;
-  }
-
-  @Override
-  public boolean isCacheIndexesOnWrite() {
-    return delegatee.isCacheIndexesOnWrite();
-  }
-
-  /**
-   * @param value true if we should cache index blocks on write
-   * @return this (for chained invocation)
-   */
-  public HColumnDescriptor setCacheIndexesOnWrite(boolean value) {
-    getDelegateeForModification().setCacheIndexesOnWrite(value);
-    return this;
-  }
-
-  @Override
-  public boolean isCacheBloomsOnWrite() {
-    return delegatee.isCacheBloomsOnWrite();
-  }
-
-  /**
-   * @param value true if we should cache bloomfilter blocks on write
-   * @return this (for chained invocation)
-   */
-  public HColumnDescriptor setCacheBloomsOnWrite(boolean value) {
-    getDelegateeForModification().setCacheBloomsOnWrite(value);
-    return this;
-  }
-
-  @Override
-  public boolean isEvictBlocksOnClose() {
-    return delegatee.isEvictBlocksOnClose();
-  }
-
-  /**
-   * @param value true if we should evict cached blocks from the blockcache on
-   * close
-   * @return this (for chained invocation)
-   */
-  public HColumnDescriptor setEvictBlocksOnClose(boolean value) {
-    getDelegateeForModification().setEvictBlocksOnClose(value);
-    return this;
-  }
-
-  @Override
-  public boolean isPrefetchBlocksOnOpen() {
-    return delegatee.isPrefetchBlocksOnOpen();
-  }
-
-  /**
-   * @param value true if we should prefetch blocks into the blockcache on open
-   * @return this (for chained invocation)
-   */
-  public HColumnDescriptor setPrefetchBlocksOnOpen(boolean value) {
-    getDelegateeForModification().setPrefetchBlocksOnOpen(value);
-    return this;
-  }
-
-  /**
-   * @see java.lang.Object#toString()
-   */
-  @Override
-  public String toString() {
-    return delegatee.toString();
-  }
-
-  /**
-   * @return Column family descriptor with only the customized attributes.
-   */
-  @Override
-  public String toStringCustomizedValues() {
-    return delegatee.toStringCustomizedValues();
-  }
-
-  public static Unit getUnit(String key) {
-    return ColumnFamilyDescriptorBuilder.getUnit(key);
-  }
-
-  public static Map<String, String> getDefaultValues() {
-    return ColumnFamilyDescriptorBuilder.getDefaultValues();
-  }
-
-  /**
-   * @see java.lang.Object#equals(java.lang.Object)
-   */
-  @Override
-  public boolean equals(Object obj) {
-    if (this == obj) {
-      return true;
-    }
-    if (obj instanceof HColumnDescriptor) {
-      return delegatee.equals(((HColumnDescriptor) obj).delegatee);
-    }
-    return false;
-  }
-
-  /**
-   * @see java.lang.Object#hashCode()
-   */
-  @Override
-  public int hashCode() {
-    return delegatee.hashCode();
-  }
-
-  @Override
-  public int compareTo(HColumnDescriptor other) {
-    return COMPARATOR.compare(this, other);
-  }
-
-  /**
-   * @return This instance serialized with pb with pb magic prefix
-   * @see #parseFrom(byte[])
-   */
-  public byte[] toByteArray() {
-    return ColumnFamilyDescriptorBuilder.toByteArray(delegatee);
-  }
-
-  /**
-   * @param bytes A pb serialized {@link HColumnDescriptor} instance with pb magic prefix
-   * @return An instance of {@link HColumnDescriptor} made from <code>bytes</code>
-   * @throws DeserializationException
-   * @see #toByteArray()
-   */
-  public static HColumnDescriptor parseFrom(final byte [] bytes) throws DeserializationException {
-    ColumnFamilyDescriptor desc = ColumnFamilyDescriptorBuilder.parseFrom(bytes);
-    if (desc instanceof ModifyableColumnFamilyDescriptor) {
-      return new HColumnDescriptor((ModifyableColumnFamilyDescriptor) desc);
-    } else {
-      return new HColumnDescriptor(new ModifyableColumnFamilyDescriptor(desc));
-    }
-  }
-
-  @Override
-  public String getConfigurationValue(String key) {
-    return delegatee.getConfigurationValue(key);
-  }
-
-  @Override
-  public Map<String, String> getConfiguration() {
-    return delegatee.getConfiguration();
-  }
-
-  /**
-   * Setter for storing a configuration setting.
-   * @param key Config key. Same as XML config key e.g. hbase.something.or.other.
-   * @param value String value. If null, removes the configuration.
-   */
-  public HColumnDescriptor setConfiguration(String key, String value) {
-    getDelegateeForModification().setConfiguration(key, value);
-    return this;
-  }
-
-  /**
-   * Remove a configuration setting represented by the key.
-   */
-  public void removeConfiguration(final String key) {
-    getDelegateeForModification().removeConfiguration(key);
-  }
-
-  @Override
-  public String getEncryptionType() {
-    return delegatee.getEncryptionType();
-  }
-
-  /**
-   * Set the encryption algorithm for use with this family
-   * @param value
-   */
-  public HColumnDescriptor setEncryptionType(String value) {
-    getDelegateeForModification().setEncryptionType(value);
-    return this;
-  }
-
-  @Override
-  public byte[] getEncryptionKey() {
-    return delegatee.getEncryptionKey();
-  }
-
-  /** Set the raw crypto key attribute for the family */
-  public HColumnDescriptor setEncryptionKey(byte[] value) {
-    getDelegateeForModification().setEncryptionKey(value);
-    return this;
-  }
-
-  @Override
-  public long getMobThreshold() {
-    return delegatee.getMobThreshold();
-  }
-
-  /**
-   * Sets the mob threshold of the family.
-   * @param value The mob threshold.
-   * @return this (for chained invocation)
-   */
-  public HColumnDescriptor setMobThreshold(long value) {
-    getDelegateeForModification().setMobThreshold(value);
-    return this;
-  }
-
-  @Override
-  public boolean isMobEnabled() {
-    return delegatee.isMobEnabled();
-  }
-
-  /**
-   * Enables the mob for the family.
-   * @param value Whether to enable the mob for the family.
-   * @return this (for chained invocation)
-   */
-  public HColumnDescriptor setMobEnabled(boolean value) {
-    getDelegateeForModification().setMobEnabled(value);
-    return this;
-  }
-
-  @Override
-  public MobCompactPartitionPolicy getMobCompactPartitionPolicy() {
-    return delegatee.getMobCompactPartitionPolicy();
-  }
-
-  /**
-   * Set the mob compact partition policy for the family.
-   * @param value policy type
-   * @return this (for chained invocation)
-   */
-  public HColumnDescriptor setMobCompactPartitionPolicy(MobCompactPartitionPolicy value) {
-    getDelegateeForModification().setMobCompactPartitionPolicy(value);
-    return this;
-  }
-
-  @Override
-  public short getDFSReplication() {
-    return delegatee.getDFSReplication();
-  }
-
-  /**
-   * Set the replication factor to hfile(s) belonging to this family
-   * @param value number of replicas the blocks(s) belonging to this CF should have, or
-   *          {@link #DEFAULT_DFS_REPLICATION} for the default replication factor set in the
-   *          filesystem
-   * @return this (for chained invocation)
-   */
-  public HColumnDescriptor setDFSReplication(short value) {
-    getDelegateeForModification().setDFSReplication(value);
-    return this;
-  }
-
-  @Override
-  public String getStoragePolicy() {
-    return delegatee.getStoragePolicy();
-  }
-
-  /**
-   * Set the storage policy for use with this family
-   * @param value the policy to set, valid setting includes: <i>"LAZY_PERSIST"</i>,
-   *          <i>"ALL_SSD"</i>, <i>"ONE_SSD"</i>, <i>"HOT"</i>, <i>"WARM"</i>, <i>"COLD"</i>
-   */
-  public HColumnDescriptor setStoragePolicy(String value) {
-    getDelegateeForModification().setStoragePolicy(value);
-    return this;
-  }
-
-  @Override
-  public Bytes getValue(Bytes key) {
-    return delegatee.getValue(key);
-  }
-
-  protected ModifyableColumnFamilyDescriptor getDelegateeForModification() {
-    return delegatee;
-  }
-}
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
deleted file mode 100644
index 02597ce..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
+++ /dev/null
@@ -1,992 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-import java.util.Optional;
-import java.util.Set;
-import java.util.stream.Collectors;
-import java.util.stream.Stream;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
-import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor;
-import org.apache.hadoop.hbase.client.CoprocessorDescriptor;
-import org.apache.hadoop.hbase.client.CoprocessorDescriptorBuilder;
-import org.apache.hadoop.hbase.client.Durability;
-import org.apache.hadoop.hbase.client.TableDescriptor;
-import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
-import org.apache.hadoop.hbase.client.TableDescriptorBuilder.ModifyableTableDescriptor;
-import org.apache.hadoop.hbase.exceptions.DeserializationException;
-import org.apache.hadoop.hbase.security.User;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.yetus.audience.InterfaceAudience;
-
-/**
- * HTableDescriptor contains the details about an HBase table  such as the descriptors of
- * all the column families, is the table a catalog table, <code> hbase:meta </code>,
- * if the table is read only, the maximum size of the memstore,
- * when the region split should occur, coprocessors associated with it etc...
- * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
- *             Use {@link TableDescriptorBuilder} to build {@link HTableDescriptor}.
- */
-@Deprecated
-@InterfaceAudience.Public
-public class HTableDescriptor implements TableDescriptor, Comparable<HTableDescriptor> {
-  public static final String SPLIT_POLICY = TableDescriptorBuilder.SPLIT_POLICY;
-  public static final String MAX_FILESIZE = TableDescriptorBuilder.MAX_FILESIZE;
-  public static final String OWNER = TableDescriptorBuilder.OWNER;
-  public static final Bytes OWNER_KEY = TableDescriptorBuilder.OWNER_KEY;
-  public static final String READONLY = TableDescriptorBuilder.READONLY;
-  public static final String COMPACTION_ENABLED = TableDescriptorBuilder.COMPACTION_ENABLED;
-  public static final String SPLIT_ENABLED = TableDescriptorBuilder.SPLIT_ENABLED;
-  public static final String MERGE_ENABLED = TableDescriptorBuilder.MERGE_ENABLED;
-  public static final String MEMSTORE_FLUSHSIZE = TableDescriptorBuilder.MEMSTORE_FLUSHSIZE;
-  public static final String FLUSH_POLICY = TableDescriptorBuilder.FLUSH_POLICY;
-  public static final String IS_ROOT = "IS_ROOT";
-  public static final String IS_META = TableDescriptorBuilder.IS_META;
-  public static final String DURABILITY = TableDescriptorBuilder.DURABILITY;
-  public static final String REGION_REPLICATION = TableDescriptorBuilder.REGION_REPLICATION;
-  public static final String REGION_MEMSTORE_REPLICATION = TableDescriptorBuilder.REGION_MEMSTORE_REPLICATION;
-  public static final String NORMALIZATION_ENABLED = TableDescriptorBuilder.NORMALIZATION_ENABLED;
-  public static final String NORMALIZER_TARGET_REGION_COUNT =
-      TableDescriptorBuilder.NORMALIZER_TARGET_REGION_COUNT;
-  public static final String NORMALIZER_TARGET_REGION_SIZE =
-      TableDescriptorBuilder.NORMALIZER_TARGET_REGION_SIZE;
-  public static final String PRIORITY = TableDescriptorBuilder.PRIORITY;
-  public static final boolean DEFAULT_READONLY = TableDescriptorBuilder.DEFAULT_READONLY;
-  public static final boolean DEFAULT_COMPACTION_ENABLED = TableDescriptorBuilder.DEFAULT_COMPACTION_ENABLED;
-  public static final boolean DEFAULT_NORMALIZATION_ENABLED = TableDescriptorBuilder.DEFAULT_NORMALIZATION_ENABLED;
-  public static final long DEFAULT_MEMSTORE_FLUSH_SIZE = TableDescriptorBuilder.DEFAULT_MEMSTORE_FLUSH_SIZE;
-  public static final int DEFAULT_REGION_REPLICATION = TableDescriptorBuilder.DEFAULT_REGION_REPLICATION;
-  public static final boolean DEFAULT_REGION_MEMSTORE_REPLICATION = TableDescriptorBuilder.DEFAULT_REGION_MEMSTORE_REPLICATION;
-  protected final ModifyableTableDescriptor delegatee;
-
-  /**
-   * Construct a table descriptor specifying a TableName object
-   * @param name Table name.
-   * @see <a href="https://issues.apache.org/jira/browse/HBASE-174">HADOOP-1581 HBASE: (HBASE-174) Un-openable tablename bug</a>
-   */
-  public HTableDescriptor(final TableName name) {
-    this(new ModifyableTableDescriptor(name));
-  }
-
-  /**
-   * Construct a table descriptor by cloning the descriptor passed as a parameter.
-   * <p>
-   * Makes a deep copy of the supplied descriptor.
-   * Can make a modifiable descriptor from an ImmutableHTableDescriptor.
-   * @param desc The descriptor.
-   */
-  public HTableDescriptor(final HTableDescriptor desc) {
-    this(desc, true);
-  }
-
-  protected HTableDescriptor(final HTableDescriptor desc, boolean deepClone) {
-    this(deepClone ? new ModifyableTableDescriptor(desc.getTableName(), desc)
-      : desc.delegatee);
-  }
-
-  public HTableDescriptor(final TableDescriptor desc) {
-    this(new ModifyableTableDescriptor(desc.getTableName(), desc));
-  }
-
-  /**
-   * Construct a table descriptor by cloning the descriptor passed as a parameter
-   * but using a different table name.
-   * <p>
-   * Makes a deep copy of the supplied descriptor.
-   * Can make a modifiable descriptor from an ImmutableHTableDescriptor.
-   * @param name Table name.
-   * @param desc The descriptor.
-   */
-  public HTableDescriptor(final TableName name, final HTableDescriptor desc) {
-    this(new ModifyableTableDescriptor(name, desc));
-  }
-
-  protected HTableDescriptor(ModifyableTableDescriptor delegatee) {
-    this.delegatee = delegatee;
-  }
-
-  /**
-   * This is vestigial API. It will be removed in 3.0.
-   *
-   * @return always return the false
-   */
-  public boolean isRootRegion() {
-    return false;
-  }
-
-  /**
-   * Checks if this table is <code> hbase:meta </code>
-   * region.
-   *
-   * @return true if this table is <code> hbase:meta </code>
-   * region
-   */
-  @Override
-  public boolean isMetaRegion() {
-    return delegatee.isMetaRegion();
-  }
-
-  /**
-   * Checks if the table is a <code>hbase:meta</code> table
-   *
-   * @return true if table is <code> hbase:meta </code> region.
-   */
-  @Override
-  public boolean isMetaTable() {
-    return delegatee.isMetaTable();
-  }
-
-  /**
-   * @return Getter for fetching an unmodifiable map.
-   */
-  @Override
-  public Map<Bytes, Bytes> getValues() {
-    return delegatee.getValues();
-  }
-
-  /**
-   * Setter for storing metadata as a (key, value) pair in map
-   *
-   * @param key The key.
-   * @param value The value. If null, removes the setting.
-   */
-  public HTableDescriptor setValue(byte[] key, byte[] value) {
-    getDelegateeForModification().setValue(key, value);
-    return this;
-  }
-
-  /*
-   * Setter for storing metadata as a (key, value) pair in map
-   *
-   * @param key The key.
-   * @param value The value. If null, removes the setting.
-   */
-  public HTableDescriptor setValue(final Bytes key, final Bytes value) {
-    getDelegateeForModification().setValue(key, value);
-    return this;
-  }
-
-  /**
-   * Setter for storing metadata as a (key, value) pair in map
-   *
-   * @param key The key.
-   * @param value The value. If null, removes the setting.
-   */
-  public HTableDescriptor setValue(String key, String value) {
-    getDelegateeForModification().setValue(key, value);
-    return this;
-  }
-
-  /**
-   * Remove metadata represented by the key from the map
-   *
-   * @param key Key whose key and value we're to remove from HTableDescriptor
-   * parameters.
-   */
-  public void remove(final String key) {
-    getDelegateeForModification().removeValue(Bytes.toBytes(key));
-  }
-
-  /**
-   * Remove metadata represented by the key from the map
-   *
-   * @param key Key whose key and value we're to remove from HTableDescriptor
-   * parameters.
-   */
-  public void remove(Bytes key) {
-    getDelegateeForModification().removeValue(key);
-  }
-
-  /**
-   * Remove metadata represented by the key from the map
-   *
-   * @param key Key whose key and value we're to remove from HTableDescriptor
-   * parameters.
-   */
-  public void remove(final byte [] key) {
-    getDelegateeForModification().removeValue(key);
-  }
-
-  /**
-   * Check if the readOnly flag of the table is set. If the readOnly flag is
-   * set then the contents of the table can only be read from but not modified.
-   *
-   * @return true if all columns in the table should be read only
-   */
-  @Override
-  public boolean isReadOnly() {
-    return delegatee.isReadOnly();
-  }
-
-  /**
-   * Setting the table as read only sets all the columns in the table as read
-   * only. By default all tables are modifiable, but if the readOnly flag is
-   * set to true then the contents of the table can only be read but not modified.
-   *
-   * @param readOnly True if all of the columns in the table should be read
-   * only.
-   */
-  public HTableDescriptor setReadOnly(final boolean readOnly) {
-    getDelegateeForModification().setReadOnly(readOnly);
-    return this;
-  }
-
-  /**
-   * Check if the compaction enable flag of the table is true. If flag is
-   * false then no minor/major compactions will be done in real.
-   *
-   * @return true if table compaction enabled
-   */
-  @Override
-  public boolean isCompactionEnabled() {
-    return delegatee.isCompactionEnabled();
-  }
-
-  /**
-   * Setting the table compaction enable flag.
-   *
-   * @param isEnable True if enable compaction.
-   */
-  public HTableDescriptor setCompactionEnabled(final boolean isEnable) {
-    getDelegateeForModification().setCompactionEnabled(isEnable);
-    return this;
-  }
-
-  /**
-   * Check if the region split enable flag of the table is true. If flag is
-   * false then no split will be done.
-   *
-   * @return true if table region split enabled
-   */
-  @Override
-  public boolean isSplitEnabled() {
-    return delegatee.isSplitEnabled();
-  }
-
-  /**
-   * Setting the table region split enable flag.
-   *
-   * @param isEnable True if enable split.
-   */
-  public HTableDescriptor setSplitEnabled(final boolean isEnable) {
-    getDelegateeForModification().setSplitEnabled(isEnable);
-    return this;
-  }
-
-
-  /**
-   * Check if the region merge enable flag of the table is true. If flag is
-   * false then no merge will be done.
-   *
-   * @return true if table region merge enabled
-   */
-  @Override
-  public boolean isMergeEnabled() {
-    return delegatee.isMergeEnabled();
-  }
-
-  /**
-   * Setting the table region merge enable flag.
-   *
-   * @param isEnable True if enable merge.
-   */
-  public HTableDescriptor setMergeEnabled(final boolean isEnable) {
-    getDelegateeForModification().setMergeEnabled(isEnable);
-    return this;
-  }
-
-  /**
-   * Check if normalization enable flag of the table is true. If flag is
-   * false then no region normalizer won't attempt to normalize this table.
-   *
-   * @return true if region normalization is enabled for this table
-   */
-  @Override
-  public boolean isNormalizationEnabled() {
-    return delegatee.isNormalizationEnabled();
-  }
-
-  /**
-   * Setting the table normalization enable flag.
-   *
-   * @param isEnable True if enable normalization.
-   */
-  public HTableDescriptor setNormalizationEnabled(final boolean isEnable) {
-    getDelegateeForModification().setNormalizationEnabled(isEnable);
-    return this;
-  }
-
-  @Override
-  public int getNormalizerTargetRegionCount() {
-    return getDelegateeForModification().getNormalizerTargetRegionCount();
-  }
-
-  public HTableDescriptor setNormalizerTargetRegionCount(final int regionCount) {
-    getDelegateeForModification().setNormalizerTargetRegionCount(regionCount);
-    return this;
-  }
-
-  @Override
-  public long getNormalizerTargetRegionSize() {
-    return getDelegateeForModification().getNormalizerTargetRegionSize();
-  }
-
-  public HTableDescriptor setNormalizerTargetRegionSize(final long regionSize) {
-    getDelegateeForModification().setNormalizerTargetRegionSize(regionSize);
-    return this;
-  }
-
-  /**
-   * Sets the {@link Durability} setting for the table. This defaults to Durability.USE_DEFAULT.
-   * @param durability enum value
-   */
-  public HTableDescriptor setDurability(Durability durability) {
-    getDelegateeForModification().setDurability(durability);
-    return this;
-  }
-
-  /**
-   * Returns the durability setting for the table.
-   * @return durability setting for the table.
-   */
-  @Override
-  public Durability getDurability() {
-    return delegatee.getDurability();
-  }
-
-  /**
-   * Get the name of the table
-   *
-   * @return TableName
-   */
-  @Override
-  public TableName getTableName() {
-    return delegatee.getTableName();
-  }
-
-  /**
-   * Get the name of the table as a String
-   *
-   * @return name of table as a String
-   */
-  public String getNameAsString() {
-    return delegatee.getTableName().getNameAsString();
-  }
-
-  /**
-   * This sets the class associated with the region split policy which
-   * determines when a region split should occur.  The class used by
-   * default is defined in org.apache.hadoop.hbase.regionserver.RegionSplitPolicy
-   * @param clazz the class name
-   */
-  public HTableDescriptor setRegionSplitPolicyClassName(String clazz) {
-    getDelegateeForModification().setRegionSplitPolicyClassName(clazz);
-    return this;
-  }
-
-  /**
-   * This gets the class associated with the region split policy which
-   * determines when a region split should occur.  The class used by
-   * default is defined in org.apache.hadoop.hbase.regionserver.RegionSplitPolicy
-   *
-   * @return the class name of the region split policy for this table.
-   * If this returns null, the default split policy is used.
-   */
-  @Override
-   public String getRegionSplitPolicyClassName() {
-    return delegatee.getRegionSplitPolicyClassName();
-  }
-
-  /**
-   * Returns the maximum size upto which a region can grow to after which a region
-   * split is triggered. The region size is represented by the size of the biggest
-   * store file in that region.
-   *
-   * @return max hregion size for table, -1 if not set.
-   *
-   * @see #setMaxFileSize(long)
-   */
-   @Override
-  public long getMaxFileSize() {
-    return delegatee.getMaxFileSize();
-  }
-
-  /**
-   * Sets the maximum size upto which a region can grow to after which a region
-   * split is triggered. The region size is represented by the size of the biggest
-   * store file in that region, i.e. If the biggest store file grows beyond the
-   * maxFileSize, then the region split is triggered. This defaults to a value of
-   * 256 MB.
-   * <p>
-   * This is not an absolute value and might vary. Assume that a single row exceeds
-   * the maxFileSize then the storeFileSize will be greater than maxFileSize since
-   * a single row cannot be split across multiple regions
-   * </p>
-   *
-   * @param maxFileSize The maximum file size that a store file can grow to
-   * before a split is triggered.
-   */
-  public HTableDescriptor setMaxFileSize(long maxFileSize) {
-    getDelegateeForModification().setMaxFileSize(maxFileSize);
-    return this;
-  }
-
-  /**
-   * Returns the size of the memstore after which a flush to filesystem is triggered.
-   *
-   * @return memory cache flush size for each hregion, -1 if not set.
-   *
-   * @see #setMemStoreFlushSize(long)
-   */
-  @Override
-  public long getMemStoreFlushSize() {
-    return delegatee.getMemStoreFlushSize();
-  }
-
-  /**
-   * Represents the maximum size of the memstore after which the contents of the
-   * memstore are flushed to the filesystem. This defaults to a size of 64 MB.
-   *
-   * @param memstoreFlushSize memory cache flush size for each hregion
-   */
-  public HTableDescriptor setMemStoreFlushSize(long memstoreFlushSize) {
-    getDelegateeForModification().setMemStoreFlushSize(memstoreFlushSize);
-    return this;
-  }
-
-  /**
-   * This sets the class associated with the flush policy which determines determines the stores
-   * need to be flushed when flushing a region. The class used by default is defined in
-   * org.apache.hadoop.hbase.regionserver.FlushPolicy.
-   * @param clazz the class name
-   */
-  public HTableDescriptor setFlushPolicyClassName(String clazz) {
-    getDelegateeForModification().setFlushPolicyClassName(clazz);
-    return this;
-  }
-
-  /**
-   * This gets the class associated with the flush policy which determines the stores need to be
-   * flushed when flushing a region. The class used by default is defined in
-   * org.apache.hadoop.hbase.regionserver.FlushPolicy.
-   * @return the class name of the flush policy for this table. If this returns null, the default
-   *         flush policy is used.
-   */
-  @Override
-  public String getFlushPolicyClassName() {
-    return delegatee.getFlushPolicyClassName();
-  }
-
-  /**
-   * Adds a column family.
-   * For the updating purpose please use {@link #modifyFamily(HColumnDescriptor)} instead.
-   * @param family HColumnDescriptor of family to add.
-   */
-  public HTableDescriptor addFamily(final HColumnDescriptor family) {
-    getDelegateeForModification().setColumnFamily(family);
-    return this;
-  }
-
-  /**
-   * Modifies the existing column family.
-   * @param family HColumnDescriptor of family to update
-   * @return this (for chained invocation)
-   */
-  public HTableDescriptor modifyFamily(final HColumnDescriptor family) {
-    getDelegateeForModification().modifyColumnFamily(family);
-    return this;
-  }
-
-  /**
-   * Checks to see if this table contains the given column family
-   * @param familyName Family name or column name.
-   * @return true if the table contains the specified family name
-   */
-  public boolean hasFamily(final byte [] familyName) {
-    return delegatee.hasColumnFamily(familyName);
-  }
-
-  /**
-   * @return Name of this table and then a map of all of the column family
-   * descriptors.
-   * @see #getNameAsString()
-   */
-  @Override
-  public String toString() {
-    return delegatee.toString();
-  }
-
-  /**
-   * @return Name of this table and then a map of all of the column family
-   * descriptors (with only the non-default column family attributes)
-   */
-  @Override
-  public String toStringCustomizedValues() {
-    return delegatee.toStringCustomizedValues();
-  }
-
-  /**
-   * @return map of all table attributes formatted into string.
-   */
-  public String toStringTableAttributes() {
-   return delegatee.toStringTableAttributes();
-  }
-
-  /**
-   * Compare the contents of the descriptor with another one passed as a parameter.
-   * Checks if the obj passed is an instance of HTableDescriptor, if yes then the
-   * contents of the descriptors are compared.
-   *
-   * @return true if the contents of the the two descriptors exactly match
-   *
-   * @see java.lang.Object#equals(java.lang.Object)
-   */
-  @Override
-  public boolean equals(Object obj) {
-    if (this == obj) {
-      return true;
-    }
-    if (obj instanceof HTableDescriptor) {
-      return delegatee.equals(((HTableDescriptor) obj).delegatee);
-    }
-    return false;
-  }
-
-  /**
-   * @see java.lang.Object#hashCode()
-   */
-  @Override
-  public int hashCode() {
-    return delegatee.hashCode();
-  }
-
-  // Comparable
-
-  /**
-   * Compares the descriptor with another descriptor which is passed as a parameter.
-   * This compares the content of the two descriptors and not the reference.
-   *
-   * @return 0 if the contents of the descriptors are exactly matching,
-   *         1 if there is a mismatch in the contents
-   */
-  @Override
-  public int compareTo(final HTableDescriptor other) {
-    return TableDescriptor.COMPARATOR.compare(this, other);
-  }
-
-  /**
-   * Returns an unmodifiable collection of all the {@link HColumnDescriptor}
-   * of all the column families of the table.
-   * @deprecated since 2.0.0 and will be removed in 3.0.0. Use {@link #getColumnFamilies()} instead.
-   * @return Immutable collection of {@link HColumnDescriptor} of all the
-   * column families.
-   * @see #getColumnFamilies()
-   * @see <a href="https://issues.apache.org/jira/browse/HBASE-18008">HBASE-18008</a>
-   */
-  @Deprecated
-  public Collection<HColumnDescriptor> getFamilies() {
-    return Stream.of(delegatee.getColumnFamilies())
-            .map(this::toHColumnDescriptor)
-            .collect(Collectors.toList());
-  }
-
-  /**
-   * Returns the configured replicas per region
-   */
-  @Override
-  public int getRegionReplication() {
-    return delegatee.getRegionReplication();
-  }
-
-  /**
-   * Sets the number of replicas per region.
-   * @param regionReplication the replication factor per region
-   */
-  public HTableDescriptor setRegionReplication(int regionReplication) {
-    getDelegateeForModification().setRegionReplication(regionReplication);
-    return this;
-  }
-
-  /**
-   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
-   *             Use {@link #hasRegionMemStoreReplication()} instead
-   */
-  @Deprecated
-  public boolean hasRegionMemstoreReplication() {
-    return hasRegionMemStoreReplication();
-  }
-
-  /**
-   * @return true if the read-replicas memstore replication is enabled.
-   */
-  @Override
-  public boolean hasRegionMemStoreReplication() {
-    return delegatee.hasRegionMemStoreReplication();
-  }
-
-  /**
-   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
-   *             Use {@link #setRegionMemStoreReplication(boolean)} instead
-   */
-  @Deprecated
-  public HTableDescriptor setRegionMemstoreReplication(boolean memstoreReplication) {
-    return setRegionMemStoreReplication(memstoreReplication);
-  }
-
-  /**
-   * Enable or Disable the memstore replication from the primary region to the replicas.
-   * The replication will be used only for meta operations (e.g. flush, compaction, ...)
-   *
-   * @param memstoreReplication true if the new data written to the primary region
-   *                                 should be replicated.
-   *                            false if the secondaries can tollerate to have new
-   *                                  data only when the primary flushes the memstore.
-   */
-  public HTableDescriptor setRegionMemStoreReplication(boolean memstoreReplication) {
-    getDelegateeForModification().setRegionMemStoreReplication(memstoreReplication);
-    return this;
-  }
-
-  public HTableDescriptor setPriority(int priority) {
-    getDelegateeForModification().setPriority(priority);
-    return this;
-  }
-
-  @Override
-  public int getPriority() {
-    return delegatee.getPriority();
-  }
-
-  /**
-   * Returns all the column family names of the current table. The map of
-   * HTableDescriptor contains mapping of family name to HColumnDescriptors.
-   * This returns all the keys of the family map which represents the column
-   * family names of the table.
-   *
-   * @return Immutable sorted set of the keys of the families.
-   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
-   *             (<a href="https://issues.apache.org/jira/browse/HBASE-18008">HBASE-18008</a>).
-   *             Use {@link #getColumnFamilyNames()}.
-   */
-  @Deprecated
-  public Set<byte[]> getFamiliesKeys() {
-    return delegatee.getColumnFamilyNames();
-  }
-
-  /**
-   * Returns the count of the column families of the table.
-   *
-   * @return Count of column families of the table
-   */
-  @Override
-  public int getColumnFamilyCount() {
-    return delegatee.getColumnFamilyCount();
-  }
-
-  /**
-   * Returns an array all the {@link HColumnDescriptor} of the column families
-   * of the table.
-   *
-   * @return Array of all the HColumnDescriptors of the current table
-   * @deprecated since 2.0.0 and will be removed in 3.0.0.
-   * @see #getFamilies()
-   * @see <a href="https://issues.apache.org/jira/browse/HBASE-18008">HBASE-18008</a>
-   */
-  @Deprecated
-  @Override
-  public HColumnDescriptor[] getColumnFamilies() {
-    return Stream.of(delegatee.getColumnFamilies())
-            .map(this::toHColumnDescriptor)
-            .toArray(size -> new HColumnDescriptor[size]);
-  }
-
-  /**
-   * Returns the HColumnDescriptor for a specific column family with name as
-   * specified by the parameter column.
-   * @param column Column family name
-   * @return Column descriptor for the passed family name or the family on
-   * passed in column.
-   * @deprecated since 2.0.0 and will be removed in 3.0.0. Use {@link #getColumnFamily(byte[])}
-   *   instead.
-   * @see #getColumnFamily(byte[])
-   * @see <a href="https://issues.apache.org/jira/browse/HBASE-18008">HBASE-18008</a>
-   */
-  @Deprecated
-  public HColumnDescriptor getFamily(final byte[] column) {
-    return toHColumnDescriptor(delegatee.getColumnFamily(column));
-  }
-
-
-  /**
-   * Removes the HColumnDescriptor with name specified by the parameter column
-   * from the table descriptor
-   *
-   * @param column Name of the column family to be removed.
-   * @return Column descriptor for the passed family name or the family on
-   * passed in column.
-   */
-  public HColumnDescriptor removeFamily(final byte [] column) {
-    return toHColumnDescriptor(getDelegateeForModification().removeColumnFamily(column));
-  }
-
-  /**
-   * Return a HColumnDescriptor for user to keep the compatibility as much as possible.
-   * @param desc read-only ColumnFamilyDescriptor
-   * @return The older implementation of ColumnFamilyDescriptor
-   */
-  protected HColumnDescriptor toHColumnDescriptor(ColumnFamilyDescriptor desc) {
-    if (desc == null) {
-      return null;
-    } else if (desc instanceof ModifyableColumnFamilyDescriptor) {
-      return new HColumnDescriptor((ModifyableColumnFamilyDescriptor) desc);
-    } else if (desc instanceof HColumnDescriptor) {
-      return (HColumnDescriptor) desc;
-    } else {
-      return new HColumnDescriptor(new ModifyableColumnFamilyDescriptor(desc));
-    }
-  }
-
-  /**
-   * Add a table coprocessor to this table. The coprocessor
-   * type must be org.apache.hadoop.hbase.coprocessor.RegionCoprocessor.
-   * It won't check if the class can be loaded or not.
-   * Whether a coprocessor is loadable or not will be determined when
-   * a region is opened.
-   * @param className Full class name.
-   * @throws IOException
-   */
-  public HTableDescriptor addCoprocessor(String className) throws IOException {
-    getDelegateeForModification().setCoprocessor(className);
-    return this;
-  }
-
-  /**
-   * Add a table coprocessor to this table. The coprocessor
-   * type must be org.apache.hadoop.hbase.coprocessor.RegionCoprocessor.
-   * It won't check if the class can be loaded or not.
-   * Whether a coprocessor is loadable or not will be determined when
-   * a region is opened.
-   * @param jarFilePath Path of the jar file. If it's null, the class will be
-   * loaded from default classloader.
-   * @param className Full class name.
-   * @param priority Priority
-   * @param kvs Arbitrary key-value parameter pairs passed into the coprocessor.
-   * @throws IOException
-   */
-  public HTableDescriptor addCoprocessor(String className, Path jarFilePath,
-                             int priority, final Map<String, String> kvs)
-  throws IOException {
-    getDelegateeForModification().setCoprocessor(
-      CoprocessorDescriptorBuilder.newBuilder(className)
-        .setJarPath(jarFilePath == null ? null : jarFilePath.toString())
-        .setPriority(priority)
-        .setProperties(kvs == null ? Collections.emptyMap() : kvs)
-        .build());
-    return this;
-  }
-
-  /**
-   * Add a table coprocessor to this table. The coprocessor
-   * type must be org.apache.hadoop.hbase.coprocessor.RegionCoprocessor.
-   * It won't check if the class can be loaded or not.
-   * Whether a coprocessor is loadable or not will be determined when
-   * a region is opened.
-   * @param specStr The Coprocessor specification all in in one String formatted so matches
-   * {@link HConstants#CP_HTD_ATTR_VALUE_PATTERN}
-   * @throws IOException
-   */
-  public HTableDescriptor addCoprocessorWithSpec(final String specStr) throws IOException {
-    getDelegateeForModification().setCoprocessorWithSpec(specStr);
-    return this;
-  }
-
-  /**
-   * Check if the table has an attached co-processor represented by the name className
-   *
-   * @param classNameToMatch - Class name of the co-processor
-   * @return true of the table has a co-processor className
-   */
-  @Override
-  public boolean hasCoprocessor(String classNameToMatch) {
-    return delegatee.hasCoprocessor(classNameToMatch);
-  }
-
-  @Override
-  public Collection<CoprocessorDescriptor> getCoprocessorDescriptors() {
-    return delegatee.getCoprocessorDescriptors();
-  }
-
-  /**
-   * Return the list of attached co-processor represented by their name className
-   *
-   * @return The list of co-processors classNames
-   */
-  public List<String> getCoprocessors() {
-    return getCoprocessorDescriptors().stream().map(CoprocessorDescriptor::getClassName)
-      .collect(Collectors.toList());
-  }
-
-  /**
-   * Remove a coprocessor from those set on the table
-   * @param className Class name of the co-processor
-   */
-  public void removeCoprocessor(String className) {
-    getDelegateeForModification().removeCoprocessor(className);
-  }
-
-  public final static String NAMESPACE_FAMILY_INFO = TableDescriptorBuilder.NAMESPACE_FAMILY_INFO;
-  public final static byte[] NAMESPACE_FAMILY_INFO_BYTES = TableDescriptorBuilder.NAMESPACE_FAMILY_INFO_BYTES;
-  public final static byte[] NAMESPACE_COL_DESC_BYTES = TableDescriptorBuilder.NAMESPACE_COL_DESC_BYTES;
-
-  /** Table descriptor for namespace table */
-  public static final HTableDescriptor NAMESPACE_TABLEDESC
-    = new HTableDescriptor(TableDescriptorBuilder.NAMESPACE_TABLEDESC);
-
-  /**
-   * @deprecated since 0.94.1
-   * @see <a href="https://issues.apache.org/jira/browse/HBASE-6188">HBASE-6188</a>
-   */
-  @Deprecated
-  public HTableDescriptor setOwner(User owner) {
-    getDelegateeForModification().setOwner(owner);
-    return this;
-  }
-
-  /**
-   * @deprecated since 0.94.1
-   * @see <a href="https://issues.apache.org/jira/browse/HBASE-6188">HBASE-6188</a>
-   */
-  // used by admin.rb:alter(table_name,*args) to update owner.
-  @Deprecated
-  public HTableDescriptor setOwnerString(String ownerString) {
-    getDelegateeForModification().setOwnerString(ownerString);
-    return this;
-  }
-
-  /**
-   * @deprecated since 0.94.1
-   * @see <a href="https://issues.apache.org/jira/browse/HBASE-6188">HBASE-6188</a>
-   */
-  @Override
-  @Deprecated
-  public String getOwnerString() {
-    return delegatee.getOwnerString();
-  }
-
-  /**
-   * @return This instance serialized with pb with pb magic prefix
-   * @see #parseFrom(byte[])
-   */
-  public byte[] toByteArray() {
-    return TableDescriptorBuilder.toByteArray(delegatee);
-  }
-
-  /**
-   * @param bytes A pb serialized {@link HTableDescriptor} instance with pb magic prefix
-   * @return An instance of {@link HTableDescriptor} made from <code>bytes</code>
-   * @throws DeserializationException
-   * @throws IOException
-   * @see #toByteArray()
-   */
-  public static HTableDescriptor parseFrom(final byte [] bytes)
-  throws DeserializationException, IOException {
-    TableDescriptor desc = TableDescriptorBuilder.parseFrom(bytes);
-    if (desc instanceof ModifyableTableDescriptor) {
-      return new HTableDescriptor((ModifyableTableDescriptor) desc);
-    } else {
-      return new HTableDescriptor(desc);
-    }
-  }
-
-  /**
-   * Getter for accessing the configuration value by key
-   */
-  public String getConfigurationValue(String key) {
-    return delegatee.getValue(key);
-  }
-
-  /**
-   * Getter for fetching an unmodifiable map.
-   */
-  public Map<String, String> getConfiguration() {
-    return delegatee.getConfiguration();
-  }
-
-  /**
-   * Setter for storing a configuration setting in map.
-   * @param key Config key. Same as XML config key e.g. hbase.something.or.other.
-   * @param value String value. If null, removes the setting.
-   */
-  public HTableDescriptor setConfiguration(String key, String value) {
-    getDelegateeForModification().setValue(key, value);
-    return this;
-  }
-
-  /**
-   * Remove a config setting represented by the key from the map
-   */
-  public void removeConfiguration(final String key) {
-    getDelegateeForModification().removeValue(Bytes.toBytes(key));
-  }
-
-  @Override
-  public Bytes getValue(Bytes key) {
-    return delegatee.getValue(key);
-  }
-
-  @Override
-  public String getValue(String key) {
-    return delegatee.getValue(key);
-  }
-
-  @Override
-  public byte[] getValue(byte[] key) {
-    return delegatee.getValue(key);
-  }
-
-  @Override
-  public Set<byte[]> getColumnFamilyNames() {
-    return delegatee.getColumnFamilyNames();
-  }
-
-  @Override
-  public boolean hasColumnFamily(byte[] name) {
-    return delegatee.hasColumnFamily(name);
-  }
-
-  @Override
-  public ColumnFamilyDescriptor getColumnFamily(byte[] name) {
-    return delegatee.getColumnFamily(name);
-  }
-
-  protected ModifyableTableDescriptor getDelegateeForModification() {
-    return delegatee;
-  }
-
-  @Override
-  public Optional<String> getRegionServerGroup() {
-    return delegatee.getRegionServerGroup();
-  }
-}
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 87579bf..3889d32 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
@@ -587,10 +587,8 @@ public class ColumnFamilyDescriptorBuilder {
    * number of versions, compression settings, etc.
    *
    * It is used as input when creating a table or adding a column.
-   * TODO: make this package-private after removing the HColumnDescriptor
    */
-  @InterfaceAudience.Private
-  public static class ModifyableColumnFamilyDescriptor
+  private static final class ModifyableColumnFamilyDescriptor
       implements ColumnFamilyDescriptor, Comparable<ModifyableColumnFamilyDescriptor> {
 
     // Column family name
@@ -692,15 +690,6 @@ public class ColumnFamilyDescriptorBuilder {
       return this;
     }
 
-    /**
-     *
-     * @param key Key whose key and value we're to remove from HCD parameters.
-     * @return this (for chained invocation)
-     */
-    public ModifyableColumnFamilyDescriptor removeValue(final Bytes key) {
-      return setValue(key, (Bytes) null);
-    }
-
     private static <T> Bytes toBytesOrNull(T t, Function<T, byte[]> f) {
       if (t == null) {
         return null;
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 bf591a1..f5ec540 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
@@ -576,12 +576,8 @@ public class TableDescriptorBuilder {
     return new ModifyableTableDescriptor(desc);
   }
 
-  /**
-   * TODO: make this private after removing the HTableDescriptor
-   */
-  @InterfaceAudience.Private
-  public static class ModifyableTableDescriptor
-          implements TableDescriptor, Comparable<ModifyableTableDescriptor> {
+  private static final class ModifyableTableDescriptor
+    implements TableDescriptor, Comparable<ModifyableTableDescriptor> {
 
     private final TableName name;
 
@@ -602,11 +598,9 @@ public class TableDescriptorBuilder {
      * Construct a table descriptor specifying a TableName object
      *
      * @param name Table name.
-     * TODO: make this private after removing the HTableDescriptor
      */
-    @InterfaceAudience.Private
-    public ModifyableTableDescriptor(final TableName name) {
-      this(name, Collections.EMPTY_LIST, Collections.EMPTY_MAP);
+    private ModifyableTableDescriptor(final TableName name) {
+      this(name, Collections.emptyList(), Collections.emptyMap());
     }
 
     private ModifyableTableDescriptor(final TableDescriptor desc) {
@@ -620,11 +614,8 @@ public class TableDescriptorBuilder {
      * Makes a deep copy of the supplied descriptor.
      * @param name The new name
      * @param desc The descriptor.
-     * TODO: make this private after removing the HTableDescriptor
      */
-    @InterfaceAudience.Private
-    @Deprecated // only used by HTableDescriptor. remove this method if HTD is removed
-    public ModifyableTableDescriptor(final TableName name, final TableDescriptor desc) {
+    private ModifyableTableDescriptor(final TableName name, final TableDescriptor desc) {
       this(name, Arrays.asList(desc.getColumnFamilies()), desc.getValues());
     }
 
@@ -697,19 +688,6 @@ public class TableDescriptorBuilder {
     }
 
     /**
-     * Getter for fetching an unmodifiable map.
-     */
-    public Map<String, String> getConfiguration() {
-      return getValues().entrySet().stream()
-        .collect(Collectors.toMap(
-          e -> Bytes.toString(e.getKey().get(), e.getKey().getOffset(),
-            e.getKey().getLength()),
-          e -> Bytes.toString(e.getValue().get(), e.getValue().getOffset(),
-            e.getValue().getLength())
-        ));
-    }
-
-    /**
      * Setter for storing metadata as a (key, value) pair in {@link #values} map
      *
      * @param key The key.
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/TestHColumnDescriptor.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/TestHColumnDescriptor.java
deleted file mode 100644
index 8149ba0..0000000
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/TestHColumnDescriptor.java
+++ /dev/null
@@ -1,148 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
-import org.apache.hadoop.hbase.exceptions.DeserializationException;
-import org.apache.hadoop.hbase.io.compress.Compression;
-import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
-import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
-import org.apache.hadoop.hbase.regionserver.BloomType;
-import org.apache.hadoop.hbase.testclassification.MiscTests;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.apache.hadoop.hbase.util.BuilderStyleTest;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.PrettyPrinter;
-import org.junit.ClassRule;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.rules.ExpectedException;
-
-/**
- * Tests the HColumnDescriptor with appropriate arguments.
- *
- * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 together with
- *            {@link HColumnDescriptor}.
- */
-@Category({MiscTests.class, SmallTests.class})
-@Deprecated
-public class TestHColumnDescriptor {
-
-  @ClassRule
-  public static final HBaseClassTestRule CLASS_RULE =
-      HBaseClassTestRule.forClass(TestHColumnDescriptor.class);
-
-  @Rule
-  public ExpectedException expectedEx = ExpectedException.none();
-  @Test
-  public void testPb() throws DeserializationException {
-    HColumnDescriptor hcd = new HColumnDescriptor(
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(HConstants.CATALOG_FAMILY)
-        .setInMemory(true)
-        .setScope(HConstants.REPLICATION_SCOPE_LOCAL)
-        .setBloomFilterType(BloomType.NONE));
-    final int v = 123;
-    hcd.setBlocksize(v);
-    hcd.setTimeToLive(v);
-    hcd.setBlockCacheEnabled(!HColumnDescriptor.DEFAULT_BLOCKCACHE);
-    hcd.setValue("a", "b");
-    hcd.setMaxVersions(v);
-    assertEquals(v, hcd.getMaxVersions());
-    hcd.setMinVersions(v);
-    assertEquals(v, hcd.getMinVersions());
-    hcd.setKeepDeletedCells(KeepDeletedCells.TRUE);
-    hcd.setInMemory(!HColumnDescriptor.DEFAULT_IN_MEMORY);
-    boolean inmemory = hcd.isInMemory();
-    hcd.setScope(v);
-    hcd.setDataBlockEncoding(DataBlockEncoding.FAST_DIFF);
-    hcd.setBloomFilterType(BloomType.ROW);
-    hcd.setCompressionType(Algorithm.SNAPPY);
-    hcd.setMobEnabled(true);
-    hcd.setMobThreshold(1000L);
-    hcd.setDFSReplication((short) v);
-
-    byte [] bytes = hcd.toByteArray();
-    HColumnDescriptor deserializedHcd = HColumnDescriptor.parseFrom(bytes);
-    assertTrue(hcd.equals(deserializedHcd));
-    assertEquals(v, hcd.getBlocksize());
-    assertEquals(v, hcd.getTimeToLive());
-    assertEquals(v, hcd.getScope());
-    assertEquals(hcd.getValue("a"), deserializedHcd.getValue("a"));
-    assertEquals(hcd.getMaxVersions(), deserializedHcd.getMaxVersions());
-    assertEquals(hcd.getMinVersions(), deserializedHcd.getMinVersions());
-    assertEquals(hcd.getKeepDeletedCells(), deserializedHcd.getKeepDeletedCells());
-    assertEquals(inmemory, deserializedHcd.isInMemory());
-    assertEquals(hcd.getScope(), deserializedHcd.getScope());
-    assertTrue(deserializedHcd.getCompressionType().equals(Compression.Algorithm.SNAPPY));
-    assertTrue(deserializedHcd.getDataBlockEncoding().equals(DataBlockEncoding.FAST_DIFF));
-    assertTrue(deserializedHcd.getBloomFilterType().equals(BloomType.ROW));
-    assertEquals(hcd.isMobEnabled(), deserializedHcd.isMobEnabled());
-    assertEquals(hcd.getMobThreshold(), deserializedHcd.getMobThreshold());
-    assertEquals(v, deserializedHcd.getDFSReplication());
-  }
-
-  /**
-   * Tests HColumnDescriptor with empty familyName
-   */
-  @Test
-  public void testHColumnDescriptorShouldThrowIAEWhenFamilyNameEmpty() throws Exception {
-    expectedEx.expect(IllegalArgumentException.class);
-    expectedEx.expectMessage("Column Family name can not be empty");
-    new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(Bytes.toBytes(""));
-  }
-
-  @Test
-  public void testMobValuesInHColumnDescriptorShouldReadable() {
-    boolean isMob = true;
-    long threshold = 1000;
-    String policy = "weekly";
-    // We unify the format of all values saved in the descriptor.
-    // Each value is stored as bytes of string.
-    String isMobString = PrettyPrinter.format(String.valueOf(isMob),
-            HColumnDescriptor.getUnit(HColumnDescriptor.IS_MOB));
-    String thresholdString = PrettyPrinter.format(String.valueOf(threshold),
-            HColumnDescriptor.getUnit(HColumnDescriptor.MOB_THRESHOLD));
-    String policyString = PrettyPrinter.format(Bytes.toStringBinary(Bytes.toBytes(policy)),
-        HColumnDescriptor.getUnit(HColumnDescriptor.MOB_COMPACT_PARTITION_POLICY));
-    assertEquals(String.valueOf(isMob), isMobString);
-    assertEquals(String.valueOf(threshold), thresholdString);
-    assertEquals(String.valueOf(policy), policyString);
-  }
-
-  @Test
-  public void testClassMethodsAreBuilderStyle() {
-    /* HColumnDescriptor should have a builder style setup where setXXX/addXXX methods
-     * can be chainable together:
-     * . For example:
-     * HColumnDescriptor hcd
-     *   = new HColumnDescriptor()
-     *     .setFoo(foo)
-     *     .setBar(bar)
-     *     .setBuz(buz)
-     *
-     * This test ensures that all methods starting with "set" returns the declaring object
-     */
-
-    BuilderStyleTest.assertClassesAreBuilderStyle(HColumnDescriptor.class);
-  }
-
-}
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/TestHTableDescriptor.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/TestHTableDescriptor.java
deleted file mode 100644
index 4e05a82..0000000
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/TestHTableDescriptor.java
+++ /dev/null
@@ -1,369 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.regex.Pattern;
-
-import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
-import org.apache.hadoop.hbase.client.Durability;
-import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
-import org.apache.hadoop.hbase.exceptions.DeserializationException;
-import org.apache.hadoop.hbase.testclassification.MiscTests;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.apache.hadoop.hbase.util.BuilderStyleTest;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.junit.ClassRule;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.rules.TestName;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Test setting values in the descriptor
- *
- * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 together with
- *             {@link HTableDescriptor}.
- */
-@Category({MiscTests.class, SmallTests.class})
-@Deprecated
-public class TestHTableDescriptor {
-
-  @ClassRule
-  public static final HBaseClassTestRule CLASS_RULE =
-      HBaseClassTestRule.forClass(TestHTableDescriptor.class);
-
-  private static final Logger LOG = LoggerFactory.getLogger(TestHTableDescriptor.class);
-
-  @Rule
-  public TestName name = new TestName();
-
-  @Test (expected=IOException.class)
-  public void testAddCoprocessorTwice() throws IOException {
-    HTableDescriptor htd = new HTableDescriptor(TableName.META_TABLE_NAME);
-    String cpName = "a.b.c.d";
-    htd.addCoprocessor(cpName);
-    htd.addCoprocessor(cpName);
-  }
-
-  @Test
-  public void testAddCoprocessorWithSpecStr() throws IOException {
-    HTableDescriptor htd = new HTableDescriptor(TableName.META_TABLE_NAME);
-    String cpName = "a.b.c.d";
-    try {
-      htd.addCoprocessorWithSpec(cpName);
-      fail();
-    } catch (IllegalArgumentException iae) {
-      // Expected as cpName is invalid
-    }
-
-    // Try minimal spec.
-    try {
-      htd.addCoprocessorWithSpec("file:///some/path" + "|" + cpName);
-      fail();
-    } catch (IllegalArgumentException iae) {
-      // Expected to be invalid
-    }
-
-    // Try more spec.
-    String spec = "hdfs:///foo.jar|com.foo.FooRegionObserver|1001|arg1=1,arg2=2";
-    try {
-      htd.addCoprocessorWithSpec(spec);
-    } catch (IllegalArgumentException iae) {
-      fail();
-    }
-
-    // Try double add of same coprocessor
-    try {
-      htd.addCoprocessorWithSpec(spec);
-      fail();
-    } catch (IOException ioe) {
-      // Expect that the coprocessor already exists
-    }
-  }
-
-  @Test
-  public void testPb() throws DeserializationException, IOException {
-    HTableDescriptor htd = new HTableDescriptor(TableName.META_TABLE_NAME);
-    final int v = 123;
-    htd.setMaxFileSize(v);
-    htd.setDurability(Durability.ASYNC_WAL);
-    htd.setReadOnly(true);
-    htd.setRegionReplication(2);
-    byte [] bytes = htd.toByteArray();
-    HTableDescriptor deserializedHtd = HTableDescriptor.parseFrom(bytes);
-    assertEquals(htd, deserializedHtd);
-    assertEquals(v, deserializedHtd.getMaxFileSize());
-    assertTrue(deserializedHtd.isReadOnly());
-    assertEquals(Durability.ASYNC_WAL, deserializedHtd.getDurability());
-    assertEquals(2, deserializedHtd.getRegionReplication());
-  }
-
-  /**
-   * Test cps in the table description.
-   *
-   * @throws Exception if adding a coprocessor fails
-   */
-  @Test
-  public void testGetSetRemoveCP() throws Exception {
-    HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(name.getMethodName()));
-    // simple CP
-    String className = "org.apache.hadoop.hbase.coprocessor.SimpleRegionObserver";
-    // add and check that it is present
-    desc.addCoprocessor(className);
-    assertTrue(desc.hasCoprocessor(className));
-    // remove it and check that it is gone
-    desc.removeCoprocessor(className);
-    assertFalse(desc.hasCoprocessor(className));
-  }
-
-  /**
-   * Test cps in the table description.
-   *
-   * @throws Exception if adding a coprocessor fails
-   */
-  @Test
-  public void testSetListRemoveCP() throws Exception {
-    HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(name.getMethodName()));
-    // simple CP
-    String className1 = "org.apache.hadoop.hbase.coprocessor.SimpleRegionObserver";
-    String className2 = "org.apache.hadoop.hbase.coprocessor.SampleRegionWALObserver";
-    // Check that any coprocessor is present.
-    assertTrue(desc.getCoprocessors().isEmpty());
-
-    // Add the 1 coprocessor and check if present.
-    desc.addCoprocessor(className1);
-    assertTrue(desc.getCoprocessors().size() == 1);
-    assertTrue(desc.getCoprocessors().contains(className1));
-
-    // Add the 2nd coprocessor and check if present.
-    // remove it and check that it is gone
-    desc.addCoprocessor(className2);
-    assertTrue(desc.getCoprocessors().size() == 2);
-    assertTrue(desc.getCoprocessors().contains(className2));
-
-    // Remove one and check
-    desc.removeCoprocessor(className1);
-    assertTrue(desc.getCoprocessors().size() == 1);
-    assertFalse(desc.getCoprocessors().contains(className1));
-    assertTrue(desc.getCoprocessors().contains(className2));
-
-    // Remove the last and check
-    desc.removeCoprocessor(className2);
-    assertTrue(desc.getCoprocessors().isEmpty());
-    assertFalse(desc.getCoprocessors().contains(className1));
-    assertFalse(desc.getCoprocessors().contains(className2));
-  }
-
-  /**
-   * Test that we add and remove strings from settings properly.
-   */
-  @Test
-  public void testAddGetRemoveString() {
-    HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(name.getMethodName()));
-    String key = "Some";
-    String value = "value";
-    desc.setValue(key, value);
-    assertEquals(value, desc.getValue(key));
-    desc.remove(key);
-    assertEquals(null, desc.getValue(key));
-    String keyShouldNotNull = "Some2";
-    String valueIsNull = null;
-    desc.setValue(keyShouldNotNull, valueIsNull);
-    assertEquals(valueIsNull, desc.getValue(keyShouldNotNull));
-    desc.remove(keyShouldNotNull);
-    assertEquals(null, desc.getValue(keyShouldNotNull));
-  }
-
-  String[] legalTableNames = { "foo", "with-dash_under.dot", "_under_start_ok",
-    "with-dash.with_underscore", "02-01-2012.my_table_01-02", "xyz._mytable_", "9_9_0.table_02",
-    "dot1.dot2.table", "new.-mytable", "with-dash.with.dot", "legal..t2", "legal..legal.t2",
-    "trailingdots..", "trailing.dots...", "ns:mytable", "ns:_mytable_", "ns:my_table_01-02",
-    "汉", "汉:字", "_字_", "foo:字", "foo.字", "字.foo"};
-  // Avoiding "zookeeper" in here as it's tough to encode in regex
-  String[] illegalTableNames = { ".dot_start_illegal", "-dash_start_illegal", "spaces not ok",
-    "-dash-.start_illegal", "new.table with space", "01 .table", "ns:-illegaldash",
-    "new:.illegaldot", "new:illegalcolon1:", "new:illegalcolon1:2", String.valueOf((char)130),
-      String.valueOf((char)5), String.valueOf((char)65530)};
-
-  @Test
-  public void testLegalHTableNames() {
-    for (String tn : legalTableNames) {
-      TableName.isLegalFullyQualifiedTableName(Bytes.toBytes(tn));
-    }
-  }
-
-  @Test
-  public void testIllegalHTableNames() {
-    for (String tn : illegalTableNames) {
-      try {
-        TableName.isLegalFullyQualifiedTableName(Bytes.toBytes(tn));
-        fail("invalid tablename " + tn + " should have failed");
-      } catch (Exception e) {
-        // expected
-      }
-    }
-  }
-
-  @Test
-  public void testIllegalZooKeeperName() {
-    for (String name : Arrays.asList("zookeeper", "ns:zookeeper", "zookeeper:table")) {
-      try {
-        TableName.isLegalFullyQualifiedTableName(Bytes.toBytes(name));
-        fail("invalid tablename " + name + " should have failed");
-      } catch (Exception e) {
-        // expected
-      }
-    }
-  }
-
-  @Test
-  public void testLegalHTableNamesRegex() {
-    for (String tn : legalTableNames) {
-      TableName tName = TableName.valueOf(tn);
-      assertTrue("Testing: '" + tn + "'", Pattern.matches(TableName.VALID_USER_TABLE_REGEX,
-          tName.getNameAsString()));
-    }
-  }
-
-  @Test
-  public void testIllegalHTableNamesRegex() {
-    for (String tn : illegalTableNames) {
-      LOG.info("Testing: '" + tn + "'");
-      assertFalse(Pattern.matches(TableName.VALID_USER_TABLE_REGEX, tn));
-    }
-  }
-
-    /**
-   * Test default value handling for maxFileSize
-   */
-  @Test
-  public void testGetMaxFileSize() {
-    HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(name.getMethodName()));
-    assertEquals(-1, desc.getMaxFileSize());
-    desc.setMaxFileSize(1111L);
-    assertEquals(1111L, desc.getMaxFileSize());
-  }
-
-  /**
-   * Test default value handling for memStoreFlushSize
-   */
-  @Test
-  public void testGetMemStoreFlushSize() {
-    HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(name.getMethodName()));
-    assertEquals(-1, desc.getMemStoreFlushSize());
-    desc.setMemStoreFlushSize(1111L);
-    assertEquals(1111L, desc.getMemStoreFlushSize());
-  }
-
-  /**
-   * Test that we add and remove strings from configuration properly.
-   */
-  @Test
-  public void testAddGetRemoveConfiguration() throws Exception {
-    HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(name.getMethodName()));
-    String key = "Some";
-    String value = "value";
-    desc.setConfiguration(key, value);
-    assertEquals(value, desc.getConfigurationValue(key));
-    desc.removeConfiguration(key);
-    assertEquals(null, desc.getConfigurationValue(key));
-  }
-
-  @Test
-  public void testClassMethodsAreBuilderStyle() {
-    /* HTableDescriptor should have a builder style setup where setXXX/addXXX methods
-     * can be chainable together:
-     * . For example:
-     * HTableDescriptor htd
-     *   = new HTableDescriptor()
-     *     .setFoo(foo)
-     *     .setBar(bar)
-     *     .setBuz(buz)
-     *
-     * This test ensures that all methods starting with "set" returns the declaring object
-     */
-
-    BuilderStyleTest.assertClassesAreBuilderStyle(HTableDescriptor.class);
-  }
-
-  @Test
-  public void testModifyFamily() {
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(
-        TableName.valueOf(name.getMethodName()));
-    byte[] familyName = Bytes.toBytes("cf");
-    ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(familyName)
-        .setBlocksize(1000)
-        .setDFSReplication((short) 3);
-    tableDescriptor.setColumnFamily(familyDescriptor);
-    assertEquals(1000, tableDescriptor.getColumnFamily(familyName).getBlocksize());
-    assertEquals(3, tableDescriptor.getColumnFamily(familyName).getDFSReplication());
-    familyDescriptor =
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(familyName)
-        .setBlocksize(2000)
-        .setDFSReplication((short) 1);
-    tableDescriptor.modifyColumnFamily(familyDescriptor);
-    assertEquals(2000, tableDescriptor.getColumnFamily(familyName).getBlocksize());
-    assertEquals(1, tableDescriptor.getColumnFamily(familyName).getDFSReplication());
-  }
-
-  @Test(expected=IllegalArgumentException.class)
-  public void testModifyInexistentFamily() {
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(
-        TableName.valueOf(name.getMethodName()));
-    byte[] familyName = Bytes.toBytes("cf");
-    ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(familyName);
-    tableDescriptor.modifyColumnFamily(familyDescriptor);
-  }
-
-  @Test(expected=IllegalArgumentException.class)
-  public void testAddDuplicateFamilies() {
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(
-        TableName.valueOf(name.getMethodName()));
-    byte[] familyName = Bytes.toBytes("cf");
-    ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(familyName)
-        .setBlocksize(1000);
-    tableDescriptor.setColumnFamily(familyDescriptor);
-    assertEquals(1000, tableDescriptor.getColumnFamily(familyName).getBlocksize());
-    familyDescriptor =
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(familyName)
-        .setBlocksize(2000);
-    tableDescriptor.setColumnFamily(familyDescriptor);
-  }
-
-  @Test
-  public void testPriority() {
-    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name.getMethodName()));
-    htd.setPriority(42);
-    assertEquals(42, htd.getPriority());
-  }
-}
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestColumnFamilyDescriptorBuilder.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestColumnFamilyDescriptorBuilder.java
index 82479ed..d6ea1b3 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestColumnFamilyDescriptorBuilder.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestColumnFamilyDescriptorBuilder.java
@@ -18,10 +18,10 @@
 package org.apache.hadoop.hbase.client;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThrows;
 import static org.junit.Assert.assertTrue;
 
 import org.apache.hadoop.hbase.HBaseClassTestRule;
-import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeepDeletedCells;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
@@ -37,38 +37,31 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.PrettyPrinter;
 import org.junit.Assert;
 import org.junit.ClassRule;
-import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
-import org.junit.rules.ExpectedException;
 
-@Category({MiscTests.class, SmallTests.class})
+@Category({ MiscTests.class, SmallTests.class })
 public class TestColumnFamilyDescriptorBuilder {
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
-      HBaseClassTestRule.forClass(TestColumnFamilyDescriptorBuilder.class);
-
-  @Rule
-  public ExpectedException expectedEx = ExpectedException.none();
+    HBaseClassTestRule.forClass(TestColumnFamilyDescriptorBuilder.class);
 
   @Test
   public void testBuilder() throws DeserializationException {
-    ColumnFamilyDescriptorBuilder builder
-      = ColumnFamilyDescriptorBuilder.newBuilder(HConstants.CATALOG_FAMILY)
-            .setInMemory(true)
-            .setScope(HConstants.REPLICATION_SCOPE_LOCAL)
-            .setBloomFilterType(BloomType.NONE);
+    ColumnFamilyDescriptorBuilder builder =
+      ColumnFamilyDescriptorBuilder.newBuilder(HConstants.CATALOG_FAMILY).setInMemory(true)
+        .setScope(HConstants.REPLICATION_SCOPE_LOCAL).setBloomFilterType(BloomType.NONE);
     final int v = 123;
     builder.setBlocksize(v);
     builder.setTimeToLive(v);
-    builder.setBlockCacheEnabled(!HColumnDescriptor.DEFAULT_BLOCKCACHE);
+    builder.setBlockCacheEnabled(!ColumnFamilyDescriptorBuilder.DEFAULT_BLOCKCACHE);
     builder.setValue(Bytes.toBytes("a"), Bytes.toBytes("b"));
     builder.setMaxVersions(v);
     assertEquals(v, builder.build().getMaxVersions());
     builder.setMinVersions(v);
     assertEquals(v, builder.build().getMinVersions());
     builder.setKeepDeletedCells(KeepDeletedCells.TRUE);
-    builder.setInMemory(!HColumnDescriptor.DEFAULT_IN_MEMORY);
+    builder.setInMemory(!ColumnFamilyDescriptorBuilder.DEFAULT_IN_MEMORY);
     boolean inmemory = builder.build().isInMemory();
     builder.setScope(v);
     builder.setDataBlockEncoding(DataBlockEncoding.FAST_DIFF);
@@ -79,13 +72,13 @@ public class TestColumnFamilyDescriptorBuilder {
     builder.setDFSReplication((short) v);
 
     ColumnFamilyDescriptor hcd = builder.build();
-    byte [] bytes = ColumnFamilyDescriptorBuilder.toByteArray(hcd);
+    byte[] bytes = ColumnFamilyDescriptorBuilder.toByteArray(hcd);
     ColumnFamilyDescriptor deserializedHcd = ColumnFamilyDescriptorBuilder.parseFrom(bytes);
     assertTrue(hcd.equals(deserializedHcd));
     assertEquals(v, hcd.getBlocksize());
     assertEquals(v, hcd.getTimeToLive());
-    assertTrue(Bytes.equals(hcd.getValue(Bytes.toBytes("a")),
-        deserializedHcd.getValue(Bytes.toBytes("a"))));
+    assertTrue(
+      Bytes.equals(hcd.getValue(Bytes.toBytes("a")), deserializedHcd.getValue(Bytes.toBytes("a"))));
     assertEquals(hcd.getMaxVersions(), deserializedHcd.getMaxVersions());
     assertEquals(hcd.getMinVersions(), deserializedHcd.getMinVersions());
     assertEquals(hcd.getKeepDeletedCells(), deserializedHcd.getKeepDeletedCells());
@@ -104,9 +97,8 @@ public class TestColumnFamilyDescriptorBuilder {
    */
   @Test
   public void testHColumnDescriptorShouldThrowIAEWhenFamilyNameEmpty() throws Exception {
-    expectedEx.expect(IllegalArgumentException.class);
-    expectedEx.expectMessage("Column Family name can not be empty");
-    ColumnFamilyDescriptorBuilder.of("");
+    assertThrows("Column Family name can not be empty", IllegalArgumentException.class,
+      () -> ColumnFamilyDescriptorBuilder.of(""));
   }
 
   /**
@@ -114,8 +106,8 @@ public class TestColumnFamilyDescriptorBuilder {
    */
   @Test
   public void testAddGetRemoveConfiguration() {
-    ColumnFamilyDescriptorBuilder builder
-      = ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes("foo"));
+    ColumnFamilyDescriptorBuilder builder =
+      ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes("foo"));
     String key = "Some";
     String value = "value";
     builder.setConfiguration(key, value);
@@ -132,11 +124,12 @@ public class TestColumnFamilyDescriptorBuilder {
     // We unify the format of all values saved in the descriptor.
     // Each value is stored as bytes of string.
     String isMobString = PrettyPrinter.format(String.valueOf(isMob),
-            HColumnDescriptor.getUnit(HColumnDescriptor.IS_MOB));
+      ColumnFamilyDescriptorBuilder.getUnit(ColumnFamilyDescriptorBuilder.IS_MOB));
     String thresholdString = PrettyPrinter.format(String.valueOf(threshold),
-            HColumnDescriptor.getUnit(HColumnDescriptor.MOB_THRESHOLD));
+      ColumnFamilyDescriptorBuilder.getUnit(ColumnFamilyDescriptorBuilder.MOB_THRESHOLD));
     String policyString = PrettyPrinter.format(Bytes.toStringBinary(Bytes.toBytes(policy)),
-        HColumnDescriptor.getUnit(HColumnDescriptor.MOB_COMPACT_PARTITION_POLICY));
+      ColumnFamilyDescriptorBuilder
+        .getUnit(ColumnFamilyDescriptorBuilder.MOB_COMPACT_PARTITION_POLICY));
     assertEquals(String.valueOf(isMob), isMobString);
     assertEquals(String.valueOf(threshold), thresholdString);
     assertEquals(String.valueOf(policy), policyString);
@@ -144,16 +137,11 @@ public class TestColumnFamilyDescriptorBuilder {
 
   @Test
   public void testClassMethodsAreBuilderStyle() {
-    /* HColumnDescriptor should have a builder style setup where setXXX/addXXX methods
-     * can be chainable together:
-     * . For example:
-     * HColumnDescriptor hcd
-     *   = new HColumnDescriptor()
-     *     .setFoo(foo)
-     *     .setBar(bar)
-     *     .setBuz(buz)
-     *
-     * This test ensures that all methods starting with "set" returns the declaring object
+    /*
+     * ColumnFamilyDescriptorBuilder should have a builder style setup where setXXX/addXXX methods
+     * can be chainable together: . For example: ColumnFamilyDescriptorBuilder builder =
+     * ColumnFamilyDescriptorBuilder.newBuilder() .setFoo(foo) .setBar(bar) .setBuz(buz) This test
+     * ensures that all methods starting with "set" returns the declaring object
      */
 
     BuilderStyleTest.assertClassesAreBuilderStyle(ColumnFamilyDescriptorBuilder.class);
@@ -162,8 +150,8 @@ public class TestColumnFamilyDescriptorBuilder {
   @Test
   public void testSetTimeToLive() throws HBaseException {
     String ttl;
-    ColumnFamilyDescriptorBuilder builder
-      = ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes("foo"));
+    ColumnFamilyDescriptorBuilder builder =
+      ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes("foo"));
 
     ttl = "50000";
     builder.setTimeToLive(ttl);
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestTableDescriptorBuilder.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestTableDescriptorBuilder.java
index 3410e13..89d740c 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestTableDescriptorBuilder.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestTableDescriptorBuilder.java
@@ -42,39 +42,32 @@ import org.slf4j.LoggerFactory;
 /**
  * Test setting values in the descriptor.
  */
-@Category({MiscTests.class, SmallTests.class})
+@Category({ MiscTests.class, SmallTests.class })
 public class TestTableDescriptorBuilder {
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
-      HBaseClassTestRule.forClass(TestTableDescriptorBuilder.class);
+    HBaseClassTestRule.forClass(TestTableDescriptorBuilder.class);
 
   private static final Logger LOG = LoggerFactory.getLogger(TestTableDescriptorBuilder.class);
 
   @Rule
   public TestName name = new TestName();
 
-  @Test (expected=IOException.class)
+  @Test(expected = IOException.class)
   public void testAddCoprocessorTwice() throws IOException {
     String cpName = "a.b.c.d";
-    TableDescriptor htd
-      = TableDescriptorBuilder.newBuilder(TableName.META_TABLE_NAME)
-            .setCoprocessor(cpName)
-            .setCoprocessor(cpName)
-            .build();
+    TableDescriptorBuilder.newBuilder(TableName.META_TABLE_NAME).setCoprocessor(cpName)
+      .setCoprocessor(cpName).build();
   }
 
   @Test
   public void testPb() throws DeserializationException, IOException {
     final int v = 123;
-    TableDescriptor htd
-      = TableDescriptorBuilder.newBuilder(TableName.META_TABLE_NAME)
-          .setMaxFileSize(v)
-          .setDurability(Durability.ASYNC_WAL)
-          .setReadOnly(true)
-          .setRegionReplication(2)
-          .build();
+    TableDescriptor htd =
+      TableDescriptorBuilder.newBuilder(TableName.META_TABLE_NAME).setMaxFileSize(v)
+        .setDurability(Durability.ASYNC_WAL).setReadOnly(true).setRegionReplication(2).build();
 
-    byte [] bytes = TableDescriptorBuilder.toByteArray(htd);
+    byte[] bytes = TableDescriptorBuilder.toByteArray(htd);
     TableDescriptor deserializedHtd = TableDescriptorBuilder.parseFrom(bytes);
     assertEquals(htd, deserializedHtd);
     assertEquals(v, deserializedHtd.getMaxFileSize());
@@ -85,69 +78,66 @@ public class TestTableDescriptorBuilder {
 
   /**
    * Test cps in the table description.
-   *
    * @throws Exception if setting a coprocessor fails
    */
   @Test
   public void testGetSetRemoveCP() throws Exception {
     // simple CP
     String className = "org.apache.hadoop.hbase.coprocessor.SimpleRegionObserver";
-    TableDescriptor desc
-      = TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
-         .setCoprocessor(className) // add and check that it is present
-        .build();
+    TableDescriptor desc = TableDescriptorBuilder
+      .newBuilder(TableName.valueOf(name.getMethodName())).setCoprocessor(className) // add and
+                                                                                     // check that
+                                                                                     // it is
+                                                                                     // present
+      .build();
     assertTrue(desc.hasCoprocessor(className));
-    desc = TableDescriptorBuilder.newBuilder(desc)
-         .removeCoprocessor(className) // remove it and check that it is gone
-        .build();
+    desc = TableDescriptorBuilder.newBuilder(desc).removeCoprocessor(className) // remove it and
+                                                                                // check that it is
+                                                                                // gone
+      .build();
     assertFalse(desc.hasCoprocessor(className));
   }
 
   /**
    * Test cps in the table description.
-   *
    * @throws Exception if setting a coprocessor fails
    */
   @Test
   public void testSetListRemoveCP() throws Exception {
-    TableDescriptor desc
-      = TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName())).build();
+    TableDescriptor desc =
+      TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName())).build();
     // Check that any coprocessor is present.
     assertTrue(desc.getCoprocessorDescriptors().isEmpty());
 
     // simple CP
     String className1 = "org.apache.hadoop.hbase.coprocessor.SimpleRegionObserver";
     String className2 = "org.apache.hadoop.hbase.coprocessor.SampleRegionWALObserver";
-    desc = TableDescriptorBuilder.newBuilder(desc)
-            .setCoprocessor(className1) // Add the 1 coprocessor and check if present.
-            .build();
+    // Add the 1 coprocessor and check if present.
+    desc = TableDescriptorBuilder.newBuilder(desc).setCoprocessor(className1).build();
     assertTrue(desc.getCoprocessorDescriptors().size() == 1);
     assertTrue(desc.getCoprocessorDescriptors().stream().map(CoprocessorDescriptor::getClassName)
       .anyMatch(name -> name.equals(className1)));
-
+    // Add the 2nd coprocessor and check if present.
+    // remove it and check that it is gone
     desc = TableDescriptorBuilder.newBuilder(desc)
-            // Add the 2nd coprocessor and check if present.
-            // remove it and check that it is gone
-            .setCoprocessor(className2)
-            .build();
+
+      .setCoprocessor(className2).build();
     assertTrue(desc.getCoprocessorDescriptors().size() == 2);
     assertTrue(desc.getCoprocessorDescriptors().stream().map(CoprocessorDescriptor::getClassName)
       .anyMatch(name -> name.equals(className2)));
-
+    // Remove one and check
     desc = TableDescriptorBuilder.newBuilder(desc)
-            // Remove one and check
-            .removeCoprocessor(className1)
-            .build();
+
+      .removeCoprocessor(className1).build();
     assertTrue(desc.getCoprocessorDescriptors().size() == 1);
     assertFalse(desc.getCoprocessorDescriptors().stream().map(CoprocessorDescriptor::getClassName)
       .anyMatch(name -> name.equals(className1)));
     assertTrue(desc.getCoprocessorDescriptors().stream().map(CoprocessorDescriptor::getClassName)
       .anyMatch(name -> name.equals(className2)));
-
+    // Remove the last and check
     desc = TableDescriptorBuilder.newBuilder(desc)
-            // Remove the last and check
-            .removeCoprocessor(className2)
-            .build();
+
+      .removeCoprocessor(className2).build();
     assertTrue(desc.getCoprocessorDescriptors().isEmpty());
     assertFalse(desc.getCoprocessorDescriptors().stream().map(CoprocessorDescriptor::getClassName)
       .anyMatch(name -> name.equals(className1)));
@@ -162,24 +152,20 @@ public class TestTableDescriptorBuilder {
   public void testRemoveString() {
     byte[] key = Bytes.toBytes("Some");
     byte[] value = Bytes.toBytes("value");
-    TableDescriptor desc
-      = TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
-            .setValue(key, value)
-            .build();
+    TableDescriptor desc = TableDescriptorBuilder
+      .newBuilder(TableName.valueOf(name.getMethodName())).setValue(key, value).build();
     assertTrue(Bytes.equals(value, desc.getValue(key)));
-    desc = TableDescriptorBuilder.newBuilder(desc)
-            .removeValue(key)
-            .build();
+    desc = TableDescriptorBuilder.newBuilder(desc).removeValue(key).build();
     assertTrue(desc.getValue(key) == null);
   }
 
   String[] legalTableNames = { "foo", "with-dash_under.dot", "_under_start_ok",
     "with-dash.with_underscore", "02-01-2012.my_table_01-02", "xyz._mytable_", "9_9_0.table_02",
     "dot1.dot2.table", "new.-mytable", "with-dash.with.dot", "legal..t2", "legal..legal.t2",
-    "trailingdots..", "trailing.dots...", "ns:mytable", "ns:_mytable_", "ns:my_table_01-02"};
+    "trailingdots..", "trailing.dots...", "ns:mytable", "ns:_mytable_", "ns:my_table_01-02" };
   String[] illegalTableNames = { ".dot_start_illegal", "-dash_start_illegal", "spaces not ok",
     "-dash-.start_illegal", "new.table with space", "01 .table", "ns:-illegaldash",
-    "new:.illegaldot", "new:illegalcolon1:", "new:illegalcolon1:2"};
+    "new:.illegaldot", "new:illegalcolon1:", "new:illegalcolon1:2" };
 
   @Test
   public void testLegalTableNames() {
@@ -204,8 +190,8 @@ public class TestTableDescriptorBuilder {
   public void testLegalTableNamesRegex() {
     for (String tn : legalTableNames) {
       TableName tName = TableName.valueOf(tn);
-      assertTrue("Testing: '" + tn + "'", Pattern.matches(TableName.VALID_USER_TABLE_REGEX,
-          tName.getNameAsString()));
+      assertTrue("Testing: '" + tn + "'",
+        Pattern.matches(TableName.VALID_USER_TABLE_REGEX, tName.getNameAsString()));
     }
   }
 
@@ -217,17 +203,16 @@ public class TestTableDescriptorBuilder {
     }
   }
 
-    /**
+  /**
    * Test default value handling for maxFileSize
    */
   @Test
   public void testGetMaxFileSize() {
-    TableDescriptor desc = TableDescriptorBuilder
-            .newBuilder(TableName.valueOf(name.getMethodName())).build();
+    TableDescriptor desc =
+      TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName())).build();
     assertEquals(-1, desc.getMaxFileSize());
-    desc = TableDescriptorBuilder
-            .newBuilder(TableName.valueOf(name.getMethodName()))
-            .setMaxFileSize(1111L).build();
+    desc = TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+      .setMaxFileSize(1111L).build();
     assertEquals(1111L, desc.getMaxFileSize());
   }
 
@@ -236,12 +221,11 @@ public class TestTableDescriptorBuilder {
    */
   @Test
   public void testGetMemStoreFlushSize() {
-    TableDescriptor desc = TableDescriptorBuilder
-            .newBuilder(TableName.valueOf(name.getMethodName())).build();
+    TableDescriptor desc =
+      TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName())).build();
     assertEquals(-1, desc.getMemStoreFlushSize());
-    desc = TableDescriptorBuilder
-            .newBuilder(TableName.valueOf(name.getMethodName()))
-            .setMemStoreFlushSize(1111L).build();
+    desc = TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+      .setMemStoreFlushSize(1111L).build();
     assertEquals(1111L, desc.getMemStoreFlushSize());
   }
 
@@ -254,51 +238,35 @@ public class TestTableDescriptorBuilder {
   public void testModifyFamily() {
     byte[] familyName = Bytes.toBytes("cf");
     ColumnFamilyDescriptor hcd = ColumnFamilyDescriptorBuilder.newBuilder(familyName)
-            .setBlocksize(1000)
-            .setDFSReplication((short) 3)
-            .build();
-    TableDescriptor htd
-      = TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
-              .setColumnFamily(hcd)
-              .build();
+      .setBlocksize(1000).setDFSReplication((short) 3).build();
+    TableDescriptor htd = TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+      .setColumnFamily(hcd).build();
 
     assertEquals(1000, htd.getColumnFamily(familyName).getBlocksize());
     assertEquals(3, htd.getColumnFamily(familyName).getDFSReplication());
-    hcd = ColumnFamilyDescriptorBuilder.newBuilder(familyName)
-            .setBlocksize(2000)
-            .setDFSReplication((short) 1)
-            .build();
-    htd = TableDescriptorBuilder.newBuilder(htd)
-              .modifyColumnFamily(hcd)
-              .build();
+    hcd = ColumnFamilyDescriptorBuilder.newBuilder(familyName).setBlocksize(2000)
+      .setDFSReplication((short) 1).build();
+    htd = TableDescriptorBuilder.newBuilder(htd).modifyColumnFamily(hcd).build();
     assertEquals(2000, htd.getColumnFamily(familyName).getBlocksize());
     assertEquals(1, htd.getColumnFamily(familyName).getDFSReplication());
   }
 
-  @Test(expected=IllegalArgumentException.class)
+  @Test(expected = IllegalArgumentException.class)
   public void testModifyInexistentFamily() {
     byte[] familyName = Bytes.toBytes("cf");
-    ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(familyName);
-    TableDescriptor htd = TableDescriptorBuilder
-      .newBuilder(TableName.valueOf(name.getMethodName()))
-      .modifyColumnFamily(familyDescriptor)
-      .build();
+    TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+      .modifyColumnFamily(ColumnFamilyDescriptorBuilder.of(familyName)).build();
   }
 
-  @Test(expected=IllegalArgumentException.class)
+  @Test(expected = IllegalArgumentException.class)
   public void testAddDuplicateFamilies() {
     byte[] familyName = Bytes.toBytes("cf");
-    ColumnFamilyDescriptor hcd = ColumnFamilyDescriptorBuilder.newBuilder(familyName)
-            .setBlocksize(1000)
-            .build();
+    ColumnFamilyDescriptor hcd =
+      ColumnFamilyDescriptorBuilder.newBuilder(familyName).setBlocksize(1000).build();
     TableDescriptor htd = TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
-            .setColumnFamily(hcd)
-            .build();
+      .setColumnFamily(hcd).build();
     assertEquals(1000, htd.getColumnFamily(familyName).getBlocksize());
-    hcd = ColumnFamilyDescriptorBuilder.newBuilder(familyName)
-            .setBlocksize(2000)
-            .build();
+    hcd = ColumnFamilyDescriptorBuilder.newBuilder(familyName).setBlocksize(2000).build();
     // add duplicate column
     TableDescriptorBuilder.newBuilder(htd).setColumnFamily(hcd).build();
   }
@@ -306,21 +274,17 @@ public class TestTableDescriptorBuilder {
   @Test
   public void testPriority() {
     TableDescriptor htd = TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
-            .setPriority(42)
-            .build();
+      .setPriority(42).build();
     assertEquals(42, htd.getPriority());
   }
 
   @Test
   public void testStringCustomizedValues() {
     byte[] familyName = Bytes.toBytes("cf");
-    ColumnFamilyDescriptor hcd = ColumnFamilyDescriptorBuilder.newBuilder(familyName)
-            .setBlocksize(1000)
-            .build();
+    ColumnFamilyDescriptor hcd =
+      ColumnFamilyDescriptorBuilder.newBuilder(familyName).setBlocksize(1000).build();
     TableDescriptor htd = TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
-            .setColumnFamily(hcd)
-            .setDurability(Durability.ASYNC_WAL)
-            .build();
+      .setColumnFamily(hcd).setDurability(Durability.ASYNC_WAL).build();
 
     assertEquals(
       "'testStringCustomizedValues', " +
diff --git a/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestBatchCoprocessorEndpoint.java b/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestBatchCoprocessorEndpoint.java
index d4ed2f2..caf4e34 100644
--- a/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestBatchCoprocessorEndpoint.java
+++ b/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestBatchCoprocessorEndpoint.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 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.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.client.coprocessor.Batch;
 import org.apache.hadoop.hbase.testclassification.CoprocessorTests;
@@ -95,11 +96,9 @@ public class TestBatchCoprocessorEndpoint {
     util.startMiniCluster(2);
     Admin admin = util.getAdmin();
 
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(TEST_TABLE);
-    tableDescriptor.setColumnFamily(
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(TEST_FAMILY));
-    admin.createTable(tableDescriptor, new byte[][]{ROWS[rowSeperator1], ROWS[rowSeperator2]});
+    TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(TEST_TABLE)
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(TEST_FAMILY)).build();
+    admin.createTable(tableDescriptor, new byte[][] { ROWS[rowSeperator1], ROWS[rowSeperator2] });
     util.waitUntilAllRegionsAssigned(TEST_TABLE);
     admin.close();
 
diff --git a/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorEndpoint.java b/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorEndpoint.java
index 8e2c9e6..a0b100f 100644
--- a/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorEndpoint.java
+++ b/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorEndpoint.java
@@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionLocator;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.client.coprocessor.Batch;
 import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
@@ -101,11 +102,9 @@ public class TestCoprocessorEndpoint {
     util.startMiniCluster(2);
     Admin admin = util.getAdmin();
 
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(TEST_TABLE);
-    tableDescriptor.setColumnFamily(
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(TEST_FAMILY));
-    admin.createTable(tableDescriptor, new byte[][]{ROWS[rowSeperator1], ROWS[rowSeperator2]});
+    TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(TEST_TABLE)
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(TEST_FAMILY)).build();
+    admin.createTable(tableDescriptor, new byte[][] { ROWS[rowSeperator1], ROWS[rowSeperator2] });
     util.waitUntilAllRegionsAssigned(TEST_TABLE);
 
     Table table = util.getConnection().getTable(TEST_TABLE);
diff --git a/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorTableEndpoint.java b/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorTableEndpoint.java
index 24ec48f..9704404 100644
--- a/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorTableEndpoint.java
+++ b/hbase-endpoint/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorTableEndpoint.java
@@ -25,10 +25,10 @@ import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 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.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.client.coprocessor.Batch;
 import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
@@ -81,13 +81,9 @@ public class TestCoprocessorTableEndpoint {
   public void testCoprocessorTableEndpoint() throws Throwable {
     final TableName tableName = TableName.valueOf(name.getMethodName());
 
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
-    ColumnFamilyDescriptor familyDescriptor =
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(TEST_FAMILY);
-
-    tableDescriptor.setColumnFamily(familyDescriptor);
-    tableDescriptor.setCoprocessor(ColumnAggregationEndpoint.class.getName());
+    TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName)
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(TEST_FAMILY))
+      .setCoprocessor(ColumnAggregationEndpoint.class.getName()).build();
 
     createTable(tableDescriptor);
     verifyTable(tableName);
@@ -97,16 +93,12 @@ public class TestCoprocessorTableEndpoint {
   public void testDynamicCoprocessorTableEndpoint() throws Throwable {
     final TableName tableName = TableName.valueOf(name.getMethodName());
 
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
-    ColumnFamilyDescriptor familyDescriptor =
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(TEST_FAMILY);
-
-    tableDescriptor.setColumnFamily(familyDescriptor);
+    TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName)
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(TEST_FAMILY)).build();
     createTable(tableDescriptor);
 
-    tableDescriptor.setCoprocessor(ColumnAggregationEndpoint.class.getName());
-    updateTable(tableDescriptor);
+    updateTable(TableDescriptorBuilder.newBuilder(tableDescriptor)
+      .setCoprocessor(ColumnAggregationEndpoint.class.getName()).build());
 
     verifyTable(tableName);
   }
@@ -142,10 +134,9 @@ public class TestCoprocessorTableEndpoint {
       });
   }
 
-  private static final void createTable(
-      final TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor) throws Exception {
+  private static final void createTable(final TableDescriptor tableDescriptor) throws Exception {
     Admin admin = TEST_UTIL.getAdmin();
-    admin.createTable(tableDescriptor, new byte[][]{ROWS[rowSeperator1], ROWS[rowSeperator2]});
+    admin.createTable(tableDescriptor, new byte[][] { ROWS[rowSeperator1], ROWS[rowSeperator2] });
     TEST_UTIL.waitUntilAllRegionsAssigned(tableDescriptor.getTableName());
     Table table = TEST_UTIL.getConnection().getTable(tableDescriptor.getTableName());
     try {
@@ -159,8 +150,7 @@ public class TestCoprocessorTableEndpoint {
     }
   }
 
-  private static void updateTable(
-      final TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor) throws Exception {
+  private static void updateTable(TableDescriptor tableDescriptor) throws Exception {
     Admin admin = TEST_UTIL.getAdmin();
     admin.disableTable(tableDescriptor.getTableName());
     admin.modifyTable(tableDescriptor);
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngestStripeCompactions.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngestStripeCompactions.java
index feca9d0..0d511c9 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngestStripeCompactions.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngestStripeCompactions.java
@@ -19,9 +19,10 @@
 package org.apache.hadoop.hbase;
 
 import java.io.IOException;
-
 import org.apache.hadoop.conf.Configuration;
+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.regionserver.HStore;
 import org.apache.hadoop.hbase.regionserver.StoreEngine;
@@ -40,14 +41,11 @@ public class IntegrationTestIngestStripeCompactions extends IntegrationTestInges
   @Override
   protected void initTable() throws IOException {
     // Do the same as the LoadTestTool does, but with different table configuration.
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(getTablename());
-    tableDescriptor.setValue(StoreEngine.STORE_ENGINE_CLASS_KEY,
-      StripeStoreEngine.class.getName());
-    tableDescriptor.setValue(HStore.BLOCKING_STOREFILES_KEY, "100");
-    ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(
-        HFileTestUtil.DEFAULT_COLUMN_FAMILY);
+    TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(getTablename())
+      .setValue(StoreEngine.STORE_ENGINE_CLASS_KEY, StripeStoreEngine.class.getName())
+      .setValue(HStore.BLOCKING_STOREFILES_KEY, "100").build();
+    ColumnFamilyDescriptor familyDescriptor =
+      ColumnFamilyDescriptorBuilder.of(HFileTestUtil.DEFAULT_COLUMN_FAMILY);
     HBaseTestingUtility.createPreSplitLoadTestTable(util.getConfiguration(),
       tableDescriptor, familyDescriptor);
   }
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngestWithMOB.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngestWithMOB.java
index 1bd91fe..2b8c2fb 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngestWithMOB.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngestWithMOB.java
@@ -21,13 +21,13 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.testclassification.IntegrationTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.HFileTestUtil;
@@ -120,14 +120,12 @@ public class IntegrationTestIngestWithMOB extends IntegrationTestIngest {
     TableName tableName = getTablename();
     try (Connection connection = ConnectionFactory.createConnection();
          Admin admin = connection.getAdmin()) {
-      HTableDescriptor tableDesc = new HTableDescriptor(admin.getDescriptor(tableName));
+      TableDescriptor tableDesc = admin.getDescriptor(tableName);
       LOG.info("Disabling table " + getTablename());
       admin.disableTable(tableName);
       ColumnFamilyDescriptor mobColumn = tableDesc.getColumnFamily(mobColumnFamily);
       ColumnFamilyDescriptor cfd = ColumnFamilyDescriptorBuilder.newBuilder(mobColumn)
-        .setMobEnabled(true)
-        .setMobThreshold((long) threshold)
-        .build();
+        .setMobEnabled(true).setMobThreshold((long) threshold).build();
       admin.modifyColumnFamily(tableName, cfd);
       LOG.info("Enabling table " + getTablename());
       admin.enableTable(tableName);
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestLazyCfLoading.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestLazyCfLoading.java
index 8deccd4..376bc56 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestLazyCfLoading.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestLazyCfLoading.java
@@ -24,6 +24,7 @@ import java.util.Set;
 import java.util.TreeMap;
 import java.util.concurrent.atomic.AtomicLong;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
@@ -189,18 +190,16 @@ public class IntegrationTestLazyCfLoading {
     Configuration conf = util.getConfiguration();
     String encodingKey = String.format(ENCODING_KEY, this.getClass().getSimpleName());
     DataBlockEncoding blockEncoding = DataBlockEncoding.valueOf(conf.get(encodingKey, "FAST_DIFF"));
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(TABLE_NAME);
+    TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(TABLE_NAME);
     for (byte[] cf : dataGen.getColumnFamilies()) {
-      ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
-        new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(cf);
-      familyDescriptor.setDataBlockEncoding(blockEncoding);
-      tableDescriptor.setColumnFamily(familyDescriptor);
+      ColumnFamilyDescriptor familyDescriptor =
+        ColumnFamilyDescriptorBuilder.newBuilder(cf).setDataBlockEncoding(blockEncoding).build();
+      builder.setColumnFamily(familyDescriptor);
     }
     int serverCount = util.getHBaseClusterInterface().getClusterMetrics()
       .getLiveServerMetrics().size();
     byte[][] splits = new RegionSplitter.HexStringSplit().split(serverCount * REGIONS_PER_SERVER);
-    util.getAdmin().createTable(tableDescriptor, splits);
+    util.getAdmin().createTable(builder.build(), splits);
     LOG.info("Created table");
   }
 
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestMobCompaction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestMobCompaction.java
index 5e8cd80..d368f30 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestMobCompaction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestMobCompaction.java
@@ -24,29 +24,27 @@ import static org.junit.Assert.assertTrue;
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.Set;
-
 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.client.Admin;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.master.cleaner.TimeToLiveHFileCleaner;
 import org.apache.hadoop.hbase.mob.FaultyMobStoreCompactor;
 import org.apache.hadoop.hbase.mob.MobConstants;
 import org.apache.hadoop.hbase.mob.MobFileCleanerChore;
 import org.apache.hadoop.hbase.mob.MobStoreEngine;
 import org.apache.hadoop.hbase.mob.MobUtils;
-
 import org.apache.hadoop.hbase.testclassification.IntegrationTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.util.ToolRunner;
-
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -98,8 +96,8 @@ public class IntegrationTestMobCompaction extends IntegrationTestBase {
       .toBytes("01234567890123456789012345678901234567890123456789012345678901234567890123456789");
 
   private static Configuration conf;
-  private static TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor;
-  private static ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor;
+  private static TableDescriptor tableDescriptor;
+  private static ColumnFamilyDescriptor familyDescriptor;
   private static Admin admin;
   private static Table table = null;
   private static MobFileCleanerChore chore;
@@ -126,12 +124,10 @@ public class IntegrationTestMobCompaction extends IntegrationTestBase {
 
   private void createTestTable() throws IOException {
     // Create test table
-    tableDescriptor = util.createModifyableTableDescriptor("testMobCompactTable");
-    familyDescriptor = new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(fam);
-    familyDescriptor.setMobEnabled(true);
-    familyDescriptor.setMobThreshold(mobLen);
-    familyDescriptor.setMaxVersions(1);
-    tableDescriptor.setColumnFamily(familyDescriptor);
+    familyDescriptor = ColumnFamilyDescriptorBuilder.newBuilder(fam).setMobEnabled(true)
+      .setMobThreshold(mobLen).setMaxVersions(1).build();
+    tableDescriptor = util.createModifyableTableDescriptor("testMobCompactTable")
+      .setColumnFamily(familyDescriptor).build();
     table = util.createTable(tableDescriptor, null);
   }
 
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/StripeCompactionsPerformanceEvaluation.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/StripeCompactionsPerformanceEvaluation.java
index 3e94a19..7f12d07 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/StripeCompactionsPerformanceEvaluation.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/StripeCompactionsPerformanceEvaluation.java
@@ -23,7 +23,6 @@ import java.util.Locale;
 import java.util.Set;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.regionserver.DisabledRegionSplitPolicy;
@@ -267,48 +266,41 @@ public class StripeCompactionsPerformanceEvaluation extends AbstractHBaseTool {
     System.out.println(s);
   }
 
-  private TableDescriptorBuilder.ModifyableTableDescriptor createHtd(boolean isStripe)
-      throws Exception {
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(TABLE_NAME);
-    ColumnFamilyDescriptor familyDescriptor =
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(COLUMN_FAMILY);
-    tableDescriptor.setColumnFamily(familyDescriptor);
+  private TableDescriptorBuilder createHtd(boolean isStripe) throws Exception {
+    TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(TABLE_NAME)
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(COLUMN_FAMILY));
     String noSplitsPolicy = DisabledRegionSplitPolicy.class.getName();
-    tableDescriptor.setValue(HConstants.HBASE_REGION_SPLIT_POLICY_KEY, noSplitsPolicy);
+    builder.setValue(HConstants.HBASE_REGION_SPLIT_POLICY_KEY, noSplitsPolicy);
     if (isStripe) {
-      tableDescriptor.setValue(StoreEngine.STORE_ENGINE_CLASS_KEY,
-        StripeStoreEngine.class.getName());
+      builder.setValue(StoreEngine.STORE_ENGINE_CLASS_KEY, StripeStoreEngine.class.getName());
       if (initialStripeCount != null) {
-        tableDescriptor.setValue(
-            StripeStoreConfig.INITIAL_STRIPE_COUNT_KEY, initialStripeCount.toString());
-        tableDescriptor.setValue(
-            HStore.BLOCKING_STOREFILES_KEY, Long.toString(10 * initialStripeCount));
+        builder.setValue(StripeStoreConfig.INITIAL_STRIPE_COUNT_KEY, initialStripeCount.toString());
+        builder.setValue(HStore.BLOCKING_STOREFILES_KEY, Long.toString(10 * initialStripeCount));
       } else {
-        tableDescriptor.setValue(HStore.BLOCKING_STOREFILES_KEY, "500");
+        builder.setValue(HStore.BLOCKING_STOREFILES_KEY, "500");
       }
       if (splitSize != null) {
-        tableDescriptor.setValue(StripeStoreConfig.SIZE_TO_SPLIT_KEY, splitSize.toString());
+        builder.setValue(StripeStoreConfig.SIZE_TO_SPLIT_KEY, splitSize.toString());
       }
       if (splitParts != null) {
-        tableDescriptor.setValue(StripeStoreConfig.SPLIT_PARTS_KEY, splitParts.toString());
+        builder.setValue(StripeStoreConfig.SPLIT_PARTS_KEY, splitParts.toString());
       }
     } else {
-      tableDescriptor.setValue(HStore.BLOCKING_STOREFILES_KEY, "10"); // default
+      builder.setValue(HStore.BLOCKING_STOREFILES_KEY, "10"); // default
     }
-    return tableDescriptor;
+    return builder;
   }
 
-  protected void createTable(TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor)
+  private void createTable(TableDescriptorBuilder builder)
       throws Exception {
     deleteTable();
     if (util.getHBaseClusterInterface() instanceof MiniHBaseCluster) {
       LOG.warn("Test does not make a lot of sense for minicluster. Will set flush size low.");
-      tableDescriptor.setValue(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, "1048576");
+      builder.setValue(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, "1048576");
     }
     byte[][] splits = new RegionSplitter.HexStringSplit().split(
         util.getHBaseClusterInterface().getClusterMetrics().getLiveServerMetrics().size());
-    util.getAdmin().createTable(tableDescriptor, splits);
+    util.getAdmin().createTable(builder.build(), splits);
   }
 
   public static class SeqShardedDataGenerator extends LoadTestDataGenerator {
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java
index ce2314e..1d30e17 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java
@@ -64,6 +64,7 @@ import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.fs.HFileSystem;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
@@ -756,24 +757,20 @@ public class IntegrationTestBigLinkedList extends IntegrationTestBase {
       try (Connection conn = ConnectionFactory.createConnection(conf);
           Admin admin = conn.getAdmin()) {
         if (!admin.tableExists(tableName)) {
-          TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-            new TableDescriptorBuilder.ModifyableTableDescriptor(getTableName(getConf()));
-
-          ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
-            new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY_NAME);
-          // if -DuseMob=true force all data through mob path.
-          setMobProperties(conf, familyDescriptor);
-          tableDescriptor.setColumnFamily(familyDescriptor);
-          // Always add these families. Just skip writing to them when we do not test per CF flush.
-          familyDescriptor =
-            new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(BIG_FAMILY_NAME);
-          setMobProperties(conf, familyDescriptor);
-          tableDescriptor.setColumnFamily(familyDescriptor);
-
-          familyDescriptor =
-            new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(TINY_FAMILY_NAME);
-          setMobProperties(conf, familyDescriptor);
-          tableDescriptor.setColumnFamily(familyDescriptor);
+          TableDescriptor tableDescriptor = TableDescriptorBuilder
+            .newBuilder(getTableName(getConf()))
+            // if -DuseMob=true force all data through mob path.
+            .setColumnFamily(
+              setMobProperties(conf, ColumnFamilyDescriptorBuilder.newBuilder(FAMILY_NAME)).build())
+            // Always add these families. Just skip writing to them when we do not test per CF
+            // flush.
+            .setColumnFamily(
+              setMobProperties(conf, ColumnFamilyDescriptorBuilder.newBuilder(BIG_FAMILY_NAME))
+                .build())
+            .setColumnFamily(
+              setMobProperties(conf, ColumnFamilyDescriptorBuilder.newBuilder(TINY_FAMILY_NAME))
+                .build())
+            .build();
 
           // If we want to pre-split compute how many splits.
           if (conf.getBoolean(HBaseTestingUtility.PRESPLIT_TEST_TABLE_KEY,
@@ -914,12 +911,13 @@ public class IntegrationTestBigLinkedList extends IntegrationTestBase {
     }
   }
 
-  private static void setMobProperties(final Configuration conf,
-      final ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor) {
+  private static ColumnFamilyDescriptorBuilder setMobProperties(final Configuration conf,
+    final ColumnFamilyDescriptorBuilder builder) {
     if (conf.getBoolean("useMob", false)) {
-      familyDescriptor.setMobEnabled(true);
-      familyDescriptor.setMobThreshold(4);
+      builder.setMobEnabled(true);
+      builder.setMobThreshold(4);
     }
+    return builder;
   }
 
   /**
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedListWithVisibility.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedListWithVisibility.java
index 8550f4e..513a5ae 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedListWithVisibility.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedListWithVisibility.java
@@ -42,6 +42,7 @@ import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.log.HBaseMarkers;
@@ -147,14 +148,13 @@ public class IntegrationTestBigLinkedListWithVisibility extends IntegrationTestB
     private void createTable(Admin admin, TableName tableName, boolean setVersion,
         boolean acl) throws IOException {
       if (!admin.tableExists(tableName)) {
-        TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-          new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
-        ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
-          new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY_NAME);
+        ColumnFamilyDescriptorBuilder cfBuilder =
+          ColumnFamilyDescriptorBuilder.newBuilder(FAMILY_NAME);
         if (setVersion) {
-          familyDescriptor.setMaxVersions(DEFAULT_TABLES_COUNT);
+          cfBuilder.setMaxVersions(DEFAULT_TABLES_COUNT);
         }
-        tableDescriptor.setColumnFamily(familyDescriptor);
+        TableDescriptor tableDescriptor =
+          TableDescriptorBuilder.newBuilder(tableName).setColumnFamily(cfBuilder.build()).build();
         admin.createTable(tableDescriptor);
         if (acl) {
           LOG.info("Granting permissions for user " + USER.getShortName());
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestLoadAndVerify.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestLoadAndVerify.java
index 6fc4ba7..29a7160 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestLoadAndVerify.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestLoadAndVerify.java
@@ -533,11 +533,9 @@ public void cleanUpCluster() throws Exception {
 
   @Test
   public void testLoadAndVerify() throws Exception {
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(TableName.valueOf(TEST_NAME));
-
-    tableDescriptor.setColumnFamily(
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(TEST_FAMILY));
+    TableDescriptor tableDescriptor =
+      TableDescriptorBuilder.newBuilder(TableName.valueOf(TEST_NAME))
+        .setColumnFamily(ColumnFamilyDescriptorBuilder.of(TEST_FAMILY)).build();
 
     Admin admin = getTestingUtil(getConf()).getAdmin();
     admin.createTable(tableDescriptor, Bytes.toBytes(0L), Bytes.toBytes(-1L), 40);
@@ -617,10 +615,8 @@ public void cleanUpCluster() throws Exception {
 
     // create HTableDescriptor for specified table
     TableName table = getTablename();
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(table);
-    tableDescriptor.setColumnFamily(
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(TEST_FAMILY));
+    TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(table)
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(TEST_FAMILY)).build();
 
     if (doLoad) {
       try (Connection conn = ConnectionFactory.createConnection(getConf());
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestWithCellVisibilityLoadAndVerify.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestWithCellVisibilityLoadAndVerify.java
index 6c71057..e196aa7 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestWithCellVisibilityLoadAndVerify.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestWithCellVisibilityLoadAndVerify.java
@@ -365,10 +365,8 @@ public class IntegrationTestWithCellVisibilityLoadAndVerify extends IntegrationT
     IntegrationTestingUtility.setUseDistributedCluster(getConf());
     int numPresplits = getConf().getInt("loadmapper.numPresplits", 5);
     // create HTableDescriptor for specified table
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(getTablename());
-    tableDescriptor.setColumnFamily(
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(TEST_FAMILY));
+    TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(getTablename())
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(TEST_FAMILY)).build();
 
     try (Connection conn = ConnectionFactory.createConnection(getConf());
         Admin admin = conn.getAdmin()) {
diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapred/TableRecordReader.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapred/TableRecordReader.java
index d56e181..3d41d8c 100644
--- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapred/TableRecordReader.java
+++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapred/TableRecordReader.java
@@ -57,7 +57,7 @@ implements RecordReader<ImmutableBytesWritable, Result> {
   }
 
   /**
-   * @param htable the {@link org.apache.hadoop.hbase.HTableDescriptor} to scan.
+   * @param htable the table to scan.
    */
   public void setHTable(Table htable) {
     this.recordReaderImpl.setHTable(htable);
diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapred/TableRecordReaderImpl.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapred/TableRecordReaderImpl.java
index 6328b5e..3455d6a 100644
--- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapred/TableRecordReaderImpl.java
+++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapred/TableRecordReaderImpl.java
@@ -104,13 +104,13 @@ public class TableRecordReaderImpl {
   byte[] getStartRow() {
     return this.startRow;
   }
+
   /**
-   * @param htable the {@link org.apache.hadoop.hbase.HTableDescriptor} to scan.
+   * @param htable the table to scan.
    */
   public void setHTable(Table htable) {
     Configuration conf = htable.getConfiguration();
-    logScannerActivity = conf.getBoolean(
-      ConnectionConfiguration.LOG_SCANNER_ACTIVITY, false);
+    logScannerActivity = conf.getBoolean(ConnectionConfiguration.LOG_SCANNER_ACTIVITY, false);
     logPerRowCount = conf.getInt(LOG_PER_ROW_COUNT, 100);
     this.htable = htable;
   }
diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/ImportTsv.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/ImportTsv.java
index 5cb8e85..9b71479 100644
--- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/ImportTsv.java
+++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/ImportTsv.java
@@ -26,7 +26,6 @@ import java.util.ArrayList;
 import java.util.Base64;
 import java.util.HashSet;
 import java.util.Set;
-
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
@@ -36,17 +35,16 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotEnabledException;
 import org.apache.hadoop.hbase.TableNotFoundException;
-import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
-import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 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.RegionLocator;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
@@ -59,9 +57,10 @@ import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
-import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
-import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
 import org.apache.hbase.thirdparty.com.google.common.base.Splitter;
 import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
@@ -596,17 +595,15 @@ public class ImportTsv extends Configured implements Tool {
 
   private static void createTable(Admin admin, TableName tableName, String[] columns)
       throws IOException {
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
+    TableDescriptorBuilder builder =
+      TableDescriptorBuilder.newBuilder(tableName);
     Set<String> cfSet = getColumnFamilies(columns);
     for (String cf : cfSet) {
-      ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
-        new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(Bytes.toBytes(cf));
-      tableDescriptor.setColumnFamily(familyDescriptor);
+      builder.setColumnFamily(ColumnFamilyDescriptorBuilder.of(cf));
     }
     LOG.warn(format("Creating table '%s' with '%s' columns and default descriptors.",
       tableName, cfSet));
-    admin.createTable(tableDescriptor);
+    admin.createTable(builder.build());
   }
 
   private static void deleteTable(Configuration conf, String[] args) {
diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableRecordReaderImpl.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableRecordReaderImpl.java
index 8aef458..21589fd 100644
--- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableRecordReaderImpl.java
+++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableRecordReaderImpl.java
@@ -118,13 +118,11 @@ public class TableRecordReaderImpl {
 
   /**
    * Sets the HBase table.
-   *
-   * @param htable  The {@link org.apache.hadoop.hbase.HTableDescriptor} to scan.
+   * @param htable The table to scan.
    */
   public void setHTable(Table htable) {
     Configuration conf = htable.getConfiguration();
-    logScannerActivity = conf.getBoolean(
-      ConnectionConfiguration.LOG_SCANNER_ACTIVITY, false);
+    logScannerActivity = conf.getBoolean(ConnectionConfiguration.LOG_SCANNER_ACTIVITY, false);
     logPerRowCount = conf.getInt(LOG_PER_ROW_COUNT, 100);
     this.htable = htable;
   }
diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
index b7f3a3e..134ffa6 100644
--- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
+++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
@@ -400,30 +400,30 @@ public class PerformanceEvaluation extends Configured implements Tool {
    * Create an HTableDescriptor from provided TestOptions.
    */
   protected static TableDescriptor getTableDescriptor(TestOptions opts) {
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(TableName.valueOf(opts.tableName));
+    TableDescriptorBuilder builder =
+      TableDescriptorBuilder.newBuilder(TableName.valueOf(opts.tableName));
 
     for (int family = 0; family < opts.families; family++) {
       byte[] familyName = Bytes.toBytes(FAMILY_NAME_BASE + family);
-      ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDesc =
-        new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(familyName);
-      familyDesc.setDataBlockEncoding(opts.blockEncoding);
-      familyDesc.setCompressionType(opts.compression);
-      familyDesc.setBloomFilterType(opts.bloomType);
-      familyDesc.setBlocksize(opts.blockSize);
+      ColumnFamilyDescriptorBuilder cfBuilder =
+        ColumnFamilyDescriptorBuilder.newBuilder(familyName);
+      cfBuilder.setDataBlockEncoding(opts.blockEncoding);
+      cfBuilder.setCompressionType(opts.compression);
+      cfBuilder.setBloomFilterType(opts.bloomType);
+      cfBuilder.setBlocksize(opts.blockSize);
       if (opts.inMemoryCF) {
-        familyDesc.setInMemory(true);
+        cfBuilder.setInMemory(true);
       }
-      familyDesc.setInMemoryCompaction(opts.inMemoryCompaction);
-      tableDescriptor.setColumnFamily(familyDesc);
+      cfBuilder.setInMemoryCompaction(opts.inMemoryCompaction);
+      builder.setColumnFamily(cfBuilder.build());
     }
     if (opts.replicas != DEFAULT_OPTS.replicas) {
-      tableDescriptor.setRegionReplication(opts.replicas);
+      builder.setRegionReplication(opts.replicas);
     }
     if (opts.splitPolicy != null && !opts.splitPolicy.equals(DEFAULT_OPTS.splitPolicy)) {
-      tableDescriptor.setRegionSplitPolicyClassName(opts.splitPolicy);
+      builder.setRegionSplitPolicyClassName(opts.splitPolicy);
     }
-    return tableDescriptor;
+    return builder.build();
   }
 
   /**
@@ -692,7 +692,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
     int randomSleep = 0;
     boolean inMemoryCF = false;
     int presplitRegions = 0;
-    int replicas = HTableDescriptor.DEFAULT_REGION_REPLICATION;
+    int replicas = TableDescriptorBuilder.DEFAULT_REGION_REPLICATION;
     String splitPolicy = null;
     Compression.Algorithm compression = Compression.Algorithm.NONE;
     BloomType bloomType = BloomType.ROW;
diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTimeRangeMapRed.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTimeRangeMapRed.java
index b5fa1e5..b7c2f21 100644
--- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTimeRangeMapRed.java
+++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTimeRangeMapRed.java
@@ -42,6 +42,7 @@ import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
@@ -150,13 +151,10 @@ public class TestTimeRangeMapRed {
 
   @Test
   public void testTimeRangeMapRed()
-      throws IOException, InterruptedException, ClassNotFoundException {
-    final TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(TABLE_NAME);
-    final ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY_NAME);
-    familyDescriptor.setMaxVersions(Integer.MAX_VALUE);
-    tableDescriptor.setColumnFamily(familyDescriptor);
+    throws IOException, InterruptedException, ClassNotFoundException {
+    final TableDescriptor tableDescriptor =
+      TableDescriptorBuilder.newBuilder(TABLE_NAME).setColumnFamily(ColumnFamilyDescriptorBuilder
+        .newBuilder(FAMILY_NAME).setMaxVersions(Integer.MAX_VALUE).build()).build();
     admin.createTable(tableDescriptor);
     List<Put> puts = new ArrayList<>();
     for (Map.Entry<Long, Boolean> entry : TIMESTAMP.entrySet()) {
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/SchemaResource.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/SchemaResource.java
index bef16c5..2194f54 100644
--- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/SchemaResource.java
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/SchemaResource.java
@@ -32,7 +32,6 @@ import javax.ws.rs.core.Response;
 import javax.ws.rs.core.Response.ResponseBuilder;
 import javax.ws.rs.core.UriInfo;
 import javax.xml.namespace.QName;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableExistsException;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotEnabledException;
@@ -71,9 +70,9 @@ public class SchemaResource extends ResourceBase {
     this.tableResource = tableResource;
   }
 
-  private HTableDescriptor getTableSchema() throws IOException, TableNotFoundException {
+  private TableDescriptor getTableSchema() throws IOException, TableNotFoundException {
     try (Table table = servlet.getTable(tableResource.getName())) {
-      return new HTableDescriptor(table.getDescriptor());
+      return table.getDescriptor();
     }
   }
 
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/model/ColumnSchemaModel.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/model/ColumnSchemaModel.java
index 967f6ba..6de4b77 100644
--- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/model/ColumnSchemaModel.java
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/model/ColumnSchemaModel.java
@@ -19,21 +19,18 @@
 
 package org.apache.hadoop.hbase.rest.model;
 
+import com.fasterxml.jackson.annotation.JsonAnyGetter;
+import com.fasterxml.jackson.annotation.JsonAnySetter;
 import java.io.Serializable;
 import java.util.LinkedHashMap;
 import java.util.Map;
-
 import javax.xml.bind.annotation.XmlAnyAttribute;
 import javax.xml.bind.annotation.XmlAttribute;
 import javax.xml.bind.annotation.XmlRootElement;
 import javax.xml.namespace.QName;
-
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
-
-import com.fasterxml.jackson.annotation.JsonAnyGetter;
-import com.fasterxml.jackson.annotation.JsonAnySetter;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Representation of a column family schema.
@@ -49,12 +46,12 @@ import com.fasterxml.jackson.annotation.JsonAnySetter;
 @InterfaceAudience.Private
 public class ColumnSchemaModel implements Serializable {
   private static final long serialVersionUID = 1L;
-  private static QName BLOCKCACHE = new QName(HColumnDescriptor.BLOCKCACHE);
-  private static QName BLOCKSIZE = new QName(HColumnDescriptor.BLOCKSIZE);
-  private static QName BLOOMFILTER = new QName(HColumnDescriptor.BLOOMFILTER);
-  private static QName COMPRESSION = new QName(HColumnDescriptor.COMPRESSION);
+  private static QName BLOCKCACHE = new QName(ColumnFamilyDescriptorBuilder.BLOCKCACHE);
+  private static QName BLOCKSIZE = new QName(ColumnFamilyDescriptorBuilder.BLOCKSIZE);
+  private static QName BLOOMFILTER = new QName(ColumnFamilyDescriptorBuilder.BLOOMFILTER);
+  private static QName COMPRESSION = new QName(ColumnFamilyDescriptorBuilder.COMPRESSION);
   private static QName IN_MEMORY = new QName(HConstants.IN_MEMORY);
-  private static QName TTL = new QName(HColumnDescriptor.TTL);
+  private static QName TTL = new QName(ColumnFamilyDescriptorBuilder.TTL);
   private static QName VERSIONS = new QName(HConstants.VERSIONS);
 
   private String name;
@@ -138,8 +135,8 @@ public class ColumnSchemaModel implements Serializable {
    */
   public boolean __getBlockcache() {
     Object o = attrs.get(BLOCKCACHE);
-    return o != null ? 
-      Boolean.parseBoolean(o.toString()) : HColumnDescriptor.DEFAULT_BLOCKCACHE;
+    return o != null ? Boolean.parseBoolean(o.toString()) :
+      ColumnFamilyDescriptorBuilder.DEFAULT_BLOCKCACHE;
   }
 
   /**
@@ -147,8 +144,8 @@ public class ColumnSchemaModel implements Serializable {
    */
   public int __getBlocksize() {
     Object o = attrs.get(BLOCKSIZE);
-    return o != null ? 
-      Integer.parseInt(o.toString()) : HColumnDescriptor.DEFAULT_BLOCKSIZE;
+    return o != null ? Integer.parseInt(o.toString()) :
+      ColumnFamilyDescriptorBuilder.DEFAULT_BLOCKSIZE;
   }
 
   /**
@@ -156,7 +153,7 @@ public class ColumnSchemaModel implements Serializable {
    */
   public String __getBloomfilter() {
     Object o = attrs.get(BLOOMFILTER);
-    return o != null ? o.toString() : HColumnDescriptor.DEFAULT_BLOOMFILTER;
+    return o != null ? o.toString() : ColumnFamilyDescriptorBuilder.DEFAULT_BLOOMFILTER.name();
   }
 
   /**
@@ -164,7 +161,7 @@ public class ColumnSchemaModel implements Serializable {
    */
   public String __getCompression() {
     Object o = attrs.get(COMPRESSION);
-    return o != null ? o.toString() : HColumnDescriptor.DEFAULT_COMPRESSION;
+    return o != null ? o.toString() : ColumnFamilyDescriptorBuilder.DEFAULT_COMPRESSION.name();
   }
 
   /**
@@ -173,7 +170,7 @@ public class ColumnSchemaModel implements Serializable {
   public boolean __getInMemory() {
     Object o = attrs.get(IN_MEMORY);
     return o != null ? 
-      Boolean.parseBoolean(o.toString()) : HColumnDescriptor.DEFAULT_IN_MEMORY;
+      Boolean.parseBoolean(o.toString()) : ColumnFamilyDescriptorBuilder.DEFAULT_IN_MEMORY;
   }
 
   /**
@@ -181,8 +178,7 @@ public class ColumnSchemaModel implements Serializable {
    */
   public int __getTTL() {
     Object o = attrs.get(TTL);
-    return o != null ? 
-      Integer.parseInt(o.toString()) : HColumnDescriptor.DEFAULT_TTL;
+    return o != null ? Integer.parseInt(o.toString()) : ColumnFamilyDescriptorBuilder.DEFAULT_TTL;
   }
 
   /**
@@ -190,8 +186,8 @@ public class ColumnSchemaModel implements Serializable {
    */
   public int __getVersions() {
     Object o = attrs.get(VERSIONS);
-    return o != null ? 
-      Integer.parseInt(o.toString()) : HColumnDescriptor.DEFAULT_VERSIONS;
+    return o != null ? Integer.parseInt(o.toString()) :
+      ColumnFamilyDescriptorBuilder.DEFAULT_MAX_VERSIONS;
   }
 
   /**
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/model/TableSchemaModel.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/model/TableSchemaModel.java
index 3d60490..b5578b7 100644
--- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/model/TableSchemaModel.java
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/model/TableSchemaModel.java
@@ -21,7 +21,6 @@ package org.apache.hadoop.hbase.rest.model;
 import com.fasterxml.jackson.annotation.JsonAnyGetter;
 import com.fasterxml.jackson.annotation.JsonAnySetter;
 import com.fasterxml.jackson.annotation.JsonIgnore;
-
 import java.io.IOException;
 import java.io.Serializable;
 import java.util.ArrayList;
@@ -29,28 +28,24 @@ import java.util.Iterator;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
-
 import javax.xml.bind.annotation.XmlAnyAttribute;
 import javax.xml.bind.annotation.XmlAttribute;
 import javax.xml.bind.annotation.XmlElement;
 import javax.xml.bind.annotation.XmlRootElement;
 import javax.xml.namespace.QName;
-
-import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
-import org.apache.yetus.audience.InterfaceAudience;
-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.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.rest.ProtobufMessageHandler;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
 
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.rest.protobuf.generated.TableSchemaMessage.TableSchema;
 import org.apache.hadoop.hbase.shaded.rest.protobuf.generated.ColumnSchemaMessage.ColumnSchema;
+import org.apache.hadoop.hbase.shaded.rest.protobuf.generated.TableSchemaMessage.TableSchema;
 
 /**
  * A representation of HBase table descriptors.
@@ -70,13 +65,12 @@ import org.apache.hadoop.hbase.shaded.rest.protobuf.generated.ColumnSchemaMessag
 @InterfaceAudience.Private
 public class TableSchemaModel implements Serializable, ProtobufMessageHandler {
   private static final long serialVersionUID = 1L;
-  private static final QName IS_META = new QName(HTableDescriptor.IS_META);
-  private static final QName IS_ROOT = new QName(HTableDescriptor.IS_ROOT);
-  private static final QName READONLY = new QName(HTableDescriptor.READONLY);
-  private static final QName TTL = new QName(HColumnDescriptor.TTL);
+  private static final QName IS_META = new QName(TableDescriptorBuilder.IS_META);
+  private static final QName IS_ROOT = new QName("IS_ROOT");
+  private static final QName READONLY = new QName(TableDescriptorBuilder.READONLY);
+  private static final QName TTL = new QName(ColumnFamilyDescriptorBuilder.TTL);
   private static final QName VERSIONS = new QName(HConstants.VERSIONS);
-  private static final QName COMPRESSION =
-    new QName(HColumnDescriptor.COMPRESSION);
+  private static final QName COMPRESSION = new QName(ColumnFamilyDescriptorBuilder.COMPRESSION);
 
   private String name;
   private Map<QName,Object> attrs = new LinkedHashMap<>();
@@ -242,7 +236,7 @@ public class TableSchemaModel implements Serializable, ProtobufMessageHandler {
    */
   public boolean __getReadOnly() {
     Object o = attrs.get(READONLY);
-    return o != null ? Boolean.parseBoolean(o.toString()) : HTableDescriptor.DEFAULT_READONLY;
+    return o != null ? Boolean.parseBoolean(o.toString()) : TableDescriptorBuilder.DEFAULT_READONLY;
   }
 
   /**
@@ -315,7 +309,7 @@ public class TableSchemaModel implements Serializable, ProtobufMessageHandler {
       this.addAttribute(attr.getName(), attr.getValue());
     }
     if (builder.hasReadOnly()) {
-      this.addAttribute(HTableDescriptor.READONLY, builder.getReadOnly());
+      this.addAttribute(TableDescriptorBuilder.READONLY, builder.getReadOnly());
     }
     for (ColumnSchema family : builder.getColumnsList()) {
       ColumnSchemaModel familyModel = new ColumnSchemaModel();
@@ -324,14 +318,14 @@ public class TableSchemaModel implements Serializable, ProtobufMessageHandler {
         familyModel.addAttribute(attr.getName(), attr.getValue());
       }
       if (family.hasTtl()) {
-        familyModel.addAttribute(HColumnDescriptor.TTL, family.getTtl());
+        familyModel.addAttribute(ColumnFamilyDescriptorBuilder.TTL, family.getTtl());
       }
       if (family.hasMaxVersions()) {
         familyModel.addAttribute(HConstants.VERSIONS,
           family.getMaxVersions());
       }
       if (family.hasCompression()) {
-        familyModel.addAttribute(HColumnDescriptor.COMPRESSION,
+        familyModel.addAttribute(ColumnFamilyDescriptorBuilder.COMPRESSION,
           family.getCompression());
       }
       this.addColumnFamily(familyModel);
diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/PerformanceEvaluation.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/PerformanceEvaluation.java
index b895c4d..d7a5cfd 100644
--- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/PerformanceEvaluation.java
+++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/PerformanceEvaluation.java
@@ -33,7 +33,6 @@ import java.util.Random;
 import java.util.TreeMap;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.FSDataInputStream;
@@ -76,7 +75,6 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Hash;
 import org.apache.hadoop.hbase.util.MurmurHash;
 import org.apache.hadoop.hbase.util.Pair;
-
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.NullWritable;
 import org.apache.hadoop.io.Text;
@@ -93,7 +91,6 @@ import org.apache.hadoop.mapreduce.lib.reduce.LongSumReducer;
 import org.apache.hadoop.util.LineReader;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -128,7 +125,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
   public static final byte[] QUALIFIER_NAME = Bytes.toBytes("data");
   private TableName tableName = TABLE_NAME;
 
-  protected TableDescriptorBuilder.ModifyableTableDescriptor TABLE_DESCRIPTOR;
+  protected TableDescriptor TABLE_DESCRIPTOR;
   protected Map<String, CmdDescriptor> commands = new TreeMap<>();
   protected static Cluster cluster = new Cluster();
 
@@ -539,15 +536,12 @@ public class PerformanceEvaluation extends Configured implements Tool {
 
   protected TableDescriptor getDescriptor() {
     if (TABLE_DESCRIPTOR == null) {
-      TABLE_DESCRIPTOR = new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
-      ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
-        new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY_NAME);
-      familyDescriptor.setDataBlockEncoding(blockEncoding);
-      familyDescriptor.setCompressionType(compression);
-      if (inMemoryCF) {
-        familyDescriptor.setInMemory(true);
-      }
-      TABLE_DESCRIPTOR.setColumnFamily(familyDescriptor);
+      TABLE_DESCRIPTOR =
+        TableDescriptorBuilder.newBuilder(tableName)
+          .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(FAMILY_NAME)
+            .setDataBlockEncoding(blockEncoding).setCompressionType(compression)
+            .setInMemory(inMemoryCF).build())
+          .build();
     }
     return TABLE_DESCRIPTOR;
   }
diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestScannersWithFilters.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestScannersWithFilters.java
index a6add87..e1fcaf8 100644
--- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestScannersWithFilters.java
+++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestScannersWithFilters.java
@@ -44,6 +44,7 @@ import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.filter.BinaryComparator;
 import org.apache.hadoop.hbase.filter.Filter;
@@ -140,12 +141,9 @@ public class TestScannersWithFilters {
       REST_TEST_UTIL.getServletPort()));
     Admin admin = TEST_UTIL.getAdmin();
     if (!admin.tableExists(TABLE)) {
-      TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-        new TableDescriptorBuilder.ModifyableTableDescriptor(TABLE);
-      tableDescriptor.setColumnFamily(
-        new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILIES[0]));
-      tableDescriptor.setColumnFamily(
-        new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILIES[1]));
+      TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(TABLE)
+        .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILIES[0]))
+        .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILIES[1])).build();
       admin.createTable(tableDescriptor);
       Table table = TEST_UTIL.getConnection().getTable(TABLE);
       // Insert first half
diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteAdminRetries.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteAdminRetries.java
index 4d0359d..6f4dee5 100644
--- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteAdminRetries.java
+++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteAdminRetries.java
@@ -19,8 +19,8 @@ package org.apache.hadoop.hbase.rest.client;
 
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.anyString;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyString;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
@@ -31,8 +31,8 @@ import java.util.regex.Pattern;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.testclassification.RestTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -117,12 +117,12 @@ public class TestRemoteAdminRetries {
   }
 
   @Test
-  @SuppressWarnings("deprecation")
   public void testFailingCreateTable() throws Exception {
     testTimedOutCall(new CallExecutor() {
       @Override
       public void run() throws Exception {
-        remoteAdmin.createTable(new HTableDescriptor(TableName.valueOf("TestTable")));
+        remoteAdmin
+          .createTable(TableDescriptorBuilder.newBuilder(TableName.valueOf("TestTable")).build());
       }
     });
     verify(client, times(RETRIES)).put(anyString(), anyString(), any());
diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteTable.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteTable.java
index 2d3af51..66baf16 100644
--- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteTable.java
+++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteTable.java
@@ -122,17 +122,11 @@ public class TestRemoteTable {
       admin.deleteTable(TABLE);
     }
 
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(TABLE);
-    tableDescriptor.setColumnFamily(
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(COLUMN_1)
-        .setMaxVersions(3));
-    tableDescriptor.setColumnFamily(
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(COLUMN_2)
-        .setMaxVersions(3));
-    tableDescriptor.setColumnFamily(
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(COLUMN_3)
-        .setMaxVersions(3));
+    TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(TABLE)
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(COLUMN_1).setMaxVersions(3).build())
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(COLUMN_2).setMaxVersions(3).build())
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(COLUMN_3).setMaxVersions(3).build())
+      .build();
     admin.createTable(tableDescriptor);
     try (Table table = TEST_UTIL.getConnection().getTable(TABLE)) {
       Put put = new Put(ROW_1);
diff --git a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/MasterStatusTmpl.jamon b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/MasterStatusTmpl.jamon
index 19759b5..d95a359 100644
--- a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/MasterStatusTmpl.jamon
+++ b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/MasterStatusTmpl.jamon
@@ -38,7 +38,6 @@ org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
 org.apache.hadoop.hbase.HBaseConfiguration;
 org.apache.hadoop.hbase.HConstants;
-org.apache.hadoop.hbase.HTableDescriptor;
 org.apache.hadoop.hbase.NamespaceDescriptor;
 org.apache.hadoop.hbase.ServerName;
 org.apache.hadoop.hbase.TableName;
@@ -610,8 +609,7 @@ AssignmentManager assignmentManager = master.getAssignmentManager();
 <tbody>
     <%for TableDescriptor desc : tables%>
     <%java>
-      HTableDescriptor htDesc = new HTableDescriptor(desc);
-      TableName tableName = htDesc.getTableName();
+      TableName tableName = desc.getTableName();
       TableState tableState = master.getTableStateManager().getTableState(tableName);
       Map<RegionState.State, List<RegionInfo>> tableRegions =
           master.getAssignmentManager().getRegionStates()
@@ -649,7 +647,7 @@ AssignmentManager assignmentManager = master.getAssignmentManager();
         <td><% failedRegionsCount %></td>
         <td><% splitRegionsCount %></td>
         <td><% otherRegionsCount %></td>
-        <td><% htDesc.toStringCustomizedValues() %></td>
+        <td><% desc.toStringCustomizedValues() %></td>
     </tr>
     </%for>
     <p><% tables.size() %> table(s) in set. [<a href=tablesDetailed.jsp>Details</a>]</p>
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java
index b67fffa..f4847b9 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java
@@ -23,22 +23,22 @@ import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
-
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import java.util.concurrent.CopyOnWriteArrayList;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
+import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.util.JVMClusterUtil;
 import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
 import org.apache.hadoop.hbase.util.Threads;
-
-import java.util.concurrent.CopyOnWriteArrayList;
-import org.apache.hadoop.hbase.master.HMaster;
-import org.apache.hadoop.hbase.util.JVMClusterUtil;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * This class creates a single process HBase cluster. One thread is created for
@@ -451,23 +451,18 @@ public class LocalHBaseCluster {
 
   /**
    * Test things basically work.
-   * @param args
-   * @throws IOException
    */
   public static void main(String[] args) throws IOException {
     Configuration conf = HBaseConfiguration.create();
     LocalHBaseCluster cluster = new LocalHBaseCluster(conf);
     cluster.startup();
-    Connection connection = ConnectionFactory.createConnection(conf);
-    Admin admin = connection.getAdmin();
-    try {
-      HTableDescriptor htd =
-        new HTableDescriptor(TableName.valueOf(cluster.getClass().getName()));
+    try (Connection connection = ConnectionFactory.createConnection(conf);
+      Admin admin = connection.getAdmin()) {
+      TableDescriptor htd =
+        TableDescriptorBuilder.newBuilder(TableName.valueOf(cluster.getClass().getName())).build();
       admin.createTable(htd);
     } finally {
-      admin.close();
+      cluster.shutdown();
     }
-    connection.close();
-    cluster.shutdown();
   }
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/constraint/Constraints.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/constraint/Constraints.java
index ee942db..301adc1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/constraint/Constraints.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/constraint/Constraints.java
@@ -286,7 +286,6 @@ public final class Constraints {
    * Write the configuration to a String
    * @param conf to write
    * @return String representation of that configuration
-   * @throws IOException
    */
   private static String serializeConfiguration(Configuration conf) throws IOException {
     // write the configuration out to the data stream
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/LruBlockCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/LruBlockCache.java
index 5587c40..2978eed 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/LruBlockCache.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/LruBlockCache.java
@@ -50,46 +50,41 @@ import org.apache.hbase.thirdparty.com.google.common.base.Objects;
 import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
 
 /**
- * A block cache implementation that is memory-aware using {@link HeapSize},
- * memory-bound using an LRU eviction algorithm, and concurrent: backed by a
- * {@link ConcurrentHashMap} and with a non-blocking eviction thread giving
- * constant-time {@link #cacheBlock} and {@link #getBlock} operations.<p>
- *
+ * A block cache implementation that is memory-aware using {@link HeapSize}, memory-bound using an
+ * LRU eviction algorithm, and concurrent: backed by a {@link ConcurrentHashMap} and with a
+ * non-blocking eviction thread giving constant-time {@link #cacheBlock} and {@link #getBlock}
+ * operations.
+ * </p>
  * Contains three levels of block priority to allow for scan-resistance and in-memory families
- * {@link org.apache.hadoop.hbase.HColumnDescriptor#setInMemory(boolean)} (An in-memory column
- * family is a column family that should be served from memory if possible):
- * single-access, multiple-accesses, and in-memory priority.
- * A block is added with an in-memory priority flag if
- * {@link org.apache.hadoop.hbase.HColumnDescriptor#isInMemory()}, otherwise a block becomes a
- * single access priority the first time it is read into this block cache.  If a block is
- * accessed again while in cache, it is marked as a multiple access priority block.  This
- * delineation of blocks is used to prevent scans from thrashing the cache adding a
- * least-frequently-used element to the eviction algorithm.<p>
- *
- * Each priority is given its own chunk of the total cache to ensure
- * fairness during eviction.  Each priority will retain close to its maximum
- * size, however, if any priority is not using its entire chunk the others
- * are able to grow beyond their chunk size.<p>
- *
- * Instantiated at a minimum with the total size and average block size.
- * All sizes are in bytes.  The block size is not especially important as this
- * cache is fully dynamic in its sizing of blocks.  It is only used for
- * pre-allocating data structures and in initial heap estimation of the map.<p>
- *
- * The detailed constructor defines the sizes for the three priorities (they
- * should total to the <code>maximum size</code> defined).  It also sets the levels that
- * trigger and control the eviction thread.<p>
- *
- * The <code>acceptable size</code> is the cache size level which triggers the eviction
- * process to start.  It evicts enough blocks to get the size below the
- * minimum size specified.<p>
- *
- * Eviction happens in a separate thread and involves a single full-scan
- * of the map.  It determines how many bytes must be freed to reach the minimum
- * size, and then while scanning determines the fewest least-recently-used
- * blocks necessary from each of the three priorities (would be 3 times bytes
- * to free).  It then uses the priority chunk sizes to evict fairly according
- * to the relative sizes and usage.
+ * {@link org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder#setInMemory(boolean)} (An
+ * in-memory column family is a column family that should be served from memory if possible):
+ * single-access, multiple-accesses, and in-memory priority. A block is added with an in-memory
+ * priority flag if {@link org.apache.hadoop.hbase.client.ColumnFamilyDescriptor#isInMemory()},
+ * otherwise a block becomes a single access priority the first time it is read into this block
+ * cache. If a block is accessed again while in cache, it is marked as a multiple access priority
+ * block. This delineation of blocks is used to prevent scans from thrashing the cache adding a
+ * least-frequently-used element to the eviction algorithm.
+ * <p/>
+ * Each priority is given its own chunk of the total cache to ensure fairness during eviction. Each
+ * priority will retain close to its maximum size, however, if any priority is not using its entire
+ * chunk the others are able to grow beyond their chunk size.
+ * <p/>
+ * Instantiated at a minimum with the total size and average block size. All sizes are in bytes. The
+ * block size is not especially important as this cache is fully dynamic in its sizing of blocks. It
+ * is only used for pre-allocating data structures and in initial heap estimation of the map.
+ * <p/>
+ * The detailed constructor defines the sizes for the three priorities (they should total to the
+ * <code>maximum size</code> defined). It also sets the levels that trigger and control the eviction
+ * thread.
+ * <p/>
+ * The <code>acceptable size</code> is the cache size level which triggers the eviction process to
+ * start. It evicts enough blocks to get the size below the minimum size specified.
+ * <p/>
+ * Eviction happens in a separate thread and involves a single full-scan of the map. It determines
+ * how many bytes must be freed to reach the minimum size, and then while scanning determines the
+ * fewest least-recently-used blocks necessary from each of the three priorities (would be 3 times
+ * bytes to free). It then uses the priority chunk sizes to evict fairly according to the relative
+ * sizes and usage.
  */
 @InterfaceAudience.Private
 public class LruBlockCache implements FirstLevelBlockCache {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaUtil.java
index 1d816d6..36dcd26 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/quotas/QuotaUtil.java
@@ -23,7 +23,6 @@ import java.util.Arrays;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
@@ -40,6 +39,7 @@ import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.regionserver.BloomType;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -75,22 +75,15 @@ public class QuotaUtil extends QuotaTableUtil {
   public static final long DEFAULT_WRITE_CAPACITY_UNIT = 1024;
 
   /** Table descriptor for Quota internal table */
-  public static final TableDescriptorBuilder.ModifyableTableDescriptor QUOTA_TABLE_DESC =
-    new TableDescriptorBuilder.ModifyableTableDescriptor(QUOTA_TABLE_NAME);
-  static {
-    QUOTA_TABLE_DESC.setColumnFamily(
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(QUOTA_FAMILY_INFO)
-        .setScope(HConstants.REPLICATION_SCOPE_LOCAL)
-        .setBloomFilterType(BloomType.ROW)
-        .setMaxVersions(1)
-    );
-    QUOTA_TABLE_DESC.setColumnFamily(
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(QUOTA_FAMILY_USAGE)
-        .setScope(HConstants.REPLICATION_SCOPE_LOCAL)
-        .setBloomFilterType(BloomType.ROW)
-        .setMaxVersions(1)
-    );
-  }
+  public static final TableDescriptor QUOTA_TABLE_DESC =
+    TableDescriptorBuilder.newBuilder(QUOTA_TABLE_NAME)
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(QUOTA_FAMILY_INFO)
+        .setScope(HConstants.REPLICATION_SCOPE_LOCAL).setBloomFilterType(BloomType.ROW)
+        .setMaxVersions(1).build())
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(QUOTA_FAMILY_USAGE)
+        .setScope(HConstants.REPLICATION_SCOPE_LOCAL).setBloomFilterType(BloomType.ROW)
+        .setMaxVersions(1).build())
+      .build();
 
   /** Returns true if the support for quota is enabled */
   public static boolean isQuotaEnabled(final Configuration conf) {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java
index 6570e77e..6cc690f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java
@@ -40,7 +40,6 @@ import org.apache.hadoop.hbase.CellScanner;
 import org.apache.hadoop.hbase.CoprocessorEnvironment;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.PrivateCellUtil;
 import org.apache.hadoop.hbase.TableName;
@@ -211,19 +210,14 @@ public class VisibilityController implements MasterCoprocessor, RegionCoprocesso
   public void postStartMaster(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException {
     // Need to create the new system table for labels here
     if (!MetaTableAccessor.tableExists(ctx.getEnvironment().getConnection(), LABELS_TABLE_NAME)) {
-      TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-        new TableDescriptorBuilder.ModifyableTableDescriptor(LABELS_TABLE_NAME);
-      ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
-        new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(LABELS_TABLE_FAMILY);
-      familyDescriptor.setBloomFilterType(BloomType.NONE);
-      // We will cache all the labels. No need of normal
-      // table block cache.
-      familyDescriptor.setBlockCacheEnabled(false);
-      tableDescriptor.setColumnFamily(familyDescriptor);
-      // Let the "labels" table having only one region always. We are not expecting too many labels in
-      // the system.
-      tableDescriptor.setValue(HTableDescriptor.SPLIT_POLICY,
-          DisabledRegionSplitPolicy.class.getName());
+      // We will cache all the labels. No need of normal table block cache.
+      // Let the "labels" table having only one region always. We are not expecting too many labels
+      // in the system.
+      TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(LABELS_TABLE_NAME)
+        .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(LABELS_TABLE_FAMILY)
+          .setBloomFilterType(BloomType.NONE).setBlockCacheEnabled(false).build())
+        .setValue(TableDescriptorBuilder.SPLIT_POLICY, DisabledRegionSplitPolicy.class.getName())
+        .build();
       try (Admin admin = ctx.getEnvironment().getConnection().getAdmin()) {
         admin.createTable(tableDescriptor);
       }
diff --git a/hbase-server/src/main/resources/hbase-webapps/master/rsgroup.jsp b/hbase-server/src/main/resources/hbase-webapps/master/rsgroup.jsp
index 94590c8..6a28de1 100644
--- a/hbase-server/src/main/resources/hbase-webapps/master/rsgroup.jsp
+++ b/hbase-server/src/main/resources/hbase-webapps/master/rsgroup.jsp
@@ -23,10 +23,10 @@
   import="java.util.Date"
   import="java.util.List"
   import="java.util.Map"
+  import="java.util.function.Function"
   import="java.util.regex.Pattern"
   import="java.util.stream.Stream"
   import="java.util.stream.Collectors"
-  import="org.apache.hadoop.hbase.HTableDescriptor"
   import="org.apache.hadoop.hbase.ServerName"
   import="org.apache.hadoop.hbase.TableName"
   import="org.apache.hadoop.hbase.client.Admin"
@@ -438,8 +438,8 @@
         try (Admin admin = master.getConnection().getAdmin()) {
             tables = master.isInitialized() ? admin.listTableDescriptors(true) : null;
         }
-         Map<TableName, HTableDescriptor> tableDescriptors
-            = tables.stream().collect(Collectors.toMap(TableDescriptor::getTableName, p -> new HTableDescriptor(p)));
+         Map<TableName, TableDescriptor> tableDescriptors = tables.stream().collect(
+           Collectors.toMap(TableDescriptor::getTableName, Function.identity()));
     %>
          <table class="table table-striped">
          <tr>
@@ -454,8 +454,8 @@
              <th>Description</th>
          </tr>
          <% for(TableName tableName : rsGroupTables) {
-             HTableDescriptor htDesc = tableDescriptors.get(tableName);
-             if(htDesc == null) {
+             TableDescriptor desc = tableDescriptors.get(tableName);
+             if(desc == null) {
          %>
                <tr>
                  <td><%= tableName.getNamespaceAsString() %></td>
@@ -500,7 +500,7 @@
                   <td><%= failedRegionsCount %></td>
                   <td><%= splitRegionsCount %></td>
                   <td><%= otherRegionsCount %></td>
-                  <td><%= htDesc.toStringCustomizedValues() %></td>
+                  <td><%= desc.toStringCustomizedValues() %></td>
                 </tr>
            <% }
             } %>
diff --git a/hbase-server/src/main/resources/hbase-webapps/master/table.jsp b/hbase-server/src/main/resources/hbase-webapps/master/table.jsp
index 7683242..e46b277 100644
--- a/hbase-server/src/main/resources/hbase-webapps/master/table.jsp
+++ b/hbase-server/src/main/resources/hbase-webapps/master/table.jsp
@@ -35,7 +35,6 @@
   import="org.apache.hadoop.conf.Configuration"
   import="org.apache.hadoop.hbase.HConstants"
   import="org.apache.hadoop.hbase.HRegionLocation"
-  import="org.apache.hadoop.hbase.HTableDescriptor"
   import="org.apache.hadoop.hbase.NotServingRegionException"
   import="org.apache.hadoop.hbase.RegionMetrics"
   import="org.apache.hadoop.hbase.RegionMetricsBuilder"
@@ -46,6 +45,7 @@
   import="org.apache.hadoop.hbase.TableNotFoundException"
   import="org.apache.hadoop.hbase.client.AsyncAdmin"
   import="org.apache.hadoop.hbase.client.AsyncConnection"
+  import="org.apache.hadoop.hbase.client.ColumnFamilyDescriptor"
   import="org.apache.hadoop.hbase.client.CompactionState"
   import="org.apache.hadoop.hbase.client.RegionInfo"
   import="org.apache.hadoop.hbase.client.RegionInfoBuilder"
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
index 0c4f7fb..ede1196 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
@@ -1565,10 +1565,9 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
       new Configuration(getConfiguration()));
   }
 
-  public Table createTable(TableName tableName, byte[][] families,
-      int numVersions, byte[] startKey, byte[] endKey, int numRegions)
-  throws IOException{
-    HTableDescriptor desc = createTableDescriptor(tableName, families, numVersions);
+  public Table createTable(TableName tableName, byte[][] families, int numVersions, byte[] startKey,
+    byte[] endKey, int numRegions) throws IOException {
+    TableDescriptor desc = createTableDescriptor(tableName, families, numVersions);
 
     getAdmin().createTable(desc, startKey, endKey, numRegions);
     // HBaseAdmin only waits for regions to appear in hbase:meta we
@@ -1579,14 +1578,11 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
 
   /**
    * Create a table.
-   * @param htd
-   * @param families
    * @param c Configuration to use
    * @return A Table instance for the created table.
-   * @throws IOException
    */
   public Table createTable(TableDescriptor htd, byte[][] families, Configuration c)
-  throws IOException {
+    throws IOException {
     return createTable(htd, families, null, c);
   }
 
@@ -1678,35 +1674,25 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
    * @param replicaCount the replica count
    * @param c Configuration to use
    * @return A Table instance for the created table.
-   * @throws IOException
    */
   public Table createTable(TableName tableName, byte[][] families, byte[][] splitKeys,
-      int replicaCount, final Configuration c) throws IOException {
-    HTableDescriptor htd = new HTableDescriptor(tableName);
-    htd.setRegionReplication(replicaCount);
+    int replicaCount, final Configuration c) throws IOException {
+    TableDescriptor htd =
+      TableDescriptorBuilder.newBuilder(tableName).setRegionReplication(replicaCount).build();
     return createTable(htd, families, splitKeys, c);
   }
 
   /**
    * Create a table.
-   * @param tableName
-   * @param family
-   * @param numVersions
    * @return A Table instance for the created table.
-   * @throws IOException
    */
-  public Table createTable(TableName tableName, byte[] family, int numVersions)
-  throws IOException {
-    return createTable(tableName, new byte[][]{family}, numVersions);
+  public Table createTable(TableName tableName, byte[] family, int numVersions) throws IOException {
+    return createTable(tableName, new byte[][] { family }, numVersions);
   }
 
   /**
    * Create a table.
-   * @param tableName
-   * @param families
-   * @param numVersions
    * @return A Table instance for the created table.
-   * @throws IOException
    */
   public Table createTable(TableName tableName, byte[][] families, int numVersions)
       throws IOException {
@@ -1715,30 +1701,23 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
 
   /**
    * Create a table.
-   * @param tableName
-   * @param families
-   * @param numVersions
-   * @param splitKeys
    * @return A Table instance for the created table.
-   * @throws IOException
    */
   public Table createTable(TableName tableName, byte[][] families, int numVersions,
       byte[][] splitKeys) throws IOException {
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
+    TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableName);
     for (byte[] family : families) {
-      ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
-        new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family)
-          .setMaxVersions(numVersions);
+      ColumnFamilyDescriptorBuilder cfBuilder = ColumnFamilyDescriptorBuilder.newBuilder(family)
+        .setMaxVersions(numVersions);
       if (isNewVersionBehaviorEnabled()) {
-        familyDescriptor.setNewVersionBehavior(true);
+        cfBuilder.setNewVersionBehavior(true);
       }
-      tableDescriptor.setColumnFamily(familyDescriptor);
+      builder.setColumnFamily(cfBuilder.build());
     }
     if (splitKeys != null) {
-      getAdmin().createTable(tableDescriptor, splitKeys);
+      getAdmin().createTable(builder.build(), splitKeys);
     } else {
-      getAdmin().createTable(tableDescriptor);
+      getAdmin().createTable(builder.build());
     }
     // HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are
     // assigned
@@ -1748,11 +1727,7 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
 
   /**
    * Create a table with multiple regions.
-   * @param tableName
-   * @param families
-   * @param numVersions
    * @return A Table instance for the created table.
-   * @throws IOException
    */
   public Table createMultiRegionTable(TableName tableName, byte[][] families, int numVersions)
       throws IOException {
@@ -1761,28 +1736,20 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
 
   /**
    * Create a table.
-   * @param tableName
-   * @param families
-   * @param numVersions
-   * @param blockSize
    * @return A Table instance for the created table.
-   * @throws IOException
    */
   public Table createTable(TableName tableName, byte[][] families,
     int numVersions, int blockSize) throws IOException {
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
+    TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableName);
     for (byte[] family : families) {
-      ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
-        new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family)
-          .setMaxVersions(numVersions)
-          .setBlocksize(blockSize);
+      ColumnFamilyDescriptorBuilder cfBuilder = ColumnFamilyDescriptorBuilder.newBuilder(family)
+        .setMaxVersions(numVersions).setBlocksize(blockSize);
       if (isNewVersionBehaviorEnabled()) {
-        familyDescriptor.setNewVersionBehavior(true);
+        cfBuilder.setNewVersionBehavior(true);
       }
-      tableDescriptor.setColumnFamily(familyDescriptor);
+      builder.setColumnFamily(cfBuilder.build());
     }
-    getAdmin().createTable(tableDescriptor);
+    getAdmin().createTable(builder.build());
     // HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are
     // assigned
     waitUntilAllRegionsAssigned(tableName);
@@ -1791,22 +1758,19 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
 
   public Table createTable(TableName tableName, byte[][] families,
       int numVersions, int blockSize, String cpName) throws IOException {
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
+    TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableName);
     for (byte[] family : families) {
-      ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
-        new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family)
-          .setMaxVersions(numVersions)
-          .setBlocksize(blockSize);
+      ColumnFamilyDescriptorBuilder cfBuilder = ColumnFamilyDescriptorBuilder.newBuilder(family)
+        .setMaxVersions(numVersions).setBlocksize(blockSize);
       if (isNewVersionBehaviorEnabled()) {
-        familyDescriptor.setNewVersionBehavior(true);
+        cfBuilder.setNewVersionBehavior(true);
       }
-      tableDescriptor.setColumnFamily(familyDescriptor);
+      builder.setColumnFamily(cfBuilder.build());
     }
     if (cpName != null) {
-      tableDescriptor.setCoprocessor(cpName);
+      builder.setCoprocessor(cpName);
     }
-    getAdmin().createTable(tableDescriptor);
+    getAdmin().createTable(builder.build());
     // HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are
     // assigned
     waitUntilAllRegionsAssigned(tableName);
@@ -1815,28 +1779,22 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
 
   /**
    * Create a table.
-   * @param tableName
-   * @param families
-   * @param numVersions
    * @return A Table instance for the created table.
-   * @throws IOException
    */
-  public Table createTable(TableName tableName, byte[][] families,
-      int[] numVersions) throws IOException {
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
+  public Table createTable(TableName tableName, byte[][] families, int[] numVersions)
+    throws IOException {
+    TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableName);
     int i = 0;
     for (byte[] family : families) {
-      ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
-        new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family)
-          .setMaxVersions(numVersions[i]);
+      ColumnFamilyDescriptorBuilder cfBuilder =
+        ColumnFamilyDescriptorBuilder.newBuilder(family).setMaxVersions(numVersions[i]);
       if (isNewVersionBehaviorEnabled()) {
-        familyDescriptor.setNewVersionBehavior(true);
+        cfBuilder.setNewVersionBehavior(true);
       }
-      tableDescriptor.setColumnFamily(familyDescriptor);
+      builder.setColumnFamily(cfBuilder.build());
       i++;
     }
-    getAdmin().createTable(tableDescriptor);
+    getAdmin().createTable(builder.build());
     // HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are
     // assigned
     waitUntilAllRegionsAssigned(tableName);
@@ -1845,23 +1803,17 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
 
   /**
    * Create a table.
-   * @param tableName
-   * @param family
-   * @param splitRows
    * @return A Table instance for the created table.
-   * @throws IOException
    */
   public Table createTable(TableName tableName, byte[] family, byte[][] splitRows)
-      throws IOException {
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
-    ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family);
+    throws IOException {
+    TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableName);
+    ColumnFamilyDescriptorBuilder cfBuilder = ColumnFamilyDescriptorBuilder.newBuilder(family);
     if (isNewVersionBehaviorEnabled()) {
-      familyDescriptor.setNewVersionBehavior(true);
+      cfBuilder.setNewVersionBehavior(true);
     }
-    tableDescriptor.setColumnFamily(familyDescriptor);
-    getAdmin().createTable(tableDescriptor, splitRows);
+    builder.setColumnFamily(cfBuilder.build());
+    getAdmin().createTable(builder.build(), splitRows);
     // HBaseAdmin only waits for regions to appear in hbase:meta we should wait until they are
     // assigned
     waitUntilAllRegionsAssigned(tableName);
@@ -1870,10 +1822,7 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
 
   /**
    * Create a table with multiple regions.
-   * @param tableName
-   * @param family
    * @return A Table instance for the created table.
-   * @throws IOException
    */
   public Table createMultiRegionTable(TableName tableName, byte[] family) throws IOException {
     return createTable(tableName, family, KEYS_FOR_HBA_CREATE_TABLE);
@@ -1896,10 +1845,10 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
    * Set the number of Region replicas.
    */
   public static void setReplicas(Admin admin, TableName table, int replicaCount)
-      throws IOException, InterruptedException {
+    throws IOException, InterruptedException {
     admin.disableTable(table);
-    HTableDescriptor desc = new HTableDescriptor(admin.getDescriptor(table));
-    desc.setRegionReplication(replicaCount);
+    TableDescriptor desc = TableDescriptorBuilder.newBuilder(admin.getDescriptor(table))
+      .setRegionReplication(replicaCount).build();
     admin.modifyTable(desc);
     admin.enableTable(table);
   }
@@ -1931,7 +1880,6 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
 
   // ==========================================================================
   // Canned table and table descriptor creation
-  // TODO replace HBaseTestCase
 
   public final static byte [] fam1 = Bytes.toBytes("colfamily11");
   public final static byte [] fam2 = Bytes.toBytes("colfamily21");
@@ -1944,52 +1892,40 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
   public static final byte [] START_KEY_BYTES = {FIRST_CHAR, FIRST_CHAR, FIRST_CHAR};
   public static final String START_KEY = new String(START_KEY_BYTES, HConstants.UTF8_CHARSET);
 
-  public TableDescriptorBuilder.ModifyableTableDescriptor createModifyableTableDescriptor(
-      final String name) {
+  public TableDescriptorBuilder createModifyableTableDescriptor(final String name) {
     return createModifyableTableDescriptor(TableName.valueOf(name),
-      HColumnDescriptor.DEFAULT_MIN_VERSIONS,
-      MAXVERSIONS, HConstants.FOREVER, HColumnDescriptor.DEFAULT_KEEP_DELETED);
-  }
-
-  public HTableDescriptor createTableDescriptor(final TableName name,
-      final int minVersions, final int versions, final int ttl, KeepDeletedCells keepDeleted) {
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(name);
-    for (byte[] cfName : new byte[][]{fam1, fam2, fam3}) {
-      ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
-        new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(cfName)
-          .setMinVersions(minVersions)
-          .setMaxVersions(versions)
-          .setKeepDeletedCells(keepDeleted)
-          .setBlockCacheEnabled(false)
-          .setTimeToLive(ttl);
+      ColumnFamilyDescriptorBuilder.DEFAULT_MIN_VERSIONS, MAXVERSIONS, HConstants.FOREVER,
+      ColumnFamilyDescriptorBuilder.DEFAULT_KEEP_DELETED);
+  }
+
+  public TableDescriptor createTableDescriptor(final TableName name, final int minVersions,
+    final int versions, final int ttl, KeepDeletedCells keepDeleted) {
+    TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(name);
+    for (byte[] cfName : new byte[][] { fam1, fam2, fam3 }) {
+      ColumnFamilyDescriptorBuilder cfBuilder = ColumnFamilyDescriptorBuilder.newBuilder(cfName)
+        .setMinVersions(minVersions).setMaxVersions(versions).setKeepDeletedCells(keepDeleted)
+        .setBlockCacheEnabled(false).setTimeToLive(ttl);
       if (isNewVersionBehaviorEnabled()) {
-        familyDescriptor.setNewVersionBehavior(true);
+        cfBuilder.setNewVersionBehavior(true);
       }
-      tableDescriptor.setColumnFamily(familyDescriptor);
-    }
-    return new HTableDescriptor(tableDescriptor);
-  }
-
-  public TableDescriptorBuilder.ModifyableTableDescriptor createModifyableTableDescriptor(
-      final TableName name, final int minVersions, final int versions, final int ttl,
-      KeepDeletedCells keepDeleted) {
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(name);
-    for (byte[] cfName : new byte[][]{fam1, fam2, fam3}) {
-      ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
-        new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(cfName)
-          .setMinVersions(minVersions)
-          .setMaxVersions(versions)
-          .setKeepDeletedCells(keepDeleted)
-          .setBlockCacheEnabled(false)
-          .setTimeToLive(ttl);
+      builder.setColumnFamily(cfBuilder.build());
+    }
+    return builder.build();
+  }
+
+  public TableDescriptorBuilder createModifyableTableDescriptor(final TableName name,
+    final int minVersions, final int versions, final int ttl, KeepDeletedCells keepDeleted) {
+    TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(name);
+    for (byte[] cfName : new byte[][] { fam1, fam2, fam3 }) {
+      ColumnFamilyDescriptorBuilder cfBuilder = ColumnFamilyDescriptorBuilder.newBuilder(cfName)
+        .setMinVersions(minVersions).setMaxVersions(versions).setKeepDeletedCells(keepDeleted)
+        .setBlockCacheEnabled(false).setTimeToLive(ttl);
       if (isNewVersionBehaviorEnabled()) {
-        familyDescriptor.setNewVersionBehavior(true);
+        cfBuilder.setNewVersionBehavior(true);
       }
-      tableDescriptor.setColumnFamily(familyDescriptor);
+      builder.setColumnFamily(cfBuilder.build());
     }
-    return tableDescriptor;
+    return builder;
   }
 
   /**
@@ -1997,31 +1933,27 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
    * @param name Name to give table.
    * @return Column descriptor.
    */
-  public HTableDescriptor createTableDescriptor(final TableName name) {
-    return createTableDescriptor(name,  HColumnDescriptor.DEFAULT_MIN_VERSIONS,
-        MAXVERSIONS, HConstants.FOREVER, HColumnDescriptor.DEFAULT_KEEP_DELETED);
+  public TableDescriptor createTableDescriptor(final TableName name) {
+    return createTableDescriptor(name, ColumnFamilyDescriptorBuilder.DEFAULT_MIN_VERSIONS,
+      MAXVERSIONS, HConstants.FOREVER, ColumnFamilyDescriptorBuilder.DEFAULT_KEEP_DELETED);
   }
 
-  public HTableDescriptor createTableDescriptor(final TableName tableName,
-      byte[] family) {
-    return createTableDescriptor(tableName, new byte[][] {family}, 1);
+  public TableDescriptor createTableDescriptor(final TableName tableName, byte[] family) {
+    return createTableDescriptor(tableName, new byte[][] { family }, 1);
   }
 
-  public HTableDescriptor createTableDescriptor(final TableName tableName,
-      byte[][] families, int maxVersions) {
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
-
+  public TableDescriptor createTableDescriptor(final TableName tableName, byte[][] families,
+    int maxVersions) {
+    TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableName);
     for (byte[] family : families) {
-      ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
-        new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family)
-          .setMaxVersions(maxVersions);
+      ColumnFamilyDescriptorBuilder cfBuilder =
+        ColumnFamilyDescriptorBuilder.newBuilder(family).setMaxVersions(maxVersions);
       if (isNewVersionBehaviorEnabled()) {
-        familyDescriptor.setNewVersionBehavior(true);
+        cfBuilder.setNewVersionBehavior(true);
       }
-      tableDescriptor.setColumnFamily(familyDescriptor);
+      builder.setColumnFamily(cfBuilder.build());
     }
-    return new HTableDescriptor(tableDescriptor);
+    return builder.build();
   }
 
   /**
@@ -2078,28 +2010,26 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
   public HRegion createLocalHRegionWithInMemoryFlags(TableName tableName, byte[] startKey,
     byte[] stopKey, boolean isReadOnly, Durability durability, WAL wal, boolean[] compactedMemStore,
     byte[]... families) throws IOException {
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
-    tableDescriptor.setReadOnly(isReadOnly);
+    TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableName);
+    builder.setReadOnly(isReadOnly);
     int i = 0;
     for (byte[] family : families) {
-      ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
-        new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family);
+      ColumnFamilyDescriptorBuilder cfBuilder = ColumnFamilyDescriptorBuilder.newBuilder(family);
       if (compactedMemStore != null && i < compactedMemStore.length) {
-        familyDescriptor.setInMemoryCompaction(MemoryCompactionPolicy.BASIC);
+        cfBuilder.setInMemoryCompaction(MemoryCompactionPolicy.BASIC);
       } else {
-        familyDescriptor.setInMemoryCompaction(MemoryCompactionPolicy.NONE);
+        cfBuilder.setInMemoryCompaction(MemoryCompactionPolicy.NONE);
 
       }
       i++;
       // Set default to be three versions.
-      familyDescriptor.setMaxVersions(Integer.MAX_VALUE);
-      tableDescriptor.setColumnFamily(familyDescriptor);
+      cfBuilder.setMaxVersions(Integer.MAX_VALUE);
+      builder.setColumnFamily(cfBuilder.build());
     }
-    tableDescriptor.setDurability(durability);
-    RegionInfo info = RegionInfoBuilder.newBuilder(tableDescriptor.getTableName())
-      .setStartKey(startKey).setEndKey(stopKey).build();
-    return createLocalHRegion(info, tableDescriptor, wal);
+    builder.setDurability(durability);
+    RegionInfo info =
+      RegionInfoBuilder.newBuilder(tableName).setStartKey(startKey).setEndKey(stopKey).build();
+    return createLocalHRegion(info, builder.build(), wal);
   }
 
   //
@@ -2776,7 +2706,6 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
     } catch (SecurityException e) {
       throw new RuntimeException(e);
     } catch (NoSuchFieldException e) {
-      // TODO Auto-generated catch block
       throw new RuntimeException(e);
     } catch (IllegalArgumentException e) {
       throw new RuntimeException(e);
@@ -3892,20 +3821,17 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
    * logs a warning and continues.
    * @return the number of regions the table was split into
    */
-  public static int createPreSplitLoadTestTable(Configuration conf,
-      TableName tableName, byte[] columnFamily, Algorithm compression,
-      DataBlockEncoding dataBlockEncoding, int numRegionsPerServer, int regionReplication,
-      Durability durability)
-          throws IOException {
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
-    tableDescriptor.setDurability(durability);
-    tableDescriptor.setRegionReplication(regionReplication);
-    ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(columnFamily);
-    familyDescriptor.setDataBlockEncoding(dataBlockEncoding);
-    familyDescriptor.setCompressionType(compression);
-    return createPreSplitLoadTestTable(conf, tableDescriptor, familyDescriptor,
+  public static int createPreSplitLoadTestTable(Configuration conf, TableName tableName,
+    byte[] columnFamily, Algorithm compression, DataBlockEncoding dataBlockEncoding,
+    int numRegionsPerServer, int regionReplication, Durability durability) throws IOException {
+    TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableName);
+    builder.setDurability(durability);
+    builder.setRegionReplication(regionReplication);
+    ColumnFamilyDescriptorBuilder cfBuilder =
+      ColumnFamilyDescriptorBuilder.newBuilder(columnFamily);
+    cfBuilder.setDataBlockEncoding(dataBlockEncoding);
+    cfBuilder.setCompressionType(compression);
+    return createPreSplitLoadTestTable(conf, builder.build(), cfBuilder.build(),
       numRegionsPerServer);
   }
 
@@ -3914,24 +3840,21 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
    * logs a warning and continues.
    * @return the number of regions the table was split into
    */
-  public static int createPreSplitLoadTestTable(Configuration conf,
-      TableName tableName, byte[][] columnFamilies, Algorithm compression,
-      DataBlockEncoding dataBlockEncoding, int numRegionsPerServer, int regionReplication,
-      Durability durability)
-          throws IOException {
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
-    tableDescriptor.setDurability(durability);
-    tableDescriptor.setRegionReplication(regionReplication);
+  public static int createPreSplitLoadTestTable(Configuration conf, TableName tableName,
+    byte[][] columnFamilies, Algorithm compression, DataBlockEncoding dataBlockEncoding,
+    int numRegionsPerServer, int regionReplication, Durability durability) throws IOException {
+    TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableName);
+    builder.setDurability(durability);
+    builder.setRegionReplication(regionReplication);
     ColumnFamilyDescriptor[] hcds = new ColumnFamilyDescriptor[columnFamilies.length];
     for (int i = 0; i < columnFamilies.length; i++) {
-      ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
-        new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(columnFamilies[i]);
-      familyDescriptor.setDataBlockEncoding(dataBlockEncoding);
-      familyDescriptor.setCompressionType(compression);
-      hcds[i] = familyDescriptor;
+      ColumnFamilyDescriptorBuilder cfBuilder =
+        ColumnFamilyDescriptorBuilder.newBuilder(columnFamilies[i]);
+      cfBuilder.setDataBlockEncoding(dataBlockEncoding);
+      cfBuilder.setCompressionType(compression);
+      hcds[i] = cfBuilder.build();
     }
-    return createPreSplitLoadTestTable(conf, tableDescriptor, hcds, numRegionsPerServer);
+    return createPreSplitLoadTestTable(conf, builder.build(), hcds, numRegionsPerServer);
   }
 
   /**
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHColumnDescriptorDefaultVersions.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestColumnFamilyDescriptorDefaultVersions.java
similarity index 72%
rename from hbase-server/src/test/java/org/apache/hadoop/hbase/TestHColumnDescriptorDefaultVersions.java
rename to hbase-server/src/test/java/org/apache/hadoop/hbase/TestColumnFamilyDescriptorDefaultVersions.java
index 0416b1a..eab1916 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHColumnDescriptorDefaultVersions.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestColumnFamilyDescriptorDefaultVersions.java
@@ -42,15 +42,15 @@ import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
 
 /**
- * Verify that the HColumnDescriptor version is set correctly by default, hbase-site.xml, and user
- * input
+ * Verify that the ColumnFamilyDescriptor version is set correctly by default, hbase-site.xml, and
+ * user input
  */
-@Category({MiscTests.class, MediumTests.class})
-public class TestHColumnDescriptorDefaultVersions {
+@Category({ MiscTests.class, MediumTests.class })
+public class TestColumnFamilyDescriptorDefaultVersions {
 
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
-      HBaseClassTestRule.forClass(TestHColumnDescriptorDefaultVersions.class);
+    HBaseClassTestRule.forClass(TestColumnFamilyDescriptorDefaultVersions.class);
 
   @Rule
   public TestName name = new TestName();
@@ -60,7 +60,6 @@ public class TestHColumnDescriptorDefaultVersions {
 
   /**
    * Start up a mini cluster and put a small table of empty regions into it.
-   * @throws Exception
    */
   @BeforeClass
   public static void beforeAllTests() throws Exception {
@@ -82,11 +81,8 @@ public class TestHColumnDescriptorDefaultVersions {
   public void testCreateTableWithDefault() throws IOException {
     Admin admin = TEST_UTIL.getAdmin();
     // Create a table with one family
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(TABLE_NAME);
-    ColumnFamilyDescriptor familyDescriptor =
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY);
-    tableDescriptor.setColumnFamily(familyDescriptor);
+    TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(TABLE_NAME)
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY)).build();
     admin.createTable(tableDescriptor);
     admin.disableTable(TABLE_NAME);
     try {
@@ -105,12 +101,10 @@ public class TestHColumnDescriptorDefaultVersions {
 
     Admin admin = TEST_UTIL.getAdmin();
     // Create a table with one family
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(TABLE_NAME);
-    ColumnFamilyDescriptor familyDescriptor =
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY)
-        .setMaxVersions(TEST_UTIL.getConfiguration().getInt("hbase.column.max.version", 1));
-    tableDescriptor.setColumnFamily(familyDescriptor);
+    TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(TABLE_NAME)
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(FAMILY)
+        .setMaxVersions(TEST_UTIL.getConfiguration().getInt("hbase.column.max.version", 1)).build())
+      .build();
     admin.createTable(tableDescriptor);
     admin.disableTable(TABLE_NAME);
     try {
@@ -129,12 +123,9 @@ public class TestHColumnDescriptorDefaultVersions {
 
     Admin admin = TEST_UTIL.getAdmin();
     // Create a table with one family
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(TABLE_NAME);
-    ColumnFamilyDescriptor familyDescriptor =
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY)
-        .setMaxVersions(5);
-    tableDescriptor.setColumnFamily(familyDescriptor);
+    TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(TABLE_NAME)
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(FAMILY).setMaxVersions(5).build())
+      .build();
     admin.createTable(tableDescriptor);
     admin.disableTable(TABLE_NAME);
     try {
@@ -148,41 +139,39 @@ public class TestHColumnDescriptorDefaultVersions {
 
   @Test
   public void testHColumnDescriptorCachedMaxVersions() throws Exception {
-    ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY);
-    familyDescriptor.setMaxVersions(5);
+    ColumnFamilyDescriptor familyDescriptor =
+      ColumnFamilyDescriptorBuilder.newBuilder(FAMILY).setMaxVersions(5).build();
     // Verify the max version
     assertEquals(5, familyDescriptor.getMaxVersions());
 
     // modify the max version
-    familyDescriptor.setValue(Bytes.toBytes(HConstants.VERSIONS), Bytes.toBytes("8"));
+    familyDescriptor = ColumnFamilyDescriptorBuilder.newBuilder(familyDescriptor)
+      .setValue(Bytes.toBytes(HConstants.VERSIONS), Bytes.toBytes("8")).build();
     // Verify the max version
     assertEquals(8, familyDescriptor.getMaxVersions());
   }
 
   private void verifyHColumnDescriptor(int expected, final TableName tableName,
-      final byte[]... families) throws IOException {
+    final byte[]... families) throws IOException {
     Admin admin = TEST_UTIL.getAdmin();
 
     // Verify descriptor from master
     TableDescriptor htd = admin.getDescriptor(tableName);
     ColumnFamilyDescriptor[] hcds = htd.getColumnFamilies();
-    verifyHColumnDescriptor(expected, hcds, tableName, families);
+    verifyColumnFamilyDescriptor(expected, hcds, tableName, families);
 
     // Verify descriptor from HDFS
     MasterFileSystem mfs = TEST_UTIL.getMiniHBaseCluster().getMaster().getMasterFileSystem();
     Path tableDir = CommonFSUtils.getTableDir(mfs.getRootDir(), tableName);
     TableDescriptor td = FSTableDescriptors.getTableDescriptorFromFs(mfs.getFileSystem(), tableDir);
     hcds = td.getColumnFamilies();
-    verifyHColumnDescriptor(expected, hcds, tableName, families);
+    verifyColumnFamilyDescriptor(expected, hcds, tableName, families);
   }
 
-  private void verifyHColumnDescriptor(int expected, final ColumnFamilyDescriptor[] hcds,
-      final TableName tableName,
-      final byte[]... families) {
+  private void verifyColumnFamilyDescriptor(int expected, final ColumnFamilyDescriptor[] hcds,
+    final TableName tableName, final byte[]... families) {
     for (ColumnFamilyDescriptor hcd : hcds) {
       assertEquals(expected, hcd.getMaxVersions());
     }
   }
-
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMultiVersions.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMultiVersions.java
index 5a36b81..edf553b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMultiVersions.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMultiVersions.java
@@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
@@ -98,15 +99,11 @@ public class TestMultiVersions {
    */
   @Test
   public void testTimestamps() throws Exception {
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(
-        TableName.valueOf(name.getMethodName()));
-    ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(
-        TimestampTestBase.FAMILY_NAME);
-
-    familyDescriptor.setMaxVersions(3);
-    tableDescriptor.setColumnFamily(familyDescriptor);
+    TableDescriptor tableDescriptor =
+      TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+        .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(TimestampTestBase.FAMILY_NAME)
+          .setMaxVersions(3).build())
+        .build();
     this.admin.createTable(tableDescriptor);
     Table table = UTIL.getConnection().getTable(tableDescriptor.getTableName());
     // TODO: Remove these deprecated classes or pull them in here if this is
@@ -144,14 +141,11 @@ public class TestMultiVersions {
     final byte [] value2 = Bytes.toBytes("value2");
     final long timestamp1 = 100L;
     final long timestamp2 = 200L;
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(
-        TableName.valueOf(name.getMethodName()));
-    ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(contents);
-
-    familyDescriptor.setMaxVersions(3);
-    tableDescriptor.setColumnFamily(familyDescriptor);
+    TableDescriptor tableDescriptor =
+      TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+        .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(contents)
+          .setMaxVersions(3).build())
+        .build();
     this.admin.createTable(tableDescriptor);
     Put put = new Put(row, timestamp1);
     put.addColumn(contents, contents, value1);
@@ -205,21 +199,14 @@ public class TestMultiVersions {
    * crazyness.
    *
    * <p>Tests five cases of scans and timestamps.
-   * @throws Exception
    */
   @Test
   public void testScanMultipleVersions() throws Exception {
     final TableName tableName = TableName.valueOf(name.getMethodName());
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
-
-    tableDescriptor.setColumnFamily(
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(
-        HConstants.CATALOG_FAMILY));
-    final byte [][] rows = new byte[][] {
-      Bytes.toBytes("row_0200"),
-      Bytes.toBytes("row_0800")
-    };
+    TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName)
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(HConstants.CATALOG_FAMILY)).build();
+
+    final byte[][] rows = new byte[][] { Bytes.toBytes("row_0200"), Bytes.toBytes("row_0800") };
     final byte [][] splitRows = new byte[][] {Bytes.toBytes("row_0500")};
     final long [] timestamp = new long[] {100L, 1000L};
     this.admin.createTable(tableDescriptor, splitRows);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionRebalancing.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionRebalancing.java
index 170182e..08ef065 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionRebalancing.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRegionRebalancing.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionLocator;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.testclassification.FlakeyTests;
@@ -77,7 +78,7 @@ public class TestRegionRebalancing {
   private static final Logger LOG = LoggerFactory.getLogger(TestRegionRebalancing.class);
   private final HBaseTestingUtility UTIL = new HBaseTestingUtility();
   private RegionLocator regionLocator;
-  private TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor;
+  private TableDescriptor tableDescriptor;
   private String balancerName;
 
   public TestRegionRebalancing(String balancerName) {
@@ -96,10 +97,8 @@ public class TestRegionRebalancing {
     // set minCostNeedBalance to 0, make sure balancer run
     UTIL.startMiniCluster(1);
 
-    this.tableDescriptor = new TableDescriptorBuilder.ModifyableTableDescriptor(
-      TableName.valueOf("test"));
-    this.tableDescriptor.setColumnFamily(
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY_NAME));
+    this.tableDescriptor = TableDescriptorBuilder.newBuilder(TableName.valueOf("test"))
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY_NAME)).build();
   }
 
   /**
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestSerialization.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestSerialization.java
index d7d0542..69cf139 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestSerialization.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestSerialization.java
@@ -141,9 +141,9 @@ public class TestSerialization {
   @Test
   public void testTableDescriptor() throws Exception {
     final String name = "testTableDescriptor";
-    HTableDescriptor htd = createTableDescriptor(name);
-    byte[] mb = htd.toByteArray();
-    HTableDescriptor deserializedHtd = HTableDescriptor.parseFrom(mb);
+    TableDescriptor htd = createTableDescriptor(name);
+    byte[] mb = TableDescriptorBuilder.toByteArray(htd);
+    TableDescriptor deserializedHtd = TableDescriptorBuilder.parseFrom(mb);
     assertEquals(htd.getTableName(), deserializedHtd.getTableName());
   }
 
@@ -296,7 +296,7 @@ public class TestSerialization {
    * @param name Name to give table.
    * @return Column descriptor.
    */
-  protected HTableDescriptor createTableDescriptor(final String name) {
+  protected TableDescriptor createTableDescriptor(final String name) {
     return createTableDescriptor(name, MAXVERSIONS);
   }
 
@@ -306,18 +306,15 @@ public class TestSerialization {
    * @param versions How many versions to allow per column.
    * @return Column descriptor.
    */
-  protected HTableDescriptor createTableDescriptor(final String name, final int versions) {
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(TableName.valueOf(name));
-    tableDescriptor
-      .setColumnFamily(new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(fam1)
-        .setMaxVersions(versions).setBlockCacheEnabled(false));
-    tableDescriptor
-      .setColumnFamily(new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(fam2)
-        .setMaxVersions(versions).setBlockCacheEnabled(false));
-    tableDescriptor
-      .setColumnFamily(new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(fam3)
-        .setMaxVersions(versions).setBlockCacheEnabled(false));
-    return new HTableDescriptor(tableDescriptor);
+  protected TableDescriptor createTableDescriptor(final String name, final int versions) {
+    TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(TableName.valueOf(name));
+    builder
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(fam1).setMaxVersions(versions)
+        .setBlockCacheEnabled(false).build())
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(fam2).setMaxVersions(versions)
+        .setBlockCacheEnabled(false).build())
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(fam3).setMaxVersions(versions)
+        .setBlockCacheEnabled(false).build());
+    return builder.build();
   }
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java
index 09c9ab2..5bb1cc2 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.client;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThrows;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
@@ -36,7 +37,6 @@ import java.util.stream.Collectors;
 import org.apache.hadoop.hbase.ClusterMetrics.Option;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableExistsException;
@@ -85,8 +85,8 @@ public class TestAdmin2 extends TestAdminBase {
   public void testCreateBadTables() throws IOException {
     String msg = null;
     try {
-      ADMIN.createTable(new HTableDescriptor(TableName.META_TABLE_NAME));
-    } catch(TableExistsException e) {
+      ADMIN.createTable(TableDescriptorBuilder.newBuilder(TableName.META_TABLE_NAME).build());
+    } catch (TableExistsException e) {
       msg = e.toString();
     }
     assertTrue("Unexcepted exception message " + msg, msg != null &&
@@ -94,13 +94,9 @@ public class TestAdmin2 extends TestAdminBase {
       msg.contains(TableName.META_TABLE_NAME.getNameAsString()));
 
     // Now try and do concurrent creation with a bunch of threads.
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(
-        TableName.valueOf(name.getMethodName()));
-    ColumnFamilyDescriptor familyDescriptor =
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(
-        HConstants.CATALOG_FAMILY);
-    tableDescriptor.setColumnFamily(familyDescriptor);
+    TableDescriptor tableDescriptor =
+      TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+        .setColumnFamily(ColumnFamilyDescriptorBuilder.of(HConstants.CATALOG_FAMILY)).build();
     int count = 10;
     Thread [] threads = new Thread [count];
     final AtomicInteger successes = new AtomicInteger(0);
@@ -141,22 +137,15 @@ public class TestAdmin2 extends TestAdminBase {
 
   /**
    * Test for hadoop-1581 'HBASE: Unopenable tablename bug'.
-   * @throws Exception
    */
   @Test
   public void testTableNameClash() throws Exception {
     final String name = this.name.getMethodName();
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor1 =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(
-        TableName.valueOf(name + "SOMEUPPERCASE"));
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor2 =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(TableName.valueOf(name));
-    tableDescriptor1.setColumnFamily(
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(
-        HConstants.CATALOG_FAMILY));
-    tableDescriptor2.setColumnFamily(
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(
-        HConstants.CATALOG_FAMILY));
+    TableDescriptor tableDescriptor1 =
+      TableDescriptorBuilder.newBuilder(TableName.valueOf(name + "SOMEUPPERCASE"))
+        .setColumnFamily(ColumnFamilyDescriptorBuilder.of(HConstants.CATALOG_FAMILY)).build();
+    TableDescriptor tableDescriptor2 = TableDescriptorBuilder.newBuilder(TableName.valueOf(name))
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(HConstants.CATALOG_FAMILY)).build();
     ADMIN.createTable(tableDescriptor1);
     ADMIN.createTable(tableDescriptor2);
     // Before fix, below would fail throwing a NoServerForRegionException.
@@ -181,11 +170,8 @@ public class TestAdmin2 extends TestAdminBase {
       byte [] startKey = { 1, 1, 1, 1, 1, 1, 1, 1, 1, 1 };
       byte [] endKey =   { 9, 9, 9, 9, 9, 9, 9, 9, 9, 9 };
       Admin hbaseadmin = TEST_UTIL.getAdmin();
-      TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-        new TableDescriptorBuilder.ModifyableTableDescriptor(TableName.valueOf(name));
-      tableDescriptor.setColumnFamily(
-        new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(
-          HConstants.CATALOG_FAMILY));
+      TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(TableName.valueOf(name))
+        .setColumnFamily(ColumnFamilyDescriptorBuilder.of(HConstants.CATALOG_FAMILY)).build();
       hbaseadmin.createTable(tableDescriptor, startKey, endKey, expectedRegions);
     } finally {
       TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_RPC_TIMEOUT_KEY, oldTimeout);
@@ -214,26 +200,18 @@ public class TestAdmin2 extends TestAdminBase {
    */
   @Test
   public void testTableNames() throws IOException {
-    byte[][] illegalNames = new byte[][] {
-        Bytes.toBytes("-bad"),
-        Bytes.toBytes(".bad")
-    };
+    byte[][] illegalNames = new byte[][] { Bytes.toBytes("-bad"), Bytes.toBytes(".bad") };
     for (byte[] illegalName : illegalNames) {
-      try {
-        new HTableDescriptor(TableName.valueOf(illegalName));
-        throw new IOException("Did not detect '" +
-            Bytes.toString(illegalName) + "' as an illegal user table name");
-      } catch (IllegalArgumentException e) {
-        // expected
-      }
+      assertThrows(
+        "Did not detect '" + Bytes.toString(illegalName) + "' as an illegal user table name",
+        IllegalArgumentException.class, () -> TableName.valueOf(illegalName));
     }
     byte[] legalName = Bytes.toBytes("g-oo.d");
     try {
-      new HTableDescriptor(TableName.valueOf(legalName));
+      TableName.valueOf(legalName);
     } catch (IllegalArgumentException e) {
-      throw new IOException("Legal user table name: '" +
-        Bytes.toString(legalName) + "' caused IllegalArgumentException: " +
-        e.getMessage());
+      fail("Legal user table name: '" + Bytes.toString(legalName) +
+        "' caused IllegalArgumentException: " + e.getMessage());
     }
   }
 
@@ -404,11 +382,8 @@ public class TestAdmin2 extends TestAdminBase {
     byte [] endKey =   { 9, 9, 9, 9, 9, 9, 9, 9, 9, 9 };
 
 
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
-    tableDescriptor.setColumnFamily(
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(
-        HConstants.CATALOG_FAMILY));
+    TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName)
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(HConstants.CATALOG_FAMILY)).build();
     ADMIN.createTable(tableDescriptor, startKey, endKey, expectedRegions);
 
     List<RegionInfo> RegionInfos = ADMIN.getRegions(tableName);
@@ -505,11 +480,8 @@ public class TestAdmin2 extends TestAdminBase {
     TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME).close();
 
     // Create the test table and open it
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
-    tableDescriptor.setColumnFamily(
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(
-        HConstants.CATALOG_FAMILY));
+    TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName)
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(HConstants.CATALOG_FAMILY)).build();
     ADMIN.createTable(tableDescriptor);
     Table table = TEST_UTIL.getConnection().getTable(tableName);
 
@@ -541,14 +513,9 @@ public class TestAdmin2 extends TestAdminBase {
     }
     // Before the fix for HBASE-6146, the below table creation was failing as the hbase:meta table
     // actually getting disabled by the disableTable() call.
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(
-        TableName.valueOf(Bytes.toBytes(name.getMethodName())));
-    ColumnFamilyDescriptor familyDescriptor =
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(
-        Bytes.toBytes("cf1"));
-
-    tableDescriptor.setColumnFamily(familyDescriptor);
+    TableDescriptor tableDescriptor =
+      TableDescriptorBuilder.newBuilder(TableName.valueOf(Bytes.toBytes(name.getMethodName())))
+        .setColumnFamily(ColumnFamilyDescriptorBuilder.of(Bytes.toBytes("cf1"))).build();
     TEST_UTIL.getAdmin().createTable(tableDescriptor);
   }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi3.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi3.java
index 6ba960b..4a71baf 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi3.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi3.java
@@ -19,9 +19,9 @@ package org.apache.hadoop.hbase.client;
 
 import static org.apache.hadoop.hbase.TableName.META_TABLE_NAME;
 import static org.hamcrest.CoreMatchers.instanceOf;
+import static org.hamcrest.MatcherAssert.assertThat;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
@@ -35,7 +35,6 @@ import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.TableDescriptorBuilder.ModifyableTableDescriptor;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -147,9 +146,8 @@ public class TestAsyncTableAdminApi3 extends TestAsyncAdminBase {
     }
     TableDescriptor desc = builder.build();
     admin.createTable(desc).join();
-    ModifyableTableDescriptor modifyableDesc = ((ModifyableTableDescriptor) desc);
     TableDescriptor confirmedHtd = admin.getDescriptor(tableName).get();
-    assertEquals(0, modifyableDesc.compareTo((ModifyableTableDescriptor) confirmedHtd));
+    assertEquals(0, TableDescriptor.COMPARATOR.compare(desc, confirmedHtd));
   }
 
   @Test
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBlockEvictionFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBlockEvictionFromClient.java
index dbf0ce8..b8c994c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBlockEvictionFromClient.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBlockEvictionFromClient.java
@@ -36,8 +36,6 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionLocation;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
@@ -575,14 +573,15 @@ public class TestBlockEvictionFromClient {
     Table table = null;
     try {
       final TableName tableName = TableName.valueOf(name.getMethodName());
-      HTableDescriptor desc = TEST_UTIL.createTableDescriptor(tableName);
+      TableDescriptor desc = TEST_UTIL.createTableDescriptor(tableName);
       // This test expects rpc refcount of cached data blocks to be 0 after split. After split,
       // two daughter regions are opened and a compaction is scheduled to get rid of reference
       // of the parent region hfiles. Compaction will increase refcount of cached data blocks by 1.
       // It is flakey since compaction can kick in anytime. To solve this issue, table is created
       // with compaction disabled.
-      desc.setCompactionEnabled(false);
-      table = TEST_UTIL.createTable(desc, FAMILIES_1, null, BloomType.ROW, 1024, null);
+      table = TEST_UTIL.createTable(
+        TableDescriptorBuilder.newBuilder(desc).setCompactionEnabled(false).build(), FAMILIES_1,
+        null, BloomType.ROW, 1024, null);
       // get the block cache and region
       RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName);
       String regionName = locator.getAllRegionLocations().get(0).getRegion().getEncodedName();
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestEnableTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestEnableTable.java
index 92bc15e..166fade 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestEnableTable.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestEnableTable.java
@@ -76,23 +76,18 @@ public class TestEnableTable {
    * We were only clearing rows that had a hregioninfo column in hbase:meta.  Mangled rows that
    * were missing the hregioninfo because of error were being left behind messing up any
    * subsequent table made with the same name. HBASE-12980
-   * @throws IOException
-   * @throws InterruptedException
    */
   @Test
   public void testDeleteForSureClearsAllTableRowsFromMeta()
-  throws IOException, InterruptedException {
+    throws IOException, InterruptedException {
     final TableName tableName = TableName.valueOf(name.getMethodName());
     final Admin admin = TEST_UTIL.getAdmin();
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
-    ColumnFamilyDescriptor familyDescriptor =
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILYNAME);
-    tableDescriptor.setColumnFamily(familyDescriptor);
+    TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName)
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILYNAME)).build();
     try {
       createTable(TEST_UTIL, tableDescriptor, HBaseTestingUtility.KEYS_FOR_HBA_CREATE_TABLE);
     } catch (Exception e) {
-      e.printStackTrace();
+      LOG.error("", e);
       fail("Got an exception while creating " + tableName);
     }
     // Now I have a nice table, mangle it by removing the HConstants.REGIONINFO_QUALIFIER_STR
@@ -115,7 +110,7 @@ public class TestEnableTable {
       try {
         deleteTable(TEST_UTIL, tableName);
       } catch (Exception e) {
-        e.printStackTrace();
+        LOG.error("", e);
         fail("Got an exception while deleting " + tableName);
       }
       int rowCount = 0;
@@ -163,13 +158,12 @@ public class TestEnableTable {
     }
   }
 
-  public static void createTable(HBaseTestingUtility testUtil,
-      TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor, byte [][] splitKeys)
-      throws Exception {
+  public static void createTable(HBaseTestingUtility testUtil, TableDescriptor tableDescriptor,
+    byte[][] splitKeys) throws Exception {
     // NOTE: We need a latch because admin is not sync,
     // so the postOp coprocessor method may be called after the admin operation returned.
-    MasterSyncObserver observer = testUtil.getHBaseCluster().getMaster()
-      .getMasterCoprocessorHost().findCoprocessor(MasterSyncObserver.class);
+    MasterSyncObserver observer = testUtil.getHBaseCluster().getMaster().getMasterCoprocessorHost()
+      .findCoprocessor(MasterSyncObserver.class);
     observer.tableCreationLatch = new CountDownLatch(1);
     Admin admin = testUtil.getAdmin();
     if (splitKeys != null) {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
index 70d832f..bd56e5f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
@@ -113,18 +113,18 @@ public class TestFromClientSide extends FromClientSideBase {
    */
   @Test
   public void testDuplicateAppend() throws Exception {
-    TableDescriptorBuilder.ModifyableTableDescriptor mtd = TEST_UTIL
+    TableDescriptorBuilder builder = TEST_UTIL
       .createModifyableTableDescriptor(name.getTableName(),
         ColumnFamilyDescriptorBuilder.DEFAULT_MIN_VERSIONS, 3, HConstants.FOREVER,
         ColumnFamilyDescriptorBuilder.DEFAULT_KEEP_DELETED);
     Map<String, String> kvs = new HashMap<>();
     kvs.put(SleepAtFirstRpcCall.SLEEP_TIME_CONF_KEY, "2000");
-    mtd.setCoprocessor(CoprocessorDescriptorBuilder
+    builder.setCoprocessor(CoprocessorDescriptorBuilder
       .newBuilder(SleepAtFirstRpcCall.class.getName())
       .setPriority(1)
       .setProperties(kvs)
       .build());
-    TEST_UTIL.createTable(mtd, new byte[][] { ROW }).close();
+    TEST_UTIL.createTable(builder.build(), new byte[][] { ROW }).close();
 
     Configuration c = new Configuration(TEST_UTIL.getConfiguration());
     c.setInt(HConstants.HBASE_CLIENT_PAUSE, 50);
@@ -156,18 +156,18 @@ public class TestFromClientSide extends FromClientSideBase {
    */
   @Test
   public void testDuplicateBatchAppend() throws Exception {
-    TableDescriptorBuilder.ModifyableTableDescriptor mtd = TEST_UTIL
+    TableDescriptorBuilder builder = TEST_UTIL
       .createModifyableTableDescriptor(name.getTableName(),
         ColumnFamilyDescriptorBuilder.DEFAULT_MIN_VERSIONS, 3, HConstants.FOREVER,
         ColumnFamilyDescriptorBuilder.DEFAULT_KEEP_DELETED);
     Map<String, String> kvs = new HashMap<>();
     kvs.put(SleepAtFirstRpcCall.SLEEP_TIME_CONF_KEY, "2000");
-    mtd.setCoprocessor(CoprocessorDescriptorBuilder
+    builder.setCoprocessor(CoprocessorDescriptorBuilder
       .newBuilder(SleepAtFirstRpcCall.class.getName())
       .setPriority(1)
       .setProperties(kvs)
       .build());
-    TEST_UTIL.createTable(mtd, new byte[][] { ROW }).close();
+    TEST_UTIL.createTable(builder.build(), new byte[][] { ROW }).close();
 
     Configuration c = new Configuration(TEST_UTIL.getConfiguration());
     c.setInt(HConstants.HBASE_CLIENT_PAUSE, 50);
@@ -200,7 +200,8 @@ public class TestFromClientSide extends FromClientSideBase {
   /**
    * Basic client side validation of HBASE-4536
    */
-  @Test public void testKeepDeletedCells() throws Exception {
+  @Test
+  public void testKeepDeletedCells() throws Exception {
     final TableName tableName = name.getTableName();
     final byte[] FAMILY = Bytes.toBytes("family");
     final byte[] C0 = Bytes.toBytes("c0");
@@ -208,13 +209,11 @@ public class TestFromClientSide extends FromClientSideBase {
     final byte[] T1 = Bytes.toBytes("T1");
     final byte[] T2 = Bytes.toBytes("T2");
     final byte[] T3 = Bytes.toBytes("T3");
-    ColumnFamilyDescriptor familyDescriptor =
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY)
-        .setKeepDeletedCells(KeepDeletedCells.TRUE).setMaxVersions(3);
 
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
-    tableDescriptor.setColumnFamily(familyDescriptor);
+    TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName)
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(FAMILY)
+        .setKeepDeletedCells(KeepDeletedCells.TRUE).setMaxVersions(3).build())
+      .build();
     TEST_UTIL.getAdmin().createTable(tableDescriptor);
     try (Table h = TEST_UTIL.getConnection().getTable(tableName)) {
       long ts = System.currentTimeMillis();
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java
index f600ab7..5e127d6 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java
@@ -43,10 +43,8 @@ import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.Coprocessor;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionLocation;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.RegionMetrics;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
@@ -308,8 +306,8 @@ public class TestFromClientSide3 {
 
         // change the compaction.min config option for this table to 5
         LOG.info("hbase.hstore.compaction.min should now be 5");
-        HTableDescriptor htd = new HTableDescriptor(table.getDescriptor());
-        htd.setValue("hbase.hstore.compaction.min", String.valueOf(5));
+        TableDescriptor htd = TableDescriptorBuilder.newBuilder(table.getDescriptor())
+          .setValue("hbase.hstore.compaction.min", String.valueOf(5)).build();
         admin.modifyTable(htd);
         LOG.info("alter status finished");
 
@@ -327,9 +325,10 @@ public class TestFromClientSide3 {
 
         // change an individual CF's config option to 2 & online schema update
         LOG.info("hbase.hstore.compaction.min should now be 2");
-        HColumnDescriptor hcd = new HColumnDescriptor(htd.getFamily(FAMILY));
-        hcd.setValue("hbase.hstore.compaction.min", String.valueOf(2));
-        htd.modifyFamily(hcd);
+        htd = TableDescriptorBuilder.newBuilder(htd)
+          .modifyColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(htd.getColumnFamily(FAMILY))
+            .setValue("hbase.hstore.compaction.min", String.valueOf(2)).build())
+          .build();
         admin.modifyTable(htd);
         LOG.info("alter status finished");
 
@@ -356,9 +355,10 @@ public class TestFromClientSide3 {
         // Finally, ensure that we can remove a custom config value after we made it
         LOG.info("Removing CF config value");
         LOG.info("hbase.hstore.compaction.min should now be 5");
-        hcd = new HColumnDescriptor(htd.getFamily(FAMILY));
-        hcd.setValue("hbase.hstore.compaction.min", null);
-        htd.modifyFamily(hcd);
+        htd = TableDescriptorBuilder.newBuilder(htd)
+          .modifyColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(htd.getColumnFamily(FAMILY))
+            .setValue("hbase.hstore.compaction.min", null).build())
+          .build();
         admin.modifyTable(htd);
         LOG.info("alter status finished");
         assertNull(table.getDescriptor().getColumnFamily(FAMILY)
@@ -735,14 +735,10 @@ public class TestFromClientSide3 {
     });
   }
 
-  private void testPreBatchMutate(TableName tableName, Runnable rn)throws Exception {
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
-    ColumnFamilyDescriptor familyDescriptor =
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY);
-
-    tableDescriptor.setCoprocessor(WaitingForScanObserver.class.getName());
-    tableDescriptor.setColumnFamily(familyDescriptor);
+  private void testPreBatchMutate(TableName tableName, Runnable rn) throws Exception {
+    TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName)
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY))
+      .setCoprocessor(WaitingForScanObserver.class.getName()).build();
     TEST_UTIL.getAdmin().createTable(tableDescriptor);
     // Don't use waitTableAvailable(), because the scanner will mess up the co-processor
 
@@ -774,14 +770,10 @@ public class TestFromClientSide3 {
 
   @Test
   public void testLockLeakWithDelta() throws Exception, Throwable {
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
-    ColumnFamilyDescriptor familyDescriptor =
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY);
-
-    tableDescriptor.setCoprocessor(WaitingForMultiMutationsObserver.class.getName());
-    tableDescriptor.setValue("hbase.rowlock.wait.duration", String.valueOf(5000));
-    tableDescriptor.setColumnFamily(familyDescriptor);
+    TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName)
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY))
+      .setCoprocessor(WaitingForMultiMutationsObserver.class.getName())
+      .setValue("hbase.rowlock.wait.duration", String.valueOf(5000)).build();
     TEST_UTIL.getAdmin().createTable(tableDescriptor);
     TEST_UTIL.waitTableAvailable(tableName, WAITTABLE_MILLIS);
 
@@ -833,15 +825,11 @@ public class TestFromClientSide3 {
 
   @Test
   public void testMultiRowMutations() throws Exception, Throwable {
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
-    ColumnFamilyDescriptor familyDescriptor =
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY);
-
-    tableDescriptor.setCoprocessor(MultiRowMutationEndpoint.class.getName());
-    tableDescriptor.setCoprocessor(WaitingForMultiMutationsObserver.class.getName());
-    tableDescriptor.setValue("hbase.rowlock.wait.duration", String.valueOf(5000));
-    tableDescriptor.setColumnFamily(familyDescriptor);
+    TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName)
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY))
+      .setCoprocessor(MultiRowMutationEndpoint.class.getName())
+      .setCoprocessor(WaitingForMultiMutationsObserver.class.getName())
+      .setValue("hbase.rowlock.wait.duration", String.valueOf(5000)).build();
     TEST_UTIL.getAdmin().createTable(tableDescriptor);
     TEST_UTIL.waitTableAvailable(tableName, WAITTABLE_MILLIS);
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide4.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide4.java
index 78aa56d..4c5985a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide4.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide4.java
@@ -24,6 +24,7 @@ import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
+
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -37,9 +38,7 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.ClusterMetrics.Option;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
-import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
 import org.apache.hadoop.hbase.TableName;
@@ -1309,29 +1308,31 @@ public class TestFromClientSide4 extends FromClientSideBase {
       // Test user metadata
       Admin admin = TEST_UTIL.getAdmin();
       // make a modifiable descriptor
-      HTableDescriptor desc = new HTableDescriptor(a.getDescriptor());
+      TableDescriptor desc = a.getDescriptor();
       // offline the table
       admin.disableTable(tableAname);
       // add a user attribute to HTD
-      desc.setValue(attrName, attrValue);
+      TableDescriptorBuilder builder =
+        TableDescriptorBuilder.newBuilder(desc).setValue(attrName, attrValue);
       // add a user attribute to HCD
-      for (HColumnDescriptor c : desc.getFamilies()) {
-        c.setValue(attrName, attrValue);
+      for (ColumnFamilyDescriptor c : desc.getColumnFamilies()) {
+        builder.modifyColumnFamily(
+          ColumnFamilyDescriptorBuilder.newBuilder(c).setValue(attrName, attrValue).build());
       }
       // update metadata for all regions of this table
-      admin.modifyTable(desc);
+      admin.modifyTable(builder.build());
       // enable the table
       admin.enableTable(tableAname);
 
       // Test that attribute changes were applied
-      desc = new HTableDescriptor(a.getDescriptor());
+      desc = a.getDescriptor();
       assertEquals("wrong table descriptor returned", desc.getTableName(), tableAname);
       // check HTD attribute
       value = desc.getValue(attrName);
       assertNotNull("missing HTD attribute value", value);
       assertFalse("HTD attribute value is incorrect", Bytes.compareTo(value, attrValue) != 0);
       // check HCD attribute
-      for (HColumnDescriptor c : desc.getFamilies()) {
+      for (ColumnFamilyDescriptor c : desc.getColumnFamilies()) {
         value = c.getValue(attrName);
         assertNotNull("missing HCD attribute value", value);
         assertFalse("HCD attribute value is incorrect", Bytes.compareTo(value, attrValue) != 0);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide5.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide5.java
index 6fb50bd..3213c0f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide5.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide5.java
@@ -2240,13 +2240,9 @@ public class TestFromClientSide5 extends FromClientSideBase {
   @Test
   public void testCellSizeLimit() throws IOException {
     final TableName tableName = name.getTableName();
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(tableName)
-        .setValue(HRegion.HBASE_MAX_CELL_SIZE_KEY, Integer.toString(10 * 1024));
-    ColumnFamilyDescriptor familyDescriptor =
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY);
-
-    tableDescriptor.setColumnFamily(familyDescriptor);
+    TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName)
+      .setValue(HRegion.HBASE_MAX_CELL_SIZE_KEY, Integer.toString(10 * 1024))
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY)).build();
     try (Admin admin = TEST_UTIL.getAdmin()) {
       admin.createTable(tableDescriptor);
     }
@@ -2279,12 +2275,9 @@ public class TestFromClientSide5 extends FromClientSideBase {
   @Test
   public void testCellSizeNoLimit() throws IOException {
     final TableName tableName = name.getTableName();
-    ColumnFamilyDescriptor familyDescriptor =
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY);
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(tableName)
-        .setValue(HRegion.HBASE_MAX_CELL_SIZE_KEY, Integer.toString(0));
-    tableDescriptor.setColumnFamily(familyDescriptor);
+    TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName)
+      .setValue(HRegion.HBASE_MAX_CELL_SIZE_KEY, Integer.toString(0))
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY)).build();
 
     try (Admin admin = TEST_UTIL.getAdmin()) {
       admin.createTable(tableDescriptor);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIllegalTableDescriptor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIllegalTableDescriptor.java
index 021a1d2..1a9e5b5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIllegalTableDescriptor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIllegalTableDescriptor.java
@@ -26,7 +26,6 @@ import static org.mockito.Mockito.verify;
 
 import java.io.IOException;
 import java.lang.reflect.Field;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
@@ -84,107 +83,104 @@ public class TestIllegalTableDescriptor {
 
   @Test
   public void testIllegalTableDescriptor() throws Exception {
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(
-        TableName.valueOf(name.getMethodName()));
-    ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY);
+    TableDescriptorBuilder builder =
+      TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()));
+    ColumnFamilyDescriptorBuilder cfBuilder = ColumnFamilyDescriptorBuilder.newBuilder(FAMILY);
 
     // create table with 0 families
-    checkTableIsIllegal(tableDescriptor);
-    tableDescriptor.setColumnFamily(familyDescriptor);
-    checkTableIsLegal(tableDescriptor);
-
-    tableDescriptor.setMaxFileSize(1024); // 1K
-    checkTableIsIllegal(tableDescriptor);
-    tableDescriptor.setMaxFileSize(0);
-    checkTableIsIllegal(tableDescriptor);
-    tableDescriptor.setMaxFileSize(1024 * 1024 * 1024); // 1G
-    checkTableIsLegal(tableDescriptor);
-
-    tableDescriptor.setMemStoreFlushSize(1024);
-    checkTableIsIllegal(tableDescriptor);
-    tableDescriptor.setMemStoreFlushSize(0);
-    checkTableIsIllegal(tableDescriptor);
-    tableDescriptor.setMemStoreFlushSize(128 * 1024 * 1024); // 128M
-    checkTableIsLegal(tableDescriptor);
-
-    tableDescriptor.setRegionSplitPolicyClassName("nonexisting.foo.class");
-    checkTableIsIllegal(tableDescriptor);
-    tableDescriptor.setRegionSplitPolicyClassName(null);
-    checkTableIsLegal(tableDescriptor);
-
-    tableDescriptor.setValue(HConstants.HBASE_REGION_SPLIT_POLICY_KEY, "nonexisting.foo.class");
-    checkTableIsIllegal(tableDescriptor);
-    tableDescriptor.removeValue(Bytes.toBytes(HConstants.HBASE_REGION_SPLIT_POLICY_KEY));
-    checkTableIsLegal(tableDescriptor);
-
-    familyDescriptor.setBlocksize(0);
-    checkTableIsIllegal(tableDescriptor);
-    familyDescriptor.setBlocksize(1024 * 1024 * 128); // 128M
-    checkTableIsIllegal(tableDescriptor);
-    familyDescriptor.setBlocksize(1024);
-    checkTableIsLegal(tableDescriptor);
-
-    familyDescriptor.setTimeToLive(0);
-    checkTableIsIllegal(tableDescriptor);
-    familyDescriptor.setTimeToLive(-1);
-    checkTableIsIllegal(tableDescriptor);
-    familyDescriptor.setTimeToLive(1);
-    checkTableIsLegal(tableDescriptor);
-
-    familyDescriptor.setMinVersions(-1);
-    checkTableIsIllegal(tableDescriptor);
-    familyDescriptor.setMinVersions(3);
+    checkTableIsIllegal(builder.build());
+    checkTableIsLegal(builder.setColumnFamily(cfBuilder.build()).build());
+
+    builder.setMaxFileSize(1024); // 1K
+    checkTableIsIllegal(builder.build());
+    builder.setMaxFileSize(0);
+    checkTableIsIllegal(builder.build());
+    builder.setMaxFileSize(1024 * 1024 * 1024); // 1G
+    checkTableIsLegal(builder.build());
+
+    builder.setMemStoreFlushSize(1024);
+    checkTableIsIllegal(builder.build());
+    builder.setMemStoreFlushSize(0);
+    checkTableIsIllegal(builder.build());
+    builder.setMemStoreFlushSize(128 * 1024 * 1024); // 128M
+    checkTableIsLegal(builder.build());
+
+    builder.setRegionSplitPolicyClassName("nonexisting.foo.class");
+    checkTableIsIllegal(builder.build());
+    builder.setRegionSplitPolicyClassName(null);
+    checkTableIsLegal(builder.build());
+
+    builder.setValue(HConstants.HBASE_REGION_SPLIT_POLICY_KEY, "nonexisting.foo.class");
+    checkTableIsIllegal(builder.build());
+    builder.removeValue(Bytes.toBytes(HConstants.HBASE_REGION_SPLIT_POLICY_KEY));
+    checkTableIsLegal(builder.build());
+
+    cfBuilder.setBlocksize(0);
+    checkTableIsIllegal(builder.modifyColumnFamily(cfBuilder.build()).build());
+    cfBuilder.setBlocksize(1024 * 1024 * 128); // 128M
+    checkTableIsIllegal(builder.modifyColumnFamily(cfBuilder.build()).build());
+    cfBuilder.setBlocksize(1024);
+    checkTableIsLegal(builder.modifyColumnFamily(cfBuilder.build()).build());
+
+    cfBuilder.setTimeToLive(0);
+    checkTableIsIllegal(builder.modifyColumnFamily(cfBuilder.build()).build());
+    cfBuilder.setTimeToLive(-1);
+    checkTableIsIllegal(builder.modifyColumnFamily(cfBuilder.build()).build());
+    cfBuilder.setTimeToLive(1);
+    checkTableIsLegal(builder.modifyColumnFamily(cfBuilder.build()).build());
+
+    cfBuilder.setMinVersions(-1);
+    checkTableIsIllegal(builder.modifyColumnFamily(cfBuilder.build()).build());
+    cfBuilder.setMinVersions(3);
     try {
-      familyDescriptor.setMaxVersions(2);
+      cfBuilder.setMaxVersions(2);
       fail();
     } catch (IllegalArgumentException ex) {
       // expected
-      familyDescriptor.setMaxVersions(10);
+      cfBuilder.setMaxVersions(10);
     }
-    checkTableIsLegal(tableDescriptor);
+    checkTableIsLegal(builder.modifyColumnFamily(cfBuilder.build()).build());
 
     // HBASE-13776 Setting illegal versions for HColumnDescriptor
     //  does not throw IllegalArgumentException
     // finally, minVersions must be less than or equal to maxVersions
-    familyDescriptor.setMaxVersions(4);
-    familyDescriptor.setMinVersions(5);
-    checkTableIsIllegal(tableDescriptor);
-    familyDescriptor.setMinVersions(3);
+    cfBuilder.setMaxVersions(4);
+    cfBuilder.setMinVersions(5);
+    checkTableIsIllegal(builder.modifyColumnFamily(cfBuilder.build()).build());
+    cfBuilder.setMinVersions(3);
 
-    familyDescriptor.setScope(-1);
-    checkTableIsIllegal(tableDescriptor);
-    familyDescriptor.setScope(0);
-    checkTableIsLegal(tableDescriptor);
+    cfBuilder.setScope(-1);
+    checkTableIsIllegal(builder.modifyColumnFamily(cfBuilder.build()).build());
+    cfBuilder.setScope(0);
+    checkTableIsLegal(builder.modifyColumnFamily(cfBuilder.build()).build());
 
-    familyDescriptor.setValue(ColumnFamilyDescriptorBuilder.IN_MEMORY_COMPACTION, "INVALID");
-    checkTableIsIllegal(tableDescriptor);
-    familyDescriptor.setValue(ColumnFamilyDescriptorBuilder.IN_MEMORY_COMPACTION, "NONE");
-    checkTableIsLegal(tableDescriptor);
+    cfBuilder.setValue(ColumnFamilyDescriptorBuilder.IN_MEMORY_COMPACTION, "INVALID");
+    checkTableIsIllegal(builder.modifyColumnFamily(cfBuilder.build()).build());
+    cfBuilder.setValue(ColumnFamilyDescriptorBuilder.IN_MEMORY_COMPACTION, "NONE");
+    checkTableIsLegal(builder.modifyColumnFamily(cfBuilder.build()).build());
 
     try {
-      familyDescriptor.setDFSReplication((short) -1);
+      cfBuilder.setDFSReplication((short) -1);
       fail("Illegal value for setDFSReplication did not throw");
     } catch (IllegalArgumentException e) {
       // pass
     }
     // set an illegal DFS replication value by hand
-    familyDescriptor.setValue(ColumnFamilyDescriptorBuilder.DFS_REPLICATION, "-1");
-    checkTableIsIllegal(tableDescriptor);
+    cfBuilder.setValue(ColumnFamilyDescriptorBuilder.DFS_REPLICATION, "-1");
+    checkTableIsIllegal(builder.modifyColumnFamily(cfBuilder.build()).build());
     try {
-      familyDescriptor.setDFSReplication((short) -1);
+      cfBuilder.setDFSReplication((short) -1);
       fail("Should throw exception if an illegal value is explicitly being set");
     } catch (IllegalArgumentException e) {
       // pass
     }
 
     // check the conf settings to disable sanity checks
-    tableDescriptor.setMemStoreFlushSize(0);
+    builder.setMemStoreFlushSize(0);
 
     // Check that logs warn on invalid table but allow it.
-    tableDescriptor.setValue(TableDescriptorChecker.TABLE_SANITY_CHECKS, Boolean.FALSE.toString());
-    checkTableIsLegal(tableDescriptor);
+    builder.setValue(TableDescriptorChecker.TABLE_SANITY_CHECKS, Boolean.FALSE.toString());
+    checkTableIsLegal(builder.build());
 
     verify(LOGGER).warn(contains("MEMSTORE_FLUSHSIZE for table "
         + "descriptor or \"hbase.hregion.memstore.flush.size\" (0) is too small, which might "
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIncrementsFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIncrementsFromClientSide.java
index 58c88b5..237eada 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIncrementsFromClientSide.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIncrementsFromClientSide.java
@@ -101,16 +101,16 @@ public class TestIncrementsFromClientSide {
    */
   @Test
   public void testDuplicateIncrement() throws Exception {
-    TableDescriptorBuilder.ModifyableTableDescriptor mtd =
+    TableDescriptorBuilder builder =
       TEST_UTIL.createModifyableTableDescriptor(name.getMethodName());
     Map<String, String> kvs = new HashMap<>();
     kvs.put(SleepAtFirstRpcCall.SLEEP_TIME_CONF_KEY, "2000");
-    mtd.setCoprocessor(CoprocessorDescriptorBuilder
+    builder.setCoprocessor(CoprocessorDescriptorBuilder
       .newBuilder(SleepAtFirstRpcCall.class.getName())
       .setPriority(1)
       .setProperties(kvs)
       .build());
-    TEST_UTIL.createTable(mtd, new byte[][] { ROW }).close();
+    TEST_UTIL.createTable(builder.build(), new byte[][] { ROW }).close();
 
     Configuration c = new Configuration(TEST_UTIL.getConfiguration());
     c.setInt(HConstants.HBASE_CLIENT_PAUSE, 50);
@@ -141,16 +141,16 @@ public class TestIncrementsFromClientSide {
    */
   @Test
   public void testDuplicateBatchIncrement() throws Exception {
-    TableDescriptorBuilder.ModifyableTableDescriptor mtd =
+    TableDescriptorBuilder builder =
       TEST_UTIL.createModifyableTableDescriptor(name.getMethodName());
     Map<String, String> kvs = new HashMap<>();
     kvs.put(SleepAtFirstRpcCall.SLEEP_TIME_CONF_KEY, "2000");
-    mtd.setCoprocessor(CoprocessorDescriptorBuilder
+    builder.setCoprocessor(CoprocessorDescriptorBuilder
       .newBuilder(SleepAtFirstRpcCall.class.getName())
       .setPriority(1)
       .setProperties(kvs)
       .build());
-    TEST_UTIL.createTable(mtd, new byte[][] { ROW }).close();
+    TEST_UTIL.createTable(builder.build(), new byte[][] { ROW }).close();
 
     Configuration c = new Configuration(TEST_UTIL.getConfiguration());
     c.setInt(HConstants.HBASE_CLIENT_PAUSE, 50);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIntraRowPagination.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIntraRowPagination.java
index 3567e50..7cada26 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIntraRowPagination.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIntraRowPagination.java
@@ -55,18 +55,15 @@ public class TestIntraRowPagination {
     byte [][] FAMILIES = HTestConst.makeNAscii(HTestConst.DEFAULT_CF_BYTES, 3);
     byte [][] QUALIFIERS = HTestConst.makeNAscii(HTestConst.DEFAULT_QUALIFIER_BYTES, 10);
 
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(
-        TableName.valueOf(HTestConst.DEFAULT_TABLE_BYTES));
+    TableDescriptorBuilder builder =
+      TableDescriptorBuilder.newBuilder(TableName.valueOf(HTestConst.DEFAULT_TABLE_BYTES));
 
     RegionInfo info = RegionInfoBuilder.newBuilder(HTestConst.DEFAULT_TABLE).build();
     for (byte[] family : FAMILIES) {
-      ColumnFamilyDescriptor familyDescriptor =
-        new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family);
-      tableDescriptor.setColumnFamily(familyDescriptor);
+      builder.setColumnFamily(ColumnFamilyDescriptorBuilder.of(family));
     }
     HRegion region = HBaseTestingUtility.createRegionAndWAL(info, TEST_UTIL.getDataTestDir(),
-        TEST_UTIL.getConfiguration(), tableDescriptor);
+        TEST_UTIL.getConfiguration(), builder.build());
     try {
       Put put;
       Scan scan;
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java
index 5019fb3..3215ea8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java
@@ -34,9 +34,7 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-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.Waiter;
 import org.apache.hadoop.hbase.coprocessor.CoreCoprocessor;
@@ -278,12 +276,14 @@ public class TestReplicaWithCluster {
   @Test
   public void testCreateDeleteTable() throws IOException {
     // Create table then get the single region for our new table.
-    HTableDescriptor hdt = HTU.createTableDescriptor(TableName.valueOf("testCreateDeleteTable"),
-      HColumnDescriptor.DEFAULT_MIN_VERSIONS, 3, HConstants.FOREVER,
-      HColumnDescriptor.DEFAULT_KEEP_DELETED);
-    hdt.setRegionReplication(NB_SERVERS);
-    hdt.addCoprocessor(SlowMeCopro.class.getName());
-    Table table = HTU.createTable(hdt, new byte[][]{f}, null);
+    TableDescriptorBuilder builder =
+      HTU.createModifyableTableDescriptor(TableName.valueOf("testCreateDeleteTable"),
+        ColumnFamilyDescriptorBuilder.DEFAULT_MIN_VERSIONS, 3, HConstants.FOREVER,
+        ColumnFamilyDescriptorBuilder.DEFAULT_KEEP_DELETED);
+    builder.setRegionReplication(NB_SERVERS);
+    builder.setCoprocessor(SlowMeCopro.class.getName());
+    TableDescriptor hdt = builder.build();
+    Table table = HTU.createTable(hdt, new byte[][] { f }, null);
 
     Put p = new Put(row);
     p.addColumn(f, row, row);
@@ -372,17 +372,14 @@ public class TestReplicaWithCluster {
   @SuppressWarnings("deprecation")
   @Test
   public void testReplicaAndReplication() throws Exception {
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
+    TableDescriptorBuilder builder =
       HTU.createModifyableTableDescriptor("testReplicaAndReplication");
-    tableDescriptor.setRegionReplication(NB_SERVERS);
+    builder.setRegionReplication(NB_SERVERS);
+    builder.setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(row)
+      .setScope(HConstants.REPLICATION_SCOPE_GLOBAL).build());
 
-    ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(row);
-
-    familyDescriptor.setScope(HConstants.REPLICATION_SCOPE_GLOBAL);
-    tableDescriptor.setColumnFamily(familyDescriptor);
-
-    tableDescriptor.setCoprocessor(SlowMeCopro.class.getName());
+    builder.setCoprocessor(SlowMeCopro.class.getName());
+    TableDescriptor tableDescriptor = builder.build();
     HTU.getAdmin().createTable(tableDescriptor, HBaseTestingUtility.KEYS_FOR_HBA_CREATE_TABLE);
 
     Configuration conf2 = HBaseConfiguration.create(HTU.getConfiguration());
@@ -461,12 +458,13 @@ public class TestReplicaWithCluster {
   public void testBulkLoad() throws IOException {
     // Create table then get the single region for our new table.
     LOG.debug("Creating test table");
-    HTableDescriptor hdt = HTU.createTableDescriptor(TableName.valueOf("testBulkLoad"),
-      HColumnDescriptor.DEFAULT_MIN_VERSIONS, 3, HConstants.FOREVER,
-      HColumnDescriptor.DEFAULT_KEEP_DELETED);
-    hdt.setRegionReplication(NB_SERVERS);
-    hdt.addCoprocessor(SlowMeCopro.class.getName());
-    Table table = HTU.createTable(hdt, new byte[][]{f}, null);
+    TableDescriptorBuilder builder = HTU.createModifyableTableDescriptor(
+      TableName.valueOf("testBulkLoad"), ColumnFamilyDescriptorBuilder.DEFAULT_MIN_VERSIONS, 3,
+      HConstants.FOREVER, ColumnFamilyDescriptorBuilder.DEFAULT_KEEP_DELETED);
+    builder.setRegionReplication(NB_SERVERS);
+    builder.setCoprocessor(SlowMeCopro.class.getName());
+    TableDescriptor hdt = builder.build();
+    Table table = HTU.createTable(hdt, new byte[][] { f }, null);
 
     // create hfiles to load.
     LOG.debug("Creating test data");
@@ -475,7 +473,7 @@ public class TestReplicaWithCluster {
     final byte[] qual = Bytes.toBytes("qual");
     final byte[] val  = Bytes.toBytes("val");
     Map<byte[], List<Path>> family2Files = new TreeMap<>(Bytes.BYTES_COMPARATOR);
-    for (HColumnDescriptor col : hdt.getColumnFamilies()) {
+    for (ColumnFamilyDescriptor col : hdt.getColumnFamilies()) {
       Path hfile = new Path(dir, col.getNameAsString());
       TestHRegionServerBulkLoad.createHFile(HTU.getTestFileSystem(), hfile, col.getName(), qual,
         val, numRows);
@@ -519,11 +517,13 @@ public class TestReplicaWithCluster {
   @Test
   public void testReplicaGetWithPrimaryDown() throws IOException {
     // Create table then get the single region for our new table.
-    HTableDescriptor hdt = HTU.createTableDescriptor(TableName.valueOf("testCreateDeleteTable"),
-      HColumnDescriptor.DEFAULT_MIN_VERSIONS, 3, HConstants.FOREVER,
-      HColumnDescriptor.DEFAULT_KEEP_DELETED);
-    hdt.setRegionReplication(NB_SERVERS);
-    hdt.addCoprocessor(RegionServerStoppedCopro.class.getName());
+    TableDescriptorBuilder builder =
+      HTU.createModifyableTableDescriptor(TableName.valueOf("testCreateDeleteTable"),
+        ColumnFamilyDescriptorBuilder.DEFAULT_MIN_VERSIONS, 3, HConstants.FOREVER,
+        ColumnFamilyDescriptorBuilder.DEFAULT_KEEP_DELETED);
+    builder.setRegionReplication(NB_SERVERS);
+    builder.setCoprocessor(RegionServerStoppedCopro.class.getName());
+    TableDescriptor hdt = builder.build();
     try {
       Table table = HTU.createTable(hdt, new byte[][] { f }, null);
 
@@ -555,12 +555,13 @@ public class TestReplicaWithCluster {
   @Test
   public void testReplicaScanWithPrimaryDown() throws IOException {
     // Create table then get the single region for our new table.
-    HTableDescriptor hdt = HTU.createTableDescriptor(TableName.valueOf("testCreateDeleteTable"),
-      HColumnDescriptor.DEFAULT_MIN_VERSIONS, 3, HConstants.FOREVER,
-      HColumnDescriptor.DEFAULT_KEEP_DELETED);
-    hdt.setRegionReplication(NB_SERVERS);
-    hdt.addCoprocessor(RegionServerStoppedCopro.class.getName());
-
+    TableDescriptorBuilder builder =
+      HTU.createModifyableTableDescriptor(TableName.valueOf("testCreateDeleteTable"),
+        ColumnFamilyDescriptorBuilder.DEFAULT_MIN_VERSIONS, 3, HConstants.FOREVER,
+        ColumnFamilyDescriptorBuilder.DEFAULT_KEEP_DELETED);
+    builder.setRegionReplication(NB_SERVERS);
+    builder.setCoprocessor(RegionServerStoppedCopro.class.getName());
+    TableDescriptor hdt = builder.build();
     try {
       Table table = HTU.createTable(hdt, new byte[][] { f }, null);
 
@@ -601,16 +602,16 @@ public class TestReplicaWithCluster {
   @Test
   public void testReplicaGetWithAsyncRpcClientImpl() throws IOException {
     HTU.getConfiguration().setBoolean("hbase.ipc.client.specificThreadForWriting", true);
-    HTU.getConfiguration().set(
-        "hbase.rpc.client.impl", "org.apache.hadoop.hbase.ipc.AsyncRpcClient");
+    HTU.getConfiguration().set("hbase.rpc.client.impl",
+      "org.apache.hadoop.hbase.ipc.AsyncRpcClient");
     // Create table then get the single region for our new table.
-    HTableDescriptor hdt = HTU.createTableDescriptor(
-      TableName.valueOf("testReplicaGetWithAsyncRpcClientImpl"),
-      HColumnDescriptor.DEFAULT_MIN_VERSIONS, 3, HConstants.FOREVER,
-      HColumnDescriptor.DEFAULT_KEEP_DELETED);
-    hdt.setRegionReplication(NB_SERVERS);
-    hdt.addCoprocessor(SlowMeCopro.class.getName());
-
+    TableDescriptorBuilder builder =
+      HTU.createModifyableTableDescriptor(TableName.valueOf("testReplicaGetWithAsyncRpcClientImpl"),
+        ColumnFamilyDescriptorBuilder.DEFAULT_MIN_VERSIONS, 3, HConstants.FOREVER,
+        ColumnFamilyDescriptorBuilder.DEFAULT_KEEP_DELETED);
+    builder.setRegionReplication(NB_SERVERS);
+    builder.setCoprocessor(SlowMeCopro.class.getName());
+    TableDescriptor hdt = builder.build();
     try {
       Table table = HTU.createTable(hdt, new byte[][] { f }, null);
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicasClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicasClient.java
index d7fc8e0..a64f296 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicasClient.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicasClient.java
@@ -31,10 +31,8 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionLocation;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.NotServingRegionException;
 import org.apache.hadoop.hbase.StartMiniClusterOption;
 import org.apache.hadoop.hbase.TableName;
@@ -195,11 +193,12 @@ public class TestReplicasClient {
     HTU.startMiniCluster(option);
 
     // Create table then get the single region for our new table.
-    HTableDescriptor hdt = HTU.createTableDescriptor(
+    TableDescriptorBuilder builder = HTU.createModifyableTableDescriptor(
       TableName.valueOf(TestReplicasClient.class.getSimpleName()),
-      HColumnDescriptor.DEFAULT_MIN_VERSIONS, 3, HConstants.FOREVER,
-      HColumnDescriptor.DEFAULT_KEEP_DELETED);
-    hdt.addCoprocessor(SlowMeCopro.class.getName());
+      ColumnFamilyDescriptorBuilder.DEFAULT_MIN_VERSIONS, 3, HConstants.FOREVER,
+      ColumnFamilyDescriptorBuilder.DEFAULT_KEEP_DELETED);
+    builder.setCoprocessor(SlowMeCopro.class.getName());
+    TableDescriptor hdt = builder.build();
     HTU.createTable(hdt, new byte[][]{f}, null);
     TABLE_NAME = hdt.getTableName();
     try (RegionLocator locator = HTU.getConnection().getRegionLocator(hdt.getTableName())) {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java
index 1dce029..00b8a64 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java
@@ -768,8 +768,7 @@ public class TestScannersFromClientSide {
       table.put(new Put(ROW).addColumn(FAMILY, QUALIFIER, ts, value));
       assertArrayEquals(value, table.get(new Get(ROW)).getValue(FAMILY, QUALIFIER));
       TEST_UTIL.getAdmin().modifyColumnFamily(tableName,
-        new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY)
-          .setTimeToLive(5));
+        ColumnFamilyDescriptorBuilder.newBuilder(FAMILY).setTimeToLive(5).build());
       try (ResultScanner scanner = table.getScanner(FAMILY)) {
         assertNull(scanner.next());
       }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestServerBusyException.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestServerBusyException.java
index 0dce2f3..9e6f702 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestServerBusyException.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestServerBusyException.java
@@ -28,8 +28,6 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
@@ -185,8 +183,8 @@ public class TestServerBusyException {
 
   @Test()
   public void testServerBusyException() throws Exception {
-    HTableDescriptor hdt = TEST_UTIL.createTableDescriptor(TableName.valueOf(name.getMethodName()));
-    hdt.addCoprocessor(SleepCoprocessor.class.getName());
+    TableDescriptor hdt = TEST_UTIL.createModifyableTableDescriptor(name.getMethodName())
+      .setCoprocessor(SleepCoprocessor.class.getName()).build();
     Configuration c = new Configuration(TEST_UTIL.getConfiguration());
     TEST_UTIL.createTable(hdt, new byte[][] { FAM_NAM }, c);
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSizeFailures.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSizeFailures.java
index ecb6973..85a912a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSizeFailures.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSizeFailures.java
@@ -23,7 +23,6 @@ import java.util.ArrayList;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map.Entry;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.TableName;
@@ -60,7 +59,6 @@ public class TestSizeFailures {
     //((Log4JLogger)RpcServer.LOG).getLogger().setLevel(Level.ALL);
     //((Log4JLogger)RpcClient.LOG).getLogger().setLevel(Level.ALL);
     //((Log4JLogger)ScannerCallable.LOG).getLogger().setLevel(Level.ALL);
-    Configuration conf = TEST_UTIL.getConfiguration();
     TEST_UTIL.startMiniCluster(SLAVES);
 
     // Write a bunch of data
@@ -70,12 +68,8 @@ public class TestSizeFailures {
       qualifiers.add(Bytes.toBytes(Integer.toString(i)));
     }
 
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(TABLENAME);
-    ColumnFamilyDescriptor familyDescriptor =
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY);
-
-    tableDescriptor.setColumnFamily(familyDescriptor);
+    TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(TABLENAME)
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY)).build();
     byte[][] splits = new byte[9][2];
     for (int i = 1; i < 10; i++) {
       int split = 48 + i;
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotCloneIndependence.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotCloneIndependence.java
index 9b35e70..cfa0b6d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotCloneIndependence.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotCloneIndependence.java
@@ -25,7 +25,6 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
 import org.apache.hadoop.hbase.regionserver.ConstantSizeRegionSplitPolicy;
@@ -65,7 +64,6 @@ public class TestSnapshotCloneIndependence {
   protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
 
   protected static final int NUM_RS = 2;
-  private static final String STRING_TABLE_NAME = "test";
   private static final String TEST_FAM_STR = "fam";
   protected static final byte[] TEST_FAM = Bytes.toBytes(TEST_FAM_STR);
   private static final int CLEANER_INTERVAL = 100;
@@ -321,8 +319,7 @@ public class TestSnapshotCloneIndependence {
   private void runTestSnapshotMetadataChangesIndependent() throws Exception {
     // Add a new column family to the original table
     byte[] TEST_FAM_2 = Bytes.toBytes("fam2");
-    ColumnFamilyDescriptor familyDescriptor =
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(TEST_FAM_2);
+    ColumnFamilyDescriptor familyDescriptor = ColumnFamilyDescriptorBuilder.of(TEST_FAM_2);
 
     admin.disableTable(originalTableName);
     admin.addColumnFamily(originalTableName, familyDescriptor);
@@ -334,19 +331,18 @@ public class TestSnapshotCloneIndependence {
     // get a description of the cloned table
     // get a list of its families
     // assert that the family is there
-    HTableDescriptor originalTableDescriptor = new HTableDescriptor(originalTable.getDescriptor());
-    HTableDescriptor clonedTableDescriptor =
-      new HTableDescriptor(admin.getDescriptor(cloneTableName));
+    TableDescriptor originalTableDescriptor = originalTable.getDescriptor();
+    TableDescriptor clonedTableDescriptor = admin.getDescriptor(cloneTableName);
 
     Assert.assertTrue("The original family was not found. There is something wrong. ",
-      originalTableDescriptor.hasFamily(TEST_FAM));
+      originalTableDescriptor.hasColumnFamily(TEST_FAM));
     Assert.assertTrue("The original family was not found in the clone. There is something wrong. ",
-      clonedTableDescriptor.hasFamily(TEST_FAM));
+      clonedTableDescriptor.hasColumnFamily(TEST_FAM));
 
     Assert.assertTrue("The new family was not found. ",
-      originalTableDescriptor.hasFamily(TEST_FAM_2));
+      originalTableDescriptor.hasColumnFamily(TEST_FAM_2));
     Assert.assertTrue("The new family was not found. ",
-      !clonedTableDescriptor.hasFamily(TEST_FAM_2));
+      !clonedTableDescriptor.hasColumnFamily(TEST_FAM_2));
   }
 
   /**
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java
index 9c57920..63bf72f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java
@@ -31,7 +31,6 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
@@ -119,9 +118,9 @@ public class TestSnapshotFromClient {
   }
 
   protected void createTable() throws Exception {
-    HTableDescriptor htd = new HTableDescriptor(TABLE_NAME);
-    htd.setRegionReplication(getNumReplicas());
-    UTIL.createTable(htd, new byte[][]{TEST_FAM}, null);
+    TableDescriptor htd =
+      TableDescriptorBuilder.newBuilder(TABLE_NAME).setRegionReplication(getNumReplicas()).build();
+    UTIL.createTable(htd, new byte[][] { TEST_FAM }, null);
   }
 
   protected int getNumReplicas() {
@@ -321,7 +320,7 @@ public class TestSnapshotFromClient {
     try {
       admin = UTIL.getAdmin();
 
-      HTableDescriptor htd = new HTableDescriptor(tableName);
+      TableDescriptor htd = TableDescriptorBuilder.newBuilder(tableName).build();
       UTIL.createTable(htd, new byte[][] { TEST_FAM }, UTIL.getConfiguration());
 
       String table1Snapshot1 = "Table1Snapshot1";
@@ -406,7 +405,7 @@ public class TestSnapshotFromClient {
     try {
       admin = UTIL.getAdmin();
 
-      HTableDescriptor htd = new HTableDescriptor(tableName);
+      TableDescriptor htd = TableDescriptorBuilder.newBuilder(tableName).build();
       UTIL.createTable(htd, new byte[][] { TEST_FAM }, UTIL.getConfiguration());
 
       String table1Snapshot1 = "Table1Snapshot1";
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotMetadata.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotMetadata.java
index 77462f8..4c5afb1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotMetadata.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotMetadata.java
@@ -29,7 +29,6 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
 import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
@@ -91,7 +90,7 @@ public class TestSnapshotMetadata {
 
   private Admin admin;
   private String originalTableDescription;
-  private HTableDescriptor originalTableDescriptor;
+  private TableDescriptor originalTableDescriptor;
   TableName originalTableName;
 
   private static FileSystem fs;
@@ -155,35 +154,26 @@ public class TestSnapshotMetadata {
     originalTableName = TableName.valueOf(sourceTableNameAsString);
 
     // enable replication on a column family
-    ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor maxVersionsColumn =
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(MAX_VERSIONS_FAM);
-    ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor bloomFilterColumn =
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(BLOOMFILTER_FAM);
-    ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor dataBlockColumn =
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(COMPRESSED_FAM);
-    ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor blockSizeColumn =
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(BLOCKSIZE_FAM);
-
-    maxVersionsColumn.setMaxVersions(MAX_VERSIONS);
-    bloomFilterColumn.setBloomFilterType(BLOOM_TYPE);
-    dataBlockColumn.setDataBlockEncoding(DATA_BLOCK_ENCODING_TYPE);
-    blockSizeColumn.setBlocksize(BLOCK_SIZE);
-
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(
-        TableName.valueOf(sourceTableNameAsString));
-    tableDescriptor.setColumnFamily(maxVersionsColumn);
-    tableDescriptor.setColumnFamily(bloomFilterColumn);
-    tableDescriptor.setColumnFamily(dataBlockColumn);
-    tableDescriptor.setColumnFamily(blockSizeColumn);
-    tableDescriptor.setValue(TEST_CUSTOM_VALUE, TEST_CUSTOM_VALUE);
-    tableDescriptor.setValue(TEST_CONF_CUSTOM_VALUE, TEST_CONF_CUSTOM_VALUE);
-    assertTrue(tableDescriptor.getConfiguration().size() > 0);
+    ColumnFamilyDescriptor maxVersionsColumn = ColumnFamilyDescriptorBuilder
+      .newBuilder(MAX_VERSIONS_FAM).setMaxVersions(MAX_VERSIONS).build();
+    ColumnFamilyDescriptor bloomFilterColumn = ColumnFamilyDescriptorBuilder
+      .newBuilder(BLOOMFILTER_FAM).setBloomFilterType(BLOOM_TYPE).build();
+    ColumnFamilyDescriptor dataBlockColumn = ColumnFamilyDescriptorBuilder
+      .newBuilder(COMPRESSED_FAM).setDataBlockEncoding(DATA_BLOCK_ENCODING_TYPE).build();
+    ColumnFamilyDescriptor blockSizeColumn =
+      ColumnFamilyDescriptorBuilder.newBuilder(BLOCKSIZE_FAM).setBlocksize(BLOCK_SIZE).build();
+
+    TableDescriptor tableDescriptor = TableDescriptorBuilder
+      .newBuilder(TableName.valueOf(sourceTableNameAsString)).setColumnFamily(maxVersionsColumn)
+      .setColumnFamily(bloomFilterColumn).setColumnFamily(dataBlockColumn)
+      .setColumnFamily(blockSizeColumn).setValue(TEST_CUSTOM_VALUE, TEST_CUSTOM_VALUE)
+      .setValue(TEST_CONF_CUSTOM_VALUE, TEST_CONF_CUSTOM_VALUE).build();
+    assertTrue(tableDescriptor.getValues().size() > 0);
 
     admin.createTable(tableDescriptor);
     Table original = UTIL.getConnection().getTable(originalTableName);
     originalTableName = TableName.valueOf(sourceTableNameAsString);
-    originalTableDescriptor = new HTableDescriptor(admin.getDescriptor(originalTableName));
+    originalTableDescriptor = admin.getDescriptor(originalTableName);
     originalTableDescription = originalTableDescriptor.toStringCustomizedValues();
 
     original.close();
@@ -212,7 +202,7 @@ public class TestSnapshotMetadata {
 
     admin.cloneSnapshot(snapshotName, clonedTableName);
     Table clonedTable = UTIL.getConnection().getTable(clonedTableName);
-    HTableDescriptor cloneHtd = new HTableDescriptor(admin.getDescriptor(clonedTableName));
+    TableDescriptor cloneHtd = admin.getDescriptor(clonedTableName);
     assertEquals(
       originalTableDescription.replace(originalTableName.getNameAsString(),clonedTableNameAsString),
       cloneHtd.toStringCustomizedValues());
@@ -220,12 +210,9 @@ public class TestSnapshotMetadata {
     // Verify the custom fields
     assertEquals(originalTableDescriptor.getValues().size(),
                         cloneHtd.getValues().size());
-    assertEquals(originalTableDescriptor.getConfiguration().size(),
-                        cloneHtd.getConfiguration().size());
     assertEquals(TEST_CUSTOM_VALUE, cloneHtd.getValue(TEST_CUSTOM_VALUE));
-    assertEquals(TEST_CONF_CUSTOM_VALUE, cloneHtd.getConfigurationValue(TEST_CONF_CUSTOM_VALUE));
+    assertEquals(TEST_CONF_CUSTOM_VALUE, cloneHtd.getValue(TEST_CONF_CUSTOM_VALUE));
     assertEquals(originalTableDescriptor.getValues(), cloneHtd.getValues());
-    assertEquals(originalTableDescriptor.getConfiguration(), cloneHtd.getConfiguration());
 
     admin.enableTable(originalTableName);
     clonedTable.close();
@@ -304,8 +291,7 @@ public class TestSnapshotMetadata {
       final byte[] newFamilyName = Bytes.toBytes(newFamilyNameAsString);
 
       admin.disableTable(originalTableName);
-      ColumnFamilyDescriptor familyDescriptor =
-        new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(newFamilyName);
+      ColumnFamilyDescriptor familyDescriptor = ColumnFamilyDescriptorBuilder.of(newFamilyName);
       admin.addColumnFamily(originalTableName, familyDescriptor);
       assertTrue("New column family was not added.",
         admin.getDescriptor(originalTableName).toString().contains(newFamilyNameAsString));
@@ -321,9 +307,8 @@ public class TestSnapshotMetadata {
 
     // verify that the descrption is reverted
     try (Table original = UTIL.getConnection().getTable(originalTableName)) {
-      assertEquals(originalTableDescriptor,
-        new HTableDescriptor(admin.getDescriptor(originalTableName)));
-      assertEquals(originalTableDescriptor, new HTableDescriptor(original.getDescriptor()));
+      assertEquals(originalTableDescriptor, admin.getDescriptor(originalTableName));
+      assertEquals(originalTableDescriptor, original.getDescriptor());
     }
   }
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotTemporaryDirectory.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotTemporaryDirectory.java
index e61b1d2..aa2ecd4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotTemporaryDirectory.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotTemporaryDirectory.java
@@ -37,7 +37,6 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
 import org.apache.hadoop.hbase.regionserver.ConstantSizeRegionSplitPolicy;
@@ -128,8 +127,8 @@ public class TestSnapshotTemporaryDirectory {
 
   @Before
   public void setup() throws Exception {
-    HTableDescriptor htd = new HTableDescriptor(TABLE_NAME);
-    htd.setRegionReplication(getNumReplicas());
+    TableDescriptor htd =
+      TableDescriptorBuilder.newBuilder(TABLE_NAME).setRegionReplication(getNumReplicas()).build();
     UTIL.createTable(htd, new byte[][] { TEST_FAM }, UTIL.getConfiguration());
   }
 
@@ -426,7 +425,7 @@ public class TestSnapshotTemporaryDirectory {
     Admin admin = UTIL.getAdmin();
     TableName tableName2 = TableName.valueOf("testListTableSnapshots");
     try {
-      HTableDescriptor htd = new HTableDescriptor(tableName2);
+      TableDescriptor htd = TableDescriptorBuilder.newBuilder(tableName2).build();
       UTIL.createTable(htd, new byte[][] { TEST_FAM }, UTIL.getConfiguration());
 
       String table1Snapshot1 = "Table1Snapshot1";
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorConfiguration.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorConfiguration.java
index 94b3948..84d4046 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorConfiguration.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorConfiguration.java
@@ -29,10 +29,11 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.CoprocessorEnvironment;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
 import org.apache.hadoop.hbase.master.MasterServices;
 import org.apache.hadoop.hbase.regionserver.HRegion;
@@ -70,10 +71,11 @@ public class TestCoprocessorConfiguration {
   }
   private static final TableName TABLENAME = TableName.valueOf("TestCoprocessorConfiguration");
   private static final RegionInfo REGIONINFO = RegionInfoBuilder.newBuilder(TABLENAME).build();
-  private static final HTableDescriptor TABLEDESC = new HTableDescriptor(TABLENAME);
+  private static final TableDescriptor TABLEDESC;
   static {
     try {
-      TABLEDESC.addCoprocessor(TableCoprocessor.class.getName());
+      TABLEDESC = TableDescriptorBuilder.newBuilder(TABLENAME)
+        .setCoprocessor(TableCoprocessor.class.getName()).build();
     } catch (IOException e) {
       throw new RuntimeException(e);
     }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java
index 03e031a..a21b63a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java
@@ -409,14 +409,11 @@ public class TestCoprocessorInterface {
     return r;
   }
 
-  HRegion initHRegion (TableName tableName, String callingMethod,
-      Configuration conf, Class<?> [] implClasses, byte [][] families)
-      throws IOException {
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
+  HRegion initHRegion(TableName tableName, String callingMethod, Configuration conf,
+    Class<?>[] implClasses, byte[][] families) throws IOException {
+    TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableName);
     for (byte[] family : families) {
-      tableDescriptor.setColumnFamily(
-        new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family));
+      builder.setColumnFamily(ColumnFamilyDescriptorBuilder.of(family));
     }
     ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     RegionInfo info = RegionInfoBuilder.newBuilder(tableName)
@@ -425,7 +422,7 @@ public class TestCoprocessorInterface {
         .setSplit(false)
         .build();
     Path path = new Path(DIR + callingMethod);
-    HRegion r = HBaseTestingUtility.createRegionAndWAL(info, path, conf, tableDescriptor);
+    HRegion r = HBaseTestingUtility.createRegionAndWAL(info, path, conf, builder.build());
 
     // this following piece is a hack.
     RegionCoprocessorHost host =
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorMetrics.java
index 0ca7e0c..d7ac38a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorMetrics.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorMetrics.java
@@ -398,12 +398,11 @@ public class TestCoprocessorMetrics {
   public void testRegionObserverSingleRegion() throws IOException {
     final TableName tableName = TableName.valueOf(name.getMethodName());
     try (Connection connection = ConnectionFactory.createConnection(UTIL.getConfiguration());
-         Admin admin = connection.getAdmin()) {
-      admin.createTable(
-        new TableDescriptorBuilder.ModifyableTableDescriptor(tableName)
-          .setColumnFamily(new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(foo))
-          // add the coprocessor for the region
-          .setCoprocessor(CustomRegionObserver.class.getName()));
+      Admin admin = connection.getAdmin()) {
+      admin.createTable(TableDescriptorBuilder.newBuilder(tableName)
+        .setColumnFamily(ColumnFamilyDescriptorBuilder.of(foo))
+        // add the coprocessor for the region
+        .setCoprocessor(CustomRegionObserver.class.getName()).build());
       try (Table table = connection.getTable(tableName)) {
         table.get(new Get(foo));
         table.get(new Get(foo)); // 2 gets
@@ -418,12 +417,10 @@ public class TestCoprocessorMetrics {
     final TableName tableName = TableName.valueOf(name.getMethodName());
     try (Connection connection = ConnectionFactory.createConnection(UTIL.getConfiguration());
          Admin admin = connection.getAdmin()) {
-      admin.createTable(
-        new TableDescriptorBuilder.ModifyableTableDescriptor(tableName)
-          .setColumnFamily(
-            new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(foo))
-          // add the coprocessor for the region
-          .setCoprocessor(CustomRegionObserver.class.getName()), new byte[][]{foo});
+      admin.createTable(TableDescriptorBuilder.newBuilder(tableName)
+        .setColumnFamily(ColumnFamilyDescriptorBuilder.of(foo))
+        // add the coprocessor for the region
+        .setCoprocessor(CustomRegionObserver.class.getName()).build(), new byte[][] { foo });
       // create with 2 regions
       try (Table table = connection.getTable(tableName);
            RegionLocator locator = connection.getRegionLocator(tableName)) {
@@ -444,18 +441,14 @@ public class TestCoprocessorMetrics {
     final TableName tableName2 = TableName.valueOf(name.getMethodName() + "2");
     try (Connection connection = ConnectionFactory.createConnection(UTIL.getConfiguration());
          Admin admin = connection.getAdmin()) {
-      admin.createTable(
-        new TableDescriptorBuilder.ModifyableTableDescriptor(tableName1)
-          .setColumnFamily(
-            new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(foo))
-          // add the coprocessor for the region
-          .setCoprocessor(CustomRegionObserver.class.getName()));
-      admin.createTable(
-        new TableDescriptorBuilder.ModifyableTableDescriptor(tableName2)
-          .setColumnFamily(
-            new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(foo))
-          // add the coprocessor for the region
-          .setCoprocessor(CustomRegionObserver.class.getName()));
+      admin.createTable(TableDescriptorBuilder.newBuilder(tableName1)
+        .setColumnFamily(ColumnFamilyDescriptorBuilder.of(foo))
+        // add the coprocessor for the region
+        .setCoprocessor(CustomRegionObserver.class.getName()).build());
+      admin.createTable(TableDescriptorBuilder.newBuilder(tableName2)
+        .setColumnFamily(ColumnFamilyDescriptorBuilder.of(foo))
+        // add the coprocessor for the region
+        .setCoprocessor(CustomRegionObserver.class.getName()).build());
       try (Table table1 = connection.getTable(tableName1);
            Table table2 = connection.getTable(tableName2)) {
         table1.get(new Get(bar));
@@ -470,13 +463,11 @@ public class TestCoprocessorMetrics {
     final TableName tableName = TableName.valueOf(name.getMethodName());
     try (Connection connection = ConnectionFactory.createConnection(UTIL.getConfiguration());
          Admin admin = connection.getAdmin()) {
-      admin.createTable(
-        new TableDescriptorBuilder.ModifyableTableDescriptor(tableName)
-          .setColumnFamily(
-            new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(foo))
-          // add the coprocessor for the region. We add two different coprocessors
-          .setCoprocessor(CustomRegionObserver.class.getName())
-          .setCoprocessor(CustomRegionObserver2.class.getName()));
+      admin.createTable(TableDescriptorBuilder.newBuilder(tableName)
+        .setColumnFamily(ColumnFamilyDescriptorBuilder.of(foo))
+        // add the coprocessor for the region. We add two different coprocessors
+        .setCoprocessor(CustomRegionObserver.class.getName())
+        .setCoprocessor(CustomRegionObserver2.class.getName()).build());
       try (Table table = connection.getTable(tableName)) {
         table.get(new Get(foo));
         table.get(new Get(foo)); // 2 gets
@@ -493,12 +484,10 @@ public class TestCoprocessorMetrics {
     final TableName tableName = TableName.valueOf(name.getMethodName());
     try (Connection connection = ConnectionFactory.createConnection(UTIL.getConfiguration());
          Admin admin = connection.getAdmin()) {
-      admin.createTable(
-        new TableDescriptorBuilder.ModifyableTableDescriptor(tableName)
-          .setColumnFamily(
-            new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(foo))
-          // add the coprocessor for the region
-          .setCoprocessor(CustomRegionObserver.class.getName()), new byte[][]{foo});
+      admin.createTable(TableDescriptorBuilder.newBuilder(tableName)
+        .setColumnFamily(ColumnFamilyDescriptorBuilder.of(foo))
+        // add the coprocessor for the region
+        .setCoprocessor(CustomRegionObserver.class.getName()).build(), new byte[][] { foo });
       // create with 2 regions
       try (Table table = connection.getTable(tableName)) {
         table.get(new Get(foo));
@@ -538,12 +527,10 @@ public class TestCoprocessorMetrics {
     final TableName tableName = TableName.valueOf(name.getMethodName());
     try (Connection connection = ConnectionFactory.createConnection(UTIL.getConfiguration());
          Admin admin = connection.getAdmin()) {
-      admin.createTable(
-        new TableDescriptorBuilder.ModifyableTableDescriptor(tableName)
-          .setColumnFamily(
-            new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(foo))
-          // add the coprocessor for the region
-          .setCoprocessor(CustomRegionEndpoint.class.getName()));
+      admin.createTable(TableDescriptorBuilder.newBuilder(tableName)
+        .setColumnFamily(ColumnFamilyDescriptorBuilder.of(foo))
+        // add the coprocessor for the region
+        .setCoprocessor(CustomRegionEndpoint.class.getName()).build());
 
       try (Table table = connection.getTable(tableName)) {
         List<Mutation> mutations = Lists.newArrayList(new Put(foo), new Put(bar));
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterCoprocessorExceptionWithAbort.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterCoprocessorExceptionWithAbort.java
index a0b0403..2a4cf61 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterCoprocessorExceptionWithAbort.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterCoprocessorExceptionWithAbort.java
@@ -33,7 +33,6 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.TableDescriptor;
@@ -89,11 +88,9 @@ public class TestMasterCoprocessorExceptionWithAbort {
     public void run() {
       // create a table : master coprocessor will throw an exception and not
       // catch it.
-      TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-        new TableDescriptorBuilder.ModifyableTableDescriptor(TableName.valueOf(TEST_TABLE));
-      ColumnFamilyDescriptor familyDescriptor =
-        new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(TEST_FAMILY);
-      tableDescriptor.setColumnFamily(familyDescriptor);
+      TableDescriptor tableDescriptor =
+        TableDescriptorBuilder.newBuilder(TableName.valueOf(TEST_TABLE))
+          .setColumnFamily(ColumnFamilyDescriptorBuilder.of(TEST_FAMILY)).build();
       try {
         Admin admin = UTIL.getAdmin();
         admin.createTable(tableDescriptor);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterCoprocessorExceptionWithRemove.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterCoprocessorExceptionWithRemove.java
index c4636fb..5379bb3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterCoprocessorExceptionWithRemove.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterCoprocessorExceptionWithRemove.java
@@ -196,11 +196,9 @@ public class TestMasterCoprocessorExceptionWithRemove {
         BuggyMasterObserver.class.getName();
     assertTrue(HMaster.getLoadedCoprocessors().contains(coprocessorName));
 
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor1 =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(TableName.valueOf(TEST_TABLE1));
-
-    tableDescriptor1.setColumnFamily(
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(TEST_FAMILY1));
+    TableDescriptor tableDescriptor1 =
+      TableDescriptorBuilder.newBuilder(TableName.valueOf(TEST_TABLE1))
+        .setColumnFamily(ColumnFamilyDescriptorBuilder.of(TEST_FAMILY1)).build();
 
     boolean threwDNRE = false;
     try {
@@ -229,10 +227,9 @@ public class TestMasterCoprocessorExceptionWithRemove {
 
     // Verify that BuggyMasterObserver has been removed due to its misbehavior
     // by creating another table: should not have a problem this time.
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor2 =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(TableName.valueOf(TEST_TABLE2));
-    tableDescriptor2.setColumnFamily(
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(TEST_FAMILY2));
+    TableDescriptor tableDescriptor2 =
+      TableDescriptorBuilder.newBuilder(TableName.valueOf(TEST_TABLE2))
+        .setColumnFamily(ColumnFamilyDescriptorBuilder.of(TEST_FAMILY2)).build();
     Admin admin = UTIL.getAdmin();
     try {
       admin.createTable(tableDescriptor2);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
index eef3229..7120758 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
@@ -33,7 +33,6 @@ import org.apache.hadoop.hbase.CoprocessorEnvironment;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HRegionLocation;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.ServerName;
@@ -1324,11 +1323,8 @@ public class TestMasterObserver {
     assertFalse("No table created yet", cp.wasCreateTableCalled());
 
     // create a table
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
-
-    tableDescriptor.setColumnFamily(
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(TEST_FAMILY));
+    TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName)
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(TEST_FAMILY)).build();
     try(Connection connection = ConnectionFactory.createConnection(UTIL.getConfiguration());
         Admin admin = connection.getAdmin()) {
       tableCreationLatch = new CountDownLatch(1);
@@ -1371,7 +1367,8 @@ public class TestMasterObserver {
       assertTrue(admin.isTableDisabled(tableName));
 
       // modify table
-      tableDescriptor.setMaxFileSize(512 * 1024 * 1024);
+      tableDescriptor = TableDescriptorBuilder.newBuilder(tableDescriptor)
+        .setMaxFileSize(512 * 1024 * 1024).build();
       modifyTableSync(admin, tableName, tableDescriptor);
       assertTrue("Test table should have been modified",
         cp.wasModifyTableCalled());
@@ -1412,7 +1409,8 @@ public class TestMasterObserver {
         cp.wasDisableTableActionCalled());
 
       // modify table
-      tableDescriptor.setMaxFileSize(512 * 1024 * 1024);
+      tableDescriptor = TableDescriptorBuilder.newBuilder(tableDescriptor)
+        .setMaxFileSize(512 * 1024 * 1024).build();
       modifyTableSync(admin, tableName, tableDescriptor);
       assertTrue("Test table should have been modified",
         cp.wasModifyTableCalled());
@@ -1455,11 +1453,8 @@ public class TestMasterObserver {
     cp.resetStates();
 
     // create a table
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
-
-    tableDescriptor.setColumnFamily(
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(TEST_FAMILY));
+    TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName)
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(TEST_FAMILY)).build();
     Admin admin = UTIL.getAdmin();
 
     tableCreationLatch = new CountDownLatch(1);
@@ -1535,12 +1530,11 @@ public class TestMasterObserver {
   }
 
   private void modifyTableSync(Admin admin, TableName tableName, TableDescriptor tableDescriptor)
-      throws IOException {
+    throws IOException {
     admin.modifyTable(tableDescriptor);
-    //wait until modify table finishes
-    for (int t = 0; t < 100; t++) { //10 sec timeout
-      HTableDescriptor td = new HTableDescriptor(
-        admin.getDescriptor(tableDescriptor.getTableName()));
+    // wait until modify table finishes
+    for (int t = 0; t < 100; t++) { // 10 sec timeout
+      TableDescriptor td = admin.getDescriptor(tableDescriptor.getTableName());
       if (td.equals(tableDescriptor)) {
         break;
       }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestOpenTableInCoprocessor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestOpenTableInCoprocessor.java
index 2caf376..5ee837a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestOpenTableInCoprocessor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestOpenTableInCoprocessor.java
@@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.testclassification.CoprocessorTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -160,23 +161,18 @@ public class TestOpenTableInCoprocessor {
     runCoprocessorConnectionToRemoteTable(CustomThreadPoolCoprocessor.class, completedWithPool);
   }
 
-  private void runCoprocessorConnectionToRemoteTable(Class clazz, boolean[] completeCheck)
+  private void runCoprocessorConnectionToRemoteTable(Class<?> clazz, boolean[] completeCheck)
       throws Throwable {
     // Check if given class implements RegionObserver.
-    assert(RegionObserver.class.isAssignableFrom(clazz));
-    TableDescriptorBuilder.ModifyableTableDescriptor primaryDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(primaryTable);
-
-    primaryDescriptor.setColumnFamily(
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family));
+    assert (RegionObserver.class.isAssignableFrom(clazz));
     // add our coprocessor
-    primaryDescriptor.setCoprocessor(clazz.getName());
+    TableDescriptor primaryDescriptor = TableDescriptorBuilder.newBuilder(primaryTable)
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(family)).setCoprocessor(clazz.getName())
+      .build();
 
-    TableDescriptorBuilder.ModifyableTableDescriptor otherDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(otherTable);
 
-    otherDescriptor.setColumnFamily(
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family));
+    TableDescriptor otherDescriptor = TableDescriptorBuilder.newBuilder(otherTable)
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(family)).build();
 
 
     Admin admin = UTIL.getAdmin();
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverForAddingMutationsFromCoprocessors.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverForAddingMutationsFromCoprocessors.java
index 9e2bd7d..15954f8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverForAddingMutationsFromCoprocessors.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverForAddingMutationsFromCoprocessors.java
@@ -39,6 +39,7 @@ import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -98,17 +99,14 @@ public class TestRegionObserverForAddingMutationsFromCoprocessors {
   }
 
   private void createTable(String coprocessor) throws IOException {
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(tableName)
-        .setColumnFamily(new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(dummy))
-        .setColumnFamily(new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(test))
-        .setCoprocessor(coprocessor);
+    TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName)
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(dummy))
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(test)).setCoprocessor(coprocessor).build();
     util.getAdmin().createTable(tableDescriptor);
   }
 
   /**
    * Test various multiput operations.
-   * @throws Exception
    */
   @Test
   public void testMulti() throws Exception {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverInterface.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverInterface.java
index d252d5e..b679c32 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverInterface.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverInterface.java
@@ -577,12 +577,9 @@ public class TestRegionObserverInterface {
       admin.deleteTable(compactTable);
     }
 
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(compactTable);
-
-    tableDescriptor.setColumnFamily(
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(A));
-    tableDescriptor.setCoprocessor(EvenOnlyCompactor.class.getName());
+    TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(compactTable)
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(A))
+      .setCoprocessor(EvenOnlyCompactor.class.getName()).build();
     admin.createTable(tableDescriptor);
 
     Table table = util.getConnection().getTable(compactTable);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java
index 713a3e5..ace8c2a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java
@@ -185,12 +185,13 @@ public class TestRegionObserverScannerOpenHook {
 
   HRegion initHRegion(byte[] tableName, String callingMethod, Configuration conf,
       byte[]... families) throws IOException {
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(TableName.valueOf(tableName));
+    TableDescriptorBuilder builder =
+      TableDescriptorBuilder.newBuilder(TableName.valueOf(tableName));
     for (byte[] family : families) {
-      tableDescriptor.setColumnFamily(
-        new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family));
+      builder.setColumnFamily(
+        ColumnFamilyDescriptorBuilder.of(family));
     }
+    TableDescriptor tableDescriptor = builder.build();
     ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     RegionInfo info = RegionInfoBuilder.newBuilder(tableDescriptor.getTableName()).build();
     Path path = new Path(DIR + callingMethod);
@@ -307,24 +308,16 @@ public class TestRegionObserverScannerOpenHook {
     UTIL.startMiniCluster();
     byte[] ROW = Bytes.toBytes("testRow");
     byte[] A = Bytes.toBytes("A");
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(
-        TableName.valueOf(name.getMethodName()));
-
-    tableDescriptor.setColumnFamily(
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(A));
-    tableDescriptor.setCoprocessor(
-      CoprocessorDescriptorBuilder.newBuilder(EmptyRegionObsever.class.getName())
-        .setJarPath(null)
-        .setPriority(Coprocessor.PRIORITY_USER)
-        .setProperties(Collections.emptyMap())
-        .build());
-    tableDescriptor.setCoprocessor(
-      CoprocessorDescriptorBuilder.newBuilder(NoDataFromCompaction.class.getName())
-        .setJarPath(null)
-        .setPriority(Coprocessor.PRIORITY_HIGHEST)
-        .setProperties(Collections.emptyMap())
-        .build());
+    TableDescriptor tableDescriptor =
+      TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+        .setColumnFamily(ColumnFamilyDescriptorBuilder.of(A))
+        .setCoprocessor(CoprocessorDescriptorBuilder
+          .newBuilder(EmptyRegionObsever.class.getName()).setJarPath(null)
+          .setPriority(Coprocessor.PRIORITY_USER).setProperties(Collections.emptyMap()).build())
+        .setCoprocessor(CoprocessorDescriptorBuilder
+          .newBuilder(NoDataFromCompaction.class.getName()).setJarPath(null)
+          .setPriority(Coprocessor.PRIORITY_HIGHEST).setProperties(Collections.emptyMap()).build())
+        .build();
 
     Admin admin = UTIL.getAdmin();
     admin.createTable(tableDescriptor);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverStacking.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverStacking.java
index 69a1454..eff278a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverStacking.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverStacking.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.regionserver.ChunkCreator;
 import org.apache.hadoop.hbase.regionserver.HRegion;
@@ -119,16 +120,14 @@ public class TestRegionObserverStacking extends TestCase {
     }
   }
 
-  HRegion initHRegion (byte [] tableName, String callingMethod,
-      Configuration conf, byte [] ... families) throws IOException {
-
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(TableName.valueOf(tableName));
-
+  HRegion initHRegion(byte[] tableName, String callingMethod, Configuration conf,
+    byte[]... families) throws IOException {
+    TableDescriptorBuilder builder =
+      TableDescriptorBuilder.newBuilder(TableName.valueOf(tableName));
     for (byte[] family : families) {
-      tableDescriptor.setColumnFamily(
-        new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family));
+      builder.setColumnFamily(ColumnFamilyDescriptorBuilder.of(family));
     }
+    TableDescriptor tableDescriptor = builder.build();
     ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     RegionInfo info = RegionInfoBuilder.newBuilder(tableDescriptor.getTableName()).build();
     Path path = new Path(DIR + callingMethod);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/FilterTestingCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/FilterTestingCluster.java
index 50dd3e3..88ecb19 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/FilterTestingCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/FilterTestingCluster.java
@@ -26,7 +26,6 @@ import static org.junit.Assert.assertTrue;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
@@ -37,6 +36,7 @@ import org.apache.hadoop.hbase.ZooKeeperConnectionException;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.testclassification.FilterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -57,12 +57,8 @@ public class FilterTestingCluster {
 
   protected static void createTable(TableName tableName, String columnFamilyName) {
     assertNotNull("HBaseAdmin is not initialized successfully.", admin);
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
-
-    tableDescriptor.setColumnFamily(
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(
-        Bytes.toBytes(columnFamilyName)));
+    TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName)
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(Bytes.toBytes(columnFamilyName))).build();
 
     try {
       admin.createTable(tableDescriptor);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestDependentColumnFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestDependentColumnFilter.java
index b249005..a13aed2 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestDependentColumnFilter.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestDependentColumnFilter.java
@@ -37,6 +37,7 @@ import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionInfoBuilder;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.filter.Filter.ReturnCode;
 import org.apache.hadoop.hbase.regionserver.HRegion;
@@ -84,18 +85,13 @@ public class TestDependentColumnFilter {
   public void setUp() throws Exception {
     testVals = makeTestVals();
 
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(
-        TableName.valueOf(this.getClass().getSimpleName()));
-
-    ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor hcd0 =
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILIES[0])
-        .setMaxVersions(3);
-    tableDescriptor.setColumnFamily(hcd0);
-    ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor hcd1 =
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILIES[1])
-        .setMaxVersions(3);
-    tableDescriptor.setColumnFamily(hcd1);
+    TableDescriptor tableDescriptor =
+      TableDescriptorBuilder.newBuilder(TableName.valueOf(this.getClass().getSimpleName()))
+        .setColumnFamily(
+          ColumnFamilyDescriptorBuilder.newBuilder(FAMILIES[0]).setMaxVersions(3).build())
+        .setColumnFamily(
+          ColumnFamilyDescriptorBuilder.newBuilder(FAMILIES[1]).setMaxVersions(3).build())
+        .build();
     RegionInfo info = RegionInfoBuilder.newBuilder(tableDescriptor.getTableName()).build();
     this.region = HBaseTestingUtility.createRegionAndWAL(info, TEST_UTIL.getDataTestDir(),
         TEST_UTIL.getConfiguration(), tableDescriptor);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java
index eb88529..816524c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java
@@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionInfoBuilder;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.filter.FilterList.Operator;
 import org.apache.hadoop.hbase.regionserver.HRegion;
@@ -150,23 +151,15 @@ public class TestFilter {
 
   @Before
   public void setUp() throws Exception {
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(TableName.valueOf("TestFilter"));
-
-    ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor family0 =
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILIES[0])
-        .setVersions(100, 100);
-    tableDescriptor.setColumnFamily(family0);
-    tableDescriptor.setColumnFamily(
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILIES[1]));
-    tableDescriptor.setColumnFamily(
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILIES_1[0]));
-    tableDescriptor.setColumnFamily(
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILIES_1[1]));
-    tableDescriptor.setColumnFamily(
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(NEW_FAMILIES[0]));
-    tableDescriptor.setColumnFamily(
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(NEW_FAMILIES[1]));
+    TableDescriptor tableDescriptor = TableDescriptorBuilder
+      .newBuilder(TableName.valueOf("TestFilter"))
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(FAMILIES[0]).setMinVersions(100)
+        .setMaxVersions(100).build())
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILIES[1]))
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILIES_1[0]))
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILIES_1[1]))
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(NEW_FAMILIES[0]))
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(NEW_FAMILIES[1])).build();
     RegionInfo info = RegionInfoBuilder.newBuilder(tableDescriptor.getTableName()).build();
     this.region = HBaseTestingUtility.createRegionAndWAL(info, TEST_UTIL.getDataTestDir(),
         TEST_UTIL.getConfiguration(), tableDescriptor);
@@ -1519,12 +1512,9 @@ public class TestFilter {
   public void testFilterListWithPrefixFilter() throws IOException {
     byte[] family = Bytes.toBytes("f1");
     byte[] qualifier = Bytes.toBytes("q1");
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(
-        TableName.valueOf(name.getMethodName()));
-
-    tableDescriptor.setColumnFamily(
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family));
+    TableDescriptor tableDescriptor =
+      TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+        .setColumnFamily(ColumnFamilyDescriptorBuilder.of(family)).build();
     RegionInfo info = RegionInfoBuilder.newBuilder(tableDescriptor.getTableName()).build();
     HRegion testRegion = HBaseTestingUtility.createRegionAndWAL(info, TEST_UTIL.getDataTestDir(),
       TEST_UTIL.getConfiguration(), tableDescriptor);
@@ -2293,12 +2283,9 @@ public class TestFilter {
   @Ignore("TODO: intentionally disabled?")
   public void testNestedFilterListWithSCVF() throws IOException {
     byte[] columnStatus = Bytes.toBytes("S");
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(
-        TableName.valueOf(name.getMethodName()));
-
-    tableDescriptor.setColumnFamily(
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILIES[0]));
+    TableDescriptor tableDescriptor =
+      TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+        .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILIES[0])).build();
     RegionInfo info = RegionInfoBuilder.newBuilder(tableDescriptor.getTableName()).build();
     HRegion testRegion = HBaseTestingUtility.createRegionAndWAL(info, TEST_UTIL.getDataTestDir(),
         TEST_UTIL.getConfiguration(), tableDescriptor);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterFromRegionSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterFromRegionSide.java
index 7be3bad..ac06289 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterFromRegionSide.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterFromRegionSide.java
@@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionInfoBuilder;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.InternalScanner;
@@ -83,18 +84,16 @@ public class TestFilterFromRegionSide {
 
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(TABLE_NAME);
+    TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(TABLE_NAME);
 
     for (byte[] family : FAMILIES) {
-      ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
-        new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family);
-      tableDescriptor.setColumnFamily(familyDescriptor);
+      builder.setColumnFamily(ColumnFamilyDescriptorBuilder.of(family));
     }
+    TableDescriptor tableDescriptor = builder.build();
     RegionInfo info = RegionInfoBuilder.newBuilder(tableDescriptor.getTableName()).build();
     REGION = HBaseTestingUtility.createRegionAndWAL(info, TEST_UTIL.getDataTestDir(),
       TEST_UTIL.getConfiguration(), tableDescriptor);
-    for(Put put:createPuts(ROWS, FAMILIES, QUALIFIERS, VALUE)){
+    for (Put put : createPuts(ROWS, FAMILIES, QUALIFIERS, VALUE)) {
       REGION.put(put);
     }
   }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterWrapper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterWrapper.java
index a00b1b1..bd9e83c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterWrapper.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterWrapper.java
@@ -17,7 +17,10 @@
  */
 package org.apache.hadoop.hbase.filter;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -42,6 +45,7 @@ import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.testclassification.FilterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -149,13 +153,8 @@ public class TestFilterWrapper {
   private static void createTable() {
     assertNotNull("HBaseAdmin is not initialized successfully.", admin);
     if (admin != null) {
-
-      TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-        new TableDescriptorBuilder.ModifyableTableDescriptor(name);
-
-      ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
-        new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(Bytes.toBytes("f1"));
-      tableDescriptor.setColumnFamily(familyDescriptor);
+      TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(name)
+        .setColumnFamily(ColumnFamilyDescriptorBuilder.of(Bytes.toBytes("f1"))).build();
 
       try {
         admin.createTable(tableDescriptor);
@@ -163,7 +162,6 @@ public class TestFilterWrapper {
       } catch (IOException e) {
         assertNull("Exception found while creating table", e);
       }
-
     }
   }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestInvocationRecordFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestInvocationRecordFilter.java
index b752d12..1a4296c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestInvocationRecordFilter.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestInvocationRecordFilter.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionInfoBuilder;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.InternalScanner;
@@ -68,14 +69,11 @@ public class TestInvocationRecordFilter {
 
   @Before
   public void setUp() throws Exception {
-    TableDescriptorBuilder.ModifyableTableDescriptor htd =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(TableName.valueOf(TABLE_NAME_BYTES));
-
-    htd.setColumnFamily(
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY_NAME_BYTES));
+    TableDescriptor htd = TableDescriptorBuilder.newBuilder(TableName.valueOf(TABLE_NAME_BYTES))
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY_NAME_BYTES)).build();
     RegionInfo info = RegionInfoBuilder.newBuilder(htd.getTableName()).build();
     this.region = HBaseTestingUtility.createRegionAndWAL(info, TEST_UTIL.getDataTestDir(),
-        TEST_UTIL.getConfiguration(), htd);
+      TEST_UTIL.getConfiguration(), htd);
 
     Put put = new Put(ROW_BYTES);
     for (int i = 0; i < 10; i += 2) {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestSeekBeforeWithReverseScan.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestSeekBeforeWithReverseScan.java
index 29f1fdd..0e7218f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestSeekBeforeWithReverseScan.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestSeekBeforeWithReverseScan.java
@@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionInfoBuilder;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter;
 import org.apache.hadoop.hbase.regionserver.HRegion;
@@ -63,12 +64,9 @@ public class TestSeekBeforeWithReverseScan {
   @Before
   public void setUp() throws Exception {
     TableName tableName = TableName.valueOf(getClass().getSimpleName());
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
-
-    tableDescriptor.setColumnFamily(
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(cfName)
-        .setDataBlockEncoding(DataBlockEncoding.FAST_DIFF));
+    TableDescriptor tableDescriptor =
+      TableDescriptorBuilder.newBuilder(tableName).setColumnFamily(ColumnFamilyDescriptorBuilder
+        .newBuilder(cfName).setDataBlockEncoding(DataBlockEncoding.FAST_DIFF).build()).build();
     RegionInfo info = RegionInfoBuilder.newBuilder(tableName).build();
     Path path = testUtil.getDataTestDir(getClass().getSimpleName());
     region = HBaseTestingUtility.createRegionAndWAL(info, path, testUtil.getConfiguration(),
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerSelectionUsingKeyRange.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerSelectionUsingKeyRange.java
index f2b7801..e9f6192 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerSelectionUsingKeyRange.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerSelectionUsingKeyRange.java
@@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionInfoBuilder;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.regionserver.BloomType;
 import org.apache.hadoop.hbase.regionserver.HRegion;
@@ -102,14 +103,11 @@ public class TestScannerSelectionUsingKeyRange {
   public void testScannerSelection() throws IOException {
     Configuration conf = TEST_UTIL.getConfiguration();
     conf.setInt("hbase.hstore.compactionThreshold", 10000);
-    ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY_BYTES)
-        .setBlockCacheEnabled(true)
-        .setBloomFilterType(bloomType);
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(TABLE);
+    TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(TABLE)
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(FAMILY_BYTES)
+        .setBlockCacheEnabled(true).setBloomFilterType(bloomType).build())
+      .build();
 
-    tableDescriptor.setColumnFamily(familyDescriptor);
     RegionInfo info = RegionInfoBuilder.newBuilder(TABLE).build();
     HRegion region = HBaseTestingUtility.createRegionAndWAL(info, TEST_UTIL.getDataTestDir(), conf,
       tableDescriptor);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java
index e653234..53bebec 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java
@@ -48,6 +48,7 @@ import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionInfoBuilder;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Table;
+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.testclassification.MasterTests;
@@ -241,11 +242,8 @@ public class TestMaster {
     int msgInterval = conf.getInt("hbase.regionserver.msginterval", 100);
     // insert some data into META
     TableName tableName = TableName.valueOf("testFlushSeqId");
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
-
-    tableDescriptor.setColumnFamily(
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(Bytes.toBytes("cf")));
+    TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName)
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(Bytes.toBytes("cf"))).build();
     Table table = TEST_UTIL.createTable(tableDescriptor, null);
     // flush META region
     TEST_UTIL.flush(TableName.META_TABLE_NAME);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetricsWrapper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetricsWrapper.java
index 1241e26..ff88be1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetricsWrapper.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetricsWrapper.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.master.assignment.RegionStates;
 import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot;
@@ -125,12 +126,9 @@ public class TestMasterMetricsWrapper {
     TableName table = TableName.valueOf("testRegionNumber");
     try {
       RegionInfo hri;
-      TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-        new TableDescriptorBuilder.ModifyableTableDescriptor(table);
-
       byte[] FAMILY = Bytes.toBytes("FAMILY");
-      tableDescriptor.setColumnFamily(
-        new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY));
+      TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(table)
+        .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY)).build();
       TEST_UTIL.getAdmin().createTable(tableDescriptor, Bytes.toBytes("A"),
         Bytes.toBytes("Z"), 5);
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement.java
index e9e4a39..3983040 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlacement.java
@@ -551,12 +551,8 @@ public class TestRegionPlacement {
       splitKeys[i - 1] = new byte[] { splitKey, splitKey, splitKey };
     }
 
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
-
-    tableDescriptor.setColumnFamily(
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(
-        HConstants.CATALOG_FAMILY));
+    TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName)
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(HConstants.CATALOG_FAMILY)).build();
     admin.createTable(tableDescriptor, splitKeys);
 
     try (RegionLocator r = CONNECTION.getRegionLocator(tableName)) {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlansWithThrottle.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlansWithThrottle.java
index 7108340..5084064 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlansWithThrottle.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionPlansWithThrottle.java
@@ -28,6 +28,7 @@ import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
@@ -65,9 +66,8 @@ public class TestRegionPlansWithThrottle {
   public void testExecuteRegionPlansWithThrottling() throws Exception {
     final TableName tableName = TableName.valueOf("testExecuteRegionPlansWithThrottling");
 
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(tableName).setColumnFamily(
-        new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(Bytes.toBytes("cf")));
+    TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName)
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(Bytes.toBytes("cf"))).build();
 
     UTIL.getAdmin().createTable(tableDescriptor);
     Table table = UTIL.getConnection().getTable(tableName);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredNodeTableImport.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredNodeTableImport.java
index 6a4fc8c..40345e8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredNodeTableImport.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredNodeTableImport.java
@@ -21,6 +21,7 @@ import static org.apache.hadoop.hbase.favored.FavoredNodeAssignmentHelper.FAVORE
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
+
 import java.util.List;
 import java.util.Set;
 import org.apache.hadoop.conf.Configuration;
@@ -32,6 +33,7 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.favored.FavoredNodesManager;
 import org.apache.hadoop.hbase.master.HMaster;
@@ -44,6 +46,7 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
 
 /*
@@ -71,7 +74,6 @@ public class TestFavoredNodeTableImport {
 
   @Test
   public void testTableCreation() throws Exception {
-
     conf.set(HConstants.HBASE_MASTER_LOADBALANCER_CLASS, StochasticLoadBalancer.class.getName());
 
     LOG.info("Starting up cluster");
@@ -83,12 +85,9 @@ public class TestFavoredNodeTableImport {
     admin.balancerSwitch(false, true);
 
     String tableName = "testFNImport";
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(TableName.valueOf(tableName));
-
-    tableDescriptor.setColumnFamily(
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(
-        HConstants.CATALOG_FAMILY));
+    TableDescriptor tableDescriptor =
+      TableDescriptorBuilder.newBuilder(TableName.valueOf(tableName))
+        .setColumnFamily(ColumnFamilyDescriptorBuilder.of(HConstants.CATALOG_FAMILY)).build();
     admin.createTable(tableDescriptor, Bytes.toBytes("a"), Bytes.toBytes("z"), REGION_NUM);
     UTIL.waitTableAvailable(tableDescriptor.getTableName());
     admin.balancerSwitch(true, true);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java
index 8a1ee8a..d3e1e83 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java
@@ -43,6 +43,7 @@ import org.apache.hadoop.hbase.Waiter;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.favored.FavoredNodeAssignmentHelper;
 import org.apache.hadoop.hbase.favored.FavoredNodesManager;
@@ -117,12 +118,8 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
   public void testBasicBalance() throws Exception {
 
     TableName tableName = TableName.valueOf("testBasicBalance");
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
-
-    tableDescriptor.setColumnFamily(
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(
-        HConstants.CATALOG_FAMILY));
+    TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName)
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(HConstants.CATALOG_FAMILY)).build();
     admin.createTable(tableDescriptor, Bytes.toBytes("aaa"), Bytes.toBytes("zzz"), REGION_NUM);
     TEST_UTIL.waitTableAvailable(tableName);
     TEST_UTIL.loadTable(admin.getConnection().getTable(tableName), HConstants.CATALOG_FAMILY);
@@ -154,12 +151,8 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
   public void testRoundRobinAssignment() throws Exception {
 
     TableName tableName = TableName.valueOf("testRoundRobinAssignment");
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
-
-    tableDescriptor.setColumnFamily(
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(
-        HConstants.CATALOG_FAMILY));
+    TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName)
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(HConstants.CATALOG_FAMILY)).build();
     admin.createTable(tableDescriptor, Bytes.toBytes("aaa"), Bytes.toBytes("zzz"), REGION_NUM);
     TEST_UTIL.waitTableAvailable(tableName);
     TEST_UTIL.loadTable(admin.getConnection().getTable(tableName), HConstants.CATALOG_FAMILY);
@@ -180,14 +173,10 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
 
   @Test
   public void testBasicRegionPlacementAndReplicaLoad() throws Exception {
-
     String tableName = "testBasicRegionPlacement";
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(TableName.valueOf(tableName));
-
-    tableDescriptor.setColumnFamily(
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(
-        HConstants.CATALOG_FAMILY));
+    TableDescriptor tableDescriptor =
+      TableDescriptorBuilder.newBuilder(TableName.valueOf(tableName))
+        .setColumnFamily(ColumnFamilyDescriptorBuilder.of(HConstants.CATALOG_FAMILY)).build();
     admin.createTable(tableDescriptor, Bytes.toBytes("aaa"), Bytes.toBytes("zzz"), REGION_NUM);
     TEST_UTIL.waitTableAvailable(tableDescriptor.getTableName());
 
@@ -226,12 +215,9 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
   public void testRandomAssignmentWithNoFavNodes() throws Exception {
 
     final String tableName = "testRandomAssignmentWithNoFavNodes";
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(TableName.valueOf(tableName));
-
-    tableDescriptor.setColumnFamily(
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(
-        HConstants.CATALOG_FAMILY));
+    TableDescriptor tableDescriptor =
+      TableDescriptorBuilder.newBuilder(TableName.valueOf(tableName))
+        .setColumnFamily(ColumnFamilyDescriptorBuilder.of(HConstants.CATALOG_FAMILY)).build();
     admin.createTable(tableDescriptor);
     TEST_UTIL.waitTableAvailable(tableDescriptor.getTableName());
 
@@ -261,12 +247,8 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
   public void testBalancerWithoutFavoredNodes() throws Exception {
 
     TableName tableName = TableName.valueOf("testBalancerWithoutFavoredNodes");
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
-
-    tableDescriptor.setColumnFamily(
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(
-        HConstants.CATALOG_FAMILY));
+    TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName)
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(HConstants.CATALOG_FAMILY)).build();
     admin.createTable(tableDescriptor, Bytes.toBytes("aaa"), Bytes.toBytes("zzz"), REGION_NUM);
     TEST_UTIL.waitTableAvailable(tableName);
 
@@ -301,14 +283,9 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
 
   @Ignore @Test
   public void testMisplacedRegions() throws Exception {
-
     TableName tableName = TableName.valueOf("testMisplacedRegions");
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
-
-    tableDescriptor.setColumnFamily(
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(
-        HConstants.CATALOG_FAMILY));
+    TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName)
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(HConstants.CATALOG_FAMILY)).build();
     admin.createTable(tableDescriptor, Bytes.toBytes("aaa"), Bytes.toBytes("zzz"), REGION_NUM);
     TEST_UTIL.waitTableAvailable(tableName);
 
@@ -352,14 +329,9 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
 
   @Test
   public void test2FavoredNodesDead() throws Exception {
-
     TableName tableName = TableName.valueOf("testAllFavoredNodesDead");
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
-
-    tableDescriptor.setColumnFamily(
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(
-        HConstants.CATALOG_FAMILY));
+    TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName)
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(HConstants.CATALOG_FAMILY)).build();
     admin.createTable(tableDescriptor, Bytes.toBytes("aaa"), Bytes.toBytes("zzz"), REGION_NUM);
     TEST_UTIL.waitTableAvailable(tableName);
 
@@ -394,14 +366,9 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
 
   @Ignore @Test
   public void testAllFavoredNodesDead() throws Exception {
-
     TableName tableName = TableName.valueOf("testAllFavoredNodesDead");
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
-
-    tableDescriptor.setColumnFamily(
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(
-        HConstants.CATALOG_FAMILY));
+    TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName)
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(HConstants.CATALOG_FAMILY)).build();
     admin.createTable(tableDescriptor, Bytes.toBytes("aaa"), Bytes.toBytes("zzz"), REGION_NUM);
     TEST_UTIL.waitTableAvailable(tableName);
 
@@ -460,14 +427,9 @@ public class TestFavoredStochasticLoadBalancer extends BalancerTestBase {
 
   @Ignore @Test
   public void testAllFavoredNodesDeadMasterRestarted() throws Exception {
-
     TableName tableName = TableName.valueOf("testAllFavoredNodesDeadMasterRestarted");
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
-
-    tableDescriptor.setColumnFamily(
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(
-        HConstants.CATALOG_FAMILY));
+    TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName)
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(HConstants.CATALOG_FAMILY)).build();
     admin.createTable(tableDescriptor, Bytes.toBytes("aaa"), Bytes.toBytes("zzz"), REGION_NUM);
     TEST_UTIL.waitTableAvailable(tableName);
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteColumnFamilyProcedureFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteColumnFamilyProcedureFromClient.java
index c7ed3b2..670d7a3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteColumnFamilyProcedureFromClient.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestDeleteColumnFamilyProcedureFromClient.java
@@ -28,13 +28,13 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.InvalidFamilyOperationException;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -101,7 +101,7 @@ public class TestDeleteColumnFamilyProcedureFromClient {
   @Test
   public void deleteColumnFamilyWithMultipleRegions() throws Exception {
     Admin admin = TEST_UTIL.getAdmin();
-    HTableDescriptor beforehtd = new HTableDescriptor(admin.getDescriptor(TABLENAME));
+    TableDescriptor beforehtd = admin.getDescriptor(TABLENAME);
 
     FileSystem fs = TEST_UTIL.getDFSCluster().getFileSystem();
 
@@ -110,7 +110,7 @@ public class TestDeleteColumnFamilyProcedureFromClient {
 
     // 2 - Check if all three families exist in descriptor
     assertEquals(3, beforehtd.getColumnFamilyCount());
-    HColumnDescriptor[] families = beforehtd.getColumnFamilies();
+    ColumnFamilyDescriptor[] families = beforehtd.getColumnFamilies();
     for (int i = 0; i < families.length; i++) {
       assertTrue(families[i].getNameAsString().equals("cf" + (i + 1)));
     }
@@ -148,9 +148,9 @@ public class TestDeleteColumnFamilyProcedureFromClient {
     admin.deleteColumnFamily(TABLENAME, Bytes.toBytes("cf2"));
 
     // 5 - Check if only 2 column families exist in the descriptor
-    HTableDescriptor afterhtd = new HTableDescriptor(admin.getDescriptor(TABLENAME));
+    TableDescriptor afterhtd = admin.getDescriptor(TABLENAME);
     assertEquals(2, afterhtd.getColumnFamilyCount());
-    HColumnDescriptor[] newFamilies = afterhtd.getColumnFamilies();
+    ColumnFamilyDescriptor[] newFamilies = afterhtd.getColumnFamilies();
     assertTrue(newFamilies[0].getNameAsString().equals("cf1"));
     assertTrue(newFamilies[1].getNameAsString().equals("cf3"));
 
@@ -179,7 +179,7 @@ public class TestDeleteColumnFamilyProcedureFromClient {
   @Test
   public void deleteColumnFamilyTwice() throws Exception {
     Admin admin = TEST_UTIL.getAdmin();
-    HTableDescriptor beforehtd = new HTableDescriptor(admin.getDescriptor(TABLENAME));
+    TableDescriptor beforehtd = admin.getDescriptor(TABLENAME);
     String cfToDelete = "cf1";
 
     FileSystem fs = TEST_UTIL.getDFSCluster().getFileSystem();
@@ -188,7 +188,7 @@ public class TestDeleteColumnFamilyProcedureFromClient {
     assertTrue(admin.isTableAvailable(TABLENAME));
 
     // 2 - Check if all the target column family exist in descriptor
-    HColumnDescriptor[] families = beforehtd.getColumnFamilies();
+    ColumnFamilyDescriptor[] families = beforehtd.getColumnFamilies();
     Boolean foundCF = false;
     for (int i = 0; i < families.length; i++) {
       if (families[i].getNameAsString().equals(cfToDelete)) {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyTableProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyTableProcedure.java
index fefba23..4461f2f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyTableProcedure.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestModifyTableProcedure.java
@@ -20,11 +20,11 @@ package org.apache.hadoop.hbase.master.procedure;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
+
 import java.io.IOException;
 import org.apache.hadoop.hbase.ConcurrentTableModificationException;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.InvalidFamilyOperationException;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
@@ -71,31 +71,31 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
     UTIL.getAdmin().disableTable(tableName);
 
     // Modify the table descriptor
-    HTableDescriptor htd = new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
+    TableDescriptor htd = UTIL.getAdmin().getDescriptor(tableName);
 
     // Test 1: Modify 1 property
     long newMaxFileSize = htd.getMaxFileSize() * 2;
-    htd.setMaxFileSize(newMaxFileSize);
-    htd.setRegionReplication(3);
+    htd = TableDescriptorBuilder.newBuilder(htd).setMaxFileSize(newMaxFileSize)
+      .setRegionReplication(3).build();
 
-    long procId1 = ProcedureTestingUtility.submitAndWait(
-        procExec, new ModifyTableProcedure(procExec.getEnvironment(), htd));
+    long procId1 = ProcedureTestingUtility.submitAndWait(procExec,
+      new ModifyTableProcedure(procExec.getEnvironment(), htd));
     ProcedureTestingUtility.assertProcNotFailed(procExec.getResult(procId1));
 
-    HTableDescriptor currentHtd = new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
+    TableDescriptor currentHtd = UTIL.getAdmin().getDescriptor(tableName);
     assertEquals(newMaxFileSize, currentHtd.getMaxFileSize());
 
     // Test 2: Modify multiple properties
     boolean newReadOnlyOption = htd.isReadOnly() ? false : true;
     long newMemStoreFlushSize = htd.getMemStoreFlushSize() * 2;
-    htd.setReadOnly(newReadOnlyOption);
-    htd.setMemStoreFlushSize(newMemStoreFlushSize);
+    htd = TableDescriptorBuilder.newBuilder(htd).setReadOnly(newReadOnlyOption)
+      .setMemStoreFlushSize(newMemStoreFlushSize).build();
 
     long procId2 = ProcedureTestingUtility.submitAndWait(
         procExec, new ModifyTableProcedure(procExec.getEnvironment(), htd));
     ProcedureTestingUtility.assertProcNotFailed(procExec.getResult(procId2));
 
-    currentHtd = new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
+    currentHtd = UTIL.getAdmin().getDescriptor(tableName);
     assertEquals(newReadOnlyOption, currentHtd.isReadOnly());
     assertEquals(newMemStoreFlushSize, currentHtd.getMemStoreFlushSize());
   }
@@ -106,8 +106,8 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
     final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
 
     MasterProcedureTestingUtility.createTable(procExec, tableName, null, "cf1");
-    HTableDescriptor currentHtd = new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
-    assertEquals(1, currentHtd.getFamiliesKeys().size());
+    TableDescriptor currentHtd = UTIL.getAdmin().getDescriptor(tableName);
+    assertEquals(1, currentHtd.getColumnFamilyNames().size());
 
     // Test 1: Modify the table descriptor online
     String cf2 = "cf2";
@@ -122,9 +122,9 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
         procExec.getEnvironment(), tableDescriptorBuilder.build()));
     ProcedureTestingUtility.assertProcNotFailed(procExec.getResult(procId));
 
-    currentHtd = new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
-    assertEquals(2, currentHtd.getFamiliesKeys().size());
-    assertTrue(currentHtd.hasFamily(Bytes.toBytes(cf2)));
+    currentHtd = UTIL.getAdmin().getDescriptor(tableName);
+    assertEquals(2, currentHtd.getColumnFamilyNames().size());
+    assertTrue(currentHtd.hasColumnFamily(Bytes.toBytes(cf2)));
 
     // Test 2: Modify the table descriptor offline
     UTIL.getAdmin().disableTable(tableName);
@@ -141,9 +141,9 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
         new ModifyTableProcedure(procExec.getEnvironment(), tableDescriptorBuilder.build()));
     ProcedureTestingUtility.assertProcNotFailed(procExec.getResult(procId2));
 
-    currentHtd = new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
-    assertTrue(currentHtd.hasFamily(Bytes.toBytes(cf3)));
-    assertEquals(3, currentHtd.getFamiliesKeys().size());
+    currentHtd = UTIL.getAdmin().getDescriptor(tableName);
+    assertTrue(currentHtd.hasColumnFamily(Bytes.toBytes(cf3)));
+    assertEquals(3, currentHtd.getColumnFamilyNames().size());
   }
 
   @Test
@@ -155,44 +155,43 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
     final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
 
     MasterProcedureTestingUtility.createTable(procExec, tableName, null, cf1, cf2, cf3);
-    HTableDescriptor currentHtd = new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
-    assertEquals(3, currentHtd.getFamiliesKeys().size());
+    TableDescriptor currentHtd = UTIL.getAdmin().getDescriptor(tableName);
+    assertEquals(3, currentHtd.getColumnFamilyNames().size());
 
     // Test 1: Modify the table descriptor
-    HTableDescriptor htd = new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
-    htd.removeFamily(Bytes.toBytes(cf2));
+    TableDescriptor htd = UTIL.getAdmin().getDescriptor(tableName);
+    htd = TableDescriptorBuilder.newBuilder(htd).removeColumnFamily(Bytes.toBytes(cf2)).build();
 
     long procId = ProcedureTestingUtility.submitAndWait(
         procExec, new ModifyTableProcedure(procExec.getEnvironment(), htd));
     ProcedureTestingUtility.assertProcNotFailed(procExec.getResult(procId));
 
-    currentHtd = new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
-    assertEquals(2, currentHtd.getFamiliesKeys().size());
-    assertFalse(currentHtd.hasFamily(Bytes.toBytes(cf2)));
+    currentHtd = UTIL.getAdmin().getDescriptor(tableName);
+    assertEquals(2, currentHtd.getColumnFamilyNames().size());
+    assertFalse(currentHtd.hasColumnFamily(Bytes.toBytes(cf2)));
 
     // Test 2: Modify the table descriptor offline
     UTIL.getAdmin().disableTable(tableName);
     ProcedureTestingUtility.waitNoProcedureRunning(procExec);
 
-    HTableDescriptor htd2 =
-        new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
-    htd2.removeFamily(Bytes.toBytes(cf3));
+    TableDescriptor htd2 = UTIL.getAdmin().getDescriptor(tableName);
     // Disable Sanity check
-    htd2.setConfiguration(TableDescriptorChecker.TABLE_SANITY_CHECKS, Boolean.FALSE.toString());
+    htd2 = TableDescriptorBuilder.newBuilder(htd2).removeColumnFamily(Bytes.toBytes(cf3))
+      .setValue(TableDescriptorChecker.TABLE_SANITY_CHECKS, Boolean.FALSE.toString())
+      .build();
 
     long procId2 =
         ProcedureTestingUtility.submitAndWait(procExec,
           new ModifyTableProcedure(procExec.getEnvironment(), htd2));
     ProcedureTestingUtility.assertProcNotFailed(procExec.getResult(procId2));
 
-    currentHtd = new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
-    assertEquals(1, currentHtd.getFamiliesKeys().size());
-    assertFalse(currentHtd.hasFamily(Bytes.toBytes(cf3)));
+    currentHtd = UTIL.getAdmin().getDescriptor(tableName);
+    assertEquals(1, currentHtd.getColumnFamilyNames().size());
+    assertFalse(currentHtd.hasColumnFamily(Bytes.toBytes(cf3)));
 
-    //Removing the last family will fail
-    HTableDescriptor htd3 =
-        new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
-    htd3.removeFamily(Bytes.toBytes(cf1));
+    // Removing the last family will fail
+    TableDescriptor htd3 = UTIL.getAdmin().getDescriptor(tableName);
+    htd3 = TableDescriptorBuilder.newBuilder(htd3).removeColumnFamily(Bytes.toBytes(cf1)).build();
     long procId3 =
         ProcedureTestingUtility.submitAndWait(procExec,
             new ModifyTableProcedure(procExec.getEnvironment(), htd3));
@@ -201,8 +200,8 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
     Throwable cause = ProcedureTestingUtility.getExceptionCause(result);
     assertTrue("expected DoNotRetryIOException, got " + cause,
         cause instanceof DoNotRetryIOException);
-    assertEquals(1, currentHtd.getFamiliesKeys().size());
-    assertTrue(currentHtd.hasFamily(Bytes.toBytes(cf1)));
+    assertEquals(1, currentHtd.getColumnFamilyNames().size());
+    assertTrue(currentHtd.hasColumnFamily(Bytes.toBytes(cf1)));
   }
 
   @Test
@@ -277,11 +276,11 @@ public class TestModifyTableProcedure extends TestTableDDLProcedureBase {
     MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId);
 
     // Validate descriptor
-    HTableDescriptor currentHtd = new HTableDescriptor(UTIL.getAdmin().getDescriptor(tableName));
+    TableDescriptor currentHtd = UTIL.getAdmin().getDescriptor(tableName);
     assertEquals(newCompactionEnableOption, currentHtd.isCompactionEnabled());
-    assertEquals(2, currentHtd.getFamiliesKeys().size());
-    assertTrue(currentHtd.hasFamily(Bytes.toBytes(cf2)));
-    assertFalse(currentHtd.hasFamily(Bytes.toBytes(cf3)));
+    assertEquals(2, currentHtd.getColumnFamilyNames().size());
+    assertTrue(currentHtd.hasColumnFamily(Bytes.toBytes(cf2)));
+    assertFalse(currentHtd.hasColumnFamily(Bytes.toBytes(cf3)));
 
     // cf2 should be added cf3 should be removed
     MasterProcedureTestingUtility.validateTableCreation(UTIL.getHBaseCluster().getMaster(),
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestRestoreSnapshotProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestRestoreSnapshotProcedure.java
index 64ecb42..afe6b96 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestRestoreSnapshotProcedure.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestRestoreSnapshotProcedure.java
@@ -21,14 +21,15 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotEquals;
 import static org.junit.Assert.assertTrue;
+
 import java.io.IOException;
 import java.util.List;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotDisabledException;
 import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.SnapshotDescription;
 import org.apache.hadoop.hbase.client.TableDescriptor;
@@ -49,6 +50,7 @@ import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos;
 
@@ -118,10 +120,10 @@ public class TestRestoreSnapshotProcedure extends TestTableDDLProcedureBase {
     snapshot = ProtobufUtil.createHBaseProtosSnapshotDesc(snapshotList.get(0));
 
     // modify the table
-    ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor columnFamilyDescriptor3 =
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(CF3);
-    ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor columnFamilyDescriptor4 =
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(CF4);
+    ColumnFamilyDescriptor columnFamilyDescriptor3 =
+      ColumnFamilyDescriptorBuilder.of(CF3);
+    ColumnFamilyDescriptor columnFamilyDescriptor4 =
+      ColumnFamilyDescriptorBuilder.of(CF4);
     admin.addColumnFamily(snapshotTableName, columnFamilyDescriptor3);
     admin.addColumnFamily(snapshotTableName, columnFamilyDescriptor4);
     admin.deleteColumnFamily(snapshotTableName, CF2);
@@ -130,27 +132,26 @@ public class TestRestoreSnapshotProcedure extends TestTableDDLProcedureBase {
     SnapshotTestingUtils.loadData(UTIL, snapshotTableName, rowCountCF3, CF3);
     SnapshotTestingUtils.loadData(UTIL, snapshotTableName, rowCountCF4, CF4);
     SnapshotTestingUtils.loadData(UTIL, snapshotTableName, rowCountCF1addition, CF1);
-    HTableDescriptor currentHTD = new HTableDescriptor(admin.getDescriptor(snapshotTableName));
-    assertTrue(currentHTD.hasFamily(CF1));
-    assertFalse(currentHTD.hasFamily(CF2));
-    assertTrue(currentHTD.hasFamily(CF3));
-    assertTrue(currentHTD.hasFamily(CF4));
-    assertNotEquals(currentHTD.getFamiliesKeys().size(), snapshotHTD.getColumnFamilies().length);
-    SnapshotTestingUtils.verifyRowCount(
-      UTIL, snapshotTableName, rowCountCF1 + rowCountCF3 + rowCountCF4 + rowCountCF1addition);
+    TableDescriptor currentHTD = admin.getDescriptor(snapshotTableName);
+    assertTrue(currentHTD.hasColumnFamily(CF1));
+    assertFalse(currentHTD.hasColumnFamily(CF2));
+    assertTrue(currentHTD.hasColumnFamily(CF3));
+    assertTrue(currentHTD.hasColumnFamily(CF4));
+    assertNotEquals(currentHTD.getColumnFamilyNames().size(),
+      snapshotHTD.getColumnFamilies().length);
+    SnapshotTestingUtils.verifyRowCount(UTIL, snapshotTableName,
+      rowCountCF1 + rowCountCF3 + rowCountCF4 + rowCountCF1addition);
     admin.disableTable(snapshotTableName);
   }
 
-  private static TableDescriptor createHTableDescriptor(
-      final TableName tableName, final byte[] ... family) {
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
+  private static TableDescriptor createHTableDescriptor(final TableName tableName,
+    final byte[]... family) {
+    TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableName);
 
     for (int i = 0; i < family.length; ++i) {
-      tableDescriptor.setColumnFamily(
-        new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family[i]));
+      builder.setColumnFamily(ColumnFamilyDescriptorBuilder.of(family[i]));
     }
-    return tableDescriptor;
+    return builder.build();
   }
 
   @Test
@@ -222,13 +223,13 @@ public class TestRestoreSnapshotProcedure extends TestTableDDLProcedureBase {
     try {
       UTIL.getAdmin().enableTable(snapshotTableName);
 
-      HTableDescriptor currentHTD =
-        new HTableDescriptor(UTIL.getAdmin().getDescriptor(snapshotTableName));
-      assertTrue(currentHTD.hasFamily(CF1));
-      assertTrue(currentHTD.hasFamily(CF2));
-      assertFalse(currentHTD.hasFamily(CF3));
-      assertFalse(currentHTD.hasFamily(CF4));
-      assertEquals(currentHTD.getFamiliesKeys().size(), snapshotHTD.getColumnFamilies().length);
+      TableDescriptor currentHTD = UTIL.getAdmin().getDescriptor(snapshotTableName);
+      assertTrue(currentHTD.hasColumnFamily(CF1));
+      assertTrue(currentHTD.hasColumnFamily(CF2));
+      assertFalse(currentHTD.hasColumnFamily(CF3));
+      assertFalse(currentHTD.hasColumnFamily(CF4));
+      assertEquals(currentHTD.getColumnFamilyNames().size(),
+        snapshotHTD.getColumnFamilies().length);
       SnapshotTestingUtils.verifyRowCount(UTIL, snapshotTableName, rowCountCF1 + rowCountCF2);
     } finally {
       UTIL.getAdmin().disableTable(snapshotTableName);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTableDescriptorModificationFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTableDescriptorModificationFromClient.java
index cdd1aa3..9196921 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTableDescriptorModificationFromClient.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTableDescriptorModificationFromClient.java
@@ -25,11 +25,10 @@ import java.util.Set;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.InvalidFamilyOperationException;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
+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;
@@ -91,11 +90,8 @@ public class TestTableDescriptorModificationFromClient {
   public void testModifyTable() throws IOException {
     Admin admin = TEST_UTIL.getAdmin();
     // Create a table with one family
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(TABLE_NAME);
-
-    tableDescriptor.setColumnFamily(
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY_0));
+    TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(TABLE_NAME)
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY_0)).build();
     admin.createTable(tableDescriptor);
     admin.disableTable(TABLE_NAME);
     try {
@@ -103,13 +99,9 @@ public class TestTableDescriptorModificationFromClient {
       verifyTableDescriptor(TABLE_NAME, FAMILY_0);
 
       // Modify the table adding another family and verify the descriptor
-      TableDescriptorBuilder.ModifyableTableDescriptor modifiedtableDescriptor =
-        new TableDescriptorBuilder.ModifyableTableDescriptor(TABLE_NAME);
-
-      modifiedtableDescriptor.setColumnFamily(
-        new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY_0));
-      modifiedtableDescriptor.setColumnFamily(
-        new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY_1));
+      TableDescriptor modifiedtableDescriptor = TableDescriptorBuilder.newBuilder(TABLE_NAME)
+        .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY_0))
+        .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY_1)).build();
       admin.modifyTable(modifiedtableDescriptor);
       verifyTableDescriptor(TABLE_NAME, FAMILY_0, FAMILY_1);
     } finally {
@@ -121,11 +113,8 @@ public class TestTableDescriptorModificationFromClient {
   public void testAddColumn() throws IOException {
     Admin admin = TEST_UTIL.getAdmin();
     // Create a table with two families
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(TABLE_NAME);
-
-    tableDescriptor.setColumnFamily(
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY_0));
+    TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(TABLE_NAME)
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY_0)).build();
     admin.createTable(tableDescriptor);
     admin.disableTable(TABLE_NAME);
     try {
@@ -133,8 +122,7 @@ public class TestTableDescriptorModificationFromClient {
       verifyTableDescriptor(TABLE_NAME, FAMILY_0);
 
       // Modify the table removing one family and verify the descriptor
-      admin.addColumnFamily(TABLE_NAME,
-        new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY_1));
+      admin.addColumnFamily(TABLE_NAME, ColumnFamilyDescriptorBuilder.of(FAMILY_1));
       verifyTableDescriptor(TABLE_NAME, FAMILY_0, FAMILY_1);
     } finally {
       admin.deleteTable(TABLE_NAME);
@@ -145,11 +133,8 @@ public class TestTableDescriptorModificationFromClient {
   public void testAddSameColumnFamilyTwice() throws IOException {
     Admin admin = TEST_UTIL.getAdmin();
     // Create a table with one families
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(TABLE_NAME);
-
-    tableDescriptor.setColumnFamily(
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY_0));
+    TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(TABLE_NAME)
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY_0)).build();
     admin.createTable(tableDescriptor);
     admin.disableTable(TABLE_NAME);
     try {
@@ -157,14 +142,12 @@ public class TestTableDescriptorModificationFromClient {
       verifyTableDescriptor(TABLE_NAME, FAMILY_0);
 
       // Modify the table removing one family and verify the descriptor
-      admin.addColumnFamily(TABLE_NAME,
-        new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY_1));
+      admin.addColumnFamily(TABLE_NAME, ColumnFamilyDescriptorBuilder.of(FAMILY_1));
       verifyTableDescriptor(TABLE_NAME, FAMILY_0, FAMILY_1);
 
       try {
         // Add same column family again - expect failure
-        admin.addColumnFamily(TABLE_NAME,
-          new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY_1));
+        admin.addColumnFamily(TABLE_NAME, ColumnFamilyDescriptorBuilder.of(FAMILY_1));
         Assert.fail("Delete a non-exist column family should fail");
       } catch (InvalidFamilyOperationException e) {
         // Expected.
@@ -179,14 +162,11 @@ public class TestTableDescriptorModificationFromClient {
   public void testModifyColumnFamily() throws IOException {
     Admin admin = TEST_UTIL.getAdmin();
 
-    ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor cfDescriptor =
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY_0);
+    ColumnFamilyDescriptor cfDescriptor = ColumnFamilyDescriptorBuilder.of(FAMILY_0);
     int blockSize = cfDescriptor.getBlocksize();
     // Create a table with one families
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(TABLE_NAME);
-
-    tableDescriptor.setColumnFamily(cfDescriptor);
+    TableDescriptor tableDescriptor =
+      TableDescriptorBuilder.newBuilder(TABLE_NAME).setColumnFamily(cfDescriptor).build();
     admin.createTable(tableDescriptor);
     admin.disableTable(TABLE_NAME);
     try {
@@ -194,13 +174,14 @@ public class TestTableDescriptorModificationFromClient {
       verifyTableDescriptor(TABLE_NAME, FAMILY_0);
 
       int newBlockSize = 2 * blockSize;
-      cfDescriptor.setBlocksize(newBlockSize);
+      cfDescriptor =
+        ColumnFamilyDescriptorBuilder.newBuilder(cfDescriptor).setBlocksize(newBlockSize).build();
 
       // Modify colymn family
       admin.modifyColumnFamily(TABLE_NAME, cfDescriptor);
 
-      HTableDescriptor htd = new HTableDescriptor(admin.getDescriptor(TABLE_NAME));
-      HColumnDescriptor hcfd = htd.getFamily(FAMILY_0);
+      TableDescriptor htd = admin.getDescriptor(TABLE_NAME);
+      ColumnFamilyDescriptor hcfd = htd.getColumnFamily(FAMILY_0);
       assertTrue(hcfd.getBlocksize() == newBlockSize);
     } finally {
       admin.deleteTable(TABLE_NAME);
@@ -211,15 +192,11 @@ public class TestTableDescriptorModificationFromClient {
   public void testModifyNonExistingColumnFamily() throws IOException {
     Admin admin = TEST_UTIL.getAdmin();
 
-    ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor cfDescriptor =
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY_1);
+    ColumnFamilyDescriptor cfDescriptor = ColumnFamilyDescriptorBuilder.of(FAMILY_1);
     int blockSize = cfDescriptor.getBlocksize();
     // Create a table with one families
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(TABLE_NAME);
-
-    tableDescriptor.setColumnFamily(
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY_0));
+    TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(TABLE_NAME)
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY_0)).build();
     admin.createTable(tableDescriptor);
     admin.disableTable(TABLE_NAME);
     try {
@@ -227,7 +204,8 @@ public class TestTableDescriptorModificationFromClient {
       verifyTableDescriptor(TABLE_NAME, FAMILY_0);
 
       int newBlockSize = 2 * blockSize;
-      cfDescriptor.setBlocksize(newBlockSize);
+      cfDescriptor =
+        ColumnFamilyDescriptorBuilder.newBuilder(cfDescriptor).setBlocksize(newBlockSize).build();
 
       // Modify a column family that is not in the table.
       try {
@@ -246,13 +224,9 @@ public class TestTableDescriptorModificationFromClient {
   public void testDeleteColumn() throws IOException {
     Admin admin = TEST_UTIL.getAdmin();
     // Create a table with two families
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(TABLE_NAME);
-
-    tableDescriptor.setColumnFamily(
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY_0));
-    tableDescriptor.setColumnFamily(
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY_1));
+    TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(TABLE_NAME)
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY_0))
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY_1)).build();
     admin.createTable(tableDescriptor);
     admin.disableTable(TABLE_NAME);
     try {
@@ -271,13 +245,9 @@ public class TestTableDescriptorModificationFromClient {
   public void testDeleteSameColumnFamilyTwice() throws IOException {
     Admin admin = TEST_UTIL.getAdmin();
     // Create a table with two families
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(TABLE_NAME);
-
-    tableDescriptor.setColumnFamily(
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY_0));
-    tableDescriptor.setColumnFamily(
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY_1));
+    TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(TABLE_NAME)
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY_0))
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY_1)).build();
     admin.createTable(tableDescriptor);
     admin.disableTable(TABLE_NAME);
     try {
@@ -305,7 +275,7 @@ public class TestTableDescriptorModificationFromClient {
     Admin admin = TEST_UTIL.getAdmin();
 
     // Verify descriptor from master
-    HTableDescriptor htd = new HTableDescriptor(admin.getDescriptor(tableName));
+    TableDescriptor htd = admin.getDescriptor(tableName);
     verifyTableDescriptor(htd, tableName, families);
 
     // Verify descriptor from HDFS
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/MobStressToolRunner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/MobStressToolRunner.java
index cf6a422..9b63b9d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/MobStressToolRunner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/MobStressToolRunner.java
@@ -22,7 +22,6 @@ import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
 import java.util.Arrays;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -30,6 +29,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
@@ -37,6 +37,7 @@ import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.master.cleaner.TimeToLiveHFileCleaner;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -56,7 +57,6 @@ import org.slf4j.LoggerFactory;
  This class is used by MobStressTool only. This is not a unit test
 
  */
-@SuppressWarnings("deprecation")
 public class MobStressToolRunner {
   private static final Logger LOG = LoggerFactory.getLogger(MobStressToolRunner.class);
 
@@ -71,8 +71,8 @@ public class MobStressToolRunner {
       .toBytes("01234567890123456789012345678901234567890123456789012345678901234567890123456789");
 
   private Configuration conf;
-  private TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor;
-  private ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor;
+  private TableDescriptor tableDescriptor;
+  private ColumnFamilyDescriptor familyDescriptor;
   private Admin admin;
   private long count = 500000;
   private double failureProb = 0.1;
@@ -82,7 +82,6 @@ public class MobStressToolRunner {
   private static volatile boolean run = true;
 
   public MobStressToolRunner() {
-
   }
 
   public void init(Configuration conf, long numRows) throws IOException {
@@ -90,15 +89,13 @@ public class MobStressToolRunner {
     this.count = numRows;
     initConf();
     printConf();
-    tableDescriptor = new TableDescriptorBuilder.ModifyableTableDescriptor(
-      TableName.valueOf("testMobCompactTable"));
     Connection conn = ConnectionFactory.createConnection(this.conf);
     this.admin = conn.getAdmin();
-    this.familyDescriptor = new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(fam);
-    this.familyDescriptor.setMobEnabled(true);
-    this.familyDescriptor.setMobThreshold(mobLen);
-    this.familyDescriptor.setMaxVersions(1);
-    this.tableDescriptor.setColumnFamily(familyDescriptor);
+    this.familyDescriptor = ColumnFamilyDescriptorBuilder.newBuilder(fam).setMobEnabled(true)
+      .setMobThreshold(mobLen).setMaxVersions(1).build();
+    this.tableDescriptor =
+      TableDescriptorBuilder.newBuilder(TableName.valueOf("testMobCompactTable"))
+        .setColumnFamily(familyDescriptor).build();
     if (admin.tableExists(tableDescriptor.getTableName())) {
       admin.disableTable(tableDescriptor.getTableName());
       admin.deleteTable(tableDescriptor.getTableName());
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestDefaultMobStoreFlusher.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestDefaultMobStoreFlusher.java
index 1fa0a68..e0d7d2c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestDefaultMobStoreFlusher.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestDefaultMobStoreFlusher.java
@@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -46,86 +47,74 @@ public class TestDefaultMobStoreFlusher {
 
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
-      HBaseClassTestRule.forClass(TestDefaultMobStoreFlusher.class);
-
- private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
- private final static byte [] row1 = Bytes.toBytes("row1");
- private final static byte [] row2 = Bytes.toBytes("row2");
- private final static byte [] family = Bytes.toBytes("family");
- private final static byte [] qf1 = Bytes.toBytes("qf1");
- private final static byte [] qf2 = Bytes.toBytes("qf2");
- private final static byte [] value1 = Bytes.toBytes("value1");
- private final static byte [] value2 = Bytes.toBytes("value2");
-
- @Rule
- public TestName name = new TestName();
-
- @BeforeClass
- public static void setUpBeforeClass() throws Exception {
-   TEST_UTIL.startMiniCluster(1);
- }
+    HBaseClassTestRule.forClass(TestDefaultMobStoreFlusher.class);
+
+  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private final static byte[] row1 = Bytes.toBytes("row1");
+  private final static byte[] row2 = Bytes.toBytes("row2");
+  private final static byte[] family = Bytes.toBytes("family");
+  private final static byte[] qf1 = Bytes.toBytes("qf1");
+  private final static byte[] qf2 = Bytes.toBytes("qf2");
+  private final static byte[] value1 = Bytes.toBytes("value1");
+  private final static byte[] value2 = Bytes.toBytes("value2");
+
+  @Rule
+  public TestName name = new TestName();
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    TEST_UTIL.startMiniCluster(1);
+  }
 
- @AfterClass
- public static void tearDownAfterClass() throws Exception {
-   TEST_UTIL.shutdownMiniCluster();
- }
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    TEST_UTIL.shutdownMiniCluster();
+  }
 
   @Test
   public void testFlushNonMobFile() throws Exception {
     final TableName tableName = TableName.valueOf(name.getMethodName());
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
-
-    ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family);
-    familyDescriptor.setMaxVersions(4);
-    tableDescriptor.setColumnFamily(familyDescriptor);
-
+    TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName)
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(family).setMaxVersions(4).build())
+      .build();
     testFlushFile(tableDescriptor);
   }
 
   @Test
   public void testFlushMobFile() throws Exception {
     final TableName tableName = TableName.valueOf(name.getMethodName());
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
-
-    ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor hcd =
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family);
-    hcd.setMobEnabled(true);
-    hcd.setMobThreshold(3L);
-    hcd.setMaxVersions(4);
-    tableDescriptor.setColumnFamily(hcd);
-
+    TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName)
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(family).setMobEnabled(true)
+        .setMobThreshold(3L).setMaxVersions(4).build())
+      .build();
     testFlushFile(tableDescriptor);
   }
 
-  private void testFlushFile(TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor)
-      throws Exception {
+  private void testFlushFile(TableDescriptor tableDescriptor) throws Exception {
     Table table = null;
     try {
       table = TEST_UTIL.createTable(tableDescriptor, null);
 
-      //put data
+      // put data
       Put put0 = new Put(row1);
       put0.addColumn(family, qf1, 1, value1);
       table.put(put0);
 
-      //put more data
+      // put more data
       Put put1 = new Put(row2);
       put1.addColumn(family, qf2, 1, value2);
       table.put(put1);
 
-      //flush
+      // flush
       TEST_UTIL.flush(tableDescriptor.getTableName());
 
-      //Scan
+      // Scan
       Scan scan = new Scan();
       scan.addColumn(family, qf1);
       scan.readVersions(4);
       ResultScanner scanner = table.getScanner(scan);
 
-      //Compare
+      // Compare
       int size = 0;
       for (Result result : scanner) {
         size++;
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionOptMode.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionOptMode.java
index 09f7f58..cacc90e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionOptMode.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionOptMode.java
@@ -18,14 +18,11 @@
  */
 package org.apache.hadoop.hbase.mob;
 import java.io.IOException;
-
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
 import org.junit.experimental.categories.Category;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 /**
   * Mob file compaction chore in a generational non-batch mode test.
@@ -42,11 +39,9 @@ import org.slf4j.LoggerFactory;
   * 11 Verifies that number of MOB files in a mob directory is 20.
   * 12 Runs scanner and checks all 3 * 1000 rows.
  */
-@SuppressWarnings("deprecation")
 @Category(LargeTests.class)
 public class TestMobCompactionOptMode extends TestMobCompactionWithDefaults {
-  private static final Logger LOG =
-      LoggerFactory.getLogger(TestMobCompactionOptMode.class);
+
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
       HBaseClassTestRule.forClass(TestMobCompactionOptMode.class);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionOptRegionBatchMode.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionOptRegionBatchMode.java
index 117b9ee..46ef3b0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionOptRegionBatchMode.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionOptRegionBatchMode.java
@@ -46,7 +46,6 @@ import org.slf4j.LoggerFactory;
   * 11 Verifies that number of MOB files in a mob directory is 20.
   * 12 Runs scanner and checks all 3 * 1000 rows.
  */
-@SuppressWarnings("deprecation")
 @Category(LargeTests.class)
 public class TestMobCompactionOptRegionBatchMode extends TestMobCompactionWithDefaults {
   private static final Logger LOG =
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionRegularRegionBatchMode.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionRegularRegionBatchMode.java
index 5151789..bc70425 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionRegularRegionBatchMode.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionRegularRegionBatchMode.java
@@ -46,7 +46,6 @@ import org.slf4j.LoggerFactory;
   * 11 Verifies that number of MOB files in a mob directory is 20.
   * 12 Runs scanner and checks all 3 * 1000 rows.
  */
-@SuppressWarnings("deprecation")
 @Category(LargeTests.class)
 public class TestMobCompactionRegularRegionBatchMode extends TestMobCompactionWithDefaults {
   private static final Logger LOG =
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionWithDefaults.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionWithDefaults.java
index 22fb31f..149de20 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionWithDefaults.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionWithDefaults.java
@@ -26,7 +26,6 @@ import java.util.Arrays;
 import java.util.List;
 import java.util.Random;
 import java.util.stream.Collectors;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -43,7 +42,6 @@ import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.TableDescriptor;
-import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.RegionSplitter;
@@ -75,7 +73,6 @@ import org.slf4j.LoggerFactory;
   * 11 Verifies that number of MOB files in a mob directory is 20.
   * 12 Runs scanner and checks all 3 * 1000 rows.
  */
-@SuppressWarnings("deprecation")
 @Category(LargeTests.class)
 public class TestMobCompactionWithDefaults {
   private static final Logger LOG =
@@ -97,8 +94,8 @@ public class TestMobCompactionWithDefaults {
 
   @Rule
   public TestName test = new TestName();
-  protected TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor;
-  private ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor;
+  protected TableDescriptor tableDescriptor;
+  private ColumnFamilyDescriptor familyDescriptor;
   protected Admin admin;
   protected TableName table = null;
   protected int numRegions = 20;
@@ -130,14 +127,12 @@ public class TestMobCompactionWithDefaults {
 
   @Before
   public void setUp() throws Exception {
-    tableDescriptor = HTU.createModifyableTableDescriptor(test.getMethodName());
     admin = HTU.getAdmin();
     cleanerChore = new MobFileCleanerChore();
-    familyDescriptor = new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(fam);
-    familyDescriptor.setMobEnabled(true);
-    familyDescriptor.setMobThreshold(mobLen);
-    familyDescriptor.setMaxVersions(1);
-    tableDescriptor.setColumnFamily(familyDescriptor);
+    familyDescriptor = ColumnFamilyDescriptorBuilder.newBuilder(fam).setMobEnabled(true)
+      .setMobThreshold(mobLen).setMaxVersions(1).build();
+    tableDescriptor = HTU.createModifyableTableDescriptor(test.getMethodName())
+      .setColumnFamily(familyDescriptor).build();
     RegionSplitter.UniformSplit splitAlgo = new RegionSplitter.UniformSplit();
     byte[][] splitKeys = splitAlgo.split(numRegions);
     table = HTU.createTable(tableDescriptor, splitKeys).getName();
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobDataBlockEncoding.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobDataBlockEncoding.java
index 2b1595d..fefa898 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobDataBlockEncoding.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobDataBlockEncoding.java
@@ -22,11 +22,13 @@ import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -52,9 +54,8 @@ public class TestMobDataBlockEncoding {
   protected final byte[] qf3 = Bytes.toBytes("qualifier3");
   private static Table table;
   private static Admin admin;
-  private static ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor
-    columnFamilyDescriptor;
-  private static TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor;
+  private static ColumnFamilyDescriptor columnFamilyDescriptor;
+  private static TableDescriptor tableDescriptor;
   private static Random random = new Random();
   private static long defaultThreshold = 10;
 
@@ -70,15 +71,10 @@ public class TestMobDataBlockEncoding {
 
   public void setUp(long threshold, String TN, DataBlockEncoding encoding)
       throws Exception {
-    tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(TableName.valueOf(TN));
-    columnFamilyDescriptor =
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family);
-    columnFamilyDescriptor.setMobEnabled(true);
-    columnFamilyDescriptor.setMobThreshold(threshold);
-    columnFamilyDescriptor.setMaxVersions(4);
-    columnFamilyDescriptor.setDataBlockEncoding(encoding);
-    tableDescriptor.setColumnFamily(columnFamilyDescriptor);
+    columnFamilyDescriptor = ColumnFamilyDescriptorBuilder.newBuilder(family).setMobEnabled(true)
+      .setMobThreshold(threshold).setMaxVersions(4).setDataBlockEncoding(encoding).build();
+    tableDescriptor = TableDescriptorBuilder.newBuilder(TableName.valueOf(TN))
+      .setColumnFamily(columnFamilyDescriptor).build();
     admin = TEST_UTIL.getAdmin();
     admin.createTable(tableDescriptor);
     table = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration())
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileCleanerChore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileCleanerChore.java
index 274d045..5465122 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileCleanerChore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileCleanerChore.java
@@ -22,7 +22,6 @@ import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
 import java.util.Arrays;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -31,13 +30,14 @@ import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.CompactionState;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.master.cleaner.TimeToLiveHFileCleaner;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -59,7 +59,6 @@ import org.slf4j.LoggerFactory;
   * 6. Runs Mob cleaner chore
   * 7 Verifies that number of MOB files in a mob directory is 1.
  */
-@SuppressWarnings("deprecation")
 @Category(MediumTests.class)
 public class TestMobFileCleanerChore {
   private static final Logger LOG = LoggerFactory.getLogger(TestMobFileCleanerChore.class);
@@ -77,8 +76,8 @@ public class TestMobFileCleanerChore {
       .toBytes("01234567890123456789012345678901234567890123456789012345678901234567890123456789");
 
   private Configuration conf;
-  private TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor;
-  private ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor;
+  private TableDescriptor tableDescriptor;
+  private ColumnFamilyDescriptor familyDescriptor;
   private Admin admin;
   private Table table = null;
   private MobFileCleanerChore chore;
@@ -90,7 +89,6 @@ public class TestMobFileCleanerChore {
   @Before
   public void setUp() throws Exception {
     HTU = new HBaseTestingUtility();
-    tableDescriptor = HTU.createModifyableTableDescriptor("testMobCompactTable");
     conf = HTU.getConfiguration();
 
     initConf();
@@ -98,11 +96,10 @@ public class TestMobFileCleanerChore {
     HTU.startMiniCluster();
     admin = HTU.getAdmin();
     chore = new MobFileCleanerChore();
-    familyDescriptor = new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(fam);
-    familyDescriptor.setMobEnabled(true);
-    familyDescriptor.setMobThreshold(mobLen);
-    familyDescriptor.setMaxVersions(1);
-    tableDescriptor.setColumnFamily(familyDescriptor);
+    familyDescriptor = ColumnFamilyDescriptorBuilder.newBuilder(fam).setMobEnabled(true)
+      .setMobThreshold(mobLen).setMaxVersions(1).build();
+    tableDescriptor = HTU.createModifyableTableDescriptor("testMobCompactTable")
+      .setColumnFamily(familyDescriptor).build();
     table = HTU.createTable(tableDescriptor, null);
   }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobStoreCompaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobStoreCompaction.java
index de1dba9..1865c0b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobStoreCompaction.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobStoreCompaction.java
@@ -96,8 +96,8 @@ public class TestMobStoreCompaction {
   private Configuration conf = null;
 
   private HRegion region = null;
-  private TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = null;
-  private ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor = null;
+  private TableDescriptor tableDescriptor = null;
+  private ColumnFamilyDescriptor familyDescriptor = null;
   private long mobCellThreshold = 1000;
 
   private FileSystem fs;
@@ -112,13 +112,10 @@ public class TestMobStoreCompaction {
     HBaseTestingUtility UTIL = new HBaseTestingUtility(conf);
 
     compactionThreshold = conf.getInt("hbase.hstore.compactionThreshold", 3);
-    tableDescriptor = UTIL.createModifyableTableDescriptor(name.getMethodName());
-    familyDescriptor =
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(COLUMN_FAMILY);
-    familyDescriptor.setMobEnabled(true);
-    familyDescriptor.setMobThreshold(mobThreshold);
-    familyDescriptor.setMaxVersions(1);
-    tableDescriptor.modifyColumnFamily(familyDescriptor);
+    familyDescriptor = ColumnFamilyDescriptorBuilder.newBuilder(COLUMN_FAMILY).setMobEnabled(true)
+      .setMobThreshold(mobThreshold).setMaxVersions(1).build();
+    tableDescriptor = UTIL.createModifyableTableDescriptor(name.getMethodName())
+      .modifyColumnFamily(familyDescriptor).build();
 
     RegionInfo regionInfo = RegionInfoBuilder.newBuilder(tableDescriptor.getTableName()).build();
     region = HBaseTestingUtility.createRegionAndWAL(regionInfo,
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobStoreScanner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobStoreScanner.java
index eec4e99..32c62ca 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobStoreScanner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobStoreScanner.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.ConnectionConfiguration;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
@@ -40,6 +41,7 @@ import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.io.hfile.CorruptHFileException;
 import org.apache.hadoop.hbase.io.hfile.TestHFile;
@@ -73,8 +75,8 @@ public class TestMobStoreScanner {
   protected final byte[] qf3 = Bytes.toBytes("qualifier3");
   private static Table table;
   private static Admin admin;
-  private static ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor;
-  private static TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor;
+  private static ColumnFamilyDescriptor familyDescriptor;
+  private static TableDescriptor tableDescriptor;
   private static Random random = new Random();
   private static long defaultThreshold = 10;
   private FileSystem fs;
@@ -99,16 +101,13 @@ public class TestMobStoreScanner {
   public void setUp(long threshold, TableName tn) throws Exception {
     conf = TEST_UTIL.getConfiguration();
     fs = FileSystem.get(conf);
-    tableDescriptor = new TableDescriptorBuilder.ModifyableTableDescriptor(tn);
-    familyDescriptor = new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family);
-    familyDescriptor.setMobEnabled(true);
-    familyDescriptor.setMobThreshold(threshold);
-    familyDescriptor.setMaxVersions(4);
-    tableDescriptor.setColumnFamily(familyDescriptor);
+    familyDescriptor = ColumnFamilyDescriptorBuilder.newBuilder(family).setMobEnabled(true)
+      .setMobThreshold(threshold).setMaxVersions(4).build();
+    tableDescriptor =
+      TableDescriptorBuilder.newBuilder(tn).setColumnFamily(familyDescriptor).build();
     admin = TEST_UTIL.getAdmin();
     admin.createTable(tableDescriptor);
-    table = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration())
-            .getTable(tn);
+    table = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration()).getTable(tn);
   }
 
   /**
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java
index bc491c8..7d67387 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java
@@ -344,10 +344,8 @@ public class TestNamespaceAuditor {
     ADMIN.createNamespace(nspDesc);
     final TableName tableTwo = TableName.valueOf(nsp1 + TableName.NAMESPACE_DELIM + "table2");
     byte[] columnFamily = Bytes.toBytes("info");
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(tableTwo);
-    tableDescriptor.setColumnFamily(
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(columnFamily));
+    TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableTwo)
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(columnFamily)).build();
     ADMIN.createTable(tableDescriptor, Bytes.toBytes("0"), Bytes.toBytes("9"), initialRegions);
     Connection connection = ConnectionFactory.createConnection(UTIL.getConfiguration());
     try (Table table = connection.getTable(tableTwo)) {
@@ -442,11 +440,8 @@ public class TestNamespaceAuditor {
     ADMIN.createNamespace(nspDesc);
     final TableName tableOne = TableName.valueOf(nsp1 + TableName.NAMESPACE_DELIM + "table1");
     byte[] columnFamily = Bytes.toBytes("info");
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(tableOne);
-
-    tableDescriptor.setColumnFamily(
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(columnFamily));
+    TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableOne)
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(columnFamily)).build();
     MasterSyncObserver.throwExceptionInPreCreateTableAction = true;
     try {
       try {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java
index a5a1d8f..db96342 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java
@@ -296,19 +296,18 @@ public class TestAtomicOperation {
     initHRegion(tableName, callingMethod, null, families);
   }
 
-  private void initHRegion (byte [] tableName, String callingMethod, int [] maxVersions,
-    byte[] ... families)
-  throws IOException {
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(TableName.valueOf(tableName));
-
-    int i=0;
-    for(byte [] family : families) {
-      ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
-        new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family);
-      familyDescriptor.setMaxVersions(maxVersions != null ? maxVersions[i++] : 1);
-      tableDescriptor.setColumnFamily(familyDescriptor);
+  private void initHRegion(byte[] tableName, String callingMethod, int[] maxVersions,
+    byte[]... families) throws IOException {
+    TableDescriptorBuilder builder =
+      TableDescriptorBuilder.newBuilder(TableName.valueOf(tableName));
+
+    int i = 0;
+    for (byte[] family : families) {
+      ColumnFamilyDescriptor familyDescriptor = ColumnFamilyDescriptorBuilder.newBuilder(family)
+        .setMaxVersions(maxVersions != null ? maxVersions[i++] : 1).build();
+      builder.setColumnFamily(familyDescriptor);
     }
+    TableDescriptor tableDescriptor = builder.build();
     RegionInfo info = RegionInfoBuilder.newBuilder(tableDescriptor.getTableName()).build();
     region = TEST_UTIL.createLocalHRegion(info, tableDescriptor);
   }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksScanned.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksScanned.java
index c3af661..69b1446 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksScanned.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksScanned.java
@@ -75,34 +75,25 @@ public class TestBlocksScanned {
 
   @Test
   public void testBlocksScanned() throws Exception {
-    byte [] tableName = Bytes.toBytes("TestBlocksScanned");
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(TableName.valueOf(tableName));
-
-    tableDescriptor.setColumnFamily(
-        new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY)
-        .setMaxVersions(10)
-        .setBlockCacheEnabled(true)
-        .setBlocksize(BLOCK_SIZE)
-        .setCompressionType(Compression.Algorithm.NONE)
-        );
+    byte[] tableName = Bytes.toBytes("TestBlocksScanned");
+    TableDescriptor tableDescriptor =
+      TableDescriptorBuilder.newBuilder(TableName.valueOf(tableName))
+        .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(FAMILY).setMaxVersions(10)
+          .setBlockCacheEnabled(true).setBlocksize(BLOCK_SIZE)
+          .setCompressionType(Compression.Algorithm.NONE).build())
+        .build();
     _testBlocksScanned(tableDescriptor);
   }
 
   @Test
   public void testBlocksScannedWithEncoding() throws Exception {
-    byte [] tableName = Bytes.toBytes("TestBlocksScannedWithEncoding");
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(TableName.valueOf(tableName));
-
-    tableDescriptor.setColumnFamily(
-        new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY)
-        .setMaxVersions(10)
-        .setBlockCacheEnabled(true)
-        .setDataBlockEncoding(DataBlockEncoding.FAST_DIFF)
-        .setBlocksize(BLOCK_SIZE)
-        .setCompressionType(Compression.Algorithm.NONE)
-        );
+    byte[] tableName = Bytes.toBytes("TestBlocksScannedWithEncoding");
+    TableDescriptor tableDescriptor =
+      TableDescriptorBuilder.newBuilder(TableName.valueOf(tableName))
+        .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(FAMILY).setMaxVersions(10)
+          .setBlockCacheEnabled(true).setDataBlockEncoding(DataBlockEncoding.FAST_DIFF)
+          .setBlocksize(BLOCK_SIZE).setCompressionType(Compression.Algorithm.NONE).build())
+        .build();
     _testBlocksScanned(tableDescriptor);
   }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoad.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoad.java
index fd36b41..f1f26ba 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoad.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoad.java
@@ -256,21 +256,18 @@ public class TestBulkLoad {
     return new Pair<>(new byte[]{0x00, 0x01, 0x02}, getNotExistFilePath());
   }
 
-
   private HRegion testRegionWithFamiliesAndSpecifiedTableName(TableName tableName,
     byte[]... families) throws IOException {
     RegionInfo hRegionInfo = RegionInfoBuilder.newBuilder(tableName).build();
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
+    TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableName);
 
     for (byte[] family : families) {
-      tableDescriptor.setColumnFamily(
-        new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family));
+      builder.setColumnFamily(ColumnFamilyDescriptorBuilder.of(family));
     }
     ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     // TODO We need a way to do this without creating files
     return HRegion.createHRegion(hRegionInfo, new Path(testFolder.newFolder().toURI()), conf,
-      tableDescriptor, log);
+      builder.build(), log);
   }
 
   private HRegion testRegionWithFamilies(byte[]... families) throws IOException {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestColumnSeeking.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestColumnSeeking.java
index fc82991..cf39dc0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestColumnSeeking.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestColumnSeeking.java
@@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionInfoBuilder;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
@@ -71,14 +72,10 @@ public class TestColumnSeeking {
     byte[] familyBytes = Bytes.toBytes("Family");
     TableName table = TableName.valueOf(name.getMethodName());
 
-    ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(familyBytes)
-        .setMaxVersions(1000);
-    familyDescriptor.setMaxVersions(3);
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(table);
-
-    tableDescriptor.setColumnFamily(familyDescriptor);
+    ColumnFamilyDescriptor familyDescriptor = ColumnFamilyDescriptorBuilder.newBuilder(familyBytes)
+      .setMaxVersions(1000).setMaxVersions(3).build();
+    TableDescriptor tableDescriptor =
+      TableDescriptorBuilder.newBuilder(table).setColumnFamily(familyDescriptor).build();
     RegionInfo info = RegionInfoBuilder.newBuilder(table).build();
     // Set this so that the archiver writes to the temp dir as well.
     HRegion region = TEST_UTIL.createLocalHRegion(info, tableDescriptor);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactSplitThread.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactSplitThread.java
index cc2921b..f191e48 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactSplitThread.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactSplitThread.java
@@ -25,11 +25,11 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -112,11 +112,9 @@ public class TestCompactSplitThread {
     Configuration conf = TEST_UTIL.getConfiguration();
     Connection conn = ConnectionFactory.createConnection(conf);
     try {
-      TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-        new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
-      tableDescriptor.setColumnFamily(
-        new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family));
-      tableDescriptor.setCompactionEnabled(false);
+      TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName)
+        .setColumnFamily(ColumnFamilyDescriptorBuilder.of(family)).setCompactionEnabled(false)
+        .build();
       TEST_UTIL.getAdmin().createTable(tableDescriptor);
       TEST_UTIL.waitTableAvailable(tableName);
       HRegionServer regionServer = TEST_UTIL.getRSForFirstRegionInTable(tableName);
@@ -162,8 +160,8 @@ public class TestCompactSplitThread {
 
   @Test
   public void testFlushWithTableCompactionDisabled() throws Exception {
-    HTableDescriptor htd = new HTableDescriptor(tableName);
-    htd.setCompactionEnabled(false);
+    TableDescriptor htd =
+      TableDescriptorBuilder.newBuilder(tableName).setCompactionEnabled(false).build();
     TEST_UTIL.createTable(htd, new byte[][] { family }, null);
 
     // load the table
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingMemStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingMemStore.java
index aeb5045..e4b720e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingMemStore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingMemStore.java
@@ -40,10 +40,12 @@ import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValueTestUtil;
 import org.apache.hadoop.hbase.MemoryCompactionPolicy;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionInfoBuilder;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.exceptions.IllegalArgumentIOException;
 import org.apache.hadoop.hbase.io.util.MemorySizeUtil;
@@ -108,12 +110,9 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
     conf.setFloat(MemStoreLAB.CHUNK_POOL_MAXSIZE_KEY, 0.2f);
     conf.setInt(HRegion.MEMSTORE_PERIODIC_FLUSH_INTERVAL, 1000);
     HBaseTestingUtility hbaseUtility = new HBaseTestingUtility(conf);
-    ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY);
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(TableName.valueOf("foobar"));
-
-    tableDescriptor.setColumnFamily(familyDescriptor);
+    ColumnFamilyDescriptor familyDescriptor = ColumnFamilyDescriptorBuilder.of(FAMILY);
+    TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(TableName.valueOf("foobar"))
+      .setColumnFamily(familyDescriptor).build();
     RegionInfo info = RegionInfoBuilder.newBuilder(TableName.valueOf("foobar")).build();
     WAL wal = HBaseTestingUtility.createWal(conf, hbaseUtility.getDataTestDir(), info);
     this.region = HRegion.createHRegion(info, hbaseUtility.getDataTestDir(), conf,
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java
index 7c8494a..2174d51 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java
@@ -52,11 +52,13 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HTestConst;
 import org.apache.hadoop.hbase.Waiter;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Durability;
 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.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.io.hfile.HFileScanner;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
@@ -99,7 +101,7 @@ public class TestCompaction {
   protected Configuration conf = UTIL.getConfiguration();
 
   private HRegion r = null;
-  private TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor = null;
+  private TableDescriptor tableDescriptor = null;
   private static final byte [] COLUMN_FAMILY = fam1;
   private final byte [] STARTROW = Bytes.toBytes(START_KEY);
   private static final byte [] COLUMN_FAMILY_TEXT = COLUMN_FAMILY;
@@ -129,17 +131,16 @@ public class TestCompaction {
 
   @Before
   public void setUp() throws Exception {
-    this.tableDescriptor = UTIL.createModifyableTableDescriptor(name.getMethodName());
+    TableDescriptorBuilder builder = UTIL.createModifyableTableDescriptor(name.getMethodName());
     if (name.getMethodName().equals("testCompactionSeqId")) {
       UTIL.getConfiguration().set("hbase.hstore.compaction.kv.max", "10");
-      UTIL.getConfiguration().set(
-          DefaultStoreEngine.DEFAULT_COMPACTOR_CLASS_KEY,
-          DummyCompactor.class.getName());
-      ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
-        new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY);
-      familyDescriptor.setMaxVersions(65536);
-      this.tableDescriptor.setColumnFamily(familyDescriptor);
+      UTIL.getConfiguration().set(DefaultStoreEngine.DEFAULT_COMPACTOR_CLASS_KEY,
+        DummyCompactor.class.getName());
+      ColumnFamilyDescriptor familyDescriptor =
+        ColumnFamilyDescriptorBuilder.newBuilder(FAMILY).setMaxVersions(65536).build();
+      builder.setColumnFamily(familyDescriptor);
     }
+    this.tableDescriptor = builder.build();
     this.r = UTIL.createLocalHRegion(tableDescriptor, null, null);
   }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionPolicy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionPolicy.java
index 0173008..33dc8d5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionPolicy.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionPolicy.java
@@ -28,9 +28,11 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionConfiguration;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequestImpl;
@@ -91,16 +93,15 @@ public class TestCompactionPolicy {
     Path basedir = new Path(DIR);
     String logName = "logs";
     Path logdir = new Path(DIR, logName);
-    ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(Bytes.toBytes("family"));
+    ColumnFamilyDescriptor familyDescriptor =
+      ColumnFamilyDescriptorBuilder.of(Bytes.toBytes("family"));
     FileSystem fs = FileSystem.get(conf);
 
     fs.delete(logdir, true);
 
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(
-        TableName.valueOf(Bytes.toBytes("table")));
-    tableDescriptor.setColumnFamily(familyDescriptor);
+    TableDescriptor tableDescriptor =
+      TableDescriptorBuilder.newBuilder(TableName.valueOf(Bytes.toBytes("table")))
+        .setColumnFamily(familyDescriptor).build();
     RegionInfo info = RegionInfoBuilder.newBuilder(tableDescriptor.getTableName()).build();
 
     hlog = new FSHLog(fs, basedir, logName, conf);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDeleteMobTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDeleteMobTable.java
index f894787..7804d70 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDeleteMobTable.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDeleteMobTable.java
@@ -87,18 +87,13 @@ public class TestDeleteMobTable {
     return mobVal;
   }
 
-  private TableDescriptorBuilder.ModifyableTableDescriptor createTableDescriptor(
-      TableName tableName, boolean hasMob) {
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
-    ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY);
+  private TableDescriptor createTableDescriptor(TableName tableName, boolean hasMob) {
+    ColumnFamilyDescriptorBuilder builder = ColumnFamilyDescriptorBuilder.newBuilder(FAMILY);
     if (hasMob) {
-      familyDescriptor.setMobEnabled(true);
-      familyDescriptor.setMobThreshold(0);
+      builder.setMobEnabled(true);
+      builder.setMobThreshold(0);
     }
-    tableDescriptor.setColumnFamily(familyDescriptor);
-    return tableDescriptor;
+    return TableDescriptorBuilder.newBuilder(tableName).setColumnFamily(builder.build()).build();
   }
 
   private Table createTableWithOneFile(TableDescriptor tableDescriptor) throws IOException {
@@ -173,12 +168,10 @@ public class TestDeleteMobTable {
   @Test
   public void testMobFamilyDelete() throws Exception {
     final TableName tableName = TableName.valueOf(name.getMethodName());
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      createTableDescriptor(tableName, true);
+    TableDescriptor tableDescriptor = createTableDescriptor(tableName, true);
     ColumnFamilyDescriptor familyDescriptor = tableDescriptor.getColumnFamily(FAMILY);
-    tableDescriptor.setColumnFamily(
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(
-        Bytes.toBytes("family2")));
+    tableDescriptor = TableDescriptorBuilder.newBuilder(tableDescriptor)
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(Bytes.toBytes("family2"))).build();
 
     Table table = createTableWithOneFile(tableDescriptor);
     try {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFSErrorsExposed.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFSErrorsExposed.java
index 4cafac5..732d0f5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFSErrorsExposed.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFSErrorsExposed.java
@@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.fs.HFileSystem;
 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
@@ -201,13 +202,9 @@ public class TestFSErrorsExposed {
       byte[] fam = Bytes.toBytes("fam");
 
       Admin admin = util.getAdmin();
-      TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-        new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
-      tableDescriptor.setColumnFamily(
-        new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(fam)
-          .setMaxVersions(1)
-          .setBlockCacheEnabled(false)
-      );
+      TableDescriptor tableDescriptor =
+        TableDescriptorBuilder.newBuilder(tableName).setColumnFamily(ColumnFamilyDescriptorBuilder
+          .newBuilder(fam).setMaxVersions(1).setBlockCacheEnabled(false).build()).build();
       admin.createTable(tableDescriptor);
 
       // Make a new Configuration so it makes a new connection that has the
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
index fdda388..f44137d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
@@ -3382,13 +3382,11 @@ public class TestHRegion {
     byte[] value2 = Bytes.toBytes("value2");
 
     final int maxVersions = 3;
-    ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(fam1);
-    familyDescriptor.setMaxVersions(maxVersions);
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(
-        TableName.valueOf("testFilterAndColumnTracker"));
-    tableDescriptor.setColumnFamily(familyDescriptor);
+    TableDescriptor tableDescriptor =
+      TableDescriptorBuilder.newBuilder(TableName.valueOf("testFilterAndColumnTracker"))
+        .setColumnFamily(
+          ColumnFamilyDescriptorBuilder.newBuilder(fam1).setMaxVersions(maxVersions).build())
+        .build();
     ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     RegionInfo info = RegionInfoBuilder.newBuilder(tableDescriptor.getTableName()).build();
     Path logDir = TEST_UTIL.getDataTestDirOnTestFS(method + ".log");
@@ -4695,14 +4693,10 @@ public class TestHRegion {
     byte[] qf1 = Bytes.toBytes("col");
     byte[] val1 = Bytes.toBytes("value1");
     // Create Table
-    ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(fam1)
-        .setMaxVersions(Integer.MAX_VALUE)
-        .setBloomFilterType(BloomType.ROWCOL);
-
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
-    tableDescriptor.setColumnFamily(familyDescriptor);
+    TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName)
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(fam1)
+        .setMaxVersions(Integer.MAX_VALUE).setBloomFilterType(BloomType.ROWCOL).build())
+      .build();
     RegionInfo info = RegionInfoBuilder.newBuilder(tableDescriptor.getTableName()).build();
     this.region = TEST_UTIL.createLocalHRegion(info, tableDescriptor);
     int num_unique_rows = 10;
@@ -4752,13 +4746,10 @@ public class TestHRegion {
     byte[] FAMILY = Bytes.toBytes("family");
 
     // Create table
-    ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY)
-        .setMaxVersions(Integer.MAX_VALUE)
-        .setBloomFilterType(BloomType.ROWCOL);
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(TableName.valueOf(TABLE));
-    tableDescriptor.setColumnFamily(familyDescriptor);
+    TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(TableName.valueOf(TABLE))
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(FAMILY)
+        .setMaxVersions(Integer.MAX_VALUE).setBloomFilterType(BloomType.ROWCOL).build())
+      .build();
     RegionInfo info = RegionInfoBuilder.newBuilder(tableDescriptor.getTableName()).build();
     this.region = TEST_UTIL.createLocalHRegion(info, tableDescriptor);
     // For row:0, col:0: insert versions 1 through 5.
@@ -4797,14 +4788,10 @@ public class TestHRegion {
     byte[] familyName = Bytes.toBytes("familyName");
 
     // Create Table
-    ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(familyName)
-        .setMaxVersions(Integer.MAX_VALUE)
-        .setBloomFilterType(BloomType.ROWCOL);
-
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
-    tableDescriptor.setColumnFamily(familyDescriptor);
+    TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName)
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(familyName)
+        .setMaxVersions(Integer.MAX_VALUE).setBloomFilterType(BloomType.ROWCOL).build())
+      .build();
     RegionInfo info = RegionInfoBuilder.newBuilder(tableDescriptor.getTableName()).build();
     this.region = TEST_UTIL.createLocalHRegion(info, tableDescriptor);
     // Insert some data
@@ -5345,14 +5332,13 @@ public class TestHRegion {
         Bytes.toBytes("cf1"), Bytes.toBytes("cf2"), Bytes.toBytes("cf3")
     };
     byte[] cq = Bytes.toBytes("cq");
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(
+    TableDescriptorBuilder builder =
+      TableDescriptorBuilder.newBuilder(
         TableName.valueOf(name.getMethodName()));
     for (byte[] family : families) {
-      tableDescriptor.setColumnFamily(
-        new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family));
+      builder.setColumnFamily(ColumnFamilyDescriptorBuilder.of(family));
     }
-
+    TableDescriptor tableDescriptor = builder.build();
     long time = System.currentTimeMillis();
     RegionInfo primaryHri = RegionInfoBuilder.newBuilder(tableDescriptor.getTableName())
       .setRegionId(time).setReplicaId(0).build();
@@ -5396,14 +5382,12 @@ public class TestHRegion {
         Bytes.toBytes("cf1"), Bytes.toBytes("cf2"), Bytes.toBytes("cf3")
     };
     byte[] cq = Bytes.toBytes("cq");
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(
-        TableName.valueOf(name.getMethodName()));
+    TableDescriptorBuilder builder =
+      TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()));
     for (byte[] family : families) {
-      tableDescriptor.setColumnFamily(
-        new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family));
+      builder.setColumnFamily(ColumnFamilyDescriptorBuilder.of(family));
     }
-
+    TableDescriptor tableDescriptor = builder.build();
     long time = System.currentTimeMillis();
     RegionInfo primaryHri = RegionInfoBuilder.newBuilder(tableDescriptor.getTableName())
       .setRegionId(time).setReplicaId(0).build();
@@ -5456,14 +5440,12 @@ public class TestHRegion {
         Bytes.toBytes("cf1"), Bytes.toBytes("cf2"), Bytes.toBytes("cf3")
     };
     byte[] cq = Bytes.toBytes("cq");
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(
-        TableName.valueOf(name.getMethodName()));
+    TableDescriptorBuilder builder =
+      TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()));
     for (byte[] family : families) {
-      tableDescriptor.setColumnFamily(
-        new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family));
+      builder.setColumnFamily(ColumnFamilyDescriptorBuilder.of(family));
     }
-
+    TableDescriptor tableDescriptor = builder.build();
     long time = System.currentTimeMillis();
     RegionInfo primaryHri = RegionInfoBuilder.newBuilder(tableDescriptor.getTableName())
       .setRegionId(time).setReplicaId(0).build();
@@ -6492,10 +6474,8 @@ public class TestHRegion {
   @Test
   public void testFlushedFileWithNoTags() throws Exception {
     final TableName tableName = TableName.valueOf(name.getMethodName());
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
-    tableDescriptor.setColumnFamily(
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(fam1));
+    TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName)
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(fam1)).build();
     RegionInfo info = RegionInfoBuilder.newBuilder(tableName).build();
     Path path = TEST_UTIL.getDataTestDir(getClass().getSimpleName());
     region = HBaseTestingUtility.createRegionAndWAL(info, path,
@@ -6665,13 +6645,11 @@ public class TestHRegion {
     final byte[] q3 = Bytes.toBytes("q3");
     final byte[] q4 = Bytes.toBytes("q4");
 
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(
-        TableName.valueOf(name.getMethodName()));
-    ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(fam1);
-    familyDescriptor.setTimeToLive(10); // 10 seconds
-    tableDescriptor.setColumnFamily(familyDescriptor);
+    // 10 seconds
+    TableDescriptor tableDescriptor =
+      TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+        .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(fam1).setTimeToLive(10).build())
+        .build();
 
     Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
     conf.setInt(HFile.FORMAT_VERSION_KEY, HFile.MIN_FORMAT_VERSION_WITH_TAGS);
@@ -7173,11 +7151,9 @@ public class TestHRegion {
     final ServerName serverName = ServerName.valueOf(name.getMethodName(), 100, 42);
     final RegionServerServices rss = spy(TEST_UTIL.createMockRegionServerService(serverName));
 
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(
-        TableName.valueOf(name.getMethodName()));
-    tableDescriptor.setColumnFamily(
-      new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(fam1));
+    TableDescriptor tableDescriptor =
+      TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+        .setColumnFamily(ColumnFamilyDescriptorBuilder.of(fam1)).build();
     RegionInfo hri = RegionInfoBuilder.newBuilder(tableDescriptor.getTableName()).build();
     region = HRegion.openHRegion(hri, tableDescriptor, rss.getWAL(hri),
       TEST_UTIL.getConfiguration(), rss, null);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionOnCluster.java
index 7685815c..85848a0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionOnCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionOnCluster.java
@@ -37,6 +37,7 @@ import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -81,10 +82,8 @@ public class TestHRegionOnCluster {
       HMaster master = cluster.getMaster();
 
       // Create table
-      TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-        new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
-      tableDescriptor.setColumnFamily(
-        new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(FAMILY));
+      TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName)
+        .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY)).build();
       hbaseAdmin = master.getConnection().getAdmin();
       hbaseAdmin.createTable(tableDescriptor);
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestIsDeleteFailure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestIsDeleteFailure.java
index 8420358..13e3110 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestIsDeleteFailure.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestIsDeleteFailure.java
@@ -21,7 +21,6 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.CompareOperator;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Mutation;
@@ -29,6 +28,8 @@ import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.filter.BinaryComparator;
 import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
 import org.apache.hadoop.hbase.testclassification.FilterTests;
@@ -73,7 +74,8 @@ public class TestIsDeleteFailure {
 
   @Test
   public void testIsDeleteFailure() throws Exception {
-    final HTableDescriptor table = new HTableDescriptor(TableName.valueOf(name.getMethodName()));
+    final TableDescriptor table =
+      TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName())).build();
     final byte[] family = Bytes.toBytes("0");
     final byte[] c1 = Bytes.toBytes("C01");
     final byte[] c2 = Bytes.toBytes("C02");
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestJoinedScanners.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestJoinedScanners.java
index d11e046..ca3ae63 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestJoinedScanners.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestJoinedScanners.java
@@ -21,7 +21,6 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Random;
-
 import org.apache.hadoop.hbase.CompareOperator;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
@@ -29,6 +28,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.StartMiniClusterOption;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.Put;
@@ -52,6 +52,7 @@ import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine;
 import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLineParser;
 import org.apache.hbase.thirdparty.org.apache.commons.cli.GnuParser;
@@ -111,14 +112,14 @@ public class TestJoinedScanners {
     byte[][] families = {cf_essential, cf_joined};
 
     final TableName tableName = TableName.valueOf(name.getMethodName());
-    TableDescriptorBuilder.ModifyableTableDescriptor tableDescriptor =
-      new TableDescriptorBuilder.ModifyableTableDescriptor(tableName);
+    TableDescriptorBuilder builder =
+      TableDescriptorBuilder.newBuilder(tableName);
     for (byte[] family : families) {
-      ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor familyDescriptor =
-        new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(family);
-      familyDescriptor.setDataBlockEncoding(blockEncoding);
-      tableDescriptor.setColumnFamily(familyDescriptor);
+      ColumnFamilyDescriptor familyDescriptor = ColumnFamilyDescriptorBuilder.newBuilder(family)
+        .setDataBlockEncoding(blockEncoding).build();
+      builder.setColumnFamily(familyDescriptor);
     }
+    TableDescriptor tableDescriptor = builder.build();
     TEST_UTIL.getAdmin().createTable(tableDescriptor);
     Table ht = TEST_UTIL.getConnection().getTable(tableName);
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestKeepDeletes.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestKeepDeletes.java
index b2c9ef2..ca9e0f6 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestKeepDeletes.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestKeepDeletes.java
@@ -23,6 +23,7 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
+
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
@@ -31,7 +32,6 @@ import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.KeepDeletedCells;
 import org.apache.hadoop.hbase.PrivateCellUtil;
 import org.apache.hadoop.hbase.TableName;
@@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -104,7 +105,7 @@ public class TestKeepDeletes {
   @Test
   public void testBasicScenario() throws Exception {
     // keep 3 versions, rows do not expire
-    HTableDescriptor htd = hbu.createTableDescriptor(TableName.valueOf(name.getMethodName()), 0, 3,
+    TableDescriptor htd = hbu.createTableDescriptor(TableName.valueOf(name.getMethodName()), 0, 3,
         HConstants.FOREVER, KeepDeletedCells.TRUE);
     HRegion region = hbu.createLocalHRegion(htd, null, null);
 
@@ -201,7 +202,7 @@ public class TestKeepDeletes {
   @Test
   public void testRawScanWithoutKeepingDeletes() throws Exception {
     // KEEP_DELETED_CELLS is NOT enabled
-    HTableDescriptor htd = hbu.createTableDescriptor(TableName.valueOf(name.getMethodName()), 0, 3,
+    TableDescriptor htd = hbu.createTableDescriptor(TableName.valueOf(name.getMethodName()), 0, 3,
         HConstants.FOREVER, KeepDeletedCells.FALSE);
     HRegion region = hbu.createLocalHRegion(htd, null, null);
 
@@ -246,7 +247,7 @@ public class TestKeepDeletes {
   @Test
   public void testWithoutKeepingDeletes() throws Exception {
     // KEEP_DELETED_CELLS is NOT enabled
-    HTableDescriptor htd = hbu.createTableDescriptor(TableName.valueOf(name.getMethodName()), 0, 3,
+    TableDescriptor htd = hbu.createTableDescriptor(TableName.valueOf(name.getMethodName()), 0, 3,
         HConstants.FOREVER, KeepDeletedCells.FALSE);
     HRegion region = hbu.createLocalHRegion(htd, null, null);
 
@@ -300,7 +301,7 @@ public class TestKeepDeletes {
    */
   @Test
   public void testRawScanWithColumns() throws Exception {
-    HTableDescriptor htd = hbu.createTableDescriptor(TableName.valueOf(name.getMethodName()), 0, 3,
+    TableDescriptor htd = hbu.createTableDescriptor(TableName.valueOf(name.getMethodName()), 0, 3,
         HConstants.FOREVER, KeepDeletedCells.TRUE);
     Region region = hbu.createLocalHRegion(htd, null, null);
 
@@ -324,7 +325,7 @@ public class TestKeepDeletes {
    */
   @Test
   public void testRawScan() throws Exception {
-    HTableDescriptor htd = hbu.createTableDescriptor(TableName.valueOf(name.getMethodName()), 0, 3,
+    TableDescriptor htd = hbu.createTableDescriptor(TableName.valueOf(name.getMethodName()), 0, 3,
         HConstants.FOREVER, KeepDeletedCells.TRUE);
     Region region = hbu.createLocalHRegion(htd, null, null);
 
@@ -414,7 +415,7 @@ public class TestKeepDeletes {
    */
... 3784 lines suppressed ...