HBASE-25484 Add trace support for WAL sync (#2892)
Signed-off-by: Guanghao Zhang <zghao@apache.org>
This commit is contained in:
parent
03e12bfa4a
commit
2be2c63f0d
hbase-common/src/main/java/org/apache/hadoop/hbase/trace
hbase-server/src
main/java/org/apache/hadoop/hbase/regionserver/wal
test/java/org/apache/hadoop/hbase
|
@ -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() {
|
||||
}
|
||||
|
||||
|
|
|
@ -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;
|
||||
|
@ -571,6 +570,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
|
||||
|
@ -672,7 +700,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++) {
|
||||
|
@ -815,6 +843,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/>
|
||||
|
@ -832,13 +864,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 {
|
||||
|
@ -876,8 +905,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 {
|
||||
if (this.closed) {
|
||||
|
@ -888,8 +916,7 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
|
|||
return null;
|
||||
}
|
||||
Map<byte[], List<byte[]>> regionsToFlush = null;
|
||||
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.
|
||||
|
@ -914,10 +941,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 {
|
||||
|
@ -925,6 +950,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() {
|
||||
|
@ -1099,7 +1129,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);
|
||||
if (timeInNanos > this.rollOnSyncNs) {
|
||||
// A single sync took too long.
|
||||
|
@ -1122,8 +1151,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());
|
||||
|
@ -1135,14 +1163,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;
|
||||
}
|
||||
|
@ -1176,13 +1202,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"));
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -24,8 +24,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;
|
||||
|
@ -53,7 +51,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;
|
||||
|
@ -345,7 +342,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.
|
||||
|
@ -394,13 +391,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();
|
||||
|
@ -408,9 +399,6 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
|
|||
sync.done(txid, null);
|
||||
iter.remove();
|
||||
finished++;
|
||||
if (addSyncTrace) {
|
||||
addTimeAnnotation(sync, "writer synced");
|
||||
}
|
||||
} else {
|
||||
break;
|
||||
}
|
||||
|
@ -419,7 +407,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()) {
|
||||
|
@ -428,9 +416,6 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
|
|||
for (SyncFuture sync : syncFutures) {
|
||||
maxSyncTxid = Math.max(maxSyncTxid, sync.getTxid());
|
||||
sync.done(maxSyncTxid, null);
|
||||
if (addSyncTrace) {
|
||||
addTimeAnnotation(sync, "writer synced");
|
||||
}
|
||||
}
|
||||
highestSyncedTxid.set(maxSyncTxid);
|
||||
int finished = syncFutures.size();
|
||||
|
@ -444,7 +429,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
|
||||
|
@ -452,7 +437,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);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -460,7 +445,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();
|
||||
|
@ -501,7 +486,7 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
|
|||
// stamped some region sequence id.
|
||||
if (unackedAppends.isEmpty()) {
|
||||
highestSyncedTxid.set(highestProcessedAppendTxid);
|
||||
finishSync(false);
|
||||
finishSync();
|
||||
trySetReadyForRolling();
|
||||
}
|
||||
return;
|
||||
|
@ -648,12 +633,12 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
|
|||
|
||||
@Override
|
||||
protected long append(RegionInfo hri, WALKeyImpl key, WALEdit edits, boolean inMemstore)
|
||||
throws IOException {
|
||||
if (markerEditOnly() && !edits.isMetaEdit()) {
|
||||
throw new IOException("WAL is closing, only marker edit is allowed");
|
||||
}
|
||||
long txid = stampSequenceIdAndPublishToRingBuffer(hri, key, edits, inMemstore,
|
||||
waitingConsumePayloads);
|
||||
throws IOException {
|
||||
if (markerEditOnly() && !edits.isMetaEdit()) {
|
||||
throw new IOException("WAL is closing, only marker edit is allowed");
|
||||
}
|
||||
long txid =
|
||||
stampSequenceIdAndPublishToRingBuffer(hri, key, edits, inMemstore, waitingConsumePayloads);
|
||||
if (shouldScheduleConsumer()) {
|
||||
consumeExecutor.execute(consumer);
|
||||
}
|
||||
|
@ -661,61 +646,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);
|
||||
}
|
||||
|
||||
protected final AsyncWriter createAsyncWriter(FileSystem fs, Path path) throws IOException {
|
||||
|
|
|
@ -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) {
|
||||
|
@ -643,15 +640,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;
|
||||
}
|
||||
|
@ -663,9 +656,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?
|
||||
|
@ -794,7 +784,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);
|
||||
}
|
||||
|
@ -820,33 +810,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() {
|
||||
|
@ -1048,8 +1022,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
|
||||
|
|
|
@ -346,9 +346,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);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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() {
|
||||
|
|
|
@ -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() {
|
||||
|
|
|
@ -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
|
||||
|
|
Loading…
Reference in New Issue