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 org.apache.commons.logging.Log;
23 import org.apache.commons.logging.LogFactory;
24 import org.apache.hadoop.conf.Configuration;
25 import org.apache.hadoop.hbase.DroppedSnapshotException;
26 import org.apache.hadoop.hbase.HConstants;
27 import org.apache.hadoop.hbase.RemoteExceptionHandler;
28 import org.apache.hadoop.hbase.util.Bytes;
29 import org.apache.hadoop.util.StringUtils;
30
31 import com.google.common.base.Preconditions;
32
33 import java.io.IOException;
34 import java.lang.management.ManagementFactory;
35 import java.util.ConcurrentModificationException;
36 import java.util.EnumSet;
37 import java.util.HashMap;
38 import java.util.Map;
39 import java.util.Set;
40 import java.util.SortedMap;
41 import java.util.TreeSet;
42 import java.util.concurrent.BlockingQueue;
43 import java.util.concurrent.DelayQueue;
44 import java.util.concurrent.Delayed;
45 import java.util.concurrent.TimeUnit;
46 import java.util.concurrent.atomic.AtomicBoolean;
47 import java.util.concurrent.locks.Condition;
48 import java.util.concurrent.locks.ReentrantLock;
49
50
51
52
53
54
55
56
57
58
59 class MemStoreFlusher extends Thread 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 if (limit >= 0.9f || limit < 0.1f) {
139 LOG.warn("Setting global memstore limit to default of " + defaultLimit +
140 " because supplied value outside allowed range of 0.1 -> 0.9");
141 }
142 return (long)(max * limit);
143 }
144
145
146
147
148
149
150
151 private boolean flushOneForGlobalPressure() {
152 SortedMap<Long, HRegion> regionsBySize =
153 server.getCopyOfOnlineRegionsSortedBySize();
154
155
156
157 Set<byte[]> excludedRegionNames = new TreeSet<byte[]>(Bytes.BYTES_COMPARATOR);
158
159 boolean flushedOne = false;
160 while (!flushedOne) {
161
162 HRegion bestFlushableRegion = getBiggestMemstoreRegion(
163 regionsBySize, excludedRegionNames, true);
164
165 HRegion bestAnyRegion = getBiggestMemstoreRegion(
166 regionsBySize, excludedRegionNames, false);
167
168 if (bestAnyRegion == null) {
169 LOG.fatal("Above memory mark but there are no flushable regions!");
170 return false;
171 }
172
173 HRegion regionToFlush;
174 if (bestAnyRegion.memstoreSize.get() > 2 * bestFlushableRegion.memstoreSize.get()) {
175
176
177
178
179 LOG.info("Under global heap pressure: " +
180 "Region " + bestAnyRegion.getRegionNameAsString() + " has too many " +
181 "store files, but is " +
182 StringUtils.humanReadableInt(bestAnyRegion.memstoreSize.get()) +
183 " vs best flushable region's " +
184 StringUtils.humanReadableInt(bestFlushableRegion.memstoreSize.get()) +
185 ". Choosing the bigger.");
186 regionToFlush = bestAnyRegion;
187 } else {
188 regionToFlush = bestFlushableRegion;
189 }
190
191 Preconditions.checkState(regionToFlush.memstoreSize.get() > 0);
192
193 LOG.info("Flush of region " + regionToFlush + " due to global heap pressure");
194 flushedOne = flushRegion(regionToFlush, true);
195 if (!flushedOne) {
196 LOG.info("Excluding unflushable region " + regionToFlush +
197 " - trying to find a different region to flush.");
198 excludedRegionNames.add(regionToFlush.getRegionName());
199 }
200 }
201 return true;
202 }
203
204 @Override
205 public void run() {
206 while (!this.server.isStopped()) {
207 FlushQueueEntry fqe = null;
208 try {
209 wakeupPending.set(false);
210 fqe = flushQueue.poll(threadWakeFrequency, TimeUnit.MILLISECONDS);
211 if (fqe == null || fqe instanceof WakeupFlushThread) {
212 if (isAboveLowWaterMark()) {
213 LOG.info("Flush thread woke up with memory above low water.");
214 if (!flushOneForGlobalPressure()) {
215
216
217
218
219
220 lock.lock();
221 try {
222 Thread.sleep(1000);
223 flushOccurred.signalAll();
224 } finally {
225 lock.unlock();
226 }
227 }
228
229 wakeupFlushThread();
230 }
231 continue;
232 }
233 FlushRegionEntry fre = (FlushRegionEntry)fqe;
234 if (!flushRegion(fre)) {
235 break;
236 }
237 } catch (InterruptedException ex) {
238 continue;
239 } catch (ConcurrentModificationException ex) {
240 continue;
241 } catch (Exception ex) {
242 LOG.error("Cache flusher failed for entry " + fqe);
243 if (!server.checkFileSystem()) {
244 break;
245 }
246 }
247 }
248 this.regionsInQueue.clear();
249 this.flushQueue.clear();
250
251
252 lock.lock();
253 try {
254 flushOccurred.signalAll();
255 } finally {
256 lock.unlock();
257 }
258 LOG.info(getName() + " exiting");
259 }
260
261 private void wakeupFlushThread() {
262 if (wakeupPending.compareAndSet(false, true)) {
263 flushQueue.add(new WakeupFlushThread());
264 }
265 }
266
267 private HRegion getBiggestMemstoreRegion(
268 SortedMap<Long, HRegion> regionsBySize,
269 Set<byte[]> excludedRegionNames,
270 boolean checkStoreFileCount) {
271 synchronized (regionsInQueue) {
272 for (HRegion region : regionsBySize.values()) {
273 if (excludedRegionNames.contains(region.getRegionName())) {
274 continue;
275 }
276
277 if (checkStoreFileCount && isTooManyStoreFiles(region)) {
278 continue;
279 }
280 return region;
281 }
282 }
283 return null;
284 }
285
286
287
288
289 private boolean isAboveHighWaterMark() {
290 return server.getGlobalMemStoreSize() >= globalMemStoreLimit;
291 }
292
293
294
295
296 private boolean isAboveLowWaterMark() {
297 return server.getGlobalMemStoreSize() >= globalMemStoreLimitLowMark;
298 }
299
300 public void requestFlush(HRegion r) {
301 synchronized (regionsInQueue) {
302 if (!regionsInQueue.containsKey(r)) {
303
304
305 FlushRegionEntry fqe = new FlushRegionEntry(r);
306 this.regionsInQueue.put(r, fqe);
307 this.flushQueue.add(fqe);
308 }
309 }
310 }
311
312 public int getFlushQueueSize() {
313 return flushQueue.size();
314 }
315
316
317
318
319 void interruptIfNecessary() {
320 lock.lock();
321 try {
322 this.interrupt();
323 } finally {
324 lock.unlock();
325 }
326 }
327
328
329
330
331
332
333
334
335
336 private boolean flushRegion(final FlushRegionEntry fqe) {
337 HRegion region = fqe.region;
338 if (!fqe.region.getRegionInfo().isMetaRegion() &&
339 isTooManyStoreFiles(region)) {
340 if (fqe.isMaximumWait(this.blockingWaitTime)) {
341 LOG.info("Waited " + (System.currentTimeMillis() - fqe.createTime) +
342 "ms on a compaction to clean up 'too many store files'; waited " +
343 "long enough... proceeding with flush of " +
344 region.getRegionNameAsString());
345 } else {
346
347 if (fqe.getRequeueCount() <= 0) {
348
349 LOG.warn("Region " + region.getRegionNameAsString() + " has too many " +
350 "store files; delaying flush up to " + this.blockingWaitTime + "ms");
351 }
352 this.server.compactSplitThread.requestCompaction(region, getName());
353
354
355 this.flushQueue.add(fqe.requeue(this.blockingWaitTime / 100));
356
357 return true;
358 }
359 }
360 return flushRegion(region, false);
361 }
362
363
364
365
366
367
368
369
370
371
372
373
374
375 private boolean flushRegion(final HRegion region, final boolean emergencyFlush) {
376 synchronized (this.regionsInQueue) {
377 FlushRegionEntry fqe = this.regionsInQueue.remove(region);
378 if (fqe != null && emergencyFlush) {
379
380
381 flushQueue.remove(fqe);
382 }
383 lock.lock();
384 }
385 try {
386 if (region.flushcache()) {
387 server.compactSplitThread.requestCompaction(region, getName());
388 }
389 server.getMetrics().addFlush(region.getRecentFlushInfo());
390 } catch (DroppedSnapshotException ex) {
391
392
393
394
395
396 server.abort("Replay of HLog required. Forcing server shutdown", ex);
397 return false;
398 } catch (IOException ex) {
399 LOG.error("Cache flush failed" +
400 (region != null ? (" for region " + Bytes.toString(region.getRegionName())) : ""),
401 RemoteExceptionHandler.checkIOException(ex));
402 if (!server.checkFileSystem()) {
403 return false;
404 }
405 } finally {
406 flushOccurred.signalAll();
407 lock.unlock();
408 }
409 return true;
410 }
411
412 private boolean isTooManyStoreFiles(HRegion region) {
413 for (Store hstore: region.stores.values()) {
414 if (hstore.getStorefilesCount() > this.blockingStoreFilesNumber) {
415 return true;
416 }
417 }
418 return false;
419 }
420
421
422
423
424
425
426
427 public synchronized void reclaimMemStoreMemory() {
428 if (isAboveHighWaterMark()) {
429 lock.lock();
430 try {
431 while (isAboveHighWaterMark() && !server.isStopped()) {
432 wakeupFlushThread();
433 try {
434
435
436 flushOccurred.await(5, TimeUnit.SECONDS);
437 } catch (InterruptedException ie) {
438 Thread.currentThread().interrupt();
439 }
440 }
441 } finally {
442 lock.unlock();
443 }
444 } else if (isAboveLowWaterMark()) {
445 wakeupFlushThread();
446 }
447 }
448
449 interface FlushQueueEntry extends Delayed {}
450
451
452
453
454 static class WakeupFlushThread implements FlushQueueEntry {
455 @Override
456 public long getDelay(TimeUnit unit) {
457 return 0;
458 }
459
460 @Override
461 public int compareTo(Delayed o) {
462 return -1;
463 }
464 }
465
466
467
468
469
470
471
472
473
474 static class FlushRegionEntry implements FlushQueueEntry {
475 private final HRegion region;
476
477 private final long createTime;
478 private long whenToExpire;
479 private int requeueCount = 0;
480
481 FlushRegionEntry(final HRegion r) {
482 this.region = r;
483 this.createTime = System.currentTimeMillis();
484 this.whenToExpire = this.createTime;
485 }
486
487
488
489
490
491 public boolean isMaximumWait(final long maximumWait) {
492 return (System.currentTimeMillis() - this.createTime) > maximumWait;
493 }
494
495
496
497
498
499 public int getRequeueCount() {
500 return this.requeueCount;
501 }
502
503
504
505
506
507
508
509 public FlushRegionEntry requeue(final long when) {
510 this.whenToExpire = System.currentTimeMillis() + when;
511 this.requeueCount++;
512 return this;
513 }
514
515 @Override
516 public long getDelay(TimeUnit unit) {
517 return unit.convert(this.whenToExpire - System.currentTimeMillis(),
518 TimeUnit.MILLISECONDS);
519 }
520
521 @Override
522 public int compareTo(Delayed other) {
523 return Long.valueOf(getDelay(TimeUnit.MILLISECONDS) -
524 other.getDelay(TimeUnit.MILLISECONDS)).intValue();
525 }
526
527 @Override
528 public String toString() {
529 return "[flush region " + Bytes.toString(region.getRegionName()) + "]";
530 }
531 }
532 }