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