You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ha...@apache.org on 2021/12/07 04:27:38 UTC
[hbase] branch master updated: HBASE-26526 Introduce a timeout to shutdown of WAL (#3297)
This is an automated email from the ASF dual-hosted git repository.
haxiaolin pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hbase.git
The following commit(s) were added to refs/heads/master by this push:
new ca3ba49 HBASE-26526 Introduce a timeout to shutdown of WAL (#3297)
ca3ba49 is described below
commit ca3ba494cbc322b0824d2d755bcf4191c3a525ed
Author: Xiaolin Ha <ha...@apache.org>
AuthorDate: Tue Dec 7 12:26:59 2021 +0800
HBASE-26526 Introduce a timeout to shutdown of WAL (#3297)
Signed-off-by: Andrew Purtell <ap...@apache.org>
---
.../hbase/regionserver/wal/AbstractFSWAL.java | 61 +++++++++++++++++-----
1 file changed, 48 insertions(+), 13 deletions(-)
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java
index 2e99fa8..5379e29 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java
@@ -39,6 +39,7 @@ import java.util.List;
import java.util.Map;
import java.util.OptionalLong;
import java.util.Set;
+import java.util.concurrent.Callable;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentNavigableMap;
import java.util.concurrent.ConcurrentSkipListMap;
@@ -46,7 +47,9 @@ import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
@@ -147,6 +150,10 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
public static final String RING_BUFFER_SLOT_COUNT =
"hbase.regionserver.wal.disruptor.event.count";
+ public static final String WAL_SHUTDOWN_WAIT_TIMEOUT_MS =
+ "hbase.wal.shutdown.wait.timeout.ms";
+ public static final int DEFAULT_WAL_SHUTDOWN_WAIT_TIMEOUT_MS = 15 * 1000;
+
/**
* file system instance
*/
@@ -278,6 +285,8 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
protected final AtomicBoolean shutdown = new AtomicBoolean(false);
+ protected final long walShutdownTimeout;
+
private long nextLogTooOldNs = System.nanoTime();
/**
@@ -336,8 +345,8 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
protected final AtomicBoolean rollRequested = new AtomicBoolean(false);
- private final ExecutorService logArchiveExecutor = Executors.newSingleThreadExecutor(
- new ThreadFactoryBuilder().setDaemon(true).setNameFormat("WAL-Archiver-%d").build());
+ private final ExecutorService logArchiveOrShutdownExecutor = Executors.newSingleThreadExecutor(
+ new ThreadFactoryBuilder().setDaemon(true).setNameFormat("WAL-Archive-Or-Shutdown-%d").build());
private final int archiveRetries;
@@ -499,7 +508,8 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
SURVIVED_TOO_LONG_SEC_KEY, SURVIVED_TOO_LONG_SEC_DEFAULT));
this.useHsync = conf.getBoolean(HRegion.WAL_HSYNC_CONF_KEY, HRegion.DEFAULT_WAL_HSYNC);
archiveRetries = this.conf.getInt("hbase.regionserver.walroll.archive.retries", 0);
-
+ this.walShutdownTimeout = conf.getLong(WAL_SHUTDOWN_WAIT_TIMEOUT_MS,
+ DEFAULT_WAL_SHUTDOWN_WAIT_TIMEOUT_MS);
}
/**
@@ -766,7 +776,7 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
final List<Pair<Path, Long>> localLogsToArchive = logsToArchive;
// make it async
for (Pair<Path, Long> log : localLogsToArchive) {
- logArchiveExecutor.execute(() -> {
+ logArchiveOrShutdownExecutor.execute(() -> {
archive(log);
});
this.walFile2Props.remove(log.getFirst());
@@ -980,17 +990,42 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
i.logCloseRequested();
}
}
- rollWriterLock.lock();
- try {
- doShutdown();
- if (syncFutureCache != null) {
- syncFutureCache.clear();
+
+ Future<Void> future = logArchiveOrShutdownExecutor.submit(new Callable<Void>() {
+ @Override
+ public Void call() throws Exception {
+ if (rollWriterLock.tryLock(walShutdownTimeout, TimeUnit.SECONDS)) {
+ try {
+ doShutdown();
+ if (syncFutureCache != null) {
+ syncFutureCache.clear();
+ }
+ } finally {
+ rollWriterLock.unlock();
+ }
+ } else {
+ throw new IOException("Waiting for rollWriterLock timeout");
+ }
+ return null;
}
- if (logArchiveExecutor != null) {
- logArchiveExecutor.shutdownNow();
+ });
+ logArchiveOrShutdownExecutor.shutdown();
+
+ try {
+ future.get(walShutdownTimeout, TimeUnit.MILLISECONDS);
+ } catch (InterruptedException e) {
+ throw new InterruptedIOException("Interrupted when waiting for shutdown WAL");
+ } catch (TimeoutException e) {
+ throw new TimeoutIOException("We have waited " + walShutdownTimeout + "ms, but"
+ + " the shutdown of WAL doesn't complete! Please check the status of underlying "
+ + "filesystem or increase the wait time by the config \""
+ + WAL_SHUTDOWN_WAIT_TIMEOUT_MS + "\"", e);
+ } catch (ExecutionException e) {
+ if (e.getCause() instanceof IOException) {
+ throw (IOException) e.getCause();
+ } else {
+ throw new IOException(e.getCause());
}
- } finally {
- rollWriterLock.unlock();
}
}