1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.regionserver;
20
21 import java.io.IOException;
22 import java.util.ArrayList;
23 import java.util.Iterator;
24 import java.util.List;
25 import java.util.concurrent.BlockingQueue;
26 import java.util.concurrent.Executors;
27 import java.util.concurrent.PriorityBlockingQueue;
28 import java.util.concurrent.RejectedExecutionException;
29 import java.util.concurrent.RejectedExecutionHandler;
30 import java.util.concurrent.ThreadFactory;
31 import java.util.concurrent.ThreadPoolExecutor;
32 import java.util.concurrent.TimeUnit;
33
34 import org.apache.commons.logging.Log;
35 import org.apache.commons.logging.LogFactory;
36 import org.apache.hadoop.classification.InterfaceAudience;
37 import org.apache.hadoop.conf.Configuration;
38 import org.apache.hadoop.hbase.RemoteExceptionHandler;
39 import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
40 import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
41 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
42 import org.apache.hadoop.hbase.util.Pair;
43 import org.apache.hadoop.util.StringUtils;
44
45 import com.google.common.base.Preconditions;
46
47
48
49
50 @InterfaceAudience.Private
51 public class CompactSplitThread implements CompactionRequestor {
52 static final Log LOG = LogFactory.getLog(CompactSplitThread.class);
53
54 private final HRegionServer server;
55 private final Configuration conf;
56
57 private final ThreadPoolExecutor largeCompactions;
58 private final ThreadPoolExecutor smallCompactions;
59 private final ThreadPoolExecutor splits;
60 private final ThreadPoolExecutor mergePool;
61
62
63
64
65
66
67 private int regionSplitLimit;
68
69
70 CompactSplitThread(HRegionServer server) {
71 super();
72 this.server = server;
73 this.conf = server.getConfiguration();
74 this.regionSplitLimit = conf.getInt("hbase.regionserver.regionSplitLimit",
75 Integer.MAX_VALUE);
76
77 int largeThreads = Math.max(1, conf.getInt(
78 "hbase.regionserver.thread.compaction.large", 1));
79 int smallThreads = conf.getInt(
80 "hbase.regionserver.thread.compaction.small", 1);
81
82 int splitThreads = conf.getInt("hbase.regionserver.thread.split", 1);
83
84
85 Preconditions.checkArgument(largeThreads > 0 && smallThreads > 0);
86
87 final String n = Thread.currentThread().getName();
88
89 this.largeCompactions = new ThreadPoolExecutor(largeThreads, largeThreads,
90 60, TimeUnit.SECONDS, new PriorityBlockingQueue<Runnable>(),
91 new ThreadFactory() {
92 @Override
93 public Thread newThread(Runnable r) {
94 Thread t = new Thread(r);
95 t.setName(n + "-largeCompactions-" + System.currentTimeMillis());
96 return t;
97 }
98 });
99 this.largeCompactions.setRejectedExecutionHandler(new Rejection());
100 this.smallCompactions = new ThreadPoolExecutor(smallThreads, smallThreads,
101 60, TimeUnit.SECONDS, new PriorityBlockingQueue<Runnable>(),
102 new ThreadFactory() {
103 @Override
104 public Thread newThread(Runnable r) {
105 Thread t = new Thread(r);
106 t.setName(n + "-smallCompactions-" + System.currentTimeMillis());
107 return t;
108 }
109 });
110 this.smallCompactions
111 .setRejectedExecutionHandler(new Rejection());
112 this.splits = (ThreadPoolExecutor)
113 Executors.newFixedThreadPool(splitThreads,
114 new ThreadFactory() {
115 @Override
116 public Thread newThread(Runnable r) {
117 Thread t = new Thread(r);
118 t.setName(n + "-splits-" + System.currentTimeMillis());
119 return t;
120 }
121 });
122 int mergeThreads = conf.getInt("hbase.regionserver.thread.merge", 1);
123 this.mergePool = (ThreadPoolExecutor) Executors.newFixedThreadPool(
124 mergeThreads, new ThreadFactory() {
125 @Override
126 public Thread newThread(Runnable r) {
127 Thread t = new Thread(r);
128 t.setName(n + "-merges-" + System.currentTimeMillis());
129 return t;
130 }
131 });
132 }
133
134 @Override
135 public String toString() {
136 return "compaction_queue=("
137 + largeCompactions.getQueue().size() + ":"
138 + smallCompactions.getQueue().size() + ")"
139 + ", split_queue=" + splits.getQueue().size()
140 + ", merge_queue=" + mergePool.getQueue().size();
141 }
142
143 public String dumpQueue() {
144 StringBuffer queueLists = new StringBuffer();
145 queueLists.append("Compaction/Split Queue dump:\n");
146 queueLists.append(" LargeCompation Queue:\n");
147 BlockingQueue<Runnable> lq = largeCompactions.getQueue();
148 Iterator it = lq.iterator();
149 while(it.hasNext()){
150 queueLists.append(" "+it.next().toString());
151 queueLists.append("\n");
152 }
153
154 if( smallCompactions != null ){
155 queueLists.append("\n");
156 queueLists.append(" SmallCompation Queue:\n");
157 lq = smallCompactions.getQueue();
158 it = lq.iterator();
159 while(it.hasNext()){
160 queueLists.append(" "+it.next().toString());
161 queueLists.append("\n");
162 }
163 }
164
165 queueLists.append("\n");
166 queueLists.append(" Split Queue:\n");
167 lq = splits.getQueue();
168 it = lq.iterator();
169 while(it.hasNext()){
170 queueLists.append(" "+it.next().toString());
171 queueLists.append("\n");
172 }
173
174 queueLists.append("\n");
175 queueLists.append(" Region Merge Queue:\n");
176 lq = mergePool.getQueue();
177 it = lq.iterator();
178 while (it.hasNext()) {
179 queueLists.append(" " + it.next().toString());
180 queueLists.append("\n");
181 }
182
183 return queueLists.toString();
184 }
185
186 public synchronized void requestRegionsMerge(final HRegion a,
187 final HRegion b, final boolean forcible) {
188 try {
189 mergePool.execute(new RegionMergeRequest(a, b, this.server, forcible));
190 if (LOG.isDebugEnabled()) {
191 LOG.debug("Region merge requested for " + a + "," + b + ", forcible="
192 + forcible + ". " + this);
193 }
194 } catch (RejectedExecutionException ree) {
195 LOG.warn("Could not execute merge for " + a + "," + b + ", forcible="
196 + forcible, ree);
197 }
198 }
199
200 public synchronized boolean requestSplit(final HRegion r) {
201
202 if (shouldSplitRegion() && r.getCompactPriority() >= Store.PRIORITY_USER) {
203 byte[] midKey = r.checkSplit();
204 if (midKey != null) {
205 requestSplit(r, midKey);
206 return true;
207 }
208 }
209 return false;
210 }
211
212 public synchronized void requestSplit(final HRegion r, byte[] midKey) {
213 if (midKey == null) {
214 LOG.debug("Region " + r.getRegionNameAsString() +
215 " not splittable because midkey=null");
216 return;
217 }
218 try {
219 this.splits.execute(new SplitRequest(r, midKey, this.server));
220 if (LOG.isDebugEnabled()) {
221 LOG.debug("Split requested for " + r + ". " + this);
222 }
223 } catch (RejectedExecutionException ree) {
224 LOG.info("Could not execute split for " + r, ree);
225 }
226 }
227
228 @Override
229 public synchronized List<CompactionRequest> requestCompaction(final HRegion r, final String why)
230 throws IOException {
231 return requestCompaction(r, why, null);
232 }
233
234 @Override
235 public synchronized List<CompactionRequest> requestCompaction(final HRegion r, final String why,
236 List<Pair<CompactionRequest, Store>> requests) throws IOException {
237 return requestCompaction(r, why, Store.NO_PRIORITY, requests);
238 }
239
240 @Override
241 public synchronized CompactionRequest requestCompaction(final HRegion r, final Store s,
242 final String why, CompactionRequest request) throws IOException {
243 return requestCompaction(r, s, why, Store.NO_PRIORITY, request);
244 }
245
246 @Override
247 public synchronized List<CompactionRequest> requestCompaction(final HRegion r, final String why,
248 int p, List<Pair<CompactionRequest, Store>> requests) throws IOException {
249
250 List<CompactionRequest> ret;
251 if (requests == null) {
252 ret = new ArrayList<CompactionRequest>(r.getStores().size());
253 for (Store s : r.getStores().values()) {
254 ret.add(requestCompaction(r, s, why, p, null));
255 }
256 } else {
257 ret = new ArrayList<CompactionRequest>(requests.size());
258 for (Pair<CompactionRequest, Store> pair : requests) {
259 ret.add(requestCompaction(r, pair.getSecond(), why, p, pair.getFirst()));
260 }
261 }
262 return ret;
263 }
264
265
266
267
268
269
270
271
272
273 public synchronized CompactionRequest requestCompaction(final HRegion r, final Store s,
274 final String why, int priority, CompactionRequest request) throws IOException {
275 if (this.server.isStopped()) {
276 return null;
277 }
278 CompactionContext compaction = s.requestCompaction(priority, request);
279 if (compaction == null) {
280 if(LOG.isDebugEnabled()) {
281 LOG.debug("Not compacting " + r.getRegionNameAsString() +
282 " because compaction request was cancelled");
283 }
284 return null;
285 }
286
287 assert compaction.hasSelection();
288 if (priority != Store.NO_PRIORITY) {
289 compaction.getRequest().setPriority(priority);
290 }
291 ThreadPoolExecutor pool = s.throttleCompaction(compaction.getRequest().getSize())
292 ? largeCompactions : smallCompactions;
293 pool.execute(new CompactionRunner(s, r, compaction));
294 if (LOG.isDebugEnabled()) {
295 String type = (pool == smallCompactions) ? "Small " : "Large ";
296 LOG.debug(type + "Compaction requested: " + compaction
297 + (why != null && !why.isEmpty() ? "; Because: " + why : "")
298 + "; " + this);
299 }
300 return compaction.getRequest();
301 }
302
303
304
305
306 void interruptIfNecessary() {
307 splits.shutdown();
308 mergePool.shutdown();
309 largeCompactions.shutdown();
310 smallCompactions.shutdown();
311 }
312
313 private void waitFor(ThreadPoolExecutor t, String name) {
314 boolean done = false;
315 while (!done) {
316 try {
317 done = t.awaitTermination(60, TimeUnit.SECONDS);
318 LOG.info("Waiting for " + name + " to finish...");
319 if (!done) {
320 t.shutdownNow();
321 }
322 } catch (InterruptedException ie) {
323 LOG.warn("Interrupted waiting for " + name + " to finish...");
324 }
325 }
326 }
327
328 void join() {
329 waitFor(splits, "Split Thread");
330 waitFor(mergePool, "Merge Thread");
331 waitFor(largeCompactions, "Large Compaction Thread");
332 waitFor(smallCompactions, "Small Compaction Thread");
333 }
334
335
336
337
338
339
340
341 public int getCompactionQueueSize() {
342 return largeCompactions.getQueue().size() + smallCompactions.getQueue().size();
343 }
344
345 private boolean shouldSplitRegion() {
346 return (regionSplitLimit > server.getNumberOfOnlineRegions());
347 }
348
349
350
351
352 public int getRegionSplitLimit() {
353 return this.regionSplitLimit;
354 }
355
356 private class CompactionRunner implements Runnable, Comparable<CompactionRunner> {
357 private final Store store;
358 private final HRegion region;
359 private final CompactionContext compaction;
360
361 public CompactionRunner(Store store, HRegion region, CompactionContext compaction) {
362 super();
363 this.store = store;
364 this.region = region;
365 this.compaction = compaction;
366 }
367
368 @Override
369 public void run() {
370 Preconditions.checkNotNull(server);
371 if (server.isStopped()) {
372 return;
373 }
374 this.compaction.getRequest().beforeExecute();
375 try {
376
377
378 long start = EnvironmentEdgeManager.currentTimeMillis();
379 boolean completed = region.compact(compaction, store);
380 long now = EnvironmentEdgeManager.currentTimeMillis();
381 LOG.info(((completed) ? "Completed" : "Aborted") + " compaction: " +
382 this + "; duration=" + StringUtils.formatTimeDiff(now, start));
383 if (completed) {
384
385 if (store.getCompactPriority() <= 0) {
386 requestCompaction(region, store, "Recursive enqueue", null);
387 } else {
388
389 requestSplit(region);
390 }
391 }
392 } catch (IOException ex) {
393 LOG.error("Compaction failed " + this, RemoteExceptionHandler.checkIOException(ex));
394 server.checkFileSystem();
395 } catch (Exception ex) {
396 LOG.error("Compaction failed " + this, ex);
397 server.checkFileSystem();
398 } finally {
399 LOG.debug("CompactSplitThread Status: " + CompactSplitThread.this);
400 }
401 this.compaction.getRequest().afterExecute();
402 }
403
404 @Override
405 public int compareTo(CompactionRunner o) {
406
407 return this.compaction.getRequest().compareTo(o.compaction.getRequest());
408 }
409 }
410
411
412
413
414 private static class Rejection implements RejectedExecutionHandler {
415 @Override
416 public void rejectedExecution(Runnable runnable, ThreadPoolExecutor pool) {
417 if (runnable instanceof CompactionRunner) {
418 CompactionRunner runner = (CompactionRunner)runnable;
419 LOG.debug("Compaction Rejected: " + runner);
420 runner.store.cancelRequestedCompaction(runner.compaction);
421 }
422 }
423 }
424 }