1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18 package org.apache.hadoop.hbase.master.cleaner;
19
20 import java.util.concurrent.LinkedBlockingQueue;
21 import java.util.concurrent.ThreadPoolExecutor;
22 import java.util.concurrent.TimeUnit;
23
24 import org.apache.commons.logging.Log;
25 import org.apache.commons.logging.LogFactory;
26 import org.apache.hadoop.conf.Configuration;
27 import org.apache.hadoop.hbase.DaemonThreadFactory;
28 import org.apache.hadoop.hbase.classification.InterfaceAudience;
29 import org.apache.hadoop.hbase.conf.ConfigurationObserver;
30
31
32
33
34 @InterfaceAudience.Private
35 public class DirScanPool implements ConfigurationObserver {
36 private static final Log LOG = LogFactory.getLog(DirScanPool.class);
37 private volatile int size;
38 private final ThreadPoolExecutor pool;
39 private int cleanerLatch;
40 private boolean reconfigNotification;
41
42 public DirScanPool(Configuration conf) {
43 String poolSize = conf.get(CleanerChore.CHORE_POOL_SIZE, CleanerChore.DEFAULT_CHORE_POOL_SIZE);
44 size = CleanerChore.calculatePoolSize(poolSize);
45
46
47 size = size == 0 ? CleanerChore.calculatePoolSize(CleanerChore.DEFAULT_CHORE_POOL_SIZE) : size;
48 pool = initializePool(size);
49 LOG.info("Cleaner pool size is " + size);
50 cleanerLatch = 0;
51 }
52
53 private static ThreadPoolExecutor initializePool(int size) {
54 ThreadPoolExecutor executor = new ThreadPoolExecutor(size, size, 1L, TimeUnit.MINUTES,
55 new LinkedBlockingQueue<Runnable>(), new DaemonThreadFactory("dir-scan-pool"));
56 executor.allowCoreThreadTimeOut(true);
57 return executor;
58 }
59
60
61
62
63
64 @Override
65 public synchronized void onConfigurationChange(Configuration conf) {
66 int newSize = CleanerChore.calculatePoolSize(
67 conf.get(CleanerChore.CHORE_POOL_SIZE, CleanerChore.DEFAULT_CHORE_POOL_SIZE));
68 if (newSize == size) {
69 LOG.trace("Size from configuration is same as previous=" + newSize + ", no need to update.");
70 return;
71 }
72 size = newSize;
73
74 reconfigNotification = true;
75 }
76
77 synchronized void latchCountUp() {
78 cleanerLatch++;
79 }
80
81 synchronized void latchCountDown() {
82 cleanerLatch--;
83 notifyAll();
84 }
85
86 synchronized void execute(Runnable runnable) {
87 this.pool.execute(runnable);
88 }
89
90 public synchronized void shutdownNow() {
91 if (pool == null || pool.isShutdown()) {
92 return;
93 }
94 pool.shutdownNow();
95 }
96
97 synchronized void tryUpdatePoolSize(long timeout) {
98 if (!reconfigNotification) {
99 return;
100 }
101 reconfigNotification = false;
102 long stopTime = System.currentTimeMillis() + timeout;
103 while (cleanerLatch != 0 && timeout > 0) {
104 try {
105 wait(timeout);
106 timeout = stopTime - System.currentTimeMillis();
107 } catch (InterruptedException ie) {
108 Thread.currentThread().interrupt();
109 break;
110 }
111 }
112 LOG.info("Update chore's pool size from " + pool.getPoolSize() + " to " + size);
113 pool.setCorePoolSize(size);
114 }
115
116 public int getSize() {
117 return size;
118 }
119 }