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.io.PrintWriter;
23 import java.io.StringWriter;
24 import java.util.ArrayList;
25 import java.util.Iterator;
26 import java.util.List;
27 import java.util.concurrent.BlockingQueue;
28 import java.util.concurrent.Executors;
29 import java.util.concurrent.PriorityBlockingQueue;
30 import java.util.concurrent.RejectedExecutionException;
31 import java.util.concurrent.RejectedExecutionHandler;
32 import java.util.concurrent.ThreadFactory;
33 import java.util.concurrent.ThreadPoolExecutor;
34 import java.util.concurrent.TimeUnit;
35
36 import org.apache.commons.logging.Log;
37 import org.apache.commons.logging.LogFactory;
38 import org.apache.hadoop.hbase.classification.InterfaceAudience;
39 import org.apache.hadoop.hbase.conf.ConfigurationManager;
40 import org.apache.hadoop.hbase.conf.PropagatingConfigurationObserver;
41 import org.apache.hadoop.conf.Configuration;
42 import org.apache.hadoop.hbase.RemoteExceptionHandler;
43 import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
44 import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
45 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
46 import org.apache.hadoop.hbase.util.Pair;
47 import org.apache.hadoop.util.StringUtils;
48
49 import com.google.common.base.Preconditions;
50
51
52
53
54 @InterfaceAudience.Private
55 public class CompactSplitThread implements CompactionRequestor, PropagatingConfigurationObserver {
56 static final Log LOG = LogFactory.getLog(CompactSplitThread.class);
57
58
59 public final static String LARGE_COMPACTION_THREADS =
60 "hbase.regionserver.thread.compaction.large";
61 public final static int LARGE_COMPACTION_THREADS_DEFAULT = 1;
62
63
64 public final static String SMALL_COMPACTION_THREADS =
65 "hbase.regionserver.thread.compaction.small";
66 public final static int SMALL_COMPACTION_THREADS_DEFAULT = 1;
67
68
69 public final static String SPLIT_THREADS = "hbase.regionserver.thread.split";
70 public final static int SPLIT_THREADS_DEFAULT = 1;
71
72
73 public final static String MERGE_THREADS = "hbase.regionserver.thread.merge";
74 public final static int MERGE_THREADS_DEFAULT = 1;
75
76 public static final String REGION_SERVER_REGION_SPLIT_LIMIT =
77 "hbase.regionserver.regionSplitLimit";
78 public static final int DEFAULT_REGION_SERVER_REGION_SPLIT_LIMIT= 1000;
79
80 private final HRegionServer server;
81 private final Configuration conf;
82
83 private final ThreadPoolExecutor longCompactions;
84 private final ThreadPoolExecutor shortCompactions;
85 private final ThreadPoolExecutor splits;
86 private final ThreadPoolExecutor mergePool;
87
88
89
90
91
92
93 private int regionSplitLimit;
94
95
96 CompactSplitThread(HRegionServer server) {
97 super();
98 this.server = server;
99 this.conf = server.getConfiguration();
100 this.regionSplitLimit = conf.getInt(REGION_SERVER_REGION_SPLIT_LIMIT,
101 DEFAULT_REGION_SERVER_REGION_SPLIT_LIMIT);
102
103 int largeThreads = Math.max(1, conf.getInt(
104 LARGE_COMPACTION_THREADS, LARGE_COMPACTION_THREADS_DEFAULT));
105 int smallThreads = conf.getInt(
106 SMALL_COMPACTION_THREADS, SMALL_COMPACTION_THREADS_DEFAULT);
107
108 int splitThreads = conf.getInt(SPLIT_THREADS, SPLIT_THREADS_DEFAULT);
109
110
111 Preconditions.checkArgument(largeThreads > 0 && smallThreads > 0);
112
113 final String n = Thread.currentThread().getName();
114
115 this.longCompactions = new ThreadPoolExecutor(largeThreads, largeThreads,
116 60, TimeUnit.SECONDS, new PriorityBlockingQueue<Runnable>(),
117 new ThreadFactory() {
118 @Override
119 public Thread newThread(Runnable r) {
120 Thread t = new Thread(r);
121 t.setName(n + "-longCompactions-" + System.currentTimeMillis());
122 return t;
123 }
124 });
125 this.longCompactions.setRejectedExecutionHandler(new Rejection());
126 this.shortCompactions = new ThreadPoolExecutor(smallThreads, smallThreads,
127 60, TimeUnit.SECONDS, new PriorityBlockingQueue<Runnable>(),
128 new ThreadFactory() {
129 @Override
130 public Thread newThread(Runnable r) {
131 Thread t = new Thread(r);
132 t.setName(n + "-shortCompactions-" + System.currentTimeMillis());
133 return t;
134 }
135 });
136 this.shortCompactions
137 .setRejectedExecutionHandler(new Rejection());
138 this.splits = (ThreadPoolExecutor)
139 Executors.newFixedThreadPool(splitThreads,
140 new ThreadFactory() {
141 @Override
142 public Thread newThread(Runnable r) {
143 Thread t = new Thread(r);
144 t.setName(n + "-splits-" + System.currentTimeMillis());
145 return t;
146 }
147 });
148 int mergeThreads = conf.getInt(MERGE_THREADS, MERGE_THREADS_DEFAULT);
149 this.mergePool = (ThreadPoolExecutor) Executors.newFixedThreadPool(
150 mergeThreads, new ThreadFactory() {
151 @Override
152 public Thread newThread(Runnable r) {
153 Thread t = new Thread(r);
154 t.setName(n + "-merges-" + System.currentTimeMillis());
155 return t;
156 }
157 });
158 }
159
160 @Override
161 public String toString() {
162 return "compaction_queue=("
163 + longCompactions.getQueue().size() + ":"
164 + shortCompactions.getQueue().size() + ")"
165 + ", split_queue=" + splits.getQueue().size()
166 + ", merge_queue=" + mergePool.getQueue().size();
167 }
168
169 public String dumpQueue() {
170 StringBuffer queueLists = new StringBuffer();
171 queueLists.append("Compaction/Split Queue dump:\n");
172 queueLists.append(" LargeCompation Queue:\n");
173 BlockingQueue<Runnable> lq = longCompactions.getQueue();
174 Iterator<Runnable> it = lq.iterator();
175 while(it.hasNext()){
176 queueLists.append(" "+it.next().toString());
177 queueLists.append("\n");
178 }
179
180 if( shortCompactions != null ){
181 queueLists.append("\n");
182 queueLists.append(" SmallCompation Queue:\n");
183 lq = shortCompactions.getQueue();
184 it = lq.iterator();
185 while(it.hasNext()){
186 queueLists.append(" "+it.next().toString());
187 queueLists.append("\n");
188 }
189 }
190
191 queueLists.append("\n");
192 queueLists.append(" Split Queue:\n");
193 lq = splits.getQueue();
194 it = lq.iterator();
195 while(it.hasNext()){
196 queueLists.append(" "+it.next().toString());
197 queueLists.append("\n");
198 }
199
200 queueLists.append("\n");
201 queueLists.append(" Region Merge Queue:\n");
202 lq = mergePool.getQueue();
203 it = lq.iterator();
204 while (it.hasNext()) {
205 queueLists.append(" " + it.next().toString());
206 queueLists.append("\n");
207 }
208
209 return queueLists.toString();
210 }
211
212 public synchronized void requestRegionsMerge(final HRegion a,
213 final HRegion b, final boolean forcible) {
214 try {
215 mergePool.execute(new RegionMergeRequest(a, b, this.server, forcible));
216 if (LOG.isDebugEnabled()) {
217 LOG.debug("Region merge requested for " + a + "," + b + ", forcible="
218 + forcible + ". " + this);
219 }
220 } catch (RejectedExecutionException ree) {
221 LOG.warn("Could not execute merge for " + a + "," + b + ", forcible="
222 + forcible, ree);
223 }
224 }
225
226 public synchronized boolean requestSplit(final HRegion r) {
227
228 if (shouldSplitRegion() && r.getCompactPriority() >= Store.PRIORITY_USER) {
229 byte[] midKey = r.checkSplit();
230 if (midKey != null) {
231 requestSplit(r, midKey);
232 return true;
233 }
234 }
235 return false;
236 }
237
238 public synchronized void requestSplit(final HRegion r, byte[] midKey) {
239 if (midKey == null) {
240 LOG.debug("Region " + r.getRegionNameAsString() +
241 " not splittable because midkey=null");
242 if (r.shouldForceSplit()) {
243 r.clearSplit();
244 }
245 return;
246 }
247 try {
248 this.splits.execute(new SplitRequest(r, midKey, this.server));
249 if (LOG.isDebugEnabled()) {
250 LOG.debug("Split requested for " + r + ". " + this);
251 }
252 } catch (RejectedExecutionException ree) {
253 LOG.info("Could not execute split for " + r, ree);
254 }
255 }
256
257 @Override
258 public synchronized List<CompactionRequest> requestCompaction(final HRegion r, final String why)
259 throws IOException {
260 return requestCompaction(r, why, null);
261 }
262
263 @Override
264 public synchronized List<CompactionRequest> requestCompaction(final HRegion r, final String why,
265 List<Pair<CompactionRequest, Store>> requests) throws IOException {
266 return requestCompaction(r, why, Store.NO_PRIORITY, requests);
267 }
268
269 @Override
270 public synchronized CompactionRequest requestCompaction(final HRegion r, final Store s,
271 final String why, CompactionRequest request) throws IOException {
272 return requestCompaction(r, s, why, Store.NO_PRIORITY, request);
273 }
274
275 @Override
276 public synchronized List<CompactionRequest> requestCompaction(final HRegion r, final String why,
277 int p, List<Pair<CompactionRequest, Store>> requests) throws IOException {
278 return requestCompactionInternal(r, why, p, requests, true);
279 }
280
281 private List<CompactionRequest> requestCompactionInternal(final HRegion r, final String why,
282 int p, List<Pair<CompactionRequest, Store>> requests, boolean selectNow) throws IOException {
283
284 List<CompactionRequest> ret = null;
285 if (requests == null) {
286 ret = selectNow ? new ArrayList<CompactionRequest>(r.getStores().size()) : null;
287 for (Store s : r.getStores().values()) {
288 CompactionRequest cr = requestCompactionInternal(r, s, why, p, null, selectNow);
289 if (selectNow) ret.add(cr);
290 }
291 } else {
292 Preconditions.checkArgument(selectNow);
293 ret = new ArrayList<CompactionRequest>(requests.size());
294 for (Pair<CompactionRequest, Store> pair : requests) {
295 ret.add(requestCompaction(r, pair.getSecond(), why, p, pair.getFirst()));
296 }
297 }
298 return ret;
299 }
300
301 public CompactionRequest requestCompaction(final HRegion r, final Store s,
302 final String why, int priority, CompactionRequest request) throws IOException {
303 return requestCompactionInternal(r, s, why, priority, request, true);
304 }
305
306 public synchronized void requestSystemCompaction(
307 final HRegion r, final String why) throws IOException {
308 requestCompactionInternal(r, why, Store.NO_PRIORITY, null, false);
309 }
310
311 public void requestSystemCompaction(
312 final HRegion r, final Store s, final String why) throws IOException {
313 requestCompactionInternal(r, s, why, Store.NO_PRIORITY, null, false);
314 }
315
316
317
318
319
320
321
322
323
324 private synchronized CompactionRequest requestCompactionInternal(final HRegion r, final Store s,
325 final String why, int priority, CompactionRequest request, boolean selectNow)
326 throws IOException {
327 if (this.server.isStopped()
328 || (r.getTableDesc() != null && !r.getTableDesc().isCompactionEnabled())) {
329 return null;
330 }
331
332 CompactionContext compaction = null;
333 if (selectNow) {
334 compaction = selectCompaction(r, s, priority, request);
335 if (compaction == null) return null;
336 }
337
338
339
340 long size = selectNow ? compaction.getRequest().getSize() : 0;
341 ThreadPoolExecutor pool = (!selectNow && s.throttleCompaction(size))
342 ? longCompactions : shortCompactions;
343 pool.execute(new CompactionRunner(s, r, compaction, pool));
344 if (LOG.isDebugEnabled()) {
345 String type = (pool == shortCompactions) ? "Small " : "Large ";
346 LOG.debug(type + "Compaction requested: " + (selectNow ? compaction.toString() : "system")
347 + (why != null && !why.isEmpty() ? "; Because: " + why : "") + "; " + this);
348 }
349 return selectNow ? compaction.getRequest() : null;
350 }
351
352 private CompactionContext selectCompaction(final HRegion r, final Store s,
353 int priority, CompactionRequest request) throws IOException {
354 CompactionContext compaction = s.requestCompaction(priority, request);
355 if (compaction == null) {
356 if(LOG.isDebugEnabled()) {
357 LOG.debug("Not compacting " + r.getRegionNameAsString() +
358 " because compaction request was cancelled");
359 }
360 return null;
361 }
362 assert compaction.hasSelection();
363 if (priority != Store.NO_PRIORITY) {
364 compaction.getRequest().setPriority(priority);
365 }
366 return compaction;
367 }
368
369
370
371
372 void interruptIfNecessary() {
373 splits.shutdown();
374 mergePool.shutdown();
375 longCompactions.shutdown();
376 shortCompactions.shutdown();
377 }
378
379 private void waitFor(ThreadPoolExecutor t, String name) {
380 boolean done = false;
381 while (!done) {
382 try {
383 done = t.awaitTermination(60, TimeUnit.SECONDS);
384 LOG.info("Waiting for " + name + " to finish...");
385 if (!done) {
386 t.shutdownNow();
387 }
388 } catch (InterruptedException ie) {
389 LOG.warn("Interrupted waiting for " + name + " to finish...");
390 }
391 }
392 }
393
394 void join() {
395 waitFor(splits, "Split Thread");
396 waitFor(mergePool, "Merge Thread");
397 waitFor(longCompactions, "Large Compaction Thread");
398 waitFor(shortCompactions, "Small Compaction Thread");
399 }
400
401
402
403
404
405
406
407 public int getCompactionQueueSize() {
408 return longCompactions.getQueue().size() + shortCompactions.getQueue().size();
409 }
410
411 public int getLargeCompactionQueueSize() {
412 return longCompactions.getQueue().size();
413 }
414
415
416 public int getSmallCompactionQueueSize() {
417 return shortCompactions.getQueue().size();
418 }
419
420 public int getSplitQueueSize() {
421 return splits.getQueue().size();
422 }
423
424 private boolean shouldSplitRegion() {
425 if(server.getNumberOfOnlineRegions() > 0.9*regionSplitLimit) {
426 LOG.warn("Total number of regions is approaching the upper limit " + regionSplitLimit + ". "
427 + "Please consider taking a look at http://hbase.apache.org/book.html#ops.regionmgt");
428 }
429 return (regionSplitLimit > server.getNumberOfOnlineRegions());
430 }
431
432
433
434
435 public int getRegionSplitLimit() {
436 return this.regionSplitLimit;
437 }
438
439 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="EQ_COMPARETO_USE_OBJECT_EQUALS",
440 justification="Contrived use of compareTo")
441 private class CompactionRunner implements Runnable, Comparable<CompactionRunner> {
442 private final Store store;
443 private final HRegion region;
444 private CompactionContext compaction;
445 private int queuedPriority;
446 private ThreadPoolExecutor parent;
447
448 public CompactionRunner(Store store, HRegion region,
449 CompactionContext compaction, ThreadPoolExecutor parent) {
450 super();
451 this.store = store;
452 this.region = region;
453 this.compaction = compaction;
454 this.queuedPriority = (this.compaction == null)
455 ? store.getCompactPriority() : compaction.getRequest().getPriority();
456 this.parent = parent;
457 }
458
459 @Override
460 public String toString() {
461 return (this.compaction != null) ? ("Request = " + compaction.getRequest())
462 : ("Store = " + store.toString() + ", pri = " + queuedPriority);
463 }
464
465 @Override
466 public void run() {
467 Preconditions.checkNotNull(server);
468 if (server.isStopped()
469 || (region.getTableDesc() != null && !region.getTableDesc().isCompactionEnabled())) {
470 return;
471 }
472
473 if (this.compaction == null) {
474 int oldPriority = this.queuedPriority;
475 this.queuedPriority = this.store.getCompactPriority();
476 if (this.queuedPriority > oldPriority) {
477
478
479 this.parent.execute(this);
480 return;
481 }
482 try {
483 this.compaction = selectCompaction(this.region, this.store, queuedPriority, null);
484 } catch (IOException ex) {
485 LOG.error("Compaction selection failed " + this, ex);
486 server.checkFileSystem();
487 return;
488 }
489 if (this.compaction == null) return;
490
491
492 assert this.compaction.hasSelection();
493 ThreadPoolExecutor pool = store.throttleCompaction(
494 compaction.getRequest().getSize()) ? longCompactions : shortCompactions;
495 if (this.parent != pool) {
496 this.store.cancelRequestedCompaction(this.compaction);
497 this.compaction = null;
498 this.parent = pool;
499 this.parent.execute(this);
500 return;
501 }
502 }
503
504 assert this.compaction != null;
505
506 this.compaction.getRequest().beforeExecute();
507 try {
508
509
510 long start = EnvironmentEdgeManager.currentTime();
511 boolean completed = region.compact(compaction, store);
512 long now = EnvironmentEdgeManager.currentTime();
513 LOG.info(((completed) ? "Completed" : "Aborted") + " compaction: " +
514 this + "; duration=" + StringUtils.formatTimeDiff(now, start));
515 if (completed) {
516
517 if (store.getCompactPriority() <= 0) {
518 requestSystemCompaction(region, store, "Recursive enqueue");
519 } else {
520
521 requestSplit(region);
522 }
523 }
524 } catch (IOException ex) {
525 IOException remoteEx = RemoteExceptionHandler.checkIOException(ex);
526 LOG.error("Compaction failed " + this, remoteEx);
527 if (remoteEx != ex) {
528 LOG.info("Compaction failed at original callstack: " + formatStackTrace(ex));
529 }
530 server.checkFileSystem();
531 } catch (Exception ex) {
532 LOG.error("Compaction failed " + this, ex);
533 server.checkFileSystem();
534 } finally {
535 LOG.debug("CompactSplitThread Status: " + CompactSplitThread.this);
536 }
537 this.compaction.getRequest().afterExecute();
538 }
539
540 private String formatStackTrace(Exception ex) {
541 StringWriter sw = new StringWriter();
542 PrintWriter pw = new PrintWriter(sw);
543 ex.printStackTrace(pw);
544 pw.flush();
545 return sw.toString();
546 }
547
548 @Override
549 public int compareTo(CompactionRunner o) {
550
551 int compareVal = queuedPriority - o.queuedPriority;
552 if (compareVal != 0) return compareVal;
553 CompactionContext tc = this.compaction, oc = o.compaction;
554
555 return (tc == null) ? ((oc == null) ? 0 : 1)
556 : ((oc == null) ? -1 : tc.getRequest().compareTo(oc.getRequest()));
557 }
558 }
559
560
561
562
563 private static class Rejection implements RejectedExecutionHandler {
564 @Override
565 public void rejectedExecution(Runnable runnable, ThreadPoolExecutor pool) {
566 if (runnable instanceof CompactionRunner) {
567 CompactionRunner runner = (CompactionRunner)runnable;
568 LOG.debug("Compaction Rejected: " + runner);
569 runner.store.cancelRequestedCompaction(runner.compaction);
570 }
571 }
572 }
573
574
575
576
577 @Override
578 public void onConfigurationChange(Configuration newConf) {
579
580
581
582
583
584 int largeThreads = Math.max(1, newConf.getInt(
585 LARGE_COMPACTION_THREADS,
586 LARGE_COMPACTION_THREADS_DEFAULT));
587 if (this.longCompactions.getCorePoolSize() != largeThreads) {
588 LOG.info("Changing the value of " + LARGE_COMPACTION_THREADS +
589 " from " + this.longCompactions.getCorePoolSize() + " to " +
590 largeThreads);
591 this.longCompactions.setMaximumPoolSize(largeThreads);
592 this.longCompactions.setCorePoolSize(largeThreads);
593 }
594
595 int smallThreads = newConf.getInt(SMALL_COMPACTION_THREADS,
596 SMALL_COMPACTION_THREADS_DEFAULT);
597 if (this.shortCompactions.getCorePoolSize() != smallThreads) {
598 LOG.info("Changing the value of " + SMALL_COMPACTION_THREADS +
599 " from " + this.shortCompactions.getCorePoolSize() + " to " +
600 smallThreads);
601 this.shortCompactions.setMaximumPoolSize(smallThreads);
602 this.shortCompactions.setCorePoolSize(smallThreads);
603 }
604
605 int splitThreads = newConf.getInt(SPLIT_THREADS,
606 SPLIT_THREADS_DEFAULT);
607 if (this.splits.getCorePoolSize() != splitThreads) {
608 LOG.info("Changing the value of " + SPLIT_THREADS +
609 " from " + this.splits.getCorePoolSize() + " to " +
610 splitThreads);
611 this.splits.setMaximumPoolSize(smallThreads);
612 this.splits.setCorePoolSize(smallThreads);
613 }
614
615 int mergeThreads = newConf.getInt(MERGE_THREADS,
616 MERGE_THREADS_DEFAULT);
617 if (this.mergePool.getCorePoolSize() != mergeThreads) {
618 LOG.info("Changing the value of " + MERGE_THREADS +
619 " from " + this.mergePool.getCorePoolSize() + " to " +
620 mergeThreads);
621 this.mergePool.setMaximumPoolSize(smallThreads);
622 this.mergePool.setCorePoolSize(smallThreads);
623 }
624
625
626
627 this.conf.reloadConfiguration();
628 }
629
630 protected int getSmallCompactionThreadNum() {
631 return this.shortCompactions.getCorePoolSize();
632 }
633
634 public int getLargeCompactionThreadNum() {
635 return this.longCompactions.getCorePoolSize();
636 }
637
638
639
640
641 @Override
642 public void registerChildren(ConfigurationManager manager) {
643
644 }
645
646
647
648
649 @Override
650 public void deregisterChildren(ConfigurationManager manager) {
651
652 }
653 }