Skip to content

Commit 958759e

Browse files
committed
HBASE-25984: Avoid premature reuse of sync futures in FSHLog (apache#3371)
Signed-off-by: Viraj Jasani <[email protected]> (cherry picked from commit 5a19bcf)
1 parent 26c4e86 commit 958759e

File tree

7 files changed

+284
-20
lines changed

7 files changed

+284
-20
lines changed

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

Lines changed: 7 additions & 15 deletions
Original file line numberDiff line numberDiff line change
@@ -304,11 +304,9 @@ public WalProps(Map<byte[], Long> encodedName2HighestSequenceId, long logSize) {
304304
new ConcurrentSkipListMap<>(LOG_NAME_COMPARATOR);
305305

306306
/**
307-
* Map of {@link SyncFuture}s owned by Thread objects. Used so we reuse SyncFutures.
308-
* Thread local is used so JVM can GC the terminated thread for us. See HBASE-21228
309-
* <p>
307+
* A cache of sync futures reused by threads.
310308
*/
311-
private final ThreadLocal<SyncFuture> cachedSyncFutures;
309+
protected final SyncFutureCache syncFutureCache;
312310

313311
/**
314312
* The class name of the runtime implementation, used as prefix for logging/tracing.
@@ -478,12 +476,7 @@ public boolean accept(final Path fileName) {
478476
DEFAULT_SLOW_SYNC_ROLL_INTERVAL_MS);
479477
this.walSyncTimeoutNs = TimeUnit.MILLISECONDS.toNanos(conf.getLong(WAL_SYNC_TIMEOUT_MS,
480478
conf.getLong("hbase.regionserver.hlog.sync.timeout", DEFAULT_WAL_SYNC_TIMEOUT_MS)));
481-
this.cachedSyncFutures = new ThreadLocal<SyncFuture>() {
482-
@Override
483-
protected SyncFuture initialValue() {
484-
return new SyncFuture();
485-
}
486-
};
479+
this.syncFutureCache = new SyncFutureCache(conf);
487480
this.implClassName = getClass().getSimpleName();
488481
this.useHsync = conf.getBoolean(HRegion.WAL_HSYNC_CONF_KEY, HRegion.DEFAULT_WAL_HSYNC);
489482
archiveRetries = this.conf.getInt("hbase.regionserver.logroll.archive.retries", 0);
@@ -808,10 +801,6 @@ protected final void blockOnSync(SyncFuture syncFuture) throws IOException {
808801
}
809802
}
810803
} catch (TimeoutIOException tioe) {
811-
// SyncFuture reuse by thread, if TimeoutIOException happens, ringbuffer
812-
// still refer to it, so if this thread use it next time may get a wrong
813-
// result.
814-
this.cachedSyncFutures.remove();
815804
throw tioe;
816805
} catch (InterruptedException ie) {
817806
LOG.warn("Interrupted", ie);
@@ -913,6 +902,9 @@ public void shutdown() throws IOException {
913902
rollWriterLock.lock();
914903
try {
915904
doShutdown();
905+
if (syncFutureCache != null) {
906+
syncFutureCache.clear();
907+
}
916908
if (logArchiveExecutor != null) {
917909
logArchiveExecutor.shutdownNow();
918910
}
@@ -969,7 +961,7 @@ public void updateStore(byte[] encodedRegionName, byte[] familyName, Long sequen
969961
}
970962

971963
protected final SyncFuture getSyncFuture(long sequence, boolean forceSync) {
972-
return cachedSyncFutures.get().reset(sequence).setForceSync(forceSync);
964+
return syncFutureCache.getIfPresentOrNew().reset(sequence).setForceSync(forceSync);
973965
}
974966

975967
protected boolean isLogRollRequested() {

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

Lines changed: 11 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -265,6 +265,14 @@ public AsyncFSWAL(FileSystem fs, Abortable abortable, Path rootDir, String logDi
265265
DEFAULT_ASYNC_WAL_WAIT_ON_SHUTDOWN_IN_SECONDS);
266266
}
267267

268+
/**
269+
* Helper that marks the future as DONE and offers it back to the cache.
270+
*/
271+
private void markFutureDoneAndOffer(SyncFuture future, long txid, Throwable t) {
272+
future.done(txid, t);
273+
syncFutureCache.offer(future);
274+
}
275+
268276
private static boolean waitingRoll(int epochAndState) {
269277
return (epochAndState & 1) != 0;
270278
}
@@ -405,7 +413,7 @@ private int finishSyncLowerThanTxid(long txid, boolean addSyncTrace) {
405413
for (Iterator<SyncFuture> iter = syncFutures.iterator(); iter.hasNext();) {
406414
SyncFuture sync = iter.next();
407415
if (sync.getTxid() <= txid) {
408-
sync.done(txid, null);
416+
markFutureDoneAndOffer(sync, txid, null);
409417
iter.remove();
410418
finished++;
411419
if (addSyncTrace) {
@@ -427,7 +435,7 @@ private int finishSync(boolean addSyncTrace) {
427435
long maxSyncTxid = highestSyncedTxid.get();
428436
for (SyncFuture sync : syncFutures) {
429437
maxSyncTxid = Math.max(maxSyncTxid, sync.getTxid());
430-
sync.done(maxSyncTxid, null);
438+
markFutureDoneAndOffer(sync, maxSyncTxid, null);
431439
if (addSyncTrace) {
432440
addTimeAnnotation(sync, "writer synced");
433441
}
@@ -766,7 +774,7 @@ protected void doShutdown() throws IOException {
766774
}
767775
}
768776
// and fail them
769-
syncFutures.forEach(f -> f.done(f.getTxid(), error));
777+
syncFutures.forEach(f -> markFutureDoneAndOffer(f, f.getTxid(), error));
770778
if (!(consumeExecutor instanceof EventLoop)) {
771779
consumeExecutor.shutdown();
772780
}

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

Lines changed: 22 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -909,6 +909,14 @@ SyncFuture waitSafePoint(SyncFuture syncFuture) throws InterruptedException,
909909
return syncFuture;
910910
}
911911

912+
/**
913+
* @return if the safepoint has been attained.
914+
*/
915+
@InterfaceAudience.Private
916+
boolean isSafePointAttained() {
917+
return this.safePointAttainedLatch.getCount() == 0;
918+
}
919+
912920
/**
913921
* Called by Thread B when it attains the 'safe point'. In this method, Thread B signals Thread
914922
* A it can proceed. Thread B will be held in here until {@link #releaseSafePoint()} is called
@@ -995,6 +1003,16 @@ private void cleanupOutstandingSyncsOnException(final long sequence, final Excep
9951003
for (int i = 0; i < this.syncFuturesCount.get(); i++) {
9961004
this.syncFutures[i].done(sequence, e);
9971005
}
1006+
offerDoneSyncsBackToCache();
1007+
}
1008+
1009+
/**
1010+
* Offers the finished syncs back to the cache for reuse.
1011+
*/
1012+
private void offerDoneSyncsBackToCache() {
1013+
for (int i = 0; i < this.syncFuturesCount.get(); i++) {
1014+
syncFutureCache.offer(syncFutures[i]);
1015+
}
9981016
this.syncFuturesCount.set(0);
9991017
}
10001018

@@ -1109,7 +1127,10 @@ public void onEvent(final RingBufferTruck truck, final long sequence, boolean en
11091127
? this.exception : new DamagedWALException("On sync", this.exception));
11101128
}
11111129
attainSafePoint(sequence);
1112-
this.syncFuturesCount.set(0);
1130+
// It is critical that we offer the futures back to the cache for reuse here after the
1131+
// safe point is attained and all the clean up has been done. There have been
1132+
// issues with reusing sync futures early causing WAL lockups, see HBASE-25984.
1133+
offerDoneSyncsBackToCache();
11131134
} catch (Throwable t) {
11141135
LOG.error("UNEXPECTED!!! syncFutures.length=" + this.syncFutures.length, t);
11151136
}

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

Lines changed: 11 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -90,7 +90,8 @@ synchronized SyncFuture reset(long txid) {
9090

9191
@Override
9292
public synchronized String toString() {
93-
return "done=" + isDone() + ", txid=" + this.txid;
93+
return "done=" + isDone() + ", txid=" + this.txid + " threadID=" + t.getId() +
94+
" threadName=" + t.getName();
9495
}
9596

9697
synchronized long getTxid() {
@@ -106,6 +107,15 @@ synchronized SyncFuture setForceSync(boolean forceSync) {
106107
return this;
107108
}
108109

110+
/**
111+
* Returns the thread that owned this sync future, use with caution as we return the reference to
112+
* the actual thread object.
113+
* @return the associated thread instance.
114+
*/
115+
Thread getThread() {
116+
return t;
117+
}
118+
109119
/**
110120
* @param txid the transaction id at which this future 'completed'.
111121
* @param t Can be null. Set if we are 'completing' on error (and this 't' is the error).
Lines changed: 74 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,74 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one
3+
* or more contributor license agreements. See the NOTICE file
4+
* distributed with this work for additional information
5+
* regarding copyright ownership. The ASF licenses this file
6+
* to you under the Apache License, Version 2.0 (the
7+
* "License"); you may not use this file except in compliance
8+
* with the License. You may obtain a copy of the License at
9+
*
10+
* http://www.apache.org/licenses/LICENSE-2.0
11+
*
12+
* Unless required by applicable law or agreed to in writing, software
13+
* distributed under the License is distributed on an "AS IS" BASIS,
14+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15+
* See the License for the specific language governing permissions and
16+
* limitations under the License.
17+
*/
18+
package org.apache.hadoop.hbase.regionserver.wal;
19+
20+
import java.util.concurrent.TimeUnit;
21+
import org.apache.hadoop.conf.Configuration;
22+
import org.apache.hadoop.hbase.HConstants;
23+
import org.apache.yetus.audience.InterfaceAudience;
24+
import org.apache.hbase.thirdparty.com.google.common.cache.Cache;
25+
import org.apache.hbase.thirdparty.com.google.common.cache.CacheBuilder;
26+
27+
/**
28+
* A cache of {@link SyncFuture}s. This class supports two methods
29+
* {@link SyncFutureCache#getIfPresentOrNew()} and {@link SyncFutureCache#offer(SyncFuture)}.
30+
*
31+
* Usage pattern:
32+
* SyncFuture sf = syncFutureCache.getIfPresentOrNew();
33+
* sf.reset(...);
34+
* // Use the sync future
35+
* finally: syncFutureCache.offer(sf);
36+
*
37+
* Offering the sync future back to the cache makes it eligible for reuse within the same thread
38+
* context. Cache keyed by the accessing thread instance and automatically invalidated if it remains
39+
* unused for {@link SyncFutureCache#SYNC_FUTURE_INVALIDATION_TIMEOUT_MINS} minutes.
40+
*/
41+
@InterfaceAudience.Private
42+
public final class SyncFutureCache {
43+
44+
private static final long SYNC_FUTURE_INVALIDATION_TIMEOUT_MINS = 2;
45+
46+
private final Cache<Thread, SyncFuture> syncFutureCache;
47+
48+
public SyncFutureCache(final Configuration conf) {
49+
final int handlerCount = conf.getInt(HConstants.REGION_SERVER_HANDLER_COUNT,
50+
HConstants.DEFAULT_REGION_SERVER_HANDLER_COUNT);
51+
syncFutureCache = CacheBuilder.newBuilder().initialCapacity(handlerCount)
52+
.expireAfterWrite(SYNC_FUTURE_INVALIDATION_TIMEOUT_MINS, TimeUnit.MINUTES).build();
53+
}
54+
55+
public SyncFuture getIfPresentOrNew() {
56+
// Invalidate the entry if a mapping exists. We do not want it to be reused at the same time.
57+
SyncFuture future = syncFutureCache.asMap().remove(Thread.currentThread());
58+
return (future == null) ? new SyncFuture() : future;
59+
}
60+
61+
/**
62+
* Offers the sync future back to the cache for reuse.
63+
*/
64+
public void offer(SyncFuture syncFuture) {
65+
// It is ok to overwrite an existing mapping.
66+
syncFutureCache.asMap().put(syncFuture.getThread(), syncFuture);
67+
}
68+
69+
public void clear() {
70+
if (syncFutureCache != null) {
71+
syncFutureCache.invalidateAll();
72+
}
73+
}
74+
}

hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestFSHLog.java

Lines changed: 91 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -18,6 +18,8 @@
1818
package org.apache.hadoop.hbase.regionserver.wal;
1919

2020
import static org.junit.Assert.assertEquals;
21+
import static org.junit.Assert.assertFalse;
22+
import static org.junit.Assert.assertTrue;
2123

2224
import java.io.IOException;
2325
import java.lang.reflect.Field;
@@ -27,13 +29,15 @@
2729
import java.util.concurrent.CountDownLatch;
2830
import java.util.concurrent.ExecutorService;
2931
import java.util.concurrent.Executors;
32+
import java.util.concurrent.TimeUnit;
3033
import java.util.concurrent.atomic.AtomicBoolean;
3134
import org.apache.hadoop.conf.Configuration;
3235
import org.apache.hadoop.fs.FileSystem;
3336
import org.apache.hadoop.fs.Path;
3437
import org.apache.hadoop.hbase.HBaseClassTestRule;
3538
import org.apache.hadoop.hbase.HConstants;
3639
import org.apache.hadoop.hbase.TableName;
40+
import org.apache.hadoop.hbase.Waiter;
3741
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
3842
import org.apache.hadoop.hbase.client.Put;
3943
import org.apache.hadoop.hbase.client.RegionInfo;
@@ -49,8 +53,10 @@
4953
import org.apache.hadoop.hbase.util.Bytes;
5054
import org.apache.hadoop.hbase.util.CommonFSUtils;
5155
import org.apache.hadoop.hbase.util.Threads;
56+
import org.apache.hadoop.hbase.wal.WAL;
5257
import org.apache.hadoop.hbase.wal.WALEdit;
5358
import org.apache.hadoop.hbase.wal.WALKey;
59+
import org.apache.hadoop.hbase.wal.WALProvider;
5460
import org.junit.ClassRule;
5561
import org.junit.Rule;
5662
import org.junit.Test;
@@ -67,6 +73,8 @@ public class TestFSHLog extends AbstractTestFSWAL {
6773
public static final HBaseClassTestRule CLASS_RULE =
6874
HBaseClassTestRule.forClass(TestFSHLog.class);
6975

76+
private static final long TEST_TIMEOUT_MS = 10000;
77+
7078
@Rule
7179
public TestName name = new TestName();
7280

@@ -131,6 +139,89 @@ public void testSyncRunnerIndexOverflow() throws IOException, NoSuchFieldExcepti
131139
}
132140
}
133141

142+
/**
143+
* Test for WAL stall due to sync future overwrites. See HBASE-25984.
144+
*/
145+
@Test
146+
public void testDeadlockWithSyncOverwrites() throws Exception {
147+
final CountDownLatch blockBeforeSafePoint = new CountDownLatch(1);
148+
149+
class FailingWriter implements WALProvider.Writer {
150+
@Override public void sync(boolean forceSync) throws IOException {
151+
throw new IOException("Injected failure..");
152+
}
153+
154+
@Override public void append(WAL.Entry entry) throws IOException {
155+
}
156+
157+
@Override public long getLength() {
158+
return 0;
159+
}
160+
161+
@Override
162+
public long getSyncedLength() {
163+
return 0;
164+
}
165+
166+
@Override public void close() throws IOException {
167+
}
168+
}
169+
170+
/*
171+
* Custom FSHLog implementation with a conditional wait before attaining safe point.
172+
*/
173+
class CustomFSHLog extends FSHLog {
174+
public CustomFSHLog(FileSystem fs, Path rootDir, String logDir, String archiveDir,
175+
Configuration conf, List<WALActionsListener> listeners, boolean failIfWALExists,
176+
String prefix, String suffix) throws IOException {
177+
super(fs, rootDir, logDir, archiveDir, conf, listeners, failIfWALExists, prefix, suffix);
178+
}
179+
180+
@Override
181+
protected void beforeWaitOnSafePoint() {
182+
try {
183+
assertTrue(blockBeforeSafePoint.await(TEST_TIMEOUT_MS, TimeUnit.MILLISECONDS));
184+
} catch (InterruptedException e) {
185+
throw new RuntimeException(e);
186+
}
187+
}
188+
189+
public SyncFuture publishSyncOnRingBuffer() {
190+
long sequence = getSequenceOnRingBuffer();
191+
return publishSyncOnRingBuffer(sequence, false);
192+
}
193+
}
194+
195+
final String name = this.name.getMethodName();
196+
try (CustomFSHLog log = new CustomFSHLog(FS, CommonFSUtils.getRootDir(CONF), name,
197+
HConstants.HREGION_OLDLOGDIR_NAME, CONF, null, true, null, null)) {
198+
log.setWriter(new FailingWriter());
199+
Field ringBufferEventHandlerField =
200+
FSHLog.class.getDeclaredField("ringBufferEventHandler");
201+
ringBufferEventHandlerField.setAccessible(true);
202+
FSHLog.RingBufferEventHandler ringBufferEventHandler =
203+
(FSHLog.RingBufferEventHandler) ringBufferEventHandlerField.get(log);
204+
// Force a safe point
205+
FSHLog.SafePointZigZagLatch latch = ringBufferEventHandler.attainSafePoint();
206+
try {
207+
SyncFuture future0 = log.publishSyncOnRingBuffer();
208+
// Wait for the sync to be done.
209+
Waiter.waitFor(CONF, TEST_TIMEOUT_MS, future0::isDone);
210+
// Publish another sync from the same thread, this should not overwrite the done sync.
211+
SyncFuture future1 = log.publishSyncOnRingBuffer();
212+
assertFalse(future1.isDone());
213+
// Unblock the safe point trigger..
214+
blockBeforeSafePoint.countDown();
215+
// Wait for the safe point to be reached.
216+
// With the deadlock in HBASE-25984, this is never possible, thus blocking the sync pipeline.
217+
Waiter.waitFor(CONF, TEST_TIMEOUT_MS, latch::isSafePointAttained);
218+
} finally {
219+
// Force release the safe point, for the clean up.
220+
latch.releaseSafePoint();
221+
}
222+
}
223+
}
224+
134225
/**
135226
* Test case for https://issues.apache.org/jira/browse/HBASE-16721
136227
*/

0 commit comments

Comments
 (0)