HDFS-14934. [SBN Read] Standby NN throws many InterruptedExceptions when dfs.ha.tail-edits.period is 0. Contributed by Ayush Saxena.

(cherry picked from commit dc32f583af)
This commit is contained in:
Takanobu Asanuma 2019-12-28 21:32:15 +09:00
parent 4acbe1c05d
commit 454a0c2913
1 changed files with 3 additions and 3 deletions

View File

@ -28,6 +28,7 @@ import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import org.apache.hadoop.classification.InterfaceAudience;
@ -55,7 +56,6 @@ import org.apache.hadoop.ipc.ProtobufRpcEngine;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.security.SecurityUtil;
import org.apache.hadoop.util.StopWatch;
import org.apache.hadoop.util.concurrent.HadoopThreadPoolExecutor;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
@ -275,8 +275,8 @@ public class IPCLoggerChannel implements AsyncLogger {
int numThreads =
conf.getInt(DFSConfigKeys.DFS_QJOURNAL_PARALLEL_READ_NUM_THREADS_KEY,
DFSConfigKeys.DFS_QJOURNAL_PARALLEL_READ_NUM_THREADS_DEFAULT);
return new HadoopThreadPoolExecutor(1, numThreads, 60L,
TimeUnit.SECONDS, new LinkedBlockingQueue<>(),
return new ThreadPoolExecutor(1, numThreads, 60L, TimeUnit.SECONDS,
new LinkedBlockingQueue<>(),
new ThreadFactoryBuilder().setDaemon(true)
.setNameFormat("Logger channel (from parallel executor) to " + addr)
.setUncaughtExceptionHandler(UncaughtExceptionHandlers.systemExit())