Просмотр исходного кода

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 dc32f583afffc372f78fb45211c3e7ce13f6a4be)
Takanobu Asanuma 5 лет назад
Родитель
Сommit
ad11b25df2

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/IPCLoggerChannel.java

@@ -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())