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