HBASE-4230 Compaction threads need names
git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1159915 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
38a046ea3c
commit
1c90f10c7f
|
@ -385,6 +385,7 @@ Release 0.91.0 - Unreleased
|
|||
not shown (Nileema Shingte)
|
||||
HBASE-4229 Replace Jettison JSON encoding with Jackson in HLogPrettyPrinter
|
||||
(Riley Patterson)
|
||||
HBASE-4230 Compaction threads need names
|
||||
|
||||
TASKS
|
||||
HBASE-3559 Move report of split to master OFF the heartbeat channel
|
||||
|
|
|
@ -22,6 +22,7 @@ package org.apache.hadoop.hbase.regionserver;
|
|||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.PriorityBlockingQueue;
|
||||
import java.util.concurrent.RejectedExecutionException;
|
||||
import java.util.concurrent.ThreadFactory;
|
||||
import java.util.concurrent.ThreadPoolExecutor;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
|
@ -60,7 +61,7 @@ public class CompactSplitThread implements CompactionRequestor {
|
|||
* stop splitting after number of online regions is greater than this.
|
||||
*/
|
||||
private int regionSplitLimit;
|
||||
|
||||
|
||||
/** @param server */
|
||||
CompactSplitThread(HRegionServer server) {
|
||||
super();
|
||||
|
@ -90,20 +91,46 @@ public class CompactSplitThread implements CompactionRequestor {
|
|||
// if we have throttle threads, make sure the user also specified size
|
||||
Preconditions.checkArgument(smallThreads == 0 || throttleSize > 0);
|
||||
|
||||
final String n = Thread.currentThread().getName();
|
||||
|
||||
this.largeCompactions = new ThreadPoolExecutor(largeThreads, largeThreads,
|
||||
60, TimeUnit.SECONDS, new PriorityBlockingQueue<Runnable>());
|
||||
60, TimeUnit.SECONDS, new PriorityBlockingQueue<Runnable>(),
|
||||
new ThreadFactory() {
|
||||
@Override
|
||||
public Thread newThread(Runnable r) {
|
||||
Thread t = new Thread(r);
|
||||
t.setName(n + "-largeCompactions-" + System.currentTimeMillis());
|
||||
return t;
|
||||
}
|
||||
});
|
||||
this.largeCompactions
|
||||
.setRejectedExecutionHandler(new CompactionRequest.Rejection());
|
||||
if (smallThreads <= 0) {
|
||||
this.smallCompactions = null;
|
||||
} else {
|
||||
this.smallCompactions = new ThreadPoolExecutor(smallThreads, smallThreads,
|
||||
60, TimeUnit.SECONDS, new PriorityBlockingQueue<Runnable>());
|
||||
60, TimeUnit.SECONDS, new PriorityBlockingQueue<Runnable>(),
|
||||
new ThreadFactory() {
|
||||
@Override
|
||||
public Thread newThread(Runnable r) {
|
||||
Thread t = new Thread(r);
|
||||
t.setName(n + "-smallCompactions-" + System.currentTimeMillis());
|
||||
return t;
|
||||
}
|
||||
});
|
||||
this.smallCompactions
|
||||
.setRejectedExecutionHandler(new CompactionRequest.Rejection());
|
||||
}
|
||||
this.splits = (ThreadPoolExecutor) Executors
|
||||
.newFixedThreadPool(splitThreads);
|
||||
this.splits = (ThreadPoolExecutor)
|
||||
Executors.newFixedThreadPool(splitThreads,
|
||||
new ThreadFactory() {
|
||||
@Override
|
||||
public Thread newThread(Runnable r) {
|
||||
Thread t = new Thread(r);
|
||||
t.setName(n + "-splits-" + System.currentTimeMillis());
|
||||
return t;
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
Loading…
Reference in New Issue