HBASE-26131 Backport HBASE-25484 "Add trace support for WAL sync" to branch-2 (#3597)

8/17 commits of HBASE-22120, original commit 2be2c63f0d

Co-authored-by: Duo Zhang <zhangduo@apache.org>

Signed-off-by: Duo Zhang <zhangduo@apache.org>
Signed-off-by:  Peter Somogyi <psomogyi@apache.org>
This commit is contained in:
Tak Lon (Stephen) Wu 2021-08-19 09:15:05 -07:00
parent 321dcd4e77
commit 9cce94a2d0
8 changed files with 104 additions and 144 deletions

View File

@ -62,6 +62,8 @@ public final class TraceUtil {
public static final AttributeKey<Boolean> ROW_LOCK_READ_LOCK_KEY =
AttributeKey.booleanKey("db.hbase.rowlock.readlock");
public static final AttributeKey<String> WAL_IMPL = AttributeKey.stringKey("db.hbase.wal.impl");
private TraceUtil() {
}

View File

@ -26,7 +26,6 @@ import static org.apache.hbase.thirdparty.com.google.common.base.Preconditions.c
import com.lmax.disruptor.RingBuffer;
import io.opentelemetry.api.trace.Span;
import io.opentelemetry.context.Scope;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.io.InterruptedIOException;
@ -549,6 +548,35 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
return rollWriter(false);
}
@Override
public final void sync() throws IOException {
sync(useHsync);
}
@Override
public final void sync(long txid) throws IOException {
sync(txid, useHsync);
}
@Override
public final void sync(boolean forceSync) throws IOException {
TraceUtil.trace(() -> {
doSync(forceSync);
return null;
}, () -> createSpan("WAL.sync"));
}
@Override
public final void sync(long txid, boolean forceSync) throws IOException {
TraceUtil.trace(() -> {
doSync(txid, forceSync);
return null;
}, () -> createSpan("WAL.sync"));
}
protected abstract void doSync(boolean forceSync) throws IOException;
protected abstract void doSync(long txid, boolean forceSync) throws IOException;
/**
* This is a convenience method that computes a new filename with a given file-number.
* @param filenum to use
@ -650,7 +678,7 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
this.sequenceIdAccounting.findLower(firstWALEntry.getValue().encodedName2HighestSequenceId);
}
if (regions != null) {
List<String> listForPrint = new ArrayList();
List<String> listForPrint = new ArrayList<>();
for (Map.Entry<byte[], List<byte[]>> r : regions.entrySet()) {
StringBuilder families = new StringBuilder();
for (int i = 0; i < r.getValue().size(); i++) {
@ -768,6 +796,10 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
}
}
private Span createSpan(String name) {
return TraceUtil.createSpan(name).setAttribute(TraceUtil.WAL_IMPL, implClassName);
}
/**
* Cleans up current writer closing it and then puts in place the passed in {@code nextWriter}.
* <p/>
@ -785,13 +817,10 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
* @throws IOException if there is a problem flushing or closing the underlying FS
*/
Path replaceWriter(Path oldPath, Path newPath, W nextWriter) throws IOException {
Span span = TraceUtil.getGlobalTracer().spanBuilder("FSHFile.replaceWriter").startSpan();
try (Scope scope = span.makeCurrent()) {
return TraceUtil.trace(() -> {
doReplaceWriter(oldPath, newPath, nextWriter);
return newPath;
} finally {
span.end();
}
}, () -> createSpan("WAL.replaceWriter"));
}
protected final void blockOnSync(SyncFuture syncFuture) throws IOException {
@ -825,8 +854,7 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
return ioe;
}
@Override
public Map<byte[], List<byte[]>> rollWriter(boolean force) throws IOException {
private Map<byte[], List<byte[]>> rollWriterInternal(boolean force) throws IOException {
rollWriterLock.lock();
try {
// Return if nothing to flush.
@ -838,8 +866,7 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
LOG.debug("WAL closed. Skipping rolling of writer");
return regionsToFlush;
}
Span span = TraceUtil.getGlobalTracer().spanBuilder("FSHLog.rollWriter").startSpan();
try (Scope scope = span.makeCurrent()) {
try {
Path oldPath = getOldPath();
Path newPath = getNewPath();
// Any exception from here on is catastrophic, non-recoverable so we currently abort.
@ -864,10 +891,8 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
// If the underlying FileSystem can't do what we ask, treat as IO failure so
// we'll abort.
throw new IOException(
"Underlying FileSystem can't meet stream requirements. See RS log " + "for details.",
exception);
} finally {
span.end();
"Underlying FileSystem can't meet stream requirements. See RS log " + "for details.",
exception);
}
return regionsToFlush;
} finally {
@ -875,6 +900,11 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
}
}
@Override
public Map<byte[], List<byte[]>> rollWriter(boolean force) throws IOException {
return TraceUtil.trace(() -> rollWriterInternal(force), () -> createSpan("WAL.rollWriter"));
}
// public only until class moves to o.a.h.h.wal
/** @return the size of log files in use */
public long getLogFileSize() {
@ -1059,7 +1089,6 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
.append(TimeUnit.NANOSECONDS.toMillis(timeInNanos))
.append(" ms, current pipeline: ")
.append(Arrays.toString(getPipeline())).toString();
Span.current().addEvent(msg);
LOG.info(msg);
// A single sync took too long.
// Elsewhere in checkSlowSync, called from checkLogRoll, we will look at cumulative
@ -1082,8 +1111,7 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
}
protected final long stampSequenceIdAndPublishToRingBuffer(RegionInfo hri, WALKeyImpl key,
WALEdit edits, boolean inMemstore, RingBuffer<RingBufferTruck> ringBuffer)
throws IOException {
WALEdit edits, boolean inMemstore, RingBuffer<RingBufferTruck> ringBuffer) throws IOException {
if (this.closed) {
throw new IOException(
"Cannot append; log is closed, regionName = " + hri.getRegionNameAsString());
@ -1095,14 +1123,12 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
long txid = txidHolder.longValue();
ServerCall<?> rpcCall = RpcServer.getCurrentCall().filter(c -> c instanceof ServerCall)
.filter(c -> c.getCellScanner() != null).map(c -> (ServerCall) c).orElse(null);
Span span = TraceUtil.getGlobalTracer().spanBuilder(implClassName + ".append").startSpan();
try (Scope scope = span.makeCurrent()) {
try {
FSWALEntry entry = new FSWALEntry(txid, key, edits, hri, inMemstore, rpcCall);
entry.stampRegionSequenceId(we);
ringBuffer.get(txid).load(entry);
} finally {
ringBuffer.publish(txid);
span.end();
}
return txid;
}
@ -1145,13 +1171,14 @@ 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);
return TraceUtil.trace(() -> append(info, key, edits, true),
() -> createSpan("WAL.appendData"));
}
@Override
public long appendMarker(RegionInfo info, WALKeyImpl key, WALEdit edits)
throws IOException {
return append(info, key, edits, false);
public long appendMarker(RegionInfo info, WALKeyImpl key, WALEdit edits) throws IOException {
return TraceUtil.trace(() -> append(info, key, edits, false),
() -> createSpan("WAL.appendMarker"));
}
/**

View File

@ -23,8 +23,6 @@ import static org.apache.hadoop.hbase.util.FutureUtils.addListener;
import com.lmax.disruptor.RingBuffer;
import com.lmax.disruptor.Sequence;
import com.lmax.disruptor.Sequencer;
import io.opentelemetry.api.trace.Span;
import io.opentelemetry.context.Scope;
import java.io.IOException;
import java.lang.reflect.Field;
import java.util.ArrayDeque;
@ -52,7 +50,6 @@ import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.io.asyncfs.AsyncFSOutput;
import org.apache.hadoop.hbase.trace.TraceUtil;
import org.apache.hadoop.hbase.wal.AsyncFSWALProvider;
import org.apache.hadoop.hbase.wal.WALEdit;
import org.apache.hadoop.hbase.wal.WALKeyImpl;
@ -349,7 +346,7 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
break;
}
}
postSync(System.nanoTime() - startTimeNs, finishSync(true));
postSync(System.nanoTime() - startTimeNs, finishSync());
if (trySetReadyForRolling()) {
// we have just finished a roll, then do not need to check for log rolling, the writer will be
// closed soon.
@ -399,13 +396,7 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
}, consumeExecutor);
}
private void addTimeAnnotation(SyncFuture future, String annotation) {
Span.current().addEvent(annotation);
// TODO handle htrace API change, see HBASE-18895
// future.setSpan(scope.getSpan());
}
private int finishSyncLowerThanTxid(long txid, boolean addSyncTrace) {
private int finishSyncLowerThanTxid(long txid) {
int finished = 0;
for (Iterator<SyncFuture> iter = syncFutures.iterator(); iter.hasNext();) {
SyncFuture sync = iter.next();
@ -413,9 +404,6 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
markFutureDoneAndOffer(sync, txid, null);
iter.remove();
finished++;
if (addSyncTrace) {
addTimeAnnotation(sync, "writer synced");
}
} else {
break;
}
@ -424,7 +412,7 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
}
// try advancing the highestSyncedTxid as much as possible
private int finishSync(boolean addSyncTrace) {
private int finishSync() {
if (unackedAppends.isEmpty()) {
// All outstanding appends have been acked.
if (toWriteAppends.isEmpty()) {
@ -432,10 +420,7 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
long maxSyncTxid = highestSyncedTxid.get();
for (SyncFuture sync : syncFutures) {
maxSyncTxid = Math.max(maxSyncTxid, sync.getTxid());
markFutureDoneAndOffer(sync, maxSyncTxid, null);
if (addSyncTrace) {
addTimeAnnotation(sync, "writer synced");
}
sync.done(maxSyncTxid, null);
}
highestSyncedTxid.set(maxSyncTxid);
int finished = syncFutures.size();
@ -449,7 +434,7 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
assert lowestUnprocessedAppendTxid > highestProcessedAppendTxid;
long doneTxid = lowestUnprocessedAppendTxid - 1;
highestSyncedTxid.set(doneTxid);
return finishSyncLowerThanTxid(doneTxid, addSyncTrace);
return finishSyncLowerThanTxid(doneTxid);
}
} else {
// There are still unacked appends. So let's move the highestSyncedTxid to the txid of the
@ -457,7 +442,7 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
long lowestUnackedAppendTxid = unackedAppends.peek().getTxid();
long doneTxid = Math.max(lowestUnackedAppendTxid - 1, highestSyncedTxid.get());
highestSyncedTxid.set(doneTxid);
return finishSyncLowerThanTxid(doneTxid, addSyncTrace);
return finishSyncLowerThanTxid(doneTxid);
}
}
@ -465,7 +450,7 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
final AsyncWriter writer = this.writer;
// maybe a sync request is not queued when we issue a sync, so check here to see if we could
// finish some.
finishSync(false);
finishSync();
long newHighestProcessedAppendTxid = -1L;
for (Iterator<FSWALEntry> iter = toWriteAppends.iterator(); iter.hasNext();) {
FSWALEntry entry = iter.next();
@ -506,7 +491,7 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
// stamped some region sequence id.
if (unackedAppends.isEmpty()) {
highestSyncedTxid.set(highestProcessedAppendTxid);
finishSync(false);
finishSync();
trySetReadyForRolling();
}
return;
@ -612,61 +597,41 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
}
@Override
public void sync() throws IOException {
sync(useHsync);
}
@Override
public void sync(long txid) throws IOException {
sync(txid, useHsync);
}
@Override
public void sync(boolean forceSync) throws IOException {
Span span = TraceUtil.getGlobalTracer().spanBuilder("AsyncFSWAL.sync").startSpan();
try (Scope scope = span.makeCurrent()) {
long txid = waitingConsumePayloads.next();
SyncFuture future;
try {
future = getSyncFuture(txid, forceSync);
RingBufferTruck truck = waitingConsumePayloads.get(txid);
truck.load(future);
} finally {
waitingConsumePayloads.publish(txid);
}
if (shouldScheduleConsumer()) {
consumeExecutor.execute(consumer);
}
blockOnSync(future);
protected void doSync(boolean forceSync) throws IOException {
long txid = waitingConsumePayloads.next();
SyncFuture future;
try {
future = getSyncFuture(txid, forceSync);
RingBufferTruck truck = waitingConsumePayloads.get(txid);
truck.load(future);
} finally {
span.end();
waitingConsumePayloads.publish(txid);
}
if (shouldScheduleConsumer()) {
consumeExecutor.execute(consumer);
}
blockOnSync(future);
}
@Override
public void sync(long txid, boolean forceSync) throws IOException {
protected void doSync(long txid, boolean forceSync) throws IOException {
if (highestSyncedTxid.get() >= txid) {
return;
}
Span span = TraceUtil.getGlobalTracer().spanBuilder("AsyncFSWAL.sync").startSpan();
try (Scope scope = span.makeCurrent()) {
// here we do not use ring buffer sequence as txid
long sequence = waitingConsumePayloads.next();
SyncFuture future;
try {
future = getSyncFuture(txid, forceSync);
RingBufferTruck truck = waitingConsumePayloads.get(sequence);
truck.load(future);
} finally {
waitingConsumePayloads.publish(sequence);
}
if (shouldScheduleConsumer()) {
consumeExecutor.execute(consumer);
}
blockOnSync(future);
// here we do not use ring buffer sequence as txid
long sequence = waitingConsumePayloads.next();
SyncFuture future;
try {
future = getSyncFuture(txid, forceSync);
RingBufferTruck truck = waitingConsumePayloads.get(sequence);
truck.load(future);
} finally {
span.end();
waitingConsumePayloads.publish(sequence);
}
if (shouldScheduleConsumer()) {
consumeExecutor.execute(consumer);
}
blockOnSync(future);
}
@Override

View File

@ -30,7 +30,6 @@ import com.lmax.disruptor.TimeoutException;
import com.lmax.disruptor.dsl.Disruptor;
import com.lmax.disruptor.dsl.ProducerType;
import io.opentelemetry.api.trace.Span;
import io.opentelemetry.context.Scope;
import java.io.IOException;
import java.io.OutputStream;
import java.util.Arrays;
@ -49,7 +48,6 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.trace.TraceUtil;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ClassSize;
import org.apache.hadoop.hbase.util.CommonFSUtils;
@ -365,7 +363,6 @@ public class FSHLog extends AbstractFSWAL<Writer> {
// use assert to make sure no change breaks the logic that
// sequence and zigzagLatch will be set together
assert sequence > 0L : "Failed to get sequence from ring buffer";
Span.current().addEvent("awaiting safepoint");
syncFuture = zigzagLatch.waitSafePoint(publishSyncOnRingBuffer(sequence, false));
}
} catch (FailedSyncBeforeLogCloseException e) {
@ -646,15 +643,11 @@ public class FSHLog extends AbstractFSWAL<Writer> {
}
// I got something. Lets run. Save off current sequence number in case it changes
// while we run.
//TODO handle htrace API change, see HBASE-18895
//TraceScope scope = Trace.continueSpan(takeSyncFuture.getSpan());
long start = System.nanoTime();
Throwable lastException = null;
try {
Span.current().addEvent("syncing writer");
long unSyncedFlushSeq = highestUnsyncedTxid;
writer.sync(sf.isForceSync());
Span.current().addEvent("writer synced");
if (unSyncedFlushSeq > currentSequence) {
currentSequence = unSyncedFlushSeq;
}
@ -666,9 +659,6 @@ public class FSHLog extends AbstractFSWAL<Writer> {
LOG.warn("UNEXPECTED", e);
lastException = e;
} finally {
// reattach the span to the future before releasing.
//TODO handle htrace API change, see HBASE-18895
// takeSyncFuture.setSpan(scope.getSpan());
// First release what we 'took' from the queue.
syncCount += releaseSyncFuture(takeSyncFuture, currentSequence, lastException);
// Can we release other syncs?
@ -793,7 +783,7 @@ public class FSHLog extends AbstractFSWAL<Writer> {
}
// Sync all known transactions
private void publishSyncThenBlockOnCompletion(Scope scope, boolean forceSync) throws IOException {
private void publishSyncThenBlockOnCompletion(boolean forceSync) throws IOException {
SyncFuture syncFuture = publishSyncOnRingBuffer(forceSync);
blockOnSync(syncFuture);
}
@ -819,33 +809,17 @@ public class FSHLog extends AbstractFSWAL<Writer> {
}
@Override
public void sync() throws IOException {
sync(useHsync);
protected void doSync(boolean forceSync) throws IOException {
publishSyncThenBlockOnCompletion(forceSync);
}
@Override
public void sync(boolean forceSync) throws IOException {
Span span = TraceUtil.getGlobalTracer().spanBuilder("FSHLog.sync").startSpan();
try (Scope scope = span.makeCurrent()) {
publishSyncThenBlockOnCompletion(scope, forceSync);
}
}
@Override
public void sync(long txid) throws IOException {
sync(txid, useHsync);
}
@Override
public void sync(long txid, boolean forceSync) throws IOException {
protected void doSync(long txid, boolean forceSync) throws IOException {
if (this.highestSyncedTxid.get() >= txid) {
// Already sync'd.
return;
}
Span span = TraceUtil.getGlobalTracer().spanBuilder("FSHLog.sync").startSpan();
try (Scope scope = span.makeCurrent()) {
publishSyncThenBlockOnCompletion(scope, forceSync);
}
publishSyncThenBlockOnCompletion(forceSync);
}
boolean isLowReplicationRollEnabled() {
@ -1066,8 +1040,6 @@ public class FSHLog extends AbstractFSWAL<Writer> {
}
} else if (truck.type() == RingBufferTruck.Type.APPEND) {
FSWALEntry entry = truck.unloadAppend();
//TODO handle htrace API change, see HBASE-18895
//TraceScope scope = Trace.continueSpan(entry.detachSpan());
try {
if (this.exception != null) {
// Return to keep processing events coming off the ringbuffer

View File

@ -351,9 +351,9 @@ public class TestHRegion {
}
@Override
public void sync(long txid) throws IOException {
protected void doSync(long txid, boolean forceSync) throws IOException {
storeFlushCtx.prepare();
super.sync(txid);
super.doSync(txid, forceSync);
}
}

View File

@ -128,15 +128,15 @@ class CustomAsyncFSWAL extends AsyncFSWAL {
}
@Override
public void sync(boolean forceSync) throws IOException {
protected void doSync(boolean forceSync) throws IOException {
syncFlag = forceSync;
super.sync(forceSync);
super.doSync(forceSync);
}
@Override
public void sync(long txid, boolean forceSync) throws IOException {
protected void doSync(long txid, boolean forceSync) throws IOException {
syncFlag = forceSync;
super.sync(txid, forceSync);
super.doSync(txid, forceSync);
}
void resetSyncFlag() {

View File

@ -103,15 +103,15 @@ class CustomFSHLog extends FSHLog {
}
@Override
public void sync(boolean forceSync) throws IOException {
protected void doSync(boolean forceSync) throws IOException {
syncFlag = forceSync;
super.sync(forceSync);
super.doSync(forceSync);
}
@Override
public void sync(long txid, boolean forceSync) throws IOException {
protected void doSync(long txid, boolean forceSync) throws IOException {
syncFlag = forceSync;
super.sync(txid, forceSync);
super.doSync(txid, forceSync);
}
void resetSyncFlag() {

View File

@ -20,7 +20,6 @@
package org.apache.hadoop.hbase.wal;
import java.io.IOException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@ -50,16 +49,11 @@ public class FaultyFSLog extends FSHLog {
}
@Override
public void sync(long txid) throws IOException {
sync(txid, false);
}
@Override
public void sync(long txid, boolean forceSync) throws IOException {
protected void doSync(long txid, boolean forceSync) throws IOException {
if (this.ft == FailureType.SYNC) {
throw new IOException("sync");
}
super.sync(txid, forceSync);
super.doSync(txid, forceSync);
}
@Override