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 2018/01/11 08:01:11 UTC

[1/2] hbase git commit: HBASE-19751 Use RegionInfo directly instead of an identifier and a namespace when getting WAL

Repository: hbase
Updated Branches:
  refs/heads/master 62a818894 -> 71a1192d6


http://git-wip-us.apache.org/repos/asf/hbase/blob/71a1192d/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java
index 057b9bf..68fa33f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java
@@ -15,7 +15,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.hadoop.hbase.regionserver.wal;
 
 import static org.junit.Assert.assertEquals;
@@ -23,20 +22,21 @@ 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.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.Increment;
 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.Result;
+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;
 import org.apache.hadoop.hbase.regionserver.MemStoreLABImpl;
@@ -52,8 +52,10 @@ import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized.Parameter;
@@ -63,7 +65,7 @@ import org.junit.runners.Parameterized.Parameters;
  * Tests for WAL write durability
  */
 @RunWith(Parameterized.class)
-@Category({RegionServerTests.class, MediumTests.class})
+@Category({ RegionServerTests.class, MediumTests.class })
 public class TestDurability {
   private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
   private static FileSystem FS;
@@ -78,6 +80,9 @@ public class TestDurability {
   @Parameter
   public String walProvider;
 
+  @Rule
+  public TestName name = new TestName();
+
   @Parameters(name = "{index}: provider={0}")
   public static Iterable<Object[]> data() {
     return Arrays.asList(new Object[] { "defaultProvider" }, new Object[] { "asyncfs" });
@@ -111,12 +116,12 @@ public class TestDurability {
 
   @Test
   public void testDurability() throws Exception {
-    final WALFactory wals = new WALFactory(CONF, null, ServerName.valueOf("TestDurability",
-        16010, System.currentTimeMillis()).toString());
-    byte[] tableName = Bytes.toBytes("TestDurability");
-    final WAL wal = wals.getWAL(tableName, null);
-    HRegion region = createHRegion(tableName, "region", wal, Durability.USE_DEFAULT);
-    HRegion deferredRegion = createHRegion(tableName, "deferredRegion", wal, Durability.ASYNC_WAL);
+    WALFactory wals = new WALFactory(CONF, null,
+        ServerName.valueOf("TestDurability", 16010, System.currentTimeMillis()).toString());
+    HRegion region = createHRegion(wals, Durability.USE_DEFAULT);
+    WAL wal = region.getWAL();
+    HRegion deferredRegion = createHRegion(region.getTableDescriptor(), region.getRegionInfo(),
+      "deferredRegion", wal, Durability.ASYNC_WAL);
 
     region.put(newPut(null));
     verifyWALCount(wals, wal, 1);
@@ -175,11 +180,10 @@ public class TestDurability {
     byte[] col3 = Bytes.toBytes("col3");
 
     // Setting up region
-    final WALFactory wals = new WALFactory(CONF, null,
+    WALFactory wals = new WALFactory(CONF, null,
         ServerName.valueOf("TestIncrement", 16010, System.currentTimeMillis()).toString());
-    byte[] tableName = Bytes.toBytes("TestIncrement");
-    final WAL wal = wals.getWAL(tableName, null);
-    HRegion region = createHRegion(tableName, "increment", wal, Durability.USE_DEFAULT);
+    HRegion region = createHRegion(wals, Durability.USE_DEFAULT);
+    WAL wal = region.getWAL();
 
     // col1: amount = 0, 1 write back to WAL
     Increment inc1 = new Increment(row1);
@@ -231,8 +235,8 @@ public class TestDurability {
     assertEquals(3, Bytes.toLong(res.getValue(FAMILY, col3)));
     verifyWALCount(wals, wal, 4);
   }
-  
-  /*
+
+  /**
    * Test when returnResults set to false in increment it should not return the result instead it
    * resturn null.
    */
@@ -242,12 +246,11 @@ public class TestDurability {
     byte[] col1 = Bytes.toBytes("col1");
 
     // Setting up region
-    final WALFactory wals = new WALFactory(CONF, null,
-        ServerName.valueOf("testIncrementWithReturnResultsSetToFalse", 16010,
-            System.currentTimeMillis()).toString());
-    byte[] tableName = Bytes.toBytes("testIncrementWithReturnResultsSetToFalse");
-    final WAL wal = wals.getWAL(tableName, null);
-    HRegion region = createHRegion(tableName, "increment", wal, Durability.USE_DEFAULT);
+    WALFactory wals = new WALFactory(CONF, null,
+        ServerName
+            .valueOf("testIncrementWithReturnResultsSetToFalse", 16010, System.currentTimeMillis())
+            .toString());
+    HRegion region = createHRegion(wals, Durability.USE_DEFAULT);
 
     Increment inc1 = new Increment(row1);
     inc1.setReturnResults(false);
@@ -270,28 +273,38 @@ public class TestDurability {
     WAL.Reader reader = wals.createReader(FS, walPath);
     int count = 0;
     WAL.Entry entry = new WAL.Entry();
-    while (reader.next(entry) != null) count++;
+    while (reader.next(entry) != null) {
+      count++;
+    }
     reader.close();
     assertEquals(expected, count);
   }
 
   // lifted from TestAtomicOperation
-  private HRegion createHRegion (byte [] tableName, String callingMethod,
-      WAL log, Durability durability)
-    throws IOException {
-      HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName));
-      htd.setDurability(durability);
-      HColumnDescriptor hcd = new HColumnDescriptor(FAMILY);
-      htd.addFamily(hcd);
-      HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false);
-      Path path = new Path(DIR + callingMethod);
-      if (FS.exists(path)) {
-        if (!FS.delete(path, true)) {
-          throw new IOException("Failed delete of " + path);
-        }
+  private HRegion createHRegion(WALFactory wals, Durability durability) throws IOException {
+    TableName tableName = TableName.valueOf(name.getMethodName().replaceAll("[^A-Za-z0-9-_]", "_"));
+    TableDescriptor htd = TableDescriptorBuilder.newBuilder(tableName)
+        .addColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY)).build();
+    RegionInfo info = RegionInfoBuilder.newBuilder(tableName).build();
+    Path path = new Path(DIR, tableName.getNameAsString());
+    if (FS.exists(path)) {
+      if (!FS.delete(path, true)) {
+        throw new IOException("Failed delete of " + path);
       }
-      ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
-      return HRegion.createHRegion(info, path, CONF, htd, log);
     }
+    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
+    return HRegion.createHRegion(info, path, CONF, htd, wals.getWAL(info));
+  }
 
+  private HRegion createHRegion(TableDescriptor td, RegionInfo info, String dir, WAL wal,
+      Durability durability) throws IOException {
+    Path path = new Path(DIR, dir);
+    if (FS.exists(path)) {
+      if (!FS.delete(path, true)) {
+        throw new IOException("Failed delete of " + path);
+      }
+    }
+    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
+    return HRegion.createHRegion(info, path, CONF, td, wal);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/71a1192d/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java
index 6c2fd9f..57679c4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java
@@ -21,22 +21,23 @@ import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.util.NavigableMap;
 import java.util.TreeMap;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.MiniHBaseCluster;
 import org.apache.hadoop.hbase.ServerName;
 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.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
 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.log.HBaseMarkers;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
@@ -136,11 +137,11 @@ public class TestLogRollAbort {
 
     // Create the test table and open it
     TableName tableName = TableName.valueOf(this.getClass().getSimpleName());
-    HTableDescriptor desc = new HTableDescriptor(tableName);
-    desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
+    TableDescriptor desc = TableDescriptorBuilder.newBuilder(tableName)
+        .addColumnFamily(ColumnFamilyDescriptorBuilder.of(HConstants.CATALOG_FAMILY)).build();
 
     admin.createTable(desc);
-    Table table = TEST_UTIL.getConnection().getTable(desc.getTableName());
+    Table table = TEST_UTIL.getConnection().getTable(tableName);
     try {
       HRegionServer server = TEST_UTIL.getRSForFirstRegionInTable(tableName);
       WAL log = server.getWAL(null);
@@ -189,32 +190,26 @@ public class TestLogRollAbort {
       // put some entries in an WAL
       TableName tableName =
           TableName.valueOf(this.getClass().getName());
-      HRegionInfo regioninfo = new HRegionInfo(tableName,
-          HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW);
-      final WAL log = wals.getWAL(regioninfo.getEncodedNameAsBytes(),
-          regioninfo.getTable().getNamespace());
+      RegionInfo regionInfo = RegionInfoBuilder.newBuilder(tableName).build();
+      WAL log = wals.getWAL(regionInfo);
       MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl(1);
 
-      final int total = 20;
+      int total = 20;
       for (int i = 0; i < total; i++) {
         WALEdit kvs = new WALEdit();
         kvs.add(new KeyValue(Bytes.toBytes(i), tableName.getName(), tableName.getName()));
-        HTableDescriptor htd = new HTableDescriptor(tableName);
-        htd.addFamily(new HColumnDescriptor("column"));
         NavigableMap<byte[], Integer> scopes = new TreeMap<>(Bytes.BYTES_COMPARATOR);
-        for(byte[] fam : htd.getFamiliesKeys()) {
-          scopes.put(fam, 0);
-        }
-        log.append(regioninfo, new WALKeyImpl(regioninfo.getEncodedNameAsBytes(), tableName,
-            System.currentTimeMillis(), mvcc, scopes), kvs, true);
+        scopes.put(Bytes.toBytes("column"), 0);
+        log.append(regionInfo, new WALKeyImpl(regionInfo.getEncodedNameAsBytes(), tableName,
+            System.currentTimeMillis(), mvcc, scopes),
+          kvs, true);
       }
       // Send the data to HDFS datanodes and close the HDFS writer
       log.sync();
       ((AbstractFSWAL<?>) log).replaceWriter(((FSHLog)log).getOldPath(), null, null);
 
-      /* code taken from MasterFileSystem.getLogDirs(), which is called from MasterFileSystem.splitLog()
-       * handles RS shutdowns (as observed by the splitting process)
-       */
+      // code taken from MasterFileSystem.getLogDirs(), which is called from
+      // MasterFileSystem.splitLog() handles RS shutdowns (as observed by the splitting process)
       // rename the directory so a rogue RS doesn't create more WALs
       Path rsSplitDir = thisTestsDir.suffix(AbstractFSWALProvider.SPLITTING_EXT);
       if (!fs.rename(thisTestsDir, rsSplitDir)) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/71a1192d/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java
index 0549100..2766b31 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java
@@ -18,20 +18,21 @@
 package org.apache.hadoop.hbase.regionserver.wal;
 
 import static org.junit.Assert.assertFalse;
+
 import java.io.IOException;
 import java.util.NavigableMap;
 import java.util.TreeMap;
 import java.util.concurrent.ThreadLocalRandom;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.CategoryBasedTimeout;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.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.regionserver.MultiVersionConcurrencyControl;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
@@ -89,8 +90,8 @@ public class TestLogRollingNoCluster {
     FSUtils.setRootDir(conf, dir);
     conf.set("hbase.regionserver.hlog.writer.impl", HighLatencySyncWriter.class.getName());
     final WALFactory wals = new WALFactory(conf, null, TestLogRollingNoCluster.class.getName());
-    final WAL wal = wals.getWAL(new byte[]{}, null);
-    
+    final WAL wal = wals.getWAL(null);
+
     Appender [] appenders = null;
 
     final int numThreads = NUM_THREADS;
@@ -157,10 +158,10 @@ public class TestLogRollingNoCluster {
           WALEdit edit = new WALEdit();
           byte[] bytes = Bytes.toBytes(i);
           edit.add(new KeyValue(bytes, bytes, bytes, now, EMPTY_1K_ARRAY));
-          final HRegionInfo hri = HRegionInfo.FIRST_META_REGIONINFO;
-          final HTableDescriptor htd = TEST_UTIL.getMetaTableDescriptor();
+          RegionInfo hri = RegionInfoBuilder.FIRST_META_REGIONINFO;
+          TableDescriptor htd = TEST_UTIL.getMetaTableDescriptorBuilder().build();
           NavigableMap<byte[], Integer> scopes = new TreeMap<>(Bytes.BYTES_COMPARATOR);
-          for(byte[] fam : htd.getFamiliesKeys()) {
+          for(byte[] fam : htd.getColumnFamilyNames()) {
             scopes.put(fam, 0);
           }
           final long txid = wal.append(hri, new WALKeyImpl(hri.getEncodedNameAsBytes(),

http://git-wip-us.apache.org/repos/asf/hbase/blob/71a1192d/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java
index 6696ce3..380ea74 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java
@@ -24,17 +24,15 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.NavigableMap;
 import java.util.TreeMap;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
 import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
@@ -100,23 +98,20 @@ public class TestWALActionsListener {
     list.add(observer);
     final WALFactory wals = new WALFactory(conf, list, "testActionListener");
     DummyWALActionsListener laterobserver = new DummyWALActionsListener();
-    HRegionInfo hri = new HRegionInfo(TableName.valueOf(SOME_BYTES),
-             SOME_BYTES, SOME_BYTES, false);
-    final WAL wal = wals.getWAL(hri.getEncodedNameAsBytes(), hri.getTable().getNamespace());
+    RegionInfo hri = RegionInfoBuilder.newBuilder(TableName.valueOf(SOME_BYTES))
+        .setStartKey(SOME_BYTES).setEndKey(SOME_BYTES).build();
+    final WAL wal = wals.getWAL(hri);
     MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl();
     for (int i = 0; i < 20; i++) {
       byte[] b = Bytes.toBytes(i + "");
-      KeyValue kv = new KeyValue(b,b,b);
+      KeyValue kv = new KeyValue(b, b, b);
       WALEdit edit = new WALEdit();
       edit.add(kv);
-      HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(SOME_BYTES));
-      htd.addFamily(new HColumnDescriptor(b));
       NavigableMap<byte[], Integer> scopes = new TreeMap<>(Bytes.BYTES_COMPARATOR);
-      for(byte[] fam : htd.getFamiliesKeys()) {
-        scopes.put(fam, 0);
-      }
-      final long txid = wal.append(hri, new WALKeyImpl(hri.getEncodedNameAsBytes(),
-          TableName.valueOf(b), 0, mvcc, scopes), edit, true);
+      scopes.put(b, 0);
+      long txid = wal.append(hri,
+        new WALKeyImpl(hri.getEncodedNameAsBytes(), TableName.valueOf(b), 0, mvcc, scopes), edit,
+        true);
       wal.sync(txid);
       if (i == 10) {
         wal.registerWALActionsListener(laterobserver);

http://git-wip-us.apache.org/repos/asf/hbase/blob/71a1192d/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
index 3a49385..33216cb 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
@@ -267,7 +267,7 @@ public abstract class TestReplicationSourceManager {
     listeners.add(replication);
     final WALFactory wals = new WALFactory(utility.getConfiguration(), listeners,
         URLEncoder.encode("regionserver:60020", "UTF8"));
-    final WAL wal = wals.getWAL(hri.getEncodedNameAsBytes(), hri.getTable().getNamespace());
+    final WAL wal = wals.getWAL(hri);
     manager.init();
     HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("tableame"));
     htd.addFamily(new HColumnDescriptor(f1));

http://git-wip-us.apache.org/repos/asf/hbase/blob/71a1192d/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntryStream.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntryStream.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntryStream.java
index 6f9c2d3..43140bc 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntryStream.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntryStream.java
@@ -1,5 +1,4 @@
-/*
- *
+/**
  * 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
@@ -35,16 +34,16 @@ import java.util.OptionalLong;
 import java.util.TreeMap;
 import java.util.concurrent.PriorityBlockingQueue;
 import java.util.concurrent.atomic.AtomicLong;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.KeyValue;
 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.regionserver.MultiVersionConcurrencyControl;
 import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
 import org.apache.hadoop.hbase.replication.WALEntryFilter;
@@ -78,8 +77,8 @@ public class TestWALEntryStream {
   private static final TableName tableName = TableName.valueOf("tablename");
   private static final byte[] family = Bytes.toBytes("column");
   private static final byte[] qualifier = Bytes.toBytes("qualifier");
-  private static final HRegionInfo info =
-      new HRegionInfo(tableName, HConstants.EMPTY_START_ROW, HConstants.LAST_ROW, false);
+  private static final RegionInfo info = RegionInfoBuilder.newBuilder(tableName)
+      .setStartKey(HConstants.EMPTY_START_ROW).setEndKey(HConstants.LAST_ROW).build();
   private static final NavigableMap<byte[], Integer> scopes = getScopes();
 
   private static NavigableMap<byte[], Integer> getScopes() {
@@ -118,7 +117,7 @@ public class TestWALEntryStream {
     pathWatcher = new PathWatcher();
     listeners.add(pathWatcher);
     final WALFactory wals = new WALFactory(conf, listeners, tn.getMethodName());
-    log = wals.getWAL(info.getEncodedNameAsBytes(), info.getTable().getNamespace());
+    log = wals.getWAL(info);
   }
 
   @After

http://git-wip-us.apache.org/repos/asf/hbase/blob/71a1192d/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/IOTestProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/IOTestProvider.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/IOTestProvider.java
index 9356647..485e5b8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/IOTestProvider.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/IOTestProvider.java
@@ -26,47 +26,45 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hbase.client.RegionInfo;
 // imports for things that haven't moved from regionserver.wal yet.
 import org.apache.hadoop.hbase.regionserver.wal.FSHLog;
 import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogWriter;
 import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
 import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.wal.WAL.Entry;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
- * A WAL Provider that returns a single thread safe WAL that optionally can skip parts of our
- * normal interactions with HDFS.
- *
- * This implementation picks a directory in HDFS based on the same mechanisms as the 
- * {@link FSHLogProvider}. Users can configure how much interaction
- * we have with HDFS with the configuration property "hbase.wal.iotestprovider.operations".
- * The value should be a comma separated list of allowed operations:
+ * A WAL Provider that returns a single thread safe WAL that optionally can skip parts of our normal
+ * interactions with HDFS.
+ * <p>
+ * This implementation picks a directory in HDFS based on the same mechanisms as the
+ * {@link FSHLogProvider}. Users can configure how much interaction we have with HDFS with the
+ * configuration property "hbase.wal.iotestprovider.operations". The value should be a comma
+ * separated list of allowed operations:
  * <ul>
- *   <li><em>append</em>   : edits will be written to the underlying filesystem
- *   <li><em>sync</em>     : wal syncs will result in hflush calls
- *   <li><em>fileroll</em> : roll requests will result in creating a new file on the underlying
- *                           filesystem.
+ * <li><em>append</em> : edits will be written to the underlying filesystem</li>
+ * <li><em>sync</em> : wal syncs will result in hflush calls</li>
+ * <li><em>fileroll</em> : roll requests will result in creating a new file on the underlying
+ * filesystem.</li>
  * </ul>
- * Additionally, the special cases "all" and "none" are recognized.
- * If ommited, the value defaults to "all."
- * Behavior is undefined if "all" or "none" are paired with additional values. Behavior is also
- * undefined if values not listed above are included.
- *
- * Only those operations listed will occur between the returned WAL and HDFS. All others
- * will be no-ops.
- *
+ * Additionally, the special cases "all" and "none" are recognized. If ommited, the value defaults
+ * to "all." Behavior is undefined if "all" or "none" are paired with additional values. Behavior is
+ * also undefined if values not listed above are included.
+ * <p>
+ * Only those operations listed will occur between the returned WAL and HDFS. All others will be
+ * no-ops.
+ * <p>
  * Note that in the case of allowing "append" operations but not allowing "fileroll", the returned
  * WAL will just keep writing to the same file. This won't avoid all costs associated with file
  * management over time, becaue the data set size may result in additional HDFS block allocations.
- *
  */
 @InterfaceAudience.Private
 public class IOTestProvider implements WALProvider {
@@ -114,7 +112,7 @@ public class IOTestProvider implements WALProvider {
   }
 
   @Override
-  public WAL getWAL(final byte[] identifier, byte[] namespace) throws IOException {
+  public WAL getWAL(RegionInfo region) throws IOException {
    return log;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/71a1192d/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestBoundedRegionGroupingStrategy.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestBoundedRegionGroupingStrategy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestBoundedRegionGroupingStrategy.java
index be65ba3..3cf65c5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestBoundedRegionGroupingStrategy.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestBoundedRegionGroupingStrategy.java
@@ -20,23 +20,24 @@ package org.apache.hadoop.hbase.wal;
 
 import static org.apache.hadoop.hbase.wal.BoundedGroupingStrategy.DEFAULT_NUM_REGION_GROUPS;
 import static org.apache.hadoop.hbase.wal.BoundedGroupingStrategy.NUM_REGION_GROUPS;
-import static org.apache.hadoop.hbase.wal.RegionGroupingProvider.*;
+import static org.apache.hadoop.hbase.wal.RegionGroupingProvider.DELEGATE_PROVIDER;
+import static org.apache.hadoop.hbase.wal.RegionGroupingProvider.REGION_GROUPING_STRATEGY;
 import static org.apache.hadoop.hbase.wal.WALFactory.WAL_PROVIDER;
 import static org.junit.Assert.assertEquals;
 
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.HashSet;
-import java.util.Random;
 import java.util.Set;
-
+import java.util.concurrent.ThreadLocalRandom;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
-import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.junit.After;
@@ -179,12 +180,13 @@ public class TestBoundedRegionGroupingStrategy {
       FSUtils.setRootDir(CONF, TEST_UTIL.getDataTestDirOnTestFS());
 
       wals = new WALFactory(CONF, null, "setMembershipDedups");
-      final Set<WAL> seen = new HashSet<>(temp * 4);
-      final Random random = new Random();
+      Set<WAL> seen = new HashSet<>(temp * 4);
       int count = 0;
       // we know that this should see one of the wals more than once
       for (int i = 0; i < temp * 8; i++) {
-        final WAL maybeNewWAL = wals.getWAL(Bytes.toBytes(random.nextInt()), null);
+        WAL maybeNewWAL = wals.getWAL(RegionInfoBuilder
+            .newBuilder(TableName.valueOf("Table-" + ThreadLocalRandom.current().nextInt()))
+            .build());
         LOG.info("Iteration " + i + ", checking wal " + maybeNewWAL);
         if (seen.add(maybeNewWAL)) {
           count++;

http://git-wip-us.apache.org/repos/asf/hbase/blob/71a1192d/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestFSHLogProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestFSHLogProvider.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestFSHLogProvider.java
index da2965e..d9ee9eb 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestFSHLogProvider.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestFSHLogProvider.java
@@ -26,29 +26,28 @@ import static org.junit.Assert.assertTrue;
 import java.io.IOException;
 import java.util.HashSet;
 import java.util.NavigableMap;
-import java.util.Random;
 import java.util.Set;
 import java.util.TreeMap;
-
+import java.util.concurrent.ThreadLocalRandom;
 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.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.ServerName;
 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.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
 // imports for things that haven't moved from regionserver.wal yet.
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
-import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
@@ -63,10 +62,10 @@ import org.slf4j.LoggerFactory;
 public class TestFSHLogProvider {
   private static final Logger LOG = LoggerFactory.getLogger(TestFSHLogProvider.class);
 
-  protected static Configuration conf;
-  protected static FileSystem fs;
-  protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
-  protected MultiVersionConcurrencyControl mvcc;
+  private static Configuration conf;
+  private static FileSystem fs;
+  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private MultiVersionConcurrencyControl mvcc;
 
   @Rule
   public final TestName currentTest = new TestName();
@@ -80,10 +79,6 @@ public class TestFSHLogProvider {
     }
   }
 
-  @After
-  public void tearDown() throws Exception {
-  }
-
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
     // Make block sizes small.
@@ -149,15 +144,15 @@ public class TestFSHLogProvider {
   }
 
 
-  protected void addEdits(WAL log, HRegionInfo hri, HTableDescriptor htd,
-                        int times, NavigableMap<byte[], Integer> scopes) throws IOException {
+  private void addEdits(WAL log, RegionInfo hri, TableDescriptor htd, int times,
+      NavigableMap<byte[], Integer> scopes) throws IOException {
     final byte[] row = Bytes.toBytes("row");
     for (int i = 0; i < times; i++) {
       long timestamp = System.currentTimeMillis();
       WALEdit cols = new WALEdit();
       cols.add(new KeyValue(row, row, row, timestamp, row));
       log.append(hri, getWalKey(hri.getEncodedNameAsBytes(), htd.getTableName(), timestamp, scopes),
-          cols, true);
+        cols, true);
     }
     log.sync();
   }
@@ -181,37 +176,31 @@ public class TestFSHLogProvider {
     wal.completeCacheFlush(regionEncodedName);
   }
 
-  private static final byte[] UNSPECIFIED_REGION = new byte[]{};
-
   @Test
   public void testLogCleaning() throws Exception {
     LOG.info(currentTest.getMethodName());
-    final HTableDescriptor htd =
-        new HTableDescriptor(TableName.valueOf(currentTest.getMethodName())).addFamily(new HColumnDescriptor(
-            "row"));
-    final HTableDescriptor htd2 =
-        new HTableDescriptor(TableName.valueOf(currentTest.getMethodName() + "2"))
-            .addFamily(new HColumnDescriptor("row"));
-    NavigableMap<byte[], Integer> scopes1 = new TreeMap<>(
-        Bytes.BYTES_COMPARATOR);
-    for(byte[] fam : htd.getFamiliesKeys()) {
+    TableDescriptor htd =
+        TableDescriptorBuilder.newBuilder(TableName.valueOf(currentTest.getMethodName()))
+            .addColumnFamily(ColumnFamilyDescriptorBuilder.of("row")).build();
+    TableDescriptor htd2 =
+        TableDescriptorBuilder.newBuilder(TableName.valueOf(currentTest.getMethodName() + "2"))
+            .addColumnFamily(ColumnFamilyDescriptorBuilder.of("row")).build();
+    NavigableMap<byte[], Integer> scopes1 = new TreeMap<>(Bytes.BYTES_COMPARATOR);
+    for (byte[] fam : htd.getColumnFamilyNames()) {
       scopes1.put(fam, 0);
     }
-    NavigableMap<byte[], Integer> scopes2 = new TreeMap<>(
-        Bytes.BYTES_COMPARATOR);
-    for(byte[] fam : htd2.getFamiliesKeys()) {
+    NavigableMap<byte[], Integer> scopes2 = new TreeMap<>(Bytes.BYTES_COMPARATOR);
+    for (byte[] fam : htd2.getColumnFamilyNames()) {
       scopes2.put(fam, 0);
     }
-    final Configuration localConf = new Configuration(conf);
+    Configuration localConf = new Configuration(conf);
     localConf.set(WALFactory.WAL_PROVIDER, FSHLogProvider.class.getName());
-    final WALFactory wals = new WALFactory(localConf, null, currentTest.getMethodName());
+    WALFactory wals = new WALFactory(localConf, null, currentTest.getMethodName());
     try {
-      HRegionInfo hri = new HRegionInfo(htd.getTableName(),
-          HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW);
-      HRegionInfo hri2 = new HRegionInfo(htd2.getTableName(),
-          HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW);
+      RegionInfo hri = RegionInfoBuilder.newBuilder(htd.getTableName()).build();
+      RegionInfo hri2 = RegionInfoBuilder.newBuilder(htd2.getTableName()).build();
       // we want to mix edits from regions, so pick our own identifier.
-      final WAL log = wals.getWAL(UNSPECIFIED_REGION, null);
+      WAL log = wals.getWAL(null);
 
       // Add a single edit and make sure that rolling won't remove the file
       // Before HBASE-3198 it used to delete it
@@ -235,7 +224,7 @@ public class TestFSHLogProvider {
       // Flush the first region, we expect to see the first two files getting
       // archived. We need to append something or writer won't be rolled.
       addEdits(log, hri2, htd2, 1, scopes2);
-      log.startCacheFlush(hri.getEncodedNameAsBytes(), htd.getFamiliesKeys());
+      log.startCacheFlush(hri.getEncodedNameAsBytes(), htd.getColumnFamilyNames());
       log.completeCacheFlush(hri.getEncodedNameAsBytes());
       log.rollWriter();
       assertEquals(2, AbstractFSWALProvider.getNumRolledLogFiles(log));
@@ -244,7 +233,7 @@ public class TestFSHLogProvider {
       // since the oldest was completely flushed and the two others only contain
       // flush information
       addEdits(log, hri2, htd2, 1, scopes2);
-      log.startCacheFlush(hri2.getEncodedNameAsBytes(), htd2.getFamiliesKeys());
+      log.startCacheFlush(hri2.getEncodedNameAsBytes(), htd2.getColumnFamilyNames());
       log.completeCacheFlush(hri2.getEncodedNameAsBytes());
       log.rollWriter();
       assertEquals(0, AbstractFSWALProvider.getNumRolledLogFiles(log));
@@ -270,35 +259,28 @@ public class TestFSHLogProvider {
   @Test
   public void testWALArchiving() throws IOException {
     LOG.debug(currentTest.getMethodName());
-    HTableDescriptor table1 =
-        new HTableDescriptor(TableName.valueOf(currentTest.getMethodName() + "1")).addFamily(new HColumnDescriptor("row"));
-    HTableDescriptor table2 =
-        new HTableDescriptor(TableName.valueOf(currentTest.getMethodName() + "2")).addFamily(new HColumnDescriptor("row"));
-    NavigableMap<byte[], Integer> scopes1 = new TreeMap<>(
-        Bytes.BYTES_COMPARATOR);
-    for(byte[] fam : table1.getFamiliesKeys()) {
+    TableDescriptor table1 =
+        TableDescriptorBuilder.newBuilder(TableName.valueOf(currentTest.getMethodName() + "1"))
+            .addColumnFamily(ColumnFamilyDescriptorBuilder.of("row")).build();
+    TableDescriptor table2 =
+        TableDescriptorBuilder.newBuilder(TableName.valueOf(currentTest.getMethodName() + "2"))
+            .addColumnFamily(ColumnFamilyDescriptorBuilder.of("row")).build();
+    NavigableMap<byte[], Integer> scopes1 = new TreeMap<>(Bytes.BYTES_COMPARATOR);
+    for (byte[] fam : table1.getColumnFamilyNames()) {
       scopes1.put(fam, 0);
     }
-    NavigableMap<byte[], Integer> scopes2 = new TreeMap<>(
-        Bytes.BYTES_COMPARATOR);
-    for(byte[] fam : table2.getFamiliesKeys()) {
+    NavigableMap<byte[], Integer> scopes2 = new TreeMap<>(Bytes.BYTES_COMPARATOR);
+    for (byte[] fam : table2.getColumnFamilyNames()) {
       scopes2.put(fam, 0);
     }
-    final Configuration localConf = new Configuration(conf);
+    Configuration localConf = new Configuration(conf);
     localConf.set(WALFactory.WAL_PROVIDER, FSHLogProvider.class.getName());
-    final WALFactory wals = new WALFactory(localConf, null, currentTest.getMethodName());
+    WALFactory wals = new WALFactory(localConf, null, currentTest.getMethodName());
     try {
-      final WAL wal = wals.getWAL(UNSPECIFIED_REGION, null);
+      WAL wal = wals.getWAL(null);
       assertEquals(0, AbstractFSWALProvider.getNumRolledLogFiles(wal));
-      HRegionInfo hri1 =
-          new HRegionInfo(table1.getTableName(), HConstants.EMPTY_START_ROW,
-              HConstants.EMPTY_END_ROW);
-      HRegionInfo hri2 =
-          new HRegionInfo(table2.getTableName(), HConstants.EMPTY_START_ROW,
-              HConstants.EMPTY_END_ROW);
-      // ensure that we don't split the regions.
-      hri1.setSplit(false);
-      hri2.setSplit(false);
+      RegionInfo hri1 = RegionInfoBuilder.newBuilder(table1.getTableName()).build();
+      RegionInfo hri2 = RegionInfoBuilder.newBuilder(table2.getTableName()).build();
       // variables to mock region sequenceIds.
       // start with the testing logic: insert a waledit, and roll writer
       addEdits(wal, hri1, table1, 1, scopes1);
@@ -312,7 +294,7 @@ public class TestFSHLogProvider {
       assertEquals(2, AbstractFSWALProvider.getNumRolledLogFiles(wal));
       // add a waledit to table1, and flush the region.
       addEdits(wal, hri1, table1, 3, scopes1);
-      flushRegion(wal, hri1.getEncodedNameAsBytes(), table1.getFamiliesKeys());
+      flushRegion(wal, hri1.getEncodedNameAsBytes(), table1.getColumnFamilyNames());
       // roll log; all old logs should be archived.
       wal.rollWriter();
       assertEquals(0, AbstractFSWALProvider.getNumRolledLogFiles(wal));
@@ -326,7 +308,7 @@ public class TestFSHLogProvider {
       assertEquals(2, AbstractFSWALProvider.getNumRolledLogFiles(wal));
       // add edits for table2, and flush hri1.
       addEdits(wal, hri2, table2, 2, scopes2);
-      flushRegion(wal, hri1.getEncodedNameAsBytes(), table2.getFamiliesKeys());
+      flushRegion(wal, hri1.getEncodedNameAsBytes(), table2.getColumnFamilyNames());
       // the log : region-sequenceId map is
       // log1: region2 (unflushed)
       // log2: region1 (flushed)
@@ -336,7 +318,7 @@ public class TestFSHLogProvider {
       assertEquals(2, AbstractFSWALProvider.getNumRolledLogFiles(wal));
       // flush region2, and all logs should be archived.
       addEdits(wal, hri2, table2, 2, scopes2);
-      flushRegion(wal, hri2.getEncodedNameAsBytes(), table2.getFamiliesKeys());
+      flushRegion(wal, hri2.getEncodedNameAsBytes(), table2.getColumnFamilyNames());
       wal.rollWriter();
       assertEquals(0, AbstractFSWALProvider.getNumRolledLogFiles(wal));
     } finally {
@@ -365,18 +347,20 @@ public class TestFSHLogProvider {
    */
   @Test
   public void setMembershipDedups() throws IOException {
-    final Configuration localConf = new Configuration(conf);
+    Configuration localConf = new Configuration(conf);
     localConf.set(WALFactory.WAL_PROVIDER, FSHLogProvider.class.getName());
-    final WALFactory wals = new WALFactory(localConf, null, currentTest.getMethodName());
+    WALFactory wals = new WALFactory(localConf, null, currentTest.getMethodName());
     try {
       final Set<WAL> seen = new HashSet<>(1);
-      final Random random = new Random();
       assertTrue("first attempt to add WAL from default provider should work.",
-          seen.add(wals.getWAL(Bytes.toBytes(random.nextInt()), null)));
+        seen.add(wals.getWAL(null)));
       for (int i = 0; i < 1000; i++) {
-        assertFalse("default wal provider is only supposed to return a single wal, which should "
-            + "compare as .equals itself.",
-          seen.add(wals.getWAL(Bytes.toBytes(random.nextInt()), null)));
+        assertFalse(
+          "default wal provider is only supposed to return a single wal, which should " +
+            "compare as .equals itself.",
+          seen.add(wals.getWAL(RegionInfoBuilder
+              .newBuilder(TableName.valueOf("Table-" + ThreadLocalRandom.current().nextInt()))
+              .build())));
       }
     } finally {
       wals.close();

http://git-wip-us.apache.org/repos/asf/hbase/blob/71a1192d/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestSecureWAL.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestSecureWAL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestSecureWAL.java
index 0814fcb..b4160e9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestSecureWAL.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestSecureWAL.java
@@ -1,4 +1,4 @@
-/*
+/**
  * 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
@@ -25,7 +25,6 @@ import java.util.Arrays;
 import java.util.List;
 import java.util.NavigableMap;
 import java.util.TreeMap;
-
 import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
@@ -33,12 +32,11 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
 import org.apache.hadoop.hbase.io.crypto.KeyProviderForTesting;
 import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
 import org.apache.hadoop.hbase.regionserver.wal.SecureAsyncProtobufLogWriter;
@@ -106,14 +104,9 @@ public class TestSecureWAL {
   @Test
   public void testSecureWAL() throws Exception {
     TableName tableName = TableName.valueOf(name.getMethodName().replaceAll("[^a-zA-Z0-9]", "_"));
-    HTableDescriptor htd = new HTableDescriptor(tableName);
-    htd.addFamily(new HColumnDescriptor(tableName.getName()));
     NavigableMap<byte[], Integer> scopes = new TreeMap<>(Bytes.BYTES_COMPARATOR);
-    for(byte[] fam : htd.getFamiliesKeys()) {
-      scopes.put(fam, 0);
-    }
-    HRegionInfo regioninfo = new HRegionInfo(tableName,
-      HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, false);
+    scopes.put(tableName.getName(), 0);
+    RegionInfo regionInfo = RegionInfoBuilder.newBuilder(tableName).build();
     final int total = 10;
     final byte[] row = Bytes.toBytes("row");
     final byte[] family = Bytes.toBytes("family");
@@ -123,15 +116,14 @@ public class TestSecureWAL {
         new WALFactory(TEST_UTIL.getConfiguration(), null, tableName.getNameAsString());
 
     // Write the WAL
-    final WAL wal =
-        wals.getWAL(regioninfo.getEncodedNameAsBytes(), regioninfo.getTable().getNamespace());
+    final WAL wal = wals.getWAL(regionInfo);
 
     MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl();
 
     for (int i = 0; i < total; i++) {
       WALEdit kvs = new WALEdit();
       kvs.add(new KeyValue(row, family, Bytes.toBytes(i), value));
-      wal.append(regioninfo, new WALKeyImpl(regioninfo.getEncodedNameAsBytes(), tableName,
+      wal.append(regionInfo, new WALKeyImpl(regionInfo.getEncodedNameAsBytes(), tableName,
           System.currentTimeMillis(), mvcc, scopes), kvs, true);
     }
     wal.sync();

http://git-wip-us.apache.org/repos/asf/hbase/blob/71a1192d/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALFactory.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALFactory.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALFactory.java
index 2c19c12..f0f35e7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALFactory.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALFactory.java
@@ -1,5 +1,4 @@
-/*
- *
+/**
  * 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
@@ -30,7 +29,6 @@ import java.net.BindException;
 import java.util.List;
 import java.util.NavigableMap;
 import java.util.TreeMap;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
@@ -41,14 +39,15 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.Coprocessor;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.ServerName;
 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.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
 import org.apache.hadoop.hbase.coprocessor.SampleRegionWALCoprocessor;
 import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
@@ -169,27 +168,24 @@ public class TestWALFactory {
     final byte [] rowName = tableName.getName();
     final MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl(1);
     final int howmany = 3;
-    HRegionInfo[] infos = new HRegionInfo[3];
+    RegionInfo[] infos = new RegionInfo[3];
     Path tabledir = FSUtils.getTableDir(hbaseWALDir, tableName);
     fs.mkdirs(tabledir);
-    for(int i = 0; i < howmany; i++) {
-      infos[i] = new HRegionInfo(tableName,
-                Bytes.toBytes("" + i), Bytes.toBytes("" + (i+1)), false);
+    for (int i = 0; i < howmany; i++) {
+      infos[i] = RegionInfoBuilder.newBuilder(tableName).setStartKey(Bytes.toBytes("" + i))
+          .setEndKey(Bytes.toBytes("" + (i + 1))).build();
       fs.mkdirs(new Path(tabledir, infos[i].getEncodedName()));
       LOG.info("allo " + new Path(tabledir, infos[i].getEncodedName()).toString());
     }
-    HTableDescriptor htd = new HTableDescriptor(tableName);
-    htd.addFamily(new HColumnDescriptor("column"));
     NavigableMap<byte[], Integer> scopes = new TreeMap<>(Bytes.BYTES_COMPARATOR);
-    for(byte[] fam : htd.getFamiliesKeys()) {
-      scopes.put(fam, 0);
-    }
+    scopes.put(Bytes.toBytes("column"), 0);
+
 
     // Add edits for three regions.
     for (int ii = 0; ii < howmany; ii++) {
       for (int i = 0; i < howmany; i++) {
         final WAL log =
-            wals.getWAL(infos[i].getEncodedNameAsBytes(), infos[i].getTable().getNamespace());
+            wals.getWAL(infos[i]);
         for (int j = 0; j < howmany; j++) {
           WALEdit edit = new WALEdit();
           byte [] family = Bytes.toBytes("column");
@@ -254,15 +250,10 @@ public class TestWALFactory {
     WAL.Reader reader = null;
 
     try {
-      HRegionInfo info = new HRegionInfo(tableName,
-                  null,null, false);
-      HTableDescriptor htd = new HTableDescriptor(tableName);
-      htd.addFamily(new HColumnDescriptor(tableName.getName()));
+      RegionInfo info = RegionInfoBuilder.newBuilder(tableName).build();
       NavigableMap<byte[], Integer> scopes = new TreeMap<>(Bytes.BYTES_COMPARATOR);
-      for(byte[] fam : htd.getFamiliesKeys()) {
-        scopes.put(fam, 0);
-      }
-      final WAL wal = wals.getWAL(info.getEncodedNameAsBytes(), info.getTable().getNamespace());
+      scopes.put(tableName.getName(), 0);
+      final WAL wal = wals.getWAL(info);
 
       for (int i = 0; i < total; i++) {
         WALEdit kvs = new WALEdit();
@@ -374,24 +365,18 @@ public class TestWALFactory {
   public void testAppendClose() throws Exception {
     TableName tableName =
         TableName.valueOf(currentTest.getMethodName());
-    HRegionInfo regioninfo = new HRegionInfo(tableName,
-             HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, false);
+    RegionInfo regionInfo = RegionInfoBuilder.newBuilder(tableName).build();
 
-    final WAL wal =
-        wals.getWAL(regioninfo.getEncodedNameAsBytes(), regioninfo.getTable().getNamespace());
-    final int total = 20;
+    WAL wal = wals.getWAL(regionInfo);
+    int total = 20;
 
-    HTableDescriptor htd = new HTableDescriptor(tableName);
-    htd.addFamily(new HColumnDescriptor(tableName.getName()));
     NavigableMap<byte[], Integer> scopes = new TreeMap<>(Bytes.BYTES_COMPARATOR);
-    for(byte[] fam : htd.getFamiliesKeys()) {
-      scopes.put(fam, 0);
-    }
+    scopes.put(tableName.getName(), 0);
     MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl();
     for (int i = 0; i < total; i++) {
       WALEdit kvs = new WALEdit();
       kvs.add(new KeyValue(Bytes.toBytes(i), tableName.getName(), tableName.getName()));
-      wal.append(regioninfo, new WALKeyImpl(regioninfo.getEncodedNameAsBytes(), tableName,
+      wal.append(regionInfo, new WALKeyImpl(regionInfo.getEncodedNameAsBytes(), tableName,
           System.currentTimeMillis(), mvcc, scopes),
         kvs, true);
     }
@@ -496,20 +481,18 @@ public class TestWALFactory {
 
   /**
    * Tests that we can write out an edit, close, and then read it back in again.
-   * @throws IOException
    */
   @Test
   public void testEditAdd() throws IOException {
-    final int COL_COUNT = 10;
-    final HTableDescriptor htd =
-        new HTableDescriptor(TableName.valueOf(currentTest.getMethodName())).addFamily(new HColumnDescriptor(
-            "column"));
-    NavigableMap<byte[], Integer> scopes = new TreeMap<byte[], Integer>(
-        Bytes.BYTES_COMPARATOR);
-    for(byte[] fam : htd.getFamiliesKeys()) {
+    int colCount = 10;
+    TableDescriptor htd =
+        TableDescriptorBuilder.newBuilder(TableName.valueOf(currentTest.getMethodName()))
+            .addColumnFamily(ColumnFamilyDescriptorBuilder.of("column")).build();
+    NavigableMap<byte[], Integer> scopes = new TreeMap<byte[], Integer>(Bytes.BYTES_COMPARATOR);
+    for (byte[] fam : htd.getColumnFamilyNames()) {
       scopes.put(fam, 0);
     }
-    final byte [] row = Bytes.toBytes("row");
+    byte[] row = Bytes.toBytes("row");
     WAL.Reader reader = null;
     try {
       final MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl(1);
@@ -518,21 +501,21 @@ public class TestWALFactory {
       // 1, 2, 3...
       long timestamp = System.currentTimeMillis();
       WALEdit cols = new WALEdit();
-      for (int i = 0; i < COL_COUNT; i++) {
+      for (int i = 0; i < colCount; i++) {
         cols.add(new KeyValue(row, Bytes.toBytes("column"),
             Bytes.toBytes(Integer.toString(i)),
           timestamp, new byte[] { (byte)(i + '0') }));
       }
-      HRegionInfo info = new HRegionInfo(htd.getTableName(),
-        row,Bytes.toBytes(Bytes.toString(row) + "1"), false);
-      final WAL log = wals.getWAL(info.getEncodedNameAsBytes(), info.getTable().getNamespace());
+      RegionInfo info = RegionInfoBuilder.newBuilder(htd.getTableName()).setStartKey(row)
+          .setEndKey(Bytes.toBytes(Bytes.toString(row) + "1")).build();
+      final WAL log = wals.getWAL(info);
 
       final long txid = log.append(info,
         new WALKeyImpl(info.getEncodedNameAsBytes(), htd.getTableName(), System.currentTimeMillis(),
             mvcc, scopes),
         cols, true);
       log.sync(txid);
-      log.startCacheFlush(info.getEncodedNameAsBytes(), htd.getFamiliesKeys());
+      log.startCacheFlush(info.getEncodedNameAsBytes(), htd.getColumnFamilyNames());
       log.completeCacheFlush(info.getEncodedNameAsBytes());
       log.shutdown();
       Path filename = AbstractFSWALProvider.getCurrentFileName(log);
@@ -560,21 +543,17 @@ public class TestWALFactory {
     }
   }
 
-  /**
-   * @throws IOException
-   */
   @Test
   public void testAppend() throws IOException {
-    final int COL_COUNT = 10;
-    final HTableDescriptor htd =
-        new HTableDescriptor(TableName.valueOf(currentTest.getMethodName())).addFamily(new HColumnDescriptor(
-            "column"));
-    NavigableMap<byte[], Integer> scopes = new TreeMap<byte[], Integer>(
-        Bytes.BYTES_COMPARATOR);
-    for(byte[] fam : htd.getFamiliesKeys()) {
+    int colCount = 10;
+    TableDescriptor htd =
+        TableDescriptorBuilder.newBuilder(TableName.valueOf(currentTest.getMethodName()))
+            .addColumnFamily(ColumnFamilyDescriptorBuilder.of("column")).build();
+    NavigableMap<byte[], Integer> scopes = new TreeMap<byte[], Integer>(Bytes.BYTES_COMPARATOR);
+    for (byte[] fam : htd.getColumnFamilyNames()) {
       scopes.put(fam, 0);
     }
-    final byte [] row = Bytes.toBytes("row");
+    byte[] row = Bytes.toBytes("row");
     WAL.Reader reader = null;
     final MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl(1);
     try {
@@ -582,27 +561,26 @@ public class TestWALFactory {
       // 1, 2, 3...
       long timestamp = System.currentTimeMillis();
       WALEdit cols = new WALEdit();
-      for (int i = 0; i < COL_COUNT; i++) {
+      for (int i = 0; i < colCount; i++) {
         cols.add(new KeyValue(row, Bytes.toBytes("column"),
           Bytes.toBytes(Integer.toString(i)),
           timestamp, new byte[] { (byte)(i + '0') }));
       }
-      HRegionInfo hri = new HRegionInfo(htd.getTableName(),
-          HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW);
-      final WAL log = wals.getWAL(hri.getEncodedNameAsBytes(), hri.getTable().getNamespace());
+      RegionInfo hri = RegionInfoBuilder.newBuilder(htd.getTableName()).build();
+      final WAL log = wals.getWAL(hri);
       final long txid = log.append(hri,
         new WALKeyImpl(hri.getEncodedNameAsBytes(), htd.getTableName(), System.currentTimeMillis(),
             mvcc, scopes),
         cols, true);
       log.sync(txid);
-      log.startCacheFlush(hri.getEncodedNameAsBytes(), htd.getFamiliesKeys());
+      log.startCacheFlush(hri.getEncodedNameAsBytes(), htd.getColumnFamilyNames());
       log.completeCacheFlush(hri.getEncodedNameAsBytes());
       log.shutdown();
       Path filename = AbstractFSWALProvider.getCurrentFileName(log);
       // Now open a reader on the log and assert append worked.
       reader = wals.createReader(fs, filename);
       WAL.Entry entry = reader.next();
-      assertEquals(COL_COUNT, entry.getEdit().size());
+      assertEquals(colCount, entry.getEdit().size());
       int idx = 0;
       for (Cell val : entry.getEdit().getCells()) {
         assertTrue(Bytes.equals(hri.getEncodedNameAsBytes(),
@@ -633,15 +611,11 @@ public class TestWALFactory {
     final DumbWALActionsListener visitor = new DumbWALActionsListener();
     final MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl(1);
     long timestamp = System.currentTimeMillis();
-    HTableDescriptor htd = new HTableDescriptor(tableName);
-    htd.addFamily(new HColumnDescriptor("column"));
     NavigableMap<byte[], Integer> scopes = new TreeMap<>(Bytes.BYTES_COMPARATOR);
-    for(byte[] fam : htd.getFamiliesKeys()) {
-      scopes.put(fam, 0);
-    }
-    HRegionInfo hri = new HRegionInfo(tableName,
-        HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW);
-    final WAL log = wals.getWAL(hri.getEncodedNameAsBytes(), hri.getTable().getNamespace());
+    scopes.put(Bytes.toBytes("column"), 0);
+
+    RegionInfo hri = RegionInfoBuilder.newBuilder(tableName).build();
+    final WAL log = wals.getWAL(hri);
     log.registerWALActionsListener(visitor);
     for (int i = 0; i < COL_COUNT; i++) {
       WALEdit cols = new WALEdit();
@@ -670,7 +644,7 @@ public class TestWALFactory {
   @Test
   public void testWALCoprocessorLoaded() throws Exception {
     // test to see whether the coprocessor is loaded or not.
-    WALCoprocessorHost host = wals.getWAL(UNSPECIFIED_REGION, null).getCoprocessorHost();
+    WALCoprocessorHost host = wals.getWAL(null).getCoprocessorHost();
     Coprocessor c = host.findCoprocessor(SampleRegionWALCoprocessor.class);
     assertNotNull(c);
   }
@@ -690,7 +664,4 @@ public class TestWALFactory {
       increments++;
     }
   }
-
-  private static final byte[] UNSPECIFIED_REGION = new byte[]{};
-
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/71a1192d/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALMethods.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALMethods.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALMethods.java
index e33e18d..5364504 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALMethods.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALMethods.java
@@ -99,7 +99,7 @@ public class TestWALMethods {
 
     final Configuration walConf = new Configuration(util.getConfiguration());
     FSUtils.setRootDir(walConf, regiondir);
-    (new WALFactory(walConf, null, "dummyLogName")).getWAL(new byte[] {}, null);
+    (new WALFactory(walConf, null, "dummyLogName")).getWAL(null);
 
     NavigableSet<Path> files = WALSplitter.getSplitEditFilesSorted(fs, regiondir);
     assertEquals(7, files.size());

http://git-wip-us.apache.org/repos/asf/hbase/blob/71a1192d/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALReaderOnSecureWAL.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALReaderOnSecureWAL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALReaderOnSecureWAL.java
index 3672f9b..dcb6f5a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALReaderOnSecureWAL.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALReaderOnSecureWAL.java
@@ -1,4 +1,4 @@
-/*
+/**
  * 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
@@ -25,22 +25,20 @@ import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.NavigableMap;
 import java.util.TreeMap;
-
 import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.ByteBufferKeyValue;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.ByteBufferKeyValue;
 import org.apache.hadoop.hbase.ServerName;
 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.io.crypto.KeyProviderForTesting;
 import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
 // imports for things that haven't moved from regionserver.wal yet.
@@ -85,7 +83,6 @@ public class TestWALReaderOnSecureWAL {
     FSUtils.setRootDir(conf, TEST_UTIL.getDataTestDir());
   }
 
-  @SuppressWarnings("deprecation")
   private Path writeWAL(final WALFactory wals, final String tblName, boolean offheap) throws IOException {
     Configuration conf = TEST_UTIL.getConfiguration();
     String clsName = conf.get(WALCellCodec.WAL_CELL_CODEC_CLASS_KEY, WALCellCodec.class.getName());
@@ -93,22 +90,16 @@ public class TestWALReaderOnSecureWAL {
       WALCellCodec.class);
     try {
       TableName tableName = TableName.valueOf(tblName);
-      HTableDescriptor htd = new HTableDescriptor(tableName);
-      htd.addFamily(new HColumnDescriptor(tableName.getName()));
       NavigableMap<byte[], Integer> scopes = new TreeMap<>(Bytes.BYTES_COMPARATOR);
-      for(byte[] fam : htd.getFamiliesKeys()) {
-        scopes.put(fam, 0);
-      }
-      HRegionInfo regioninfo = new HRegionInfo(tableName,
-        HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, false);
+      scopes.put(tableName.getName(), 0);
+      RegionInfo regionInfo = RegionInfoBuilder.newBuilder(tableName).build();
       final int total = 10;
       final byte[] row = Bytes.toBytes("row");
       final byte[] family = Bytes.toBytes("family");
       final MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl(1);
 
       // Write the WAL
-      WAL wal =
-          wals.getWAL(regioninfo.getEncodedNameAsBytes(), regioninfo.getTable().getNamespace());
+      WAL wal = wals.getWAL(regionInfo);
       for (int i = 0; i < total; i++) {
         WALEdit kvs = new WALEdit();
         KeyValue kv = new KeyValue(row, family, Bytes.toBytes(i), value);
@@ -120,7 +111,7 @@ public class TestWALReaderOnSecureWAL {
         } else {
           kvs.add(kv);
         }
-        wal.append(regioninfo, new WALKeyImpl(regioninfo.getEncodedNameAsBytes(), tableName,
+        wal.append(regionInfo, new WALKeyImpl(regionInfo.getEncodedNameAsBytes(), tableName,
             System.currentTimeMillis(), mvcc, scopes), kvs, true);
       }
       wal.sync();

http://git-wip-us.apache.org/repos/asf/hbase/blob/71a1192d/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALRootDir.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALRootDir.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALRootDir.java
index c71fb4b3..e46dc21 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALRootDir.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALRootDir.java
@@ -1,4 +1,4 @@
-/*
+/**
  * 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
@@ -17,17 +17,22 @@
  */
 package org.apache.hadoop.hbase.wal;
 
+import static org.junit.Assert.assertEquals;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
 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.HBaseTestingUtility;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.KeyValue;
+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.regionserver.MultiVersionConcurrencyControl;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -40,13 +45,6 @@ import org.junit.experimental.categories.Category;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
-import static org.junit.Assert.assertEquals;
-
 @Category(MediumTests.class)
 public class TestWALRootDir {
   private static final Logger LOG = LoggerFactory.getLogger(TestWALRootDir.class);
@@ -54,10 +52,9 @@ public class TestWALRootDir {
   private static Configuration conf;
   private static FileSystem fs;
   private static FileSystem walFs;
-  static final TableName tableName = TableName.valueOf("TestWALWALDir");
+  private static final TableName tableName = TableName.valueOf("TestWALWALDir");
   private static final byte [] rowName = Bytes.toBytes("row");
   private static final byte [] family = Bytes.toBytes("column");
-  private static HTableDescriptor htd;
   private static Path walRootDir;
   private static Path rootDir;
   private static WALFactory wals;
@@ -75,8 +72,6 @@ public class TestWALRootDir {
     walRootDir = TEST_UTIL.createWALRootDir();
     fs = FSUtils.getRootDirFileSystem(conf);
     walFs = FSUtils.getWALFileSystem(conf);
-    htd = new HTableDescriptor(tableName);
-    htd.addFamily(new HColumnDescriptor(family));
   }
 
   @AfterClass
@@ -87,9 +82,9 @@ public class TestWALRootDir {
 
   @Test
   public void testWALRootDir() throws Exception {
-    HRegionInfo regionInfo = new HRegionInfo(tableName);
+    RegionInfo regionInfo = RegionInfoBuilder.newBuilder(tableName).build();
     wals = new WALFactory(conf, null, "testWALRootDir");
-    WAL log = wals.getWAL(regionInfo.getEncodedNameAsBytes(), regionInfo.getTable().getNamespace());
+    WAL log = wals.getWAL(regionInfo);
 
     assertEquals(1, getWALFiles(walFs, walRootDir).size());
     byte [] value = Bytes.toBytes("value");
@@ -117,7 +112,7 @@ public class TestWALRootDir {
         new Path(walRootDir, HConstants.HREGION_LOGDIR_NAME)).size());
   }
 
-  protected WALKeyImpl getWalKey(final long time, HRegionInfo hri, final long startPoint) {
+  private WALKeyImpl getWalKey(final long time, RegionInfo hri, final long startPoint) {
     return new WALKeyImpl(hri.getEncodedNameAsBytes(), tableName, time,
         new MultiVersionConcurrencyControl(startPoint));
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/71a1192d/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java
index 556afd9..a1206aa 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java
@@ -23,10 +23,6 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
-import org.apache.hbase.thirdparty.com.google.common.base.Joiner;
-import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList;
-import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableMap;
-
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.lang.reflect.Method;
@@ -45,7 +41,6 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.stream.Collectors;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
@@ -58,18 +53,16 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.ServerName;
 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.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.wal.FaultyProtobufLogReader;
 import org.apache.hadoop.hbase.regionserver.wal.InstrumentedLogWriter;
 import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader;
 import org.apache.hadoop.hbase.security.User;
-import org.apache.hbase.thirdparty.com.google.protobuf.ByteString;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -98,6 +91,14 @@ import org.mockito.stubbing.Answer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hbase.thirdparty.com.google.common.base.Joiner;
+import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList;
+import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableMap;
+import org.apache.hbase.thirdparty.com.google.protobuf.ByteString;
+
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
+
 /**
  * Testing {@link WAL} splitting code.
  */
@@ -373,10 +374,10 @@ public class TestWALSplit {
    */
   @Test (timeout=300000)
   public void testRecoveredEditsPathForMeta() throws IOException {
-    byte [] encoded = HRegionInfo.FIRST_META_REGIONINFO.getEncodedNameAsBytes();
+    byte[] encoded = RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedNameAsBytes();
     Path tdir = FSUtils.getTableDir(HBASEDIR, TableName.META_TABLE_NAME);
     Path regiondir = new Path(tdir,
-        HRegionInfo.FIRST_META_REGIONINFO.getEncodedName());
+      RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedName());
     fs.mkdirs(regiondir);
     long now = System.currentTimeMillis();
     Entry entry =
@@ -386,7 +387,7 @@ public class TestWALSplit {
     Path p = WALSplitter.getRegionSplitEditsPath(fs, entry, HBASEDIR,
         FILENAME_BEING_SPLIT);
     String parentOfParent = p.getParent().getParent().getName();
-    assertEquals(parentOfParent, HRegionInfo.FIRST_META_REGIONINFO.getEncodedName());
+    assertEquals(parentOfParent, RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedName());
   }
 
   /**
@@ -395,10 +396,10 @@ public class TestWALSplit {
    */
   @Test (timeout=300000)
   public void testOldRecoveredEditsFileSidelined() throws IOException {
-    byte [] encoded = HRegionInfo.FIRST_META_REGIONINFO.getEncodedNameAsBytes();
+    byte [] encoded = RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedNameAsBytes();
     Path tdir = FSUtils.getTableDir(HBASEDIR, TableName.META_TABLE_NAME);
     Path regiondir = new Path(tdir,
-        HRegionInfo.FIRST_META_REGIONINFO.getEncodedName());
+      RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedName());
     fs.mkdirs(regiondir);
     long now = System.currentTimeMillis();
     Entry entry =
@@ -412,7 +413,7 @@ public class TestWALSplit {
     Path p = WALSplitter.getRegionSplitEditsPath(fs, entry, HBASEDIR,
         FILENAME_BEING_SPLIT);
     String parentOfParent = p.getParent().getParent().getName();
-    assertEquals(parentOfParent, HRegionInfo.FIRST_META_REGIONINFO.getEncodedName());
+    assertEquals(parentOfParent, RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedName());
     WALFactory.createRecoveredEditsWriter(fs, p, conf).close();
   }
 
@@ -459,7 +460,7 @@ public class TestWALSplit {
 
   @Test (timeout=300000)
   public void testSplitLeavesCompactionEventsEdits() throws IOException{
-    HRegionInfo hri = new HRegionInfo(TABLE_NAME);
+    RegionInfo hri = RegionInfoBuilder.newBuilder(TABLE_NAME).build();
     REGIONS.clear();
     REGIONS.add(hri.getEncodedName());
     Path regionDir = new Path(FSUtils.getTableDir(HBASEDIR, TABLE_NAME), hri.getEncodedName());
@@ -1132,7 +1133,7 @@ public class TestWALSplit {
     REGIONS.add(regionName);
     generateWALs(-1);
 
-    wals.getWAL(Bytes.toBytes(regionName), null);
+    wals.getWAL(null);
     FileStatus[] logfiles = fs.listStatus(WALDIR);
     assertTrue("There should be some log file",
         logfiles != null && logfiles.length > 0);
@@ -1337,7 +1338,7 @@ public class TestWALSplit {
     return count;
   }
 
-  private static void appendCompactionEvent(Writer w, HRegionInfo hri, String[] inputs,
+  private static void appendCompactionEvent(Writer w, RegionInfo hri, String[] inputs,
       String output) throws IOException {
     WALProtos.CompactionDescriptor.Builder desc = WALProtos.CompactionDescriptor.newBuilder();
     desc.setTableName(ByteString.copyFrom(hri.getTable().toBytes()))

http://git-wip-us.apache.org/repos/asf/hbase/blob/71a1192d/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/WALPerformanceEvaluation.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/WALPerformanceEvaluation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/WALPerformanceEvaluation.java
index 93e6373..a9bad00 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/WALPerformanceEvaluation.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/WALPerformanceEvaluation.java
@@ -20,6 +20,11 @@ package org.apache.hadoop.hbase.wal;
 
 import static com.codahale.metrics.MetricRegistry.name;
 
+import com.codahale.metrics.ConsoleReporter;
+import com.codahale.metrics.Histogram;
+import com.codahale.metrics.Meter;
+import com.codahale.metrics.MetricFilter;
+import com.codahale.metrics.MetricRegistry;
 import java.io.IOException;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -30,7 +35,7 @@ import java.util.Random;
 import java.util.Set;
 import java.util.TreeMap;
 import java.util.concurrent.TimeUnit;
-
+import java.util.stream.IntStream;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.FileStatus;
@@ -39,14 +44,15 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Cell;
 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.MockRegionServerServices;
 import org.apache.hadoop.hbase.TableName;
+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.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.io.crypto.KeyProviderForTesting;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.LogRoller;
@@ -71,12 +77,6 @@ import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.codahale.metrics.ConsoleReporter;
-import com.codahale.metrics.Histogram;
-import com.codahale.metrics.Meter;
-import com.codahale.metrics.MetricFilter;
-import com.codahale.metrics.MetricRegistry;
-
 // imports for things that haven't moved from regionserver.wal yet.
 
 /**
@@ -131,11 +131,10 @@ public final class WALPerformanceEvaluation extends Configured implements Tool {
     private final boolean noSync;
     private final HRegion region;
     private final int syncInterval;
-    private final HTableDescriptor htd;
     private final Sampler loopSampler;
     private final NavigableMap<byte[], Integer> scopes;
 
-    WALPutBenchmark(final HRegion region, final HTableDescriptor htd,
+    WALPutBenchmark(final HRegion region, final TableDescriptor htd,
         final long numIterations, final boolean noSync, final int syncInterval,
         final double traceFreq) {
       this.numIterations = numIterations;
@@ -143,9 +142,8 @@ public final class WALPerformanceEvaluation extends Configured implements Tool {
       this.syncInterval = syncInterval;
       this.numFamilies = htd.getColumnFamilyCount();
       this.region = region;
-      this.htd = htd;
       scopes = new TreeMap<>(Bytes.BYTES_COMPARATOR);
-      for(byte[] fam : htd.getFamiliesKeys()) {
+      for(byte[] fam : htd.getColumnFamilyNames()) {
         scopes.put(fam, 0);
       }
       String spanReceivers = getConf().get("hbase.trace.spanreceiver.classes");
@@ -320,7 +318,7 @@ public final class WALPerformanceEvaluation extends Configured implements Tool {
       if (rootRegionDir == null) {
         rootRegionDir = TEST_UTIL.getDataTestDirOnTestFS("WALPerformanceEvaluation");
       }
-      rootRegionDir = rootRegionDir.makeQualified(fs);
+      rootRegionDir = rootRegionDir.makeQualified(fs.getUri(), fs.getWorkingDirectory());
       cleanRegionRootDir(fs, rootRegionDir);
       FSUtils.setRootDir(getConf(), rootRegionDir);
       final WALFactory wals = new WALFactory(getConf(), null, "wals");
@@ -334,7 +332,7 @@ public final class WALPerformanceEvaluation extends Configured implements Tool {
         for(int i = 0; i < numRegions; i++) {
           // Initialize Table Descriptor
           // a table per desired region means we can avoid carving up the key space
-          final HTableDescriptor htd = createHTableDescriptor(i, numFamilies);
+          final TableDescriptor htd = createHTableDescriptor(i, numFamilies);
           regions[i] = openRegion(fs, rootRegionDir, htd, wals, roll, roller);
           benchmarks[i] = TraceUtil.wrap(new WALPutBenchmark(regions[i], htd, numIterations, noSync,
               syncInterval, traceFreq), "");
@@ -401,14 +399,14 @@ public final class WALPerformanceEvaluation extends Configured implements Tool {
     return(0);
   }
 
-  private static HTableDescriptor createHTableDescriptor(final int regionNum,
+  private static TableDescriptor createHTableDescriptor(final int regionNum,
       final int numFamilies) {
-    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(TABLE_NAME + ":" + regionNum));
-    for (int i = 0; i < numFamilies; ++i) {
-      HColumnDescriptor colDef = new HColumnDescriptor(FAMILY_PREFIX + i);
-      htd.addFamily(colDef);
-    }
-    return htd;
+    TableDescriptorBuilder builder =
+        TableDescriptorBuilder.newBuilder(TableName.valueOf(TABLE_NAME + ":" + regionNum));
+    IntStream.range(0, numFamilies)
+        .mapToObj(i -> ColumnFamilyDescriptorBuilder.of(FAMILY_PREFIX + i))
+        .forEachOrdered(builder::addColumnFamily);
+    return builder.build();
   }
 
   /**
@@ -495,13 +493,12 @@ public final class WALPerformanceEvaluation extends Configured implements Tool {
 
   private final Set<WAL> walsListenedTo = new HashSet<>();
 
-  private HRegion openRegion(final FileSystem fs, final Path dir, final HTableDescriptor htd,
+  private HRegion openRegion(final FileSystem fs, final Path dir, final TableDescriptor htd,
       final WALFactory wals, final long whenToRoll, final LogRoller roller) throws IOException {
     // Initialize HRegion
     RegionInfo regionInfo = RegionInfoBuilder.newBuilder(htd.getTableName()).build();
     // Initialize WAL
-    final WAL wal =
-        wals.getWAL(regionInfo.getEncodedNameAsBytes(), regionInfo.getTable().getNamespace());
+    final WAL wal = wals.getWAL(regionInfo);
     // If we haven't already, attach a listener to this wal to handle rolls and metrics.
     if (walsListenedTo.add(wal)) {
       roller.addWAL(wal);


[2/2] hbase git commit: HBASE-19751 Use RegionInfo directly instead of an identifier and a namespace when getting WAL

Posted by zh...@apache.org.
HBASE-19751 Use RegionInfo directly instead of an identifier and a namespace when getting WAL


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

Branch: refs/heads/master
Commit: 71a1192d671a93cc17b82e4355f2ace97c41dae5
Parents: 62a8188
Author: zhangduo <zh...@apache.org>
Authored: Thu Jan 11 15:47:08 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Thu Jan 11 15:47:34 2018 +0800

----------------------------------------------------------------------
 .../hbase/mapreduce/TestWALRecordReader.java    |  50 +++----
 .../hbase/regionserver/HRegionServer.java       |  14 +-
 .../org/apache/hadoop/hbase/util/HBaseFsck.java |   6 +-
 .../hadoop/hbase/wal/AbstractFSWALProvider.java |   3 +-
 .../hadoop/hbase/wal/DisabledWALProvider.java   |   2 +-
 .../hbase/wal/RegionGroupingProvider.java       |  40 +++---
 .../org/apache/hadoop/hbase/wal/WALFactory.java |  64 ++++-----
 .../apache/hadoop/hbase/wal/WALProvider.java    |  17 ++-
 .../hadoop/hbase/HBaseTestingUtility.java       |   5 +-
 .../hbase/coprocessor/TestWALObserver.java      | 117 +++++++----------
 .../regionserver/TestCacheOnWriteInSchema.java  |  52 ++++----
 .../TestCompactionArchiveConcurrentClose.java   |  31 ++---
 .../TestCompactionArchiveIOException.java       |  42 +++---
 .../hbase/regionserver/TestDefaultMemStore.java |  45 +++----
 .../hbase/regionserver/TestHMobStore.java       |   3 +-
 .../hadoop/hbase/regionserver/TestHRegion.java  |  13 +-
 .../regionserver/TestHRegionReplayEvents.java   |  43 +++---
 .../hadoop/hbase/regionserver/TestHStore.java   |  14 +-
 .../TestStoreFileRefresherChore.java            |  51 ++++----
 .../TestWALMonotonicallyIncreasingSeqId.java    |   2 +-
 .../wal/AbstractTestLogRolling.java             |   9 +-
 .../hbase/regionserver/wal/TestDurability.java  |  93 +++++++------
 .../regionserver/wal/TestLogRollAbort.java      |  39 +++---
 .../wal/TestLogRollingNoCluster.java            |  17 +--
 .../wal/TestWALActionsListener.java             |  25 ++--
 .../TestReplicationSourceManager.java           |   2 +-
 .../regionserver/TestWALEntryStream.java        |  13 +-
 .../apache/hadoop/hbase/wal/IOTestProvider.java |  48 ++++---
 .../wal/TestBoundedRegionGroupingStrategy.java  |  16 ++-
 .../hadoop/hbase/wal/TestFSHLogProvider.java    | 130 ++++++++-----------
 .../apache/hadoop/hbase/wal/TestSecureWAL.java  |  22 +---
 .../apache/hadoop/hbase/wal/TestWALFactory.java | 127 +++++++-----------
 .../apache/hadoop/hbase/wal/TestWALMethods.java |   2 +-
 .../hbase/wal/TestWALReaderOnSecureWAL.java     |  25 ++--
 .../apache/hadoop/hbase/wal/TestWALRootDir.java |  35 +++--
 .../apache/hadoop/hbase/wal/TestWALSplit.java   |  37 +++---
 .../hbase/wal/WALPerformanceEvaluation.java     |  47 ++++---
 37 files changed, 591 insertions(+), 710 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/71a1192d/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALRecordReader.java
----------------------------------------------------------------------
diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALRecordReader.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALRecordReader.java
index 18bb135..c8db903 100644
--- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALRecordReader.java
+++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALRecordReader.java
@@ -24,30 +24,28 @@ import static org.junit.Assert.assertTrue;
 import java.util.List;
 import java.util.NavigableMap;
 import java.util.TreeMap;
-import java.util.concurrent.atomic.AtomicLong;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
 import org.apache.hadoop.hbase.mapreduce.WALInputFormat.WALKeyRecordReader;
 import org.apache.hadoop.hbase.mapreduce.WALInputFormat.WALRecordReader;
 import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
-import org.apache.hadoop.hbase.wal.WALEdit;
+import org.apache.hadoop.hbase.testclassification.MapReduceTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.wal.WAL;
+import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.wal.WALFactory;
+import org.apache.hadoop.hbase.wal.WALKey;
 import org.apache.hadoop.hbase.wal.WALKeyImpl;
-import org.apache.hadoop.hbase.testclassification.MapReduceTests;
-import org.apache.hadoop.hbase.testclassification.MediumTests;
-import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.MapReduceTestUtil;
 import org.junit.AfterClass;
@@ -61,7 +59,7 @@ import org.slf4j.LoggerFactory;
 /**
  * JUnit tests for the WALRecordReader
  */
-@Category({MapReduceTests.class, MediumTests.class})
+@Category({ MapReduceTests.class, MediumTests.class })
 public class TestWALRecordReader {
   private static final Logger LOG = LoggerFactory.getLogger(TestWALRecordReader.class);
   private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
@@ -74,11 +72,9 @@ public class TestWALRecordReader {
   static final TableName tableName = TableName.valueOf(getName());
   private static final byte [] rowName = tableName.getName();
   // visible for TestHLogRecordReader
-  static final HRegionInfo info = new HRegionInfo(tableName,
-      Bytes.toBytes(""), Bytes.toBytes(""), false);
-  private static final byte [] family = Bytes.toBytes("column");
-  private static final byte [] value = Bytes.toBytes("value");
-  private static HTableDescriptor htd;
+  static final RegionInfo info = RegionInfoBuilder.newBuilder(tableName).build();
+  private static final byte[] family = Bytes.toBytes("column");
+  private static final byte[] value = Bytes.toBytes("value");
   private static Path logDir;
   protected MultiVersionConcurrencyControl mvcc;
   protected static NavigableMap<byte[], Integer> scopes = new TreeMap<>(Bytes.BYTES_COMPARATOR);
@@ -93,6 +89,7 @@ public class TestWALRecordReader {
     walFs.delete(walRootDir, true);
     mvcc = new MultiVersionConcurrencyControl();
   }
+
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
     // Make block sizes small.
@@ -108,9 +105,6 @@ public class TestWALRecordReader {
     walRootDir = TEST_UTIL.createWALRootDir();
     walFs = FSUtils.getWALFileSystem(conf);
     logDir = new Path(walRootDir, HConstants.HREGION_LOGDIR_NAME);
-
-    htd = new HTableDescriptor(tableName);
-    htd.addFamily(new HColumnDescriptor(family));
   }
 
   @AfterClass
@@ -127,7 +121,7 @@ public class TestWALRecordReader {
   @Test
   public void testPartialRead() throws Exception {
     final WALFactory walfactory = new WALFactory(conf, null, getName());
-    WAL log = walfactory.getWAL(info.getEncodedNameAsBytes(), info.getTable().getNamespace());
+    WAL log = walfactory.getWAL(info);
     // This test depends on timestamp being millisecond based and the filename of the WAL also
     // being millisecond based.
     long ts = System.currentTimeMillis();
@@ -186,9 +180,8 @@ public class TestWALRecordReader {
   @Test
   public void testWALRecordReader() throws Exception {
     final WALFactory walfactory = new WALFactory(conf, null, getName());
-    WAL log = walfactory.getWAL(info.getEncodedNameAsBytes(), info.getTable().getNamespace());
+    WAL log = walfactory.getWAL(info);
     byte [] value = Bytes.toBytes("value");
-    final AtomicLong sequenceId = new AtomicLong(0);
     WALEdit edit = new WALEdit();
     edit.add(new KeyValue(rowName, family, Bytes.toBytes("1"),
         System.currentTimeMillis(), value));
@@ -245,7 +238,7 @@ public class TestWALRecordReader {
     return new WALKeyImpl(info.getEncodedNameAsBytes(), tableName, time, mvcc, scopes);
   }
 
-  protected WALRecordReader getReader() {
+  private WALRecordReader<WALKey> getReader() {
     return new WALKeyRecordReader();
   }
 
@@ -253,7 +246,7 @@ public class TestWALRecordReader {
    * Create a new reader from the split, and match the edits against the passed columns.
    */
   private void testSplit(InputSplit split, byte[]... columns) throws Exception {
-    final WALRecordReader reader = getReader();
+    WALRecordReader<WALKey> reader = getReader();
     reader.initialize(split, MapReduceTestUtil.createDummyMapTaskAttemptContext(conf));
 
     for (byte[] column : columns) {
@@ -262,15 +255,12 @@ public class TestWALRecordReader {
       if (!Bytes.equals(column, 0, column.length, cell.getQualifierArray(),
         cell.getQualifierOffset(), cell.getQualifierLength())) {
         assertTrue(
-          "expected ["
-              + Bytes.toString(column)
-              + "], actual ["
-              + Bytes.toString(cell.getQualifierArray(), cell.getQualifierOffset(),
-                cell.getQualifierLength()) + "]", false);
+          "expected [" + Bytes.toString(column) + "], actual [" + Bytes.toString(
+            cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength()) + "]",
+          false);
       }
     }
     assertFalse(reader.nextKeyValue());
     reader.close();
   }
-
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/71a1192d/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index 449119d..0fda442 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -2116,8 +2116,6 @@ public class HRegionServer extends HasThread implements
     return healthy;
   }
 
-  private static final byte[] UNSPECIFIED_REGION = new byte[]{};
-
   @Override
   public List<WAL> getWALs() throws IOException {
     return walFactory.getWALs();
@@ -2125,17 +2123,7 @@ public class HRegionServer extends HasThread implements
 
   @Override
   public WAL getWAL(RegionInfo regionInfo) throws IOException {
-    WAL wal;
-    // _ROOT_ and hbase:meta regions have separate WAL.
-    if (regionInfo != null && regionInfo.isMetaRegion()
-        && regionInfo.getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID) {
-      wal = walFactory.getMetaWAL(regionInfo.getEncodedNameAsBytes());
-    } else if (regionInfo == null) {
-      wal = walFactory.getWAL(UNSPECIFIED_REGION, null);
-    } else {
-      byte[] namespace = regionInfo.getTable().getNamespace();
-      wal = walFactory.getWAL(regionInfo.getEncodedNameAsBytes(), namespace);
-    }
+    WAL wal = walFactory.getWAL(regionInfo);
     if (this.walRoller != null) {
       this.walRoller.addWAL(wal);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/71a1192d/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
index 8fa5f85..062e516 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
@@ -1482,9 +1482,9 @@ public class HBaseFsck extends Configured implements Closeable {
     // unless I pass along via the conf.
     Configuration confForWAL = new Configuration(c);
     confForWAL.set(HConstants.HBASE_DIR, rootdir.toString());
-    WAL wal = (new WALFactory(confForWAL,
-        Collections.<WALActionsListener> singletonList(new MetricsWAL()), walFactoryID))
-            .getWAL(metaHRI.getEncodedNameAsBytes(), metaHRI.getTable().getNamespace());
+    WAL wal =
+      new WALFactory(confForWAL, Collections.<WALActionsListener> singletonList(new MetricsWAL()),
+          walFactoryID).getWAL(metaHRI);
     HRegion meta = HRegion.createHRegion(metaHRI, rootdir, c, metaDescriptor, wal);
     MasterFileSystem.setInfoFamilyCachingForMeta(metaDescriptor, true);
     return meta;

http://git-wip-us.apache.org/repos/asf/hbase/blob/71a1192d/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AbstractFSWALProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AbstractFSWALProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AbstractFSWALProvider.java
index 103b05f..74d502e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AbstractFSWALProvider.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AbstractFSWALProvider.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.yetus.audience.InterfaceStability;
 import org.slf4j.Logger;
@@ -128,7 +129,7 @@ public abstract class AbstractFSWALProvider<T extends AbstractFSWAL<?>> implemen
   }
 
   @Override
-  public T getWAL(byte[] identifier, byte[] namespace) throws IOException {
+  public T getWAL(RegionInfo region) throws IOException {
     T walCopy = wal;
     if (walCopy == null) {
       // only lock when need to create wal, and need to lock since

http://git-wip-us.apache.org/repos/asf/hbase/blob/71a1192d/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DisabledWALProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DisabledWALProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DisabledWALProvider.java
index 280d95f..725f9ff 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DisabledWALProvider.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DisabledWALProvider.java
@@ -74,7 +74,7 @@ class DisabledWALProvider implements WALProvider {
   }
 
   @Override
-  public WAL getWAL(final byte[] identifier, byte[] namespace) throws IOException {
+  public WAL getWAL(RegionInfo region) throws IOException {
     return disabled;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/71a1192d/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/RegionGroupingProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/RegionGroupingProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/RegionGroupingProvider.java
index b8c9484..a3e54a5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/RegionGroupingProvider.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/RegionGroupingProvider.java
@@ -27,15 +27,17 @@ import java.util.Collections;
 import java.util.List;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
-
+import java.util.concurrent.locks.Lock;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.client.RegionInfo;
 // imports for classes still in regionserver.wal
 import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.IdLock;
+import org.apache.hadoop.hbase.util.KeyLocker;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * A WAL Provider that returns a WAL per group of regions.
@@ -131,7 +133,7 @@ public class RegionGroupingProvider implements WALProvider {
   /** A group-provider mapping, make sure one-one rather than many-one mapping */
   private final ConcurrentMap<String, WALProvider> cached = new ConcurrentHashMap<>();
 
-  private final IdLock createLock = new IdLock();
+  private final KeyLocker<String> createLock = new KeyLocker<>();
 
   private RegionGroupingStrategy strategy = null;
   private WALFactory factory = null;
@@ -177,33 +179,39 @@ public class RegionGroupingProvider implements WALProvider {
     return wals;
   }
 
-  private WAL getWAL(final String group) throws IOException {
+  private WAL getWAL(String group) throws IOException {
     WALProvider provider = cached.get(group);
     if (provider == null) {
-      IdLock.Entry lockEntry = null;
+      Lock lock = createLock.acquireLock(group);
       try {
-        lockEntry = createLock.getLockEntry(group.hashCode());
         provider = cached.get(group);
         if (provider == null) {
           provider = createProvider(group);
           cached.put(group, provider);
         }
       } finally {
-        if (lockEntry != null) {
-          createLock.releaseLockEntry(lockEntry);
-        }
+        lock.unlock();
       }
     }
-    return provider.getWAL(null, null);
+    return provider.getWAL(null);
   }
 
   @Override
-  public WAL getWAL(final byte[] identifier, byte[] namespace) throws IOException {
-    final String group;
+  public WAL getWAL(RegionInfo region) throws IOException {
+    String group;
     if (META_WAL_PROVIDER_ID.equals(this.providerId)) {
       group = META_WAL_GROUP_NAME;
     } else {
-      group = strategy.group(identifier, namespace);
+      byte[] id;
+      byte[] namespace;
+      if (region != null) {
+        id = region.getEncodedNameAsBytes();
+        namespace = region.getTable().getNamespace();
+      } else {
+        id = HConstants.EMPTY_BYTE_ARRAY;
+        namespace = null;
+      }
+      group = strategy.group(id, namespace);
     }
     return getWAL(group);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/71a1192d/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALFactory.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALFactory.java
index 1ed4168..d70b8cd 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALFactory.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALFactory.java
@@ -1,5 +1,4 @@
 /**
- *
  * 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
@@ -16,25 +15,18 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
-
 package org.apache.hadoop.hbase.wal;
 
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-
 import java.io.IOException;
 import java.io.InterruptedIOException;
 import java.util.Collections;
 import java.util.List;
 import java.util.OptionalLong;
 import java.util.concurrent.atomic.AtomicReference;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hbase.client.RegionInfo;
 // imports for things that haven't moved from regionserver.wal yet.
 import org.apache.hadoop.hbase.regionserver.wal.MetricsWAL;
 import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader;
@@ -45,6 +37,11 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.LeaseNotRecoveredException;
 import org.apache.hadoop.hbase.wal.WAL.Reader;
 import org.apache.hadoop.hbase.wal.WALProvider.Writer;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 
 /**
  * Entry point for users of the Write Ahead Log.
@@ -91,11 +88,11 @@ public class WALFactory implements WALFileLengthProvider {
   static final String DEFAULT_META_WAL_PROVIDER = Providers.defaultProvider.name();
 
   final String factoryId;
-  final WALProvider provider;
+  private final WALProvider provider;
   // The meta updates are written to a different wal. If this
   // regionserver holds meta regions, then this ref will be non-null.
   // lazily intialized; most RegionServers don't deal with META
-  final AtomicReference<WALProvider> metaProvider = new AtomicReference<>();
+  private final AtomicReference<WALProvider> metaProvider = new AtomicReference<>();
 
   /**
    * Configuration-specified WAL Reader used when a custom reader is requested
@@ -236,32 +233,35 @@ public class WALFactory implements WALFileLengthProvider {
     return provider.getWALs();
   }
 
-  /**
-   * @param identifier may not be null, contents will not be altered
-   * @param namespace could be null, and will use default namespace if null
-   */
-  public WAL getWAL(final byte[] identifier, final byte[] namespace) throws IOException {
-    return provider.getWAL(identifier, namespace);
+  private WALProvider getMetaProvider() throws IOException {
+    for (;;) {
+      WALProvider provider = this.metaProvider.get();
+      if (provider != null) {
+        return provider;
+      }
+      provider = getProvider(META_WAL_PROVIDER, DEFAULT_META_WAL_PROVIDER,
+        Collections.<WALActionsListener> singletonList(new MetricsWAL()),
+        AbstractFSWALProvider.META_WAL_PROVIDER_ID);
+      if (metaProvider.compareAndSet(null, provider)) {
+        return provider;
+      } else {
+        // someone is ahead of us, close and try again.
+        provider.close();
+      }
+    }
   }
 
   /**
-   * @param identifier may not be null, contents will not be altered
+   * @param region the region which we want to get a WAL for it. Could be null.
    */
-  public WAL getMetaWAL(final byte[] identifier) throws IOException {
-    WALProvider metaProvider = this.metaProvider.get();
-    if (null == metaProvider) {
-      final WALProvider temp = getProvider(META_WAL_PROVIDER, DEFAULT_META_WAL_PROVIDER,
-          Collections.<WALActionsListener>singletonList(new MetricsWAL()),
-          AbstractFSWALProvider.META_WAL_PROVIDER_ID);
-      if (this.metaProvider.compareAndSet(null, temp)) {
-        metaProvider = temp;
-      } else {
-        // reference must now be to a provider created in another thread.
-        temp.close();
-        metaProvider = this.metaProvider.get();
-      }
+  public WAL getWAL(RegionInfo region) throws IOException {
+    // use different WAL for hbase:meta
+    if (region != null && region.isMetaRegion() &&
+      region.getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID) {
+      return getMetaProvider().getWAL(region);
+    } else {
+      return provider.getWAL(region);
     }
-    return metaProvider.getWAL(identifier, null);
   }
 
   public Reader createReader(final FileSystem fs, final Path path) throws IOException {

http://git-wip-us.apache.org/repos/asf/hbase/blob/71a1192d/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALProvider.java
index c38f419..0586d1d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALProvider.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALProvider.java
@@ -22,11 +22,10 @@ import java.io.Closeable;
 import java.io.IOException;
 import java.util.List;
 import java.util.concurrent.CompletableFuture;
-
 import org.apache.hadoop.conf.Configuration;
-import org.apache.yetus.audience.InterfaceAudience;
-// imports for things that haven't moved from regionserver.wal yet.
+import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * The Write Ahead Log (WAL) stores all durable edits to the HRegion.
@@ -48,17 +47,17 @@ public interface WALProvider {
    * @param listeners may be null
    * @param providerId differentiate between providers from one factory. may be null
    */
-  void init(final WALFactory factory, final Configuration conf,
-      final List<WALActionsListener> listeners, final String providerId) throws IOException;
+  void init(WALFactory factory, Configuration conf, List<WALActionsListener> listeners,
+      String providerId) throws IOException;
 
   /**
-   * @param identifier may not be null. contents will not be altered.
-   * @param namespace could be null, and will use default namespace if null
+   * @param region the region which we want to get a WAL for it. Could be null.
    * @return a WAL for writing entries for the given region.
    */
-  WAL getWAL(final byte[] identifier, byte[] namespace) throws IOException;
+  WAL getWAL(RegionInfo region) throws IOException;
 
-  /** @return the List of WALs that are used by this server
+  /**
+   * @return the List of WALs that are used by this server
    */
   List<WAL> getWALs();
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/71a1192d/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 304b3cb..fe8902a 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
@@ -2310,9 +2310,8 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
     Configuration confForWAL = new Configuration(conf);
     confForWAL.set(HConstants.HBASE_DIR, rootDir.toString());
     return (new WALFactory(confForWAL,
-        Collections.<WALActionsListener>singletonList(new MetricsWAL()),
-        "hregion-" + RandomStringUtils.randomNumeric(8))).
-        getWAL(hri.getEncodedNameAsBytes(), hri.getTable().getNamespace());
+        Collections.<WALActionsListener> singletonList(new MetricsWAL()),
+        "hregion-" + RandomStringUtils.randomNumeric(8))).getWAL(hri);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/71a1192d/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestWALObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestWALObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestWALObserver.java
index a6628bb..166dfdd 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestWALObserver.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestWALObserver.java
@@ -1,5 +1,4 @@
-/*
- *
+/**
  * 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
@@ -16,7 +15,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.hadoop.hbase.coprocessor;
 
 import static org.junit.Assert.assertEquals;
@@ -31,7 +29,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.NavigableMap;
 import java.util.TreeMap;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -40,18 +37,19 @@ import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.Coprocessor;
 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.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
+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.RegionInfoBuilder;
+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.MultiVersionConcurrencyControl;
 import org.apache.hadoop.hbase.regionserver.wal.WALCoprocessorHost;
-import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.testclassification.CoprocessorTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -61,6 +59,7 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
 import org.apache.hadoop.hbase.wal.WAL;
+import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.wal.WALFactory;
 import org.apache.hadoop.hbase.wal.WALKeyImpl;
 import org.apache.hadoop.hbase.wal.WALSplitter;
@@ -172,17 +171,17 @@ public class TestWALObserver {
    */
   @Test
   public void testWALObserverWriteToWAL() throws Exception {
-    final WAL log = wals.getWAL(UNSPECIFIED_REGION, null);
+    final WAL log = wals.getWAL(null);
     verifyWritesSeen(log, getCoprocessor(log, SampleRegionWALCoprocessor.class), false);
   }
 
   private void verifyWritesSeen(final WAL log, final SampleRegionWALCoprocessor cp,
       final boolean seesLegacy) throws Exception {
-    HRegionInfo hri = createBasic3FamilyHRegionInfo(Bytes.toString(TEST_TABLE));
-    final HTableDescriptor htd = createBasic3FamilyHTD(Bytes
+    RegionInfo hri = createBasicHRegionInfo(Bytes.toString(TEST_TABLE));
+    TableDescriptor htd = createBasic3FamilyHTD(Bytes
         .toString(TEST_TABLE));
     NavigableMap<byte[], Integer> scopes = new TreeMap<>(Bytes.BYTES_COMPARATOR);
-    for(byte[] fam : htd.getFamiliesKeys()) {
+    for (byte[] fam : htd.getColumnFamilyNames()) {
       scopes.put(fam, 0);
     }
     Path basedir = new Path(this.hbaseRootDir, Bytes.toString(TEST_TABLE));
@@ -268,14 +267,14 @@ public class TestWALObserver {
    */
   @Test
   public void testEmptyWALEditAreNotSeen() throws Exception {
-    final HRegionInfo hri = createBasic3FamilyHRegionInfo(Bytes.toString(TEST_TABLE));
-    final HTableDescriptor htd = createBasic3FamilyHTD(Bytes.toString(TEST_TABLE));
-    final MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl();
+    RegionInfo hri = createBasicHRegionInfo(Bytes.toString(TEST_TABLE));
+    TableDescriptor htd = createBasic3FamilyHTD(Bytes.toString(TEST_TABLE));
+    MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl();
     NavigableMap<byte[], Integer> scopes = new TreeMap<>(Bytes.BYTES_COMPARATOR);
-    for(byte[] fam : htd.getFamiliesKeys()) {
+    for(byte[] fam : htd.getColumnFamilyNames()) {
       scopes.put(fam, 0);
     }
-    WAL log = wals.getWAL(UNSPECIFIED_REGION, null);
+    WAL log = wals.getWAL(null);
     try {
       SampleRegionWALCoprocessor cp = getCoprocessor(log, SampleRegionWALCoprocessor.class);
 
@@ -304,14 +303,14 @@ public class TestWALObserver {
   public void testWALCoprocessorReplay() throws Exception {
     // WAL replay is handled at HRegion::replayRecoveredEdits(), which is
     // ultimately called by HRegion::initialize()
-    final TableName tableName = TableName.valueOf(currentTest.getMethodName());
-    final HTableDescriptor htd = getBasic3FamilyHTableDescriptor(tableName);
+    TableName tableName = TableName.valueOf(currentTest.getMethodName());
+    TableDescriptor htd = getBasic3FamilyHTableDescriptor(tableName);
     MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl();
     // final HRegionInfo hri =
     // createBasic3FamilyHRegionInfo(Bytes.toString(tableName));
     // final HRegionInfo hri1 =
     // createBasic3FamilyHRegionInfo(Bytes.toString(tableName));
-    final HRegionInfo hri = new HRegionInfo(tableName, null, null);
+    RegionInfo hri = RegionInfoBuilder.newBuilder(tableName).build();
 
     final Path basedir =
         FSUtils.getTableDir(this.hbaseRootDir, tableName);
@@ -321,21 +320,21 @@ public class TestWALObserver {
     final Configuration newConf = HBaseConfiguration.create(this.conf);
 
     // WAL wal = new WAL(this.fs, this.dir, this.oldLogDir, this.conf);
-    WAL wal = wals.getWAL(UNSPECIFIED_REGION, null);
+    WAL wal = wals.getWAL(null);
     // Put p = creatPutWith2Families(TEST_ROW);
     WALEdit edit = new WALEdit();
     long now = EnvironmentEdgeManager.currentTime();
     final int countPerFamily = 1000;
     NavigableMap<byte[], Integer> scopes = new TreeMap<>(Bytes.BYTES_COMPARATOR);
-    for (HColumnDescriptor hcd : htd.getFamilies()) {
-      scopes.put(hcd.getName(), 0);
+    for (byte[] fam : htd.getColumnFamilyNames()) {
+      scopes.put(fam, 0);
     }
-    for (HColumnDescriptor hcd : htd.getFamilies()) {
-      addWALEdits(tableName, hri, TEST_ROW, hcd.getName(), countPerFamily,
-          EnvironmentEdgeManager.getDelegate(), wal, scopes, mvcc);
+    for (byte[] fam : htd.getColumnFamilyNames()) {
+      addWALEdits(tableName, hri, TEST_ROW, fam, countPerFamily,
+        EnvironmentEdgeManager.getDelegate(), wal, scopes, mvcc);
     }
     wal.append(hri, new WALKeyImpl(hri.getEncodedNameAsBytes(), tableName, now, mvcc, scopes), edit,
-        true);
+      true);
     // sync to fs.
     wal.sync();
 
@@ -345,14 +344,12 @@ public class TestWALObserver {
       public Void run() throws Exception {
         Path p = runWALSplit(newConf);
         LOG.info("WALSplit path == " + p);
-        FileSystem newFS = FileSystem.get(newConf);
         // Make a new wal for new region open.
         final WALFactory wals2 = new WALFactory(conf, null,
             ServerName.valueOf(currentTest.getMethodName()+"2", 16010, System.currentTimeMillis()).toString());
-        WAL wal2 = wals2.getWAL(UNSPECIFIED_REGION, null);
+        WAL wal2 = wals2.getWAL(null);
         HRegion region = HRegion.openHRegion(newConf, FileSystem.get(newConf), hbaseRootDir,
             hri, htd, wal2, TEST_UTIL.getHBaseCluster().getRegionServer(0), null);
-        long seqid2 = region.getOpenSeqNum();
 
         SampleRegionWALCoprocessor cp2 =
           region.getCoprocessorHost().findCoprocessor(SampleRegionWALCoprocessor.class);
@@ -374,13 +371,13 @@ public class TestWALObserver {
    */
   @Test
   public void testWALObserverLoaded() throws Exception {
-    WAL log = wals.getWAL(UNSPECIFIED_REGION, null);
+    WAL log = wals.getWAL(null);
     assertNotNull(getCoprocessor(log, SampleRegionWALCoprocessor.class));
   }
 
   @Test
   public void testWALObserverRoll() throws Exception {
-    final WAL wal = wals.getWAL(UNSPECIFIED_REGION, null);
+    final WAL wal = wals.getWAL(null);
     final SampleRegionWALCoprocessor cp = getCoprocessor(wal, SampleRegionWALCoprocessor.class);
     cp.setTestValues(TEST_TABLE, null, null, null, null, null, null, null);
 
@@ -399,20 +396,12 @@ public class TestWALObserver {
     return (SampleRegionWALCoprocessor) c;
   }
 
-  /*
-   * Creates an HRI around an HTD that has <code>tableName</code> and three
-   * column families named.
-   *
-   * @param tableName Name of table to use when we create HTableDescriptor.
+  /**
+   * Creates an HRI around an HTD that has <code>tableName</code>.
+   * @param tableName Name of table to use.
    */
-  private HRegionInfo createBasic3FamilyHRegionInfo(final String tableName) {
-    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName));
-
-    for (int i = 0; i < TEST_FAMILY.length; i++) {
-      HColumnDescriptor a = new HColumnDescriptor(TEST_FAMILY[i]);
-      htd.addFamily(a);
-    }
-    return new HRegionInfo(htd.getTableName(), null, null, false);
+  private RegionInfo createBasicHRegionInfo(String tableName) {
+    return RegionInfoBuilder.newBuilder(TableName.valueOf(tableName)).build();
   }
 
   /*
@@ -463,12 +452,10 @@ public class TestWALObserver {
     return splits.get(0);
   }
 
-  private static final byte[] UNSPECIFIED_REGION = new byte[]{};
-
-  private void addWALEdits(final TableName tableName, final HRegionInfo hri, final byte[] rowName,
+  private void addWALEdits(final TableName tableName, final RegionInfo hri, final byte[] rowName,
       final byte[] family, final int count, EnvironmentEdge ee, final WAL wal,
       final NavigableMap<byte[], Integer> scopes, final MultiVersionConcurrencyControl mvcc)
-          throws IOException {
+      throws IOException {
     String familyStr = Bytes.toString(family);
     long txid = -1;
     for (int j = 0; j < count; j++) {
@@ -478,33 +465,25 @@ public class TestWALObserver {
       edit.add(new KeyValue(rowName, family, qualifierBytes, ee.currentTime(), columnBytes));
       // uses WALKeyImpl instead of HLogKey on purpose. will only work for tests where we don't care
       // about legacy coprocessors
-      txid = wal.append(hri, new WALKeyImpl(hri.getEncodedNameAsBytes(), tableName,
-          ee.currentTime(), mvcc), edit, true);
+      txid = wal.append(hri,
+        new WALKeyImpl(hri.getEncodedNameAsBytes(), tableName, ee.currentTime(), mvcc), edit, true);
     }
     if (-1 != txid) {
       wal.sync(txid);
     }
   }
 
-  private HTableDescriptor getBasic3FamilyHTableDescriptor(
-      final TableName tableName) {
-    HTableDescriptor htd = new HTableDescriptor(tableName);
-
-    for (int i = 0; i < TEST_FAMILY.length; i++) {
-      HColumnDescriptor a = new HColumnDescriptor(TEST_FAMILY[i]);
-      htd.addFamily(a);
-    }
-    return htd;
+  private TableDescriptor getBasic3FamilyHTableDescriptor(TableName tableName) {
+    TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableName);
+    Arrays.stream(TEST_FAMILY).map(ColumnFamilyDescriptorBuilder::of)
+        .forEachOrdered(builder::addColumnFamily);
+    return builder.build();
   }
 
-  private HTableDescriptor createBasic3FamilyHTD(final String tableName) {
-    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName));
-    HColumnDescriptor a = new HColumnDescriptor(Bytes.toBytes("a"));
-    htd.addFamily(a);
-    HColumnDescriptor b = new HColumnDescriptor(Bytes.toBytes("b"));
-    htd.addFamily(b);
-    HColumnDescriptor c = new HColumnDescriptor(Bytes.toBytes("c"));
-    htd.addFamily(c);
-    return htd;
+  private TableDescriptor createBasic3FamilyHTD(String tableName) {
+    return TableDescriptorBuilder.newBuilder(TableName.valueOf(tableName))
+        .addColumnFamily(ColumnFamilyDescriptorBuilder.of("a"))
+        .addColumnFamily(ColumnFamilyDescriptorBuilder.of("b"))
+        .addColumnFamily(ColumnFamilyDescriptorBuilder.of("c")).build();
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/71a1192d/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java
index 02a21b6..0f4ece6 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java
@@ -26,16 +26,18 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
 import java.util.Random;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.KeyValue;
 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.fs.HFileSystem;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
 import org.apache.hadoop.hbase.io.hfile.BlockCache;
@@ -106,18 +108,19 @@ public class TestCacheOnWriteInSchema {
       return blockType == blockType1 || blockType == blockType2;
     }
 
-    public void modifyFamilySchema(HColumnDescriptor family) {
+    public ColumnFamilyDescriptorBuilder modifyFamilySchema(ColumnFamilyDescriptorBuilder builder) {
       switch (this) {
-      case DATA_BLOCKS:
-        family.setCacheDataOnWrite(true);
-        break;
-      case BLOOM_BLOCKS:
-        family.setCacheBloomsOnWrite(true);
-        break;
-      case INDEX_BLOCKS:
-        family.setCacheIndexesOnWrite(true);
-        break;
+        case DATA_BLOCKS:
+          builder.setCacheDataOnWrite(true);
+          break;
+        case BLOOM_BLOCKS:
+          builder.setCacheBloomsOnWrite(true);
+          break;
+        case INDEX_BLOCKS:
+          builder.setCacheIndexesOnWrite(true);
+          break;
       }
+      return builder;
     }
   }
 
@@ -158,23 +161,22 @@ public class TestCacheOnWriteInSchema {
     fs = HFileSystem.get(conf);
 
     // Create the schema
-    HColumnDescriptor hcd = new HColumnDescriptor(family);
-    hcd.setBloomFilterType(BloomType.ROWCOL);
-    cowType.modifyFamilySchema(hcd);
-    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(table));
-    htd.addFamily(hcd);
+    ColumnFamilyDescriptor hcd = cowType
+        .modifyFamilySchema(
+          ColumnFamilyDescriptorBuilder.newBuilder(family).setBloomFilterType(BloomType.ROWCOL))
+        .build();
+    TableDescriptor htd =
+        TableDescriptorBuilder.newBuilder(TableName.valueOf(table)).addColumnFamily(hcd).build();
 
     // Create a store based on the schema
-    final String id = TestCacheOnWriteInSchema.class.getName();
-    final Path logdir = new Path(FSUtils.getRootDir(conf),
-      AbstractFSWALProvider.getWALDirectoryName(id));
+    String id = TestCacheOnWriteInSchema.class.getName();
+    Path logdir = new Path(FSUtils.getRootDir(conf), AbstractFSWALProvider.getWALDirectoryName(id));
     fs.delete(logdir, true);
 
-    HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false);
+    RegionInfo info = RegionInfoBuilder.newBuilder(htd.getTableName()).build();
     walFactory = new WALFactory(conf, null, id);
 
-    region = TEST_UTIL.createLocalHRegion(info, htd,
-        walFactory.getWAL(info.getEncodedNameAsBytes(), info.getTable().getNamespace()));
+    region = TEST_UTIL.createLocalHRegion(info, htd, walFactory.getWAL(info));
     store = new HStore(region, hcd, conf);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/71a1192d/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveConcurrentClose.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveConcurrentClose.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveConcurrentClose.java
index 95dec3d..d2e1866 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveConcurrentClose.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveConcurrentClose.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.regionserver;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
 
 import java.io.IOException;
 import java.io.InterruptedIOException;
@@ -29,17 +28,18 @@ import java.util.Collection;
 import java.util.List;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicReference;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.Stoppable;
 import org.apache.hadoop.hbase.TableName;
+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.RegionInfoBuilder;
+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;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -85,10 +85,10 @@ public class TestCompactionArchiveConcurrentClose {
     byte[] col = Bytes.toBytes("c");
     byte[] val = Bytes.toBytes("val");
 
-    final TableName tableName = TableName.valueOf(name.getMethodName());
-    HTableDescriptor htd = new HTableDescriptor(tableName);
-    htd.addFamily(new HColumnDescriptor(fam));
-    HRegionInfo info = new HRegionInfo(tableName, null, null, false);
+    TableName tableName = TableName.valueOf(name.getMethodName());
+    TableDescriptor htd = TableDescriptorBuilder.newBuilder(tableName)
+        .addColumnFamily(ColumnFamilyDescriptorBuilder.of(fam)).build();
+    RegionInfo info = RegionInfoBuilder.newBuilder(tableName).build();
     HRegion region = initHRegion(htd, info);
     RegionServerServices rss = mock(RegionServerServices.class);
     List<HRegion> regions = new ArrayList<>();
@@ -157,20 +157,17 @@ public class TestCompactionArchiveConcurrentClose {
     }
   }
 
-  private HRegion initHRegion(HTableDescriptor htd, HRegionInfo info)
-      throws IOException {
+  private HRegion initHRegion(TableDescriptor htd, RegionInfo info) throws IOException {
     Configuration conf = testUtil.getConfiguration();
     Path tableDir = FSUtils.getTableDir(testDir, htd.getTableName());
 
-    HRegionFileSystem fs = new WaitingHRegionFileSystem(conf, tableDir.getFileSystem(conf),
-        tableDir, info);
+    HRegionFileSystem fs =
+        new WaitingHRegionFileSystem(conf, tableDir.getFileSystem(conf), tableDir, info);
     ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     final Configuration walConf = new Configuration(conf);
     FSUtils.setRootDir(walConf, tableDir);
     final WALFactory wals = new WALFactory(walConf, null, "log_" + info.getEncodedName());
-    HRegion region =
-        new HRegion(fs, wals.getWAL(info.getEncodedNameAsBytes(), info.getTable().getNamespace()),
-            conf, htd, null);
+    HRegion region = new HRegion(fs, wals.getWAL(info), conf, htd, null);
 
     region.initialize();
 
@@ -180,7 +177,7 @@ public class TestCompactionArchiveConcurrentClose {
   private class WaitingHRegionFileSystem extends HRegionFileSystem {
 
     public WaitingHRegionFileSystem(final Configuration conf, final FileSystem fs,
-        final Path tableDir, final HRegionInfo regionInfo) {
+        final Path tableDir, final RegionInfo regionInfo) {
       super(conf, fs, tableDir, regionInfo);
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/71a1192d/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveIOException.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveIOException.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveIOException.java
index 2b555a5..9c085e5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveIOException.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveIOException.java
@@ -1,4 +1,4 @@
-/*
+/**
  * 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
@@ -15,38 +15,36 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.hadoop.hbase.regionserver;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.eq;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.eq;
 import static org.mockito.Mockito.doThrow;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.spy;
 
-import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList;
-
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.Stoppable;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.backup.FailedArchiveException;
+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.RegionInfoBuilder;
+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.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
@@ -59,6 +57,8 @@ import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
 import org.mockito.Mockito;
 
+import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList;
+
 /**
  * Tests that archiving compacted files behaves correctly when encountering exceptions.
  */
@@ -93,11 +93,11 @@ public class TestCompactionArchiveIOException {
     byte[] col = Bytes.toBytes("c");
     byte[] val = Bytes.toBytes("val");
 
-    final TableName tableName = TableName.valueOf(name.getMethodName());
-    HTableDescriptor htd = new HTableDescriptor(tableName);
-    htd.addFamily(new HColumnDescriptor(fam));
-    HRegionInfo info = new HRegionInfo(tableName, null, null, false);
-    final HRegion region = initHRegion(htd, info);
+    TableName tableName = TableName.valueOf(name.getMethodName());
+    TableDescriptor htd = TableDescriptorBuilder.newBuilder(tableName)
+        .addColumnFamily(ColumnFamilyDescriptorBuilder.of(fam)).build();
+    RegionInfo info = RegionInfoBuilder.newBuilder(tableName).build();
+    HRegion region = initHRegion(htd, info);
     RegionServerServices rss = mock(RegionServerServices.class);
     List<HRegion> regions = new ArrayList<>();
     regions.add(region);
@@ -172,29 +172,25 @@ public class TestCompactionArchiveIOException {
     }
   }
 
-  private HRegion initHRegion(HTableDescriptor htd, HRegionInfo info)
-      throws IOException {
+  private HRegion initHRegion(TableDescriptor htd, RegionInfo info) throws IOException {
     Configuration conf = testUtil.getConfiguration();
     ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     Path tableDir = FSUtils.getTableDir(testDir, htd.getTableName());
     Path regionDir = new Path(tableDir, info.getEncodedName());
     Path storeDir = new Path(regionDir, htd.getColumnFamilies()[0].getNameAsString());
 
-
     FileSystem errFS = spy(testUtil.getTestFileSystem());
     // Prior to HBASE-16964, when an exception is thrown archiving any compacted file,
     // none of the other files are cleared from the compactedfiles list.
     // Simulate this condition with a dummy file
-    doThrow(new IOException("Error for test"))
-        .when(errFS).rename(eq(new Path(storeDir, ERROR_FILE)), any());
+    doThrow(new IOException("Error for test")).when(errFS)
+        .rename(eq(new Path(storeDir, ERROR_FILE)), any());
 
     HRegionFileSystem fs = new HRegionFileSystem(conf, errFS, tableDir, info);
     final Configuration walConf = new Configuration(conf);
     FSUtils.setRootDir(walConf, tableDir);
     final WALFactory wals = new WALFactory(walConf, null, "log_" + info.getEncodedName());
-    HRegion region =
-        new HRegion(fs, wals.getWAL(info.getEncodedNameAsBytes(), info.getTable().getNamespace()),
-            conf, htd, null);
+    HRegion region = new HRegion(fs, wals.getWAL(info), conf, htd, null);
 
     region.initialize();
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/71a1192d/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java
index 7772f7b..b89fb0e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java
@@ -30,7 +30,6 @@ import java.util.List;
 import java.util.Objects;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.atomic.AtomicReference;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.CategoryBasedTimeout;
@@ -39,18 +38,19 @@ import org.apache.hadoop.hbase.CellComparatorImpl;
 import org.apache.hadoop.hbase.CellUtil;
 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.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.KeepDeletedCells;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValueTestUtil;
 import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.TableName;
+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.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.UnexpectedStateException;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
@@ -68,6 +68,7 @@ import org.junit.rules.TestName;
 import org.junit.rules.TestRule;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hbase.thirdparty.com.google.common.base.Joiner;
 import org.apache.hbase.thirdparty.com.google.common.collect.Iterables;
 import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
@@ -451,22 +452,16 @@ public class TestDefaultMemStore {
 
     final MultiVersionConcurrencyControl mvcc;
     final MemStore memstore;
-    final AtomicLong startSeqNum;
 
     AtomicReference<Throwable> caughtException;
 
 
-    public ReadOwnWritesTester(int id,
-                               MemStore memstore,
-                               MultiVersionConcurrencyControl mvcc,
-                               AtomicReference<Throwable> caughtException,
-                               AtomicLong startSeqNum)
-    {
+    public ReadOwnWritesTester(int id, MemStore memstore, MultiVersionConcurrencyControl mvcc,
+        AtomicReference<Throwable> caughtException) {
       this.mvcc = mvcc;
       this.memstore = memstore;
       this.caughtException = caughtException;
       row = Bytes.toBytes(id);
-      this.startSeqNum = startSeqNum;
     }
 
     @Override
@@ -505,14 +500,13 @@ public class TestDefaultMemStore {
 
   @Test
   public void testReadOwnWritesUnderConcurrency() throws Throwable {
-
     int NUM_THREADS = 8;
 
     ReadOwnWritesTester threads[] = new ReadOwnWritesTester[NUM_THREADS];
     AtomicReference<Throwable> caught = new AtomicReference<>();
 
     for (int i = 0; i < NUM_THREADS; i++) {
-      threads[i] = new ReadOwnWritesTester(i, memstore, mvcc, caught, this.startSeqNum);
+      threads[i] = new ReadOwnWritesTester(i, memstore, mvcc, caught);
       threads[i].start();
     }
 
@@ -921,7 +915,8 @@ public class TestDefaultMemStore {
       EnvironmentEdgeManager.injectEdge(edge);
       HBaseTestingUtility hbaseUtility = HBaseTestingUtility.createLocalHTU(conf);
       String cf = "foo";
-      HRegion region = hbaseUtility.createTestRegion("foobar", new HColumnDescriptor(cf));
+      HRegion region =
+          hbaseUtility.createTestRegion("foobar", ColumnFamilyDescriptorBuilder.of(cf));
 
       edge.setCurrentTimeMillis(1234);
       Put p = new Put(Bytes.toBytes("r"));
@@ -950,20 +945,16 @@ public class TestDefaultMemStore {
     EnvironmentEdgeManager.injectEdge(edge);
     edge.setCurrentTimeMillis(1234);
     WALFactory wFactory = new WALFactory(conf, null, "1234");
-    HRegion meta = HRegion.createHRegion(HRegionInfo.FIRST_META_REGIONINFO, testDir,
+    HRegion meta = HRegion.createHRegion(RegionInfoBuilder.FIRST_META_REGIONINFO, testDir,
         conf, FSTableDescriptors.createMetaTableDescriptor(conf),
-        wFactory.getMetaWAL(HRegionInfo.FIRST_META_REGIONINFO.
-            getEncodedNameAsBytes()));
+        wFactory.getWAL(RegionInfoBuilder.FIRST_META_REGIONINFO));
     // parameterized tests add [#] suffix get rid of [ and ].
-    HRegionInfo hri =
-        new HRegionInfo(TableName.valueOf(name.getMethodName().replaceAll("[\\[\\]]", "_")),
-        Bytes.toBytes("row_0200"), Bytes.toBytes("row_0300"));
-    HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(
-        name.getMethodName().replaceAll("[\\[\\]]", "_")));
-    desc.addFamily(new HColumnDescriptor("foo".getBytes()));
-    HRegion r =
-        HRegion.createHRegion(hri, testDir, conf, desc,
-            wFactory.getWAL(hri.getEncodedNameAsBytes(), hri.getTable().getNamespace()));
+    TableDescriptor desc = TableDescriptorBuilder
+        .newBuilder(TableName.valueOf(name.getMethodName().replaceAll("[\\[\\]]", "_")))
+        .addColumnFamily(ColumnFamilyDescriptorBuilder.of("foo")).build();
+    RegionInfo hri = RegionInfoBuilder.newBuilder(desc.getTableName())
+        .setStartKey(Bytes.toBytes("row_0200")).setEndKey(Bytes.toBytes("row_0300")).build();
+    HRegion r = HRegion.createHRegion(hri, testDir, conf, desc, wFactory.getWAL(hri));
     addRegionToMETA(meta, r);
     edge.setCurrentTimeMillis(1234 + 100);
     StringBuilder sb = new StringBuilder();

http://git-wip-us.apache.org/repos/asf/hbase/blob/71a1192d/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java
index 32e3856..9845d8a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java
@@ -162,8 +162,7 @@ public class TestHMobStore {
     final Configuration walConf = new Configuration(conf);
     FSUtils.setRootDir(walConf, basedir);
     final WALFactory wals = new WALFactory(walConf, null, methodName);
-    region = new HRegion(tableDir, wals.getWAL(info.getEncodedNameAsBytes(),
-            info.getTable().getNamespace()), fs, conf, info, htd, null);
+    region = new HRegion(tableDir, wals.getWAL(info), fs, conf, info, htd, null);
     store = new HMobStore(region, hcd, conf);
     if(testStore) {
       init(conf, hcd);

http://git-wip-us.apache.org/repos/asf/hbase/blob/71a1192d/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 5ddd4df..3c11b31 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
@@ -63,7 +63,6 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReference;
-
 import org.apache.commons.lang3.RandomStringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
@@ -110,6 +109,7 @@ import org.apache.hadoop.hbase.client.Increment;
 import org.apache.hadoop.hbase.client.Mutation;
 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.Result;
 import org.apache.hadoop.hbase.client.RowMutations;
 import org.apache.hadoop.hbase.client.Scan;
@@ -180,8 +180,10 @@ import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
 import org.apache.hbase.thirdparty.com.google.protobuf.ByteString;
+
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor;
@@ -378,9 +380,8 @@ public class TestHRegion {
     final Path logDir = TEST_UTIL.getDataTestDirOnTestFS(callingMethod + ".log");
     final Configuration walConf = new Configuration(conf);
     FSUtils.setRootDir(walConf, logDir);
-    return (new WALFactory(walConf,
-        Collections.<WALActionsListener>singletonList(new MetricsWAL()), callingMethod))
-        .getWAL(tableName.toBytes(), tableName.getNamespace());
+    return new WALFactory(walConf, Collections.<WALActionsListener> singletonList(new MetricsWAL()),
+        callingMethod).getWAL(RegionInfoBuilder.newBuilder(tableName).build());
   }
 
   @Test
@@ -960,7 +961,7 @@ public class TestHRegion {
     final Configuration walConf = new Configuration(TEST_UTIL.getConfiguration());
     FSUtils.setRootDir(walConf, logDir);
     final WALFactory wals = new WALFactory(walConf, null, method);
-    final WAL wal = wals.getWAL(tableName.getName(), tableName.getNamespace());
+    final WAL wal = wals.getWAL(RegionInfoBuilder.newBuilder(tableName).build());
 
     this.region = initHRegion(tableName, HConstants.EMPTY_START_ROW,
       HConstants.EMPTY_END_ROW, false, Durability.USE_DEFAULT, wal, family);
@@ -4690,7 +4691,7 @@ public class TestHRegion {
     // deal with classes which have a field of an inner class. See discussions in HBASE-15536.
     walConf.set(WALFactory.WAL_PROVIDER, "filesystem");
     final WALFactory wals = new WALFactory(walConf, null, UUID.randomUUID().toString());
-    final WAL wal = spy(wals.getWAL(tableName.getName(), tableName.getNamespace()));
+    final WAL wal = spy(wals.getWAL(RegionInfoBuilder.newBuilder(tableName).build()));
     this.region = initHRegion(tableName, HConstants.EMPTY_START_ROW,
         HConstants.EMPTY_END_ROW, false, tableDurability, wal,
         new byte[][] { family });

http://git-wip-us.apache.org/repos/asf/hbase/blob/71a1192d/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java
index 7a2438b..d5e7e75 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java
@@ -1,4 +1,4 @@
-/*
+/**
  * 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
@@ -15,7 +15,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.hadoop.hbase.regionserver;
 
 import static org.apache.hadoop.hbase.regionserver.TestHRegion.assertGet;
@@ -27,8 +26,8 @@ import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.anyBoolean;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyBoolean;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.times;
@@ -43,7 +42,6 @@ import java.util.Map;
 import java.util.Objects;
 import java.util.Random;
 import java.util.UUID;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.Path;
@@ -51,7 +49,6 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
@@ -59,6 +56,8 @@ import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.Get;
 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.executor.ExecutorService;
@@ -89,8 +88,10 @@ import org.junit.rules.TestName;
 import org.mockito.Mockito;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
 import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
+
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.MutationType;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.BulkLoadDescriptor;
@@ -132,7 +133,7 @@ public class TestHRegionReplayEvents {
   private TableDescriptor htd;
   private long time;
   private RegionServerServices rss;
-  private HRegionInfo primaryHri, secondaryHri;
+  private RegionInfo primaryHri, secondaryHri;
   private HRegion primaryRegion, secondaryRegion;
   private WALFactory wals;
   private WAL walPrimary, walSecondary;
@@ -156,18 +157,14 @@ public class TestHRegionReplayEvents {
 
     time = System.currentTimeMillis();
     ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
-    primaryHri = new HRegionInfo(htd.getTableName(),
-      HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW,
-      false, time, 0);
-    secondaryHri = new HRegionInfo(htd.getTableName(),
-      HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW,
-      false, time, 1);
+    primaryHri =
+        RegionInfoBuilder.newBuilder(htd.getTableName()).setRegionId(time).setReplicaId(0).build();
+    secondaryHri =
+        RegionInfoBuilder.newBuilder(htd.getTableName()).setRegionId(time).setReplicaId(1).build();
 
     wals = TestHRegion.createWALFactory(CONF, rootDir);
-    walPrimary = wals.getWAL(primaryHri.getEncodedNameAsBytes(),
-        primaryHri.getTable().getNamespace());
-    walSecondary = wals.getWAL(secondaryHri.getEncodedNameAsBytes(),
-        secondaryHri.getTable().getNamespace());
+    walPrimary = wals.getWAL(primaryHri);
+    walSecondary = wals.getWAL(secondaryHri);
 
     rss = mock(RegionServerServices.class);
     when(rss.getServerName()).thenReturn(ServerName.valueOf("foo", 1, 1));
@@ -1150,8 +1147,8 @@ public class TestHRegionReplayEvents {
 
     // test for region open and close
     secondaryRegion = HRegion.openHRegion(secondaryHri, htd, walSecondary, CONF, rss, null);
-    verify(walSecondary, times(0)).append((HRegionInfo)any(),
-      (WALKeyImpl)any(), (WALEdit)any(),  anyBoolean());
+    verify(walSecondary, times(0)).append(any(RegionInfo.class), any(WALKeyImpl.class),
+      any(WALEdit.class), anyBoolean());
 
     // test for replay prepare flush
     putDataByReplay(secondaryRegion, 0, 10, cq, families);
@@ -1166,12 +1163,12 @@ public class TestHRegionReplayEvents {
           primaryRegion.getRegionInfo().getRegionName()))
       .build());
 
-    verify(walSecondary, times(0)).append((HRegionInfo)any(),
-      (WALKeyImpl)any(), (WALEdit)any(), anyBoolean());
+    verify(walSecondary, times(0)).append(any(RegionInfo.class), any(WALKeyImpl.class),
+      any(WALEdit.class), anyBoolean());
 
     secondaryRegion.close();
-    verify(walSecondary, times(0)).append((HRegionInfo)any(),
-      (WALKeyImpl)any(), (WALEdit)any(),  anyBoolean());
+    verify(walSecondary, times(0)).append(any(RegionInfo.class), any(WALKeyImpl.class),
+      any(WALEdit.class), anyBoolean());
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/71a1192d/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java
index e28e484..b8d3ec7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java
@@ -21,7 +21,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 static org.mockito.Matchers.any;
+import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.times;
@@ -47,7 +47,6 @@ import java.util.concurrent.Executors;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
@@ -65,7 +64,6 @@ import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.MemoryCompactionPolicy;
 import org.apache.hadoop.hbase.PrivateCellUtil;
@@ -73,6 +71,8 @@ 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.Get;
+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;
@@ -112,6 +112,7 @@ import org.junit.rules.TestName;
 import org.mockito.Mockito;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
 
 /**
@@ -205,12 +206,12 @@ public class TestHStore {
     fs.delete(logdir, true);
     ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false,
       MemStoreLABImpl.CHUNK_SIZE_DEFAULT, 1, 0, null);
-    HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false);
+    RegionInfo info = RegionInfoBuilder.newBuilder(htd.getTableName()).build();
     Configuration walConf = new Configuration(conf);
     FSUtils.setRootDir(walConf, basedir);
     WALFactory wals = new WALFactory(walConf, null, methodName);
-    region = new HRegion(new HRegionFileSystem(conf, fs, tableDir, info),
-        wals.getWAL(info.getEncodedNameAsBytes(), info.getTable().getNamespace()), conf, htd, null);
+    region = new HRegion(new HRegionFileSystem(conf, fs, tableDir, info), wals.getWAL(info), conf,
+        htd, null);
   }
 
   private HStore init(String methodName, Configuration conf, TableDescriptorBuilder builder,
@@ -1006,7 +1007,6 @@ public class TestHStore {
     assertEquals(0, this.store.getStorefilesCount());
   }
 
-  @SuppressWarnings("unchecked")
   @Test
   public void testRefreshStoreFilesNotChanged() throws IOException {
     init(name.getMethodName());

http://git-wip-us.apache.org/repos/asf/hbase/blob/71a1192d/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileRefresherChore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileRefresherChore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileRefresherChore.java
index 416b194..35c744f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileRefresherChore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileRefresherChore.java
@@ -26,31 +26,33 @@ import static org.mockito.Mockito.when;
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.List;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.testclassification.RegionServerTests;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.Stoppable;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.Get;
 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.Result;
-import org.apache.hadoop.hbase.wal.WALFactory;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
+import org.apache.hadoop.hbase.testclassification.RegionServerTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.StoppableImplementation;
+import org.apache.hadoop.hbase.wal.WALFactory;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
@@ -73,20 +75,20 @@ public class TestStoreFileRefresherChore {
     FSUtils.setRootDir(TEST_UTIL.getConfiguration(), testDir);
   }
 
-  private HTableDescriptor getTableDesc(TableName tableName, byte[]... families) {
-    HTableDescriptor htd = new HTableDescriptor(tableName);
-    for (byte[] family : families) {
-      HColumnDescriptor hcd = new HColumnDescriptor(family);
-      // Set default to be three versions.
-      hcd.setMaxVersions(Integer.MAX_VALUE);
-      htd.addFamily(hcd);
-    }
-    return htd;
+  private TableDescriptor getTableDesc(TableName tableName, int regionReplication,
+      byte[]... families) {
+    TableDescriptorBuilder builder =
+        TableDescriptorBuilder.newBuilder(tableName).setRegionReplication(regionReplication);
+    Arrays.stream(families).map(family -> ColumnFamilyDescriptorBuilder.newBuilder(family)
+        .setMaxVersions(Integer.MAX_VALUE).build()).forEachOrdered(builder::addColumnFamily);
+    return builder.build();
   }
 
   static class FailingHRegionFileSystem extends HRegionFileSystem {
     boolean fail = false;
-    FailingHRegionFileSystem(Configuration conf, FileSystem fs, Path tableDir, HRegionInfo regionInfo) {
+
+    FailingHRegionFileSystem(Configuration conf, FileSystem fs, Path tableDir,
+        RegionInfo regionInfo) {
       super(conf, fs, tableDir, regionInfo);
     }
 
@@ -99,21 +101,21 @@ public class TestStoreFileRefresherChore {
     }
   }
 
-  private HRegion initHRegion(HTableDescriptor htd, byte[] startKey, byte[] stopKey, int replicaId)
+  private HRegion initHRegion(TableDescriptor htd, byte[] startKey, byte[] stopKey, int replicaId)
       throws IOException {
     Configuration conf = TEST_UTIL.getConfiguration();
     Path tableDir = FSUtils.getTableDir(testDir, htd.getTableName());
 
-    HRegionInfo info = new HRegionInfo(htd.getTableName(), startKey, stopKey, false, 0, replicaId);
-
-    HRegionFileSystem fs = new FailingHRegionFileSystem(conf, tableDir.getFileSystem(conf), tableDir,
-      info);
+    RegionInfo info = RegionInfoBuilder.newBuilder(htd.getTableName()).setStartKey(startKey)
+        .setEndKey(stopKey).setRegionId(0L).setReplicaId(replicaId).build();
+    HRegionFileSystem fs =
+        new FailingHRegionFileSystem(conf, tableDir.getFileSystem(conf), tableDir, info);
     final Configuration walConf = new Configuration(conf);
     FSUtils.setRootDir(walConf, tableDir);
     final WALFactory wals = new WALFactory(walConf, null, "log_" + replicaId);
     ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     HRegion region =
-        new HRegion(fs, wals.getWAL(info.getEncodedNameAsBytes(), info.getTable().getNamespace()),
+        new HRegion(fs, wals.getWAL(info),
             conf, htd, null);
 
     region.initialize();
@@ -188,8 +190,7 @@ public class TestStoreFileRefresherChore {
     when(regionServer.getOnlineRegionsLocalContext()).thenReturn(regions);
     when(regionServer.getConfiguration()).thenReturn(TEST_UTIL.getConfiguration());
 
-    HTableDescriptor htd = getTableDesc(TableName.valueOf(name.getMethodName()), families);
-    htd.setRegionReplication(2);
+    TableDescriptor htd = getTableDesc(TableName.valueOf(name.getMethodName()), 2, families);
     HRegion primary = initHRegion(htd, HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, 0);
     HRegion replica1 = initHRegion(htd, HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, 1);
     regions.add(primary);

http://git-wip-us.apache.org/repos/asf/hbase/blob/71a1192d/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALMonotonicallyIncreasingSeqId.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALMonotonicallyIncreasingSeqId.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALMonotonicallyIncreasingSeqId.java
index 99db208..a5148b3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALMonotonicallyIncreasingSeqId.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALMonotonicallyIncreasingSeqId.java
@@ -119,7 +119,7 @@ public class TestWALMonotonicallyIncreasingSeqId {
     wals = new WALFactory(walConf, null, "log_" + replicaId);
     ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     HRegion region = HRegion.createHRegion(info, TEST_UTIL.getDefaultRootDirPath(), conf, htd,
-      wals.getWAL(info.getEncodedNameAsBytes(), info.getTable().getNamespace()));
+      wals.getWAL(info));
     return region;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/71a1192d/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestLogRolling.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestLogRolling.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestLogRolling.java
index 264b4c8..37c3b37 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestLogRolling.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestLogRolling.java
@@ -22,7 +22,6 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
@@ -160,12 +159,12 @@ public abstract class AbstractTestLogRolling  {
   /**
    * Tests that log rolling doesn't hang when no data is written.
    */
-  @Test(timeout=120000)
+  @Test(timeout = 120000)
   public void testLogRollOnNothingWritten() throws Exception {
     final Configuration conf = TEST_UTIL.getConfiguration();
-    final WALFactory wals = new WALFactory(conf, null,
-        ServerName.valueOf("test.com",8080, 1).toString());
-    final WAL newLog = wals.getWAL(new byte[]{}, null);
+    final WALFactory wals =
+        new WALFactory(conf, null, ServerName.valueOf("test.com", 8080, 1).toString());
+    final WAL newLog = wals.getWAL(null);
     try {
       // Now roll the log before we write anything.
       newLog.rollWriter(true);