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 2019/10/31 05:28:24 UTC

[hbase] branch branch-2 updated: HBASE-23221 Polish the WAL interface after HBASE-23181 (#774)

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

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


The following commit(s) were added to refs/heads/branch-2 by this push:
     new 471538c  HBASE-23221 Polish the WAL interface after HBASE-23181 (#774)
471538c is described below

commit 471538ca9eaddf2489d4b546ad15a370a81ed196
Author: Michael Stack <sa...@users.noreply.github.com>
AuthorDate: Wed Oct 30 22:02:49 2019 -0700

    HBASE-23221 Polish the WAL interface after HBASE-23181 (#774)
    
    Removes the closeRegion flag added by HBASE-23181 and instead
    relies on reading meta WALEdit content. Modified how qualifier is
    written when the meta WALEdit is for a RegionEventDescriptor
    so the 'type' is added to the qualifer so can figure type
    w/o having to deserialize protobuf value content: e.g.
    HBASE::REGION_EVENT::REGION_CLOSE
    
    Added doc on WALEdit and tried to formalize the 'meta' WALEdit
    type and how it works. Needs complete redo in part as suggested
    by HBASE-8457. Meantime, some doc and cleanup.
    
    Also changed the LogRoller constructor to remove redundant param.
    Because of constructor change, need to change also
    TestFailedAppendAndSync, TestWALLockup, TestAsyncFSWAL &
    WALPerformanceEvaluation.java
    
    Signed-off-by: Duo Zhang <zh...@apache.org>
    Signed-off-by: Lijin Bin <bi...@apache.org>
---
 .../java/org/apache/hadoop/hbase/CellUtil.java     |  13 +-
 .../apache/hadoop/hbase/mapreduce/WALPlayer.java   |   3 +-
 .../org/apache/hadoop/hbase/master/HMaster.java    |   2 +-
 .../apache/hadoop/hbase/regionserver/HRegion.java  |   7 +-
 .../hadoop/hbase/regionserver/HRegionServer.java   |   2 +-
 .../hadoop/hbase/regionserver/LogRoller.java       |  24 ++-
 .../hbase/regionserver/wal/AbstractFSWAL.java      |  40 ++---
 .../hadoop/hbase/regionserver/wal/AsyncFSWAL.java  |   6 +-
 .../hadoop/hbase/regionserver/wal/FSHLog.java      |   4 +-
 .../hadoop/hbase/regionserver/wal/FSWALEntry.java  |  27 ++-
 .../regionserver/wal/SequenceIdAccounting.java     |  11 +-
 .../hadoop/hbase/regionserver/wal/WALUtil.java     |  41 +++--
 .../ReplicationSourceWALActionListener.java        |   3 +-
 .../hadoop/hbase/wal/DisabledWALProvider.java      |  10 +-
 .../hbase/wal/LogRecoveredEditsOutputSink.java     |   2 +-
 .../main/java/org/apache/hadoop/hbase/wal/WAL.java |  21 ++-
 .../java/org/apache/hadoop/hbase/wal/WALEdit.java  | 189 +++++++++++++++------
 .../hadoop/hbase/regionserver/TestBulkLoad.java    |  37 ++--
 .../regionserver/TestFailedAppendAndSync.java      |  18 +-
 .../hadoop/hbase/regionserver/TestHRegion.java     |  27 ++-
 .../hbase/regionserver/TestRecoveredEdits.java     |  12 +-
 .../hadoop/hbase/regionserver/TestWALLockup.java   |  24 ++-
 .../hbase/regionserver/wal/AbstractTestFSWAL.java  |   4 +-
 .../regionserver/wal/AbstractTestWALReplay.java    |   2 +-
 .../hbase/regionserver/wal/TestAsyncFSWAL.java     |  10 +-
 .../org/apache/hadoop/hbase/wal/FaultyFSLog.java   |   6 +-
 .../org/apache/hadoop/hbase/wal/TestWALSplit.java  |   6 +-
 .../hadoop/hbase/wal/WALPerformanceEvaluation.java |   2 +-
 28 files changed, 315 insertions(+), 238 deletions(-)

diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
index 20ad149..3d4ae6c 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
@@ -756,7 +756,7 @@ public final class CellUtil {
 
   /**
    * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0. Instead use
-   *             {@link #matchingRows(Cell, byte[]))}
+   *   {@link #matchingRows(Cell, byte[])}
    */
   @Deprecated
   public static boolean matchingRow(final Cell left, final byte[] buf) {
@@ -894,8 +894,15 @@ public final class CellUtil {
   }
 
   public static boolean matchingColumn(final Cell left, final byte[] fam, final byte[] qual) {
-    if (!matchingFamily(left, fam)) return false;
-    return matchingQualifier(left, qual);
+    return matchingFamily(left, fam) && matchingQualifier(left, qual);
+  }
+
+  /**
+   * @return True if matching column family and the qualifier starts with <code>qual</code>
+   */
+  public static boolean matchingColumnFamilyAndQualifierPrefix(final Cell left, final byte[] fam,
+      final byte[] qual) {
+    return matchingFamily(left, fam) && PrivateCellUtil.qualifierStartsWith(left, qual);
   }
 
   /**
diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java
index 5251887..58e8448 100644
--- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java
+++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java
@@ -199,11 +199,10 @@ public class WALPlayer extends Configured implements Tool {
           Delete del = null;
           Cell lastCell = null;
           for (Cell cell : value.getCells()) {
-            // filtering WAL meta entries
+            // Filtering WAL meta marker entries.
             if (WALEdit.isMetaEditFamily(cell)) {
               continue;
             }
-
             // Allow a subclass filter out this cell.
             if (filter(context, cell)) {
               // A WALEdit may contain multiple operations (HBASE-3584) and/or
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index 880466e..ec9824b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -3566,7 +3566,7 @@ public class HMaster extends HRegionServer implements MasterServices {
     if (cpHost != null) {
       cpHost.preListReplicationPeers(regex);
     }
-    LOG.info(getClientIdAuditPrefix() + " list replication peers, regex=" + regex);
+    LOG.debug("{} list replication peers, regex={}", getClientIdAuditPrefix(), regex);
     Pattern pattern = regex == null ? null : Pattern.compile(regex);
     List<ReplicationPeerDescription> peers =
       this.replicationPeerManager.listPeers(pattern);
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 2423c08..fbeecc6 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
@@ -1725,7 +1725,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
 
       status.setStatus("Writing region close event to WAL");
       // Always write close marker to wal even for read only table. This is not a big problem as we
-      // do not write any data into the region.
+      // do not write any data into the region; it is just a meta edit in the WAL file.
       if (!abort && wal != null && getRegionServerServices() != null &&
         RegionReplicaUtil.isDefaultReplica(getRegionInfo())) {
         writeRegionCloseMarker(wal);
@@ -2691,7 +2691,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
       }
     }
     MemStoreSize mss = this.memStoreSizing.getMemStoreSize();
-    LOG.info("Flushing " + storesToFlush.size() + "/" + stores.size() + " column families," +
+    LOG.info("Flushing " + this.getRegionInfo().getEncodedName() + " " +
+        storesToFlush.size() + "/" + stores.size() + " column families," +
         " dataSize=" + StringUtils.byteDesc(mss.getDataSize()) +
         " heapSize=" + StringUtils.byteDesc(mss.getHeapSize()) +
         ((perCfExtras != null && perCfExtras.length() > 0)? perCfExtras.toString(): "") +
@@ -4817,7 +4818,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
           for (Cell cell: val.getCells()) {
             // Check this edit is for me. Also, guard against writing the special
             // METACOLUMN info such as HBASE::CACHEFLUSH entries
-            if (CellUtil.matchingFamily(cell, WALEdit.METAFAMILY)) {
+            if (WALEdit.isMetaEditFamily(cell)) {
               // if region names don't match, skipp replaying compaction marker
               if (!checkRowWithinBoundary) {
                 //this is a special edit, we should handle it
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index 79985ec..109cc1a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -1926,7 +1926,7 @@ public class HRegionServer extends HasThread implements
       healthCheckChore = new HealthCheckChore(sleepTime, this, getConfiguration());
     }
 
-    this.walRoller = new LogRoller(this, this);
+    this.walRoller = new LogRoller(this);
     this.flushThroughputController = FlushThroughputControllerFactory.create(this, conf);
     this.procedureResultReporter = new RemoteProcedureResultReporter(this);
 
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/LogRoller.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/LogRoller.java
index 3f057d0..272925c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/LogRoller.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/LogRoller.java
@@ -1,4 +1,4 @@
-/**
+/*
  *
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
@@ -28,7 +28,6 @@ import java.util.Map.Entry;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL;
 import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException;
 import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
@@ -56,7 +55,6 @@ import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesti
 public class LogRoller extends HasThread implements Closeable {
   private static final Logger LOG = LoggerFactory.getLogger(LogRoller.class);
   private final ConcurrentMap<WAL, Boolean> walNeedsRoll = new ConcurrentHashMap<>();
-  private final Server server;
   protected final RegionServerServices services;
   private volatile long lastRollTime = System.currentTimeMillis();
   // Period to roll log.
@@ -99,16 +97,14 @@ public class LogRoller extends HasThread implements Closeable {
     }
   }
 
-  /** @param server */
-  public LogRoller(final Server server, final RegionServerServices services) {
+  public LogRoller(RegionServerServices services) {
     super("LogRoller");
-    this.server = server;
     this.services = services;
-    this.rollPeriod = this.server.getConfiguration().
+    this.rollPeriod = this.services.getConfiguration().
       getLong("hbase.regionserver.logroll.period", 3600000);
-    this.threadWakeFrequency = this.server.getConfiguration().
+    this.threadWakeFrequency = this.services.getConfiguration().
       getInt(HConstants.THREAD_WAKE_FREQUENCY, 10 * 1000);
-    this.checkLowReplicationInterval = this.server.getConfiguration().getLong(
+    this.checkLowReplicationInterval = this.services.getConfiguration().getLong(
         "hbase.regionserver.hlog.check.lowreplication.interval", 30 * 1000);
   }
 
@@ -144,7 +140,7 @@ public class LogRoller extends HasThread implements Closeable {
         LOG.warn("Failed to shutdown wal", e);
       }
     }
-    server.abort(reason, cause);
+    this.services.abort(reason, cause);
   }
 
   @Override
@@ -156,7 +152,7 @@ public class LogRoller extends HasThread implements Closeable {
       periodic = (now - this.lastRollTime) > this.rollPeriod;
       if (periodic) {
         // Time for periodic roll, fall through
-        LOG.debug("Wal roll period {} ms elapsed", this.rollPeriod);
+        LOG.debug("WAL roll period {} ms elapsed", this.rollPeriod);
       } else {
         synchronized (this) {
           if (walNeedsRoll.values().stream().anyMatch(Boolean::booleanValue)) {
@@ -183,9 +179,9 @@ public class LogRoller extends HasThread implements Closeable {
           WAL wal = entry.getKey();
           // reset the flag in front to avoid missing roll request before we return from rollWriter.
           walNeedsRoll.put(wal, Boolean.FALSE);
-            // Force the roll if the logroll.period is elapsed or if a roll was requested.
-            // The returned value is an array of actual region names.
-            byte[][]   regionsToFlush = wal.rollWriter(periodic || entry.getValue().booleanValue());
+          // Force the roll if the logroll.period is elapsed or if a roll was requested.
+          // The returned value is an array of actual region names.
+          byte[][] regionsToFlush = wal.rollWriter(periodic || entry.getValue().booleanValue());
           if (regionsToFlush != null) {
             for (byte[] r : regionsToFlush) {
               scheduleFlush(Bytes.toString(r));
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 c0ab9c8..71d6012 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
@@ -261,8 +261,8 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
   private static final class WalProps {
 
     /**
-     * Map the encoded region name to the highest sequence id. Contain all the regions it has
-     * entries of
+     * 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;
 
@@ -610,9 +610,9 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
   }
 
   /**
-   * If the number of un-archived WAL files is greater than maximum allowed, check the first
-   * (oldest) WAL file, and returns those regions which should be flushed so that it can be
-   * archived.
+   * If the number of un-archived WAL files ('live' WALs) is greater than maximum allowed,
+   * check the first (oldest) WAL, and return those regions which should be flushed so that
+   * it can be let-go/'archived'.
    * @return regions (encodedRegionNames) to flush in order to archive oldest WAL file.
    */
   byte[][] findRegionsToForceFlush() throws IOException {
@@ -888,10 +888,6 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
   /**
    * updates the sequence number of a specific store. depending on the flag: replaces current seq
    * number if the given seq id is bigger, or even if it is lower than existing one
-   * @param encodedRegionName
-   * @param familyName
-   * @param sequenceid
-   * @param onlyIfGreater
    */
   @Override
   public void updateStore(byte[] encodedRegionName, byte[] familyName, Long sequenceid,
@@ -1015,7 +1011,7 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
   }
 
   protected final long stampSequenceIdAndPublishToRingBuffer(RegionInfo hri, WALKeyImpl key,
-    WALEdit edits, boolean inMemstore, boolean closeRegion, RingBuffer<RingBufferTruck> ringBuffer)
+    WALEdit edits, boolean inMemstore, RingBuffer<RingBufferTruck> ringBuffer)
     throws IOException {
     if (this.closed) {
       throw new IOException(
@@ -1029,7 +1025,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, closeRegion, rpcCall);
+      FSWALEntry entry = new FSWALEntry(txid, key, edits, hri, inMemstore, rpcCall);
       entry.stampRegionSequenceId(we);
       ringBuffer.get(txid).load(entry);
     } finally {
@@ -1067,13 +1063,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, false);
+    return append(info, key, edits, true);
   }
 
   @Override
-  public long appendMarker(RegionInfo info, WALKeyImpl key, WALEdit edits, boolean closeRegion)
+  public long appendMarker(RegionInfo info, WALKeyImpl key, WALEdit edits)
     throws IOException {
-    return append(info, key, edits, false, closeRegion);
+    return append(info, key, edits, false);
   }
 
   /**
@@ -1097,17 +1093,17 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
    * @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.
-   * @param inMemstore Always true except for case where we are writing a region event marker, for
-   *          example, a compaction completion record into the WAL; in this case the entry is just
-   *          so we can finish an unfinished compaction -- it is not an edit for memstore.
-   * @param closeRegion Whether this is a region close marker, i.e, the last wal edit for this
-   *          region on this region server. The WAL implementation should remove all the related
-   *          stuff, for example, the sequence id accounting.
+   * @param inMemstore Always true except for case where we are writing a region event meta
+   *          marker edit, for example, a compaction completion record into the WAL or noting a
+   *          Region Open event. In these cases the entry is just so we can finish an unfinished
+   *          compaction after a crash when the new Server reads the WAL on recovery, etc. These
+   *          transition event 'Markers' do not go via the memstore. When memstore is false,
+   *          we presume a Marker event edit.
    * @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,
-    boolean closeRegion) throws IOException;
+  protected abstract long append(RegionInfo info, 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 b1adc29..9eab96f 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
@@ -564,9 +564,9 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
   }
 
   @Override
-  protected long append(RegionInfo hri, WALKeyImpl key, WALEdit edits, boolean inMemstore,
-    boolean closeRegion) throws IOException {
-    long txid = stampSequenceIdAndPublishToRingBuffer(hri, key, edits, inMemstore, closeRegion,
+  protected long append(RegionInfo hri, WALKeyImpl key, WALEdit edits, boolean inMemstore)
+      throws IOException {
+    long txid = stampSequenceIdAndPublishToRingBuffer(hri, 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 7eab65d..71ae251 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
@@ -436,8 +436,8 @@ public class FSHLog extends AbstractFSWAL<Writer> {
 
   @Override
   protected long append(final RegionInfo hri, final WALKeyImpl key, final WALEdit edits,
-    final boolean inMemstore, boolean closeRegion) throws IOException {
-    return stampSequenceIdAndPublishToRingBuffer(hri, key, edits, inMemstore, closeRegion,
+    final boolean inMemstore) throws IOException {
+    return stampSequenceIdAndPublishToRingBuffer(hri, key, edits, inMemstore,
       disruptor.getRingBuffer());
   }
 
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 c174650..44c96de 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
@@ -40,7 +40,7 @@ import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUti
 /**
  * A WAL Entry for {@link AbstractFSWAL} implementation.  Immutable.
  * A subclass of {@link Entry} that carries extra info across the ring buffer such as
- * region sequence id (we want to use this later, just before we write the WAL to ensure region
+ * region sequenceid (we want to use this later, just before we write the WAL to ensure region
  * edits maintain order).  The extra info added here is not 'serialized' as part of the WALEdit
  * hence marked 'transient' to underline this fact.  It also adds mechanism so we can wait on
  * the assign of the region sequence id.  See #stampRegionSequenceId().
@@ -50,17 +50,32 @@ class FSWALEntry extends Entry {
   // The below data members are denoted 'transient' just to highlight these are not persisted;
   // they are only in memory and held here while passing over the ring buffer.
   private final transient long txid;
+
+  /**
+   * If false, means this is a meta edit written by the hbase system itself. It was not in
+   * memstore. HBase uses these edit types to note in the log operational transitions such
+   * as compactions, flushes, or region open/closes.
+   */
   private final transient boolean inMemstore;
+
+  /**
+   * 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;
 
+  /**
+   * @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, boolean closeRegion, ServerCall<?> rpcCall) {
+    final boolean inMemstore, ServerCall<?> rpcCall) {
     super(key, edit);
     this.inMemstore = inMemstore;
-    this.closeRegion = closeRegion;
+    this.closeRegion = !inMemstore && edit.isRegionCloseMarker();
     this.regionInfo = regionInfo;
     this.txid = txid;
     if (inMemstore) {
@@ -68,7 +83,7 @@ class FSWALEntry extends Entry {
       Set<byte[]> families = edit.getFamilies();
       this.familyNames = families != null ? families : collectFamilies(edit.getCells());
     } else {
-      this.familyNames = Collections.<byte[]> emptySet();
+      this.familyNames = Collections.emptySet();
     }
     this.rpcCall = rpcCall;
     if (rpcCall != null) {
@@ -83,7 +98,7 @@ class FSWALEntry extends Entry {
     } else {
       Set<byte[]> set = new TreeSet<>(Bytes.BYTES_COMPARATOR);
       for (Cell cell: cells) {
-        if (!CellUtil.matchingFamily(cell, WALEdit.METAFAMILY)) {
+        if (!WALEdit.isMetaEditFamily(cell)) {
           set.add(CellUtil.cloneFamily(cell));
         }
       }
@@ -94,7 +109,7 @@ class FSWALEntry extends Entry {
   @Override
   public String toString() {
     return "sequence=" + this.txid + ", " + super.toString();
-  };
+  }
 
   boolean isInMemStore() {
     return this.inMemstore;
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SequenceIdAccounting.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SequenceIdAccounting.java
index 8e87b40..9b5a4d7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SequenceIdAccounting.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SequenceIdAccounting.java
@@ -38,13 +38,11 @@ import org.slf4j.LoggerFactory;
 import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 
 /**
- * <p>
- * Accounting of sequence ids per region and then by column family. So we can our accounting
+ * Accounting of sequence ids per region and then by column family. So we can keep our accounting
  * current, call startCacheFlush and then finishedCacheFlush or abortCacheFlush so this instance can
  * keep abreast of the state of sequence id persistence. Also call update per append.
- * </p>
  * <p>
- * For the implementation, we assume that all the {@code encodedRegionName} passed in is gotten by
+ * For the implementation, we assume that all the {@code encodedRegionName} passed in are gotten by
  * {@link org.apache.hadoop.hbase.client.RegionInfo#getEncodedNameAsBytes()}. So it is safe to use
  * it as a hash key. And for family name, we use {@link ImmutableByteArray} as key. This is because
  * hash based map is much faster than RBTree or CSLM and here we are on the critical write path. See
@@ -53,8 +51,8 @@ import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesti
  */
 @InterfaceAudience.Private
 class SequenceIdAccounting {
-
   private static final Logger LOG = LoggerFactory.getLogger(SequenceIdAccounting.class);
+
   /**
    * This lock ties all operations on {@link SequenceIdAccounting#flushingSequenceIds} and
    * {@link #lowestUnflushedSequenceIds} Maps. {@link #lowestUnflushedSequenceIds} has the
@@ -110,7 +108,6 @@ class SequenceIdAccounting {
 
   /**
    * Returns the lowest unflushed sequence id for the region.
-   * @param encodedRegionName
    * @return Lowest outstanding unflushed sequenceid for <code>encodedRegionName</code>. Will
    * return {@link HConstants#NO_SEQNUM} when none.
    */
@@ -125,8 +122,6 @@ class SequenceIdAccounting {
   }
 
   /**
-   * @param encodedRegionName
-   * @param familyName
    * @return Lowest outstanding unflushed sequenceid for <code>encodedRegionname</code> and
    *         <code>familyName</code>. Returned sequenceid may be for an edit currently being
    *         flushed.
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 cab8da0..b64e128 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
@@ -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
@@ -68,7 +67,7 @@ public class WALUtil {
     NavigableMap<byte[], Integer> replicationScope, RegionInfo hri, final CompactionDescriptor c,
     MultiVersionConcurrencyControl mvcc) throws IOException {
     WALKeyImpl walKey =
-      writeMarker(wal, replicationScope, hri, WALEdit.createCompaction(hri, c), false, mvcc, null);
+      writeMarker(wal, replicationScope, hri, WALEdit.createCompaction(hri, c), mvcc, null);
     if (LOG.isTraceEnabled()) {
       LOG.trace("Appended compaction marker " + TextFormat.shortDebugString(c));
     }
@@ -84,7 +83,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), false, mvcc, null, sync);
+      WALEdit.createFlushWALEdit(hri, f), mvcc, null, sync);
     if (LOG.isTraceEnabled()) {
       LOG.trace("Appended flush marker " + TextFormat.shortDebugString(f));
     }
@@ -96,11 +95,11 @@ public class WALUtil {
    * only. Not for external client consumption.
    */
   public static WALKeyImpl writeRegionEventMarker(WAL wal,
-    NavigableMap<byte[], Integer> replicationScope, RegionInfo hri, final RegionEventDescriptor r,
-    final MultiVersionConcurrencyControl mvcc) throws IOException {
-    WALKeyImpl walKey =
-      writeMarker(wal, replicationScope, hri, WALEdit.createRegionEventWALEdit(hri, r),
-        r.getEventType() == RegionEventDescriptor.EventType.REGION_CLOSE, mvcc, null);
+      NavigableMap<byte[], Integer> replicationScope, RegionInfo hri, RegionEventDescriptor r,
+      MultiVersionConcurrencyControl mvcc)
+    throws IOException {
+    WALKeyImpl walKey = writeMarker(wal, replicationScope, hri,
+        WALEdit.createRegionEventWALEdit(hri, r), mvcc, null);
     if (LOG.isTraceEnabled()) {
       LOG.trace("Appended region event marker " + TextFormat.shortDebugString(r));
     }
@@ -118,11 +117,11 @@ public class WALUtil {
    * @throws IOException We will throw an IOException if we can not append to the HLog.
    */
   public static WALKeyImpl writeBulkLoadMarkerAndSync(final WAL wal,
-    final NavigableMap<byte[], Integer> replicationScope, final RegionInfo hri,
-    final WALProtos.BulkLoadDescriptor desc, final MultiVersionConcurrencyControl mvcc)
+      final NavigableMap<byte[], Integer> replicationScope, final RegionInfo hri,
+      final WALProtos.BulkLoadDescriptor desc, final MultiVersionConcurrencyControl mvcc)
     throws IOException {
     WALKeyImpl walKey = writeMarker(wal, replicationScope, hri,
-      WALEdit.createBulkLoadEvent(hri, desc), false, mvcc, null);
+      WALEdit.createBulkLoadEvent(hri, desc), mvcc, null);
     if (LOG.isTraceEnabled()) {
       LOG.trace("Appended Bulk Load marker " + TextFormat.shortDebugString(desc));
     }
@@ -130,11 +129,11 @@ public class WALUtil {
   }
 
   private static WALKeyImpl writeMarker(final WAL wal,
-    final NavigableMap<byte[], Integer> replicationScope, final RegionInfo hri, final WALEdit edit,
-    boolean closeRegion, final MultiVersionConcurrencyControl mvcc,
-    final Map<String, byte[]> extendedAttributes) throws IOException {
+      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, closeRegion, mvcc,
+    return doFullMarkerAppendTransaction(wal, replicationScope, hri, edit, mvcc,
       extendedAttributes, true);
   }
 
@@ -146,16 +145,16 @@ public class WALUtil {
    * This write is for internal use only. Not for external client consumption.
    * @return WALKeyImpl that was added to the WAL.
    */
-  private static WALKeyImpl doFullMarkerAppendTransaction(final WAL wal,
-    final NavigableMap<byte[], Integer> replicationScope, final RegionInfo hri, final WALEdit edit,
-    boolean closeRegion, final MultiVersionConcurrencyControl mvcc,
-    final Map<String, byte[]> extendedAttributes, final boolean sync) throws IOException {
+  private static WALKeyImpl doFullMarkerAppendTransaction(WAL wal,
+      NavigableMap<byte[], Integer> replicationScope, RegionInfo hri, final WALEdit edit,
+      MultiVersionConcurrencyControl mvcc, Map<String, byte[]> extendedAttributes, boolean sync)
+    throws IOException {
     // TODO: Pass in current time to use?
     WALKeyImpl walKey = new WALKeyImpl(hri.getEncodedNameAsBytes(), hri.getTable(),
       System.currentTimeMillis(), mvcc, replicationScope, extendedAttributes);
     long trx = MultiVersionConcurrencyControl.NONE;
     try {
-      trx = wal.appendMarker(hri, walKey, edit, closeRegion);
+      trx = wal.appendMarker(hri, walKey, edit);
       if (sync) {
         wal.sync(trx);
       }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALActionListener.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALActionListener.java
index 27b25c4..d25ab07 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALActionListener.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALActionListener.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.replication.regionserver;
 import java.io.IOException;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
 import org.apache.hadoop.hbase.replication.ReplicationUtils;
 import org.apache.hadoop.hbase.wal.WALEdit;
@@ -74,7 +73,7 @@ class ReplicationSourceWALActionListener implements WALActionsListener {
     }
     // For replay, or if all the cells are markers, do not need to store replication scope.
     if (logEdit.isReplay() ||
-      logEdit.getCells().stream().allMatch(c -> CellUtil.matchingFamily(c, WALEdit.METAFAMILY))) {
+      logEdit.getCells().stream().allMatch(c -> WALEdit.isMetaEditFamily(c))) {
       ((WALKeyImpl) logKey).clearReplicationScope();
     }
   }
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 7e894ae..a60a4ce 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
@@ -162,17 +162,17 @@ class DisabledWALProvider implements WALProvider {
 
     @Override
     public long appendData(RegionInfo info, WALKeyImpl key, WALEdit edits) throws IOException {
-      return append(info, key, edits, true, false);
+      return append(info, key, edits, true);
     }
 
     @Override
-    public long appendMarker(RegionInfo info, WALKeyImpl key, WALEdit edits, boolean closeRegion)
+    public long appendMarker(RegionInfo info, WALKeyImpl key, WALEdit edits)
       throws IOException {
-      return append(info, key, edits, false, closeRegion);
+      return append(info, key, edits, false);
     }
 
-    private long append(RegionInfo info, WALKeyImpl key, WALEdit edits, boolean inMemstore,
-      boolean closeRegion) throws IOException {
+    private long append(RegionInfo info, WALKeyImpl key, WALEdit edits, boolean inMemstore)
+        throws IOException {
       WriteEntry writeEntry = key.getMvcc().begin();
       if (!edits.isReplay()) {
         for (Cell cell : edits.getCells()) {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/LogRecoveredEditsOutputSink.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/LogRecoveredEditsOutputSink.java
index aa649e4..9fc43b1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/LogRecoveredEditsOutputSink.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/LogRecoveredEditsOutputSink.java
@@ -362,7 +362,7 @@ public class LogRecoveredEditsOutputSink extends OutputSink {
     // We make the assumption that most cells will be kept.
     ArrayList<Cell> keptCells = new ArrayList<>(logEntry.getEdit().getCells().size());
     for (Cell cell : logEntry.getEdit().getCells()) {
-      if (CellUtil.matchingFamily(cell, WALEdit.METAFAMILY)) {
+      if (WALEdit.isMetaEditFamily(cell)) {
         keptCells.add(cell);
       } else {
         byte[] family = CellUtil.cloneFamily(cell);
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 64adbcc..7f7c412 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
  * distributed with this work for additional information
@@ -81,7 +81,7 @@ public interface WAL extends Closeable, WALFileLengthProvider {
    *         can clean logs. Returns null if nothing to flush. Names are actual
    *         region names as returned by {@link RegionInfo#getEncodedName()}
    */
-  byte[][] rollWriter(boolean force) throws FailedLogCloseException, IOException;
+  byte[][] rollWriter(boolean force) throws IOException;
 
   /**
    * Stop accepting new writes. If we have unsynced writes still in buffer, sync them.
@@ -99,7 +99,7 @@ public interface WAL extends Closeable, WALFileLengthProvider {
 
   /**
    * Append a set of data edits to the WAL. 'Data' here means that the content in the edits will
-   * also be added to memstore.
+   * also 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
@@ -110,13 +110,16 @@ public interface WAL extends Closeable, WALFileLengthProvider {
    *          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, boolean)
+   * @see #appendMarker(RegionInfo, WALKeyImpl, WALEdit)
    */
   long appendData(RegionInfo info, WALKeyImpl key, WALEdit edits) throws IOException;
 
   /**
-   * Append a marker edit to the WAL. A marker could be a FlushDescriptor, a compaction marker, or
-   * region event marker. The difference here is that, a marker will not be added to memstore.
+   * 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
+   * 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
@@ -125,15 +128,11 @@ public interface WAL extends Closeable, WALFileLengthProvider {
    * @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.
-   * @param closeRegion Whether this is a region close marker, i.e, the last wal edit for this
-   *          region on this region server. The WAL implementation should remove all the related
-   *          stuff, for example, the sequence id accounting.
    * @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, boolean closeRegion)
-    throws IOException;
+  long appendMarker(RegionInfo info, WALKeyImpl key, WALEdit edits) throws IOException;
 
   /**
    * updates the seuence number of a specific store.
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 2140002..cfa414d 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
@@ -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
@@ -37,8 +36,6 @@ 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.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor;
@@ -52,45 +49,103 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDe
  * single record, in PB format, followed (optionally) by Cells written via the WALCellEncoder.
  * <p>This class is LimitedPrivate for CPs to read-only. The {@link #add} methods are
  * classified as private methods, not for use by CPs.</p>
+ *
+ * <p>A particular WALEdit 'type' is the 'meta' type used to mark key operational
+ * events in the WAL such as compaction, flush, or region open. These meta types do not traverse
+ * hbase memstores. They are edits made by the hbase system rather than edit data submitted by
+ * clients. They only show in the WAL. These 'Meta' types have not been formally specified
+ * (or made into an explicit class type). They evolved organically. HBASE-8457 suggests codifying
+ * a WALEdit 'type' by adding a type field to WALEdit that gets serialized into the WAL. TODO.
+ * Would have to work on the consumption-side. Reading WALs on replay we seem to consume
+ * a Cell-at-a-time rather than by WALEdit. We are already in the below going out of our
+ * way to figure particular types --  e.g. if a compaction, replay, or close meta Marker -- during
+ * 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>
+ *
  * <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
  * column family name to-hand.. just shove it into the WALEdit if available. Doing this, we can
  * save on a parse of each Cell to figure column family down the line when we go to add the
  * WALEdit to the WAL file. See the hand-off in FSWALEntry Constructor.
+ * @see WALKey
  */
 // 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 {
-  private static final Logger LOG = LoggerFactory.getLogger(WALEdit.class);
-
-  // TODO: Get rid of this; see HBASE-8457
+  // Below defines are for writing WALEdit 'meta' Cells..
+  // TODO: Get rid of this system of special 'meta' Cells. See HBASE-8457. It suggests
+  // adding a type to WALEdit itself for use denoting meta Edits and their types.
   public static final byte [] METAFAMILY = Bytes.toBytes("METAFAMILY");
-  @VisibleForTesting
+
+  /**
+   * @deprecated Since 2.3.0. Not used.
+   */
+  @Deprecated
   public static final byte [] METAROW = Bytes.toBytes("METAROW");
+
+  /**
+   * @deprecated Since 2.3.0. Make it protected, internal-use only. Use
+   *   {@link #isCompactionMarker(Cell)}
+   */
+  @Deprecated
   @VisibleForTesting
   public static final byte[] COMPACTION = Bytes.toBytes("HBASE::COMPACTION");
+
+  /**
+   * @deprecated Since 2.3.0. Make it protected, internal-use only.
+   */
+  @Deprecated
   @VisibleForTesting
   public static final byte [] FLUSH = Bytes.toBytes("HBASE::FLUSH");
-  @VisibleForTesting
-  public static final byte [] REGION_EVENT = Bytes.toBytes("HBASE::REGION_EVENT");
+
+  /**
+   * Qualifier for region event meta 'Marker' WALEdits start with the
+   * {@link #REGION_EVENT_PREFIX} prefix ('HBASE::REGION_EVENT::'). After the prefix,
+   * we note the type of the event which we get from the RegionEventDescriptor protobuf
+   * instance type (A RegionEventDescriptor protobuf instance is written as the meta Marker
+   * Cell value). Adding a type suffix means we do not have to deserialize the protobuf to
+   * figure out what type of event this is.. .just read the qualifier suffix. For example,
+   * a close region event descriptor will have a qualifier of HBASE::REGION_EVENT::REGION_CLOSE.
+   * See WAL.proto and the EventType in RegionEventDescriptor protos for all possible
+   * event types.
+   */
+  private static final String REGION_EVENT_STR = "HBASE::REGION_EVENT";
+  private static final String REGION_EVENT_PREFIX_STR = REGION_EVENT_STR + "::";
+  private static final byte [] REGION_EVENT_PREFIX = Bytes.toBytes(REGION_EVENT_PREFIX_STR);
+
+  /**
+   * @deprecated Since 2.3.0. Remove. Not for external use. Not used.
+   */
+  @Deprecated
+  public static final byte [] REGION_EVENT = Bytes.toBytes(REGION_EVENT_STR);
+
+  /**
+   * We use this define figuring if we are carrying a close event.
+   */
+  private static final byte [] REGION_EVENT_CLOSE =
+      createRegionEventDescriptorQualifier(RegionEventDescriptor.EventType.REGION_CLOSE);
+
   @VisibleForTesting
   public static final byte [] BULK_LOAD = Bytes.toBytes("HBASE::BULK_LOAD");
 
-  private final boolean replay;
+  private final transient boolean replay;
 
-  private ArrayList<Cell> cells = null;
+  private ArrayList<Cell> cells;
 
   /**
    * All the Cell families in <code>cells</code>. Updated by {@link #add(Cell)} and
    * {@link #add(Map)}. This Set is passed to the FSWALEntry so it does not have
    * to recalculate the Set of families in a transaction; makes for a bunch of CPU savings.
-   * An optimization that saves on CPU-expensive Cell-parsing.
    */
   private Set<byte []> families = null;
 
   public WALEdit() {
-    this(false);
+    this(1, false);
   }
 
   /**
@@ -100,8 +155,8 @@ public class WALEdit implements HeapSize {
    * @see <a href="https://issues.apache.org/jira/browse/HBASE-20781">HBASE-20781</a>
    */
   @Deprecated
-  public WALEdit(boolean isReplay) {
-    this(1, isReplay);
+  public WALEdit(boolean replay) {
+    this(1, replay);
   }
 
   /**
@@ -125,7 +180,7 @@ public class WALEdit implements HeapSize {
 
   private Set<byte[]> getOrCreateFamilies() {
     if (this.families == null) {
-      this.families = new TreeSet<byte []>(Bytes.BYTES_COMPARATOR);
+      this.families = new TreeSet<>(Bytes.BYTES_COMPARATOR);
     }
     return this.families;
   }
@@ -140,22 +195,26 @@ public class WALEdit implements HeapSize {
 
   /**
    * @return True is <code>f</code> is {@link #METAFAMILY}
+   * @deprecated Since 2.3.0. Do not expose. Make protected.
    */
+  @Deprecated
   public static boolean isMetaEditFamily(final byte [] f) {
     return Bytes.equals(METAFAMILY, f);
   }
 
+  /**
+   * Replaying WALs can read Cell-at-a-time so need this method in those cases.
+   */
   public static boolean isMetaEditFamily(Cell cell) {
     return CellUtil.matchingFamily(cell, METAFAMILY);
   }
 
+  /**
+   * @return True if this is a meta edit; has one edit only and its columnfamily
+   *   is {@link #METAFAMILY}.
+   */
   public boolean isMetaEdit() {
-    for (Cell cell: cells) {
-      if (!isMetaEditFamily(cell)) {
-        return false;
-      }
-    }
-    return true;
+    return this.families != null && this.families.size() == 1 && this.families.contains(METAFAMILY);
   }
 
   /**
@@ -215,7 +274,7 @@ public class WALEdit implements HeapSize {
     cells.clear();
     cells.ensureCapacity(expectedCount);
     while (cells.size() < expectedCount && cellDecoder.advance()) {
-      cells.add(cellDecoder.current());
+      add(cellDecoder.current());
     }
     return cells.size();
   }
@@ -241,7 +300,7 @@ public class WALEdit implements HeapSize {
   public String toString() {
     StringBuilder sb = new StringBuilder();
 
-    sb.append("[#edits: " + cells.size() + " = <");
+    sb.append("[#edits: ").append(cells.size()).append(" = <");
     for (Cell cell : cells) {
       sb.append(cell);
       sb.append("; ");
@@ -257,30 +316,61 @@ public class WALEdit implements HeapSize {
   }
 
   public static FlushDescriptor getFlushDescriptor(Cell cell) throws IOException {
-    if (CellUtil.matchingColumn(cell, METAFAMILY, FLUSH)) {
-      return FlushDescriptor.parseFrom(CellUtil.cloneValue(cell));
-    }
-    return null;
+    return CellUtil.matchingColumn(cell, METAFAMILY, FLUSH)?
+        FlushDescriptor.parseFrom(CellUtil.cloneValue(cell)): null;
   }
 
+  /**
+   * @return A meta Marker WALEdit that has a single Cell whose value is the passed in
+   *   <code>regionEventDesc</code> serialized and whose row is this region,
+   *   columnfamily is {@link #METAFAMILY} and qualifier is
+   *   {@link #REGION_EVENT_PREFIX} + {@link RegionEventDescriptor#getEventType()};
+   *   for example HBASE::REGION_EVENT::REGION_CLOSE.
+   */
   public static WALEdit createRegionEventWALEdit(RegionInfo hri,
       RegionEventDescriptor regionEventDesc) {
-    KeyValue kv = new KeyValue(getRowForRegion(hri), METAFAMILY, REGION_EVENT,
-      EnvironmentEdgeManager.currentTime(), regionEventDesc.toByteArray());
+    return createRegionEventWALEdit(getRowForRegion(hri), regionEventDesc);
+  }
+
+  @VisibleForTesting
+  public static WALEdit createRegionEventWALEdit(byte [] rowForRegion,
+      RegionEventDescriptor regionEventDesc) {
+    KeyValue kv = new KeyValue(rowForRegion, METAFAMILY,
+        createRegionEventDescriptorQualifier(regionEventDesc.getEventType()),
+        EnvironmentEdgeManager.currentTime(), regionEventDesc.toByteArray());
     return new WALEdit().add(kv, METAFAMILY);
   }
 
+  /**
+   * @return Cell qualifier for the passed in RegionEventDescriptor Type; e.g. we'll
+   *   return something like a byte array with HBASE::REGION_EVENT::REGION_OPEN in it.
+   */
+  @VisibleForTesting
+  public static byte [] createRegionEventDescriptorQualifier(RegionEventDescriptor.EventType t) {
+    return Bytes.toBytes(REGION_EVENT_PREFIX_STR + t.toString());
+  }
+
+  /**
+   * Public so can be accessed from regionserver.wal package.
+   * @return True if this is a Marker Edit and it is a RegionClose type.
+   */
+  public boolean isRegionCloseMarker() {
+    return isMetaEdit() && PrivateCellUtil.matchingQualifier(this.cells.get(0),
+        REGION_EVENT_CLOSE, 0, REGION_EVENT_CLOSE.length);
+  }
+
+  /**
+   * @return Returns a RegionEventDescriptor made by deserializing the content of the
+   *   passed in <code>cell</code>, IFF the <code>cell</code> is a RegionEventDescriptor
+   *   type WALEdit.
+   */
   public static RegionEventDescriptor getRegionEventDescriptor(Cell cell) throws IOException {
-    if (CellUtil.matchingColumn(cell, METAFAMILY, REGION_EVENT)) {
-      return RegionEventDescriptor.parseFrom(CellUtil.cloneValue(cell));
-    }
-    return null;
+    return CellUtil.matchingColumnFamilyAndQualifierPrefix(cell, METAFAMILY, REGION_EVENT_PREFIX)?
+      RegionEventDescriptor.parseFrom(CellUtil.cloneValue(cell)): null;
   }
 
   /**
-   * Create a compaction WALEdit
-   * @param c
-   * @return A WALEdit that has <code>c</code> serialized as its value
+   * @return A Marker WALEdit that has <code>c</code> serialized as its value
    */
   public static WALEdit createCompaction(final RegionInfo hri, final CompactionDescriptor c) {
     byte [] pbbytes = c.toByteArray();
@@ -305,10 +395,7 @@ public class WALEdit implements HeapSize {
    * @return deserialized CompactionDescriptor or null.
    */
   public static CompactionDescriptor getCompaction(Cell kv) throws IOException {
-    if (isCompactionMarker(kv)) {
-      return CompactionDescriptor.parseFrom(CellUtil.cloneValue(kv));
-    }
-    return null;
+    return isCompactionMarker(kv)? CompactionDescriptor.parseFrom(CellUtil.cloneValue(kv)): null;
   }
 
   /**
@@ -328,12 +415,9 @@ public class WALEdit implements HeapSize {
    * @return The WALEdit for the BulkLoad
    */
   public static WALEdit createBulkLoadEvent(RegionInfo hri,
-                                            WALProtos.BulkLoadDescriptor bulkLoadDescriptor) {
-    KeyValue kv = new KeyValue(getRowForRegion(hri),
-        METAFAMILY,
-        BULK_LOAD,
-        EnvironmentEdgeManager.currentTime(),
-        bulkLoadDescriptor.toByteArray());
+      WALProtos.BulkLoadDescriptor bulkLoadDescriptor) {
+    KeyValue kv = new KeyValue(getRowForRegion(hri), METAFAMILY, BULK_LOAD,
+        EnvironmentEdgeManager.currentTime(), bulkLoadDescriptor.toByteArray());
     return new WALEdit().add(kv, METAFAMILY);
   }
 
@@ -343,17 +427,16 @@ public class WALEdit implements HeapSize {
    * @return deserialized BulkLoadDescriptor or null.
    */
   public static WALProtos.BulkLoadDescriptor getBulkLoadDescriptor(Cell cell) throws IOException {
-    if (CellUtil.matchingColumn(cell, METAFAMILY, BULK_LOAD)) {
-      return WALProtos.BulkLoadDescriptor.parseFrom(CellUtil.cloneValue(cell));
-    }
-    return null;
+    return CellUtil.matchingColumn(cell, METAFAMILY, BULK_LOAD)?
+      WALProtos.BulkLoadDescriptor.parseFrom(CellUtil.cloneValue(cell)):  null;
   }
 
   /**
    * Append the given map of family->edits to a WALEdit data structure.
    * This does not write to the WAL itself.
    * Note that as an optimization, we will stamp the Set of column families into the WALEdit
-   * to save on our having to calculate it subsequently way down in the actual WAL writing.
+   * to save on our having to calculate column families subsequently down in the actual WAL
+   * writing.
    *
    * @param familyMap map of family->edits
    */
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 23f6c29..0e3fac9 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
@@ -22,7 +22,6 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 import static org.mockito.ArgumentMatchers.any;
-import static org.mockito.ArgumentMatchers.anyBoolean;
 import static org.mockito.ArgumentMatchers.anyLong;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.verify;
@@ -113,20 +112,19 @@ public class TestBulkLoad {
     storeFileName = (new Path(storeFileName)).getName();
     List<String> storeFileNames = new ArrayList<>();
     storeFileNames.add(storeFileName);
-    when(log.appendMarker(any(), any(),
-            argThat(bulkLogWalEdit(WALEdit.BULK_LOAD, tableName.toBytes(),
-                    familyName, storeFileNames)),
-            anyBoolean())).thenAnswer(new Answer() {
-              @Override
-              public Object answer(InvocationOnMock invocation) {
-                WALKeyImpl walKey = invocation.getArgument(1);
-                MultiVersionConcurrencyControl mvcc = walKey.getMvcc();
-                if (mvcc != null) {
-                  MultiVersionConcurrencyControl.WriteEntry we = mvcc.begin();
-                  walKey.setWriteEntry(we);
-                }
-                return 01L;
-              };
+    when(log.appendMarker(any(), any(), argThat(
+        bulkLogWalEdit(WALEdit.BULK_LOAD, tableName.toBytes(), familyName, storeFileNames)))).
+        thenAnswer(new Answer() {
+          @Override
+          public Object answer(InvocationOnMock invocation) {
+            WALKeyImpl walKey = invocation.getArgument(1);
+            MultiVersionConcurrencyControl mvcc = walKey.getMvcc();
+            if (mvcc != null) {
+              MultiVersionConcurrencyControl.WriteEntry we = mvcc.begin();
+              walKey.setWriteEntry(we);
+            }
+            return 01L;
+          };
     });
     testRegionWithFamiliesAndSpecifiedTableName(tableName, family1)
         .bulkLoadHFiles(familyPaths, false, null);
@@ -141,8 +139,7 @@ public class TestBulkLoad {
   @Test
   public void shouldBulkLoadSingleFamilyHLog() throws IOException {
     when(log.appendMarker(any(),
-            any(), argThat(bulkLogWalEditType(WALEdit.BULK_LOAD)),
-            anyBoolean())).thenAnswer(new Answer() {
+            any(), argThat(bulkLogWalEditType(WALEdit.BULK_LOAD)))).thenAnswer(new Answer() {
               @Override
               public Object answer(InvocationOnMock invocation) {
                 WALKeyImpl walKey = invocation.getArgument(1);
@@ -161,8 +158,7 @@ public class TestBulkLoad {
   @Test
   public void shouldBulkLoadManyFamilyHLog() throws IOException {
     when(log.appendMarker(any(),
-            any(), argThat(bulkLogWalEditType(WALEdit.BULK_LOAD)),
-            anyBoolean())).thenAnswer(new Answer() {
+            any(), argThat(bulkLogWalEditType(WALEdit.BULK_LOAD)))).thenAnswer(new Answer() {
               @Override
               public Object answer(InvocationOnMock invocation) {
                 WALKeyImpl walKey = invocation.getArgument(1);
@@ -182,8 +178,7 @@ public class TestBulkLoad {
   @Test
   public void shouldBulkLoadManyFamilyHLogEvenWhenTableNameNamespaceSpecified() throws IOException {
     when(log.appendMarker(any(),
-            any(), argThat(bulkLogWalEditType(WALEdit.BULK_LOAD)),
-            anyBoolean())).thenAnswer(new Answer() {
+            any(), argThat(bulkLogWalEditType(WALEdit.BULK_LOAD)))).thenAnswer(new Answer() {
               @Override
               public Object answer(InvocationOnMock invocation) {
                 WALKeyImpl walKey = invocation.getArgument(1);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFailedAppendAndSync.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFailedAppendAndSync.java
index 0639493..4e6a1fe 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFailedAppendAndSync.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFailedAppendAndSync.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
@@ -31,7 +31,6 @@ import org.apache.hadoop.hbase.DroppedSnapshotException;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.Put;
@@ -162,18 +161,17 @@ public class TestFailedAppendAndSync {
       }
 
     // Make up mocked server and services.
-    Server server = mock(Server.class);
-    when(server.getConfiguration()).thenReturn(CONF);
-    when(server.isStopped()).thenReturn(false);
-    when(server.isAborted()).thenReturn(false);
     RegionServerServices services = mock(RegionServerServices.class);
+    when(services.getConfiguration()).thenReturn(CONF);
+    when(services.isStopped()).thenReturn(false);
+    when(services.isAborted()).thenReturn(false);
     // OK. Now I have my mocked up Server and RegionServerServices and my dodgy WAL, go ahead with
     // the test.
     FileSystem fs = FileSystem.get(CONF);
     Path rootDir = new Path(dir + getName());
     DodgyFSLog dodgyWAL = new DodgyFSLog(fs, rootDir, getName(), CONF);
     dodgyWAL.init();
-    LogRoller logRoller = new LogRoller(server, services);
+    LogRoller logRoller = new LogRoller(services);
     logRoller.addWAL(dodgyWAL);
     logRoller.start();
 
@@ -224,7 +222,7 @@ public class TestFailedAppendAndSync {
       // to just continue.
 
       // So, should be no abort at this stage. Verify.
-      Mockito.verify(server, Mockito.atLeast(0)).
+      Mockito.verify(services, Mockito.atLeast(0)).
         abort(Mockito.anyString(), (Throwable)Mockito.anyObject());
       try {
         dodgyWAL.throwAppendException = false;
@@ -240,7 +238,7 @@ public class TestFailedAppendAndSync {
       // happens. If it don't we'll timeout the whole test. That is fine.
       while (true) {
         try {
-          Mockito.verify(server, Mockito.atLeast(1)).
+          Mockito.verify(services, Mockito.atLeast(1)).
             abort(Mockito.anyString(), (Throwable)Mockito.anyObject());
           break;
         } catch (WantedButNotInvoked t) {
@@ -249,7 +247,7 @@ public class TestFailedAppendAndSync {
       }
     } finally {
       // To stop logRoller, its server has to say it is stopped.
-      Mockito.when(server.isStopped()).thenReturn(true);
+      Mockito.when(services.isStopped()).thenReturn(true);
       if (logRoller != null) logRoller.close();
       if (region != null) {
         try {
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 5d8660e..34d7ff6 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
@@ -29,7 +29,6 @@ import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 import static org.mockito.ArgumentMatchers.any;
-import static org.mockito.ArgumentMatchers.anyBoolean;
 import static org.mockito.ArgumentMatchers.anyLong;
 import static org.mockito.Mockito.doThrow;
 import static org.mockito.Mockito.mock;
@@ -5639,7 +5638,7 @@ public class TestHRegion {
       TEST_UTIL.getConfiguration(), rss, null);
 
     verify(wal, times(1)).appendMarker(any(RegionInfo.class), any(WALKeyImpl.class),
-      editCaptor.capture(), anyBoolean());
+      editCaptor.capture());
 
     WALEdit edit = editCaptor.getValue();
     assertNotNull(edit);
@@ -5721,16 +5720,16 @@ public class TestHRegion {
           return 1L;
         }
       });
-    when(wal.appendMarker(any(RegionInfo.class), any(WALKeyImpl.class), any(WALEdit.class),
-      anyBoolean())).thenAnswer(new Answer<Long>() {
-        @Override
-        public Long answer(InvocationOnMock invocation) throws Throwable {
-          WALKeyImpl key = invocation.getArgument(1);
-          MultiVersionConcurrencyControl.WriteEntry we = key.getMvcc().begin();
-          key.setWriteEntry(we);
-          return 1L;
-        }
-      });
+    when(wal.appendMarker(any(RegionInfo.class), any(WALKeyImpl.class), any(WALEdit.class))).
+        thenAnswer(new Answer<Long>() {
+          @Override
+          public Long answer(InvocationOnMock invocation) throws Throwable {
+            WALKeyImpl key = invocation.getArgument(1);
+            MultiVersionConcurrencyControl.WriteEntry we = key.getMvcc().begin();
+            key.setWriteEntry(we);
+            return 1L;
+          }
+        });
     return wal;
   }
 
@@ -5763,8 +5762,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(), anyBoolean());
+    verify(wal, times(2)).appendMarker(any(RegionInfo.class),
+        (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/TestRecoveredEdits.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRecoveredEdits.java
index 6212613..10b9d8f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRecoveredEdits.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRecoveredEdits.java
@@ -30,7 +30,6 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparatorImpl;
-import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
@@ -195,15 +194,18 @@ public class TestRecoveredEdits {
         WALEdit val = entry.getEdit();
         count++;
         // Check this edit is for this region.
-        if (!Bytes
-            .equals(key.getEncodedRegionName(), region.getRegionInfo().getEncodedNameAsBytes())) {
+        if (!Bytes.equals(key.getEncodedRegionName(),
+            region.getRegionInfo().getEncodedNameAsBytes())) {
           continue;
         }
         Cell previous = null;
         for (Cell cell : val.getCells()) {
-          if (CellUtil.matchingFamily(cell, WALEdit.METAFAMILY)) continue;
-          if (previous != null && CellComparatorImpl.COMPARATOR.compareRows(previous, cell) == 0)
+          if (WALEdit.isMetaEditFamily(cell)) {
             continue;
+          }
+          if (previous != null && CellComparatorImpl.COMPARATOR.compareRows(previous, cell) == 0) {
+            continue;
+          }
           previous = cell;
           walCells.add(cell);
         }
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 d9a76e8..a50ef78 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
@@ -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
@@ -204,11 +204,10 @@ public class TestWALLockup {
   @Test
   public void testLockupWhenSyncInMiddleOfZigZagSetup() throws IOException {
     // Mocked up server and regionserver services. Needed below.
-    Server server = Mockito.mock(Server.class);
-    Mockito.when(server.getConfiguration()).thenReturn(CONF);
-    Mockito.when(server.isStopped()).thenReturn(false);
-    Mockito.when(server.isAborted()).thenReturn(false);
     RegionServerServices services = Mockito.mock(RegionServerServices.class);
+    Mockito.when(services.getConfiguration()).thenReturn(CONF);
+    Mockito.when(services.isStopped()).thenReturn(false);
+    Mockito.when(services.isAborted()).thenReturn(false);
 
     // OK. Now I have my mocked up Server & RegionServerServices and dodgy WAL, go ahead with test.
     FileSystem fs = FileSystem.get(CONF);
@@ -217,7 +216,7 @@ public class TestWALLockup {
     dodgyWAL.init();
     Path originalWAL = dodgyWAL.getCurrentFileName();
     // I need a log roller running.
-    LogRoller logRoller = new LogRoller(server, services);
+    LogRoller logRoller = new LogRoller(services);
     logRoller.addWAL(dodgyWAL);
     // There is no 'stop' once a logRoller is running.. it just dies.
     logRoller.start();
@@ -294,7 +293,7 @@ public class TestWALLockup {
       }
     } finally {
       // To stop logRoller, its server has to say it is stopped.
-      Mockito.when(server.isStopped()).thenReturn(true);
+      Mockito.when(services.isStopped()).thenReturn(true);
       Closeables.close(logRoller, true);
       try {
         if (region != null) {
@@ -380,11 +379,10 @@ public class TestWALLockup {
     }
 
     // Mocked up server and regionserver services. Needed below.
-    final Server server = Mockito.mock(Server.class);
-    Mockito.when(server.getConfiguration()).thenReturn(CONF);
-    Mockito.when(server.isStopped()).thenReturn(false);
-    Mockito.when(server.isAborted()).thenReturn(false);
     RegionServerServices services = Mockito.mock(RegionServerServices.class);
+    Mockito.when(services.getConfiguration()).thenReturn(CONF);
+    Mockito.when(services.isStopped()).thenReturn(false);
+    Mockito.when(services.isAborted()).thenReturn(false);
 
     // OK. Now I have my mocked up Server & RegionServerServices and dodgy WAL, go ahead with test.
     FileSystem fs = FileSystem.get(CONF);
@@ -392,7 +390,7 @@ public class TestWALLockup {
     final DodgyFSLog dodgyWAL = new DodgyFSLog(fs, rootDir, getName(), CONF);
     dodgyWAL.init();
     // I need a log roller running.
-    LogRoller logRoller = new LogRoller(server, services);
+    LogRoller logRoller = new LogRoller(services);
     logRoller.addWAL(dodgyWAL);
     // There is no 'stop' once a logRoller is running.. it just dies.
     logRoller.start();
@@ -433,7 +431,7 @@ public class TestWALLockup {
 
     } finally {
       // To stop logRoller, its server has to say it is stopped.
-      Mockito.when(server.isStopped()).thenReturn(true);
+      Mockito.when(services.isStopped()).thenReturn(true);
       if (logRoller != null) {
         logRoller.close();
       }
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 987e1b9..513a15f 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
@@ -429,7 +429,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, false);
+        wal.append(info, logkey, edits, true);
         region.getMVCC().completeAndWait(logkey.getWriteEntry());
       }
       region.flush(true);
@@ -479,7 +479,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, false);
+      wal.append(ri, 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 704141c..d05fcce 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
@@ -1157,7 +1157,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, false, null);
+      createWALEdit(rowName, family, ee, index), hri, true, null);
     entry.stampRegionSequenceId(mvcc.begin());
     return entry;
   }
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 c43b918..ee02e6b 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
@@ -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,6 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.RegionInfo;
@@ -124,9 +123,8 @@ public class TestAsyncFSWAL extends AbstractTestFSWAL {
 
   @Test
   public void testBrokenWriter() throws Exception {
-    Server server = mock(Server.class);
-    when(server.getConfiguration()).thenReturn(CONF);
     RegionServerServices services = mock(RegionServerServices.class);
+    when(services.getConfiguration()).thenReturn(CONF);
     TableDescriptor td = TableDescriptorBuilder.newBuilder(TableName.valueOf("table"))
         .setColumnFamily(ColumnFamilyDescriptorBuilder.of("row")).build();
     RegionInfo ri = RegionInfoBuilder.newBuilder(td.getTableName()).build();
@@ -138,7 +136,7 @@ public class TestAsyncFSWAL extends AbstractTestFSWAL {
     long timestamp = System.currentTimeMillis();
     String testName = currentTest.getMethodName();
     AtomicInteger failedCount = new AtomicInteger(0);
-    try (LogRoller roller = new LogRoller(server, services);
+    try (LogRoller roller = new LogRoller(services);
         AsyncFSWAL wal = new AsyncFSWAL(FS, CommonFSUtils.getWALRootDir(CONF), DIR.toString(),
             testName, CONF, null, true, null, null, GROUP, CHANNEL_CLASS) {
 
@@ -196,7 +194,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, false);
+              wal.append(ri, 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/wal/FaultyFSLog.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/FaultyFSLog.java
index 01de1f4..f58d364 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
@@ -63,12 +63,12 @@ public class FaultyFSLog extends FSHLog {
   }
 
   @Override
-  protected long append(RegionInfo info, WALKeyImpl key, WALEdit edits, boolean inMemstore,
-    boolean closeRegion) throws IOException {
+  protected long append(RegionInfo info, WALKeyImpl key, WALEdit edits, boolean inMemstore)
+      throws IOException {
     if (this.ft == FailureType.APPEND) {
       throw new IOException("append");
     }
-    return super.append(info, key, edits, inMemstore, closeRegion);
+    return super.append(info, key, edits, inMemstore);
   }
 }
 
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 8865de1..cfeaac8 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
@@ -1376,12 +1376,10 @@ public class TestWALSplit {
         1,
         ServerName.parseServerName("ServerName:9099"), ImmutableMap.<byte[], List<Path>>of());
     final long time = EnvironmentEdgeManager.currentTime();
-    KeyValue kv = new KeyValue(Bytes.toBytes(region), WALEdit.METAFAMILY, WALEdit.REGION_EVENT,
-        time, regionOpenDesc.toByteArray());
     final WALKeyImpl walKey = new WALKeyImpl(Bytes.toBytes(region), TABLE_NAME, 1, time,
         HConstants.DEFAULT_CLUSTER_ID);
-    w.append(
-        new Entry(walKey, new WALEdit().add(kv)));
+    WALEdit we = WALEdit.createRegionEventWALEdit(Bytes.toBytes(region), regionOpenDesc);
+    w.append(new Entry(walKey, we));
     w.sync(false);
   }
 
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 7e6ed8f..3578cce 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
@@ -321,7 +321,7 @@ public final class WALPerformanceEvaluation extends Configured implements Tool {
       final HRegion[] regions = new HRegion[numRegions];
       final Runnable[] benchmarks = new Runnable[numRegions];
       final MockRegionServerServices mockServices = new MockRegionServerServices(getConf());
-      final LogRoller roller = new LogRoller(mockServices, mockServices);
+      final LogRoller roller = new LogRoller(mockServices);
       Threads.setDaemonThreadRunning(roller.getThread(), "WALPerfEval.logRoller");
 
       try {