Skip to content

Commit 26b9e76

Browse files
committed
HBASE-22301 Consider rolling the WAL if the HDFS write pipeline is slow
1 parent 36b4c0f commit 26b9e76

File tree

9 files changed

+417
-52
lines changed

9 files changed

+417
-52
lines changed

hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/wal/MetricsWALSource.java

Lines changed: 17 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -58,10 +58,19 @@ public interface MetricsWALSource extends BaseSource {
5858
String SYNC_TIME = "syncTime";
5959
String SYNC_TIME_DESC = "The time it took to sync the WAL to HDFS.";
6060
String ROLL_REQUESTED = "rollRequest";
61-
String ROLL_REQUESTED_DESC = "How many times a log roll has been requested total";
61+
String ROLL_REQUESTED_DESC = "How many times a roll has been requested total";
62+
String ERROR_ROLL_REQUESTED = "errorRollRequest";
63+
String ERROR_ROLL_REQUESTED_DESC =
64+
"How many times a roll was requested due to I/O or other errors.";
6265
String LOW_REPLICA_ROLL_REQUESTED = "lowReplicaRollRequest";
6366
String LOW_REPLICA_ROLL_REQUESTED_DESC =
64-
"How many times a log roll was requested due to too few DN's in the write pipeline.";
67+
"How many times a roll was requested due to too few datanodes in the write pipeline.";
68+
String SLOW_SYNC_ROLL_REQUESTED = "slowSyncRollRequest";
69+
String SLOW_SYNC_ROLL_REQUESTED_DESC =
70+
"How many times a roll was requested due to sync too slow on the write pipeline.";
71+
String SIZE_ROLL_REQUESTED = "sizeRollRequest";
72+
String SIZE_ROLL_REQUESTED_DESC =
73+
"How many times a roll was requested due to file size roll threshold.";
6574
String WRITTEN_BYTES = "writtenBytes";
6675
String WRITTEN_BYTES_DESC = "Size (in bytes) of the data written to the WAL.";
6776

@@ -92,8 +101,14 @@ public interface MetricsWALSource extends BaseSource {
92101

93102
void incrementLogRollRequested();
94103

104+
void incrementErrorLogRoll();
105+
95106
void incrementLowReplicationLogRoll();
96107

108+
void incrementSlowSyncLogRoll();
109+
110+
void incrementSizeLogRoll();
111+
97112
void incrementWrittenBytes(long val);
98113

99114
long getWrittenBytes();

hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/wal/MetricsWALSourceImpl.java

Lines changed: 27 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -39,7 +39,10 @@ public class MetricsWALSourceImpl extends BaseSourceImpl implements MetricsWALSo
3939
private final MutableFastCounter appendCount;
4040
private final MutableFastCounter slowAppendCount;
4141
private final MutableFastCounter logRollRequested;
42-
private final MutableFastCounter lowReplicationLogRollRequested;
42+
private final MutableFastCounter errorRollRequested;
43+
private final MutableFastCounter lowReplicationRollRequested;
44+
private final MutableFastCounter slowSyncRollRequested;
45+
private final MutableFastCounter sizeRollRequested;
4346
private final MutableFastCounter writtenBytes;
4447

4548
public MetricsWALSourceImpl() {
@@ -61,8 +64,14 @@ public MetricsWALSourceImpl(String metricsName,
6164
syncTimeHisto = this.getMetricsRegistry().newTimeHistogram(SYNC_TIME, SYNC_TIME_DESC);
6265
logRollRequested =
6366
this.getMetricsRegistry().newCounter(ROLL_REQUESTED, ROLL_REQUESTED_DESC, 0L);
64-
lowReplicationLogRollRequested = this.getMetricsRegistry()
67+
errorRollRequested = this.getMetricsRegistry()
68+
.newCounter(ERROR_ROLL_REQUESTED, ERROR_ROLL_REQUESTED_DESC, 0L);
69+
lowReplicationRollRequested = this.getMetricsRegistry()
6570
.newCounter(LOW_REPLICA_ROLL_REQUESTED, LOW_REPLICA_ROLL_REQUESTED_DESC, 0L);
71+
slowSyncRollRequested = this.getMetricsRegistry()
72+
.newCounter(SLOW_SYNC_ROLL_REQUESTED, SLOW_SYNC_ROLL_REQUESTED_DESC, 0L);
73+
sizeRollRequested = this.getMetricsRegistry()
74+
.newCounter(SIZE_ROLL_REQUESTED, SIZE_ROLL_REQUESTED_DESC, 0L);
6675
writtenBytes = this.getMetricsRegistry().newCounter(WRITTEN_BYTES, WRITTEN_BYTES_DESC, 0l);
6776
}
6877

@@ -96,9 +105,24 @@ public void incrementLogRollRequested() {
96105
logRollRequested.incr();
97106
}
98107

108+
@Override
109+
public void incrementErrorLogRoll() {
110+
errorRollRequested.incr();
111+
}
112+
99113
@Override
100114
public void incrementLowReplicationLogRoll() {
101-
lowReplicationLogRollRequested.incr();
115+
lowReplicationRollRequested.incr();
116+
}
117+
118+
@Override
119+
public void incrementSlowSyncLogRoll() {
120+
slowSyncRollRequested.incr();
121+
}
122+
123+
@Override
124+
public void incrementSizeLogRoll() {
125+
sizeRollRequested.incr();
102126
}
103127

104128
@Override

hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/LogRoller.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -71,7 +71,7 @@ public void addWAL(final WAL wal) {
7171
if (null == walNeedsRoll.putIfAbsent(wal, Boolean.FALSE)) {
7272
wal.registerWALActionsListener(new WALActionsListener.Base() {
7373
@Override
74-
public void logRollRequested(boolean lowReplicas) {
74+
public void logRollRequested(WALActionsListener.RollRequestReason reason) {
7575
walNeedsRoll.put(wal, Boolean.TRUE);
7676
// TODO logs will contend with each other here, replace with e.g. DelayedQueue
7777
synchronized(rollLog) {

hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java

Lines changed: 137 additions & 25 deletions
Original file line numberDiff line numberDiff line change
@@ -17,6 +17,10 @@
1717
*/
1818
package org.apache.hadoop.hbase.regionserver.wal;
1919

20+
import static org.apache.hadoop.hbase.regionserver.wal.WALActionsListener.RollRequestReason.ERROR;
21+
import static org.apache.hadoop.hbase.regionserver.wal.WALActionsListener.RollRequestReason.LOW_REPLICATION;
22+
import static org.apache.hadoop.hbase.regionserver.wal.WALActionsListener.RollRequestReason.SIZE;
23+
import static org.apache.hadoop.hbase.regionserver.wal.WALActionsListener.RollRequestReason.SLOW_SYNC;
2024
import static org.apache.hadoop.hbase.wal.DefaultWALProvider.WAL_FILE_NAME_DELIMITER;
2125

2226
import java.io.FileNotFoundException;
@@ -160,9 +164,18 @@ public class FSHLog implements WAL {
160164

161165
private static final Log LOG = LogFactory.getLog(FSHLog.class);
162166

163-
private static final int DEFAULT_SLOW_SYNC_TIME_MS = 100; // in ms
167+
static final String SLOW_SYNC_TIME_MS ="hbase.regionserver.wal.slowsync.ms";
168+
static final int DEFAULT_SLOW_SYNC_TIME_MS = 100; // in ms
169+
static final String ROLL_ON_SYNC_TIME_MS = "hbase.regionserver.wal.roll.on.sync.ms";
170+
static final int DEFAULT_ROLL_ON_SYNC_TIME_MS = 10000; // in ms
171+
static final String SLOW_SYNC_ROLL_THRESHOLD = "hbase.regionserver.wal.slowsync.roll.threshold";
172+
static final int DEFAULT_SLOW_SYNC_ROLL_THRESHOLD = 100; // 100 slow sync warnings
173+
static final String SLOW_SYNC_ROLL_INTERVAL_MS =
174+
"hbase.regionserver.wal.slowsync.roll.interval.ms";
175+
static final int DEFAULT_SLOW_SYNC_ROLL_INTERVAL_MS = 60 * 1000; // in ms, 1 minute
164176

165-
private static final int DEFAULT_WAL_SYNC_TIMEOUT_MS = 5 * 60 * 1000; // in ms, 5min
177+
static final String WAL_SYNC_TIMEOUT_MS = "hbase.regionserver.wal.sync.timeout";
178+
static final int DEFAULT_WAL_SYNC_TIMEOUT_MS = 5 * 60 * 1000; // in ms, 5min
166179

167180
/**
168181
* The nexus at which all incoming handlers meet. Does appends and sync with an ordering.
@@ -280,7 +293,10 @@ public WALCoprocessorHost getCoprocessorHost() {
280293

281294
private final boolean useHsync;
282295

283-
private final int slowSyncNs;
296+
private final long slowSyncNs, rollOnSyncNs;
297+
private final int slowSyncRollThreshold;
298+
private final int slowSyncCheckInterval;
299+
private final AtomicInteger slowSyncCount = new AtomicInteger();
284300

285301
private final long walSyncTimeout;
286302

@@ -350,9 +366,13 @@ public WALCoprocessorHost getCoprocessorHost() {
350366

351367
private final AtomicInteger closeErrorCount = new AtomicInteger();
352368

369+
protected volatile boolean rollRequested;
370+
353371
// Last time to check low replication on hlog's pipeline
354372
private volatile long lastTimeCheckLowReplication = EnvironmentEdgeManager.currentTime();
355373

374+
// Last time we asked to roll the log due to a slow sync
375+
private volatile long lastTimeCheckSlowSync = EnvironmentEdgeManager.currentTime();
356376

357377
/**
358378
* WAL Comparator; it compares the timestamp (log filenum), present in the log file name.
@@ -540,11 +560,16 @@ public boolean accept(final Path fileName) {
540560
// rollWriter sets this.hdfs_out if it can.
541561
rollWriter();
542562

543-
this.slowSyncNs =
544-
1000000 * conf.getInt("hbase.regionserver.hlog.slowsync.ms",
545-
DEFAULT_SLOW_SYNC_TIME_MS);
546-
this.walSyncTimeout = conf.getLong("hbase.regionserver.hlog.sync.timeout",
547-
DEFAULT_WAL_SYNC_TIMEOUT_MS);
563+
this.slowSyncNs = TimeUnit.MILLISECONDS.toNanos(conf.getInt(SLOW_SYNC_TIME_MS,
564+
conf.getInt("hbase.regionserver.hlog.slowsync.ms", DEFAULT_SLOW_SYNC_TIME_MS)));
565+
this.rollOnSyncNs = TimeUnit.MILLISECONDS.toNanos(conf.getInt(ROLL_ON_SYNC_TIME_MS,
566+
DEFAULT_ROLL_ON_SYNC_TIME_MS));
567+
this.slowSyncRollThreshold = conf.getInt(SLOW_SYNC_ROLL_THRESHOLD,
568+
DEFAULT_SLOW_SYNC_ROLL_THRESHOLD);
569+
this.slowSyncCheckInterval = conf.getInt(SLOW_SYNC_ROLL_INTERVAL_MS,
570+
DEFAULT_SLOW_SYNC_ROLL_INTERVAL_MS);
571+
this.walSyncTimeout = conf.getLong(WAL_SYNC_TIMEOUT_MS,
572+
conf.getLong("hbase.regionserver.hlog.sync.timeout", DEFAULT_WAL_SYNC_TIMEOUT_MS));
548573

549574
// This is the 'writer' -- a single threaded executor. This single thread 'consumes' what is
550575
// put on the ring buffer.
@@ -720,6 +745,11 @@ private void preemptiveSync(final ProtobufLogWriter nextWriter) {
720745
// NewPath could be equal to oldPath if replaceWriter fails.
721746
newPath = replaceWriter(oldPath, newPath, nextWriter, nextHdfsOut);
722747
tellListenersAboutPostLogRoll(oldPath, newPath);
748+
// Reset rollRequested status
749+
rollRequested = false;
750+
// We got a new writer, so reset the slow sync count
751+
lastTimeCheckSlowSync = EnvironmentEdgeManager.currentTime();
752+
slowSyncCount.set(0);
723753
// Can we delete any of the old log files?
724754
if (getNumRolledLogFiles() > 0) {
725755
cleanOldLogs();
@@ -1303,8 +1333,11 @@ public void run() {
13031333
syncCount += releaseSyncFuture(takeSyncFuture, currentSequence, lastException);
13041334
// Can we release other syncs?
13051335
syncCount += releaseSyncFutures(currentSequence, lastException);
1306-
if (lastException != null) requestLogRoll();
1307-
else checkLogRoll();
1336+
if (lastException != null) {
1337+
requestLogRoll();
1338+
} else {
1339+
checkLogRoll();
1340+
}
13081341
}
13091342
postSync(System.nanoTime() - start, syncCount);
13101343
} catch (InterruptedException e) {
@@ -1321,24 +1354,37 @@ public void run() {
13211354
* Schedule a log roll if needed.
13221355
*/
13231356
public void checkLogRoll() {
1357+
// If we have already requested a roll, do nothing
1358+
if (isLogRollRequested()) {
1359+
return;
1360+
}
13241361
// Will return immediately if we are in the middle of a WAL log roll currently.
1325-
if (!rollWriterLock.tryLock()) return;
1326-
boolean lowReplication;
1327-
try {
1328-
lowReplication = checkLowReplication();
1329-
} finally {
1330-
rollWriterLock.unlock();
1362+
if (!rollWriterLock.tryLock()) {
1363+
return;
13311364
}
13321365
try {
1333-
if (lowReplication || (writer != null && writer.getLength() > logrollsize)) {
1334-
requestLogRoll(lowReplication);
1366+
if (checkLowReplication()) {
1367+
LOG.warn("Requesting log roll because of low replication, current pipeline: " +
1368+
Arrays.toString(getPipeLine()));
1369+
requestLogRoll(LOW_REPLICATION);
1370+
} else if (writer != null && writer.getLength() > logrollsize) {
1371+
if (LOG.isDebugEnabled()) {
1372+
LOG.debug("Requesting log roll because of file size threshold; length=" +
1373+
writer.getLength() + ", logrollsize=" + logrollsize);
1374+
}
1375+
requestLogRoll(SIZE);
1376+
} else if (checkSlowSync()) {
1377+
// We log this already in checkSlowSync
1378+
requestLogRoll(SLOW_SYNC);
13351379
}
13361380
} catch (IOException e) {
13371381
LOG.warn("Writer.getLength() failed; continuing", e);
1382+
} finally {
1383+
rollWriterLock.unlock();
13381384
}
13391385
}
13401386

1341-
/*
1387+
/**
13421388
* @return true if number of replicas for the WAL is lower than threshold
13431389
*/
13441390
private boolean checkLowReplication() {
@@ -1389,6 +1435,41 @@ private boolean checkLowReplication() {
13891435
return logRollNeeded;
13901436
}
13911437

1438+
/**
1439+
* @return true if we exceeded the slow sync roll threshold over the last check
1440+
* interval
1441+
*/
1442+
private boolean checkSlowSync() {
1443+
boolean result = false;
1444+
long now = EnvironmentEdgeManager.currentTime();
1445+
long elapsedTime = now - lastTimeCheckSlowSync;
1446+
if (elapsedTime >= slowSyncCheckInterval) {
1447+
if (slowSyncCount.get() >= slowSyncRollThreshold) {
1448+
if (elapsedTime >= (2 * slowSyncCheckInterval)) {
1449+
// If two or more slowSyncCheckInterval have elapsed this is a corner case
1450+
// where a train of slow syncs almost triggered us but then there was a long
1451+
// interval from then until the one more that pushed us over. If so, we
1452+
// should do nothing and let the count reset.
1453+
if (LOG.isDebugEnabled()) {
1454+
LOG.debug("checkSlowSync triggered but we decided to ignore it; " +
1455+
"count=" + slowSyncCount.get() + ", threshold=" + slowSyncRollThreshold +
1456+
", elapsedTime=" + elapsedTime + " ms, slowSyncCheckInterval=" +
1457+
slowSyncCheckInterval + " ms");
1458+
}
1459+
// Fall through to count reset below
1460+
} else {
1461+
LOG.warn("Requesting log roll because we exceeded slow sync threshold; count=" +
1462+
slowSyncCount.get() + ", threshold=" + slowSyncRollThreshold +
1463+
", current pipeline: " + Arrays.toString(getPipeLine()));
1464+
result = true;
1465+
}
1466+
}
1467+
lastTimeCheckSlowSync = now;
1468+
slowSyncCount.set(0);
1469+
}
1470+
return result;
1471+
}
1472+
13921473
private SyncFuture publishSyncOnRingBuffer(long sequence) {
13931474
return publishSyncOnRingBuffer(sequence, null, false);
13941475
}
@@ -1452,10 +1533,23 @@ private void postSync(final long timeInNanos, final int handlerSyncs) {
14521533
if (timeInNanos > this.slowSyncNs) {
14531534
String msg =
14541535
new StringBuilder().append("Slow sync cost: ")
1455-
.append(timeInNanos / 1000000).append(" ms, current pipeline: ")
1536+
.append(TimeUnit.NANOSECONDS.toMillis(timeInNanos))
1537+
.append(" ms, current pipeline: ")
14561538
.append(Arrays.toString(getPipeLine())).toString();
14571539
Trace.addTimelineAnnotation(msg);
14581540
LOG.info(msg);
1541+
// A single sync took too long.
1542+
// Elsewhere in checkSlowSync, called from checkLogRoll, we will look at cumulative
1543+
// effects. Here we have a single data point that indicates we should take immediate
1544+
// action, so do so.
1545+
if (timeInNanos > this.rollOnSyncNs) {
1546+
LOG.warn("Requesting log roll because we exceeded slow sync threshold; time=" +
1547+
TimeUnit.NANOSECONDS.toMillis(timeInNanos) + " ms, threshold=" +
1548+
TimeUnit.NANOSECONDS.toMillis(rollOnSyncNs) + " ms, current pipeline: " +
1549+
Arrays.toString(getPipeLine()));
1550+
requestLogRoll(SLOW_SYNC);
1551+
}
1552+
slowSyncCount.incrementAndGet(); // it's fine to unconditionally increment this
14591553
}
14601554
if (!listeners.isEmpty()) {
14611555
for (WALActionsListener listener : listeners) {
@@ -1539,15 +1633,24 @@ public void sync(long txid, boolean forceSync) throws IOException {
15391633
}
15401634
}
15411635

1636+
protected boolean isLogRollRequested() {
1637+
return rollRequested;
1638+
}
1639+
15421640
// public only until class moves to o.a.h.h.wal
15431641
public void requestLogRoll() {
1544-
requestLogRoll(false);
1642+
requestLogRoll(ERROR);
15451643
}
15461644

1547-
private void requestLogRoll(boolean tooFewReplicas) {
1645+
private void requestLogRoll(final WALActionsListener.RollRequestReason reason) {
1646+
// If we have already requested a roll, don't do it again
1647+
if (rollRequested) {
1648+
return;
1649+
}
15481650
if (!this.listeners.isEmpty()) {
1651+
rollRequested = true; // No point to assert this unless there is a registered listener
15491652
for (WALActionsListener i: this.listeners) {
1550-
i.logRollRequested(tooFewReplicas);
1653+
i.logRollRequested(reason);
15511654
}
15521655
}
15531656
}
@@ -1599,8 +1702,7 @@ boolean isLowReplicationRollEnabled() {
15991702

16001703
public static final long FIXED_OVERHEAD = ClassSize.align(
16011704
ClassSize.OBJECT + (5 * ClassSize.REFERENCE) +
1602-
ClassSize.ATOMIC_INTEGER + Bytes.SIZEOF_INT + (3 * Bytes.SIZEOF_LONG));
1603-
1705+
ClassSize.ATOMIC_INTEGER + (3 * Bytes.SIZEOF_INT) + (4 * Bytes.SIZEOF_LONG));
16041706

16051707
private static void split(final Configuration conf, final Path p) throws IOException {
16061708
FileSystem fs = FSUtils.getWALFileSystem(conf);
@@ -2083,4 +2185,14 @@ DatanodeInfo[] getPipeLine() {
20832185
public long getLastTimeCheckLowReplication() {
20842186
return this.lastTimeCheckLowReplication;
20852187
}
2188+
2189+
@VisibleForTesting
2190+
Writer getWriter() {
2191+
return this.writer;
2192+
}
2193+
2194+
@VisibleForTesting
2195+
void setWriter(Writer writer) {
2196+
this.writer = writer;
2197+
}
20862198
}

hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/MetricsWAL.java

Lines changed: 16 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -72,10 +72,23 @@ public void postAppend(final long size, final long time, final WALKey logkey,
7272
}
7373

7474
@Override
75-
public void logRollRequested(boolean underReplicated) {
75+
public void logRollRequested(WALActionsListener.RollRequestReason reason) {
7676
source.incrementLogRollRequested();
77-
if (underReplicated) {
78-
source.incrementLowReplicationLogRoll();
77+
switch (reason) {
78+
case ERROR:
79+
source.incrementErrorLogRoll();
80+
break;
81+
case LOW_REPLICATION:
82+
source.incrementLowReplicationLogRoll();
83+
break;
84+
case SIZE:
85+
source.incrementSizeLogRoll();
86+
break;
87+
case SLOW_SYNC:
88+
source.incrementSlowSyncLogRoll();
89+
break;
90+
default:
91+
break;
7992
}
8093
}
8194
}

0 commit comments

Comments
 (0)