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

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/053e6154/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestTableDescriptorBuilder.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestTableDescriptorBuilder.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestTableDescriptorBuilder.java
new file mode 100644
index 0000000..c4ecacf
--- /dev/null
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestTableDescriptorBuilder.java
@@ -0,0 +1,376 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.client;
+
+import org.apache.hadoop.hbase.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+import java.util.regex.Pattern;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.testclassification.MiscTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.BuilderStyleTest;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+
+/**
+ * Test setting values in the descriptor
+ */
+@Category({MiscTests.class, SmallTests.class})
+public class TestTableDescriptorBuilder {
+  private static final Log LOG = LogFactory.getLog(TestTableDescriptorBuilder.class);
+
+  @Rule
+  public TestName name = new TestName();
+
+  @Test (expected=IOException.class)
+  public void testAddCoprocessorTwice() throws IOException {
+    String cpName = "a.b.c.d";
+    TableDescriptor htd
+      = TableDescriptorBuilder.newBuilder(TableName.META_TABLE_NAME)
+            .addCoprocessor(cpName)
+            .addCoprocessor(cpName)
+            .build();
+  }
+
+  @Test
+  public void testAddCoprocessorWithSpecStr() throws IOException {
+    String cpName = "a.b.c.d";
+    TableDescriptorBuilder builder
+      = TableDescriptorBuilder.newBuilder(TableName.META_TABLE_NAME);
+
+    try {
+      builder.addCoprocessorWithSpec(cpName);
+      fail();
+    } catch (IllegalArgumentException iae) {
+      // Expected as cpName is invalid
+    }
+
+    // Try minimal spec.
+    try {
+      builder.addCoprocessorWithSpec("file:///some/path" + "|" + cpName);
+      fail();
+    } catch (IllegalArgumentException iae) {
+      // Expected to be invalid
+    }
+
+    // Try more spec.
+    String spec = "hdfs:///foo.jar|com.foo.FooRegionObserver|1001|arg1=1,arg2=2";
+    try {
+      builder.addCoprocessorWithSpec(spec);
+    } catch (IllegalArgumentException iae) {
+      fail();
+    }
+
+    // Try double add of same coprocessor
+    try {
+      builder.addCoprocessorWithSpec(spec);
+      fail();
+    } catch (IOException ioe) {
+      // Expect that the coprocessor already exists
+    }
+  }
+
+  @Test
+  public void testPb() throws DeserializationException, IOException {
+    final int v = 123;
+    TableDescriptor htd
+      = TableDescriptorBuilder.newBuilder(TableName.META_TABLE_NAME)
+          .setMaxFileSize(v)
+          .setDurability(Durability.ASYNC_WAL)
+          .setReadOnly(true)
+          .setRegionReplication(2)
+          .build();
+
+    byte [] bytes = TableDescriptorBuilder.toByteArray(htd);
+    TableDescriptor deserializedHtd = TableDescriptorBuilder.newBuilder(bytes).build();
+    assertEquals(htd, deserializedHtd);
+    assertEquals(v, deserializedHtd.getMaxFileSize());
+    assertTrue(deserializedHtd.isReadOnly());
+    assertEquals(Durability.ASYNC_WAL, deserializedHtd.getDurability());
+    assertEquals(deserializedHtd.getRegionReplication(), 2);
+  }
+
+  /**
+   * Test cps in the table description
+   * @throws Exception
+   */
+  @Test
+  public void testGetSetRemoveCP() throws Exception {
+    // simple CP
+    String className = "org.apache.hadoop.hbase.coprocessor.SimpleRegionObserver";
+    TableDescriptor desc
+      = TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+         .addCoprocessor(className) // add and check that it is present
+        .build();
+    assertTrue(desc.hasCoprocessor(className));
+    desc = TableDescriptorBuilder.newBuilder(desc)
+         .removeCoprocessor(className) // remove it and check that it is gone
+        .build();
+    assertFalse(desc.hasCoprocessor(className));
+  }
+
+  /**
+   * Test cps in the table description
+   * @throws Exception
+   */
+  @Test
+  public void testSetListRemoveCP() throws Exception {
+    TableDescriptor desc
+      = TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName())).build();
+    // Check that any coprocessor is present.
+    assertTrue(desc.getCoprocessors().isEmpty());
+
+    // simple CP
+    String className1 = "org.apache.hadoop.hbase.coprocessor.SimpleRegionObserver";
+    String className2 = "org.apache.hadoop.hbase.coprocessor.SampleRegionWALObserver";
+    desc = TableDescriptorBuilder.newBuilder(desc)
+            .addCoprocessor(className1) // Add the 1 coprocessor and check if present.
+            .build();
+    assertTrue(desc.getCoprocessors().size() == 1);
+    assertTrue(desc.getCoprocessors().contains(className1));
+
+    desc = TableDescriptorBuilder.newBuilder(desc)
+            // Add the 2nd coprocessor and check if present.
+            // remove it and check that it is gone
+            .addCoprocessor(className2)
+            .build();
+    assertTrue(desc.getCoprocessors().size() == 2);
+    assertTrue(desc.getCoprocessors().contains(className2));
+
+    desc = TableDescriptorBuilder.newBuilder(desc)
+            // Remove one and check
+            .removeCoprocessor(className1)
+            .build();
+    assertTrue(desc.getCoprocessors().size() == 1);
+    assertFalse(desc.getCoprocessors().contains(className1));
+    assertTrue(desc.getCoprocessors().contains(className2));
+
+    desc = TableDescriptorBuilder.newBuilder(desc)
+            // Remove the last and check
+            .removeCoprocessor(className2)
+            .build();
+    assertTrue(desc.getCoprocessors().isEmpty());
+    assertFalse(desc.getCoprocessors().contains(className1));
+    assertFalse(desc.getCoprocessors().contains(className2));
+  }
+
+  /**
+   * Test that we add and remove strings from settings properly.
+   * @throws Exception
+   */
+  @Test
+  public void testRemoveString() throws Exception {
+    byte[] key = Bytes.toBytes("Some");
+    byte[] value = Bytes.toBytes("value");
+    TableDescriptor desc
+      = TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+            .setValue(key, value)
+            .build();
+    assertTrue(Bytes.equals(value, desc.getValue(key)));
+    desc = TableDescriptorBuilder.newBuilder(desc)
+            .remove(key)
+            .build();
+    assertTrue(desc.getValue(key) == null);
+  }
+
+  String legalTableNames[] = { "foo", "with-dash_under.dot", "_under_start_ok",
+      "with-dash.with_underscore", "02-01-2012.my_table_01-02", "xyz._mytable_", "9_9_0.table_02"
+      , "dot1.dot2.table", "new.-mytable", "with-dash.with.dot", "legal..t2", "legal..legal.t2",
+      "trailingdots..", "trailing.dots...", "ns:mytable", "ns:_mytable_", "ns:my_table_01-02"};
+  String illegalTableNames[] = { ".dot_start_illegal", "-dash_start_illegal", "spaces not ok",
+      "-dash-.start_illegal", "new.table with space", "01 .table", "ns:-illegaldash",
+      "new:.illegaldot", "new:illegalcolon1:", "new:illegalcolon1:2"};
+
+  @Test
+  public void testLegalTableNames() {
+    for (String tn : legalTableNames) {
+      TableName.isLegalFullyQualifiedTableName(Bytes.toBytes(tn));
+    }
+  }
+
+  @Test
+  public void testIllegalTableNames() {
+    for (String tn : illegalTableNames) {
+      try {
+        TableName.isLegalFullyQualifiedTableName(Bytes.toBytes(tn));
+        fail("invalid tablename " + tn + " should have failed");
+      } catch (Exception e) {
+        // expected
+      }
+    }
+  }
+
+  @Test
+  public void testLegalTableNamesRegex() {
+    for (String tn : legalTableNames) {
+      TableName tName = TableName.valueOf(tn);
+      assertTrue("Testing: '" + tn + "'", Pattern.matches(TableName.VALID_USER_TABLE_REGEX,
+          tName.getNameAsString()));
+    }
+  }
+
+  @Test
+  public void testIllegalTableNamesRegex() {
+    for (String tn : illegalTableNames) {
+      LOG.info("Testing: '" + tn + "'");
+      assertFalse(Pattern.matches(TableName.VALID_USER_TABLE_REGEX, tn));
+    }
+  }
+
+    /**
+   * Test default value handling for maxFileSize
+   */
+  @Test
+  public void testGetMaxFileSize() {
+    TableDescriptor desc = TableDescriptorBuilder
+            .newBuilder(TableName.valueOf(name.getMethodName())).build();
+    assertEquals(-1, desc.getMaxFileSize());
+    desc = TableDescriptorBuilder
+            .newBuilder(TableName.valueOf(name.getMethodName()))
+            .setMaxFileSize(1111L).build();
+    assertEquals(1111L, desc.getMaxFileSize());
+  }
+
+  /**
+   * Test default value handling for memStoreFlushSize
+   */
+  @Test
+  public void testGetMemStoreFlushSize() {
+    TableDescriptor desc = TableDescriptorBuilder
+            .newBuilder(TableName.valueOf(name.getMethodName())).build();
+    assertEquals(-1, desc.getMemStoreFlushSize());
+    desc = TableDescriptorBuilder
+            .newBuilder(TableName.valueOf(name.getMethodName()))
+            .setMemStoreFlushSize(1111L).build();
+    assertEquals(1111L, desc.getMemStoreFlushSize());
+  }
+
+  /**
+   * Test that we add and remove strings from configuration properly.
+   */
+  @Test
+  public void testAddGetRemoveConfiguration() {
+    String key = "Some";
+    String value = "value";
+    TableDescriptor desc = TableDescriptorBuilder
+            .newBuilder(TableName.valueOf(name.getMethodName()))
+            .setConfiguration(key, value)
+            .build();
+    assertEquals(value, desc.getConfigurationValue(key));
+    desc = TableDescriptorBuilder
+            .newBuilder(desc)
+            .removeConfiguration(key)
+            .build();
+    assertEquals(null, desc.getConfigurationValue(key));
+  }
+
+  @Test
+  public void testClassMethodsAreBuilderStyle() {
+    BuilderStyleTest.assertClassesAreBuilderStyle(TableDescriptorBuilder.class);
+  }
+
+  @Test
+  public void testModifyFamily() {
+    byte[] familyName = Bytes.toBytes("cf");
+    HColumnDescriptor hcd = new HColumnDescriptor(familyName);
+    hcd.setBlocksize(1000);
+    hcd.setDFSReplication((short) 3);
+    TableDescriptor htd
+      = TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+              .addFamily(hcd)
+              .build();
+
+    assertEquals(1000, htd.getFamily(familyName).getBlocksize());
+    assertEquals(3, htd.getFamily(familyName).getDFSReplication());
+    hcd = new HColumnDescriptor(familyName);
+    hcd.setBlocksize(2000);
+    hcd.setDFSReplication((short) 1);
+    htd = TableDescriptorBuilder.newBuilder(htd)
+              .modifyFamily(hcd)
+              .build();
+    assertEquals(2000, htd.getFamily(familyName).getBlocksize());
+    assertEquals(1, htd.getFamily(familyName).getDFSReplication());
+  }
+
+  @Test(expected=IllegalArgumentException.class)
+  public void testModifyInexistentFamily() {
+    byte[] familyName = Bytes.toBytes("cf");
+    HColumnDescriptor hcd = new HColumnDescriptor(familyName);
+    TableDescriptor htd = TableDescriptorBuilder
+            .newBuilder(TableName.valueOf(name.getMethodName()))
+            .modifyFamily(hcd)
+            .build();
+  }
+
+  @Test(expected=IllegalArgumentException.class)
+  public void testAddDuplicateFamilies() {
+    byte[] familyName = Bytes.toBytes("cf");
+    HColumnDescriptor hcd = new HColumnDescriptor(familyName);
+    hcd.setBlocksize(1000);
+    TableDescriptor htd = TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+            .addFamily(hcd)
+            .build();
+    assertEquals(1000, htd.getFamily(familyName).getBlocksize());
+    hcd = new HColumnDescriptor(familyName);
+    hcd.setBlocksize(2000);
+    // add duplicate column
+    TableDescriptorBuilder.newBuilder(htd).addFamily(hcd).build();
+  }
+
+  @Test
+  public void testPriority() {
+    TableDescriptor htd = TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+            .setPriority(42)
+            .build();
+    assertEquals(42, htd.getPriority());
+  }
+
+  @Test
+  public void testSerialReplicationScope() {
+    HColumnDescriptor hcdWithScope = new HColumnDescriptor(Bytes.toBytes("cf0"));
+    hcdWithScope.setScope(HConstants.REPLICATION_SCOPE_SERIAL);
+    HColumnDescriptor hcdWithoutScope = new HColumnDescriptor(Bytes.toBytes("cf1"));
+    TableDescriptor htd = TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+            .addFamily(hcdWithoutScope)
+            .build();
+    assertFalse(htd.hasSerialReplicationScope());
+
+    htd = TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+            .addFamily(hcdWithScope)
+            .build();
+    assertTrue(htd.hasSerialReplicationScope());
+
+    htd = TableDescriptorBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
+            .addFamily(hcdWithScope)
+            .addFamily(hcdWithoutScope)
+            .build();
+    assertTrue(htd.hasSerialReplicationScope());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/053e6154/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestUnmodifyableHTableDescriptor.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestUnmodifyableHTableDescriptor.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestUnmodifyableHTableDescriptor.java
deleted file mode 100644
index dca0c1f..0000000
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestUnmodifyableHTableDescriptor.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.client;
-
-import org.apache.hadoop.hbase.testclassification.ClientTests;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.apache.hadoop.hbase.util.BuilderStyleTest;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-@Category({ClientTests.class, SmallTests.class})
-public class TestUnmodifyableHTableDescriptor {
-
-    @Test
-    public void testClassMethodsAreBuilderStyle() {
-    /* UnmodifyableHTableDescriptor should have a builder style setup where setXXX/addXXX methods
-     * can be chainable together:
-     * . For example:
-     * UnmodifyableHTableDescriptor d
-     *   = new UnmodifyableHTableDescriptor()
-     *     .setFoo(foo)
-     *     .setBar(bar)
-     *     .setBuz(buz)
-     *
-     * This test ensures that all methods starting with "set" returns the declaring object
-     */
-
-        BuilderStyleTest.assertClassesAreBuilderStyle(UnmodifyableHTableDescriptor.class);
-    }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/053e6154/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/RowResourceBase.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/RowResourceBase.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/RowResourceBase.java
index b4b00a9..406b4e5 100644
--- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/RowResourceBase.java
+++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/RowResourceBase.java
@@ -111,7 +111,7 @@ public class RowResourceBase {
     if (admin.tableExists(TABLE_NAME)) {
       TEST_UTIL.deleteTable(TABLE_NAME);
     }
-    HTableDescriptor htd = new HTableDescriptor(TABLE);
+    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(TABLE));
     htd.addFamily(new HColumnDescriptor(CFA));
     htd.addFamily(new HColumnDescriptor(CFB));
     admin.createTable(htd);

http://git-wip-us.apache.org/repos/asf/hbase/blob/053e6154/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteAdminRetries.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteAdminRetries.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteAdminRetries.java
index 7c888e0..b926d82 100644
--- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteAdminRetries.java
+++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteAdminRetries.java
@@ -32,6 +32,7 @@ import java.util.regex.Pattern;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.testclassification.RestTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -116,7 +117,7 @@ public class TestRemoteAdminRetries {
     testTimedOutCall(new CallExecutor() {
       @Override
       public void run() throws Exception {
-        remoteAdmin.createTable(new HTableDescriptor(Bytes.toBytes("TestTable")));
+        remoteAdmin.createTable(new HTableDescriptor(TableName.valueOf("TestTable")));
       }
     });
     verify(client, times(RETRIES)).put(anyString(), anyString(), any(byte[].class));

http://git-wip-us.apache.org/repos/asf/hbase/blob/053e6154/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManager.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManager.java b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManager.java
index ab5c09f..e1fd82c 100644
--- a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManager.java
+++ b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManager.java
@@ -39,7 +39,6 @@ public interface RSGroupInfoManager {
   //Assigned before user tables
   TableName RSGROUP_TABLE_NAME =
       TableName.valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "rsgroup");
-  byte[] RSGROUP_TABLE_NAME_BYTES = RSGROUP_TABLE_NAME.toBytes();
   String rsGroupZNode = "rsgroup";
   byte[] META_FAMILY_BYTES = Bytes.toBytes("m");
   byte[] META_QUALIFIER_BYTES = Bytes.toBytes("i");

http://git-wip-us.apache.org/repos/asf/hbase/blob/053e6154/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java
index 9f77c77..f2c6118 100644
--- a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java
+++ b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java
@@ -117,7 +117,7 @@ class RSGroupInfoManagerImpl implements RSGroupInfoManager {
   /** Table descriptor for <code>hbase:rsgroup</code> catalog table */
   private final static HTableDescriptor RSGROUP_TABLE_DESC;
   static {
-    RSGROUP_TABLE_DESC = new HTableDescriptor(RSGROUP_TABLE_NAME_BYTES);
+    RSGROUP_TABLE_DESC = new HTableDescriptor(RSGROUP_TABLE_NAME);
     RSGROUP_TABLE_DESC.addFamily(new HColumnDescriptor(META_FAMILY_BYTES));
     RSGROUP_TABLE_DESC.setRegionSplitPolicyClassName(DisabledRegionSplitPolicy.class.getName());
     try {

http://git-wip-us.apache.org/repos/asf/hbase/blob/053e6154/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroups.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroups.java b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroups.java
index d6bd43b..6ef162b 100644
--- a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroups.java
+++ b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroups.java
@@ -189,7 +189,7 @@ public class TestRSGroups extends TestRSGroupsBase {
     final byte[] tableName = Bytes.toBytes(tablePrefix + "_testCreateAndAssign");
     admin.modifyNamespace(NamespaceDescriptor.create("default")
         .addConfiguration(RSGroupInfo.NAMESPACE_DESC_PROP_GROUP, "default").build());
-    final HTableDescriptor desc = new HTableDescriptor(tableName);
+    final HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
     desc.addFamily(new HColumnDescriptor("f"));
     admin.createTable(desc);
     //wait for created table to be assigned

http://git-wip-us.apache.org/repos/asf/hbase/blob/053e6154/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreTool.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreTool.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreTool.java
index d34701f..f744ecb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreTool.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreTool.java
@@ -176,7 +176,7 @@ public class RestoreTool {
         LOG.debug("Found descriptor " + tableDescriptor + " through " + incrBackupId);
 
         TableName newTableName = newTableNames[i];
-        HTableDescriptor newTableDescriptor = admin.getTableDescriptor(newTableName);
+        HTableDescriptor newTableDescriptor = new HTableDescriptor(admin.getTableDescriptor(newTableName));
         List<HColumnDescriptor> families = Arrays.asList(tableDescriptor.getColumnFamilies());
         List<HColumnDescriptor> existingFamilies =
             Arrays.asList(newTableDescriptor.getColumnFamilies());
@@ -325,7 +325,7 @@ public class RestoreTool {
           LOG.debug("find table descriptor but no archive dir for table " + tableName
               + ", will only create table");
         }
-        tableDescriptor.setName(newTableName);
+        tableDescriptor = new HTableDescriptor(newTableName, tableDescriptor);
         checkAndCreateTable(conn, tableBackupPath, tableName, newTableName, null, tableDescriptor,
           truncateIfExists);
         return;
@@ -338,7 +338,7 @@ public class RestoreTool {
     if (tableDescriptor == null) {
       tableDescriptor = new HTableDescriptor(newTableName);
     } else {
-      tableDescriptor.setName(newTableName);
+      tableDescriptor = new HTableDescriptor(newTableName, tableDescriptor);
     }
 
     // record all region dirs:

http://git-wip-us.apache.org/repos/asf/hbase/blob/053e6154/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java
index c2ca3eb..70fe5c5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java
@@ -48,6 +48,7 @@ import org.apache.hadoop.hbase.TableDescriptors;
 import org.apache.hadoop.hbase.TableInfoMissingException;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.regionserver.BloomType;
 
@@ -128,74 +129,69 @@ public class FSTableDescriptors implements TableDescriptors {
   @VisibleForTesting
   public static HTableDescriptor createMetaTableDescriptor(final Configuration conf)
       throws IOException {
-    HTableDescriptor metaDescriptor = new HTableDescriptor(
-        TableName.META_TABLE_NAME,
-        new HColumnDescriptor[] {
-            new HColumnDescriptor(HConstants.CATALOG_FAMILY)
-                .setMaxVersions(conf.getInt(HConstants.HBASE_META_VERSIONS,
+    return new HTableDescriptor(TableDescriptorBuilder.newBuilder(TableName.META_TABLE_NAME)
+          .addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)
+            .setMaxVersions(conf.getInt(HConstants.HBASE_META_VERSIONS,
                     HConstants.DEFAULT_HBASE_META_VERSIONS))
-                .setInMemory(true)
-                .setBlocksize(conf.getInt(HConstants.HBASE_META_BLOCK_SIZE,
+            .setInMemory(true)
+            .setBlocksize(conf.getInt(HConstants.HBASE_META_BLOCK_SIZE,
                     HConstants.DEFAULT_HBASE_META_BLOCK_SIZE))
-                .setScope(HConstants.REPLICATION_SCOPE_LOCAL)
-                    // Disable blooms for meta.  Needs work.  Seems to mess w/ getClosestOrBefore.
-                .setBloomFilterType(BloomType.NONE)
-                    // Enable cache of data blocks in L1 if more than one caching tier deployed:
-                    // e.g. if using CombinedBlockCache (BucketCache).
-                .setCacheDataInL1(true),
-            new HColumnDescriptor(HConstants.REPLICATION_BARRIER_FAMILY)
-                .setMaxVersions(conf.getInt(HConstants.HBASE_META_VERSIONS,
+            .setScope(HConstants.REPLICATION_SCOPE_LOCAL)
+            // Disable blooms for meta.  Needs work.  Seems to mess w/ getClosestOrBefore.
+            .setBloomFilterType(BloomType.NONE)
+            // Enable cache of data blocks in L1 if more than one caching tier deployed:
+            // e.g. if using CombinedBlockCache (BucketCache).
+            .setCacheDataInL1(true))
+          .addFamily(new HColumnDescriptor(HConstants.REPLICATION_BARRIER_FAMILY)
+            .setMaxVersions(conf.getInt(HConstants.HBASE_META_VERSIONS,
                     HConstants.DEFAULT_HBASE_META_VERSIONS))
-                .setInMemory(true)
-                .setBlocksize(conf.getInt(HConstants.HBASE_META_BLOCK_SIZE,
+            .setInMemory(true)
+            .setBlocksize(conf.getInt(HConstants.HBASE_META_BLOCK_SIZE,
                     HConstants.DEFAULT_HBASE_META_BLOCK_SIZE))
-                .setScope(HConstants.REPLICATION_SCOPE_LOCAL)
-                // Disable blooms for meta.  Needs work.  Seems to mess w/ getClosestOrBefore.
-                .setBloomFilterType(BloomType.NONE)
-                // Enable cache of data blocks in L1 if more than one caching tier deployed:
-                // e.g. if using CombinedBlockCache (BucketCache).
-                .setCacheDataInL1(true),
-            new HColumnDescriptor(HConstants.REPLICATION_POSITION_FAMILY)
-                .setMaxVersions(conf.getInt(HConstants.HBASE_META_VERSIONS,
+            .setScope(HConstants.REPLICATION_SCOPE_LOCAL)
+            // Disable blooms for meta.  Needs work.  Seems to mess w/ getClosestOrBefore.
+            .setBloomFilterType(BloomType.NONE)
+            // Enable cache of data blocks in L1 if more than one caching tier deployed:
+            // e.g. if using CombinedBlockCache (BucketCache).
+            .setCacheDataInL1(true))
+          .addFamily(new HColumnDescriptor(HConstants.REPLICATION_POSITION_FAMILY)
+            .setMaxVersions(conf.getInt(HConstants.HBASE_META_VERSIONS,
                     HConstants.DEFAULT_HBASE_META_VERSIONS))
-                .setInMemory(true)
-                .setBlocksize(conf.getInt(HConstants.HBASE_META_BLOCK_SIZE,
+            .setInMemory(true)
+            .setBlocksize(conf.getInt(HConstants.HBASE_META_BLOCK_SIZE,
                     HConstants.DEFAULT_HBASE_META_BLOCK_SIZE))
-                .setScope(HConstants.REPLICATION_SCOPE_LOCAL)
-                // Disable blooms for meta.  Needs work.  Seems to mess w/ getClosestOrBefore.
-                .setBloomFilterType(BloomType.NONE)
-                // Enable cache of data blocks in L1 if more than one caching tier deployed:
-                // e.g. if using CombinedBlockCache (BucketCache).
-                .setCacheDataInL1(true),
-            new HColumnDescriptor(HConstants.REPLICATION_META_FAMILY)
-                .setMaxVersions(conf.getInt(HConstants.HBASE_META_VERSIONS,
+            .setScope(HConstants.REPLICATION_SCOPE_LOCAL)
+            // Disable blooms for meta.  Needs work.  Seems to mess w/ getClosestOrBefore.
+            .setBloomFilterType(BloomType.NONE)
+            // Enable cache of data blocks in L1 if more than one caching tier deployed:
+            // e.g. if using CombinedBlockCache (BucketCache).
+            .setCacheDataInL1(true))
+          .addFamily(new HColumnDescriptor(HConstants.REPLICATION_META_FAMILY)
+            .setMaxVersions(conf.getInt(HConstants.HBASE_META_VERSIONS,
                     HConstants.DEFAULT_HBASE_META_VERSIONS))
-                .setInMemory(true)
-                .setBlocksize(conf.getInt(HConstants.HBASE_META_BLOCK_SIZE,
+            .setInMemory(true)
+            .setBlocksize(conf.getInt(HConstants.HBASE_META_BLOCK_SIZE,
                     HConstants.DEFAULT_HBASE_META_BLOCK_SIZE))
-                .setScope(HConstants.REPLICATION_SCOPE_LOCAL)
-                // Disable blooms for meta.  Needs work.  Seems to mess w/ getClosestOrBefore.
-                .setBloomFilterType(BloomType.NONE)
-                // Enable cache of data blocks in L1 if more than one caching tier deployed:
-                // e.g. if using CombinedBlockCache (BucketCache).
-                .setCacheDataInL1(true),
-            new HColumnDescriptor(HConstants.TABLE_FAMILY)
-                // Ten is arbitrary number.  Keep versions to help debugging.
-                .setMaxVersions(10)
-                .setInMemory(true)
-                .setBlocksize(8 * 1024)
-                .setScope(HConstants.REPLICATION_SCOPE_LOCAL)
-                    // Disable blooms for meta.  Needs work.  Seems to mess w/ getClosestOrBefore.
-                .setBloomFilterType(BloomType.NONE)
-                    // Enable cache of data blocks in L1 if more than one caching tier deployed:
-                    // e.g. if using CombinedBlockCache (BucketCache).
-                .setCacheDataInL1(true)
-        }) {
-    };
-    metaDescriptor.addCoprocessor(
-        "org.apache.hadoop.hbase.coprocessor.MultiRowMutationEndpoint",
-        null, Coprocessor.PRIORITY_SYSTEM, null);
-    return metaDescriptor;
+            .setScope(HConstants.REPLICATION_SCOPE_LOCAL)
+            // Disable blooms for meta.  Needs work.  Seems to mess w/ getClosestOrBefore.
+            .setBloomFilterType(BloomType.NONE)
+            // Enable cache of data blocks in L1 if more than one caching tier deployed:
+            // e.g. if using CombinedBlockCache (BucketCache).
+            .setCacheDataInL1(true))
+          .addFamily(new HColumnDescriptor(HConstants.TABLE_FAMILY)
+            // Ten is arbitrary number.  Keep versions to help debugging.
+            .setMaxVersions(10)
+            .setInMemory(true)
+            .setBlocksize(8 * 1024)
+            .setScope(HConstants.REPLICATION_SCOPE_LOCAL)
+            // Disable blooms for meta.  Needs work.  Seems to mess w/ getClosestOrBefore.
+            .setBloomFilterType(BloomType.NONE)
+            // Enable cache of data blocks in L1 if more than one caching tier deployed:
+            // e.g. if using CombinedBlockCache (BucketCache).
+            .setCacheDataInL1(true))
+          .addCoprocessor("org.apache.hadoop.hbase.coprocessor.MultiRowMutationEndpoint",
+            null, Coprocessor.PRIORITY_SYSTEM, null)
+          .build());
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/053e6154/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
index c0ddbfc..e0edfa3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
@@ -1726,7 +1726,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
   public static void setReplicas(Admin admin, TableName table, int replicaCount)
       throws IOException, InterruptedException {
     admin.disableTable(table);
-    HTableDescriptor desc = admin.getTableDescriptor(table);
+    HTableDescriptor desc = new HTableDescriptor(admin.getTableDescriptor(table));
     desc.setRegionReplication(replicaCount);
     admin.modifyTable(desc.getTableName(), desc);
     admin.enableTable(table);

http://git-wip-us.apache.org/repos/asf/hbase/blob/053e6154/hbase-server/src/test/java/org/apache/hadoop/hbase/TestFSTableDescriptorForceCreation.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestFSTableDescriptorForceCreation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestFSTableDescriptorForceCreation.java
index 7744631..7457f43 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestFSTableDescriptorForceCreation.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestFSTableDescriptorForceCreation.java
@@ -59,7 +59,7 @@ public class TestFSTableDescriptorForceCreation {
     // Cleanup old tests if any detritus laying around.
     Path rootdir = new Path(UTIL.getDataTestDir(), name);
     FSTableDescriptors fstd = new FSTableDescriptors(UTIL.getConfiguration(), fs, rootdir);
-    HTableDescriptor htd = new HTableDescriptor(name);
+    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name));
     fstd.add(htd);
     assertFalse("Should not create new table descriptor", fstd.createTableDescriptor(htd, false));
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/053e6154/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java
index b7430fe..f6dbb41 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java
@@ -92,7 +92,7 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
       TEST_UTIL.createTable(tables[i], FAMILY);
     }
 
-    HTableDescriptor[] tableDescs = admin.listTables().get();
+    TableDescriptor[] tableDescs = admin.listTables().get();
     int size = tableDescs.length;
     assertTrue(size >= tables.length);
     for (int i = 0; i < tables.length && i < size; i++) {
@@ -140,13 +140,13 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
     htd.addFamily(fam2);
     htd.addFamily(fam3);
     admin.createTable(htd).join();
-    HTableDescriptor confirmedHtd = admin.getTableDescriptor(htd.getTableName()).get();
-    assertEquals(htd.compareTo(confirmedHtd), 0);
+    TableDescriptor confirmedHtd = admin.getTableDescriptor(htd.getTableName()).get();
+    assertEquals(htd.compareTo(new HTableDescriptor(confirmedHtd)), 0);
   }
 
   @Test(timeout = 300000)
   public void testCreateTable() throws Exception {
-    HTableDescriptor[] tables = admin.listTables().get();
+    TableDescriptor[] tables = admin.listTables().get();
     int numTables = tables.length;
     final  TableName tableName = TableName.valueOf(name.getMethodName());
     admin.createTable(new HTableDescriptor(tableName).addFamily(new HColumnDescriptor(FAMILY)))
@@ -452,7 +452,7 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
           } catch (Exception e) {
           }
         });
-    HTableDescriptor[] failed = admin.deleteTables(Pattern.compile("testDeleteTables.*")).get();
+    TableDescriptor[] failed = admin.deleteTables(Pattern.compile("testDeleteTables.*")).get();
     assertEquals(0, failed.length);
     Arrays.stream(tables).forEach((table) -> {
       admin.tableExists(table).thenAccept((exist) -> assertFalse(exist)).join();
@@ -727,7 +727,7 @@ public class TestAsyncTableAdminApi extends TestAsyncAdminBase {
       // Modify colymn family
       admin.modifyColumnFamily(tableName, cfDescriptor).join();
 
-      HTableDescriptor htd = admin.getTableDescriptor(tableName).get();
+      TableDescriptor htd = admin.getTableDescriptor(tableName).get();
       HColumnDescriptor hcfd = htd.getFamily(FAMILY_0);
       assertTrue(hcfd.getBlocksize() == newBlockSize);
     } finally {

http://git-wip-us.apache.org/repos/asf/hbase/blob/053e6154/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableBatch.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableBatch.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableBatch.java
index 30b3b5b..61ff2be 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableBatch.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableBatch.java
@@ -216,7 +216,7 @@ public class TestAsyncTableBatch {
   @Test
   public void testPartialSuccess() throws IOException, InterruptedException, ExecutionException {
     Admin admin = TEST_UTIL.getAdmin();
-    HTableDescriptor htd = admin.getTableDescriptor(TABLE_NAME);
+    HTableDescriptor htd = new HTableDescriptor(admin.getTableDescriptor(TABLE_NAME));
     htd.addCoprocessor(ErrorInjectObserver.class.getName());
     admin.modifyTable(TABLE_NAME, htd);
     AsyncTableBase table = tableGetter.apply(TABLE_NAME);

http://git-wip-us.apache.org/repos/asf/hbase/blob/053e6154/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdminWithClusters.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdminWithClusters.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdminWithClusters.java
index 56f4141..f46354c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdminWithClusters.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdminWithClusters.java
@@ -77,7 +77,7 @@ public class TestReplicationAdminWithClusters extends TestReplicationBase {
 
   @Test(timeout = 300000)
   public void disableNotFullReplication() throws Exception {
-    HTableDescriptor table = admin2.getTableDescriptor(tableName);
+    HTableDescriptor table = new HTableDescriptor(admin2.getTableDescriptor(tableName));
     HColumnDescriptor f = new HColumnDescriptor("notReplicatedFamily");
     table.addFamily(f);
     admin1.disableTable(tableName);
@@ -125,7 +125,7 @@ public class TestReplicationAdminWithClusters extends TestReplicationBase {
 
   @Test(timeout = 300000)
   public void testEnableReplicationWhenTableDescriptorIsNotSameInClusters() throws Exception {
-    HTableDescriptor table = admin2.getTableDescriptor(tableName);
+    HTableDescriptor table = new HTableDescriptor(admin2.getTableDescriptor(tableName));
     HColumnDescriptor f = new HColumnDescriptor("newFamily");
     table.addFamily(f);
     admin2.disableTable(tableName);

http://git-wip-us.apache.org/repos/asf/hbase/blob/053e6154/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizerOnCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizerOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizerOnCluster.java
index 19df42a..70a78de 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizerOnCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizerOnCluster.java
@@ -140,7 +140,7 @@ public class TestSimpleRegionNormalizerOnCluster {
       region.flush(true);
     }
 
-    HTableDescriptor htd = admin.getTableDescriptor(TABLENAME);
+    HTableDescriptor htd = new HTableDescriptor(admin.getTableDescriptor(TABLENAME));
     htd.setNormalizationEnabled(true);
     admin.modifyTable(TABLENAME, htd);
 
@@ -217,7 +217,7 @@ public class TestSimpleRegionNormalizerOnCluster {
       region.flush(true);
     }
 
-    HTableDescriptor htd = admin.getTableDescriptor(tableName);
+    HTableDescriptor htd = new HTableDescriptor(admin.getTableDescriptor(tableName));
     htd.setNormalizationEnabled(true);
     admin.modifyTable(tableName, htd);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/053e6154/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionKeyRotation.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionKeyRotation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionKeyRotation.java
index 3e7477d..4f6d4c2 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionKeyRotation.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionKeyRotation.java
@@ -195,7 +195,7 @@ public class TestEncryptionKeyRotation {
     // Start the cluster back up
     TEST_UTIL.startMiniHBaseCluster(1, 1);
     // Verify the table can still be loaded
-    TEST_UTIL.waitTableAvailable(htd.getName(), 5000);
+    TEST_UTIL.waitTableAvailable(htd.getTableName(), 5000);
     // Double check that the store file keys can be unwrapped
     storeFilePaths = findStorefilePaths(htd.getTableName());
     assertTrue(storeFilePaths.size() > 0);
@@ -266,7 +266,7 @@ public class TestEncryptionKeyRotation {
     HColumnDescriptor hcd = htd.getFamilies().iterator().next();
     // Create the test table
     TEST_UTIL.getAdmin().createTable(htd);
-    TEST_UTIL.waitTableAvailable(htd.getName(), 5000);
+    TEST_UTIL.waitTableAvailable(htd.getTableName(), 5000);
     // Create a store file
     Table table = TEST_UTIL.getConnection().getTable(htd.getTableName());
     try {

http://git-wip-us.apache.org/repos/asf/hbase/blob/053e6154/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionRandomKeying.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionRandomKeying.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionRandomKeying.java
index 3d8eeed..cd1f1e7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionRandomKeying.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionRandomKeying.java
@@ -101,7 +101,7 @@ public class TestEncryptionRandomKeying {
 
     // Create the test table
     TEST_UTIL.getAdmin().createTable(htd);
-    TEST_UTIL.waitTableAvailable(htd.getName(), 5000);
+    TEST_UTIL.waitTableAvailable(htd.getTableName(), 5000);
 
     // Create a store file
     Table table = TEST_UTIL.getConnection().getTable(htd.getTableName());

http://git-wip-us.apache.org/repos/asf/hbase/blob/053e6154/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
index 095f4bd..53f0892 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
@@ -5734,7 +5734,7 @@ public class TestHRegion {
       LOG.info("RegionEventDescriptor from WAL: " + desc);
 
       assertEquals(RegionEventDescriptor.EventType.REGION_OPEN, desc.getEventType());
-      assertTrue(Bytes.equals(desc.getTableName().toByteArray(), htd.getName()));
+      assertTrue(Bytes.equals(desc.getTableName().toByteArray(), htd.getTableName().toBytes()));
       assertTrue(Bytes.equals(desc.getEncodedRegionName().toByteArray(),
         hri.getEncodedNameAsBytes()));
       assertTrue(desc.getLogSequenceNumber() > 0);
@@ -5854,7 +5854,7 @@ public class TestHRegion {
       LOG.info("RegionEventDescriptor from WAL: " + desc);
 
       assertEquals(RegionEventDescriptor.EventType.REGION_OPEN, desc.getEventType());
-      assertTrue(Bytes.equals(desc.getTableName().toByteArray(), htd.getName()));
+      assertTrue(Bytes.equals(desc.getTableName().toByteArray(), htd.getTableName().toBytes()));
       assertTrue(Bytes.equals(desc.getEncodedRegionName().toByteArray(),
         hri.getEncodedNameAsBytes()));
       assertTrue(desc.getLogSequenceNumber() > 0);
@@ -5940,7 +5940,7 @@ public class TestHRegion {
     LOG.info("RegionEventDescriptor from WAL: " + desc);
 
     assertEquals(RegionEventDescriptor.EventType.REGION_CLOSE, desc.getEventType());
-    assertTrue(Bytes.equals(desc.getTableName().toByteArray(), htd.getName()));
+    assertTrue(Bytes.equals(desc.getTableName().toByteArray(), htd.getTableName().toBytes()));
     assertTrue(Bytes.equals(desc.getEncodedRegionName().toByteArray(),
       hri.getEncodedNameAsBytes()));
     assertTrue(desc.getLogSequenceNumber() > 0);

http://git-wip-us.apache.org/repos/asf/hbase/blob/053e6154/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCoprocessorWhitelistMasterObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCoprocessorWhitelistMasterObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCoprocessorWhitelistMasterObserver.java
index aeb909e..bc74f4f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCoprocessorWhitelistMasterObserver.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestCoprocessorWhitelistMasterObserver.java
@@ -107,7 +107,7 @@ public class TestCoprocessorWhitelistMasterObserver extends SecureTestUtil {
     UTIL.waitUntilAllRegionsAssigned(TEST_TABLE);
     Connection connection = ConnectionFactory.createConnection(conf);
     Table t = connection.getTable(TEST_TABLE);
-    HTableDescriptor htd = t.getTableDescriptor();
+    HTableDescriptor htd = new HTableDescriptor(t.getTableDescriptor());
     htd.addCoprocessor("net.clayb.hbase.coprocessor.NotWhitelisted",
       new Path(coprocessorPath),
       Coprocessor.PRIORITY_USER, null);
@@ -153,7 +153,7 @@ public class TestCoprocessorWhitelistMasterObserver extends SecureTestUtil {
     // coprocessor file
     admin.disableTable(TEST_TABLE);
     Table t = connection.getTable(TEST_TABLE);
-    HTableDescriptor htd = t.getTableDescriptor();
+    HTableDescriptor htd = new HTableDescriptor(t.getTableDescriptor());
     htd.addCoprocessor("net.clayb.hbase.coprocessor.Whitelisted",
       new Path(coprocessorPath),
       Coprocessor.PRIORITY_USER, null);

http://git-wip-us.apache.org/repos/asf/hbase/blob/053e6154/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/MobSnapshotTestingUtils.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/MobSnapshotTestingUtils.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/MobSnapshotTestingUtils.java
index d301214..3e1abb9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/MobSnapshotTestingUtils.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/MobSnapshotTestingUtils.java
@@ -147,7 +147,7 @@ public class MobSnapshotTestingUtils {
 
     @Override
     public HTableDescriptor createHtd(final String tableName) {
-      HTableDescriptor htd = new HTableDescriptor(tableName);
+      HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName));
       HColumnDescriptor hcd = new HColumnDescriptor(TEST_FAMILY);
       hcd.setMobEnabled(true);
       hcd.setMobThreshold(0L);

http://git-wip-us.apache.org/repos/asf/hbase/blob/053e6154/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java
index ccad85b..3c10dee 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java
@@ -680,7 +680,7 @@ public final class SnapshotTestingUtils {
     }
 
     public HTableDescriptor createHtd(final String tableName) {
-      HTableDescriptor htd = new HTableDescriptor(tableName);
+      HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName));
       htd.addFamily(new HColumnDescriptor(TEST_FAMILY));
       return htd;
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/053e6154/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSTableDescriptors.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSTableDescriptors.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSTableDescriptors.java
index ee048b7..8337eb0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSTableDescriptors.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSTableDescriptors.java
@@ -261,7 +261,7 @@ public class TestFSTableDescriptors {
     final int count = 10;
     // Write out table infos.
     for (int i = 0; i < count; i++) {
-      HTableDescriptor htd = new HTableDescriptor(name + i);
+      HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name + i));
       htds.createTableDescriptor(htd);
     }
 
@@ -294,7 +294,7 @@ public class TestFSTableDescriptors {
     final int count = 4;
     // Write out table infos.
     for (int i = 0; i < count; i++) {
-      HTableDescriptor htd = new HTableDescriptor(name + i);
+      HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name + i));
       htds.createTableDescriptor(htd);
     }
     // add hbase:meta
@@ -321,7 +321,7 @@ public class TestFSTableDescriptors {
     final int count = 10;
     // Write out table infos via non-cached FSTableDescriptors
     for (int i = 0; i < count; i++) {
-      HTableDescriptor htd = new HTableDescriptor(name + i);
+      HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name + i));
       nonchtds.createTableDescriptor(htd);
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/053e6154/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckEncryption.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckEncryption.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckEncryption.java
index 17ab004..c1d5778 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckEncryption.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckEncryption.java
@@ -92,7 +92,7 @@ public class TestHBaseFsckEncryption {
       cfKey));
     htd.addFamily(hcd);
     TEST_UTIL.getAdmin().createTable(htd);
-    TEST_UTIL.waitTableAvailable(htd.getName(), 5000);
+    TEST_UTIL.waitTableAvailable(htd.getTableName(), 5000);
   }
 
   @After

http://git-wip-us.apache.org/repos/asf/hbase/blob/053e6154/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckOneRS.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckOneRS.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckOneRS.java
index 1d09dfa..74ef414 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckOneRS.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckOneRS.java
@@ -248,7 +248,7 @@ public class TestHBaseFsckOneRS extends BaseTestHBaseFsck {
       status = FSTableDescriptors.getTableInfoPath(fs, hbaseTableDir);
       assertNotNull(status);
 
-      HTableDescriptor htd = admin.getTableDescriptor(tableName);
+      HTableDescriptor htd = new HTableDescriptor(admin.getTableDescriptor(tableName));
       htd.setValue("NOT_DEFAULT", "true");
       admin.disableTable(tableName);
       admin.modifyTable(tableName, htd);

http://git-wip-us.apache.org/repos/asf/hbase/blob/053e6154/hbase-shell/src/main/ruby/hbase/admin.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/hbase/admin.rb b/hbase-shell/src/main/ruby/hbase/admin.rb
index 41904be..2db89b7 100644
--- a/hbase-shell/src/main/ruby/hbase/admin.rb
+++ b/hbase-shell/src/main/ruby/hbase/admin.rb
@@ -589,7 +589,7 @@ module Hbase
       table_name = TableName.valueOf(table_name_str)
 
       # Get table descriptor
-      htd = @admin.getTableDescriptor(table_name)
+      htd = org.apache.hadoop.hbase.HTableDescriptor.new(@admin.getTableDescriptor(table_name))
       hasTableUpdate = false
 
       # Process all args
@@ -1206,15 +1206,6 @@ module Hbase
       htd.setNormalizationEnabled(
         JBoolean.valueOf(arg.delete(NORMALIZATION_ENABLED))) if arg[NORMALIZATION_ENABLED]
       htd.setMemStoreFlushSize(JLong.valueOf(arg.delete(MEMSTORE_FLUSHSIZE))) if arg[MEMSTORE_FLUSHSIZE]
-      # DEFERRED_LOG_FLUSH is deprecated and was replaced by DURABILITY.  To keep backward compatible, it still exists.
-      # However, it has to be set before DURABILITY so that DURABILITY could overwrite if both args are set
-      if arg.include?(DEFERRED_LOG_FLUSH)
-        if arg.delete(DEFERRED_LOG_FLUSH).to_s.upcase == "TRUE"
-          htd.setDurability(org.apache.hadoop.hbase.client.Durability.valueOf("ASYNC_WAL"))
-        else
-          htd.setDurability(org.apache.hadoop.hbase.client.Durability.valueOf("SYNC_WAL"))
-        end
-      end
       htd.setDurability(org.apache.hadoop.hbase.client.Durability.valueOf(arg.delete(DURABILITY))) if arg[DURABILITY]
       htd.setPriority(JInteger.valueOf(arg.delete(org.apache.hadoop.hbase.HTableDescriptor::PRIORITY))) if arg[org.apache.hadoop.hbase.HTableDescriptor::PRIORITY]
       htd.setFlushPolicyClassName(arg.delete(org.apache.hadoop.hbase.HTableDescriptor::FLUSH_POLICY)) if arg[org.apache.hadoop.hbase.HTableDescriptor::FLUSH_POLICY]

http://git-wip-us.apache.org/repos/asf/hbase/blob/053e6154/hbase-shell/src/main/ruby/shell/commands/alter_async.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands/alter_async.rb b/hbase-shell/src/main/ruby/shell/commands/alter_async.rb
index e0f6deb..05335fb 100644
--- a/hbase-shell/src/main/ruby/shell/commands/alter_async.rb
+++ b/hbase-shell/src/main/ruby/shell/commands/alter_async.rb
@@ -40,8 +40,8 @@ or a shorter version:
 
   hbase> alter_async 'ns1:t1', 'delete' => 'f1'
 
-You can also change table-scope attributes like MAX_FILESIZE
-MEMSTORE_FLUSHSIZE, READONLY, and DEFERRED_LOG_FLUSH.
+You can also change table-scope attributes like MAX_FILESIZE,
+MEMSTORE_FLUSHSIZE, and READONLY.
 
 For example, to change the max size of a family to 128MB, do: