HBASE-22881 Fix non-daemon threads in hbase server implementation (#512) (#558)

Signed-off-by: stack <stack@apache.org>
This commit is contained in:
linkaline 2019-08-30 07:14:34 +08:00 committed by Michael Stack
parent 3b6f6b623d
commit db85526254
19 changed files with 64 additions and 115 deletions

View File

@ -807,7 +807,8 @@ public class TestClientNoCluster extends Configured implements Tool {
// Have them all share the same connection so they all share the same instance of
// ManyServersManyRegionsConnection so I can keep an eye on how many requests by server.
final ExecutorService pool = Executors.newCachedThreadPool(Threads.getNamedThreadFactory("p"));
final ExecutorService pool = Executors.newCachedThreadPool(
Threads.newDaemonThreadFactory("p"));
// Executors.newFixedThreadPool(servers * 10, Threads.getNamedThreadFactory("p"));
// Share a connection so I can keep counts in the 'server' on concurrency.
final Connection sharedConnection = ConnectionFactory.createConnection(getConf()/*, pool*/);

View File

@ -204,7 +204,7 @@ public class Threads {
* @param prefix The prefix of every created Thread's name
* @return a {@link java.util.concurrent.ThreadFactory} that names threads
*/
public static ThreadFactory getNamedThreadFactory(final String prefix) {
private static ThreadFactory getNamedThreadFactory(final String prefix) {
SecurityManager s = System.getSecurityManager();
final ThreadGroup threadGroup = (s != null) ? s.getThreadGroup() : Thread.currentThread()
.getThreadGroup();

View File

@ -227,7 +227,9 @@ public class HFileArchiver {
@Override
public Thread newThread(Runnable r) {
final String name = "HFileArchiver-" + threadNumber.getAndIncrement();
return new Thread(r, name);
Thread t = new Thread(r, name);
t.setDaemon(true);
return t;
}
};
}

View File

@ -23,21 +23,21 @@ import java.util.List;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.RejectedExecutionException;
import java.util.concurrent.SynchronousQueue;
import java.util.concurrent.ThreadFactory;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.hbase.TableName;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.master.locking.LockManager;
import org.apache.hadoop.hbase.mob.MobUtils;
import org.apache.hadoop.hbase.procedure2.LockType;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
/**
* The mob compaction thread used in {@link MasterRpcServices}
@ -55,14 +55,11 @@ public class MasterMobCompactionThread {
this.conf = master.getConfiguration();
final String n = Thread.currentThread().getName();
// this pool is used to run the mob compaction
this.masterMobPool = new ThreadPoolExecutor(1, 2, 60, TimeUnit.SECONDS,
new SynchronousQueue<>(), new ThreadFactory() {
@Override
public Thread newThread(Runnable r) {
String name = n + "-MasterMobCompaction-" + EnvironmentEdgeManager.currentTime();
return new Thread(r, name);
}
});
this.masterMobPool = new ThreadPoolExecutor(1, 2, 60,
TimeUnit.SECONDS, new SynchronousQueue<>(),
new ThreadFactoryBuilder().setDaemon(true)
.setNameFormat(n + "-MasterMobCompaction-" + EnvironmentEdgeManager.currentTime())
.build());
((ThreadPoolExecutor) this.masterMobPool).allowCoreThreadTimeOut(true);
// this pool is used in the mob compaction to compact the mob files by partitions
// in parallel

View File

@ -667,7 +667,7 @@ public class SplitTableRegionProcedure
LOG.info("pid=" + getProcId() + " splitting " + nbFiles + " storefiles, region=" +
getParentRegion().getShortNameToLog() + ", threads=" + maxThreads);
final ExecutorService threadPool = Executors.newFixedThreadPool(maxThreads,
Threads.getNamedThreadFactory("StoreFileSplitter-%1$d"));
Threads.newDaemonThreadFactory("StoreFileSplitter-%1$d"));
final List<Future<Pair<Path, Path>>> futures = new ArrayList<Future<Pair<Path, Path>>>(nbFiles);
// Split each store file.

View File

@ -25,11 +25,9 @@ import java.util.concurrent.Callable;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorCompletionService;
import java.util.concurrent.Future;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.DaemonThreadFactory;
@ -47,8 +45,11 @@ import org.apache.hadoop.hbase.procedure.ZKProcedureMemberRpcs;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.zookeeper.KeeperException;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -213,10 +214,8 @@ public class RegionServerFlushTableProcedureManager extends RegionServerProcedur
RegionServerFlushTableProcedureManager.FLUSH_TIMEOUT_MILLIS_DEFAULT);
int threads = conf.getInt(CONCURENT_FLUSH_TASKS_KEY, DEFAULT_CONCURRENT_FLUSH_TASKS);
this.name = name;
executor = new ThreadPoolExecutor(threads, threads, keepAlive, TimeUnit.MILLISECONDS,
new LinkedBlockingQueue<>(), new DaemonThreadFactory("rs("
+ name + ")-flush-proc-pool"));
executor.allowCoreThreadTimeOut(true);
executor = Threads.getBoundedCachedThreadPool(threads, keepAlive, TimeUnit.MILLISECONDS,
new DaemonThreadFactory("rs(" + name + ")-flush-proc-pool-"));
taskPool = new ExecutorCompletionService<>(executor);
}

View File

@ -31,7 +31,6 @@ import java.util.concurrent.BlockingQueue;
import java.util.concurrent.Executors;
import java.util.concurrent.RejectedExecutionException;
import java.util.concurrent.RejectedExecutionHandler;
import java.util.concurrent.ThreadFactory;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
@ -56,8 +55,10 @@ import org.apache.hadoop.util.StringUtils;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
/**
* Compact region on request and then run split if appropriate
@ -120,37 +121,22 @@ public class CompactSplit implements CompactionRequester, PropagatingConfigurati
final String n = Thread.currentThread().getName();
StealJobQueue<Runnable> stealJobQueue = new StealJobQueue<Runnable>(COMPARATOR);
this.longCompactions = new ThreadPoolExecutor(largeThreads, largeThreads,
60, TimeUnit.SECONDS, stealJobQueue,
new ThreadFactory() {
@Override
public Thread newThread(Runnable r) {
String name = n + "-longCompactions-" + System.currentTimeMillis();
return new Thread(r, name);
}
});
this.longCompactions = new ThreadPoolExecutor(largeThreads, largeThreads, 60,
TimeUnit.SECONDS, stealJobQueue,
new ThreadFactoryBuilder()
.setNameFormat(n + "-longCompactions-" + System.currentTimeMillis())
.setDaemon(true).build());
this.longCompactions.setRejectedExecutionHandler(new Rejection());
this.longCompactions.prestartAllCoreThreads();
this.shortCompactions = new ThreadPoolExecutor(smallThreads, smallThreads,
60, TimeUnit.SECONDS, stealJobQueue.getStealFromQueue(),
new ThreadFactory() {
@Override
public Thread newThread(Runnable r) {
String name = n + "-shortCompactions-" + System.currentTimeMillis();
return new Thread(r, name);
}
});
this.shortCompactions
.setRejectedExecutionHandler(new Rejection());
this.splits = (ThreadPoolExecutor)
Executors.newFixedThreadPool(splitThreads,
new ThreadFactory() {
@Override
public Thread newThread(Runnable r) {
String name = n + "-splits-" + System.currentTimeMillis();
return new Thread(r, name);
}
});
this.shortCompactions = new ThreadPoolExecutor(smallThreads, smallThreads, 60,
TimeUnit.SECONDS, stealJobQueue.getStealFromQueue(),
new ThreadFactoryBuilder()
.setNameFormat(n + "-shortCompactions-" + System.currentTimeMillis())
.setDaemon(true).build());
this.shortCompactions.setRejectedExecutionHandler(new Rejection());
this.splits = (ThreadPoolExecutor) Executors.newFixedThreadPool(splitThreads,
new ThreadFactoryBuilder().setNameFormat(n + "-splits-" + System.currentTimeMillis())
.setDaemon(true).build());
// compaction throughput controller
this.compactionThroughputController =

View File

@ -26,7 +26,6 @@ import java.util.concurrent.Callable;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorCompletionService;
import java.util.concurrent.Future;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
@ -36,6 +35,7 @@ import org.apache.hadoop.hbase.DaemonThreadFactory;
import org.apache.hadoop.hbase.DroppedSnapshotException;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
@ -283,10 +283,8 @@ public class RegionServerSnapshotManager extends RegionServerProcedureManager {
RegionServerSnapshotManager.SNAPSHOT_TIMEOUT_MILLIS_DEFAULT);
int threads = conf.getInt(CONCURENT_SNAPSHOT_TASKS_KEY, DEFAULT_CONCURRENT_SNAPSHOT_TASKS);
this.name = name;
executor = new ThreadPoolExecutor(threads, threads, keepAlive, TimeUnit.MILLISECONDS,
new LinkedBlockingQueue<>(), new DaemonThreadFactory("rs("
+ name + ")-snapshot-pool"));
executor.allowCoreThreadTimeOut(true);
executor = Threads.getBoundedCachedThreadPool(threads, keepAlive, TimeUnit.MILLISECONDS,
new DaemonThreadFactory("rs(" + name + ")-snapshot-pool-"));
taskPool = new ExecutorCompletionService<>(executor);
}

View File

@ -222,7 +222,8 @@ public class FSHLog extends AbstractFSWAL<Writer> {
// Using BlockingWaitStrategy. Stuff that is going on here takes so long it makes no sense
// spinning as other strategies do.
this.disruptor = new Disruptor<>(RingBufferTruck::new,
getPreallocatedEventCount(), Threads.getNamedThreadFactory(hostingThreadName + ".append"),
getPreallocatedEventCount(),
Threads.newDaemonThreadFactory(hostingThreadName + ".append"),
ProducerType.MULTI, new BlockingWaitStrategy());
// Advance the ring buffer sequence so that it starts from 1 instead of 0,
// because SyncFuture.NOT_DONE = 0.

View File

@ -32,7 +32,6 @@ import java.util.concurrent.CompletionService;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorCompletionService;
import java.util.concurrent.Future;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import java.util.regex.Matcher;
@ -57,8 +56,10 @@ import org.apache.hadoop.hbase.replication.HBaseReplicationEndpoint;
import org.apache.hadoop.hbase.replication.regionserver.ReplicationSinkManager.SinkPeer;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.wal.WAL.Entry;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -141,9 +142,8 @@ public class HBaseInterClusterReplicationEndpoint extends HBaseReplicationEndpoi
// per sink thread pool
this.maxThreads = this.conf.getInt(HConstants.REPLICATION_SOURCE_MAXTHREADS_KEY,
HConstants.REPLICATION_SOURCE_MAXTHREADS_DEFAULT);
this.exec = new ThreadPoolExecutor(maxThreads, maxThreads, 60, TimeUnit.SECONDS,
new LinkedBlockingQueue<>());
this.exec.allowCoreThreadTimeOut(true);
this.exec = Threads.getBoundedCachedThreadPool(maxThreads, 60, TimeUnit.SECONDS,
new ThreadFactoryBuilder().setDaemon(true).setNameFormat("SinkThread-%d").build());
this.abortable = ctx.getAbortable();
// Set the size limit for replication RPCs to 95% of the max request size.
// We could do with less slop if we have an accurate estimate of encoded size. Being

View File

@ -27,7 +27,6 @@ import java.util.Map.Entry;
import java.util.concurrent.Callable;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Future;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
@ -53,6 +52,7 @@ import org.apache.hadoop.hbase.security.token.FsDelegationToken;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.Threads;
/**
* It is used for replicating HFile entries. It will first copy parallely all the hfiles to a local
@ -105,12 +105,9 @@ public class HFileReplicator {
this.maxCopyThreads =
this.conf.getInt(REPLICATION_BULKLOAD_COPY_MAXTHREADS_KEY,
REPLICATION_BULKLOAD_COPY_MAXTHREADS_DEFAULT);
ThreadFactoryBuilder builder = new ThreadFactoryBuilder();
builder.setNameFormat("HFileReplicationCallable-%1$d");
this.exec =
new ThreadPoolExecutor(maxCopyThreads, maxCopyThreads, 60, TimeUnit.SECONDS,
new LinkedBlockingQueue<>(), builder.build());
this.exec.allowCoreThreadTimeOut(true);
this.exec = Threads.getBoundedCachedThreadPool(maxCopyThreads, 60, TimeUnit.SECONDS,
new ThreadFactoryBuilder().setDaemon(true)
.setNameFormat("HFileReplicationCallable-%1$d").build());
this.copiesPerThread =
conf.getInt(REPLICATION_BULKLOAD_COPY_HFILES_PERTHREAD_KEY,
REPLICATION_BULKLOAD_COPY_HFILES_PERTHREAD_DEFAULT);

View File

@ -196,8 +196,8 @@ public class ReplicationSourceManager implements ReplicationListener {
int nbWorkers = conf.getInt("replication.executor.workers", 1);
// use a short 100ms sleep since this could be done inline with a RS startup
// even if we fail, other region servers can take care of it
this.executor = new ThreadPoolExecutor(nbWorkers, nbWorkers, 100, TimeUnit.MILLISECONDS,
new LinkedBlockingQueue<>());
this.executor = new ThreadPoolExecutor(nbWorkers, nbWorkers, 100,
TimeUnit.MILLISECONDS, new LinkedBlockingQueue<>());
ThreadFactoryBuilder tfb = new ThreadFactoryBuilder();
tfb.setNameFormat("ReplicationExecutor-%d");
tfb.setDaemon(true);

View File

@ -558,7 +558,7 @@ public final class SnapshotManifest {
public static ThreadPoolExecutor createExecutor(final Configuration conf, final String name) {
int maxThreads = conf.getInt("hbase.snapshot.thread.pool.max", 8);
return Threads.getBoundedCachedThreadPool(maxThreads, 30L, TimeUnit.SECONDS,
Threads.getNamedThreadFactory(name));
Threads.newDaemonThreadFactory(name));
}
/**

View File

@ -108,7 +108,7 @@ public class HFileContentValidator extends AbstractHBaseTool {
int availableProcessors = Runtime.getRuntime().availableProcessors();
int numThreads = conf.getInt("hfilevalidator.numthreads", availableProcessors);
return Executors.newFixedThreadPool(numThreads,
Threads.getNamedThreadFactory("hfile-validator"));
Threads.newDaemonThreadFactory("hfile-validator"));
}
@Override

View File

@ -593,8 +593,7 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
}
/**
* @param conn the HBase cluster connection
* @param tableName the table name of the table to load into
* @param table the table to load into
* @param pool the ExecutorService
* @param queue the queue for LoadQueueItem
* @param startEndKeys start and end keys

View File

@ -1623,7 +1623,8 @@ public abstract class FSUtils extends CommonFSUtils {
// run in multiple threads
ThreadPoolExecutor tpe = new ThreadPoolExecutor(threadPoolSize,
threadPoolSize, 60, TimeUnit.SECONDS,
new ArrayBlockingQueue<>(statusList.length));
new ArrayBlockingQueue<>(statusList.length),
Threads.newDaemonThreadFactory("FSRegionQuery"));
try {
// ignore all file status items that are not of interest
for (FileStatus regionStatus : statusList) {

View File

@ -28,7 +28,6 @@ import java.util.concurrent.Callable;
import java.util.concurrent.CompletionService;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorCompletionService;
import java.util.concurrent.ThreadFactory;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
@ -235,14 +234,7 @@ public abstract class ModifyRegionUtils {
"hbase.hregion.open.and.init.threads.max", 16));
ThreadPoolExecutor regionOpenAndInitThreadPool = Threads
.getBoundedCachedThreadPool(maxThreads, 30L, TimeUnit.SECONDS,
new ThreadFactory() {
private int count = 1;
@Override
public Thread newThread(Runnable r) {
return new Thread(r, threadNamePrefix + "-" + count++);
}
});
Threads.newDaemonThreadFactory(threadNamePrefix));
return regionOpenAndInitThreadPool;
}
}

View File

@ -124,9 +124,9 @@ public class SimpleRSProcedureManager extends RegionServerProcedureManager {
public SimpleSubprocedurePool(String name, Configuration conf) {
this.name = name;
executor = new ThreadPoolExecutor(1, 1, 500, TimeUnit.SECONDS,
new LinkedBlockingQueue<>(),
new DaemonThreadFactory("rs(" + name + ")-procedure-pool"));
executor = new ThreadPoolExecutor(1, 1, 500,
TimeUnit.SECONDS, new LinkedBlockingQueue<>(),
new DaemonThreadFactory("rs(" + name + ")-procedure-pool-"));
taskPool = new ExecutorCompletionService<>(executor);
}

View File

@ -26,10 +26,8 @@ import java.util.concurrent.Callable;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.ThreadFactory;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.LongAdder;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.client.Table;
@ -131,27 +129,6 @@ public class IncrementCoalescer implements IncrementCoalescerMBean {
}
static class DaemonThreadFactory implements ThreadFactory {
static final AtomicInteger poolNumber = new AtomicInteger(1);
final ThreadGroup group;
final AtomicInteger threadNumber = new AtomicInteger(1);
final String namePrefix;
DaemonThreadFactory() {
SecurityManager s = System.getSecurityManager();
group = (s != null) ? s.getThreadGroup() : Thread.currentThread().getThreadGroup();
namePrefix = "ICV-" + poolNumber.getAndIncrement() + "-thread-";
}
@Override
public Thread newThread(Runnable r) {
Thread t = new Thread(group, r, namePrefix + threadNumber.getAndIncrement(), 0);
if (!t.isDaemon()) t.setDaemon(true);
if (t.getPriority() != Thread.NORM_PRIORITY) t.setPriority(Thread.NORM_PRIORITY);
return t;
}
}
private final LongAdder failedIncrements = new LongAdder();
private final LongAdder successfulCoalescings = new LongAdder();
private final LongAdder totalIncrements = new LongAdder();
@ -169,10 +146,9 @@ public class IncrementCoalescer implements IncrementCoalescerMBean {
public IncrementCoalescer(HBaseHandler hand) {
this.handler = hand;
LinkedBlockingQueue<Runnable> queue = new LinkedBlockingQueue<>();
pool =
new ThreadPoolExecutor(CORE_POOL_SIZE, CORE_POOL_SIZE, 50, TimeUnit.MILLISECONDS, queue,
Threads.newDaemonThreadFactory("IncrementCoalescer"));
pool = new ThreadPoolExecutor(CORE_POOL_SIZE, CORE_POOL_SIZE, 50,
TimeUnit.MILLISECONDS, queue,
Threads.newDaemonThreadFactory("IncrementCoalescer"));
MBeans.register("thrift", "Thrift", this);
}