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.HasThread;
46 import org.apache.hadoop.util.StringUtils;
47
48 import com.google.common.base.Preconditions;
49
50
51
52
53
54
55
56
57
58
59 class MemStoreFlusher extends HasThread implements FlushRequester {
60 static final Log LOG = LogFactory.getLog(MemStoreFlusher.class);
61
62
63 private final BlockingQueue<FlushQueueEntry> flushQueue =
64 new DelayQueue<FlushQueueEntry>();
65 private final Map<HRegion, FlushRegionEntry> regionsInQueue =
66 new HashMap<HRegion, FlushRegionEntry>();
67 private AtomicBoolean wakeupPending = new AtomicBoolean();
68
69 private final long threadWakeFrequency;
70 private final HRegionServer server;
71 private final ReentrantLock lock = new ReentrantLock();
72 private final Condition flushOccurred = lock.newCondition();
73
74 protected final long globalMemStoreLimit;
75 protected final long globalMemStoreLimitLowMark;
76
77 private static final float DEFAULT_UPPER = 0.4f;
78 private static final float DEFAULT_LOWER = 0.35f;
79 private static final String UPPER_KEY =
80 "hbase.regionserver.global.memstore.upperLimit";
81 private static final String LOWER_KEY =
82 "hbase.regionserver.global.memstore.lowerLimit";
83 private long blockingStoreFilesNumber;
84 private long blockingWaitTime;
85
86
87
88
89
90 public MemStoreFlusher(final Configuration conf,
91 final HRegionServer server) {
92 super();
93 this.server = server;
94 this.threadWakeFrequency =
95 conf.getLong(HConstants.THREAD_WAKE_FREQUENCY, 10 * 1000);
96 long max = ManagementFactory.getMemoryMXBean().getHeapMemoryUsage().getMax();
97 this.globalMemStoreLimit = globalMemStoreLimit(max, DEFAULT_UPPER,
98 UPPER_KEY, conf);
99 long lower = globalMemStoreLimit(max, DEFAULT_LOWER, LOWER_KEY, conf);
100 if (lower > this.globalMemStoreLimit) {
101 lower = this.globalMemStoreLimit;
102 LOG.info("Setting globalMemStoreLimitLowMark == globalMemStoreLimit " +
103 "because supplied " + LOWER_KEY + " was > " + UPPER_KEY);
104 }
105 this.globalMemStoreLimitLowMark = lower;
106 this.blockingStoreFilesNumber =
107 conf.getInt("hbase.hstore.blockingStoreFiles", 7);
108 if (this.blockingStoreFilesNumber == -1) {
109 this.blockingStoreFilesNumber = 1 +
110 conf.getInt("hbase.hstore.compactionThreshold", 3);
111 }
112 this.blockingWaitTime = conf.getInt("hbase.hstore.blockingWaitTime",
113 90000);
114 LOG.info("globalMemStoreLimit=" +
115 StringUtils.humanReadableInt(this.globalMemStoreLimit) +
116 ", globalMemStoreLimitLowMark=" +
117 StringUtils.humanReadableInt(this.globalMemStoreLimitLowMark) +
118 ", maxHeap=" + StringUtils.humanReadableInt(max));
119 }
120
121
122
123
124
125
126
127
128
129
130 static long globalMemStoreLimit(final long max,
131 final float defaultLimit, final String key, final Configuration c) {
132 float limit = c.getFloat(key, defaultLimit);
133 return getMemStoreLimit(max, limit, defaultLimit);
134 }
135
136 static long getMemStoreLimit(final long max, final float limit,
137 final float defaultLimit) {
138 float effectiveLimit = limit;
139 if (limit >= 0.9f || limit < 0.1f) {
140 LOG.warn("Setting global memstore limit to default of " + defaultLimit +
141 " because supplied value outside allowed range of 0.1 -> 0.9");
142 effectiveLimit = defaultLimit;
143 }
144 return (long)(max * effectiveLimit);
145 }
146
147
148
149
150
151
152
153 private boolean flushOneForGlobalPressure() {
154 SortedMap<Long, HRegion> regionsBySize =
155 server.getCopyOfOnlineRegionsSortedBySize();
156
157 Set<HRegion> excludedRegions = new HashSet<HRegion>();
158
159 boolean flushedOne = false;
160 while (!flushedOne) {
161
162
163 HRegion bestFlushableRegion = getBiggestMemstoreRegion(
164 regionsBySize, excludedRegions, true);
165
166 HRegion bestAnyRegion = getBiggestMemstoreRegion(
167 regionsBySize, excludedRegions, false);
168
169 if (bestAnyRegion == null) {
170 LOG.error("Above memory mark but there are no flushable regions!");
171 return false;
172 }
173
174 HRegion regionToFlush;
175 if (bestFlushableRegion != null &&
176 bestAnyRegion.memstoreSize.get() > 2 * bestFlushableRegion.memstoreSize.get()) {
177
178
179
180
181 if (LOG.isDebugEnabled()) {
182 LOG.debug("Under global heap pressure: " +
183 "Region " + bestAnyRegion.getRegionNameAsString() + " has too many " +
184 "store files, but is " +
185 StringUtils.humanReadableInt(bestAnyRegion.memstoreSize.get()) +
186 " vs best flushable region's " +
187 StringUtils.humanReadableInt(bestFlushableRegion.memstoreSize.get()) +
188 ". Choosing the bigger.");
189 }
190 regionToFlush = bestAnyRegion;
191 } else {
192 if (bestFlushableRegion == null) {
193 regionToFlush = bestAnyRegion;
194 } else {
195 regionToFlush = bestFlushableRegion;
196 }
197 }
198
199 Preconditions.checkState(regionToFlush.memstoreSize.get() > 0);
200
201 LOG.info("Flush of region " + regionToFlush + " due to global heap pressure");
202 flushedOne = flushRegion(regionToFlush, true);
203 if (!flushedOne) {
204 LOG.info("Excluding unflushable region " + regionToFlush +
205 " - trying to find a different region to flush.");
206 excludedRegions.add(regionToFlush);
207 }
208 }
209 return true;
210 }
211
212 @Override
213 public void run() {
214 while (!this.server.isStopped()) {
215 FlushQueueEntry fqe = null;
216 try {
217 wakeupPending.set(false);
218 fqe = flushQueue.poll(threadWakeFrequency, TimeUnit.MILLISECONDS);
219 if (fqe == null || fqe instanceof WakeupFlushThread) {
220 if (isAboveLowWaterMark()) {
221 LOG.debug("Flush thread woke up because memory above low water=" +
222 StringUtils.humanReadableInt(this.globalMemStoreLimitLowMark));
223 if (!flushOneForGlobalPressure()) {
224
225
226
227
228
229 lock.lock();
230 try {
231 Thread.sleep(1000);
232 flushOccurred.signalAll();
233 } finally {
234 lock.unlock();
235 }
236 }
237
238 wakeupFlushThread();
239 }
240 continue;
241 }
242 FlushRegionEntry fre = (FlushRegionEntry)fqe;
243 if (!flushRegion(fre)) {
244 break;
245 }
246 } catch (InterruptedException ex) {
247 continue;
248 } catch (ConcurrentModificationException ex) {
249 continue;
250 } catch (Exception ex) {
251 LOG.error("Cache flusher failed for entry " + fqe, ex);
252 if (!server.checkFileSystem()) {
253 break;
254 }
255 }
256 }
257 this.regionsInQueue.clear();
258 this.flushQueue.clear();
259
260
261 lock.lock();
262 try {
263 flushOccurred.signalAll();
264 } finally {
265 lock.unlock();
266 }
267 LOG.info(getName() + " exiting");
268 }
269
270 private void wakeupFlushThread() {
271 if (wakeupPending.compareAndSet(false, true)) {
272 flushQueue.add(new WakeupFlushThread());
273 }
274 }
275
276 private HRegion getBiggestMemstoreRegion(
277 SortedMap<Long, HRegion> regionsBySize,
278 Set<HRegion> excludedRegions,
279 boolean checkStoreFileCount) {
280 synchronized (regionsInQueue) {
281 for (HRegion region : regionsBySize.values()) {
282 if (excludedRegions.contains(region)) {
283 continue;
284 }
285
286 if (checkStoreFileCount && isTooManyStoreFiles(region)) {
287 continue;
288 }
289 return region;
290 }
291 }
292 return null;
293 }
294
295
296
297
298 private boolean isAboveHighWaterMark() {
299 return server.getRegionServerAccounting().
300 getGlobalMemstoreSize() >= globalMemStoreLimit;
301 }
302
303
304
305
306 private boolean isAboveLowWaterMark() {
307 return server.getRegionServerAccounting().
308 getGlobalMemstoreSize() >= globalMemStoreLimitLowMark;
309 }
310
311 public void requestFlush(HRegion r) {
312 synchronized (regionsInQueue) {
313 if (!regionsInQueue.containsKey(r)) {
314
315
316 FlushRegionEntry fqe = new FlushRegionEntry(r);
317 this.regionsInQueue.put(r, fqe);
318 this.flushQueue.add(fqe);
319 }
320 }
321 }
322
323 public int getFlushQueueSize() {
324 return flushQueue.size();
325 }
326
327
328
329
330 void interruptIfNecessary() {
331 lock.lock();
332 try {
333 this.interrupt();
334 } finally {
335 lock.unlock();
336 }
337 }
338
339
340
341
342
343
344
345
346
347 private boolean flushRegion(final FlushRegionEntry fqe) {
348 HRegion region = fqe.region;
349 if (!fqe.region.getRegionInfo().isMetaRegion() &&
350 isTooManyStoreFiles(region)) {
351 if (fqe.isMaximumWait(this.blockingWaitTime)) {
352 LOG.info("Waited " + (System.currentTimeMillis() - fqe.createTime) +
353 "ms on a compaction to clean up 'too many store files'; waited " +
354 "long enough... proceeding with flush of " +
355 region.getRegionNameAsString());
356 } else {
357
358 if (fqe.getRequeueCount() <= 0) {
359
360 LOG.warn("Region " + region.getRegionNameAsString() + " has too many " +
361 "store files; delaying flush up to " + this.blockingWaitTime + "ms");
362 if (!this.server.compactSplitThread.requestSplit(region)) {
363 this.server.compactSplitThread.requestCompaction(region, getName());
364 }
365 }
366
367
368
369 this.flushQueue.add(fqe.requeue(this.blockingWaitTime / 100));
370
371 return true;
372 }
373 }
374 return flushRegion(region, false);
375 }
376
377
378
379
380
381
382
383
384
385
386
387
388
389 private boolean flushRegion(final HRegion region, final boolean emergencyFlush) {
390 synchronized (this.regionsInQueue) {
391 FlushRegionEntry fqe = this.regionsInQueue.remove(region);
392 if (fqe != null && emergencyFlush) {
393
394
395 flushQueue.remove(fqe);
396 }
397 lock.lock();
398 }
399 try {
400 if (region.flushcache()) {
401 server.compactSplitThread.requestCompaction(region, getName());
402 }
403 server.getMetrics().addFlush(region.getRecentFlushInfo());
404 } catch (DroppedSnapshotException ex) {
405
406
407
408
409
410 server.abort("Replay of HLog required. Forcing server shutdown", ex);
411 return false;
412 } catch (IOException ex) {
413 LOG.error("Cache flush failed" +
414 (region != null ? (" for region " + Bytes.toStringBinary(region.getRegionName())) : ""),
415 RemoteExceptionHandler.checkIOException(ex));
416 if (!server.checkFileSystem()) {
417 return false;
418 }
419 } finally {
420 flushOccurred.signalAll();
421 lock.unlock();
422 }
423 return true;
424 }
425
426 private boolean isTooManyStoreFiles(HRegion region) {
427 for (Store hstore: region.stores.values()) {
428 if (hstore.getStorefilesCount() > this.blockingStoreFilesNumber) {
429 return true;
430 }
431 }
432 return false;
433 }
434
435
436
437
438
439
440
441 public synchronized void reclaimMemStoreMemory() {
442 if (isAboveHighWaterMark()) {
443 lock.lock();
444 try {
445 while (isAboveHighWaterMark() && !server.isStopped()) {
446 wakeupFlushThread();
447 try {
448
449
450 flushOccurred.await(5, TimeUnit.SECONDS);
451 } catch (InterruptedException ie) {
452 Thread.currentThread().interrupt();
453 }
454 }
455 } finally {
456 lock.unlock();
457 }
458 } else if (isAboveLowWaterMark()) {
459 wakeupFlushThread();
460 }
461 }
462
463 interface FlushQueueEntry extends Delayed {}
464
465
466
467
468 static class WakeupFlushThread implements FlushQueueEntry {
469 @Override
470 public long getDelay(TimeUnit unit) {
471 return 0;
472 }
473
474 @Override
475 public int compareTo(Delayed o) {
476 return -1;
477 }
478 }
479
480
481
482
483
484
485
486
487
488 static class FlushRegionEntry implements FlushQueueEntry {
489 private final HRegion region;
490
491 private final long createTime;
492 private long whenToExpire;
493 private int requeueCount = 0;
494
495 FlushRegionEntry(final HRegion r) {
496 this.region = r;
497 this.createTime = System.currentTimeMillis();
498 this.whenToExpire = this.createTime;
499 }
500
501
502
503
504
505 public boolean isMaximumWait(final long maximumWait) {
506 return (System.currentTimeMillis() - this.createTime) > maximumWait;
507 }
508
509
510
511
512
513 public int getRequeueCount() {
514 return this.requeueCount;
515 }
516
517
518
519
520
521
522
523 public FlushRegionEntry requeue(final long when) {
524 this.whenToExpire = System.currentTimeMillis() + when;
525 this.requeueCount++;
526 return this;
527 }
528
529 @Override
530 public long getDelay(TimeUnit unit) {
531 return unit.convert(this.whenToExpire - System.currentTimeMillis(),
532 TimeUnit.MILLISECONDS);
533 }
534
535 @Override
536 public int compareTo(Delayed other) {
537 return Long.valueOf(getDelay(TimeUnit.MILLISECONDS) -
538 other.getDelay(TimeUnit.MILLISECONDS)).intValue();
539 }
540
541 @Override
542 public String toString() {
543 return "[flush region " + Bytes.toStringBinary(region.getRegionName()) + "]";
544 }
545 }
546 }