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 <zhangduo@apache.org>
Signed-off-by: Lijin Bin <binlijin@apache.org>
This commit is contained in:
Michael Stack 2019-10-30 22:02:49 -07:00 committed by stack
parent 9ab0489eab
commit 471538ca9e
28 changed files with 323 additions and 246 deletions

View File

@ -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);
}
/**

View File

@ -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

View File

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

View File

@ -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

View File

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

View File

@ -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)) {

View File

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

View File

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

View File

@ -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());
}

View File

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

View File

@ -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.

View File

@ -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));
}
@ -122,7 +121,7 @@ public class WALUtil {
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);
}

View File

@ -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();
}
}

View File

@ -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()) {

View File

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

View File

@ -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.

View File

@ -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,
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);
}
public static RegionEventDescriptor getRegionEventDescriptor(Cell cell) throws IOException {
if (CellUtil.matchingColumn(cell, METAFAMILY, REGION_EVENT)) {
return RegionEventDescriptor.parseFrom(CellUtil.cloneValue(cell));
}
return null;
/**
* @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());
}
/**
* Create a compaction WALEdit
* @param c
* @return A WALEdit that has <code>c</code> serialized as its value
* 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 {
return CellUtil.matchingColumnFamilyAndQualifierPrefix(cell, METAFAMILY, REGION_EVENT_PREFIX)?
RegionEventDescriptor.parseFrom(CellUtil.cloneValue(cell)): null;
}
/**
* @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;
}
/**
@ -329,11 +416,8 @@ public class WALEdit implements HeapSize {
*/
public static WALEdit createBulkLoadEvent(RegionInfo hri,
WALProtos.BulkLoadDescriptor bulkLoadDescriptor) {
KeyValue kv = new KeyValue(getRowForRegion(hri),
METAFAMILY,
BULK_LOAD,
EnvironmentEdgeManager.currentTime(),
bulkLoadDescriptor.toByteArray());
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
*/

View File

@ -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,10 +112,9 @@ 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() {
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);
@ -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);

View File

@ -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 {

View File

@ -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,8 +5720,8 @@ public class TestHRegion {
return 1L;
}
});
when(wal.appendMarker(any(RegionInfo.class), any(WALKeyImpl.class), any(WALEdit.class),
anyBoolean())).thenAnswer(new Answer<Long>() {
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);
@ -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);

View File

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

View File

@ -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();
}

View File

@ -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

View File

@ -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;
}

View File

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

View File

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

View File

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

View File

@ -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 {