You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2021/03/31 07:23:22 UTC

[hbase] branch HBASE-25713 updated: hbase-wal module compiles

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

stack pushed a commit to branch HBASE-25713
in repository https://gitbox.apache.org/repos/asf/hbase.git


The following commit(s) were added to refs/heads/HBASE-25713 by this push:
     new c62ab8e  hbase-wal module compiles
c62ab8e is described below

commit c62ab8e5f2347aa843944bebea0c0694808c3f1d
Author: stack <st...@apache.org>
AuthorDate: Wed Mar 31 00:19:42 2021 -0700

    hbase-wal module compiles
---
 hbase-client/pom.xml                               |  4 ++
 .../org/apache/hadoop/hbase/client/RegionInfo.java | 11 +++
 .../MultiVersionConcurrencyControl.java            |  5 +-
 .../TestMultiVersionConcurrencyControl.java        |  0
 hbase-coprocessor/pom.xml                          |  4 ++
 .../hbase/coprocessor/MetricsCoprocessor.java      |  2 +-
 .../hbase/mapreduce/TestWALRecordReader.java       | 13 ++--
 hbase-server/pom.xml                               |  4 ++
 .../apache/hadoop/hbase/regionserver/HRegion.java  |  2 +-
 .../hbase/regionserver/wal/AbstractFSWAL.java      | 33 ++++-----
 .../hadoop/hbase/regionserver/wal/AsyncFSWAL.java  | 14 ++--
 .../hadoop/hbase/regionserver/wal/FSHLog.java      | 25 +++----
 .../hadoop/hbase/regionserver/wal/FSWALEntry.java  | 15 +----
 .../hadoop/hbase/regionserver/wal/WALUtil.java     | 23 +++----
 .../coprocessor/CurrentCoprocessorMethods.java     |  2 +-
 .../hadoop/hbase/wal/DisabledWALProvider.java      | 23 ++++---
 .../coprocessor/SampleRegionWALCoprocessor.java    | 13 ++--
 .../hbase/coprocessor/TestCoprocessorMetrics.java  | 11 ---
 ...ObserverForAddingMutationsFromCoprocessors.java | 26 +++-----
 .../hadoop/hbase/coprocessor/TestWALObserver.java  | 12 ++--
 .../hadoop/hbase/master/AbstractTestDLS.java       |  2 +-
 .../hadoop/hbase/regionserver/TestBulkLoad.java    |  8 +--
 .../regionserver/TestCompactionAfterBulkLoad.java  |  2 +-
 .../hadoop/hbase/regionserver/TestHRegion.java     | 21 +++---
 .../regionserver/TestHRegionReplayEvents.java      | 12 ++--
 .../hadoop/hbase/regionserver/TestWALLockup.java   |  8 +--
 .../hbase/regionserver/wal/AbstractTestFSWAL.java  | 12 ++--
 .../regionserver/wal/AbstractTestWALReplay.java    | 57 ++++------------
 .../hbase/regionserver/wal/TestAsyncFSWAL.java     |  3 +-
 .../hbase/regionserver/wal/TestLogRollAbort.java   |  2 +-
 .../hbase/regionserver/wal/TestLogRolling.java     |  1 -
 .../regionserver/wal/TestLogRollingNoCluster.java  |  7 +-
 .../regionserver/wal/TestWALActionsListener.java   |  5 +-
 .../TestReplicationEmptyWALRecovery.java           |  4 +-
 .../replication/TestReplicationSmallTests.java     |  5 +-
 ...stRegionReplicaReplicationEndpointNoMaster.java | 20 +-----
 .../regionserver/TestReplicationSink.java          | 61 ++++++-----------
 .../regionserver/TestReplicationSourceManager.java | 21 ++----
 .../regionserver/TestWALEntryStream.java           |  6 +-
 .../org/apache/hadoop/hbase/wal/FaultyFSLog.java   |  9 +--
 .../hadoop/hbase/wal/TestFSHLogProvider.java       | 10 +--
 .../org/apache/hadoop/hbase/wal/TestSecureWAL.java |  2 +-
 .../apache/hadoop/hbase/wal/TestWALFactory.java    | 78 +++++++++++-----------
 .../hadoop/hbase/wal/TestWALReaderOnSecureWAL.java |  2 +-
 .../apache/hadoop/hbase/wal/TestWALRootDir.java    | 10 +--
 .../org/apache/hadoop/hbase/wal/TestWALSplit.java  | 27 +++++---
 .../hadoop/hbase/wal/WALPerformanceEvaluation.java |  7 +-
 {hbase-coprocessor => hbase-wal}/pom.xml           | 18 ++++-
 .../hadoop/hbase/coprocessor/WALCoprocessor.java   |  6 +-
 .../coprocessor/WALCoprocessorEnvironment.java     |  4 +-
 .../hadoop/hbase/coprocessor/WALObserver.java      | 29 +-------
 .../hadoop/hbase/regionserver/SequenceId.java      |  0
 .../hbase/regionserver/wal/CompressionContext.java |  6 +-
 .../regionserver/wal/FailedLogCloseException.java  |  9 +--
 .../hbase/regionserver/wal/WALActionsListener.java | 19 +++---
 .../hbase/regionserver/wal/WALCellCodec.java       |  0
 .../hbase/regionserver/wal/WALCoprocessorHost.java | 43 ++++--------
 .../regionserver/WALFileLengthProvider.java        |  2 -
 .../main/java/org/apache/hadoop/hbase/wal/WAL.java | 36 ++++------
 .../java/org/apache/hadoop/hbase/wal/WALEdit.java  | 40 ++++-------
 .../java/org/apache/hadoop/hbase/wal/WALKey.java   | 13 ++--
 .../org/apache/hadoop/hbase/wal/WALKeyImpl.java    | 15 +----
 .../regionserver/wal/TestCustomWALCellCodec.java   |  0
 pom.xml                                            | 13 ++++
 64 files changed, 362 insertions(+), 535 deletions(-)

diff --git a/hbase-client/pom.xml b/hbase-client/pom.xml
index 8f61522..288993c 100644
--- a/hbase-client/pom.xml
+++ b/hbase-client/pom.xml
@@ -204,6 +204,10 @@
         </exclusion>
       </exclusions>
     </dependency>
+      <dependency>
+          <artifactId>hbase-wal</artifactId>
+          <groupId>org.apache.hbase</groupId>
+      </dependency>
   </dependencies>
 
   <profiles>
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfo.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfo.java
index e105dbc..427a49c 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfo.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfo.java
@@ -851,4 +851,15 @@ public interface RegionInfo extends Comparable<RegionInfo> {
   default int compareTo(RegionInfo other) {
     return RegionInfo.COMPARATOR.compare(this, other);
   }
+
+  /**
+   * Empty row key is not allowed in mutations because it is both the start key and the end key
+   * so we return the smallest byte[] that is bigger (in lex comparison) than byte[0] if startkey
+   * is empty.
+   * @return {@link #getStartKey()} or if length == 0, new byte [] {0}
+   */
+  default byte[] getNonEmptyStartKey() {
+    byte[] startKey = getStartKey();
+    return startKey.length == 0? new byte [] {0}: startKey;
+  }
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MultiVersionConcurrencyControl.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/regionserver/MultiVersionConcurrencyControl.java
similarity index 99%
rename from hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MultiVersionConcurrencyControl.java
rename to hbase-common/src/main/java/org/apache/hadoop/hbase/regionserver/MultiVersionConcurrencyControl.java
index d821eec..bcab681 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MultiVersionConcurrencyControl.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/regionserver/MultiVersionConcurrencyControl.java
@@ -23,11 +23,10 @@ import java.util.concurrent.atomic.AtomicLong;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ClassSize;
 import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 import org.apache.hbase.thirdparty.com.google.common.base.MoreObjects;
 import org.apache.hbase.thirdparty.com.google.common.base.MoreObjects.ToStringHelper;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Manages the read/write consistency. This provides an interface for readers to determine what
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiVersionConcurrencyControl.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiVersionConcurrencyControl.java
similarity index 100%
rename from hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiVersionConcurrencyControl.java
rename to hbase-common/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiVersionConcurrencyControl.java
diff --git a/hbase-coprocessor/pom.xml b/hbase-coprocessor/pom.xml
index 0bcfcef..2998751 100644
--- a/hbase-coprocessor/pom.xml
+++ b/hbase-coprocessor/pom.xml
@@ -98,5 +98,9 @@
       <type>test-jar</type>
       <scope>test</scope>
     </dependency>
+      <dependency>
+          <groupId>org.apache.hbase</groupId>
+          <artifactId>hbase-metrics-api</artifactId>
+      </dependency>
   </dependencies>
 </project>
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MetricsCoprocessor.java b/hbase-coprocessor/src/main/java/org/apache/hadoop/hbase/coprocessor/MetricsCoprocessor.java
similarity index 100%
rename from hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MetricsCoprocessor.java
rename to hbase-coprocessor/src/main/java/org/apache/hadoop/hbase/coprocessor/MetricsCoprocessor.java
index a77a0fe..d8a1bcb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MetricsCoprocessor.java
+++ b/hbase-coprocessor/src/main/java/org/apache/hadoop/hbase/coprocessor/MetricsCoprocessor.java
@@ -20,10 +20,10 @@
 
 package org.apache.hadoop.hbase.coprocessor;
 
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.metrics.MetricRegistries;
 import org.apache.hadoop.hbase.metrics.MetricRegistry;
 import org.apache.hadoop.hbase.metrics.MetricRegistryInfo;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Utility class for tracking metrics for various types of coprocessors. Each coprocessor instance
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 0fa73f6..76fd384 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
@@ -42,7 +42,6 @@ import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.Threads;
-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;
@@ -134,10 +133,10 @@ public class TestWALRecordReader {
     long ts = System.currentTimeMillis();
     WALEdit edit = new WALEdit();
     edit.add(new KeyValue(rowName, family, Bytes.toBytes("1"), ts, value));
-    log.appendData(info, getWalKeyImpl(ts, scopes), edit);
+    log.appendData(getWalKeyImpl(ts, scopes), edit);
     edit = new WALEdit();
     edit.add(new KeyValue(rowName, family, Bytes.toBytes("2"), ts+1, value));
-    log.appendData(info, getWalKeyImpl(ts+1, scopes), edit);
+    log.appendData(getWalKeyImpl(ts+1, scopes), edit);
     log.sync();
     Threads.sleep(10);
     LOG.info("Before 1st WAL roll " + log.toString());
@@ -149,10 +148,10 @@ public class TestWALRecordReader {
 
     edit = new WALEdit();
     edit.add(new KeyValue(rowName, family, Bytes.toBytes("3"), ts1+1, value));
-    log.appendData(info, getWalKeyImpl(ts1+1, scopes), edit);
+    log.appendData(getWalKeyImpl(ts1+1, scopes), edit);
     edit = new WALEdit();
     edit.add(new KeyValue(rowName, family, Bytes.toBytes("4"), ts1+2, value));
-    log.appendData(info, getWalKeyImpl(ts1+2, scopes), edit);
+    log.appendData(getWalKeyImpl(ts1+2, scopes), edit);
     log.sync();
     log.shutdown();
     walfactory.shutdown();
@@ -196,7 +195,7 @@ public class TestWALRecordReader {
     WALEdit edit = new WALEdit();
     edit.add(new KeyValue(rowName, family, Bytes.toBytes("1"),
         System.currentTimeMillis(), value));
-    long txid = log.appendData(info, getWalKeyImpl(System.currentTimeMillis(), scopes), edit);
+    long txid = log.appendData(getWalKeyImpl(System.currentTimeMillis(), scopes), edit);
     log.sync(txid);
 
     Thread.sleep(1); // make sure 2nd log gets a later timestamp
@@ -205,7 +204,7 @@ public class TestWALRecordReader {
 
     edit = new WALEdit();
     edit.add(new KeyValue(rowName, family, Bytes.toBytes("2"), System.currentTimeMillis(), value));
-    txid = log.appendData(info, getWalKeyImpl(System.currentTimeMillis(), scopes), edit);
+    txid = log.appendData(getWalKeyImpl(System.currentTimeMillis(), scopes), edit);
     log.sync(txid);
     log.shutdown();
     walfactory.shutdown();
diff --git a/hbase-server/pom.xml b/hbase-server/pom.xml
index 93051e0..410d7fc 100644
--- a/hbase-server/pom.xml
+++ b/hbase-server/pom.xml
@@ -258,6 +258,10 @@
     </dependency>
     <dependency>
       <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-wal</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
       <artifactId>hbase-coprocessor</artifactId>
     </dependency>
     <dependency>
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
index 9220b2f..6f3193c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
@@ -7907,7 +7907,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     }
     WriteEntry writeEntry = null;
     try {
-      long txid = this.wal.appendData(this.getRegionInfo(), walKey, walEdit);
+      long txid = this.wal.appendData(walKey, walEdit);
       // Call sync on our edit.
       if (txid != 0) {
         sync(txid, durability);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java
index 6c7f008..8345560 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java
@@ -23,7 +23,6 @@ import static org.apache.hadoop.hbase.regionserver.wal.WALActionsListener.RollRe
 import static org.apache.hadoop.hbase.wal.AbstractFSWALProvider.WAL_FILE_NAME_DELIMITER;
 import static org.apache.hbase.thirdparty.com.google.common.base.Preconditions.checkArgument;
 import static org.apache.hbase.thirdparty.com.google.common.base.Preconditions.checkNotNull;
-
 import com.lmax.disruptor.RingBuffer;
 import java.io.FileNotFoundException;
 import java.io.IOException;
@@ -59,7 +58,6 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.PrivateCellUtil;
-import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.exceptions.TimeoutIOException;
 import org.apache.hadoop.hbase.io.util.MemorySizeUtil;
 import org.apache.hadoop.hbase.ipc.RpcServer;
@@ -82,13 +80,12 @@ import org.apache.hadoop.hbase.wal.WALProvider.WriterBase;
 import org.apache.hadoop.hbase.wal.WALSplitter;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.util.StringUtils;
+import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
 import org.apache.htrace.core.TraceScope;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
-
 /**
  * Implementation of {@link WAL} to go against {@link FileSystem}; i.e. keep WALs in HDFS. Only one
  * WAL is ever being written at a time. When a WAL hits a configured maximum size, it is rolled.
@@ -265,6 +262,7 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
   protected volatile boolean closed = false;
 
   protected final AtomicBoolean shutdown = new AtomicBoolean(false);
+
   /**
    * WAL Comparator; it compares the timestamp (log filenum), present in the log file name. Throws
    * an IllegalArgumentException if used to compare paths from different wals.
@@ -273,12 +271,11 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
     (o1, o2) -> Long.compare(getFileNumFromFileName(o1), getFileNumFromFileName(o2));
 
   private static final class WalProps {
-
     /**
      * Map the encoded region name to the highest sequence id.
      * <p/>Contains all the regions it has an entry for.
      */
-    public final Map<byte[], Long> encodedName2HighestSequenceId;
+    private final Map<byte[], Long> encodedName2HighestSequenceId;
 
     /**
      * The log file size. Notice that the size may not be accurate if we do asynchronous close in
@@ -1007,7 +1004,7 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
     }
 
     // Coprocessor hook.
-    coprocessorHost.preWALWrite(entry.getRegionInfo(), entry.getKey(), entry.getEdit());
+    coprocessorHost.preWALWrite(entry.getKey(), entry.getEdit());
     if (!listeners.isEmpty()) {
       for (WALActionsListener i : listeners) {
         i.visitLogEntryBeforeWrite(entry.getKey(), entry.getEdit());
@@ -1023,7 +1020,7 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
       sequenceIdAccounting.update(encodedRegionName, entry.getFamilyNames(), regionSequenceId,
         entry.isInMemStore());
     }
-    coprocessorHost.postWALWrite(entry.getRegionInfo(), entry.getKey(), entry.getEdit());
+    coprocessorHost.postWALWrite(entry.getKey(), entry.getEdit());
     // Update metrics.
     postAppend(entry, EnvironmentEdgeManager.currentTime() - start);
     numEntries.incrementAndGet();
@@ -1071,12 +1068,12 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
     }
   }
 
-  protected final long stampSequenceIdAndPublishToRingBuffer(RegionInfo hri, WALKeyImpl key,
+  protected final long stampSequenceIdAndPublishToRingBuffer(WALKeyImpl key,
     WALEdit edits, boolean inMemstore, RingBuffer<RingBufferTruck> ringBuffer)
     throws IOException {
     if (this.closed) {
-      throw new IOException(
-        "Cannot append; log is closed, regionName = " + hri.getRegionNameAsString());
+      throw new IOException("WAL closed; cannot append; encodedRegionName=" +
+        Bytes.toString(key.getEncodedRegionName()));
     }
     MutableLong txidHolder = new MutableLong();
     MultiVersionConcurrencyControl.WriteEntry we = key.getMvcc().begin(() -> {
@@ -1086,7 +1083,7 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
     ServerCall<?> rpcCall = RpcServer.getCurrentCall().filter(c -> c instanceof ServerCall)
       .filter(c -> c.getCellScanner() != null).map(c -> (ServerCall) c).orElse(null);
     try (TraceScope scope = TraceUtil.createTrace(implClassName + ".append")) {
-      FSWALEntry entry = new FSWALEntry(txid, key, edits, hri, inMemstore, rpcCall);
+      FSWALEntry entry = new FSWALEntry(txid, key, edits, inMemstore, rpcCall);
       entry.stampRegionSequenceId(we);
       ringBuffer.get(txid).load(entry);
     } finally {
@@ -1123,14 +1120,13 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
   }
 
   @Override
-  public long appendData(RegionInfo info, WALKeyImpl key, WALEdit edits) throws IOException {
-    return append(info, key, edits, true);
+  public long appendData(WALKeyImpl key, WALEdit edits) throws IOException {
+    return append(key, edits, true);
   }
 
   @Override
-  public long appendMarker(RegionInfo info, WALKeyImpl key, WALEdit edits)
-    throws IOException {
-    return append(info, key, edits, false);
+  public long appendMarker(WALKeyImpl key, WALEdit edits) throws IOException {
+    return append(key, edits, false);
   }
 
   /**
@@ -1150,7 +1146,6 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
    * passed in WALKey <code>walKey</code> parameter. Be warned that the WriteEntry is not
    * immediately available on return from this method. It WILL be available subsequent to a sync of
    * this append; otherwise, you will just have to wait on the WriteEntry to get filled in.
-   * @param info the regioninfo associated with append
    * @param key Modified by this call; we add to it this edits region edit/sequence id.
    * @param edits Edits to append. MAY CONTAIN NO EDITS for case where we want to get an edit
    *          sequence id that is after all currently appended edits.
@@ -1163,7 +1158,7 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
    * @return Returns a 'transaction id' and <code>key</code> will have the region edit/sequence id
    *         in it.
    */
-  protected abstract long append(RegionInfo info, WALKeyImpl key, WALEdit edits, boolean inMemstore)
+  protected abstract long append(WALKeyImpl key, WALEdit edits, boolean inMemstore)
       throws IOException;
 
   protected abstract void doAppend(W writer, FSWALEntry entry) throws IOException;
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java
index 66149a3..5576a77 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.regionserver.wal;
 import static org.apache.hadoop.hbase.regionserver.wal.WALActionsListener.RollRequestReason.ERROR;
 import static org.apache.hadoop.hbase.regionserver.wal.WALActionsListener.RollRequestReason.SIZE;
 import static org.apache.hadoop.hbase.util.FutureUtils.addListener;
-
 import com.lmax.disruptor.RingBuffer;
 import com.lmax.disruptor.Sequence;
 import com.lmax.disruptor.Sequencer;
@@ -44,7 +43,6 @@ import java.util.concurrent.locks.Condition;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantLock;
 import java.util.function.Supplier;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -60,14 +58,14 @@ import org.apache.hadoop.hbase.wal.WALProvider.AsyncWriter;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
 import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
-import org.apache.htrace.core.TraceScope;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.hbase.thirdparty.io.netty.channel.Channel;
 import org.apache.hbase.thirdparty.io.netty.channel.EventLoop;
 import org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup;
 import org.apache.hbase.thirdparty.io.netty.util.concurrent.SingleThreadEventExecutor;
+import org.apache.htrace.core.TraceScope;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 
 /**
@@ -595,9 +593,9 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
   }
 
   @Override
-  protected long append(RegionInfo hri, WALKeyImpl key, WALEdit edits, boolean inMemstore)
+  protected long append(WALKeyImpl key, WALEdit edits, boolean inMemstore)
       throws IOException {
-    long txid = stampSequenceIdAndPublishToRingBuffer(hri, key, edits, inMemstore,
+    long txid = stampSequenceIdAndPublishToRingBuffer(key, edits, inMemstore,
       waitingConsumePayloads);
     if (shouldScheduleConsumer()) {
       consumeExecutor.execute(consumer);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
index 7e12097..c8879aa 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
@@ -21,7 +21,6 @@ import static org.apache.hadoop.hbase.regionserver.wal.WALActionsListener.RollRe
 import static org.apache.hadoop.hbase.regionserver.wal.WALActionsListener.RollRequestReason.LOW_REPLICATION;
 import static org.apache.hadoop.hbase.regionserver.wal.WALActionsListener.RollRequestReason.SIZE;
 import static org.apache.hadoop.hbase.regionserver.wal.WALActionsListener.RollRequestReason.SLOW_SYNC;
-
 import com.lmax.disruptor.BlockingWaitStrategy;
 import com.lmax.disruptor.EventHandler;
 import com.lmax.disruptor.ExceptionHandler;
@@ -65,8 +64,6 @@ import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
-
 /**
  * The default implementation of FSWAL.
  */
@@ -109,8 +106,10 @@ public class FSHLog extends AbstractFSWAL<Writer> {
   // We use ring buffer sequence as txid of FSWALEntry and SyncFuture.
   private static final Logger LOG = LoggerFactory.getLogger(FSHLog.class);
 
-  private static final String TOLERABLE_LOW_REPLICATION = "hbase.regionserver.hlog.tolerable.lowreplication";
-  private static final String LOW_REPLICATION_ROLL_LIMIT = "hbase.regionserver.hlog.lowreplication.rolllimit";
+  private static final String TOLERABLE_LOW_REPLICATION =
+    "hbase.regionserver.hlog.tolerable.lowreplication";
+  private static final String LOW_REPLICATION_ROLL_LIMIT =
+    "hbase.regionserver.hlog.lowreplication.rolllimit";
   private static final int DEFAULT_LOW_REPLICATION_ROLL_LIMIT = 5;
   private static final String ROLL_ERRORS_TOLERATED = "hbase.regionserver.logroll.errors.tolerated";
   private static final int DEFAULT_ROLL_ERRORS_TOLERATED = 2;
@@ -119,7 +118,8 @@ public class FSHLog extends AbstractFSWAL<Writer> {
   private static final String MAX_BATCH_COUNT = "hbase.regionserver.wal.sync.batch.count";
   private static final int DEFAULT_MAX_BATCH_COUNT = 200;
 
-  private static final String FSHLOG_WAIT_ON_SHUTDOWN_IN_SECONDS = "hbase.wal.fshlog.wait.on.shutdown.seconds";
+  private static final String FSHLOG_WAIT_ON_SHUTDOWN_IN_SECONDS =
+    "hbase.wal.fshlog.wait.on.shutdown.seconds";
   private static final int DEFAULT_FSHLOG_WAIT_ON_SHUTDOWN_IN_SECONDS = 5;
 
   /**
@@ -491,9 +491,9 @@ public class FSHLog extends AbstractFSWAL<Writer> {
   }
 
   @Override
-  protected long append(final RegionInfo hri, final WALKeyImpl key, final WALEdit edits,
+  protected long append(final WALKeyImpl key, final WALEdit edits,
     final boolean inMemstore) throws IOException {
-    return stampSequenceIdAndPublishToRingBuffer(hri, key, edits, inMemstore,
+    return stampSequenceIdAndPublishToRingBuffer(key, edits, inMemstore,
       disruptor.getRingBuffer());
   }
 
@@ -865,8 +865,8 @@ public class FSHLog extends AbstractFSWAL<Writer> {
    * To start up the drama, Thread A creates an instance of this class each time it would do this
    * zigzag dance and passes it to Thread B (these classes use Latches so it is one shot only).
    * Thread B notices the new instance (via reading a volatile reference or how ever) and it starts
-   * to work toward the 'safe point'. Thread A calls {@link #waitSafePoint(SyncFuture)} when it cannot proceed
-   * until the Thread B 'safe point' is attained. Thread A will be held inside in
+   * to work toward the 'safe point'. Thread A calls {@link #waitSafePoint(SyncFuture)} when it
+   * cannot proceed until the Thread B 'safe point' is attained. Thread A will be held inside in
    * {@link #waitSafePoint(SyncFuture)} until Thread B reaches the 'safe point'. Once there, Thread B frees
    * Thread A by calling {@link #safePointAttained()}. Thread A now knows Thread B is at the 'safe
    * point' and that it is holding there (When Thread B calls {@link #safePointAttained()} it blocks
@@ -1052,8 +1052,9 @@ public class FSHLog extends AbstractFSWAL<Writer> {
             // Failed append. Record the exception.
             this.exception = e;
             // invoking cleanupOutstandingSyncsOnException when append failed with exception,
-            // it will cleanup existing sync requests recorded in syncFutures but not offered to SyncRunner yet,
-            // so there won't be any sync future left over if no further truck published to disruptor.
+            // it will cleanup existing sync requests recorded in syncFutures but not offered to
+            // SyncRunner yet, so there won't be any sync future left over if no further truck
+            // published to disruptor.
             cleanupOutstandingSyncsOnException(sequence,
                 this.exception instanceof DamagedWALException ? this.exception
                     : new DamagedWALException("On sync", this.exception));
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSWALEntry.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSWALEntry.java
index ca51ec0..710fc28 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSWALEntry.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSWALEntry.java
@@ -25,16 +25,14 @@ import java.util.TreeSet;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.PrivateCellUtil;
-import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.ipc.ServerCall;
 import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.wal.WAL.Entry;
 import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.wal.WALKeyImpl;
-import org.apache.yetus.audience.InterfaceAudience;
-
 import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * A WAL Entry for {@link AbstractFSWAL} implementation.  Immutable.
@@ -61,8 +59,6 @@ class FSWALEntry extends Entry {
    * Set if this is a meta edit and it is of close region type.
    */
   private final transient boolean closeRegion;
-
-  private final transient RegionInfo regionInfo;
   private final transient Set<byte[]> familyNames;
   private final transient ServerCall<?> rpcCall;
 
@@ -70,12 +66,11 @@ class FSWALEntry extends Entry {
    * @param inMemstore If true, then this is a data edit, one that came from client. If false, it
    *   is a meta edit made by the hbase system itself and is for the WAL only.
    */
-  FSWALEntry(final long txid, final WALKeyImpl key, final WALEdit edit, final RegionInfo regionInfo,
-    final boolean inMemstore, ServerCall<?> rpcCall) {
+  FSWALEntry(final long txid, final WALKeyImpl key, final WALEdit edit, final boolean inMemstore,
+      ServerCall<?> rpcCall) {
     super(key, edit);
     this.inMemstore = inMemstore;
     this.closeRegion = !inMemstore && edit.isRegionCloseMarker();
-    this.regionInfo = regionInfo;
     this.txid = txid;
     if (inMemstore) {
       // construct familyNames here to reduce the work of log sinker.
@@ -117,10 +112,6 @@ class FSWALEntry extends Entry {
     return closeRegion;
   }
 
-  RegionInfo getRegionInfo() {
-    return this.regionInfo;
-  }
-
   /**
    * @return The transaction id of this edit.
    */
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALUtil.java
index 19f6b96..f90c04d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALUtil.java
@@ -23,7 +23,6 @@ import java.util.ArrayList;
 import java.util.Map;
 import java.util.NavigableMap;
 import java.util.function.Function;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -68,10 +67,10 @@ public class WALUtil {
    * @param mvcc Used by WAL to get sequence Id for the waledit.
    */
   public static WALKeyImpl writeCompactionMarker(WAL wal,
-    NavigableMap<byte[], Integer> replicationScope, RegionInfo hri, final CompactionDescriptor c,
-    MultiVersionConcurrencyControl mvcc) throws IOException {
-    WALKeyImpl walKey =
-      writeMarker(wal, replicationScope, hri, WALEdit.createCompaction(hri, c), mvcc, null);
+      NavigableMap<byte[], Integer> replicationScope, RegionInfo hri, final CompactionDescriptor c,
+      MultiVersionConcurrencyControl mvcc) throws IOException {
+   WALKeyImpl walKey = writeMarker(wal, replicationScope,
+     hri, WALEdit.createCompaction(hri.getNonEmptyStartKey(), c), mvcc, null);
     if (LOG.isTraceEnabled()) {
       LOG.trace("Appended compaction marker " + TextFormat.shortDebugString(c));
     }
@@ -87,7 +86,7 @@ public class WALUtil {
     RegionInfo hri, final FlushDescriptor f, boolean sync, MultiVersionConcurrencyControl mvcc)
     throws IOException {
     WALKeyImpl walKey = doFullMarkerAppendTransaction(wal, replicationScope, hri,
-      WALEdit.createFlushWALEdit(hri, f), mvcc, null, sync);
+      WALEdit.createFlushWALEdit(hri.getNonEmptyStartKey(), f), mvcc, null, sync);
     if (LOG.isTraceEnabled()) {
       LOG.trace("Appended flush marker " + TextFormat.shortDebugString(f));
     }
@@ -103,7 +102,7 @@ public class WALUtil {
       MultiVersionConcurrencyControl mvcc)
     throws IOException {
     WALKeyImpl walKey = writeMarker(wal, replicationScope, hri,
-        WALEdit.createRegionEventWALEdit(hri, r), mvcc, null);
+        WALEdit.createRegionEventWALEdit(hri.getNonEmptyStartKey(), r), mvcc, null);
     if (LOG.isTraceEnabled()) {
       LOG.trace("Appended region event marker " + TextFormat.shortDebugString(r));
     }
@@ -125,7 +124,7 @@ public class WALUtil {
       final WALProtos.BulkLoadDescriptor desc, final MultiVersionConcurrencyControl mvcc)
     throws IOException {
     WALKeyImpl walKey = writeMarker(wal, replicationScope, hri,
-      WALEdit.createBulkLoadEvent(hri, desc), mvcc, null);
+      WALEdit.createBulkLoadEvent(hri.getNonEmptyStartKey(), desc), mvcc, null);
     if (LOG.isTraceEnabled()) {
       LOG.trace("Appended Bulk Load marker " + TextFormat.shortDebugString(desc));
     }
@@ -133,8 +132,8 @@ public class WALUtil {
   }
 
   private static WALKeyImpl writeMarker(final WAL wal,
-      NavigableMap<byte[], Integer> replicationScope, RegionInfo hri, WALEdit edit, MultiVersionConcurrencyControl mvcc,
-      Map<String, byte[]> extendedAttributes)
+      NavigableMap<byte[], Integer> replicationScope, RegionInfo hri, WALEdit edit,
+      MultiVersionConcurrencyControl mvcc, Map<String, byte[]> extendedAttributes)
     throws IOException {
     // If sync == true in below, then timeout is not used; safe to pass UNSPECIFIED_TIMEOUT
     return doFullMarkerAppendTransaction(wal, replicationScope, hri, edit, mvcc,
@@ -156,9 +155,9 @@ public class WALUtil {
     // TODO: Pass in current time to use?
     WALKeyImpl walKey = new WALKeyImpl(hri.getEncodedNameAsBytes(), hri.getTable(),
       System.currentTimeMillis(), mvcc, replicationScope, extendedAttributes);
-    long trx = MultiVersionConcurrencyControl.NONE;
+    long trx;
     try {
-      trx = wal.appendMarker(hri, walKey, edit);
+      trx = wal.appendMarker(walKey, edit);
       if (sync) {
         wal.sync(trx);
       }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/coprocessor/CurrentCoprocessorMethods.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/coprocessor/CurrentCoprocessorMethods.java
index 265cf51..88da2ae 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/coprocessor/CurrentCoprocessorMethods.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/coprocessor/CurrentCoprocessorMethods.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
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 6c215f8..5a9241e 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
@@ -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
@@ -162,18 +162,17 @@ class DisabledWALProvider implements WALProvider {
     }
 
     @Override
-    public long appendData(RegionInfo info, WALKeyImpl key, WALEdit edits) throws IOException {
-      return append(info, key, edits, true);
+    public long appendData(WALKeyImpl key, WALEdit edits) throws IOException {
+      return append(key, edits, true);
     }
 
     @Override
-    public long appendMarker(RegionInfo info, WALKeyImpl key, WALEdit edits)
+    public long appendMarker(WALKeyImpl key, WALEdit edits)
       throws IOException {
-      return append(info, key, edits, false);
+      return append(key, edits, false);
     }
 
-    private long append(RegionInfo info, WALKeyImpl key, WALEdit edits, boolean inMemstore)
-        throws IOException {
+    private long append(WALKeyImpl key, WALEdit edits, boolean inMemstore) throws IOException {
       WriteEntry writeEntry = key.getMvcc().begin();
       if (!edits.isReplay()) {
         for (Cell cell : edits.getCells()) {
@@ -196,8 +195,10 @@ class DisabledWALProvider implements WALProvider {
     }
 
     @Override
-    public void updateStore(byte[] encodedRegionName, byte[] familyName,
-        Long sequenceid, boolean onlyIfGreater) { return; }
+    public void updateStore(byte[] encodedRegionName, byte[] familyName, Long sequenceid,
+        boolean onlyIfGreater) {
+      return;
+    }
 
     @Override
     public void sync() {
@@ -221,7 +222,9 @@ class DisabledWALProvider implements WALProvider {
 
     @Override
     public Long startCacheFlush(final byte[] encodedRegionName, Set<byte[]> flushedFamilyNames) {
-      if (closed.get()) return null;
+      if (closed.get()) {
+        return null;
+      }
       return HConstants.NO_SEQNUM;
     }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SampleRegionWALCoprocessor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SampleRegionWALCoprocessor.java
index 1da31da..c80e790 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SampleRegionWALCoprocessor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SampleRegionWALCoprocessor.java
@@ -23,7 +23,6 @@ import java.io.IOException;
 import java.util.Arrays;
 import java.util.List;
 import java.util.Optional;
-
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
@@ -92,17 +91,17 @@ public class SampleRegionWALCoprocessor implements WALCoprocessor, RegionCoproce
     return Optional.of(this);
   }
 
-  @Override
-  public void postWALWrite(ObserverContext<? extends WALCoprocessorEnvironment> env,
-      RegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {
+  // What to do here? TODO?
+  public void postWALWrite(ObserverContext<? extends WALCoprocessorEnvironment> env, WALKey logKey,
+      WALEdit logEdit) throws IOException {
     postWALWriteCalled = true;
   }
 
-  @Override
+  // What to do here? TODO?
   public void preWALWrite(ObserverContext<? extends WALCoprocessorEnvironment> env,
-      RegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {
+      WALKey logKey, WALEdit logEdit) throws IOException {
     // check table name matches or not.
-    if (!Bytes.equals(info.getTable().toBytes(), this.tableName)) {
+    if (!Bytes.equals(logKey.getTableName().toBytes(), this.tableName)) {
       return;
     }
     preWALWriteCalled = true;
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorMetrics.java
index 24f74d8..cc4ebe3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorMetrics.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorMetrics.java
@@ -22,7 +22,6 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotEquals;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
-
 import com.google.protobuf.RpcCallback;
 import com.google.protobuf.RpcController;
 import com.google.protobuf.ServiceException;
@@ -64,8 +63,6 @@ import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.testclassification.CoprocessorTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.wal.WALEdit;
-import org.apache.hadoop.hbase.wal.WALKey;
 import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
@@ -76,7 +73,6 @@ import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-
 import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
 
 /**
@@ -177,13 +173,6 @@ public class TestCoprocessorMetrics {
     private Counter walEditsCount;
 
     @Override
-    public void postWALWrite(ObserverContext<? extends WALCoprocessorEnvironment> ctx,
-                             RegionInfo info, WALKey logKey,
-                             WALEdit logEdit) throws IOException {
-      walEditsCount.increment();
-    }
-
-    @Override
     public void start(CoprocessorEnvironment env) throws IOException {
       if (env instanceof WALCoprocessorEnvironment) {
         MetricRegistry registry =
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverForAddingMutationsFromCoprocessors.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverForAddingMutationsFromCoprocessors.java
index 21435d9..64eb6b7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverForAddingMutationsFromCoprocessors.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverForAddingMutationsFromCoprocessors.java
@@ -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
@@ -19,7 +19,6 @@ package org.apache.hadoop.hbase.coprocessor;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
-
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.List;
@@ -35,7 +34,6 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Delete;
 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.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
@@ -45,6 +43,7 @@ import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.wal.WALKey;
+import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
 import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
@@ -56,8 +55,6 @@ import org.junit.rules.TestName;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
-
 @Category(MediumTests.class)
 public class TestRegionObserverForAddingMutationsFromCoprocessors {
 
@@ -107,7 +104,6 @@ public class TestRegionObserverForAddingMutationsFromCoprocessors {
 
   /**
    * Test various multiput operations.
-   * @throws Exception
    */
   @Test
   public void testMulti() throws Exception {
@@ -225,9 +221,9 @@ public class TestRegionObserverForAddingMutationsFromCoprocessors {
       Mutation mut = miniBatchOp.getOperation(0);
       List<Cell> cells = mut.getFamilyCellMap().get(test);
       Put[] puts = new Put[] {
-          new Put(Bytes.toBytes("cpPut")).addColumn(test, dummy, cells.get(0).getTimestamp(),
-            Bytes.toBytes("cpdummy")).setTTL(mut.getTTL())
-          };
+        new Put(Bytes.toBytes("cpPut")).addColumn(test, dummy, cells.get(0).getTimestamp(),
+          Bytes.toBytes("cpdummy")).setTTL(mut.getTTL())
+      };
       LOG.info("Putting:" + Arrays.toString(puts));
       miniBatchOp.addOperationsFromCP(0, puts);
     }
@@ -317,9 +313,9 @@ public class TestRegionObserverForAddingMutationsFromCoprocessors {
       if (mut instanceof Delete) {
         List<Cell> cells = mut.getFamilyCellMap().get(test);
         Delete[] deletes = new Delete[] {
-            // delete only 2 rows
-            new Delete(row1, cells.get(0).getTimestamp()),
-            new Delete(row2, cells.get(0).getTimestamp()),
+          // delete only 2 rows
+          new Delete(row1, cells.get(0).getTimestamp()),
+          new Delete(row2, cells.get(0).getTimestamp()),
         };
         LOG.info("Deleting:" + Arrays.toString(deletes));
         miniBatchOp.addOperationsFromCP(0, deletes);
@@ -335,10 +331,10 @@ public class TestRegionObserverForAddingMutationsFromCoprocessors {
       return Optional.of(this);
     }
 
-    @Override
+    // WHAT TO DO HERE?
     public void postWALWrite(ObserverContext<? extends WALCoprocessorEnvironment> ctx,
-                             RegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {
-      if (info.getTable().equals(TableName.valueOf("testCPMutationsAreWrittenToWALEdit"))) {
+        WALKey logKey, WALEdit logEdit) throws IOException {
+      if (logKey.getTableName().equals(TableName.valueOf("testCPMutationsAreWrittenToWALEdit"))) {
         savedEdit = logEdit;
       }
     }
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 6fbd3b5..9a10924 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,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
@@ -21,7 +21,6 @@ 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 java.io.IOException;
 import java.security.PrivilegedExceptionAction;
 import java.util.Arrays;
@@ -239,7 +238,7 @@ public class TestWALObserver {
     // it's where WAL write cp should occur.
     long now = EnvironmentEdgeManager.currentTime();
     // we use HLogKey here instead of WALKeyImpl directly to support legacy coprocessors.
-    long txid = log.appendData(hri, new WALKeyImpl(hri.getEncodedNameAsBytes(), hri.getTable(), now,
+    long txid = log.appendData(new WALKeyImpl(hri.getEncodedNameAsBytes(), hri.getTable(), now,
       new MultiVersionConcurrencyControl(), scopes), edit);
     log.sync(txid);
 
@@ -290,7 +289,7 @@ public class TestWALObserver {
       assertFalse(cp.isPostWALWriteCalled());
 
       final long now = EnvironmentEdgeManager.currentTime();
-      long txid = log.appendData(hri,
+      long txid = log.appendData(
         new WALKeyImpl(hri.getEncodedNameAsBytes(), hri.getTable(), now, mvcc, scopes),
         new WALEdit());
       log.sync(txid);
@@ -338,8 +337,7 @@ public class TestWALObserver {
       addWALEdits(tableName, hri, TEST_ROW, fam, countPerFamily,
         EnvironmentEdgeManager.getDelegate(), wal, scopes, mvcc);
     }
-    wal.appendData(hri, new WALKeyImpl(hri.getEncodedNameAsBytes(), tableName, now, mvcc, scopes),
-      edit);
+    wal.appendData(new WALKeyImpl(hri.getEncodedNameAsBytes(), tableName, now, mvcc, scopes), edit);
     // sync to fs.
     wal.sync();
 
@@ -454,7 +452,7 @@ 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.appendData(hri,
+      txid = wal.appendData(
         new WALKeyImpl(hri.getEncodedNameAsBytes(), tableName, ee.currentTime(), mvcc), edit);
     }
     if (-1 != txid) {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/AbstractTestDLS.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/AbstractTestDLS.java
index 24fdbc4..ef12911 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/AbstractTestDLS.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/AbstractTestDLS.java
@@ -372,7 +372,7 @@ public abstract class AbstractTestDLS {
         // HBaseTestingUtility.createMultiRegions use 5 bytes key
         byte[] qualifier = Bytes.toBytes("c" + Integer.toString(i));
         e.add(new KeyValue(row, COLUMN_FAMILY, qualifier, System.currentTimeMillis(), value));
-        log.appendData(curRegionInfo, new WALKeyImpl(curRegionInfo.getEncodedNameAsBytes(),
+        log.appendData(new WALKeyImpl(curRegionInfo.getEncodedNameAsBytes(),
           tableName, System.currentTimeMillis(), mvcc), e);
         if (0 == i % syncEvery) {
           log.sync();
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoad.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoad.java
index 3a934b7..d6a28ed 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoad.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoad.java
@@ -63,7 +63,7 @@ public class TestBulkLoad extends TestBulkloadBase {
     storeFileName = (new Path(storeFileName)).getName();
     List<String> storeFileNames = new ArrayList<>();
     storeFileNames.add(storeFileName);
-    when(log.appendMarker(any(), any(),
+    when(log.appendMarker(any(),
       argThat(bulkLogWalEdit(WALEdit.BULK_LOAD, tableName.toBytes(), familyName, storeFileNames))))
           .thenAnswer(new Answer() {
             @Override
@@ -89,7 +89,7 @@ public class TestBulkLoad extends TestBulkloadBase {
 
   @Test
   public void shouldBulkLoadSingleFamilyHLog() throws IOException {
-    when(log.appendMarker(any(), any(), argThat(bulkLogWalEditType(WALEdit.BULK_LOAD))))
+    when(log.appendMarker(any(), argThat(bulkLogWalEditType(WALEdit.BULK_LOAD))))
         .thenAnswer(new Answer() {
           @Override
           public Object answer(InvocationOnMock invocation) {
@@ -109,7 +109,7 @@ public class TestBulkLoad extends TestBulkloadBase {
   @Test
   public void shouldBulkLoadManyFamilyHLog() throws IOException {
     when(log.appendMarker(any(),
-            any(), argThat(bulkLogWalEditType(WALEdit.BULK_LOAD)))).thenAnswer(new Answer() {
+      argThat(bulkLogWalEditType(WALEdit.BULK_LOAD)))).thenAnswer(new Answer() {
               @Override
               public Object answer(InvocationOnMock invocation) {
                 WALKeyImpl walKey = invocation.getArgument(1);
@@ -128,7 +128,7 @@ public class TestBulkLoad extends TestBulkloadBase {
 
   @Test
   public void shouldBulkLoadManyFamilyHLogEvenWhenTableNameNamespaceSpecified() throws IOException {
-    when(log.appendMarker(any(), any(), argThat(bulkLogWalEditType(WALEdit.BULK_LOAD))))
+    when(log.appendMarker(any(), argThat(bulkLogWalEditType(WALEdit.BULK_LOAD))))
         .thenAnswer(new Answer() {
           @Override
           public Object answer(InvocationOnMock invocation) {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionAfterBulkLoad.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionAfterBulkLoad.java
index c736513..b6b7235 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionAfterBulkLoad.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionAfterBulkLoad.java
@@ -89,7 +89,7 @@ public class TestCompactionAfterBulkLoad extends TestBulkloadBase {
       conf.setBoolean(COMPACTION_AFTER_BULKLOAD_ENABLE, true);
       when(regionServerServices.getConfiguration()).thenReturn(conf);
       when(regionServerServices.getCompactionRequestor()).thenReturn(compactionRequester);
-      when(log.appendMarker(any(), any(), argThat(bulkLogWalEditType(WALEdit.BULK_LOAD))))
+      when(log.appendMarker(any(), argThat(bulkLogWalEditType(WALEdit.BULK_LOAD))))
           .thenAnswer(new Answer() {
             @Override
             public Object answer(InvocationOnMock invocation) {
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 de8118d..1e08e92 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
@@ -900,8 +900,8 @@ public class TestHRegion {
         long time = System.nanoTime();
         WALEdit edit = null;
         if (i == maxSeqId) {
-          edit = WALEdit.createCompaction(region.getRegionInfo(),
-          CompactionDescriptor.newBuilder()
+          edit = WALEdit.createCompaction(region.getRegionInfo().getNonEmptyStartKey(),
+            CompactionDescriptor.newBuilder()
           .setTableName(ByteString.copyFrom(tableName.getName()))
           .setFamilyName(ByteString.copyFrom(regionName))
           .setEncodedRegionName(ByteString.copyFrom(regionName))
@@ -1012,8 +1012,8 @@ public class TestHRegion {
       long time = System.nanoTime();
 
       writer.append(new WAL.Entry(new WALKeyImpl(regionName, tableName, 10, time,
-          HConstants.DEFAULT_CLUSTER_ID), WALEdit.createCompaction(region.getRegionInfo(),
-          compactionDescriptor)));
+          HConstants.DEFAULT_CLUSTER_ID), WALEdit.createCompaction(
+            region.getRegionInfo().getNonEmptyStartKey(), compactionDescriptor)));
       writer.close();
 
       // close the region now, and reopen again
@@ -5416,7 +5416,6 @@ public class TestHRegion {
 
   /**
    * Test case to check put function with memstore flushing for same row, same ts
-   * @throws Exception
    */
   @Test
   public void testPutWithMemStoreFlush() throws Exception {
@@ -5547,7 +5546,7 @@ public class TestHRegion {
     region.put(put);
 
     // verify append called or not
-    verify(wal, expectAppend ? times(1) : never()).appendData((HRegionInfo) any(),
+    verify(wal, expectAppend ? times(1) : never()).appendData(
       (WALKeyImpl) any(), (WALEdit) any());
 
     // verify sync called or not
@@ -6682,7 +6681,7 @@ public class TestHRegion {
     region = HRegion.openHRegion(hri, htd, rss.getWAL(hri),
       TEST_UTIL.getConfiguration(), rss, null);
 
-    verify(wal, times(1)).appendMarker(any(RegionInfo.class), any(WALKeyImpl.class),
+    verify(wal, times(1)).appendMarker(any(WALKeyImpl.class),
       editCaptor.capture());
 
     WALEdit edit = editCaptor.getValue();
@@ -6755,7 +6754,7 @@ public class TestHRegion {
    */
   private WAL mockWAL() throws IOException {
     WAL wal = mock(WAL.class);
-    when(wal.appendData(any(RegionInfo.class), any(WALKeyImpl.class), any(WALEdit.class)))
+    when(wal.appendData(any(WALKeyImpl.class), any(WALEdit.class)))
       .thenAnswer(new Answer<Long>() {
         @Override
         public Long answer(InvocationOnMock invocation) throws Throwable {
@@ -6765,7 +6764,7 @@ public class TestHRegion {
           return 1L;
         }
       });
-    when(wal.appendMarker(any(RegionInfo.class), any(WALKeyImpl.class), any(WALEdit.class))).
+    when(wal.appendMarker(any(WALKeyImpl.class), any(WALEdit.class))).
         thenAnswer(new Answer<Long>() {
           @Override
           public Long answer(InvocationOnMock invocation) throws Throwable {
@@ -6807,8 +6806,8 @@ public class TestHRegion {
     region.close(false);
 
     // 2 times, one for region open, the other close region
-    verify(wal, times(2)).appendMarker(any(RegionInfo.class),
-        (WALKeyImpl) any(WALKeyImpl.class), editCaptor.capture());
+    verify(wal, times(2)).appendMarker((WALKeyImpl)any(WALKeyImpl.class),
+      editCaptor.capture());
 
     WALEdit edit = editCaptor.getAllValues().get(1);
     assertNotNull(edit);
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 7b62119..968f619 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
@@ -368,8 +368,7 @@ public class TestHRegionReplayEvents {
       if (entry == null) {
         break;
       }
-      FlushDescriptor flushDesc
-      = WALEdit.getFlushDescriptor(entry.getEdit().getCells().get(0));
+      FlushDescriptor flushDesc = WALEdit.getFlushDescriptor(entry.getEdit().getCells().get(0));
       CompactionDescriptor compactionDesc
       = WALEdit.getCompaction(entry.getEdit().getCells().get(0));
       if (flushDesc != null) {
@@ -470,8 +469,7 @@ public class TestHRegionReplayEvents {
       if (entry == null) {
         break;
       }
-      FlushDescriptor flushDesc
-      = WALEdit.getFlushDescriptor(entry.getEdit().getCells().get(0));
+      FlushDescriptor flushDesc = WALEdit.getFlushDescriptor(entry.getEdit().getCells().get(0));
       if (flushDesc != null) {
         // first verify that everything is replayed and visible before flush event replay
         HStore store = secondaryRegion.getStore(Bytes.toBytes("cf1"));
@@ -1164,7 +1162,7 @@ public class TestHRegionReplayEvents {
 
     // test for region open and close
     secondaryRegion = HRegion.openHRegion(secondaryHri, htd, walSecondary, CONF, rss, null);
-    verify(walSecondary, times(0)).appendData(any(RegionInfo.class), any(WALKeyImpl.class),
+    verify(walSecondary, times(0)).appendData(any(WALKeyImpl.class),
       any(WALEdit.class));
 
     // test for replay prepare flush
@@ -1180,11 +1178,11 @@ public class TestHRegionReplayEvents {
           primaryRegion.getRegionInfo().getRegionName()))
       .build());
 
-    verify(walSecondary, times(0)).appendData(any(RegionInfo.class), any(WALKeyImpl.class),
+    verify(walSecondary, times(0)).appendData(any(WALKeyImpl.class),
       any(WALEdit.class));
 
     secondaryRegion.close();
-    verify(walSecondary, times(0)).appendData(any(RegionInfo.class), any(WALKeyImpl.class),
+    verify(walSecondary, times(0)).appendData(any(WALKeyImpl.class),
       any(WALEdit.class));
   }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
index 09d6934..daeb9bd 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.regionserver;
 
 import static org.junit.Assert.assertTrue;
-
 import java.io.IOException;
 import java.util.NavigableMap;
 import java.util.TreeMap;
@@ -54,6 +53,7 @@ import org.apache.hadoop.hbase.wal.WALKey;
 import org.apache.hadoop.hbase.wal.WALKeyImpl;
 import org.apache.hadoop.hbase.wal.WALProvider.Writer;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
+import org.apache.hbase.thirdparty.com.google.common.io.Closeables;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -66,7 +66,6 @@ import org.mockito.Mockito;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.hbase.thirdparty.com.google.common.io.Closeables;
 
 /**
  * Testing for lock up of FSHLog.
@@ -155,8 +154,9 @@ public class TestWALLockup {
         // Don't countdown latch until someone waiting on it otherwise, the above
         // afterCreatingZigZagLatch will get to the latch and no one will ever free it and we'll
         // be stuck; test won't go down
-        while (this.latch.getCount() <= 0)
+        while (this.latch.getCount() <= 0) {
           Threads.sleep(1);
+        }
         this.latch.countDown();
       }
     }
@@ -253,7 +253,7 @@ public class TestWALLockup {
       LOG.info("SET throwing of exception on append");
       dodgyWAL.throwException = true;
       // This append provokes a WAL roll request
-      dodgyWAL.appendData(region.getRegionInfo(), key, edit);
+      dodgyWAL.appendData(key, edit);
       boolean exception = false;
       try {
         dodgyWAL.sync(false);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestFSWAL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestFSWAL.java
index 6d0771e..9aeac2d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestFSWAL.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestFSWAL.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
@@ -27,7 +27,6 @@ import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
-
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
@@ -187,7 +186,7 @@ public abstract class AbstractTestFSWAL {
       WALKeyImpl key = new WALKeyImpl(hri.getEncodedNameAsBytes(), htd.getTableName(),
           SequenceId.NO_SEQUENCE_ID, timestamp, WALKey.EMPTY_UUIDS, HConstants.NO_NONCE,
           HConstants.NO_NONCE, mvcc, scopes);
-      log.appendData(hri, key, cols);
+      log.appendData(key, cols);
     }
     log.sync();
   }
@@ -203,7 +202,6 @@ public abstract class AbstractTestFSWAL {
   /**
    * tests the log comparator. Ensure that we are not mixing meta logs with non-meta logs (throws
    * exception if we do). Comparison is based on the timestamp present in the wal name.
-   * @throws Exception
    */
   @Test
   public void testWALComparator() throws Exception {
@@ -261,7 +259,6 @@ public abstract class AbstractTestFSWAL {
    * This method tests this behavior by inserting edits and rolling the wal enough times to reach
    * the max number of logs threshold. It checks whether we get the "right regions and stores" for
    * flush on rolling the wal.
-   * @throws Exception
    */
   @Test
   public void testFindMemStoresEligibleForFlush() throws Exception {
@@ -406,7 +403,6 @@ public abstract class AbstractTestFSWAL {
    * slowing appends in the background ring buffer thread while in foreground we call flush. The
    * addition of the sync over HRegion in flush should fix an issue where flush was returning before
    * all of its appends had made it out to the WAL (HBASE-11109).
-   * @throws IOException
    * @see <a href="https://issues.apache.org/jira/browse/HBASE-11109">HBASE-11109</a>
    */
   @Test
@@ -470,7 +466,7 @@ public abstract class AbstractTestFSWAL {
         final RegionInfo info = region.getRegionInfo();
         final WALKeyImpl logkey = new WALKeyImpl(info.getEncodedNameAsBytes(), tableName,
             System.currentTimeMillis(), clusterIds, -1, -1, region.getMVCC(), scopes);
-        wal.append(info, logkey, edits, true);
+        wal.append(logkey, edits, true);
         region.getMVCC().completeAndWait(logkey.getWriteEntry());
       }
       region.flush(true);
@@ -520,7 +516,7 @@ public abstract class AbstractTestFSWAL {
         new WALKeyImpl(ri.getEncodedNameAsBytes(), td.getTableName(), SequenceId.NO_SEQUENCE_ID,
           timestamp, WALKey.EMPTY_UUIDS, HConstants.NO_NONCE, HConstants.NO_NONCE, mvcc, scopes);
     try {
-      wal.append(ri, key, cols, true);
+      wal.append(key, cols, true);
       fail("Should fail since the wal has already been closed");
     } catch (IOException e) {
       // expected
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java
index 9c069bd..5d58b3c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java
@@ -28,7 +28,6 @@ import static org.mockito.ArgumentMatchers.eq;
 import static org.mockito.Mockito.doAnswer;
 import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.when;
-
 import java.io.FilterInputStream;
 import java.io.IOException;
 import java.lang.reflect.Field;
@@ -182,10 +181,6 @@ public abstract class AbstractTestWALReplay {
     }
   }
 
-  /**
-   *
-   * @throws Exception
-   */
   @Test
   public void testReplayEditsAfterRegionMovedWithMultiCF() throws Exception {
     final TableName tableName =
@@ -263,7 +258,6 @@ public abstract class AbstractTestWALReplay {
 
   /**
    * Tests for hbase-2727.
-   * @throws Exception
    * @see <a href="https://issues.apache.org/jira/browse/HBASE-2727">HBASE-2727</a>
    */
   @Test
@@ -328,16 +322,11 @@ public abstract class AbstractTestWALReplay {
   /**
    * Test case of HRegion that is only made out of bulk loaded files.  Assert
    * that we don't 'crash'.
-   * @throws IOException
-   * @throws IllegalAccessException
-   * @throws NoSuchFieldException
-   * @throws IllegalArgumentException
-   * @throws SecurityException
    */
   @Test
-  public void testRegionMadeOfBulkLoadedFilesOnly()
-  throws IOException, SecurityException, IllegalArgumentException,
-      NoSuchFieldException, IllegalAccessException, InterruptedException {
+  public void testRegionMadeOfBulkLoadedFilesOnly() throws
+        IOException, SecurityException, IllegalArgumentException,
+          NoSuchFieldException, IllegalAccessException, InterruptedException {
     final TableName tableName =
         TableName.valueOf("testRegionMadeOfBulkLoadedFilesOnly");
     final HRegionInfo hri = createBasic3FamilyHRegionInfo(tableName);
@@ -394,11 +383,6 @@ public abstract class AbstractTestWALReplay {
    * files) and an edit in the memstore.
    * This is for HBASE-10958 "[dataloss] Bulk loading with seqids can prevent some log entries
    * from being replayed"
-   * @throws IOException
-   * @throws IllegalAccessException
-   * @throws NoSuchFieldException
-   * @throws IllegalArgumentException
-   * @throws SecurityException
    */
   @Test
   public void testCompactedBulkLoadedFiles()
@@ -464,11 +448,6 @@ public abstract class AbstractTestWALReplay {
   /**
    * Test writing edits into an HRegion, closing it, splitting logs, opening
    * Region again.  Verify seqids.
-   * @throws IOException
-   * @throws IllegalAccessException
-   * @throws NoSuchFieldException
-   * @throws IllegalArgumentException
-   * @throws SecurityException
    */
   @Test
   public void testReplayEditsWrittenViaHRegion()
@@ -571,17 +550,11 @@ public abstract class AbstractTestWALReplay {
    * died.
    *
    * We restart Region again, and verify that the edits were replayed.
-   *
-   * @throws IOException
-   * @throws IllegalAccessException
-   * @throws NoSuchFieldException
-   * @throws IllegalArgumentException
-   * @throws SecurityException
    */
   @Test
-  public void testReplayEditsAfterPartialFlush()
-  throws IOException, SecurityException, IllegalArgumentException,
-      NoSuchFieldException, IllegalAccessException, InterruptedException {
+  public void testReplayEditsAfterPartialFlush() throws
+      IOException, SecurityException, IllegalArgumentException,
+        NoSuchFieldException, IllegalAccessException, InterruptedException {
     final TableName tableName =
         TableName.valueOf("testReplayEditsWrittenViaHRegion");
     final HRegionInfo hri = createBasic3FamilyHRegionInfo(tableName);
@@ -663,7 +636,6 @@ public abstract class AbstractTestWALReplay {
    * Test that we could recover the data correctly after aborting flush. In the
    * test, first we abort flush after writing some data, then writing more data
    * and flush again, at last verify the data.
-   * @throws IOException
    */
   @Test
   public void testReplayEditsAfterAbortingFlush() throws IOException {
@@ -749,10 +721,12 @@ public abstract class AbstractTestWALReplay {
     List<Cell> results = new ArrayList<>();
     while (true) {
       boolean existMore = scanner.next(results);
-      if (!results.isEmpty())
+      if (!results.isEmpty()) {
         scannedCount++;
-      if (!existMore)
+      }
+      if (!existMore) {
         break;
+      }
       results.clear();
     }
     return scannedCount;
@@ -761,7 +735,6 @@ public abstract class AbstractTestWALReplay {
   /**
    * Create an HRegion with the result of a WAL split and test we only see the
    * good edits
-   * @throws Exception
    */
   @Test
   public void testReplayEditsWrittenIntoWAL() throws Exception {
@@ -801,15 +774,13 @@ public abstract class AbstractTestWALReplay {
     long now = ee.currentTime();
     edit.add(new KeyValue(rowName, Bytes.toBytes("another family"), rowName,
       now, rowName));
-    wal.appendData(hri, new WALKeyImpl(hri.getEncodedNameAsBytes(), tableName, now, mvcc, scopes),
-      edit);
+    wal.appendData(new WALKeyImpl(hri.getEncodedNameAsBytes(), tableName, now, mvcc, scopes), edit);
 
     // Delete the c family to verify deletes make it over.
     edit = new WALEdit();
     now = ee.currentTime();
     edit.add(new KeyValue(rowName, Bytes.toBytes("c"), null, now, KeyValue.Type.DeleteFamily));
-    wal.appendData(hri, new WALKeyImpl(hri.getEncodedNameAsBytes(), tableName, now, mvcc, scopes),
-      edit);
+    wal.appendData(new WALKeyImpl(hri.getEncodedNameAsBytes(), tableName, now, mvcc, scopes), edit);
 
     // Sync.
     wal.sync();
@@ -1163,7 +1134,7 @@ public abstract class AbstractTestWALReplay {
     byte[] rowName, byte[] family, EnvironmentEdge ee, MultiVersionConcurrencyControl mvcc,
     int index, NavigableMap<byte[], Integer> scopes) throws IOException {
     FSWALEntry entry = new FSWALEntry(sequence, createWALKey(htd.getTableName(), hri, mvcc, scopes),
-      createWALEdit(rowName, family, ee, index), hri, true, null);
+      createWALEdit(rowName, family, ee, index),true, null);
     entry.stampRegionSequenceId(mvcc.begin());
     return entry;
   }
@@ -1173,7 +1144,7 @@ public abstract class AbstractTestWALReplay {
       final HTableDescriptor htd, final MultiVersionConcurrencyControl mvcc,
       NavigableMap<byte[], Integer> scopes) throws IOException {
     for (int j = 0; j < count; j++) {
-      wal.appendData(hri, createWALKey(tableName, hri, mvcc, scopes),
+      wal.appendData(createWALKey(tableName, hri, mvcc, scopes),
         createWALEdit(rowName, family, ee, j));
     }
     wal.sync();
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncFSWAL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncFSWAL.java
index f413baf..2e9729a1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncFSWAL.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncFSWAL.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.regionserver.wal;
 import static org.junit.Assert.assertNull;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
-
 import java.io.IOException;
 import java.io.UncheckedIOException;
 import java.util.List;
@@ -202,7 +201,7 @@ public class TestAsyncFSWAL extends AbstractTestFSWAL {
                 SequenceId.NO_SEQUENCE_ID, timestamp, WALKey.EMPTY_UUIDS, HConstants.NO_NONCE,
                 HConstants.NO_NONCE, mvcc, scopes);
             try {
-              wal.append(ri, key, cols, true);
+              wal.append(key, cols, true);
             } catch (IOException e) {
               // should not happen
               throw new UncheckedIOException(e);
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 d0274de..5616ffb 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
@@ -207,7 +207,7 @@ public class TestLogRollAbort {
         kvs.add(new KeyValue(Bytes.toBytes(i), tableName.getName(), tableName.getName()));
         NavigableMap<byte[], Integer> scopes = new TreeMap<>(Bytes.BYTES_COMPARATOR);
         scopes.put(Bytes.toBytes("column"), 0);
-        log.appendData(regionInfo, new WALKeyImpl(regionInfo.getEncodedNameAsBytes(), tableName,
+        log.appendData(new WALKeyImpl(regionInfo.getEncodedNameAsBytes(), tableName,
           System.currentTimeMillis(), mvcc, scopes), kvs);
       }
       // Send the data to HDFS datanodes and close the HDFS writer
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java
index 0712b59..4a12486 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java
@@ -419,7 +419,6 @@ public class TestLogRolling extends AbstractTestLogRolling {
 
   /**
    * Test that WAL is rolled when all data nodes in the pipeline have been restarted.
-   * @throws Exception
    */
   @Test
   public void testLogRollOnPipelineRestart() throws Exception {
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 a8b6490..4119e49 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
@@ -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
@@ -18,7 +18,6 @@
 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;
@@ -78,8 +77,6 @@ public class TestLogRollingNoCluster {
   /**
    * Spin up a bunch of threads and have them all append to a WAL.  Roll the
    * WAL frequently to try and trigger NPE.
-   * @throws IOException
-   * @throws InterruptedException
    */
   @Test
   public void testContendedLogRolling() throws Exception {
@@ -171,7 +168,7 @@ public class TestLogRollingNoCluster {
           for(byte[] fam: this.metaTableDescriptor.getColumnFamilyNames()) {
             scopes.put(fam, 0);
           }
-          final long txid = wal.appendData(hri, new WALKeyImpl(hri.getEncodedNameAsBytes(),
+          final long txid = wal.appendData(new WALKeyImpl(hri.getEncodedNameAsBytes(),
             TableName.META_TABLE_NAME, now, mvcc, scopes), edit);
           Threads.sleep(ThreadLocalRandom.current().nextInt(5));
           wal.sync(txid);
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 e40d7e1..7ea79a5 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
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.regionserver.wal;
 
 import static org.junit.Assert.assertEquals;
-
 import java.util.NavigableMap;
 import java.util.TreeMap;
 import org.apache.hadoop.conf.Configuration;
@@ -111,8 +110,8 @@ public class TestWALActionsListener {
       edit.add(kv);
       NavigableMap<byte[], Integer> scopes = new TreeMap<>(Bytes.BYTES_COMPARATOR);
       scopes.put(b, 0);
-      long txid = wal.appendData(hri,
-        new WALKeyImpl(hri.getEncodedNameAsBytes(), TableName.valueOf(b), 0, mvcc, scopes), edit);
+      long txid = wal.appendData(new WALKeyImpl(hri.getEncodedNameAsBytes(), TableName.valueOf(b),
+        0, mvcc, scopes), edit);
       wal.sync(txid);
       if (i == 10) {
         wal.registerWALActionsListener(laterobserver);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEmptyWALRecovery.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEmptyWALRecovery.java
index 2d72618..d6dd0f3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEmptyWALRecovery.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationEmptyWALRecovery.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
@@ -362,7 +362,7 @@ import org.junit.experimental.categories.Category;
       KeyValue kv = new KeyValue(b, famName, b);
       WALEdit edit = new WALEdit();
       edit.add(kv);
-      txId = wal.appendData(info, getWalKeyImpl(), edit);
+      txId = wal.appendData(getWalKeyImpl(), edit);
     }
     wal.sync(txId);
   }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java
index 7c8f97b..72ff165 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java
@@ -21,7 +21,6 @@ import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
-
 import java.util.ArrayList;
 import java.util.List;
 import java.util.NavigableMap;
@@ -51,6 +50,7 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.wal.WAL;
 import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.wal.WALKeyImpl;
+import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList;
 import org.junit.Before;
 import org.junit.ClassRule;
 import org.junit.Test;
@@ -62,7 +62,6 @@ import org.junit.runners.Parameterized.Parameters;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList;
 
 @RunWith(Parameterized.class)
 @Category({ ReplicationTests.class, LargeTests.class })
@@ -424,7 +423,7 @@ public class TestReplicationSmallTests extends TestReplicationBase {
     long now = EnvironmentEdgeManager.currentTime();
     edit.add(new KeyValue(rowName, famName, qualifier, now, value));
     WALKeyImpl walKey = new WALKeyImpl(hri.getEncodedNameAsBytes(), tableName, now, mvcc, scopes);
-    wal.appendData(hri, walKey, edit);
+    wal.appendData(walKey, edit);
     wal.sync();
 
     Get get = new Get(rowName);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpointNoMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpointNoMaster.java
index 1c369e2..facfd69 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpointNoMaster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpointNoMaster.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
@@ -39,14 +39,11 @@ import org.apache.hadoop.hbase.StartMiniClusterOption;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
-import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionLocator;
 import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
-import org.apache.hadoop.hbase.coprocessor.ObserverContext;
 import org.apache.hadoop.hbase.coprocessor.WALCoprocessor;
-import org.apache.hadoop.hbase.coprocessor.WALCoprocessorEnvironment;
 import org.apache.hadoop.hbase.coprocessor.WALObserver;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
@@ -59,9 +56,6 @@ import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.ReplicationTests;
 import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
 import org.apache.hadoop.hbase.wal.WAL.Entry;
-import org.apache.hadoop.hbase.wal.WALEdit;
-import org.apache.hadoop.hbase.wal.WALKey;
-import org.apache.hadoop.hbase.wal.WALKeyImpl;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Assert;
@@ -87,7 +81,7 @@ public class TestRegionReplicaReplicationEndpointNoMaster {
       HBaseClassTestRule.forClass(TestRegionReplicaReplicationEndpointNoMaster.class);
 
   private static final int NB_SERVERS = 2;
-  private static TableName tableName = TableName.valueOf(
+  private static final TableName tableName = TableName.valueOf(
     TestRegionReplicaReplicationEndpointNoMaster.class.getSimpleName());
   private static Table table;
   private static final byte[] row = "TestRegionReplicaReplicator".getBytes();
@@ -164,16 +158,6 @@ public class TestRegionReplicaReplicationEndpointNoMaster {
     public Optional<WALObserver> getWALObserver() {
       return Optional.of(this);
     }
-
-    @Override
-    public void postWALWrite(ObserverContext<? extends WALCoprocessorEnvironment> ctx,
-                             RegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {
-      // only keep primary region's edits
-      if (logKey.getTableName().equals(tableName) && info.getReplicaId() == 0) {
-        // Presume type is a WALKeyImpl
-        entries.add(new Entry((WALKeyImpl)logKey, logEdit));
-      }
-    }
   }
 
   @Test
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java
index 57900cd..7d9edfc 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.replication.regionserver;
 
 import static org.junit.Assert.assertEquals;
-
 import java.security.SecureRandom;
 import java.util.ArrayList;
 import java.util.Collections;
@@ -54,11 +53,17 @@ import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WALEntry;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.UUID;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALKey;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.ReplicationTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.HFileTestUtil;
+import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.Before;
@@ -69,14 +74,6 @@ import org.junit.experimental.categories.Category;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-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.AdminProtos.WALEntry;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.UUID;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALKey;
-
 @Category({ReplicationTests.class, LargeTests.class})
 public class TestReplicationSink {
 
@@ -119,9 +116,6 @@ public class TestReplicationSink {
   protected static String hfileArchiveDir;
   protected static String replicationClusterId;
 
-   /**
-   * @throws java.lang.Exception
-   */
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
     TEST_UTIL.getConfiguration().set("hbase.replication.source.fs.conf.provider",
@@ -136,18 +130,12 @@ public class TestReplicationSink {
     replicationClusterId = "12345";
   }
 
-  /**
-   * @throws java.lang.Exception
-   */
   @AfterClass
   public static void tearDownAfterClass() throws Exception {
     STOPPABLE.stop("Shutting down");
     TEST_UTIL.shutdownMiniCluster();
   }
 
-  /**
-   * @throws java.lang.Exception
-   */
   @Before
   public void setUp() throws Exception {
     table1 = TEST_UTIL.deleteTableData(TABLE_NAME1);
@@ -156,13 +144,12 @@ public class TestReplicationSink {
 
   /**
    * Insert a whole batch of entries
-   * @throws Exception
    */
   @Test
   public void testBatchSink() throws Exception {
     List<WALEntry> entries = new ArrayList<>(BATCH_SIZE);
     List<Cell> cells = new ArrayList<>();
-    for(int i = 0; i < BATCH_SIZE; i++) {
+    for (int i = 0; i < BATCH_SIZE; i++) {
       entries.add(createEntry(TABLE_NAME1, i, KeyValue.Type.Put, cells));
     }
     SINK.replicateEntries(entries, CellUtil.createCellScanner(cells.iterator()),
@@ -174,7 +161,6 @@ public class TestReplicationSink {
 
   /**
    * Insert a mix of puts and deletes
-   * @throws Exception
    */
   @Test
   public void testMixedPutDelete() throws Exception {
@@ -236,15 +222,13 @@ public class TestReplicationSink {
 
   /**
    * Insert to 2 different tables
-   * @throws Exception
    */
   @Test
   public void testMixedPutTables() throws Exception {
     List<WALEntry> entries = new ArrayList<>(BATCH_SIZE/2);
     List<Cell> cells = new ArrayList<>();
-    for(int i = 0; i < BATCH_SIZE; i++) {
-      entries.add(createEntry( i % 2 == 0 ? TABLE_NAME2 : TABLE_NAME1,
-              i, KeyValue.Type.Put, cells));
+    for (int i = 0; i < BATCH_SIZE; i++) {
+      entries.add(createEntry(i % 2 == 0 ? TABLE_NAME2 : TABLE_NAME1, i, KeyValue.Type.Put, cells));
     }
 
     SINK.replicateEntries(entries, CellUtil.createCellScanner(cells.iterator()),
@@ -262,7 +246,6 @@ public class TestReplicationSink {
 
   /**
    * Insert then do different types of deletes
-   * @throws Exception
    */
   @Test
   public void testMixedDeletes() throws Exception {
@@ -290,7 +273,6 @@ public class TestReplicationSink {
   /**
    * Puts are buffered, but this tests when a delete (not-buffered) is applied
    * before the actual Put that creates it.
-   * @throws Exception
    */
   @Test
   public void testApplyDeleteBeforePut() throws Exception {
@@ -390,8 +372,8 @@ public class TestReplicationSink {
           ProtobufUtil.toBulkLoadDescriptor(TABLE_NAME1,
               UnsafeByteOperations.unsafeWrap(regionInfo.getEncodedNameAsBytes()),
               storeFiles, storeFilesSize, 1);
-      edit = org.apache.hadoop.hbase.wal.WALEdit.createBulkLoadEvent(regionInfo,
-        loadDescriptor);
+      edit = org.apache.hadoop.hbase.wal.WALEdit.createBulkLoadEvent(
+        regionInfo.getNonEmptyStartKey(), loadDescriptor);
     }
     List<WALEntry> entries = new ArrayList<>(1);
 
@@ -400,12 +382,12 @@ public class TestReplicationSink {
 
     // 5. Copy the hfile to the path as it is in reality
     for (int i = 0; i < 25; i++) {
-      String pathToHfileFromNS =
-          new StringBuilder(100).append(TABLE_NAME1.getNamespaceAsString()).append(Path.SEPARATOR)
-              .append(Bytes.toString(TABLE_NAME1.getName())).append(Path.SEPARATOR)
-              .append(Bytes.toString(loadDescriptor.getEncodedRegionName().toByteArray()))
-              .append(Path.SEPARATOR).append(Bytes.toString(FAM_NAME1)).append(Path.SEPARATOR)
-              .append(hfilePrefix + i).toString();
+      String pathToHfileFromNS = new StringBuilder(100).
+        append(TABLE_NAME1.getNamespaceAsString()).append(Path.SEPARATOR).
+        append(Bytes.toString(TABLE_NAME1.getName())).append(Path.SEPARATOR).
+        append(Bytes.toString(loadDescriptor.getEncodedRegionName().toByteArray())).
+        append(Path.SEPARATOR).append(Bytes.toString(FAM_NAME1)).append(Path.SEPARATOR).
+        append(hfilePrefix + i).toString();
       String dst = baseNamespaceDir + Path.SEPARATOR + pathToHfileFromNS;
       Path dstPath = new Path(dst);
       FileUtil.copy(fs, p.get(0), fs, dstPath, false, conf);
@@ -439,14 +421,11 @@ public class TestReplicationSink {
     final long now = System.currentTimeMillis();
     KeyValue kv = null;
     if(type.getCode() == KeyValue.Type.Put.getCode()) {
-      kv = new KeyValue(rowBytes, fam, fam, now,
-          KeyValue.Type.Put, Bytes.toBytes(row));
+      kv = new KeyValue(rowBytes, fam, fam, now, KeyValue.Type.Put, Bytes.toBytes(row));
     } else if (type.getCode() == KeyValue.Type.DeleteColumn.getCode()) {
-        kv = new KeyValue(rowBytes, fam, fam,
-            now, KeyValue.Type.DeleteColumn);
+      kv = new KeyValue(rowBytes, fam, fam, now, KeyValue.Type.DeleteColumn);
     } else if (type.getCode() == KeyValue.Type.DeleteFamily.getCode()) {
-        kv = new KeyValue(rowBytes, fam, null,
-            now, KeyValue.Type.DeleteFamily);
+      kv = new KeyValue(rowBytes, fam, null, now, KeyValue.Type.DeleteFamily);
     }
     WALEntry.Builder builder = createWALEntryBuilder(table);
     cells.add(kv);
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 2745ad6..5e642e3 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
@@ -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
@@ -22,7 +22,6 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
-
 import java.io.IOException;
 import java.lang.reflect.Field;
 import java.net.URLEncoder;
@@ -96,10 +95,8 @@ import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-
 import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
 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.WALProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.BulkLoadDescriptor;
@@ -291,7 +288,7 @@ public abstract class TestReplicationSourceManager {
         wal.rollWriter();
       }
       LOG.info(Long.toString(i));
-      final long txid = wal.appendData(hri,
+      final long txid = wal.appendData(
         new WALKeyImpl(hri.getEncodedNameAsBytes(), test, System.currentTimeMillis(), mvcc, scopes),
         edit);
       wal.sync(txid);
@@ -305,7 +302,7 @@ public abstract class TestReplicationSourceManager {
     LOG.info(baseline + " and " + time);
 
     for (int i = 0; i < 3; i++) {
-      wal.appendData(hri,
+      wal.appendData(
         new WALKeyImpl(hri.getEncodedNameAsBytes(), test, System.currentTimeMillis(), mvcc, scopes),
         edit);
     }
@@ -323,7 +320,7 @@ public abstract class TestReplicationSourceManager {
     manager.logPositionAndCleanOldLogs(manager.getSources().get(0),
       new WALEntryBatch(0, manager.getSources().get(0).getCurrentPath()));
 
-    wal.appendData(hri,
+    wal.appendData(
       new WALKeyImpl(hri.getEncodedNameAsBytes(), test, System.currentTimeMillis(), mvcc, scopes),
       edit);
     wal.sync();
@@ -431,7 +428,7 @@ public abstract class TestReplicationSourceManager {
       WALProtos.CompactionDescriptor.getDefaultInstance();
     RegionInfo hri = RegionInfoBuilder.newBuilder(tableName).setStartKey(HConstants.EMPTY_START_ROW)
         .setEndKey(HConstants.EMPTY_END_ROW).build();
-    WALEdit edit = WALEdit.createCompaction(hri, compactionDescriptor);
+    WALEdit edit = WALEdit.createCompaction(hri.getNonEmptyStartKey(), compactionDescriptor);
     ReplicationSourceWALActionListener.scopeWALEdits(new WALKeyImpl(), edit, conf);
   }
 
@@ -479,7 +476,6 @@ public abstract class TestReplicationSourceManager {
    * corresponding ReplicationSourceInterface correctly cleans up the corresponding
    * replication queue and ReplicationPeer.
    * See HBASE-16096.
-   * @throws Exception
    */
   @Test
   public void testPeerRemovalCleanup() throws Exception{
@@ -575,10 +571,7 @@ public abstract class TestReplicationSourceManager {
 
   /**
    * Add a peer and wait for it to initialize
-   * @param peerId
-   * @param peerConfig
    * @param waitForSource Whether to wait for replication source to initialize
-   * @throws Exception
    */
   private void addPeerAndWait(final String peerId, final ReplicationPeerConfig peerConfig,
       final boolean waitForSource) throws Exception {
@@ -617,8 +610,6 @@ public abstract class TestReplicationSourceManager {
 
   /**
    * Remove a peer and wait for it to get cleaned up
-   * @param peerId
-   * @throws Exception
    */
   private void removePeerAndWait(final String peerId) throws Exception {
     final ReplicationPeers rp = manager.getReplicationPeers();
@@ -671,7 +662,7 @@ public abstract class TestReplicationSourceManager {
       UnsafeByteOperations.unsafeWrap(hri.getEncodedNameAsBytes()), storeFiles, storeFilesSize, 1);
 
     // 3. create bulk load wal edit event
-    WALEdit logEdit = WALEdit.createBulkLoadEvent(hri, desc);
+    WALEdit logEdit = WALEdit.createBulkLoadEvent(hri.getNonEmptyStartKey(), desc);
     return logEdit;
   }
 
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 ae9bb67..cbc04e5 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,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
@@ -582,7 +582,7 @@ public class TestWALEntryStream {
   }
 
   private void appendToLog(String key) throws IOException {
-    final long txid = log.appendData(info,
+    final long txid = log.appendData(
       new WALKeyImpl(info.getEncodedNameAsBytes(), tableName, System.currentTimeMillis(),
           mvcc, scopes), getWALEdit(key));
     log.sync(txid);
@@ -606,7 +606,7 @@ public class TestWALEntryStream {
   }
 
   private long appendToLog(int count) throws IOException {
-    return log.appendData(info, new WALKeyImpl(info.getEncodedNameAsBytes(), tableName,
+    return log.appendData(new WALKeyImpl(info.getEncodedNameAsBytes(), tableName,
       System.currentTimeMillis(), mvcc, scopes), getWALEdits(count));
   }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/FaultyFSLog.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/FaultyFSLog.java
index f58d364..549abaa 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/FaultyFSLog.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/FaultyFSLog.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
@@ -20,11 +19,9 @@
 package org.apache.hadoop.hbase.wal;
 
 import java.io.IOException;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.regionserver.wal.FSHLog;
 import org.apache.yetus.audience.InterfaceAudience;
 
@@ -63,12 +60,12 @@ public class FaultyFSLog extends FSHLog {
   }
 
   @Override
-  protected long append(RegionInfo info, WALKeyImpl key, WALEdit edits, boolean inMemstore)
+  protected long append(WALKeyImpl key, WALEdit edits, boolean inMemstore)
       throws IOException {
     if (this.ft == FailureType.APPEND) {
       throw new IOException("append");
     }
-    return super.append(info, key, edits, inMemstore);
+    return super.append(key, edits, inMemstore);
   }
 }
 
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 72e4998..a3da7fd 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
@@ -21,7 +21,6 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
-
 import java.io.IOException;
 import java.util.HashSet;
 import java.util.NavigableMap;
@@ -157,15 +156,14 @@ public class TestFSHLogProvider {
       long timestamp = System.currentTimeMillis();
       WALEdit cols = new WALEdit();
       cols.add(new KeyValue(row, row, row, timestamp, row));
-      log.appendData(hri,
-        getWalKey(hri.getEncodedNameAsBytes(), htd.getTableName(), timestamp, scopes), cols);
+      log.appendData(getWalKey(hri.getEncodedNameAsBytes(), htd.getTableName(), timestamp, scopes),
+        cols);
     }
     log.sync();
   }
 
   /**
    * used by TestDefaultWALProviderWithHLogKey
-   * @param scopes
    */
   WALKeyImpl getWalKey(final byte[] info, final TableName tableName, final long timestamp,
       NavigableMap<byte[], Integer> scopes) {
@@ -174,8 +172,6 @@ public class TestFSHLogProvider {
 
   /**
    * helper method to simulate region flush for a WAL.
-   * @param wal
-   * @param regionEncodedName
    */
   protected void flushRegion(WAL wal, byte[] regionEncodedName, Set<byte[]> flushedFamilyNames) {
     wal.startCacheFlush(regionEncodedName, flushedFamilyNames);
@@ -261,7 +257,6 @@ public class TestFSHLogProvider {
    * archiving if for all the regions which have entries in that wal file, have flushed - past
    * their maximum sequence id in that wal file.
    * <p>
-   * @throws IOException
    */
   @Test
   public void testWALArchiving() throws IOException {
@@ -337,7 +332,6 @@ public class TestFSHLogProvider {
 
   /**
    * Write to a log file with three concurrent threads and verifying all data is written.
-   * @throws Exception
    */
   @Test
   public void testConcurrentWrites() throws Exception {
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 d8f82d8..aee2a1d 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
@@ -129,7 +129,7 @@ public class TestSecureWAL {
     for (int i = 0; i < total; i++) {
       WALEdit kvs = new WALEdit();
       kvs.add(new KeyValue(row, family, Bytes.toBytes(i), value));
-      wal.appendData(regionInfo, new WALKeyImpl(regionInfo.getEncodedNameAsBytes(), tableName,
+      wal.appendData(new WALKeyImpl(regionInfo.getEncodedNameAsBytes(), tableName,
         System.currentTimeMillis(), mvcc, scopes), kvs);
     }
     wal.sync();
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 f1ac464..d053f8b 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,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
@@ -24,7 +24,6 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
-
 import java.io.IOException;
 import java.io.InputStream;
 import java.lang.reflect.Method;
@@ -35,7 +34,6 @@ import java.util.NavigableMap;
 import java.util.TreeMap;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.stream.Collectors;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
@@ -180,7 +178,6 @@ public class TestWALFactory {
   /**
    * Just write multiple logs then split.  Before fix for HADOOP-2283, this
    * would fail.
-   * @throws IOException
    */
   @Test
   public void testSplit() throws IOException {
@@ -219,7 +216,7 @@ public class TestWALFactory {
           LOG.info("Region " + i + ": " + edit);
           WALKeyImpl walKey =  new WALKeyImpl(infos[i].getEncodedNameAsBytes(), tableName,
               System.currentTimeMillis(), mvcc, scopes);
-          log.appendData(infos[i], walKey, edit);
+          log.appendData(walKey, edit);
           walKey.getWriteEntry();
         }
         log.sync();
@@ -227,11 +224,11 @@ public class TestWALFactory {
       }
     }
     wals.shutdown();
-    // The below calculation of logDir relies on insider information... WALSplitter should be connected better
-    // with the WAL system.... not requiring explicit path. The oldLogDir is just made up not used.
-    Path logDir =
-        new Path(new Path(hbaseWALDir, HConstants.HREGION_LOGDIR_NAME),
-            this.currentServername.toString());
+    // The below calculation of logDir relies on insider information... WALSplitter should be
+    // connected better with the WAL system.... not requiring explicit path. The oldLogDir is just
+    // made up not used.
+    Path logDir = new Path(new Path(hbaseWALDir, HConstants.HREGION_LOGDIR_NAME),
+      this.currentServername.toString());
     Path oldLogDir = new Path(hbaseDir, HConstants.HREGION_OLDLOGDIR_NAME);
     List<Path> splits = WALSplitter.split(hbaseWALDir, logDir, oldLogDir, fs, conf, wals);
     verifySplits(splits, howmany);
@@ -239,7 +236,6 @@ public class TestWALFactory {
 
   /**
    * Test new HDFS-265 sync.
-   * @throws Exception
    */
   @Test
   public void Broken_testSync() throws Exception {
@@ -281,7 +277,7 @@ public class TestWALFactory {
       for (int i = 0; i < total; i++) {
         WALEdit kvs = new WALEdit();
         kvs.add(new KeyValue(Bytes.toBytes(i), tableName.getName(), tableName.getName()));
-        wal.appendData(info, new WALKeyImpl(info.getEncodedNameAsBytes(), tableName,
+        wal.appendData(new WALKeyImpl(info.getEncodedNameAsBytes(), tableName,
           System.currentTimeMillis(), mvcc, scopes), kvs);
       }
       // Now call sync and try reading.  Opening a Reader before you sync just
@@ -292,7 +288,9 @@ public class TestWALFactory {
       reader = wals.createReader(fs, walPath);
       int count = 0;
       WAL.Entry entry = new WAL.Entry();
-      while ((entry = reader.next(entry)) != null) count++;
+      while ((entry = reader.next(entry)) != null) {
+        count++;
+      }
       assertEquals(total, count);
       reader.close();
       // Add test that checks to see that an open of a Reader works on a file
@@ -300,20 +298,24 @@ public class TestWALFactory {
       for (int i = 0; i < total; i++) {
         WALEdit kvs = new WALEdit();
         kvs.add(new KeyValue(Bytes.toBytes(i), tableName.getName(), tableName.getName()));
-        wal.appendData(info, new WALKeyImpl(info.getEncodedNameAsBytes(), tableName,
+        wal.appendData(new WALKeyImpl(info.getEncodedNameAsBytes(), tableName,
           System.currentTimeMillis(), mvcc, scopes), kvs);
       }
       wal.sync();
       reader = wals.createReader(fs, walPath);
       count = 0;
-      while((entry = reader.next(entry)) != null) count++;
+      while((entry = reader.next(entry)) != null) {
+        count++;
+      }
       assertTrue(count >= total);
       reader.close();
       // If I sync, should see double the edits.
       wal.sync();
       reader = wals.createReader(fs, walPath);
       count = 0;
-      while((entry = reader.next(entry)) != null) count++;
+      while((entry = reader.next(entry)) != null) {
+        count++;
+      }
       assertEquals(total * 2, count);
       reader.close();
       // Now do a test that ensures stuff works when we go over block boundary,
@@ -322,14 +324,16 @@ public class TestWALFactory {
       for (int i = 0; i < total; i++) {
         WALEdit kvs = new WALEdit();
         kvs.add(new KeyValue(Bytes.toBytes(i), tableName.getName(), value));
-        wal.appendData(info, new WALKeyImpl(info.getEncodedNameAsBytes(), tableName,
+        wal.appendData(new WALKeyImpl(info.getEncodedNameAsBytes(), tableName,
           System.currentTimeMillis(), mvcc, scopes), kvs);
       }
       // Now I should have written out lots of blocks.  Sync then read.
       wal.sync();
       reader = wals.createReader(fs, walPath);
       count = 0;
-      while((entry = reader.next(entry)) != null) count++;
+      while((entry = reader.next(entry)) != null) {
+        count++;
+      }
       assertEquals(total * 3, count);
       reader.close();
       // shutdown and ensure that Reader gets right length also.
@@ -340,12 +344,13 @@ public class TestWALFactory {
       assertEquals(total * 3, count);
       reader.close();
     } finally {
-      if (reader != null) reader.close();
+      if (reader != null) {
+        reader.close();
+      }
     }
   }
 
-  private void verifySplits(final List<Path> splits, final int howmany)
-  throws IOException {
+  private void verifySplits(final List<Path> splits, final int howmany) throws IOException {
     assertEquals(howmany * howmany, splits.size());
     for (int i = 0; i < splits.size(); i++) {
       LOG.info("Verifying=" + splits.get(i));
@@ -399,12 +404,12 @@ public class TestWALFactory {
     for (int i = 0; i < total; i++) {
       WALEdit kvs = new WALEdit();
       kvs.add(new KeyValue(Bytes.toBytes(i), tableName.getName(), tableName.getName()));
-      wal.appendData(regionInfo, new WALKeyImpl(regionInfo.getEncodedNameAsBytes(), tableName,
+      wal.appendData(new WALKeyImpl(regionInfo.getEncodedNameAsBytes(), tableName,
         System.currentTimeMillis(), mvcc, scopes), kvs);
     }
     // Now call sync to send the data to HDFS datanodes
     wal.sync();
-     int namenodePort = cluster.getNameNodePort();
+    int namenodePort = cluster.getNameNodePort();
     final Path walPath = AbstractFSWALProvider.getCurrentFileName(wal);
 
 
@@ -482,8 +487,9 @@ public class TestWALFactory {
       throw new Exception("Timed out waiting for WAL.recoverLog()");
     }
 
-    if (t.exception != null)
+    if (t.exception != null) {
       throw t.exception;
+    }
 
     // Make sure you can read all the content
     WAL.Reader reader = wals.createReader(fs, walPath);
@@ -532,7 +538,7 @@ public class TestWALFactory {
           .setEndKey(Bytes.toBytes(Bytes.toString(row) + "1")).build();
       final WAL log = wals.getWAL(info);
 
-      final long txid = log.appendData(info, new WALKeyImpl(info.getEncodedNameAsBytes(),
+      final long txid = log.appendData(new WALKeyImpl(info.getEncodedNameAsBytes(),
         htd.getTableName(), System.currentTimeMillis(), mvcc, scopes), cols);
       log.sync(txid);
       log.startCacheFlush(info.getEncodedNameAsBytes(), htd.getColumnFamilyNames());
@@ -545,7 +551,9 @@ public class TestWALFactory {
       // entry in the below... thats why we have '1'.
       for (int i = 0; i < 1; i++) {
         WAL.Entry entry = reader.next(null);
-        if (entry == null) break;
+        if (entry == null) {
+          break;
+        }
         WALKey key = entry.getKey();
         WALEdit val = entry.getEdit();
         assertTrue(Bytes.equals(info.getEncodedNameAsBytes(), key.getEncodedRegionName()));
@@ -588,7 +596,7 @@ public class TestWALFactory {
       }
       RegionInfo hri = RegionInfoBuilder.newBuilder(htd.getTableName()).build();
       final WAL log = wals.getWAL(hri);
-      final long txid = log.appendData(hri, new WALKeyImpl(hri.getEncodedNameAsBytes(),
+      final long txid = log.appendData(new WALKeyImpl(hri.getEncodedNameAsBytes(),
         htd.getTableName(), System.currentTimeMillis(), mvcc, scopes), cols);
       log.sync(txid);
       log.startCacheFlush(hri.getEncodedNameAsBytes(), htd.getColumnFamilyNames());
@@ -619,7 +627,6 @@ public class TestWALFactory {
 
   /**
    * Test that we can visit entries before they are appended
-   * @throws Exception
    */
   @Test
   public void testVisitors() throws Exception {
@@ -640,7 +647,7 @@ public class TestWALFactory {
       cols.add(new KeyValue(row, Bytes.toBytes("column"),
           Bytes.toBytes(Integer.toString(i)),
           timestamp, new byte[]{(byte) (i + '0')}));
-      log.appendData(hri, new WALKeyImpl(hri.getEncodedNameAsBytes(), tableName,
+      log.appendData(new WALKeyImpl(hri.getEncodedNameAsBytes(), tableName,
         System.currentTimeMillis(), mvcc, scopes), cols);
     }
     log.sync();
@@ -650,7 +657,7 @@ public class TestWALFactory {
     cols.add(new KeyValue(row, Bytes.toBytes("column"),
         Bytes.toBytes(Integer.toString(11)),
         timestamp, new byte[]{(byte) (11 + '0')}));
-    log.appendData(hri, new WALKeyImpl(hri.getEncodedNameAsBytes(), tableName,
+    log.appendData(new WALKeyImpl(hri.getEncodedNameAsBytes(), tableName,
       System.currentTimeMillis(), mvcc, scopes), cols);
     log.sync();
     assertEquals(COL_COUNT, visitor.increments);
@@ -669,12 +676,6 @@ public class TestWALFactory {
 
   static class DumbWALActionsListener implements WALActionsListener {
     int increments = 0;
-
-    @Override
-    public void visitLogEntryBeforeWrite(RegionInfo info, WALKey logKey, WALEdit logEdit) {
-      increments++;
-    }
-
     @Override
     public void visitLogEntryBeforeWrite(WALKey logKey, WALEdit logEdit) {
       // To change body of implemented methods use File | Settings | File
@@ -762,7 +763,8 @@ public class TestWALFactory {
   public void testReaderClosedOnBadCodec() throws IOException {
     // Create our own Configuration and WALFactory to avoid breaking other test methods
     Configuration confWithCodec = new Configuration(conf);
-    confWithCodec.setClass(WALCellCodec.WAL_CELL_CODEC_CLASS_KEY, BrokenWALCellCodec.class, Codec.class);
+    confWithCodec.setClass(WALCellCodec.WAL_CELL_CODEC_CLASS_KEY, BrokenWALCellCodec.class,
+      Codec.class);
     WALFactory customFactory = new WALFactory(confWithCodec, this.currentServername.toString());
 
     // Hack a Proxy over the FileSystem so that we can track the InputStreams opened by
@@ -802,7 +804,7 @@ public class TestWALFactory {
       cols.add(new KeyValue(row, Bytes.toBytes("column"),
         Bytes.toBytes("0"), System.currentTimeMillis(), new byte[] { 0 }));
       final WAL log = customFactory.getWAL(hri);
-      final long txid = log.appendData(hri, new WALKeyImpl(hri.getEncodedNameAsBytes(),
+      final long txid = log.appendData(new WALKeyImpl(hri.getEncodedNameAsBytes(),
         htd.getTableName(), System.currentTimeMillis(), mvcc, scopes), cols);
       // Sync the edit to the WAL
       log.sync(txid);
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 f36c5b9..02a800b 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
@@ -117,7 +117,7 @@ public class TestWALReaderOnSecureWAL {
         } else {
           kvs.add(kv);
         }
-        wal.appendData(regionInfo, new WALKeyImpl(regionInfo.getEncodedNameAsBytes(), tableName,
+        wal.appendData(new WALKeyImpl(regionInfo.getEncodedNameAsBytes(), tableName,
           System.currentTimeMillis(), mvcc, scopes), kvs);
       }
       wal.sync();
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 017b540..e109d9a 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
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.wal;
 
 import static org.junit.Assert.assertEquals;
-
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
@@ -98,8 +97,7 @@ public class TestWALRootDir {
     WALEdit edit = new WALEdit();
     edit.add(new KeyValue(rowName, family, Bytes.toBytes("1"),
         System.currentTimeMillis(), value));
-    long txid =
-      log.appendData(regionInfo, getWalKey(System.currentTimeMillis(), regionInfo, 0), edit);
+    long txid = log.appendData(getWalKey(System.currentTimeMillis(), regionInfo, 0), edit);
     log.sync(txid);
     assertEquals("Expect 1 log have been created", 1,
         getWALFiles(walFs, walRootDir).size());
@@ -109,7 +107,7 @@ public class TestWALRootDir {
         HConstants.HREGION_LOGDIR_NAME)).size());
     edit.add(new KeyValue(rowName, family, Bytes.toBytes("2"),
         System.currentTimeMillis(), value));
-    txid = log.appendData(regionInfo, getWalKey(System.currentTimeMillis(), regionInfo, 1), edit);
+    txid = log.appendData(getWalKey(System.currentTimeMillis(), regionInfo, 1), edit);
     log.sync(txid);
     log.rollWriter();
     log.shutdown();
@@ -129,7 +127,9 @@ public class TestWALRootDir {
     LOG.debug("Scanning " + dir.toString() + " for WAL files");
 
     FileStatus[] files = fs.listStatus(dir);
-    if (files == null) return Collections.emptyList();
+    if (files == null) {
+      return Collections.emptyList();
+    }
     for (FileStatus file : files) {
       if (file.isDirectory()) {
         // recurse into sub directories
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 5f22b45..62e7cbf 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
@@ -62,6 +62,7 @@ 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.regionserver.wal.WALUtil;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
@@ -224,7 +225,9 @@ public class TestWALSplit {
       long startCount = counter.get();
       zombie.start();
       // Wait till writer starts going.
-      while (startCount == counter.get()) Threads.sleep(1);
+      while (startCount == counter.get()) {
+        Threads.sleep(1);
+      }
       // Give it a second to write a few appends.
       Threads.sleep(1000);
       final Configuration conf2 = HBaseConfiguration.create(conf);
@@ -679,7 +682,9 @@ public class TestWALSplit {
     Reader in = wals.createReader(fs, splitLog[0]);
     @SuppressWarnings("unused")
     Entry entry;
-    while ((entry = in.next()) != null) ++actualCount;
+    while ((entry = in.next()) != null) {
+      ++actualCount;
+    }
     assertEquals(expectedCount, actualCount);
     in.close();
 
@@ -815,7 +820,9 @@ public class TestWALSplit {
     final Thread someOldThread = new Thread("Some-old-thread") {
       @Override
       public void run() {
-        while(!stop.get()) Threads.sleep(10);
+       while(!stop.get()) {
+         Threads.sleep(10);
+       }
       }
     };
     someOldThread.setDaemon(true);
@@ -909,8 +916,7 @@ public class TestWALSplit {
 
     final AtomicInteger count = new AtomicInteger();
 
-    CancelableProgressable localReporter
-        = new CancelableProgressable() {
+    CancelableProgressable localReporter = new CancelableProgressable() {
       @Override
       public boolean progress() {
         count.getAndIncrement();
@@ -1031,11 +1037,13 @@ public class TestWALSplit {
 
           @Override
           public Entry answer(InvocationOnMock invocation) throws Throwable {
-            if (index >= numFakeEdits) return null;
+            if (index >= numFakeEdits) {
+              return null;
+            }
 
             // Generate r0 through r4 in round robin fashion
             int regionIdx = index % regions.size();
-            byte region[] = new byte[] {(byte)'r', (byte) (0x30 + regionIdx)};
+            byte [] region = new byte[] {(byte)'r', (byte) (0x30 + regionIdx)};
 
             Entry ret = createTestEntry(TABLE_NAME, region,
                 Bytes.toBytes(index / regions.size()),
@@ -1193,7 +1201,8 @@ public class TestWALSplit {
    * @param leaveOpen index to leave un-closed. -1 to close all.
    * @return the writer that's still open, or null if all were closed.
    */
-  private Writer generateWALs(int writers, int entries, int leaveOpen, int regionEvents) throws IOException {
+  private Writer generateWALs(int writers, int entries, int leaveOpen, int regionEvents)
+      throws IOException {
     makeRegionDirs(REGIONS);
     fs.mkdirs(WALDIR);
     Writer [] ws = new Writer[writers];
@@ -1348,7 +1357,7 @@ public class TestWALSplit {
         .addAllCompactionInput(Arrays.asList(inputs))
         .addCompactionOutput(output);
 
-    WALEdit edit = WALEdit.createCompaction(hri, desc.build());
+    WALEdit edit = WALEdit.createCompaction(WALUtil.getRowForRegion(hri), desc.build());
     WALKeyImpl key = new WALKeyImpl(hri.getEncodedNameAsBytes(), TABLE_NAME, 1,
         EnvironmentEdgeManager.currentTime(), HConstants.DEFAULT_CLUSTER_ID);
     w.append(new Entry(key, edit));
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 602367c..e3cf7b4 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
@@ -19,7 +19,6 @@
 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;
@@ -183,7 +182,7 @@ public final class WALPerformanceEvaluation extends Configured implements Tool {
             RegionInfo hri = region.getRegionInfo();
             final WALKeyImpl logkey =
                 new WALKeyImpl(hri.getEncodedNameAsBytes(), hri.getTable(), now, mvcc, scopes);
-            wal.appendData(hri, logkey, walEdit);
+            wal.appendData(logkey, walEdit);
             if (!this.noSync) {
               if (++lastSync >= this.syncInterval) {
                 wal.sync();
@@ -408,9 +407,7 @@ public final class WALPerformanceEvaluation extends Configured implements Tool {
    * Verify the content of the WAL file.
    * Verify that the file has expected number of edits.
    * @param wals may not be null
-   * @param wal
    * @return Count of edits.
-   * @throws IOException
    */
   private long verify(final WALFactory wals, final Path wal, final boolean verbose)
       throws IOException {
@@ -574,9 +571,7 @@ public final class WALPerformanceEvaluation extends Configured implements Tool {
   /**
    * The guts of the {@link #main} method.
    * Call this method to avoid the {@link #main(String[])} System.exit.
-   * @param args
    * @return errCode
-   * @throws Exception
    */
   static int innerMain(final Configuration c, final String [] args) throws Exception {
     return ToolRunner.run(c, new WALPerformanceEvaluation(), args);
diff --git a/hbase-coprocessor/pom.xml b/hbase-wal/pom.xml
similarity index 85%
copy from hbase-coprocessor/pom.xml
copy to hbase-wal/pom.xml
index 0bcfcef..9e1e3bb 100644
--- a/hbase-coprocessor/pom.xml
+++ b/hbase-wal/pom.xml
@@ -29,9 +29,9 @@
     <relativePath>../hbase-build-configuration</relativePath>
   </parent>
 
-  <artifactId>hbase-coprocessor</artifactId>
-  <name>Apache HBase - Coprocessor</name>
-  <description>Base Coprocessor Types</description>
+  <artifactId>hbase-wal</artifactId>
+  <name>Apache HBase - WAL</name>
+  <description>HBase Write-ahead Log</description>
 
   <build>
     <testResources>
@@ -98,5 +98,17 @@
       <type>test-jar</type>
       <scope>test</scope>
     </dependency>
+      <dependency>
+          <groupId>org.apache.hbase</groupId>
+          <artifactId>hbase-protocol-shaded</artifactId>
+      </dependency>
+      <dependency>
+          <groupId>org.apache.hbase</groupId>
+          <artifactId>hbase-coprocessor</artifactId>
+      </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-metrics-api</artifactId>
+    </dependency>
   </dependencies>
 </project>
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/WALCoprocessor.java b/hbase-wal/src/main/java/org/apache/hadoop/hbase/coprocessor/WALCoprocessor.java
similarity index 99%
rename from hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/WALCoprocessor.java
rename to hbase-wal/src/main/java/org/apache/hadoop/hbase/coprocessor/WALCoprocessor.java
index 1deddf9..eaecee7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/WALCoprocessor.java
+++ b/hbase-wal/src/main/java/org/apache/hadoop/hbase/coprocessor/WALCoprocessor.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
@@ -18,13 +18,12 @@
 
 package org.apache.hadoop.hbase.coprocessor;
 
+import java.util.Optional;
 import org.apache.hadoop.hbase.Coprocessor;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.yetus.audience.InterfaceStability;
 
-import java.util.Optional;
-
 /**
  * WALCoprocessor don't support loading services using {@link #getServices()}.
  */
@@ -32,5 +31,4 @@ import java.util.Optional;
 @InterfaceStability.Evolving
 public interface WALCoprocessor extends Coprocessor {
   Optional<WALObserver> getWALObserver();
-
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/WALCoprocessorEnvironment.java b/hbase-wal/src/main/java/org/apache/hadoop/hbase/coprocessor/WALCoprocessorEnvironment.java
similarity index 100%
rename from hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/WALCoprocessorEnvironment.java
rename to hbase-wal/src/main/java/org/apache/hadoop/hbase/coprocessor/WALCoprocessorEnvironment.java
index 71c72a2..51a3e96 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/WALCoprocessorEnvironment.java
+++ b/hbase-wal/src/main/java/org/apache/hadoop/hbase/coprocessor/WALCoprocessorEnvironment.java
@@ -19,12 +19,12 @@
 
 package org.apache.hadoop.hbase.coprocessor;
 
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.yetus.audience.InterfaceStability;
 import org.apache.hadoop.hbase.CoprocessorEnvironment;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.metrics.MetricRegistry;
 import org.apache.hadoop.hbase.wal.WAL;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
 
 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
 @InterfaceStability.Evolving
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/WALObserver.java b/hbase-wal/src/main/java/org/apache/hadoop/hbase/coprocessor/WALObserver.java
similarity index 73%
rename from hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/WALObserver.java
rename to hbase-wal/src/main/java/org/apache/hadoop/hbase/coprocessor/WALObserver.java
index b2fa7ca..99bb20e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/WALObserver.java
+++ b/hbase-wal/src/main/java/org/apache/hadoop/hbase/coprocessor/WALObserver.java
@@ -20,12 +20,9 @@
 package org.apache.hadoop.hbase.coprocessor;
 
 import java.io.IOException;
-
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.wal.WALEdit;
-import org.apache.hadoop.hbase.wal.WALKey;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.yetus.audience.InterfaceStability;
 
@@ -36,7 +33,7 @@ import org.apache.yetus.audience.InterfaceStability;
  * Note that implementers of WALObserver will not see WALEdits that report themselves
  * as empty via {@link WALEdit#isEmpty()}.
  *
- * {@link org.apache.hadoop.hbase.coprocessor.RegionObserver} provides
+ * #org.apache.hadoop.hbase.coprocessor.RegionObserver provides
  * hooks for adding logic for WALEdits in the region context during reconstruction.
  *
  * Defines coprocessor hooks for interacting with operations on the
@@ -67,30 +64,6 @@ import org.apache.yetus.audience.InterfaceStability;
 @InterfaceStability.Evolving
 public interface WALObserver {
   /**
-   * Called before a {@link WALEdit}
-   * is writen to WAL.
-   * Do not amend the WALKey. It is InterfaceAudience.Private. Changing the WALKey will cause
-   * damage.
-   * @deprecated Since hbase-2.0.0. To be replaced with an alternative that does not expose
-   * InterfaceAudience classes such as WALKey and WALEdit. Will be removed in hbase-3.0.0.
-   */
-  @Deprecated
-  default void preWALWrite(ObserverContext<? extends WALCoprocessorEnvironment> ctx,
-      RegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {}
-
-  /**
-   * Called after a {@link WALEdit}
-   * is writen to WAL.
-   * Do not amend the WALKey. It is InterfaceAudience.Private. Changing the WALKey will cause
-   * damage.
-   * @deprecated Since hbase-2.0.0. To be replaced with an alternative that does not expose
-   * InterfaceAudience classes such as WALKey and WALEdit. Will be removed in hbase-3.0.0.
-   */
-  @Deprecated
-  default void postWALWrite(ObserverContext<? extends WALCoprocessorEnvironment> ctx,
-      RegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {}
-
-  /**
    * Called before rolling the current WAL
    * @param oldPath the path of the current wal that we are replacing
    * @param newPath the path of the wal we are going to create
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SequenceId.java b/hbase-wal/src/main/java/org/apache/hadoop/hbase/regionserver/SequenceId.java
similarity index 100%
rename from hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SequenceId.java
rename to hbase-wal/src/main/java/org/apache/hadoop/hbase/regionserver/SequenceId.java
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CompressionContext.java b/hbase-wal/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CompressionContext.java
similarity index 99%
rename from hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CompressionContext.java
rename to hbase-wal/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CompressionContext.java
index 16866e1..f0db427 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CompressionContext.java
+++ b/hbase-wal/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CompressionContext.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,18 +15,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.hadoop.hbase.regionserver.wal;
 
 import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
 import java.util.EnumMap;
 import java.util.Map;
-
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.io.TagCompressionContext;
 import org.apache.hadoop.hbase.io.util.Dictionary;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Context that holds the various dictionaries for compression in WAL.
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FailedLogCloseException.java b/hbase-wal/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FailedLogCloseException.java
similarity index 94%
rename from hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FailedLogCloseException.java
rename to hbase-wal/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FailedLogCloseException.java
index a595436..e88cfe2 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FailedLogCloseException.java
+++ b/hbase-wal/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FailedLogCloseException.java
@@ -19,12 +19,11 @@
 package org.apache.hadoop.hbase.regionserver.wal;
 
 import java.io.IOException;
-
 import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Thrown when we fail close of the write-ahead-log file.
- * Package private.  Only used inside this package.
+ * Package private.
  */
 @InterfaceAudience.Public
 public class FailedLogCloseException extends IOException {
@@ -33,18 +32,12 @@ public class FailedLogCloseException extends IOException {
   public FailedLogCloseException() {
     super();
   }
-
-  /**
-   * @param msg
-   */
   public FailedLogCloseException(String msg) {
     super(msg);
   }
-
   public FailedLogCloseException(final String msg, final Throwable t) {
     super(msg, t);
   }
-
   public FailedLogCloseException(final Throwable t) {
     super(t);
   }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALActionsListener.java b/hbase-wal/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALActionsListener.java
similarity index 91%
rename from hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALActionsListener.java
rename to hbase-wal/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALActionsListener.java
index 7fba7df..b98dc09 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALActionsListener.java
+++ b/hbase-wal/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALActionsListener.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.regionserver.wal;
 
 import java.io.IOException;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.wal.WALKey;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -33,7 +32,7 @@ import org.apache.yetus.audience.InterfaceAudience;
 public interface WALActionsListener {
 
   /** The reason for the log roll request. */
-  static enum RollRequestReason {
+  enum RollRequestReason {
     /** The length of the log exceeds the roll size threshold. */
     SIZE,
     /** Too few replicas in the writer pipeline. */
@@ -42,7 +41,7 @@ public interface WALActionsListener {
     SLOW_SYNC,
     /** I/O or other error. */
     ERROR
-  };
+  }
 
   /**
    * The WAL is going to be rolled. The oldPath can be null if this is
@@ -84,15 +83,14 @@ public interface WALActionsListener {
    */
   default void logCloseRequested() {}
 
-  /**
-  * Called before each write.
-  */
-  default void visitLogEntryBeforeWrite(RegionInfo info, WALKey logKey, WALEdit logEdit) {}
+  // /**
+  // * Called before each write.
+  // */
+  // default void visitLogEntryBeforeWrite(RegionInfo info, WALKey logKey, WALEdit logEdit) {}
 
   /**
-   * @param logKey
    * @param logEdit TODO: Retire this in favor of
-   *          {@link #visitLogEntryBeforeWrite(RegionInfo, WALKey, WALEdit)} It only exists to get
+   *          #visitLogEntryBeforeWrite(RegionInfo, WALKey, WALEdit) It only exists to get
    *          scope when replicating. Scope should be in the WALKey and not need us passing in a
    *          <code>htd</code>.
    * @throws IOException If failed to parse the WALEdit
@@ -114,8 +112,7 @@ public interface WALActionsListener {
   /**
    * For notification post writer sync.  Used by metrics system at least.
    * @param timeInNanos How long the filesystem sync took in nanoseconds.
-   * @param handlerSyncs How many sync handler calls were released by this call to filesystem
-   * sync.
+   * @param handlerSyncs How many sync handler calls were released by this call to filesystem sync.
    */
   default void postSync(final long timeInNanos, final int handlerSyncs) {}
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java b/hbase-wal/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java
similarity index 100%
rename from hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java
rename to hbase-wal/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCoprocessorHost.java b/hbase-wal/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCoprocessorHost.java
similarity index 83%
rename from hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCoprocessorHost.java
rename to hbase-wal/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCoprocessorHost.java
index 12e9448..83d8074 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCoprocessorHost.java
+++ b/hbase-wal/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCoprocessorHost.java
@@ -1,6 +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
@@ -22,10 +20,8 @@ package org.apache.hadoop.hbase.regionserver.wal;
 
 import java.io.IOException;
 import java.lang.reflect.InvocationTargetException;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.coprocessor.BaseEnvironment;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
 import org.apache.hadoop.hbase.coprocessor.MetricsCoprocessor;
@@ -45,18 +41,16 @@ import org.slf4j.LoggerFactory;
  * loaded within a {@link WAL}.
  */
 @InterfaceAudience.Private
-public class WALCoprocessorHost
-    extends CoprocessorHost<WALCoprocessor, WALCoprocessorEnvironment> {
+public class WALCoprocessorHost extends
+    CoprocessorHost<WALCoprocessor, WALCoprocessorEnvironment> {
   private static final Logger LOG = LoggerFactory.getLogger(WALCoprocessorHost.class);
 
   /**
    * Encapsulation of the environment of each coprocessor
    */
-  static class WALEnvironment extends BaseEnvironment<WALCoprocessor>
-    implements WALCoprocessorEnvironment {
-
+  static class WALEnvironment extends BaseEnvironment<WALCoprocessor> implements
+      WALCoprocessorEnvironment {
     private final WAL wal;
-
     private final MetricRegistry metricRegistry;
 
     @Override
@@ -113,7 +107,7 @@ public class WALCoprocessorHost
     super(null);
     this.wal = log;
     // load system default cp's from configuration.
-    loadSystemCoprocessors(conf, WAL_COPROCESSOR_CONF_KEY);
+    loadSystemCoprocessors(conf, CoprocessorHost.WAL_COPROCESSOR_CONF_KEY);
   }
 
   @Override
@@ -139,37 +133,24 @@ public class WALCoprocessorHost
   }
 
   private ObserverGetter<WALCoprocessor, WALObserver> walObserverGetter =
-      WALCoprocessor::getWALObserver;
+    WALCoprocessor::getWALObserver;
 
-  abstract class WALObserverOperation extends
-      ObserverOperationWithoutResult<WALObserver> {
+  abstract class WALObserverOperation extends ObserverOperationWithoutResult<WALObserver> {
     public WALObserverOperation() {
       super(walObserverGetter);
     }
   }
 
-  public void preWALWrite(final RegionInfo info, final WALKey logKey, final WALEdit logEdit)
-      throws IOException {
+  public void preWALWrite(final WALKey logKey, final WALEdit logEdit) throws IOException {
     // Not bypassable.
     if (this.coprocEnvironments.isEmpty()) {
       return;
     }
-    execOperation(new WALObserverOperation() {
-      @Override
-      public void call(WALObserver oserver) throws IOException {
-        oserver.preWALWrite(this, info, logKey, logEdit);
-      }
-    });
+    // TODO: We used to call WALObserver methods but they have been removed. Call CP instead?
   }
 
-  public void postWALWrite(final RegionInfo info, final WALKey logKey, final WALEdit logEdit)
-      throws IOException {
-    execOperation(coprocEnvironments.isEmpty() ? null : new WALObserverOperation() {
-      @Override
-      protected void call(WALObserver observer) throws IOException {
-        observer.postWALWrite(this, info, logKey, logEdit);
-      }
-    });
+  public void postWALWrite(final WALKey logKey, final WALEdit logEdit) throws IOException {
+    // TODO: We used to call WALObserver methods but they have been removed. Call CP instead?
   }
 
   /**
@@ -178,7 +159,7 @@ public class WALCoprocessorHost
    * @param newPath the path of the wal we are going to create
    */
   public void preWALRoll(Path oldPath, Path newPath) throws IOException {
-    execOperation(coprocEnvironments.isEmpty() ? null : new WALObserverOperation() {
+    execOperation(coprocEnvironments.isEmpty()? null : new WALObserverOperation() {
       @Override
       protected void call(WALObserver observer) throws IOException {
         observer.preWALRoll(this, oldPath, newPath);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALFileLengthProvider.java b/hbase-wal/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALFileLengthProvider.java
similarity index 99%
rename from hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALFileLengthProvider.java
rename to hbase-wal/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALFileLengthProvider.java
index c60faa9..da7c8ae 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALFileLengthProvider.java
+++ b/hbase-wal/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALFileLengthProvider.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.replication.regionserver;
 
 import java.util.OptionalLong;
-
 import org.apache.hadoop.fs.Path;
 import org.apache.yetus.audience.InterfaceAudience;
 
@@ -32,6 +31,5 @@ import org.apache.yetus.audience.InterfaceAudience;
 @InterfaceAudience.Private
 @FunctionalInterface
 public interface WALFileLengthProvider {
-
   OptionalLong getLogFileSizeIfBeingWritten(Path path);
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WAL.java b/hbase-wal/src/main/java/org/apache/hadoop/hbase/wal/WAL.java
similarity index 89%
rename from hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WAL.java
rename to hbase-wal/src/main/java/org/apache/hadoop/hbase/wal/WAL.java
index 20e45de..5bff009 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WAL.java
+++ b/hbase-wal/src/main/java/org/apache/hadoop/hbase/wal/WAL.java
@@ -17,22 +17,18 @@
  */
 package org.apache.hadoop.hbase.wal;
 
+import static org.apache.commons.lang3.StringUtils.isNumeric;
 import java.io.Closeable;
 import java.io.IOException;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.regionserver.wal.CompressionContext;
-import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException;
 import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
 import org.apache.hadoop.hbase.regionserver.wal.WALCoprocessorHost;
 import org.apache.hadoop.hbase.replication.regionserver.WALFileLengthProvider;
 import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.yetus.audience.InterfaceStability;
-
-import static org.apache.commons.lang3.StringUtils.isNumeric;
 
 /**
  * A Write Ahead Log (WAL) provides service for reading, writing waledits. This interface provides
@@ -42,7 +38,6 @@ import static org.apache.commons.lang3.StringUtils.isNumeric;
  * WAL.equals to determine if they have already seen a given WAL.
  */
 @InterfaceAudience.Private
-@InterfaceStability.Evolving
 public interface WAL extends Closeable, WALFileLengthProvider {
 
   /**
@@ -64,9 +59,9 @@ public interface WAL extends Closeable, WALFileLengthProvider {
    *
    * @return If lots of logs, flush the stores of returned regions so next time through we
    *         can clean logs. Returns null if nothing to flush. Names are actual
-   *         region names as returned by {@link RegionInfo#getEncodedName()}
+   *         region names as returned by RegionInfo#getEncodedName()
    */
-  Map<byte[], List<byte[]>> rollWriter() throws FailedLogCloseException, IOException;
+  Map<byte[], List<byte[]>> rollWriter() throws IOException;
 
   /**
    * Roll the log writer. That is, start writing log messages to a new file.
@@ -80,7 +75,7 @@ public interface WAL extends Closeable, WALFileLengthProvider {
    *          been written to the current writer
    * @return If lots of logs, flush the stores of returned regions so next time through we
    *         can clean logs. Returns null if nothing to flush. Names are actual
-   *         region names as returned by {@link RegionInfo#getEncodedName()}
+   *         region names as returned by RegionInfo#getEncodedName()
    */
   Map<byte[], List<byte[]>> rollWriter(boolean force) throws IOException;
 
@@ -105,35 +100,32 @@ public interface WAL extends Closeable, WALFileLengthProvider {
    * The WAL is not flushed/sync'd after this transaction completes BUT on return this edit must
    * have its region edit/sequence id assigned else it messes up our unification of mvcc and
    * sequenceid. On return <code>key</code> will have the region edit/sequence id filled in.
-   * @param info the regioninfo associated with append
    * @param key Modified by this call; we add to it this edits region edit/sequence id.
    * @param edits Edits to append. MAY CONTAIN NO EDITS for case where we want to get an edit
    *          sequence id that is after all currently appended edits.
    * @return Returns a 'transaction id' and <code>key</code> will have the region edit/sequence id
    *         in it.
-   * @see #appendMarker(RegionInfo, WALKeyImpl, WALEdit)
+   * @see #appendMarker(WALKeyImpl, WALEdit)
    */
-  long appendData(RegionInfo info, WALKeyImpl key, WALEdit edits) throws IOException;
+  long appendData(WALKeyImpl key, WALEdit edits) throws IOException;
 
   /**
    * Append an operational 'meta' event marker edit to the WAL. A marker meta edit could
    * be a FlushDescriptor, a compaction marker, or a region event marker; e.g. region open
    * or region close. The difference between a 'marker' append and a 'data' append as in
-   * {@link #appendData(RegionInfo, WALKeyImpl, WALEdit)}is that a marker will not have
+   * {@link #appendData(WALKeyImpl, WALEdit)}is that a marker will not have
    * transitioned through the memstore.
    * <p/>
    * The WAL is not flushed/sync'd after this transaction completes BUT on return this edit must
    * have its region edit/sequence id assigned else it messes up our unification of mvcc and
    * sequenceid. On return <code>key</code> will have the region edit/sequence id filled in.
-   * @param info the regioninfo associated with append
    * @param key Modified by this call; we add to it this edits region edit/sequence id.
    * @param edits Edits to append. MAY CONTAIN NO EDITS for case where we want to get an edit
    *          sequence id that is after all currently appended edits.
    * @return Returns a 'transaction id' and <code>key</code> will have the region edit/sequence id
    *         in it.
-   * @see #appendData(RegionInfo, WALKeyImpl, WALEdit)
    */
-  long appendMarker(RegionInfo info, WALKeyImpl key, WALEdit edits) throws IOException;
+  long appendMarker(WALKeyImpl key, WALEdit edits) throws IOException;
 
   /**
    * updates the seuence number of a specific store.
@@ -181,10 +173,10 @@ public interface WAL extends Closeable, WALFileLengthProvider {
    * concurrent appends while we set up cache flush.
    * @param families Families to flush. May be a subset of all families in the region.
    * @return Returns {@link HConstants#NO_SEQNUM} if we are flushing the whole region OR if
-   * we are flushing a subset of all families but there are no edits in those families not
-   * being flushed; in other words, this is effectively same as a flush of all of the region
-   * though we were passed a subset of regions. Otherwise, it returns the sequence id of the
-   * oldest/lowest outstanding edit.
+   *    we are flushing a subset of all families but there are no edits in those families not
+   *    being flushed; in other words, this is effectively same as a flush of all of the region
+   *    though we were passed a subset of regions. Otherwise, it returns the sequence id of the
+   *    oldest/lowest outstanding edit.
    * @see #completeCacheFlush(byte[], long)
    * @see #abortCacheFlush(byte[])
    */
@@ -220,7 +212,7 @@ public interface WAL extends Closeable, WALFileLengthProvider {
    * @param encodedRegionName The region to get the number for.
    * @return The earliest/lowest/oldest sequence id if present, HConstants.NO_SEQNUM if absent.
    * @deprecated Since version 1.2.0. Removing because not used and exposes subtle internal
-   * workings. Use {@link #getEarliestMemStoreSeqNum(byte[], byte[])}
+   *    workings. Use {@link #getEarliestMemStoreSeqNum(byte[], byte[])}
    */
   @Deprecated
   long getEarliestMemStoreSeqNum(byte[] encodedRegionName);
@@ -324,7 +316,7 @@ public interface WAL extends Closeable, WALFileLengthProvider {
    * @param name Name of the WAL file.
    * @return Timestamp or -1.
    */
-  public static long getTimestamp(String name) {
+  static long getTimestamp(String name) {
     String [] splits = name.split("\\.");
     if (splits.length <= 1) {
       return -1;
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALEdit.java b/hbase-wal/src/main/java/org/apache/hadoop/hbase/wal/WALEdit.java
similarity index 92%
rename from hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALEdit.java
rename to hbase-wal/src/main/java/org/apache/hadoop/hbase/wal/WALEdit.java
index cefc3a9..56ddf03 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALEdit.java
+++ b/hbase-wal/src/main/java/org/apache/hadoop/hbase/wal/WALEdit.java
@@ -23,20 +23,17 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.TreeSet;
-
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.PrivateCellUtil;
 import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.codec.Codec;
 import org.apache.hadoop.hbase.io.HeapSize;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ClassSize;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.yetus.audience.InterfaceAudience;
-
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor;
@@ -61,9 +58,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDe
  * normal processing so would make sense to do this. Current system is an awkward marking of Cell
  * columnfamily as {@link #METAFAMILY} and then setting qualifier based off meta edit type. For
  * replay-time where we read Cell-at-a-time, there are utility methods below for figuring
- * meta type. See also
- * {@link #createBulkLoadEvent(RegionInfo, WALProtos.BulkLoadDescriptor)}, etc., for where we
- * create meta WALEdit instances.</p>
+ * meta type. See also #createBulkLoadEvent(RegionInfo, WALProtos.BulkLoadDescriptor), etc., for
+ * where we create meta WALEdit instances.</p>
  *
  * <p>WALEdit will accumulate a Set of all column family names referenced by the Cells
  * {@link #add(Cell)}'d. This is an optimization. Usually when loading a WALEdit, we have the
@@ -308,8 +304,10 @@ public class WALEdit implements HeapSize {
     return sb.toString();
   }
 
-  public static WALEdit createFlushWALEdit(RegionInfo hri, FlushDescriptor f) {
-    KeyValue kv = new KeyValue(getRowForRegion(hri), METAFAMILY, FLUSH,
+  // Changed first param from RegionInfo to byte [] hbase3. Should be fine in a LimitedPrivate class
+  // going to a major version where this method is only called from a private util method.
+  public static WALEdit createFlushWALEdit(byte [] regionRow, FlushDescriptor f) {
+    KeyValue kv = new KeyValue(regionRow, METAFAMILY, FLUSH,
       EnvironmentEdgeManager.currentTime(), f.toByteArray());
     return new WALEdit().add(kv, METAFAMILY);
   }
@@ -326,11 +324,6 @@ public class WALEdit implements HeapSize {
    *   {@link #REGION_EVENT_PREFIX} + {@link RegionEventDescriptor#getEventType()};
    *   for example HBASE::REGION_EVENT::REGION_CLOSE.
    */
-  public static WALEdit createRegionEventWALEdit(RegionInfo hri,
-      RegionEventDescriptor regionEventDesc) {
-    return createRegionEventWALEdit(getRowForRegion(hri), regionEventDesc);
-  }
-
   @InterfaceAudience.Private
   public static WALEdit createRegionEventWALEdit(byte [] rowForRegion,
       RegionEventDescriptor regionEventDesc) {
@@ -371,23 +364,13 @@ public class WALEdit implements HeapSize {
   /**
    * @return A Marker WALEdit that has <code>c</code> serialized as its value
    */
-  public static WALEdit createCompaction(final RegionInfo hri, final CompactionDescriptor c) {
+  public static WALEdit createCompaction(byte [] regionRow, final CompactionDescriptor c) {
     byte [] pbbytes = c.toByteArray();
-    KeyValue kv = new KeyValue(getRowForRegion(hri), METAFAMILY, COMPACTION,
+    KeyValue kv = new KeyValue(regionRow, METAFAMILY, COMPACTION,
       EnvironmentEdgeManager.currentTime(), pbbytes);
     return new WALEdit().add(kv, METAFAMILY); //replication scope null so this won't be replicated
   }
 
-  public static byte[] getRowForRegion(RegionInfo hri) {
-    byte[] startKey = hri.getStartKey();
-    if (startKey.length == 0) {
-      // empty row key is not allowed in mutations because it is both the start key and the end key
-      // we return the smallest byte[] that is bigger (in lex comparison) than byte[0].
-      return new byte[] {0};
-    }
-    return startKey;
-  }
-
   /**
    * Deserialized and returns a CompactionDescriptor is the KeyValue contains one.
    * @param kv the key value
@@ -409,13 +392,14 @@ public class WALEdit implements HeapSize {
   /**
    * Create a bulk loader WALEdit
    *
-   * @param hri                The RegionInfo for the region in which we are bulk loading
    * @param bulkLoadDescriptor The descriptor for the Bulk Loader
    * @return The WALEdit for the BulkLoad
    */
-  public static WALEdit createBulkLoadEvent(RegionInfo hri,
+  // Changed first param from RegionInfo to byte [] hbase3. Should be fine in a LimitedPrivate class
+  // going to a major version where this method is only called from a private util method.
+  public static WALEdit createBulkLoadEvent(byte [] regionRow,
       WALProtos.BulkLoadDescriptor bulkLoadDescriptor) {
-    KeyValue kv = new KeyValue(getRowForRegion(hri), METAFAMILY, BULK_LOAD,
+    KeyValue kv = new KeyValue(regionRow, METAFAMILY, BULK_LOAD,
         EnvironmentEdgeManager.currentTime(), bulkLoadDescriptor.toByteArray());
     return new WALEdit().add(kv, METAFAMILY);
   }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKey.java b/hbase-wal/src/main/java/org/apache/hadoop/hbase/wal/WALKey.java
similarity index 99%
rename from hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKey.java
rename to hbase-wal/src/main/java/org/apache/hadoop/hbase/wal/WALKey.java
index fdbacbd..bbfeb8c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKey.java
+++ b/hbase-wal/src/main/java/org/apache/hadoop/hbase/wal/WALKey.java
@@ -17,19 +17,18 @@
  */
 package org.apache.hadoop.hbase.wal;
 
-import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.regionserver.SequenceId;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.yetus.audience.InterfaceAudience;
-
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.UUID;
+import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.regionserver.SequenceId;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
 
 
 /**
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKeyImpl.java b/hbase-wal/src/main/java/org/apache/hadoop/hbase/wal/WALKeyImpl.java
similarity index 99%
rename from hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKeyImpl.java
rename to hbase-wal/src/main/java/org/apache/hadoop/hbase/wal/WALKeyImpl.java
index b2a9956..d0dcfd1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKeyImpl.java
+++ b/hbase-wal/src/main/java/org/apache/hadoop/hbase/wal/WALKeyImpl.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
@@ -34,14 +34,12 @@ import org.apache.hadoop.hbase.regionserver.wal.CompressionContext;
 import org.apache.hadoop.hbase.regionserver.wal.WALCellCodec;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.yetus.audience.InterfaceAudience;
-
-import org.apache.hbase.thirdparty.com.google.protobuf.ByteString;
-
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FamilyScope;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.ScopeType;
+import org.apache.hbase.thirdparty.com.google.protobuf.ByteString;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Default implementation of Key for an Entry in the WAL.
@@ -278,8 +276,6 @@ public class WALKeyImpl implements WALKey {
    * @param tablename         the tablename
    * @param now               Time at which this edit was written.
    * @param clusterIds        the clusters that have consumed the change(used in Replication)
-   * @param nonceGroup
-   * @param nonce
    * @param mvcc mvcc control used to generate sequence numbers and control read/write points
    */
   public WALKeyImpl(final byte[] encodedRegionName, final TableName tablename,
@@ -296,7 +292,6 @@ public class WALKeyImpl implements WALKey {
    *
    * @param encodedRegionName Encoded name of the region as returned by
    *                          <code>HRegionInfo#getEncodedNameAsBytes()</code>.
-   * @param tablename
    * @param now               Time at which this edit was written.
    * @param clusterIds        the clusters that have consumed the change(used in Replication)
    * @param nonceGroup        the nonceGroup
@@ -319,10 +314,6 @@ public class WALKeyImpl implements WALKey {
    *
    * @param encodedRegionName Encoded name of the region as returned by
    *                          <code>HRegionInfo#getEncodedNameAsBytes()</code>.
-   * @param tablename
-   * @param logSeqNum
-   * @param nonceGroup
-   * @param nonce
    */
   // TODO: Fix being able to pass in sequenceid.
   public WALKeyImpl(final byte[] encodedRegionName,
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestCustomWALCellCodec.java b/hbase-wal/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestCustomWALCellCodec.java
similarity index 100%
rename from hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestCustomWALCellCodec.java
rename to hbase-wal/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestCustomWALCellCodec.java
diff --git a/pom.xml b/pom.xml
index 465cdb2..5fbc411 100755
--- a/pom.xml
+++ b/pom.xml
@@ -93,6 +93,7 @@
     <module>hbase-asyncfs</module>
     <module>hbase-logging</module>
     <module>hbase-coprocessor</module>
+    <module>hbase-wal</module>
   </modules>
   <scm>
     <connection>scm:git:git://gitbox.apache.org/repos/asf/hbase.git</connection>
@@ -1642,6 +1643,18 @@
       </dependency>
       <dependency>
         <groupId>org.apache.hbase</groupId>
+        <artifactId>hbase-wal</artifactId>
+        <version>${project.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.hbase</groupId>
+        <artifactId>hbase-wal</artifactId>
+        <version>${project.version}</version>
+        <type>test-jar</type>
+        <scope>test</scope>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.hbase</groupId>
         <artifactId>hbase-protocol-shaded</artifactId>
         <version>${project.version}</version>
       </dependency>