HBASE-22930 Set unique name to longCompactions/shortCompactions threads (#645)

This commit is contained in:
Pankaj 2019-09-30 13:41:21 +05:30 committed by Peter Somogyi
parent daf1faabfd
commit 4cd51aed31
1 changed files with 14 additions and 9 deletions

View File

@ -31,6 +31,7 @@ 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;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@ -133,9 +134,11 @@ public class CompactSplitThread implements CompactionRequestor, PropagatingConfi
int splitThreads = conf.getInt(SPLIT_THREADS, SPLIT_THREADS_DEFAULT);
this.splits =
(ThreadPoolExecutor) Executors.newFixedThreadPool(splitThreads, new ThreadFactory() {
AtomicInteger splitThreadCounter = new AtomicInteger(0);
@Override
public Thread newThread(Runnable r) {
String name = n + "-splits-" + System.currentTimeMillis();
String name = n + "-splits-" + splitThreadCounter.getAndIncrement();
return new Thread(r, name);
}
});
@ -155,23 +158,25 @@ public class CompactSplitThread implements CompactionRequestor, PropagatingConfi
final String n = Thread.currentThread().getName();
StealJobQueue<Runnable> stealJobQueue = new StealJobQueue<Runnable>();
this.longCompactions = new ThreadPoolExecutor(largeThreads, largeThreads, 60,
TimeUnit.SECONDS, stealJobQueue,
new ThreadFactory() {
this.longCompactions = new ThreadPoolExecutor(largeThreads, largeThreads, 60, TimeUnit.SECONDS,
stealJobQueue, new ThreadFactory() {
AtomicInteger longCompactionThreadCounter = new AtomicInteger(0);
@Override
public Thread newThread(Runnable r) {
String name = n + "-longCompactions-" + System.currentTimeMillis();
String name = n + "-longCompactions-" + longCompactionThreadCounter.getAndIncrement();
return new Thread(r, name);
}
});
this.longCompactions.setRejectedExecutionHandler(new Rejection());
this.longCompactions.prestartAllCoreThreads();
this.shortCompactions = new ThreadPoolExecutor(smallThreads, smallThreads, 60,
TimeUnit.SECONDS, stealJobQueue.getStealFromQueue(),
new ThreadFactory() {
this.shortCompactions = new ThreadPoolExecutor(smallThreads, smallThreads, 60, TimeUnit.SECONDS,
stealJobQueue.getStealFromQueue(), new ThreadFactory() {
AtomicInteger shortCompactionThreadCounter = new AtomicInteger(0);
@Override
public Thread newThread(Runnable r) {
String name = n + "-shortCompactions-" + System.currentTimeMillis();
String name = n + "-shortCompactions-" + shortCompactionThreadCounter.getAndIncrement();
return new Thread(r, name);
}
});