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.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.TimeUnit;
34 import java.util.concurrent.atomic.AtomicBoolean;
35 import java.util.concurrent.locks.Condition;
36 import java.util.concurrent.locks.ReentrantLock;
37
38 import org.apache.commons.logging.Log;
39 import org.apache.commons.logging.LogFactory;
40 import org.apache.hadoop.conf.Configuration;
41 import org.apache.hadoop.hbase.DroppedSnapshotException;
42 import org.apache.hadoop.hbase.HConstants;
43 import org.apache.hadoop.hbase.RemoteExceptionHandler;
44 import org.apache.hadoop.hbase.util.Bytes;
45 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
46 import org.apache.hadoop.hbase.util.HasThread;
47 import org.apache.hadoop.util.StringUtils;
48 import org.cliffc.high_scale_lib.Counter;
49
50 import com.google.common.base.Preconditions;
51
52
53
54
55
56
57
58
59
60
61 class MemStoreFlusher extends HasThread implements FlushRequester {
62 static final Log LOG = LogFactory.getLog(MemStoreFlusher.class);
63
64
65 private final BlockingQueue<FlushQueueEntry> flushQueue =
66 new DelayQueue<FlushQueueEntry>();
67 private final Map<HRegion, FlushRegionEntry> regionsInQueue =
68 new HashMap<HRegion, FlushRegionEntry>();
69 private AtomicBoolean wakeupPending = new AtomicBoolean();
70
71 private final long threadWakeFrequency;
72 private final HRegionServer server;
73 private final ReentrantLock lock = new ReentrantLock();
74 private final Condition flushOccurred = lock.newCondition();
75
76 protected final long globalMemStoreLimit;
77 protected final long globalMemStoreLimitLowMark;
78
79 private static final float DEFAULT_UPPER = 0.4f;
80 private static final float DEFAULT_LOWER = 0.35f;
81 private static final String UPPER_KEY =
82 "hbase.regionserver.global.memstore.upperLimit";
83 private static final String LOWER_KEY =
84 "hbase.regionserver.global.memstore.lowerLimit";
85 private long blockingStoreFilesNumber;
86 private long blockingWaitTime;
87 private final Counter updatesBlockedMsHighWater = new Counter();
88
89
90
91
92
93 public MemStoreFlusher(final Configuration conf,
94 final HRegionServer server) {
95 super();
96 this.server = server;
97 this.threadWakeFrequency =
98 conf.getLong(HConstants.THREAD_WAKE_FREQUENCY, 10 * 1000);
99 long max = ManagementFactory.getMemoryMXBean().getHeapMemoryUsage().getMax();
100 this.globalMemStoreLimit = globalMemStoreLimit(max, DEFAULT_UPPER,
101 UPPER_KEY, conf);
102 long lower = globalMemStoreLimit(max, DEFAULT_LOWER, LOWER_KEY, conf);
103 if (lower > this.globalMemStoreLimit) {
104 lower = this.globalMemStoreLimit;
105 LOG.info("Setting globalMemStoreLimitLowMark == globalMemStoreLimit " +
106 "because supplied " + LOWER_KEY + " was > " + UPPER_KEY);
107 }
108 this.globalMemStoreLimitLowMark = lower;
109 this.blockingStoreFilesNumber =
110 conf.getInt("hbase.hstore.blockingStoreFiles", 7);
111 if (this.blockingStoreFilesNumber == -1) {
112 this.blockingStoreFilesNumber = 1 +
113 conf.getInt("hbase.hstore.compactionThreshold", 3);
114 }
115 this.blockingWaitTime = conf.getInt("hbase.hstore.blockingWaitTime",
116 90000);
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 @Override
220 public void run() {
221 while (!this.server.isStopped()) {
222 FlushQueueEntry fqe = null;
223 try {
224 wakeupPending.set(false);
225 fqe = flushQueue.poll(threadWakeFrequency, TimeUnit.MILLISECONDS);
226 if (fqe == null || fqe instanceof WakeupFlushThread) {
227 if (isAboveLowWaterMark()) {
228 LOG.debug("Flush thread woke up because memory above low water=" +
229 StringUtils.humanReadableInt(this.globalMemStoreLimitLowMark));
230 if (!flushOneForGlobalPressure()) {
231
232
233
234
235
236 lock.lock();
237 try {
238 Thread.sleep(1000);
239 flushOccurred.signalAll();
240 } finally {
241 lock.unlock();
242 }
243 }
244
245 wakeupFlushThread();
246 }
247 continue;
248 }
249 FlushRegionEntry fre = (FlushRegionEntry)fqe;
250 if (!flushRegion(fre)) {
251 break;
252 }
253 } catch (InterruptedException ex) {
254 continue;
255 } catch (ConcurrentModificationException ex) {
256 continue;
257 } catch (Exception ex) {
258 LOG.error("Cache flusher failed for entry " + fqe, ex);
259 if (!server.checkFileSystem()) {
260 break;
261 }
262 }
263 }
264 this.regionsInQueue.clear();
265 this.flushQueue.clear();
266
267
268 lock.lock();
269 try {
270 flushOccurred.signalAll();
271 } finally {
272 lock.unlock();
273 }
274 LOG.info(getName() + " exiting");
275 }
276
277 private void wakeupFlushThread() {
278 if (wakeupPending.compareAndSet(false, true)) {
279 flushQueue.add(new WakeupFlushThread());
280 }
281 }
282
283 private HRegion getBiggestMemstoreRegion(
284 SortedMap<Long, HRegion> regionsBySize,
285 Set<HRegion> excludedRegions,
286 boolean checkStoreFileCount) {
287 synchronized (regionsInQueue) {
288 for (HRegion region : regionsBySize.values()) {
289 if (excludedRegions.contains(region)) {
290 continue;
291 }
292
293 if (checkStoreFileCount && isTooManyStoreFiles(region)) {
294 continue;
295 }
296 return region;
297 }
298 }
299 return null;
300 }
301
302
303
304
305 private boolean isAboveHighWaterMark() {
306 return server.getRegionServerAccounting().
307 getGlobalMemstoreSize() >= globalMemStoreLimit;
308 }
309
310
311
312
313 private boolean isAboveLowWaterMark() {
314 return server.getRegionServerAccounting().
315 getGlobalMemstoreSize() >= globalMemStoreLimitLowMark;
316 }
317
318 public void requestFlush(HRegion r) {
319 synchronized (regionsInQueue) {
320 if (!regionsInQueue.containsKey(r)) {
321
322
323 FlushRegionEntry fqe = new FlushRegionEntry(r);
324 this.regionsInQueue.put(r, fqe);
325 this.flushQueue.add(fqe);
326 }
327 }
328 }
329
330 public int getFlushQueueSize() {
331 return flushQueue.size();
332 }
333
334
335
336
337 void interruptIfNecessary() {
338 lock.lock();
339 try {
340 this.interrupt();
341 } finally {
342 lock.unlock();
343 }
344 }
345
346
347
348
349
350
351
352
353
354 private boolean flushRegion(final FlushRegionEntry fqe) {
355 HRegion region = fqe.region;
356 if (!fqe.region.getRegionInfo().isMetaRegion() &&
357 isTooManyStoreFiles(region)) {
358 if (fqe.isMaximumWait(this.blockingWaitTime)) {
359 LOG.info("Waited " + (System.currentTimeMillis() - fqe.createTime) +
360 "ms on a compaction to clean up 'too many store files'; waited " +
361 "long enough... proceeding with flush of " +
362 region.getRegionNameAsString());
363 } else {
364
365 if (fqe.getRequeueCount() <= 0) {
366
367 LOG.warn("Region " + region.getRegionNameAsString() + " has too many " +
368 "store files; delaying flush up to " + this.blockingWaitTime + "ms");
369 if (!this.server.compactSplitThread.requestSplit(region)) {
370 try {
371 this.server.compactSplitThread.requestCompaction(region, getName());
372 } catch (IOException e) {
373 LOG.error("Cache flush failed" +
374 (region != null ? (" for region " + Bytes.toStringBinary(region.getRegionName())) : ""),
375 RemoteExceptionHandler.checkIOException(e));
376 }
377 }
378 }
379
380
381
382 this.flushQueue.add(fqe.requeue(this.blockingWaitTime / 100));
383
384 return true;
385 }
386 }
387 return flushRegion(region, false);
388 }
389
390
391
392
393
394
395
396
397
398
399
400
401
402 private boolean flushRegion(final HRegion region, final boolean emergencyFlush) {
403 synchronized (this.regionsInQueue) {
404 FlushRegionEntry fqe = this.regionsInQueue.remove(region);
405 if (fqe != null && emergencyFlush) {
406
407
408 flushQueue.remove(fqe);
409 }
410 lock.lock();
411 }
412 try {
413 boolean shouldCompact = region.flushcache();
414
415 boolean shouldSplit = region.checkSplit() != null;
416 if (shouldSplit) {
417 this.server.compactSplitThread.requestSplit(region);
418 } else if (shouldCompact) {
419 server.compactSplitThread.requestCompaction(region, getName());
420 }
421
422 server.getMetrics().addFlush(region.getRecentFlushInfo());
423 } catch (DroppedSnapshotException ex) {
424
425
426
427
428
429 server.abort("Replay of HLog required. Forcing server shutdown", ex);
430 return false;
431 } catch (IOException ex) {
432 LOG.error("Cache flush failed" +
433 (region != null ? (" for region " + Bytes.toStringBinary(region.getRegionName())) : ""),
434 RemoteExceptionHandler.checkIOException(ex));
435 if (!server.checkFileSystem()) {
436 return false;
437 }
438 } finally {
439 flushOccurred.signalAll();
440 lock.unlock();
441 }
442 return true;
443 }
444
445 private boolean isTooManyStoreFiles(HRegion region) {
446 for (Store hstore: region.stores.values()) {
447 if (hstore.getStorefilesCount() > this.blockingStoreFilesNumber) {
448 return true;
449 }
450 }
451 return false;
452 }
453
454
455
456
457
458
459
460 public void reclaimMemStoreMemory() {
461 if (isAboveHighWaterMark()) {
462 lock.lock();
463 try {
464 boolean blocked = false;
465 long startTime = 0;
466 while (isAboveHighWaterMark() && !server.isStopped()) {
467 if(!blocked){
468 startTime = EnvironmentEdgeManager.currentTimeMillis();
469 LOG.info("Blocking updates on " + server.toString() +
470 ": the global memstore size " +
471 StringUtils.humanReadableInt(server.getRegionServerAccounting().getGlobalMemstoreSize()) +
472 " is >= than blocking " +
473 StringUtils.humanReadableInt(globalMemStoreLimit) + " size");
474 }
475 blocked = true;
476 wakeupFlushThread();
477 try {
478
479
480 flushOccurred.await(5, TimeUnit.SECONDS);
481 } catch (InterruptedException ie) {
482 Thread.currentThread().interrupt();
483 }
484 }
485 if(blocked){
486 final long totalTime = EnvironmentEdgeManager.currentTimeMillis() - startTime;
487 if(totalTime > 0){
488 this.updatesBlockedMsHighWater.add(totalTime);
489 }
490 LOG.info("Unblocking updates for server " + server.toString());
491 }
492 } finally {
493 lock.unlock();
494 }
495 } else if (isAboveLowWaterMark()) {
496 wakeupFlushThread();
497 }
498 }
499
500 @Override
501 public String toString() {
502 return "flush_queue="
503 + flushQueue.size();
504 }
505
506 public String dumpQueue() {
507 StringBuilder queueList = new StringBuilder();
508 queueList.append("Flush Queue Queue dump:\n");
509 queueList.append(" Flush Queue:\n");
510 java.util.Iterator<FlushQueueEntry> it = flushQueue.iterator();
511
512 while(it.hasNext()){
513 queueList.append(" "+it.next().toString());
514 queueList.append("\n");
515 }
516
517 return queueList.toString();
518 }
519
520 interface FlushQueueEntry extends Delayed {}
521
522
523
524
525 static class WakeupFlushThread implements FlushQueueEntry {
526 @Override
527 public long getDelay(TimeUnit unit) {
528 return 0;
529 }
530
531 @Override
532 public int compareTo(Delayed o) {
533 return -1;
534 }
535 }
536
537
538
539
540
541
542
543
544
545 static class FlushRegionEntry implements FlushQueueEntry {
546 private final HRegion region;
547
548 private final long createTime;
549 private long whenToExpire;
550 private int requeueCount = 0;
551
552 FlushRegionEntry(final HRegion r) {
553 this.region = r;
554 this.createTime = System.currentTimeMillis();
555 this.whenToExpire = this.createTime;
556 }
557
558
559
560
561
562 public boolean isMaximumWait(final long maximumWait) {
563 return (System.currentTimeMillis() - this.createTime) > maximumWait;
564 }
565
566
567
568
569
570 public int getRequeueCount() {
571 return this.requeueCount;
572 }
573
574
575
576
577
578
579
580 public FlushRegionEntry requeue(final long when) {
581 this.whenToExpire = System.currentTimeMillis() + when;
582 this.requeueCount++;
583 return this;
584 }
585
586 @Override
587 public long getDelay(TimeUnit unit) {
588 return unit.convert(this.whenToExpire - System.currentTimeMillis(),
589 TimeUnit.MILLISECONDS);
590 }
591
592 @Override
593 public int compareTo(Delayed other) {
594 return Long.valueOf(getDelay(TimeUnit.MILLISECONDS) -
595 other.getDelay(TimeUnit.MILLISECONDS)).intValue();
596 }
597
598 @Override
599 public String toString() {
600 return "[flush region " + Bytes.toStringBinary(region.getRegionName()) + "]";
601 }
602 }
603 }