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