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.lang.Thread.UncaughtExceptionHandler;
23 import java.lang.management.ManagementFactory;
24 import java.util.ArrayList;
25 import java.util.ConcurrentModificationException;
26 import java.util.HashMap;
27 import java.util.HashSet;
28 import java.util.List;
29 import java.util.Map;
30 import java.util.Set;
31 import java.util.SortedMap;
32 import java.util.concurrent.BlockingQueue;
33 import java.util.concurrent.DelayQueue;
34 import java.util.concurrent.Delayed;
35 import java.util.concurrent.ThreadFactory;
36 import java.util.concurrent.TimeUnit;
37 import java.util.concurrent.atomic.AtomicBoolean;
38 import java.util.concurrent.locks.ReentrantReadWriteLock;
39
40 import org.apache.commons.logging.Log;
41 import org.apache.commons.logging.LogFactory;
42 import org.apache.hadoop.hbase.classification.InterfaceAudience;
43 import org.apache.hadoop.conf.Configuration;
44 import org.apache.hadoop.hbase.DroppedSnapshotException;
45 import org.apache.hadoop.hbase.HConstants;
46 import org.apache.hadoop.hbase.RemoteExceptionHandler;
47 import org.apache.hadoop.hbase.io.util.HeapMemorySizeUtil;
48 import org.apache.hadoop.hbase.util.Bytes;
49 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
50 import org.apache.hadoop.hbase.util.HasThread;
51 import org.apache.hadoop.hbase.util.Threads;
52 import org.apache.hadoop.util.StringUtils;
53 import org.apache.htrace.Trace;
54 import org.apache.htrace.TraceScope;
55 import org.apache.hadoop.hbase.util.Counter;
56
57 import com.google.common.base.Preconditions;
58
59
60
61
62
63
64
65
66
67
68 @InterfaceAudience.Private
69 class MemStoreFlusher implements FlushRequester {
70 static final Log LOG = LogFactory.getLog(MemStoreFlusher.class);
71
72
73
74 private final BlockingQueue<FlushQueueEntry> flushQueue =
75 new DelayQueue<FlushQueueEntry>();
76 private final Map<HRegion, FlushRegionEntry> regionsInQueue =
77 new HashMap<HRegion, FlushRegionEntry>();
78 private AtomicBoolean wakeupPending = new AtomicBoolean();
79
80 private final long threadWakeFrequency;
81 private final HRegionServer server;
82 private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
83 private final Object blockSignal = new Object();
84
85 protected long globalMemStoreLimit;
86 protected float globalMemStoreLimitLowMarkPercent;
87 protected long globalMemStoreLimitLowMark;
88
89 private long blockingWaitTime;
90 private final Counter updatesBlockedMsHighWater = new Counter();
91
92 private final FlushHandler[] flushHandlers;
93 private List<FlushRequestListener> flushRequestListeners = new ArrayList<FlushRequestListener>(1);
94
95
96
97
98
99 public MemStoreFlusher(final Configuration conf,
100 final HRegionServer server) {
101 super();
102 this.server = server;
103 this.threadWakeFrequency =
104 conf.getLong(HConstants.THREAD_WAKE_FREQUENCY, 10 * 1000);
105 long max = ManagementFactory.getMemoryMXBean().getHeapMemoryUsage().getMax();
106 float globalMemStorePercent = HeapMemorySizeUtil.getGlobalMemStorePercent(conf, true);
107 this.globalMemStoreLimit = (long) (max * globalMemStorePercent);
108 this.globalMemStoreLimitLowMarkPercent =
109 HeapMemorySizeUtil.getGlobalMemStoreLowerMark(conf, globalMemStorePercent);
110 this.globalMemStoreLimitLowMark =
111 (long) (this.globalMemStoreLimit * this.globalMemStoreLimitLowMarkPercent);
112
113 this.blockingWaitTime = conf.getInt("hbase.hstore.blockingWaitTime",
114 90000);
115 int handlerCount = conf.getInt("hbase.hstore.flusher.count", 2);
116 this.flushHandlers = new FlushHandler[handlerCount];
117 LOG.info("globalMemStoreLimit=" +
118 StringUtils.humanReadableInt(this.globalMemStoreLimit) +
119 ", globalMemStoreLimitLowMark=" +
120 StringUtils.humanReadableInt(this.globalMemStoreLimitLowMark) +
121 ", maxHeap=" + StringUtils.humanReadableInt(max));
122 }
123
124 public Counter getUpdatesBlockedMsHighWater() {
125 return this.updatesBlockedMsHighWater;
126 }
127
128
129
130
131
132
133
134 private boolean flushOneForGlobalPressure() {
135 SortedMap<Long, HRegion> regionsBySize =
136 server.getCopyOfOnlineRegionsSortedBySize();
137
138 Set<HRegion> excludedRegions = new HashSet<HRegion>();
139
140 boolean flushedOne = false;
141 while (!flushedOne) {
142
143
144 HRegion bestFlushableRegion = getBiggestMemstoreRegion(
145 regionsBySize, excludedRegions, true);
146
147 HRegion bestAnyRegion = getBiggestMemstoreRegion(
148 regionsBySize, excludedRegions, false);
149
150 if (bestAnyRegion == null) {
151 LOG.error("Above memory mark but there are no flushable regions!");
152 return false;
153 }
154
155 HRegion regionToFlush;
156 if (bestFlushableRegion != null &&
157 bestAnyRegion.memstoreSize.get() > 2 * bestFlushableRegion.memstoreSize.get()) {
158
159
160
161
162 if (LOG.isDebugEnabled()) {
163 LOG.debug("Under global heap pressure: " +
164 "Region " + bestAnyRegion.getRegionNameAsString() + " has too many " +
165 "store files, but is " +
166 StringUtils.humanReadableInt(bestAnyRegion.memstoreSize.get()) +
167 " vs best flushable region's " +
168 StringUtils.humanReadableInt(bestFlushableRegion.memstoreSize.get()) +
169 ". Choosing the bigger.");
170 }
171 regionToFlush = bestAnyRegion;
172 } else {
173 if (bestFlushableRegion == null) {
174 regionToFlush = bestAnyRegion;
175 } else {
176 regionToFlush = bestFlushableRegion;
177 }
178 }
179
180 Preconditions.checkState(regionToFlush.memstoreSize.get() > 0);
181
182 LOG.info("Flush of region " + regionToFlush + " due to global heap pressure");
183 flushedOne = flushRegion(regionToFlush, true);
184 if (!flushedOne) {
185 LOG.info("Excluding unflushable region " + regionToFlush +
186 " - trying to find a different region to flush.");
187 excludedRegions.add(regionToFlush);
188 }
189 }
190 return true;
191 }
192
193 private class FlushHandler extends HasThread {
194
195 private FlushHandler(String name) {
196 super(name);
197 }
198
199 @Override
200 public void run() {
201 while (!server.isStopped()) {
202 FlushQueueEntry fqe = null;
203 try {
204 wakeupPending.set(false);
205 fqe = flushQueue.poll(threadWakeFrequency, TimeUnit.MILLISECONDS);
206 if (fqe == null || fqe instanceof WakeupFlushThread) {
207 if (isAboveLowWaterMark()) {
208 LOG.debug("Flush thread woke up because memory above low water="
209 + StringUtils.humanReadableInt(globalMemStoreLimitLowMark));
210 if (!flushOneForGlobalPressure()) {
211
212
213
214
215
216 Thread.sleep(1000);
217 wakeUpIfBlocking();
218 }
219
220 wakeupFlushThread();
221 }
222 continue;
223 }
224 FlushRegionEntry fre = (FlushRegionEntry) fqe;
225 if (!flushRegion(fre)) {
226 break;
227 }
228 } catch (InterruptedException ex) {
229 continue;
230 } catch (ConcurrentModificationException ex) {
231 continue;
232 } catch (Exception ex) {
233 LOG.error("Cache flusher failed for entry " + fqe, ex);
234 if (!server.checkFileSystem()) {
235 break;
236 }
237 }
238 }
239 synchronized (regionsInQueue) {
240 regionsInQueue.clear();
241 flushQueue.clear();
242 }
243
244
245 wakeUpIfBlocking();
246 LOG.info(getName() + " exiting");
247 }
248 }
249
250
251 private void wakeupFlushThread() {
252 if (wakeupPending.compareAndSet(false, true)) {
253 flushQueue.add(new WakeupFlushThread());
254 }
255 }
256
257 private HRegion getBiggestMemstoreRegion(
258 SortedMap<Long, HRegion> regionsBySize,
259 Set<HRegion> excludedRegions,
260 boolean checkStoreFileCount) {
261 synchronized (regionsInQueue) {
262 for (HRegion region : regionsBySize.values()) {
263 if (excludedRegions.contains(region)) {
264 continue;
265 }
266
267 if (region.writestate.flushing || !region.writestate.writesEnabled) {
268 continue;
269 }
270
271 if (checkStoreFileCount && isTooManyStoreFiles(region)) {
272 continue;
273 }
274 return region;
275 }
276 }
277 return null;
278 }
279
280
281
282
283 private boolean isAboveHighWaterMark() {
284 return server.getRegionServerAccounting().
285 getGlobalMemstoreSize() >= globalMemStoreLimit;
286 }
287
288
289
290
291 private boolean isAboveLowWaterMark() {
292 return server.getRegionServerAccounting().
293 getGlobalMemstoreSize() >= globalMemStoreLimitLowMark;
294 }
295
296 public void requestFlush(HRegion r) {
297 synchronized (regionsInQueue) {
298 if (!regionsInQueue.containsKey(r)) {
299
300
301 FlushRegionEntry fqe = new FlushRegionEntry(r);
302 this.regionsInQueue.put(r, fqe);
303 this.flushQueue.add(fqe);
304 }
305 }
306 }
307
308 public void requestDelayedFlush(HRegion r, long delay) {
309 synchronized (regionsInQueue) {
310 if (!regionsInQueue.containsKey(r)) {
311
312 FlushRegionEntry fqe = new FlushRegionEntry(r);
313 fqe.requeue(delay);
314 this.regionsInQueue.put(r, fqe);
315 this.flushQueue.add(fqe);
316 }
317 }
318 }
319
320 public int getFlushQueueSize() {
321 return flushQueue.size();
322 }
323
324
325
326
327 void interruptIfNecessary() {
328 lock.writeLock().lock();
329 try {
330 for (FlushHandler flushHander : flushHandlers) {
331 if (flushHander != null) flushHander.interrupt();
332 }
333 } finally {
334 lock.writeLock().unlock();
335 }
336 }
337
338 synchronized void start(UncaughtExceptionHandler eh) {
339 ThreadFactory flusherThreadFactory = Threads.newDaemonThreadFactory(
340 server.getServerName().toShortString() + "-MemStoreFlusher", eh);
341 for (int i = 0; i < flushHandlers.length; i++) {
342 flushHandlers[i] = new FlushHandler("MemStoreFlusher." + i);
343 flusherThreadFactory.newThread(flushHandlers[i]);
344 flushHandlers[i].start();
345 }
346 }
347
348 boolean isAlive() {
349 for (FlushHandler flushHander : flushHandlers) {
350 if (flushHander != null && flushHander.isAlive()) {
351 return true;
352 }
353 }
354 return false;
355 }
356
357 void join() {
358 for (FlushHandler flushHander : flushHandlers) {
359 if (flushHander != null) {
360 Threads.shutdown(flushHander.getThread());
361 }
362 }
363 }
364
365
366
367
368
369
370
371
372
373 private boolean flushRegion(final FlushRegionEntry fqe) {
374 HRegion region = fqe.region;
375 if (!region.getRegionInfo().isMetaRegion() &&
376 isTooManyStoreFiles(region)) {
377 if (fqe.isMaximumWait(this.blockingWaitTime)) {
378 LOG.info("Waited " + (EnvironmentEdgeManager.currentTime() - fqe.createTime) +
379 "ms on a compaction to clean up 'too many store files'; waited " +
380 "long enough... proceeding with flush of " +
381 region.getRegionNameAsString());
382 } else {
383
384 if (fqe.getRequeueCount() <= 0) {
385
386 LOG.warn("Region " + region.getRegionNameAsString() + " has too many " +
387 "store files; delaying flush up to " + this.blockingWaitTime + "ms");
388 if (!this.server.compactSplitThread.requestSplit(region)) {
389 try {
390 this.server.compactSplitThread.requestSystemCompaction(
391 region, Thread.currentThread().getName());
392 } catch (IOException e) {
393 LOG.error(
394 "Cache flush failed for region " + Bytes.toStringBinary(region.getRegionName()),
395 RemoteExceptionHandler.checkIOException(e));
396 }
397 }
398 }
399
400
401
402 this.flushQueue.add(fqe.requeue(this.blockingWaitTime / 100));
403
404 return true;
405 }
406 }
407 return flushRegion(region, false);
408 }
409
410
411
412
413
414
415
416
417
418
419
420
421
422 private boolean flushRegion(final HRegion region, final boolean emergencyFlush) {
423 long startTime = 0;
424 synchronized (this.regionsInQueue) {
425 FlushRegionEntry fqe = this.regionsInQueue.remove(region);
426
427 if (fqe != null) {
428 startTime = fqe.createTime;
429 }
430 if (fqe != null && emergencyFlush) {
431
432
433 flushQueue.remove(fqe);
434 }
435 }
436 if (startTime == 0) {
437
438
439
440 startTime = EnvironmentEdgeManager.currentTime();
441 }
442 lock.readLock().lock();
443 try {
444 notifyFlushRequest(region, emergencyFlush);
445 HRegion.FlushResult flushResult = region.flushcache();
446 boolean shouldCompact = flushResult.isCompactionNeeded();
447
448 boolean shouldSplit = region.checkSplit() != null;
449 if (shouldSplit) {
450 this.server.compactSplitThread.requestSplit(region);
451 } else if (shouldCompact) {
452 server.compactSplitThread.requestSystemCompaction(
453 region, Thread.currentThread().getName());
454 }
455 if (flushResult.isFlushSucceeded()) {
456 long endTime = EnvironmentEdgeManager.currentTime();
457 server.metricsRegionServer.updateFlushTime(endTime - startTime);
458 }
459 } catch (DroppedSnapshotException ex) {
460
461
462
463
464
465 server.abort("Replay of WAL required. Forcing server shutdown", ex);
466 return false;
467 } catch (IOException ex) {
468 LOG.error("Cache flush failed" +
469 (region != null ? (" for region " + Bytes.toStringBinary(region.getRegionName())) : ""),
470 RemoteExceptionHandler.checkIOException(ex));
471 if (!server.checkFileSystem()) {
472 return false;
473 }
474 } finally {
475 lock.readLock().unlock();
476 wakeUpIfBlocking();
477 }
478 return true;
479 }
480
481 private void notifyFlushRequest(HRegion region, boolean emergencyFlush) {
482 FlushType type = FlushType.NORMAL;
483 if (emergencyFlush) {
484 type = isAboveHighWaterMark() ? FlushType.ABOVE_HIGHER_MARK : FlushType.ABOVE_LOWER_MARK;
485 }
486 for (FlushRequestListener listener : flushRequestListeners) {
487 listener.flushRequested(type, region);
488 }
489 }
490
491 private void wakeUpIfBlocking() {
492 synchronized (blockSignal) {
493 blockSignal.notifyAll();
494 }
495 }
496
497 private boolean isTooManyStoreFiles(HRegion region) {
498 for (Store store : region.stores.values()) {
499 if (store.hasTooManyStoreFiles()) {
500 return true;
501 }
502 }
503 return false;
504 }
505
506
507
508
509
510
511
512 public void reclaimMemStoreMemory() {
513 TraceScope scope = Trace.startSpan("MemStoreFluser.reclaimMemStoreMemory");
514 if (isAboveHighWaterMark()) {
515 if (Trace.isTracing()) {
516 scope.getSpan().addTimelineAnnotation("Force Flush. We're above high water mark.");
517 }
518 long start = EnvironmentEdgeManager.currentTime();
519 synchronized (this.blockSignal) {
520 boolean blocked = false;
521 long startTime = 0;
522 boolean interrupted = false;
523 try {
524 while (isAboveHighWaterMark() && !server.isStopped()) {
525 if (!blocked) {
526 startTime = EnvironmentEdgeManager.currentTime();
527 LOG.info("Blocking updates on " + server.toString() +
528 ": the global memstore size " +
529 StringUtils.humanReadableInt(server.getRegionServerAccounting().getGlobalMemstoreSize()) +
530 " is >= than blocking " +
531 StringUtils.humanReadableInt(globalMemStoreLimit) + " size");
532 }
533 blocked = true;
534 wakeupFlushThread();
535 try {
536
537
538 blockSignal.wait(5 * 1000);
539 } catch (InterruptedException ie) {
540 LOG.warn("Interrupted while waiting");
541 interrupted = true;
542 }
543 long took = EnvironmentEdgeManager.currentTime() - start;
544 LOG.warn("Memstore is above high water mark and block " + took + "ms");
545 }
546 } finally {
547 if (interrupted) {
548 Thread.currentThread().interrupt();
549 }
550 }
551
552 if(blocked){
553 final long totalTime = EnvironmentEdgeManager.currentTime() - startTime;
554 if(totalTime > 0){
555 this.updatesBlockedMsHighWater.add(totalTime);
556 }
557 LOG.info("Unblocking updates for server " + server.toString());
558 }
559 }
560 } else if (isAboveLowWaterMark()) {
561 wakeupFlushThread();
562 }
563 scope.close();
564 }
565 @Override
566 public String toString() {
567 return "flush_queue="
568 + flushQueue.size();
569 }
570
571 public String dumpQueue() {
572 StringBuilder queueList = new StringBuilder();
573 queueList.append("Flush Queue Queue dump:\n");
574 queueList.append(" Flush Queue:\n");
575 java.util.Iterator<FlushQueueEntry> it = flushQueue.iterator();
576
577 while(it.hasNext()){
578 queueList.append(" "+it.next().toString());
579 queueList.append("\n");
580 }
581
582 return queueList.toString();
583 }
584
585
586
587
588
589 public void registerFlushRequestListener(final FlushRequestListener listener) {
590 this.flushRequestListeners.add(listener);
591 }
592
593
594
595
596
597
598 public boolean unregisterFlushRequestListener(final FlushRequestListener listener) {
599 return this.flushRequestListeners.remove(listener);
600 }
601
602
603
604
605
606 public void setGlobalMemstoreLimit(long globalMemStoreSize) {
607 this.globalMemStoreLimit = globalMemStoreSize;
608 this.globalMemStoreLimitLowMark =
609 (long) (this.globalMemStoreLimitLowMarkPercent * globalMemStoreSize);
610 reclaimMemStoreMemory();
611 }
612
613 public long getMemoryLimit() {
614 return this.globalMemStoreLimit;
615 }
616
617 interface FlushQueueEntry extends Delayed {
618 }
619
620
621
622
623 static class WakeupFlushThread implements FlushQueueEntry {
624 @Override
625 public long getDelay(TimeUnit unit) {
626 return 0;
627 }
628
629 @Override
630 public int compareTo(Delayed o) {
631 return -1;
632 }
633
634 @Override
635 public boolean equals(Object obj) {
636 return (this == obj);
637 }
638 }
639
640
641
642
643
644
645
646
647
648 static class FlushRegionEntry implements FlushQueueEntry {
649 private final HRegion region;
650
651 private final long createTime;
652 private long whenToExpire;
653 private int requeueCount = 0;
654
655 FlushRegionEntry(final HRegion r) {
656 this.region = r;
657 this.createTime = EnvironmentEdgeManager.currentTime();
658 this.whenToExpire = this.createTime;
659 }
660
661
662
663
664
665 public boolean isMaximumWait(final long maximumWait) {
666 return (EnvironmentEdgeManager.currentTime() - this.createTime) > maximumWait;
667 }
668
669
670
671
672
673 public int getRequeueCount() {
674 return this.requeueCount;
675 }
676
677
678
679
680
681
682
683 public FlushRegionEntry requeue(final long when) {
684 this.whenToExpire = EnvironmentEdgeManager.currentTime() + when;
685 this.requeueCount++;
686 return this;
687 }
688
689 @Override
690 public long getDelay(TimeUnit unit) {
691 return unit.convert(this.whenToExpire - EnvironmentEdgeManager.currentTime(),
692 TimeUnit.MILLISECONDS);
693 }
694
695 @Override
696 public int compareTo(Delayed other) {
697
698 int ret = Long.valueOf(getDelay(TimeUnit.MILLISECONDS) -
699 other.getDelay(TimeUnit.MILLISECONDS)).intValue();
700 if (ret != 0) {
701 return ret;
702 }
703 FlushQueueEntry otherEntry = (FlushQueueEntry) other;
704 return hashCode() - otherEntry.hashCode();
705 }
706
707 @Override
708 public String toString() {
709 return "[flush region " + Bytes.toStringBinary(region.getRegionName()) + "]";
710 }
711
712 @Override
713 public int hashCode() {
714 int hash = (int) getDelay(TimeUnit.MILLISECONDS);
715 return hash ^ region.hashCode();
716 }
717
718 @Override
719 public boolean equals(Object obj) {
720 if (this == obj) {
721 return true;
722 }
723 if (obj == null || getClass() != obj.getClass()) {
724 return false;
725 }
726 Delayed other = (Delayed) obj;
727 return compareTo(other) == 0;
728 }
729 }
730 }
731
732 enum FlushType {
733 NORMAL, ABOVE_LOWER_MARK, ABOVE_HIGHER_MARK;
734 }