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