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:12 UTC

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

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);