qual
+ */
+ 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 52518877c790..58e84489fa52 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 void map(WALKey key, WALEdit value, Context context)
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 8722b4f6538c..9e999d6a0c87 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
@@ -3485,7 +3485,7 @@ public Listkey 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 0307a882cac5..68f2db5c05f4 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
@@ -558,9 +558,9 @@ private boolean shouldScheduleConsumer() {
}
@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 810267113076..78259d1ba1a5 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
@@ -431,8 +431,8 @@ protected void doShutdown() throws IOException {
@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 c174650194ee..44c96dee7619 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 @@
/**
* 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- * 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. - *
*
- * 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 @@
*/
@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 This class is LimitedPrivate for CPs to read-only. The {@link #add} methods are
* classified as private methods, not for use by CPs. 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. 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 ArrayListencodedRegionName. Will
* return {@link HConstants#NO_SEQNUM} when none.
*/
@@ -125,8 +122,6 @@ long getLowestSequenceId(final byte[] encodedRegionName) {
}
/**
- * @param encodedRegionName
- * @param familyName
* @return Lowest outstanding unflushed sequenceid for encodedRegionname and
* familyName. 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 e8c7404d606f..09e1a39cb8d0 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
@@ -66,7 +65,7 @@ public static WALKeyImpl writeCompactionMarker(WAL wal,
NavigableMapkey 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.
*
* 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 key 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 214000200cba..cfa414d3f05a 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.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 @@
* single record, in PB format, followed (optionally) by Cells written via the WALCellEncoder.
* cells. 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 Setf 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 int readFromCells(Codec.Decoder cellDecoder, int expectedCount) throws IO
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 long estimatedSerializedSizeOf() {
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 static WALEdit createFlushWALEdit(RegionInfo hri, FlushDescriptor f) {
}
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
+ * regionEventDesc 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 cell, IFF the cell 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 c serialized as its value
+ * @return A Marker WALEdit that has c serialized as its value
*/
public static WALEdit createCompaction(final RegionInfo hri, final CompactionDescriptor c) {
byte [] pbbytes = c.toByteArray();
@@ -305,10 +395,7 @@ public static byte[] getRowForRegion(RegionInfo hri) {
* @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 static boolean isCompactionMarker(Cell cell) {
* @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 static WALEdit createBulkLoadEvent(RegionInfo hri,
* @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 23f6c29fddc5..0e3fac9a79ce 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.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 void verifyBulkLoadEvent() throws IOException {
storeFileName = (new Path(storeFileName)).getName();
List