1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20 package org.apache.hadoop.hbase.regionserver;
21
22 import java.util.concurrent.Executors;
23 import java.util.concurrent.PriorityBlockingQueue;
24 import java.util.concurrent.RejectedExecutionException;
25 import java.util.concurrent.ThreadFactory;
26 import java.util.concurrent.ThreadPoolExecutor;
27 import java.util.concurrent.TimeUnit;
28
29 import org.apache.commons.logging.Log;
30 import org.apache.commons.logging.LogFactory;
31 import org.apache.hadoop.conf.Configuration;
32 import org.apache.hadoop.hbase.HConstants;
33 import org.apache.hadoop.hbase.HTableDescriptor;
34 import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
35
36 import com.google.common.base.Preconditions;
37
38
39
40
41 public class CompactSplitThread implements CompactionRequestor {
42 static final Log LOG = LogFactory.getLog(CompactSplitThread.class);
43
44 private final HRegionServer server;
45 private final Configuration conf;
46
47 private final ThreadPoolExecutor largeCompactions;
48 private final ThreadPoolExecutor smallCompactions;
49 private final ThreadPoolExecutor splits;
50 private final long throttleSize;
51
52
53
54
55 public static final int PRIORITY_USER = 1;
56 public static final int NO_PRIORITY = Integer.MIN_VALUE;
57
58
59
60
61
62
63 private int regionSplitLimit;
64
65
66 CompactSplitThread(HRegionServer server) {
67 super();
68 this.server = server;
69 this.conf = server.getConfiguration();
70 this.regionSplitLimit = conf.getInt("hbase.regionserver.regionSplitLimit",
71 Integer.MAX_VALUE);
72
73 int largeThreads = Math.max(1, conf.getInt(
74 "hbase.regionserver.thread.compaction.large", 1));
75 int smallThreads = conf.getInt(
76 "hbase.regionserver.thread.compaction.small", 1);
77 if (conf.get("hbase.regionserver.thread.compaction.throttle") != null) {
78 throttleSize = conf.getLong(
79 "hbase.regionserver.thread.compaction.throttle", 0);
80 } else {
81
82 long flushSize = conf.getLong("hbase.hregion.memstore.flush.size",
83 HTableDescriptor.DEFAULT_MEMSTORE_FLUSH_SIZE);
84 long splitSize = conf.getLong("hbase.hregion.max.filesize",
85 HConstants.DEFAULT_MAX_FILE_SIZE);
86 throttleSize = Math.min(flushSize * 2, splitSize / 2);
87 }
88
89 int splitThreads = conf.getInt("hbase.regionserver.thread.split", 1);
90
91
92 Preconditions.checkArgument(smallThreads == 0 || throttleSize > 0);
93
94 final String n = Thread.currentThread().getName();
95
96 this.largeCompactions = new ThreadPoolExecutor(largeThreads, largeThreads,
97 60, TimeUnit.SECONDS, new PriorityBlockingQueue<Runnable>(),
98 new ThreadFactory() {
99 @Override
100 public Thread newThread(Runnable r) {
101 Thread t = new Thread(r);
102 t.setName(n + "-largeCompactions-" + System.currentTimeMillis());
103 return t;
104 }
105 });
106 this.largeCompactions
107 .setRejectedExecutionHandler(new CompactionRequest.Rejection());
108 if (smallThreads <= 0) {
109 this.smallCompactions = null;
110 } else {
111 this.smallCompactions = new ThreadPoolExecutor(smallThreads, smallThreads,
112 60, TimeUnit.SECONDS, new PriorityBlockingQueue<Runnable>(),
113 new ThreadFactory() {
114 @Override
115 public Thread newThread(Runnable r) {
116 Thread t = new Thread(r);
117 t.setName(n + "-smallCompactions-" + System.currentTimeMillis());
118 return t;
119 }
120 });
121 this.smallCompactions
122 .setRejectedExecutionHandler(new CompactionRequest.Rejection());
123 }
124 this.splits = (ThreadPoolExecutor)
125 Executors.newFixedThreadPool(splitThreads,
126 new ThreadFactory() {
127 @Override
128 public Thread newThread(Runnable r) {
129 Thread t = new Thread(r);
130 t.setName(n + "-splits-" + System.currentTimeMillis());
131 return t;
132 }
133 });
134 }
135
136 @Override
137 public String toString() {
138 return "compaction_queue="
139 + (smallCompactions != null ? "("
140 + largeCompactions.getQueue().size() + ":"
141 + smallCompactions.getQueue().size() + ")"
142 : largeCompactions.getQueue().size())
143 + ", split_queue=" + splits.getQueue().size();
144 }
145
146 public synchronized boolean requestSplit(final HRegion r) {
147
148 if (shouldSplitRegion() && r.getCompactPriority() >= PRIORITY_USER) {
149 byte[] midKey = r.checkSplit();
150 if (midKey != null) {
151 requestSplit(r, midKey);
152 return true;
153 }
154 }
155 return false;
156 }
157
158 public synchronized void requestSplit(final HRegion r, byte[] midKey) {
159 if (midKey == null) {
160 LOG.debug("Region " + r.getRegionNameAsString() +
161 " not splittable because midkey=null");
162 return;
163 }
164 try {
165 this.splits.execute(new SplitRequest(r, midKey, this.server));
166 if (LOG.isDebugEnabled()) {
167 LOG.debug("Split requested for " + r + ". " + this);
168 }
169 } catch (RejectedExecutionException ree) {
170 LOG.info("Could not execute split for " + r, ree);
171 }
172 }
173
174 public synchronized void requestCompaction(final HRegion r,
175 final String why) {
176 for(Store s : r.getStores().values()) {
177 requestCompaction(r, s, why, NO_PRIORITY);
178 }
179 }
180
181 public synchronized void requestCompaction(final HRegion r, final Store s,
182 final String why) {
183 requestCompaction(r, s, why, NO_PRIORITY);
184 }
185
186 public synchronized void requestCompaction(final HRegion r, final String why,
187 int p) {
188 for(Store s : r.getStores().values()) {
189 requestCompaction(r, s, why, p);
190 }
191 }
192
193
194
195
196
197
198
199 public synchronized void requestCompaction(final HRegion r, final Store s,
200 final String why, int priority) {
201 if (this.server.isStopped()) {
202 return;
203 }
204 CompactionRequest cr = s.requestCompaction();
205 if (cr != null) {
206 cr.setServer(server);
207 if (priority != NO_PRIORITY) {
208 cr.setPriority(priority);
209 }
210 ThreadPoolExecutor pool = largeCompactions;
211 if (smallCompactions != null && throttleSize > cr.getSize()) {
212
213 pool = smallCompactions;
214 }
215 pool.execute(cr);
216 if (LOG.isDebugEnabled()) {
217 String type = "";
218 if (smallCompactions != null) {
219 type = (pool == smallCompactions) ? "Small " : "Large ";
220 }
221 LOG.debug(type + "Compaction requested: " + cr
222 + (why != null && !why.isEmpty() ? "; Because: " + why : "")
223 + "; " + this);
224 }
225 }
226 }
227
228
229
230
231 void interruptIfNecessary() {
232 splits.shutdown();
233 largeCompactions.shutdown();
234 if (smallCompactions != null)
235 smallCompactions.shutdown();
236 }
237
238 private void waitFor(ThreadPoolExecutor t, String name) {
239 boolean done = false;
240 while (!done) {
241 try {
242 done = t.awaitTermination(60, TimeUnit.SECONDS);
243 LOG.debug("Waiting for " + name + " to finish...");
244 } catch (InterruptedException ie) {
245 LOG.debug("Interrupted waiting for " + name + " to finish...");
246 }
247 }
248 }
249
250 void join() {
251 waitFor(splits, "Split Thread");
252 waitFor(largeCompactions, "Large Compaction Thread");
253 if (smallCompactions != null) {
254 waitFor(smallCompactions, "Small Compaction Thread");
255 }
256 }
257
258
259
260
261
262
263
264 public int getCompactionQueueSize() {
265 int size = largeCompactions.getQueue().size();
266 if (smallCompactions != null)
267 size += smallCompactions.getQueue().size();
268 return size;
269 }
270
271 private boolean shouldSplitRegion() {
272 return (regionSplitLimit > server.getNumberOfOnlineRegions());
273 }
274
275
276
277
278 public int getRegionSplitLimit() {
279 return this.regionSplitLimit;
280 }
281 }