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 = public static final AttributeKey<Boolean> ROW_LOCK_READ_LOCK_KEY =
AttributeKey.booleanKey("db.hbase.rowlock.readlock"); AttributeKey.booleanKey("db.hbase.rowlock.readlock");
public static final AttributeKey<String> WAL_IMPL = AttributeKey.stringKey("db.hbase.wal.impl");
private TraceUtil() { 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 com.lmax.disruptor.RingBuffer;
import io.opentelemetry.api.trace.Span; import io.opentelemetry.api.trace.Span;
import io.opentelemetry.context.Scope;
import java.io.FileNotFoundException; import java.io.FileNotFoundException;
import java.io.IOException; import java.io.IOException;
import java.io.InterruptedIOException; import java.io.InterruptedIOException;
@ -549,6 +548,35 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
return rollWriter(false); 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. * This is a convenience method that computes a new filename with a given file-number.
* @param filenum to use * @param filenum to use
@ -650,7 +678,7 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
this.sequenceIdAccounting.findLower(firstWALEntry.getValue().encodedName2HighestSequenceId); this.sequenceIdAccounting.findLower(firstWALEntry.getValue().encodedName2HighestSequenceId);
} }
if (regions != null) { if (regions != null) {
List<String> listForPrint = new ArrayList(); List<String> listForPrint = new ArrayList<>();
for (Map.Entry<byte[], List<byte[]>> r : regions.entrySet()) { for (Map.Entry<byte[], List<byte[]>> r : regions.entrySet()) {
StringBuilder families = new StringBuilder(); StringBuilder families = new StringBuilder();
for (int i = 0; i < r.getValue().size(); i++) { 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}. * Cleans up current writer closing it and then puts in place the passed in {@code nextWriter}.
* <p/> * <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 * @throws IOException if there is a problem flushing or closing the underlying FS
*/ */
Path replaceWriter(Path oldPath, Path newPath, W nextWriter) throws IOException { Path replaceWriter(Path oldPath, Path newPath, W nextWriter) throws IOException {
Span span = TraceUtil.getGlobalTracer().spanBuilder("FSHFile.replaceWriter").startSpan(); return TraceUtil.trace(() -> {
try (Scope scope = span.makeCurrent()) {
doReplaceWriter(oldPath, newPath, nextWriter); doReplaceWriter(oldPath, newPath, nextWriter);
return newPath; return newPath;
} finally { }, () -> createSpan("WAL.replaceWriter"));
span.end();
}
} }
protected final void blockOnSync(SyncFuture syncFuture) throws IOException { protected final void blockOnSync(SyncFuture syncFuture) throws IOException {
@ -825,8 +854,7 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
return ioe; return ioe;
} }
@Override private Map<byte[], List<byte[]>> rollWriterInternal(boolean force) throws IOException {
public Map<byte[], List<byte[]>> rollWriter(boolean force) throws IOException {
rollWriterLock.lock(); rollWriterLock.lock();
try { try {
// Return if nothing to flush. // 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"); LOG.debug("WAL closed. Skipping rolling of writer");
return regionsToFlush; return regionsToFlush;
} }
Span span = TraceUtil.getGlobalTracer().spanBuilder("FSHLog.rollWriter").startSpan(); try {
try (Scope scope = span.makeCurrent()) {
Path oldPath = getOldPath(); Path oldPath = getOldPath();
Path newPath = getNewPath(); Path newPath = getNewPath();
// Any exception from here on is catastrophic, non-recoverable so we currently abort. // Any exception from here on is catastrophic, non-recoverable so we currently abort.
@ -866,8 +893,6 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
throw new IOException( throw new IOException(
"Underlying FileSystem can't meet stream requirements. See RS log " + "for details.", "Underlying FileSystem can't meet stream requirements. See RS log " + "for details.",
exception); exception);
} finally {
span.end();
} }
return regionsToFlush; return regionsToFlush;
} finally { } 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 // public only until class moves to o.a.h.h.wal
/** @return the size of log files in use */ /** @return the size of log files in use */
public long getLogFileSize() { public long getLogFileSize() {
@ -1059,7 +1089,6 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
.append(TimeUnit.NANOSECONDS.toMillis(timeInNanos)) .append(TimeUnit.NANOSECONDS.toMillis(timeInNanos))
.append(" ms, current pipeline: ") .append(" ms, current pipeline: ")
.append(Arrays.toString(getPipeline())).toString(); .append(Arrays.toString(getPipeline())).toString();
Span.current().addEvent(msg);
LOG.info(msg); LOG.info(msg);
// A single sync took too long. // A single sync took too long.
// Elsewhere in checkSlowSync, called from checkLogRoll, we will look at cumulative // 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, protected final long stampSequenceIdAndPublishToRingBuffer(RegionInfo hri, WALKeyImpl key,
WALEdit edits, boolean inMemstore, RingBuffer<RingBufferTruck> ringBuffer) WALEdit edits, boolean inMemstore, RingBuffer<RingBufferTruck> ringBuffer) throws IOException {
throws IOException {
if (this.closed) { if (this.closed) {
throw new IOException( throw new IOException(
"Cannot append; log is closed, regionName = " + hri.getRegionNameAsString()); "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(); long txid = txidHolder.longValue();
ServerCall<?> rpcCall = RpcServer.getCurrentCall().filter(c -> c instanceof ServerCall) ServerCall<?> rpcCall = RpcServer.getCurrentCall().filter(c -> c instanceof ServerCall)
.filter(c -> c.getCellScanner() != null).map(c -> (ServerCall) c).orElse(null); .filter(c -> c.getCellScanner() != null).map(c -> (ServerCall) c).orElse(null);
Span span = TraceUtil.getGlobalTracer().spanBuilder(implClassName + ".append").startSpan(); try {
try (Scope scope = span.makeCurrent()) {
FSWALEntry entry = new FSWALEntry(txid, key, edits, hri, inMemstore, rpcCall); FSWALEntry entry = new FSWALEntry(txid, key, edits, hri, inMemstore, rpcCall);
entry.stampRegionSequenceId(we); entry.stampRegionSequenceId(we);
ringBuffer.get(txid).load(entry); ringBuffer.get(txid).load(entry);
} finally { } finally {
ringBuffer.publish(txid); ringBuffer.publish(txid);
span.end();
} }
return txid; return txid;
} }
@ -1145,13 +1171,14 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
@Override @Override
public long appendData(RegionInfo info, WALKeyImpl key, WALEdit edits) throws IOException { 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 @Override
public long appendMarker(RegionInfo info, WALKeyImpl key, WALEdit edits) public long appendMarker(RegionInfo info, WALKeyImpl key, WALEdit edits) throws IOException {
throws IOException { return TraceUtil.trace(() -> append(info, key, edits, false),
return 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.RingBuffer;
import com.lmax.disruptor.Sequence; import com.lmax.disruptor.Sequence;
import com.lmax.disruptor.Sequencer; import com.lmax.disruptor.Sequencer;
import io.opentelemetry.api.trace.Span;
import io.opentelemetry.context.Scope;
import java.io.IOException; import java.io.IOException;
import java.lang.reflect.Field; import java.lang.reflect.Field;
import java.util.ArrayDeque; 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.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.io.asyncfs.AsyncFSOutput; 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.AsyncFSWALProvider;
import org.apache.hadoop.hbase.wal.WALEdit; import org.apache.hadoop.hbase.wal.WALEdit;
import org.apache.hadoop.hbase.wal.WALKeyImpl; import org.apache.hadoop.hbase.wal.WALKeyImpl;
@ -349,7 +346,7 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
break; break;
} }
} }
postSync(System.nanoTime() - startTimeNs, finishSync(true)); postSync(System.nanoTime() - startTimeNs, finishSync());
if (trySetReadyForRolling()) { if (trySetReadyForRolling()) {
// we have just finished a roll, then do not need to check for log rolling, the writer will be // we have just finished a roll, then do not need to check for log rolling, the writer will be
// closed soon. // closed soon.
@ -399,13 +396,7 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
}, consumeExecutor); }, consumeExecutor);
} }
private void addTimeAnnotation(SyncFuture future, String annotation) { private int finishSyncLowerThanTxid(long txid) {
Span.current().addEvent(annotation);
// TODO handle htrace API change, see HBASE-18895
// future.setSpan(scope.getSpan());
}
private int finishSyncLowerThanTxid(long txid, boolean addSyncTrace) {
int finished = 0; int finished = 0;
for (Iterator<SyncFuture> iter = syncFutures.iterator(); iter.hasNext();) { for (Iterator<SyncFuture> iter = syncFutures.iterator(); iter.hasNext();) {
SyncFuture sync = iter.next(); SyncFuture sync = iter.next();
@ -413,9 +404,6 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
markFutureDoneAndOffer(sync, txid, null); markFutureDoneAndOffer(sync, txid, null);
iter.remove(); iter.remove();
finished++; finished++;
if (addSyncTrace) {
addTimeAnnotation(sync, "writer synced");
}
} else { } else {
break; break;
} }
@ -424,7 +412,7 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
} }
// try advancing the highestSyncedTxid as much as possible // try advancing the highestSyncedTxid as much as possible
private int finishSync(boolean addSyncTrace) { private int finishSync() {
if (unackedAppends.isEmpty()) { if (unackedAppends.isEmpty()) {
// All outstanding appends have been acked. // All outstanding appends have been acked.
if (toWriteAppends.isEmpty()) { if (toWriteAppends.isEmpty()) {
@ -432,10 +420,7 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
long maxSyncTxid = highestSyncedTxid.get(); long maxSyncTxid = highestSyncedTxid.get();
for (SyncFuture sync : syncFutures) { for (SyncFuture sync : syncFutures) {
maxSyncTxid = Math.max(maxSyncTxid, sync.getTxid()); maxSyncTxid = Math.max(maxSyncTxid, sync.getTxid());
markFutureDoneAndOffer(sync, maxSyncTxid, null); sync.done(maxSyncTxid, null);
if (addSyncTrace) {
addTimeAnnotation(sync, "writer synced");
}
} }
highestSyncedTxid.set(maxSyncTxid); highestSyncedTxid.set(maxSyncTxid);
int finished = syncFutures.size(); int finished = syncFutures.size();
@ -449,7 +434,7 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
assert lowestUnprocessedAppendTxid > highestProcessedAppendTxid; assert lowestUnprocessedAppendTxid > highestProcessedAppendTxid;
long doneTxid = lowestUnprocessedAppendTxid - 1; long doneTxid = lowestUnprocessedAppendTxid - 1;
highestSyncedTxid.set(doneTxid); highestSyncedTxid.set(doneTxid);
return finishSyncLowerThanTxid(doneTxid, addSyncTrace); return finishSyncLowerThanTxid(doneTxid);
} }
} else { } else {
// There are still unacked appends. So let's move the highestSyncedTxid to the txid of the // 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 lowestUnackedAppendTxid = unackedAppends.peek().getTxid();
long doneTxid = Math.max(lowestUnackedAppendTxid - 1, highestSyncedTxid.get()); long doneTxid = Math.max(lowestUnackedAppendTxid - 1, highestSyncedTxid.get());
highestSyncedTxid.set(doneTxid); 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; 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 // maybe a sync request is not queued when we issue a sync, so check here to see if we could
// finish some. // finish some.
finishSync(false); finishSync();
long newHighestProcessedAppendTxid = -1L; long newHighestProcessedAppendTxid = -1L;
for (Iterator<FSWALEntry> iter = toWriteAppends.iterator(); iter.hasNext();) { for (Iterator<FSWALEntry> iter = toWriteAppends.iterator(); iter.hasNext();) {
FSWALEntry entry = iter.next(); FSWALEntry entry = iter.next();
@ -506,7 +491,7 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
// stamped some region sequence id. // stamped some region sequence id.
if (unackedAppends.isEmpty()) { if (unackedAppends.isEmpty()) {
highestSyncedTxid.set(highestProcessedAppendTxid); highestSyncedTxid.set(highestProcessedAppendTxid);
finishSync(false); finishSync();
trySetReadyForRolling(); trySetReadyForRolling();
} }
return; return;
@ -612,19 +597,7 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
} }
@Override @Override
public void sync() throws IOException { protected void doSync(boolean forceSync) 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(); long txid = waitingConsumePayloads.next();
SyncFuture future; SyncFuture future;
try { try {
@ -638,18 +611,13 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
consumeExecutor.execute(consumer); consumeExecutor.execute(consumer);
} }
blockOnSync(future); blockOnSync(future);
} finally {
span.end();
}
} }
@Override @Override
public void sync(long txid, boolean forceSync) throws IOException { protected void doSync(long txid, boolean forceSync) throws IOException {
if (highestSyncedTxid.get() >= txid) { if (highestSyncedTxid.get() >= txid) {
return; return;
} }
Span span = TraceUtil.getGlobalTracer().spanBuilder("AsyncFSWAL.sync").startSpan();
try (Scope scope = span.makeCurrent()) {
// here we do not use ring buffer sequence as txid // here we do not use ring buffer sequence as txid
long sequence = waitingConsumePayloads.next(); long sequence = waitingConsumePayloads.next();
SyncFuture future; SyncFuture future;
@ -664,9 +632,6 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
consumeExecutor.execute(consumer); consumeExecutor.execute(consumer);
} }
blockOnSync(future); blockOnSync(future);
} finally {
span.end();
}
} }
@Override @Override

View File

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

View File

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

View File

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

View File

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

View File

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