Index: src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java (revision 1159907) +++ src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java (working copy) @@ -22,6 +22,7 @@ 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 @@ * stop splitting after number of online regions is greater than this. */ private int regionSplitLimit; - + /** @param server */ CompactSplitThread(HRegionServer server) { super(); @@ -90,20 +91,46 @@ // 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()); + 60, TimeUnit.SECONDS, new PriorityBlockingQueue(), + 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()); + 60, TimeUnit.SECONDS, new PriorityBlockingQueue(), + 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