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 2017/12/08 23:12:41 UTC

[3/3] hbase git commit: HBASE-19134 Make WALKey an Interface; expose Read-Only version to CPs

HBASE-19134 Make WALKey an Interface; expose Read-Only version to CPs

Created a new WALKey Interface and a WALKeyImpl. The WALKey Interface
is surfaced to Coprocessors and throughout most of the code base.
WALKeyImpl is used internally by WAL and by Replication which need
access to WALKey setters.

Methods that were deprecated in WALObserver because they were exposing
Private audience Classes have been undeprecated now we have WALKey.

Moved over to use SequenceId#getSequenceId throughout. Changed
SequenceId#getSequenceId removing the IOE.


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

Branch: refs/heads/master
Commit: a5a77ae3d5bfef9396b6024ec723d83a911692ec
Parents: 542060c
Author: Michael Stack <st...@apache.org>
Authored: Wed Dec 6 22:44:52 2017 -0800
Committer: Michael Stack <st...@apache.org>
Committed: Fri Dec 8 15:12:31 2017 -0800

----------------------------------------------------------------------
 .../hbase/mapreduce/TestWALRecordReader.java    |  18 +-
 .../replication/TestReplicationSmallTests.java  |   8 +-
 .../hbase/coprocessor/RegionObserver.java       |  12 +-
 .../hbase/protobuf/ReplicationProtbufUtil.java  |  10 +-
 .../hadoop/hbase/regionserver/HRegion.java      |  28 +-
 .../hadoop/hbase/regionserver/SequenceId.java   |  10 +-
 .../hbase/regionserver/wal/AbstractFSWAL.java   |   5 +-
 .../hbase/regionserver/wal/AsyncFSWAL.java      |   9 +-
 .../wal/AsyncProtobufLogWriter.java             |   4 +-
 .../hadoop/hbase/regionserver/wal/FSHLog.java   |  14 +-
 .../hbase/regionserver/wal/FSWALEntry.java      |   6 +-
 .../regionserver/wal/ProtobufLogReader.java     |   3 +-
 .../regionserver/wal/ProtobufLogWriter.java     |   5 +-
 .../hbase/regionserver/wal/ReaderBase.java      |   4 +-
 .../regionserver/wal/WALActionsListener.java    |   7 +-
 .../regionserver/wal/WALCoprocessorHost.java    |  11 +-
 .../hadoop/hbase/regionserver/wal/WALUtil.java  |  29 +-
 .../replication/ClusterMarkingEntryFilter.java  |   4 +-
 .../replication/regionserver/Replication.java   |   5 +-
 .../VisibilityReplicationEndpoint.java          |   3 +-
 .../hadoop/hbase/wal/DisabledWALProvider.java   |   2 +-
 .../java/org/apache/hadoop/hbase/wal/WAL.java   |  12 +-
 .../org/apache/hadoop/hbase/wal/WALEdit.java    |   2 +
 .../org/apache/hadoop/hbase/wal/WALKey.java     | 613 +-----------------
 .../org/apache/hadoop/hbase/wal/WALKeyImpl.java | 633 +++++++++++++++++++
 .../apache/hadoop/hbase/wal/WALSplitter.java    |   8 +-
 .../coprocessor/TestCoprocessorMetrics.java     |   5 +-
 .../hbase/coprocessor/TestWALObserver.java      |  14 +-
 .../master/TestDistributedLogSplitting.java     |   4 +-
 .../hadoop/hbase/regionserver/TestBulkLoad.java |  10 +-
 .../hadoop/hbase/regionserver/TestHRegion.java  |  21 +-
 .../regionserver/TestHRegionReplayEvents.java   |  17 +-
 .../hbase/regionserver/TestWALLockup.java       |  15 +-
 .../regionserver/wal/AbstractTestFSWAL.java     |   8 +-
 .../wal/AbstractTestProtobufLog.java            |   4 +-
 .../regionserver/wal/AbstractTestWALReplay.java |  10 +-
 .../wal/FaultyProtobufLogReader.java            |   4 +-
 .../regionserver/wal/TestLogRollAbort.java      |   4 +-
 .../wal/TestLogRollingNoCluster.java            |   4 +-
 .../wal/TestWALActionsListener.java             |   4 +-
 .../replication/TestReplicationSource.java      |   5 +-
 .../TestReplicationWALEntryFilters.java         |  15 +-
 .../TestRegionReplicaReplicationEndpoint.java   |   4 +-
 ...egionReplicaReplicationEndpointNoMaster.java |  11 +-
 .../TestReplicationSourceManager.java           |  15 +-
 .../regionserver/TestWALEntryStream.java        |  11 +-
 .../apache/hadoop/hbase/wal/FaultyFSLog.java    |   2 +-
 .../hadoop/hbase/wal/TestFSHLogProvider.java    |   4 +-
 .../apache/hadoop/hbase/wal/TestSecureWAL.java  |   2 +-
 .../apache/hadoop/hbase/wal/TestWALFactory.java |  23 +-
 .../apache/hadoop/hbase/wal/TestWALMethods.java |   2 +-
 .../hbase/wal/TestWALReaderOnSecureWAL.java     |   2 +-
 .../apache/hadoop/hbase/wal/TestWALRootDir.java |  22 +-
 .../apache/hadoop/hbase/wal/TestWALSplit.java   |  10 +-
 .../hbase/wal/WALPerformanceEvaluation.java     |   8 +-
 55 files changed, 904 insertions(+), 821 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/a5a77ae3/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALRecordReader.java
----------------------------------------------------------------------
diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALRecordReader.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALRecordReader.java
index 4a8b8ad..65a3421 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
@@ -46,7 +46,7 @@ import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.wal.WAL;
 import org.apache.hadoop.hbase.wal.WALFactory;
-import org.apache.hadoop.hbase.wal.WALKey;
+import org.apache.hadoop.hbase.wal.WALKeyImpl;
 import org.apache.hadoop.hbase.testclassification.MapReduceTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -133,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.append(info, getWalKey(ts, scopes), edit, true);
+    log.append(info, getWalKeyImpl(ts, scopes), edit, true);
     edit = new WALEdit();
     edit.add(new KeyValue(rowName, family, Bytes.toBytes("2"), ts+1, value));
-    log.append(info, getWalKey(ts+1, scopes), edit, true);
+    log.append(info, getWalKeyImpl(ts+1, scopes), edit, true);
     log.sync();
     LOG.info("Before 1st WAL roll " + log.toString());
     log.rollWriter();
@@ -147,10 +147,10 @@ public class TestWALRecordReader {
 
     edit = new WALEdit();
     edit.add(new KeyValue(rowName, family, Bytes.toBytes("3"), ts1+1, value));
-    log.append(info, getWalKey(ts1+1, scopes), edit, true);
+    log.append(info, getWalKeyImpl(ts1+1, scopes), edit, true);
     edit = new WALEdit();
     edit.add(new KeyValue(rowName, family, Bytes.toBytes("4"), ts1+2, value));
-    log.append(info, getWalKey(ts1+2, scopes), edit, true);
+    log.append(info, getWalKeyImpl(ts1+2, scopes), edit, true);
     log.sync();
     log.shutdown();
     walfactory.shutdown();
@@ -192,7 +192,7 @@ public class TestWALRecordReader {
     WALEdit edit = new WALEdit();
     edit.add(new KeyValue(rowName, family, Bytes.toBytes("1"),
         System.currentTimeMillis(), value));
-    long txid = log.append(info, getWalKey(System.currentTimeMillis(), scopes), edit, true);
+    long txid = log.append(info, getWalKeyImpl(System.currentTimeMillis(), scopes), edit, true);
     log.sync(txid);
 
     Thread.sleep(1); // make sure 2nd log gets a later timestamp
@@ -202,7 +202,7 @@ public class TestWALRecordReader {
     edit = new WALEdit();
     edit.add(new KeyValue(rowName, family, Bytes.toBytes("2"),
         System.currentTimeMillis(), value));
-    txid = log.append(info, getWalKey(System.currentTimeMillis(), scopes), edit, true);
+    txid = log.append(info, getWalKeyImpl(System.currentTimeMillis(), scopes), edit, true);
     log.sync(txid);
     log.shutdown();
     walfactory.shutdown();
@@ -241,8 +241,8 @@ public class TestWALRecordReader {
     testSplit(splits.get(1));
   }
 
-  protected WALKey getWalKey(final long time, NavigableMap<byte[], Integer> scopes) {
-    return new WALKey(info.getEncodedNameAsBytes(), tableName, time, mvcc, scopes);
+  protected WALKeyImpl getWalKeyImpl(final long time, NavigableMap<byte[], Integer> scopes) {
+    return new WALKeyImpl(info.getEncodedNameAsBytes(), tableName, time, mvcc, scopes);
   }
 
   protected WALRecordReader getReader() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5a77ae3/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java
----------------------------------------------------------------------
diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java
index 28bf249..143f585 100644
--- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java
+++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.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
@@ -76,7 +76,7 @@ import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.JVMClusterUtil;
 import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
 import org.apache.hadoop.hbase.wal.WAL;
-import org.apache.hadoop.hbase.wal.WALKey;
+import org.apache.hadoop.hbase.wal.WALKeyImpl;
 import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.mapreduce.Job;
 import org.junit.Before;
@@ -764,7 +764,7 @@ public class TestReplicationSmallTests extends TestReplicationBase {
         .setEndKey(HConstants.EMPTY_END_ROW)
         .build();
     WALEdit edit = WALEdit.createCompaction(hri, compactionDescriptor);
-    Replication.scopeWALEdits(new WALKey(), edit,
+    Replication.scopeWALEdits(new WALKeyImpl(), edit,
       htable1.getConfiguration(), null);
   }
 
@@ -844,7 +844,7 @@ public class TestReplicationSmallTests extends TestReplicationBase {
     long now = EnvironmentEdgeManager.currentTime();
     edit.add(new KeyValue(rowName, famName, qualifier,
       now, value));
-    WALKey walKey = new WALKey(hri.getEncodedNameAsBytes(), tableName, now, mvcc, scopes);
+    WALKeyImpl walKey = new WALKeyImpl(hri.getEncodedNameAsBytes(), tableName, now, mvcc, scopes);
     wal.append(hri, walKey, edit, true);
     wal.sync();
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5a77ae3/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java
index 6b5527b..7391f6f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.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
@@ -912,26 +912,16 @@ public interface RegionObserver {
   /**
    * Called before a {@link WALEdit}
    * replayed for this region.
-   * Do not amend the WALKey. It is InterfaceAudience.Private. Changing the WALKey will cause
-   * damage.
    * @param ctx the environment provided by the region server
-   * @deprecated Since hbase-2.0.0. No replacement. To be removed in hbase-3.0.0 and replaced
-   * with something that doesn't expose IntefaceAudience.Private classes.
    */
-  @Deprecated
   default void preWALRestore(ObserverContext<? extends RegionCoprocessorEnvironment> ctx,
     RegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {}
 
   /**
    * Called after a {@link WALEdit}
    * replayed for this region.
-   * Do not amend the WALKey. It is InterfaceAudience.Private. Changing the WALKey will cause
-   * damage.
    * @param ctx the environment provided by the region server
-   * @deprecated Since hbase-2.0.0. No replacement. To be removed in hbase-3.0.0 and replaced
-   * with something that doesn't expose IntefaceAudience.Private classes.
    */
-  @Deprecated
   default void postWALRestore(ObserverContext<? extends RegionCoprocessorEnvironment> ctx,
     RegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {}
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5a77ae3/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java
index af9690a..56cbf47 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java
@@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellScanner;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.PrivateCellUtil;
+import org.apache.hadoop.hbase.wal.WALKeyImpl;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.io.SizedCellScanner;
 import org.apache.hadoop.hbase.ipc.HBaseRpcController;
@@ -108,15 +109,16 @@ public class ReplicationProtbufUtil {
     HBaseProtos.UUID.Builder uuidBuilder = HBaseProtos.UUID.newBuilder();
     for (Entry entry: entries) {
       entryBuilder.clear();
-      // TODO: this duplicates a lot in WALKey#getBuilder
+      // TODO: this duplicates a lot in WALKeyImpl#getBuilder
       WALProtos.WALKey.Builder keyBuilder = entryBuilder.getKeyBuilder();
-      WALKey key = entry.getKey();
+      WALKeyImpl key = entry.getKey();
       keyBuilder.setEncodedRegionName(
           UnsafeByteOperations.unsafeWrap(encodedRegionName == null
             ? key.getEncodedRegionName()
             : encodedRegionName));
       keyBuilder.setTableName(UnsafeByteOperations.unsafeWrap(key.getTablename().getName()));
-      keyBuilder.setLogSequenceNumber(key.getLogSeqNum());
+      long sequenceId = key.getSequenceId();
+      keyBuilder.setLogSequenceNumber(sequenceId);
       keyBuilder.setWriteTime(key.getWriteTime());
       if (key.getNonce() != HConstants.NO_NONCE) {
         keyBuilder.setNonce(key.getNonce());
@@ -129,7 +131,7 @@ public class ReplicationProtbufUtil {
         uuidBuilder.setMostSigBits(clusterId.getMostSignificantBits());
         keyBuilder.addClusterIds(uuidBuilder.build());
       }
-      if(key.getOrigLogSeqNum() > 0) {
+      if (key.getOrigLogSeqNum() > 0) {
         keyBuilder.setOrigSequenceNumber(key.getOrigLogSeqNum());
       }
       WALEdit edit = entry.getEdit();

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5a77ae3/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
----------------------------------------------------------------------
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 a8ea724..85c12e9 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
@@ -191,6 +191,7 @@ import org.apache.hadoop.hbase.wal.WAL;
 import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.wal.WALFactory;
 import org.apache.hadoop.hbase.wal.WALKey;
+import org.apache.hadoop.hbase.wal.WALKeyImpl;
 import org.apache.hadoop.hbase.wal.WALSplitter;
 import org.apache.hadoop.hbase.wal.WALSplitter.MutationReplay;
 import org.apache.hadoop.io.MultipleIOException;
@@ -3343,7 +3344,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
 
     @Override
     public long getOrigLogSeqNum() {
-      return WALKey.NO_SEQUENCE_ID;
+      return SequenceId.NO_SEQUENCE_ID;
     }
 
     @Override
@@ -4504,16 +4505,16 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
           }
 
           if (firstSeqIdInLog == -1) {
-            firstSeqIdInLog = key.getLogSeqNum();
+            firstSeqIdInLog = key.getSequenceId();
           }
-          if (currentEditSeqId > key.getLogSeqNum()) {
+          if (currentEditSeqId > key.getSequenceId()) {
             // when this condition is true, it means we have a serious defect because we need to
             // maintain increasing SeqId for WAL edits per region
             LOG.error(getRegionInfo().getEncodedName() + " : "
                  + "Found decreasing SeqId. PreId=" + currentEditSeqId + " key=" + key
                 + "; edit=" + val);
           } else {
-            currentEditSeqId = key.getLogSeqNum();
+            currentEditSeqId = key.getSequenceId();
           }
           currentReplaySeqId = (key.getOrigLogSeqNum() > 0) ?
             key.getOrigLogSeqNum() : currentEditSeqId;
@@ -4571,7 +4572,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
               continue;
             }
             // Now, figure if we should skip this edit.
-            if (key.getLogSeqNum() <= maxSeqIdInStores.get(store.getColumnFamilyDescriptor()
+            if (key.getSequenceId() <= maxSeqIdInStores.get(store.getColumnFamilyDescriptor()
                 .getName())) {
               skippedEdits++;
               continue;
@@ -7550,7 +7551,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   private WriteEntry doWALAppend(WALEdit walEdit, Durability durability, List<UUID> clusterIds,
       long now, long nonceGroup, long nonce) throws IOException {
     return doWALAppend(walEdit, durability, clusterIds, now, nonceGroup, nonce,
-        WALKey.NO_SEQUENCE_ID);
+        SequenceId.NO_SEQUENCE_ID);
   }
 
   /**
@@ -7560,16 +7561,17 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
       long now, long nonceGroup, long nonce, long origLogSeqNum) throws IOException {
     Preconditions.checkArgument(walEdit != null && !walEdit.isEmpty(),
         "WALEdit is null or empty!");
-    Preconditions.checkArgument(!walEdit.isReplay() || origLogSeqNum != WALKey.NO_SEQUENCE_ID,
+    Preconditions.checkArgument(!walEdit.isReplay() || origLogSeqNum != SequenceId.NO_SEQUENCE_ID,
         "Invalid replay sequence Id for replay WALEdit!");
     // Using default cluster id, as this can only happen in the originating cluster.
     // A slave cluster receives the final value (not the delta) as a Put. We use HLogKey
-    // here instead of WALKey directly to support legacy coprocessors.
-    WALKey walKey = walEdit.isReplay() ? new WALKey(this.getRegionInfo().getEncodedNameAsBytes(),
-        this.htableDescriptor.getTableName(), WALKey.NO_SEQUENCE_ID, now, clusterIds, nonceGroup,
-        nonce, mvcc) :
-        new WALKey(this.getRegionInfo().getEncodedNameAsBytes(),
-            this.htableDescriptor.getTableName(), WALKey.NO_SEQUENCE_ID, now, clusterIds,
+    // here instead of WALKeyImpl directly to support legacy coprocessors.
+    WALKeyImpl walKey = walEdit.isReplay()?
+        new WALKeyImpl(this.getRegionInfo().getEncodedNameAsBytes(),
+          this.htableDescriptor.getTableName(), SequenceId.NO_SEQUENCE_ID, now, clusterIds,
+            nonceGroup, nonce, mvcc) :
+        new WALKeyImpl(this.getRegionInfo().getEncodedNameAsBytes(),
+            this.htableDescriptor.getTableName(), SequenceId.NO_SEQUENCE_ID, now, clusterIds,
             nonceGroup, nonce, mvcc, this.getReplicationScope());
     if (walEdit.isReplay()) {
       walKey.setOrigLogSeqNum(origLogSeqNum);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5a77ae3/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SequenceId.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SequenceId.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SequenceId.java
index 7bfda6a..235644e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SequenceId.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SequenceId.java
@@ -19,7 +19,6 @@
 
 package org.apache.hadoop.hbase.regionserver;
 
-import java.io.IOException;
 import org.apache.yetus.audience.InterfaceAudience;
 
 /**
@@ -27,5 +26,12 @@ import org.apache.yetus.audience.InterfaceAudience;
  */
 @InterfaceAudience.Private
 public interface SequenceId {
-  public long getSequenceId() throws IOException;
+  /**
+   * Used to represent when a particular wal key doesn't know/care about the sequence ordering.
+   */
+  long NO_SEQUENCE_ID = -1;
+
+  default long getSequenceId() {
+    return NO_SEQUENCE_ID;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5a77ae3/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java
----------------------------------------------------------------------
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 534315e..246221b 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
@@ -72,6 +72,7 @@ import org.apache.hadoop.hbase.wal.WAL;
 import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.wal.WALFactory;
 import org.apache.hadoop.hbase.wal.WALKey;
+import org.apache.hadoop.hbase.wal.WALKeyImpl;
 import org.apache.hadoop.hbase.wal.WALProvider.WriterBase;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.util.StringUtils;
@@ -954,7 +955,7 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
     }
   }
 
-  protected final long stampSequenceIdAndPublishToRingBuffer(RegionInfo hri, WALKey key,
+  protected final long stampSequenceIdAndPublishToRingBuffer(RegionInfo hri, WALKeyImpl key,
       WALEdit edits, boolean inMemstore, RingBuffer<RingBufferTruck> ringBuffer)
       throws IOException {
     if (this.closed) {
@@ -1017,7 +1018,7 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
    * this append; otherwise, you will just have to wait on the WriteEntry to get filled in.
    */
   @Override
-  public abstract long append(RegionInfo info, WALKey key, WALEdit edits, boolean inMemstore)
+  public abstract long append(RegionInfo info, WALKeyImpl key, WALEdit edits, boolean inMemstore)
       throws IOException;
 
   protected abstract void doAppend(W writer, FSWALEntry entry) throws IOException;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5a77ae3/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java
----------------------------------------------------------------------
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 18007aa..832eefd 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
@@ -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
@@ -59,6 +59,7 @@ import org.apache.hadoop.hbase.trace.TraceUtil;
 import org.apache.hadoop.hbase.wal.AsyncFSWALProvider;
 import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.wal.WALKey;
+import org.apache.hadoop.hbase.wal.WALKeyImpl;
 import org.apache.hadoop.hbase.wal.WALProvider.AsyncWriter;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.ipc.RemoteException;
@@ -561,10 +562,10 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
   }
 
   @Override
-  public long append(RegionInfo hri, WALKey key, WALEdit edits, boolean inMemstore)
+  public long append(RegionInfo hri, WALKeyImpl key, WALEdit edits, boolean inMemstore)
       throws IOException {
-    long txid =
-      stampSequenceIdAndPublishToRingBuffer(hri, key, edits, inMemstore, waitingConsumePayloads);
+    long txid = stampSequenceIdAndPublishToRingBuffer(hri, key, edits, inMemstore,
+        waitingConsumePayloads);
     if (shouldScheduleConsumer()) {
       consumeExecutor.execute(consumer);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5a77ae3/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncProtobufLogWriter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncProtobufLogWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncProtobufLogWriter.java
index 482500e..454928b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncProtobufLogWriter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncProtobufLogWriter.java
@@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.io.asyncfs.AsyncFSOutputHelper;
 import org.apache.hadoop.hbase.util.CommonFSUtils.StreamLacksCapabilityException;
 import org.apache.hadoop.hbase.wal.AsyncFSWALProvider;
 import org.apache.hadoop.hbase.wal.WAL.Entry;
+import org.apache.hadoop.hbase.wal.WALKeyImpl;
 import org.apache.yetus.audience.InterfaceAudience;
 
 import org.apache.hadoop.hbase.shaded.com.google.common.base.Throwables;
@@ -113,7 +114,8 @@ public class AsyncProtobufLogWriter extends AbstractProtobufLogWriter
     int buffered = output.buffered();
     entry.setCompressionContext(compressionContext);
     try {
-      entry.getKey().getBuilder(compressor).setFollowingKvCount(entry.getEdit().size()).build()
+      entry.getKey().
+        getBuilder(compressor).setFollowingKvCount(entry.getEdit().size()).build()
           .writeDelimitedTo(asyncOutputWrapper);
     } catch (IOException e) {
       throw new AssertionError("should not happen", e);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5a77ae3/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
----------------------------------------------------------------------
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 15a6a41..3da37d3 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
@@ -57,6 +57,7 @@ import org.apache.hadoop.hbase.wal.FSHLogProvider;
 import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.wal.WALFactory;
 import org.apache.hadoop.hbase.wal.WALKey;
+import org.apache.hadoop.hbase.wal.WALKeyImpl;
 import org.apache.hadoop.hbase.wal.WALPrettyPrinter;
 import org.apache.hadoop.hbase.wal.WALProvider.Writer;
 import org.apache.hadoop.hbase.wal.WALSplitter;
@@ -441,7 +442,7 @@ public class FSHLog extends AbstractFSWAL<Writer> {
   @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "NP_NULL_ON_SOME_PATH_EXCEPTION",
       justification = "Will never be null")
   @Override
-  public long append(final RegionInfo hri, final WALKey key, final WALEdit edits,
+  public long append(final RegionInfo hri, final WALKeyImpl key, final WALEdit edits,
       final boolean inMemstore) throws IOException {
     return stampSequenceIdAndPublishToRingBuffer(hri, key, edits, inMemstore,
       disruptor.getRingBuffer());
@@ -469,17 +470,6 @@ public class FSHLog extends AbstractFSWAL<Writer> {
     private final BlockingQueue<SyncFuture> syncFutures;
     private volatile SyncFuture takeSyncFuture = null;
 
-    /**
-     * UPDATE!
-     * @param syncs the batch of calls to sync that arrived as this thread was starting; when done,
-     *          we will put the result of the actual hdfs sync call as the result.
-     * @param sequence The sequence number on the ring buffer when this thread was set running. If
-     *          this actual writer sync completes then all appends up this point have been
-     *          flushed/synced/pushed to datanodes. If we fail, then the passed in
-     *          <code>syncs</code> futures will return the exception to their clients; some of the
-     *          edits may have made it out to data nodes but we will report all that were part of
-     *          this session as failed.
-     */
     SyncRunner(final String name, final int maxHandlersCount) {
       super(name);
       // LinkedBlockingQueue because of

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5a77ae3/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSWALEntry.java
----------------------------------------------------------------------
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 a928ad5..728a075 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
@@ -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
@@ -35,7 +35,7 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.CollectionUtils;
 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.apache.yetus.audience.InterfaceAudience;
 
 import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
@@ -57,7 +57,7 @@ class FSWALEntry extends Entry {
   private final transient RegionInfo regionInfo;
   private final transient Set<byte[]> familyNames;
 
-  FSWALEntry(final long txid, final WALKey key, final WALEdit edit,
+  FSWALEntry(final long txid, final WALKeyImpl key, final WALEdit edit,
       final RegionInfo regionInfo, final boolean inMemstore) {
     super(key, edit);
     this.inMemstore = inMemstore;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5a77ae3/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogReader.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogReader.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogReader.java
index 78c055e..c199484 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogReader.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogReader.java
@@ -367,7 +367,8 @@ public class ProtobufLogReader extends ReaderBase {
         entry.getKey().readFieldsFromPb(walKey, this.byteStringUncompressor);
         if (!walKey.hasFollowingKvCount() || 0 == walKey.getFollowingKvCount()) {
           if (LOG.isTraceEnabled()) {
-            LOG.trace("WALKey has no KVs that follow it; trying the next one. current offset=" + this.inputStream.getPos());
+            LOG.trace("WALKey has no KVs that follow it; trying the next one. current offset=" +
+                this.inputStream.getPos());
           }
           continue;
         }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5a77ae3/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogWriter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogWriter.java
index d1e72f7..64acfba 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogWriter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogWriter.java
@@ -28,6 +28,7 @@ import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.wal.WALKeyImpl;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALHeader;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.WALTrailer;
@@ -50,8 +51,8 @@ public class ProtobufLogWriter extends AbstractProtobufLogWriter
   @Override
   public void append(Entry entry) throws IOException {
     entry.setCompressionContext(compressionContext);
-    entry.getKey().getBuilder(compressor).setFollowingKvCount(entry.getEdit().size()).build()
-        .writeDelimitedTo(output);
+    ((WALKeyImpl)entry.getKey()).getBuilder(compressor).
+        setFollowingKvCount(entry.getEdit().size()).build().writeDelimitedTo(output);
     for (Cell cell : entry.getEdit().getCells()) {
       // cellEncoder must assume little about the stream, since we write PB and cells in turn.
       cellEncoder.write(cell);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5a77ae3/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ReaderBase.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ReaderBase.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ReaderBase.java
index a30c29c..2a01b14 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ReaderBase.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ReaderBase.java
@@ -35,7 +35,7 @@ import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
 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;
 
 @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
 public abstract class ReaderBase implements AbstractFSWALProvider.Reader {
@@ -92,7 +92,7 @@ public abstract class ReaderBase implements AbstractFSWALProvider.Reader {
   public Entry next(Entry reuse) throws IOException {
     Entry e = reuse;
     if (e == null) {
-      e = new Entry(new WALKey(), new WALEdit());
+      e = new Entry();
     }
     if (compressionContext != null) {
       e.setCompressionContext(compressionContext);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5a77ae3/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALActionsListener.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALActionsListener.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALActionsListener.java
index a25b19d..13ffac7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALActionsListener.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALActionsListener.java
@@ -74,13 +74,8 @@ public interface WALActionsListener {
 
   /**
   * Called before each write.
-  * @param info
-  * @param logKey
-  * @param logEdit
   */
-  default void visitLogEntryBeforeWrite(
-    RegionInfo info, WALKey logKey, WALEdit logEdit
-  ) {}
+  default void visitLogEntryBeforeWrite(RegionInfo info, WALKey logKey, WALEdit logEdit) {}
 
   /**
    * @param logKey

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5a77ae3/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCoprocessorHost.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCoprocessorHost.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCoprocessorHost.java
index d28c3c4..0edd5d4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCoprocessorHost.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCoprocessorHost.java
@@ -139,11 +139,6 @@ public class WALCoprocessorHost
     }
   }
 
-  /**
-   * @deprecated Since hbase-2.0.0. No replacement. To be removed in hbase-3.0.0 and replaced
-   * with something that doesn't expose IntefaceAudience.Private classes.
-   */
-  @Deprecated
   public void preWALWrite(final RegionInfo info, final WALKey logKey, final WALEdit logEdit)
       throws IOException {
     // Not bypassable.
@@ -157,11 +152,7 @@ public class WALCoprocessorHost
       }
     });
   }
-  /**
-   * @deprecated Since hbase-2.0.0. No replacement. To be removed in hbase-3.0.0 and replaced
-   * with something that doesn't expose IntefaceAudience.Private classes.
-   */
-  @Deprecated
+
   public void postWALWrite(final RegionInfo info, final WALKey logKey, final WALEdit logEdit)
       throws IOException {
     execOperation(coprocEnvironments.isEmpty() ? null : new WALObserverOperation() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5a77ae3/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALUtil.java
----------------------------------------------------------------------
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 816a3b8..518ee8f 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
@@ -28,7 +28,7 @@ import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
 import org.apache.hadoop.hbase.wal.WAL;
 import org.apache.hadoop.hbase.wal.WALEdit;
-import org.apache.hadoop.hbase.wal.WALKey;
+import org.apache.hadoop.hbase.wal.WALKeyImpl;
 import org.apache.yetus.audience.InterfaceAudience;
 
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.TextFormat;
@@ -59,11 +59,12 @@ public class WALUtil {
    * <p>This write is for internal use only. Not for external client consumption.
    * @param mvcc Used by WAL to get sequence Id for the waledit.
    */
-  public static WALKey writeCompactionMarker(WAL wal,
+  public static WALKeyImpl writeCompactionMarker(WAL wal,
       NavigableMap<byte[], Integer> replicationScope, RegionInfo hri, final CompactionDescriptor c,
       MultiVersionConcurrencyControl mvcc)
   throws IOException {
-    WALKey walKey = writeMarker(wal, replicationScope, hri, WALEdit.createCompaction(hri, c), mvcc);
+    WALKeyImpl walKey =
+        writeMarker(wal, replicationScope, hri, WALEdit.createCompaction(hri, c), mvcc);
     if (LOG.isTraceEnabled()) {
       LOG.trace("Appended compaction marker " + TextFormat.shortDebugString(c));
     }
@@ -75,10 +76,10 @@ public class WALUtil {
    *
    * <p>This write is for internal use only. Not for external client consumption.
    */
-  public static WALKey writeFlushMarker(WAL wal, NavigableMap<byte[], Integer> replicationScope,
+  public static WALKeyImpl writeFlushMarker(WAL wal, NavigableMap<byte[], Integer> replicationScope,
       RegionInfo hri, final FlushDescriptor f, boolean sync, MultiVersionConcurrencyControl mvcc)
           throws IOException {
-    WALKey walKey = doFullAppendTransaction(wal, replicationScope, hri,
+    WALKeyImpl walKey = doFullAppendTransaction(wal, replicationScope, hri,
         WALEdit.createFlushWALEdit(hri, f), mvcc, sync);
     if (LOG.isTraceEnabled()) {
       LOG.trace("Appended flush marker " + TextFormat.shortDebugString(f));
@@ -90,11 +91,11 @@ public class WALUtil {
    * Write a region open marker indicating that the region is opened.
    * This write is for internal use only. Not for external client consumption.
    */
-  public static WALKey writeRegionEventMarker(WAL wal,
+  public static WALKeyImpl writeRegionEventMarker(WAL wal,
       NavigableMap<byte[], Integer> replicationScope, RegionInfo hri,
       final RegionEventDescriptor r, final MultiVersionConcurrencyControl mvcc)
   throws IOException {
-    WALKey walKey = writeMarker(wal, replicationScope, hri,
+    WALKeyImpl walKey = writeMarker(wal, replicationScope, hri,
         WALEdit.createRegionEventWALEdit(hri, r), mvcc);
     if (LOG.isTraceEnabled()) {
       LOG.trace("Appended region event marker " + TextFormat.shortDebugString(r));
@@ -112,19 +113,19 @@ public class WALUtil {
    * @return walKey with sequenceid filled out for this bulk load marker
    * @throws IOException We will throw an IOException if we can not append to the HLog.
    */
-  public static WALKey writeBulkLoadMarkerAndSync(final WAL wal,
+  public static WALKeyImpl writeBulkLoadMarkerAndSync(final WAL wal,
       final NavigableMap<byte[], Integer> replicationScope, final RegionInfo hri,
       final WALProtos.BulkLoadDescriptor desc, final MultiVersionConcurrencyControl mvcc)
           throws IOException {
-    WALKey walKey = writeMarker(wal, replicationScope, hri, WALEdit.createBulkLoadEvent(hri, desc),
-        mvcc);
+    WALKeyImpl walKey =
+        writeMarker(wal, replicationScope, hri, WALEdit.createBulkLoadEvent(hri, desc), mvcc);
     if (LOG.isTraceEnabled()) {
       LOG.trace("Appended Bulk Load marker " + TextFormat.shortDebugString(desc));
     }
     return walKey;
   }
 
-  private static WALKey writeMarker(final WAL wal,
+  private static WALKeyImpl writeMarker(final WAL wal,
       final NavigableMap<byte[], Integer> replicationScope, final RegionInfo hri,
       final WALEdit edit, final MultiVersionConcurrencyControl mvcc)
   throws IOException {
@@ -138,14 +139,14 @@ public class WALUtil {
    * Good for case of adding a single edit or marker to the WAL.
    *
    * <p>This write is for internal use only. Not for external client consumption.
-   * @return WALKey that was added to the WAL.
+   * @return WALKeyImpl that was added to the WAL.
    */
-  public static WALKey doFullAppendTransaction(final WAL wal,
+  public static WALKeyImpl doFullAppendTransaction(final WAL wal,
       final NavigableMap<byte[], Integer> replicationScope, final RegionInfo hri,
       final WALEdit edit, final MultiVersionConcurrencyControl mvcc, final boolean sync)
   throws IOException {
     // TODO: Pass in current time to use?
-    WALKey walKey = new WALKey(hri.getEncodedNameAsBytes(), hri.getTable(),
+    WALKeyImpl walKey = new WALKeyImpl(hri.getEncodedNameAsBytes(), hri.getTable(),
         System.currentTimeMillis(), mvcc, replicationScope);
     long trx = MultiVersionConcurrencyControl.NONE;
     try {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5a77ae3/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ClusterMarkingEntryFilter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ClusterMarkingEntryFilter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ClusterMarkingEntryFilter.java
index c629e16..6dc5001 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ClusterMarkingEntryFilter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ClusterMarkingEntryFilter.java
@@ -24,7 +24,7 @@ import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.yetus.audience.InterfaceStability;
 import org.apache.hadoop.hbase.wal.WALEdit;
-import org.apache.hadoop.hbase.wal.WALKey;
+import org.apache.hadoop.hbase.wal.WALKeyImpl;
 import org.apache.hadoop.hbase.wal.WAL.Entry;
 
 
@@ -55,7 +55,7 @@ public class ClusterMarkingEntryFilter implements WALEntryFilter {
     if (replicationEndpoint.canReplicateToSameCluster()
         || !entry.getKey().getClusterIds().contains(peerClusterId)) {
       WALEdit edit = entry.getEdit();
-      WALKey logKey = entry.getKey();
+      WALKeyImpl logKey = (WALKeyImpl)entry.getKey();
 
       if (edit != null && !edit.isEmpty()) {
         // Mark that the current cluster has the change

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5a77ae3/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
index 0e4aadd..2a2df60 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
@@ -31,6 +31,8 @@ import java.util.concurrent.TimeUnit;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.wal.WALKey;
+import org.apache.hadoop.hbase.wal.WALKeyImpl;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
 import org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
@@ -58,7 +60,6 @@ import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
 import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.replication.master.ReplicationHFileCleaner;
 import org.apache.hadoop.hbase.replication.master.ReplicationLogCleaner;
-import org.apache.hadoop.hbase.wal.WALKey;
 import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
 import org.apache.zookeeper.KeeperException;
 
@@ -266,7 +267,7 @@ public class Replication implements
       }
     }
     if ((!replicationForBulkLoadEnabled && !foundOtherEdits) || logEdit.isReplay()) {
-      logKey.serializeReplicationScope(false);
+      ((WALKeyImpl)logKey).serializeReplicationScope(false);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5a77ae3/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityReplicationEndpoint.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityReplicationEndpoint.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityReplicationEndpoint.java
index 4c86323..038b799 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityReplicationEndpoint.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityReplicationEndpoint.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.PrivateCellUtil;
 import org.apache.hadoop.hbase.Tag;
 import org.apache.hadoop.hbase.TagType;
+import org.apache.hadoop.hbase.wal.WALKeyImpl;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.replication.ReplicationEndpoint;
@@ -111,7 +112,7 @@ public class VisibilityReplicationEndpoint implements ReplicationEndpoint {
             newEdit.add(cell);
           }
         }
-        newEntries.add(new Entry(entry.getKey(), newEdit));
+        newEntries.add(new Entry(((WALKeyImpl)entry.getKey()), newEdit));
       }
       replicateContext.setEntries(newEntries);
       return delegator.replicate(replicateContext);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5a77ae3/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DisabledWALProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DisabledWALProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DisabledWALProvider.java
index 76b65a7..cedf350 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
@@ -161,7 +161,7 @@ class DisabledWALProvider implements WALProvider {
     }
 
     @Override
-    public long append(RegionInfo info, WALKey key, WALEdit edits, boolean inMemstore)
+    public long append(RegionInfo info, WALKeyImpl key, WALEdit edits, boolean inMemstore)
         throws IOException {
       if (!this.listeners.isEmpty()) {
         final long start = System.nanoTime();

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5a77ae3/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WAL.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WAL.java
index e319255..1002357 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WAL.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WAL.java
@@ -1,4 +1,4 @@
-/**
+/*
  *
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
@@ -116,7 +116,7 @@ public interface WAL extends Closeable, WALFileLengthProvider {
    * @return Returns a 'transaction id' and <code>key</code> will have the region edit/sequence id
    * in it.
    */
-  long append(RegionInfo info, WALKey key, WALEdit edits, boolean inMemstore) throws IOException;
+  long append(RegionInfo info, WALKeyImpl key, WALEdit edits, boolean inMemstore) throws IOException;
 
   /**
    * updates the seuence number of a specific store.
@@ -230,10 +230,10 @@ public interface WAL extends Closeable, WALFileLengthProvider {
    */
   class Entry {
     private final WALEdit edit;
-    private final WALKey key;
+    private final WALKeyImpl key;
 
     public Entry() {
-      this(new WALKey(), new WALEdit());
+      this(new WALKeyImpl(), new WALEdit());
     }
 
     /**
@@ -242,7 +242,7 @@ public interface WAL extends Closeable, WALFileLengthProvider {
      * @param edit log's edit
      * @param key log's key
      */
-    public Entry(WALKey key, WALEdit edit) {
+    public Entry(WALKeyImpl key, WALEdit edit) {
       this.key = key;
       this.edit = edit;
     }
@@ -261,7 +261,7 @@ public interface WAL extends Closeable, WALFileLengthProvider {
      *
      * @return key
      */
-    public WALKey getKey() {
+    public WALKeyImpl getKey() {
       return key;
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5a77ae3/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALEdit.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALEdit.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALEdit.java
index 2feb356..f5b611b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALEdit.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALEdit.java
@@ -51,6 +51,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDe
  * All the edits for a given transaction are written out as a single record, in PB format followed
  * by Cells written via the WALCellEncoder.
  */
+// TODO: Do not expose this class to Coprocessors. It has set methods. A CP might meddle.
 @InterfaceAudience.LimitedPrivate({ HBaseInterfaceAudience.REPLICATION,
     HBaseInterfaceAudience.COPROC })
 public class WALEdit implements HeapSize {
@@ -119,6 +120,7 @@ public class WALEdit implements HeapSize {
     return this.isReplay;
   }
 
+  @InterfaceAudience.Private
   public WALEdit add(Cell cell) {
     this.cells.add(cell);
     return this;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5a77ae3/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKey.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKey.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKey.java
index fd40ec4..0c818db 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKey.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKey.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
@@ -18,630 +17,88 @@
  */
 package org.apache.hadoop.hbase.wal;
 
-import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
+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.io.IOException;
-import java.io.InterruptedIOException;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.NavigableMap;
-import java.util.TreeMap;
 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.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
-import org.apache.hadoop.hbase.regionserver.SequenceId;
-// imports for things that haven't moved from regionserver.wal yet.
-import org.apache.hadoop.hbase.regionserver.wal.CompressionContext;
-import org.apache.hadoop.hbase.regionserver.wal.WALCellCodec;
-import org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString;
-import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
-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.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 
 /**
- * A Key for an entry in the WAL.
- *
- * The log intermingles edits to many tables and rows, so each log entry
- * identifies the appropriate table and row.  Within a table and row, they're
- * also sorted.
- *
- * <p>Some Transactional edits (START, COMMIT, ABORT) will not have an associated row.
- *
+ * Key for WAL Entry.
+ * Read-only. No Setters. For limited audience such as Coprocessors.
  */
-// TODO: Key and WALEdit are never used separately, or in one-to-many relation, for practical
-//       purposes. They need to be merged into WALEntry.
-@InterfaceAudience.Private
-public class WALKey implements SequenceId, Comparable<WALKey> {
-
-  /**
-   * Used to represent when a particular wal key doesn't know/care about the sequence ordering.
-   */
-  public static final long NO_SEQUENCE_ID = -1;
-
-  @InterfaceAudience.Private // For internal use only.
-  public MultiVersionConcurrencyControl getMvcc() {
-    return mvcc;
-  }
-
-  /**
-   * Use it to complete mvcc transaction. This WALKey was part of
-   * (the transaction is started when you call append; see the comment on FSHLog#append). To
-   * complete call
-   * {@link MultiVersionConcurrencyControl#complete(MultiVersionConcurrencyControl.WriteEntry)}
-   * or {@link MultiVersionConcurrencyControl#complete(MultiVersionConcurrencyControl.WriteEntry)}
-   * @return A WriteEntry gotten from local WAL subsystem.
-   * @see #setWriteEntry(MultiVersionConcurrencyControl.WriteEntry)
-   */
-  @InterfaceAudience.Private // For internal use only.
-  public MultiVersionConcurrencyControl.WriteEntry getWriteEntry() throws InterruptedIOException {
-    assert this.writeEntry != null;
-    return this.writeEntry;
-  }
-
-  @InterfaceAudience.Private // For internal use only.
-  public void setWriteEntry(MultiVersionConcurrencyControl.WriteEntry writeEntry) {
-    assert this.writeEntry == null;
-    this.writeEntry = writeEntry;
-    // Set our sequenceid now using WriteEntry.
-    this.sequenceId = writeEntry.getWriteNumber();
-  }
-
-  private byte [] encodedRegionName;
-
-  private TableName tablename;
-  /**
-   * SequenceId for this edit. Set post-construction at write-to-WAL time. Until then it is
-   * NO_SEQUENCE_ID. Change it so multiple threads can read it -- e.g. access is synchronized.
-   */
-  private long sequenceId;
-
+@InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.REPLICATION,
+    HBaseInterfaceAudience.COPROC})
+public interface WALKey extends SequenceId, Comparable<WALKey> {
   /**
-   * Used during WAL replay; the sequenceId of the edit when it came into the system.
+   * Unmodifiable empty list of UUIDs.
    */
-  private long origLogSeqNum = 0;
+  List<UUID> EMPTY_UUIDS = Collections.unmodifiableList(new ArrayList<UUID>());
 
-  /** Time at which this edit was written. */
-  private long writeTime;
-
-  /** The first element in the list is the cluster id on which the change has originated */
-  private List<UUID> clusterIds;
-
-  private NavigableMap<byte[], Integer> replicationScope;
-
-  private long nonceGroup = HConstants.NO_NONCE;
-  private long nonce = HConstants.NO_NONCE;
-  private MultiVersionConcurrencyControl mvcc;
-  /**
-   * Set in a way visible to multiple threads; e.g. synchronized getter/setters.
-   */
-  private MultiVersionConcurrencyControl.WriteEntry writeEntry;
-  public static final List<UUID> EMPTY_UUIDS = Collections.unmodifiableList(new ArrayList<UUID>());
-
-  private CompressionContext compressionContext;
-
-  public WALKey() {
-    init(null, null, 0L, HConstants.LATEST_TIMESTAMP,
-        new ArrayList<>(), HConstants.NO_NONCE, HConstants.NO_NONCE, null, null);
-  }
-
-  public WALKey(final NavigableMap<byte[], Integer> replicationScope) {
-    init(null, null, 0L, HConstants.LATEST_TIMESTAMP,
-        new ArrayList<>(), HConstants.NO_NONCE, HConstants.NO_NONCE, null, replicationScope);
-  }
-
-  @VisibleForTesting
-  public WALKey(final byte[] encodedRegionName, final TableName tablename,
-                long logSeqNum,
-      final long now, UUID clusterId) {
-    List<UUID> clusterIds = new ArrayList<>(1);
-    clusterIds.add(clusterId);
-    init(encodedRegionName, tablename, logSeqNum, now, clusterIds,
-        HConstants.NO_NONCE, HConstants.NO_NONCE, null, null);
-  }
-
-  // TODO: Fix being able to pass in sequenceid.
-  public WALKey(final byte[] encodedRegionName, final TableName tablename, final long now) {
-    init(encodedRegionName,
-        tablename,
-        NO_SEQUENCE_ID,
-        now,
-        EMPTY_UUIDS,
-        HConstants.NO_NONCE,
-        HConstants.NO_NONCE,
-        null, null);
-  }
-
-  // TODO: Fix being able to pass in sequenceid.
-  public WALKey(final byte[] encodedRegionName, final TableName tablename, final long now,
-      final NavigableMap<byte[], Integer> replicationScope) {
-    init(encodedRegionName, tablename, NO_SEQUENCE_ID, now, EMPTY_UUIDS, HConstants.NO_NONCE,
-        HConstants.NO_NONCE, null, replicationScope);
-  }
-
-  public WALKey(final byte[] encodedRegionName, final TableName tablename, final long now,
-      MultiVersionConcurrencyControl mvcc, final NavigableMap<byte[], Integer> replicationScope) {
-    init(encodedRegionName, tablename, NO_SEQUENCE_ID, now, EMPTY_UUIDS, HConstants.NO_NONCE,
-        HConstants.NO_NONCE, mvcc, replicationScope);
-  }
-
-  public WALKey(final byte[] encodedRegionName,
-                final TableName tablename,
-                final long now,
-                MultiVersionConcurrencyControl mvcc) {
-    init(encodedRegionName,
-        tablename,
-        NO_SEQUENCE_ID,
-        now,
-        EMPTY_UUIDS,
-        HConstants.NO_NONCE,
-        HConstants.NO_NONCE,
-        mvcc, null);
+  default long estimatedSerializedSizeOf() {
+    return 0;
   }
 
   /**
-   * Create the log key for writing to somewhere.
-   * We maintain the tablename mainly for debugging purposes.
-   * A regionName is always a sub-table object.
-   * <p>Used by log splitting and snapshots.
-   *
-   * @param encodedRegionName Encoded name of the region as returned by
-   *                         <code>HRegionInfo#getEncodedNameAsBytes()</code>.
-   * @param tablename         - name of table
-   * @param logSeqNum         - log sequence number
-   * @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
-   * @param nonce             the nonce
-   * @param mvcc              the mvcc associate the WALKey
-   * @param replicationScope  the non-default replication scope
-   *                          associated with the region's column families
+   * @return encoded region name
    */
-  // TODO: Fix being able to pass in sequenceid.
-  public WALKey(final byte[] encodedRegionName, final TableName tablename, long logSeqNum,
-      final long now, List<UUID> clusterIds, long nonceGroup, long nonce,
-      MultiVersionConcurrencyControl mvcc, final NavigableMap<byte[], Integer> replicationScope) {
-    init(encodedRegionName, tablename, logSeqNum, now, clusterIds, nonceGroup, nonce, mvcc,
-        replicationScope);
-  }
+  byte[] getEncodedRegionName();
 
   /**
-   * Create the log key for writing to somewhere.
-   * We maintain the tablename mainly for debugging purposes.
-   * A regionName is always a sub-table object.
-   * <p>Used by log splitting and snapshots.
-   *
-   * @param encodedRegionName Encoded name of the region as returned by
-   *                          <code>HRegionInfo#getEncodedNameAsBytes()</code>.
-   * @param tablename         - name of table
-   * @param logSeqNum         - log sequence number
-   * @param now               Time at which this edit was written.
-   * @param clusterIds        the clusters that have consumed the change(used in Replication)
+   * @return table name
    */
-  // TODO: Fix being able to pass in sequenceid.
-  public WALKey(final byte[] encodedRegionName,
-                final TableName tablename,
-                long logSeqNum,
-                final long now,
-                List<UUID> clusterIds,
-                long nonceGroup,
-                long nonce,
-                MultiVersionConcurrencyControl mvcc) {
-    init(encodedRegionName, tablename, logSeqNum, now, clusterIds, nonceGroup, nonce, mvcc, null);
-  }
+  TableName getTablename();
 
   /**
-   * Create the log key for writing to somewhere.
-   * We maintain the tablename mainly for debugging purposes.
-   * A regionName is always a sub-table object.
-   *
-   * @param encodedRegionName Encoded name of the region as returned by
-   *                          <code>HRegionInfo#getEncodedNameAsBytes()</code>.
-   * @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
+   * @return the write time
    */
-  public WALKey(final byte[] encodedRegionName, final TableName tablename,
-                final long now, List<UUID> clusterIds, long nonceGroup,
-                final long nonce, final MultiVersionConcurrencyControl mvcc) {
-    init(encodedRegionName, tablename, NO_SEQUENCE_ID, now, clusterIds, nonceGroup, nonce, mvcc,
-        null);
-  }
+  long getWriteTime();
 
   /**
-   * Create the log key for writing to somewhere.
-   * We maintain the tablename mainly for debugging purposes.
-   * A regionName is always a sub-table object.
-   *
-   * @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
-   * @param nonce             the nonce
-   * @param mvcc mvcc control used to generate sequence numbers and control read/write points
-   * @param replicationScope  the non-default replication scope of the column families
+   * @return The nonce group
    */
-  public WALKey(final byte[] encodedRegionName, final TableName tablename,
-                final long now, List<UUID> clusterIds, long nonceGroup,
-                final long nonce, final MultiVersionConcurrencyControl mvcc,
-                NavigableMap<byte[], Integer> replicationScope) {
-    init(encodedRegionName, tablename, NO_SEQUENCE_ID, now, clusterIds, nonceGroup, nonce, mvcc,
-        replicationScope);
+  default long getNonceGroup() {
+    return HConstants.NO_NONCE;
   }
 
   /**
-   * Create the log key for writing to somewhere.
-   * We maintain the tablename mainly for debugging purposes.
-   * A regionName is always a sub-table object.
-   *
-   * @param encodedRegionName Encoded name of the region as returned by
-   *                          <code>HRegionInfo#getEncodedNameAsBytes()</code>.
-   * @param tablename
-   * @param logSeqNum
-   * @param nonceGroup
-   * @param nonce
+   * @return The nonce
    */
-  // TODO: Fix being able to pass in sequenceid.
-  public WALKey(final byte[] encodedRegionName,
-                final TableName tablename,
-                long logSeqNum,
-                long nonceGroup,
-                long nonce,
-                final MultiVersionConcurrencyControl mvcc) {
-    init(encodedRegionName,
-        tablename,
-        logSeqNum,
-        EnvironmentEdgeManager.currentTime(),
-        EMPTY_UUIDS,
-        nonceGroup,
-        nonce,
-        mvcc, null);
-  }
-
-  @InterfaceAudience.Private
-  protected void init(final byte[] encodedRegionName,
-                      final TableName tablename,
-                      long logSeqNum,
-                      final long now,
-                      List<UUID> clusterIds,
-                      long nonceGroup,
-                      long nonce,
-                      MultiVersionConcurrencyControl mvcc,
-                      NavigableMap<byte[], Integer> replicationScope) {
-    this.sequenceId = logSeqNum;
-    this.writeTime = now;
-    this.clusterIds = clusterIds;
-    this.encodedRegionName = encodedRegionName;
-    this.tablename = tablename;
-    this.nonceGroup = nonceGroup;
-    this.nonce = nonce;
-    this.mvcc = mvcc;
-    if (logSeqNum != NO_SEQUENCE_ID) {
-      setSequenceId(logSeqNum);
-    }
-    this.replicationScope = replicationScope;
+  default long getNonce() {
+    return HConstants.NO_NONCE;
   }
 
-  // For deserialization. DO NOT USE. See setWriteEntry below.
-  @InterfaceAudience.Private
-  protected void setSequenceId(long sequenceId) {
-    this.sequenceId = sequenceId;
-  }
+  UUID getOriginatingClusterId();
 
   /**
-   * @param compressionContext Compression context to use
-   */
-  public void setCompressionContext(CompressionContext compressionContext) {
-    this.compressionContext = compressionContext;
-  }
-
-  /** @return encoded region name */
-  public byte [] getEncodedRegionName() {
-    return encodedRegionName;
-  }
-
-  /** @return table name */
-  public TableName getTablename() {
-    return tablename;
-  }
-
-  /** @return log sequence number
-   * @deprecated Use {@link #getSequenceId()}
-   */
-  @Deprecated
-  public long getLogSeqNum() {
-    return getSequenceId();
-  }
-
-  /**
-   * Used to set original sequenceId for WALKey during WAL replay
-   */
-  public void setOrigLogSeqNum(final long sequenceId) {
-    this.origLogSeqNum = sequenceId;
-  }
-  
-  /**
-   * Return a positive long if current WALKey is created from a replay edit; a replay edit is an
+   * Return a positive long if current WALKeyImpl is created from a replay edit; a replay edit is an
    * edit that came in when replaying WALs of a crashed server.
    * @return original sequence number of the WALEdit
    */
-  public long getOrigLogSeqNum() {
-    return this.origLogSeqNum;
-  }
-  
-  /**
-   * SequenceId is only available post WAL-assign. Calls before this will get you a
-   * {@link #NO_SEQUENCE_ID}. See the comment on FSHLog#append and #getWriteNumber in this method
-   * for more on when this sequenceId comes available.
-   * @return long the new assigned sequence number
-   */
-  @Override
-  public long getSequenceId() {
-    return this.sequenceId;
-  }
-
-  /**
-   * @return the write time
-   */
-  public long getWriteTime() {
-    return this.writeTime;
-  }
-
-  public NavigableMap<byte[], Integer> getReplicationScopes() {
-    return replicationScope;
-  }
-
-  /** @return The nonce group */
-  public long getNonceGroup() {
-    return nonceGroup;
-  }
-
-  /** @return The nonce */
-  public long getNonce() {
-    return nonce;
-  }
-
-  private void setReplicationScope(NavigableMap<byte[], Integer> replicationScope) {
-    this.replicationScope = replicationScope;
-  }
-
-  public void serializeReplicationScope(boolean serialize) {
-    if (!serialize) {
-      setReplicationScope(null);
-    }
-  }
-
-  /**
-   * Marks that the cluster with the given clusterId has consumed the change
-   */
-  public void addClusterId(UUID clusterId) {
-    if (!clusterIds.contains(clusterId)) {
-      clusterIds.add(clusterId);
-    }
-  }
-
-  /**
-   * @return the set of cluster Ids that have consumed the change
-   */
-  public List<UUID> getClusterIds() {
-    return clusterIds;
-  }
-
-  /**
-   * @return the cluster id on which the change has originated. It there is no such cluster, it
-   *         returns DEFAULT_CLUSTER_ID (cases where replication is not enabled)
-   */
-  public UUID getOriginatingClusterId(){
-    return clusterIds.isEmpty() ? HConstants.DEFAULT_CLUSTER_ID : clusterIds.get(0);
-  }
-
-  @Override
-  public String toString() {
-    return tablename + "/" + Bytes.toString(encodedRegionName) + "/" +
-      sequenceId;
-  }
+  long getOrigLogSeqNum();
 
   /**
    * Produces a string map for this key. Useful for programmatic use and
-   * manipulation of the data stored in an WALKey, for example, printing
+   * manipulation of the data stored in an WALKeyImpl, for example, printing
    * as JSON.
    *
    * @return a Map containing data from this key
    */
-  public Map<String, Object> toStringMap() {
+  default Map<String, Object> toStringMap() {
     Map<String, Object> stringMap = new HashMap<>();
-    stringMap.put("table", tablename);
-    stringMap.put("region", Bytes.toStringBinary(encodedRegionName));
+    stringMap.put("table", getTablename());
+    stringMap.put("region", Bytes.toStringBinary(getEncodedRegionName()));
     stringMap.put("sequence", getSequenceId());
     return stringMap;
   }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (this == obj) {
-      return true;
-    }
-    if (obj == null || getClass() != obj.getClass()) {
-      return false;
-    }
-    return compareTo((WALKey)obj) == 0;
-  }
-
-  @Override
-  public int hashCode() {
-    int result = Bytes.hashCode(this.encodedRegionName);
-    result ^= getSequenceId();
-    result ^= this.writeTime;
-    return result;
-  }
-
-  @Override
-  public int compareTo(WALKey o) {
-    int result = Bytes.compareTo(this.encodedRegionName, o.encodedRegionName);
-    if (result == 0) {
-      long sid = getSequenceId();
-      long otherSid = o.getSequenceId();
-      if (sid < otherSid) {
-        result = -1;
-      } else if (sid  > otherSid) {
-        result = 1;
-      }
-      if (result == 0) {
-        if (this.writeTime < o.writeTime) {
-          result = -1;
-        } else if (this.writeTime > o.writeTime) {
-          return 1;
-        }
-      }
-    }
-    // why isn't cluster id accounted for?
-    return result;
-  }
-
-  /**
-   * Drop this instance's tablename byte array and instead
-   * hold a reference to the provided tablename. This is not
-   * meant to be a general purpose setter - it's only used
-   * to collapse references to conserve memory.
-   */
-  void internTableName(TableName tablename) {
-    // We should not use this as a setter - only to swap
-    // in a new reference to the same table name.
-    assert tablename.equals(this.tablename);
-    this.tablename = tablename;
-  }
-
-  /**
-   * Drop this instance's region name byte array and instead
-   * hold a reference to the provided region name. This is not
-   * meant to be a general purpose setter - it's only used
-   * to collapse references to conserve memory.
-   */
-  void internEncodedRegionName(byte []encodedRegionName) {
-    // We should not use this as a setter - only to swap
-    // in a new reference to the same table name.
-    assert Bytes.equals(this.encodedRegionName, encodedRegionName);
-    this.encodedRegionName = encodedRegionName;
-  }
-
-  public WALProtos.WALKey.Builder getBuilder(
-      WALCellCodec.ByteStringCompressor compressor) throws IOException {
-    WALProtos.WALKey.Builder builder = WALProtos.WALKey.newBuilder();
-    if (compressionContext == null) {
-      builder.setEncodedRegionName(UnsafeByteOperations.unsafeWrap(this.encodedRegionName));
-      builder.setTableName(UnsafeByteOperations.unsafeWrap(this.tablename.getName()));
-    } else {
-      builder.setEncodedRegionName(compressor.compress(this.encodedRegionName,
-          compressionContext.regionDict));
-      builder.setTableName(compressor.compress(this.tablename.getName(),
-          compressionContext.tableDict));
-    }
-    builder.setLogSequenceNumber(getSequenceId());
-    builder.setWriteTime(writeTime);
-    if (this.origLogSeqNum > 0) {
-      builder.setOrigSequenceNumber(this.origLogSeqNum);
-    }
-    if (this.nonce != HConstants.NO_NONCE) {
-      builder.setNonce(nonce);
-    }
-    if (this.nonceGroup != HConstants.NO_NONCE) {
-      builder.setNonceGroup(nonceGroup);
-    }
-    HBaseProtos.UUID.Builder uuidBuilder = HBaseProtos.UUID.newBuilder();
-    for (UUID clusterId : clusterIds) {
-      uuidBuilder.setLeastSigBits(clusterId.getLeastSignificantBits());
-      uuidBuilder.setMostSigBits(clusterId.getMostSignificantBits());
-      builder.addClusterIds(uuidBuilder.build());
-    }
-    if (replicationScope != null) {
-      for (Map.Entry<byte[], Integer> e : replicationScope.entrySet()) {
-        ByteString family = (compressionContext == null)
-            ? UnsafeByteOperations.unsafeWrap(e.getKey())
-            : compressor.compress(e.getKey(), compressionContext.familyDict);
-        builder.addScopes(FamilyScope.newBuilder()
-            .setFamily(family).setScopeType(ScopeType.forNumber(e.getValue())));
-      }
-    }
-    return builder;
-  }
-
-  public void readFieldsFromPb(WALProtos.WALKey walKey,
-                               WALCellCodec.ByteStringUncompressor uncompressor)
-      throws IOException {
-    if (this.compressionContext != null) {
-      this.encodedRegionName = uncompressor.uncompress(
-          walKey.getEncodedRegionName(), compressionContext.regionDict);
-      byte[] tablenameBytes = uncompressor.uncompress(
-          walKey.getTableName(), compressionContext.tableDict);
-      this.tablename = TableName.valueOf(tablenameBytes);
-    } else {
-      this.encodedRegionName = walKey.getEncodedRegionName().toByteArray();
-      this.tablename = TableName.valueOf(walKey.getTableName().toByteArray());
-    }
-    clusterIds.clear();
-    for (HBaseProtos.UUID clusterId : walKey.getClusterIdsList()) {
-      clusterIds.add(new UUID(clusterId.getMostSigBits(), clusterId.getLeastSigBits()));
-    }
-    if (walKey.hasNonceGroup()) {
-      this.nonceGroup = walKey.getNonceGroup();
-    }
-    if (walKey.hasNonce()) {
-      this.nonce = walKey.getNonce();
-    }
-    this.replicationScope = null;
-    if (walKey.getScopesCount() > 0) {
-      this.replicationScope = new TreeMap<>(Bytes.BYTES_COMPARATOR);
-      for (FamilyScope scope : walKey.getScopesList()) {
-        byte[] family = (compressionContext == null) ? scope.getFamily().toByteArray() :
-          uncompressor.uncompress(scope.getFamily(), compressionContext.familyDict);
-        this.replicationScope.put(family, scope.getScopeType().getNumber());
-      }
-    }
-    setSequenceId(walKey.getLogSequenceNumber());
-    this.writeTime = walKey.getWriteTime();
-    if(walKey.hasOrigSequenceNumber()) {
-      this.origLogSeqNum = walKey.getOrigSequenceNumber();
-    }
-  }
-
-  public long estimatedSerializedSizeOf() {
-    long size = encodedRegionName != null ? encodedRegionName.length : 0;
-    size += tablename != null ? tablename.toBytes().length : 0;
-    if (clusterIds != null) {
-      size += 16 * clusterIds.size();
-    }
-    if (nonceGroup != HConstants.NO_NONCE) {
-      size += Bytes.SIZEOF_LONG; // nonce group
-    }
-    if (nonce != HConstants.NO_NONCE) {
-      size += Bytes.SIZEOF_LONG; // nonce
-    }
-    if (replicationScope != null) {
-      for (Map.Entry<byte[], Integer> scope: replicationScope.entrySet()) {
-        size += scope.getKey().length;
-        size += Bytes.SIZEOF_INT;
-      }
-    }
-    size += Bytes.SIZEOF_LONG; // sequence number
-    size += Bytes.SIZEOF_LONG; // write time
-    if (origLogSeqNum > 0) {
-      size += Bytes.SIZEOF_LONG; // original sequence number
-    }
-    return size;
-  }
 }