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