1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.io.hfile;
20
21 import java.io.IOException;
22 import java.lang.ref.WeakReference;
23 import java.nio.ByteBuffer;
24 import java.util.ArrayList;
25 import java.util.Collections;
26 import java.util.EnumMap;
27 import java.util.HashMap;
28 import java.util.List;
29 import java.util.Map;
30 import java.util.PriorityQueue;
31 import java.util.SortedSet;
32 import java.util.TreeSet;
33 import java.util.concurrent.ConcurrentHashMap;
34 import java.util.concurrent.Executors;
35 import java.util.concurrent.ScheduledExecutorService;
36 import java.util.concurrent.TimeUnit;
37 import java.util.concurrent.atomic.AtomicLong;
38 import java.util.concurrent.locks.ReentrantLock;
39
40 import org.apache.commons.logging.Log;
41 import org.apache.commons.logging.LogFactory;
42 import org.apache.hadoop.classification.InterfaceAudience;
43 import org.apache.hadoop.conf.Configuration;
44 import org.apache.hadoop.fs.FileSystem;
45 import org.apache.hadoop.fs.Path;
46 import org.apache.hadoop.hbase.io.HeapSize;
47 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
48 import org.apache.hadoop.hbase.io.hfile.CachedBlock.BlockPriority;
49 import org.apache.hadoop.hbase.io.hfile.bucket.BucketCache;
50 import org.apache.hadoop.hbase.util.Bytes;
51 import org.apache.hadoop.hbase.util.ClassSize;
52 import org.apache.hadoop.hbase.util.FSUtils;
53 import org.apache.hadoop.hbase.util.HasThread;
54 import org.apache.hadoop.hbase.util.Threads;
55 import org.apache.hadoop.util.StringUtils;
56
57 import com.google.common.util.concurrent.ThreadFactoryBuilder;
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97 @InterfaceAudience.Private
98 public class LruBlockCache implements BlockCache, HeapSize {
99
100 static final Log LOG = LogFactory.getLog(LruBlockCache.class);
101
102 static final String LRU_MIN_FACTOR_CONFIG_NAME = "hbase.lru.blockcache.min.factor";
103 static final String LRU_ACCEPTABLE_FACTOR_CONFIG_NAME = "hbase.lru.blockcache.acceptable.factor";
104
105
106
107
108 static final float DEFAULT_LOAD_FACTOR = 0.75f;
109 static final int DEFAULT_CONCURRENCY_LEVEL = 16;
110
111
112 static final float DEFAULT_MIN_FACTOR = 0.95f;
113 static final float DEFAULT_ACCEPTABLE_FACTOR = 0.99f;
114
115
116 static final float DEFAULT_SINGLE_FACTOR = 0.25f;
117 static final float DEFAULT_MULTI_FACTOR = 0.50f;
118 static final float DEFAULT_MEMORY_FACTOR = 0.25f;
119
120
121 static final int statThreadPeriod = 60 * 5;
122
123
124 private final ConcurrentHashMap<BlockCacheKey,CachedBlock> map;
125
126
127 private final ReentrantLock evictionLock = new ReentrantLock(true);
128
129
130 private volatile boolean evictionInProgress = false;
131
132
133 private final EvictionThread evictionThread;
134
135
136 private final ScheduledExecutorService scheduleThreadPool =
137 Executors.newScheduledThreadPool(1,
138 new ThreadFactoryBuilder()
139 .setNameFormat("LRU Statistics #%d")
140 .setDaemon(true)
141 .build());
142
143
144 private final AtomicLong size;
145
146
147 private final AtomicLong elements;
148
149
150 private final AtomicLong count;
151
152
153 private final CacheStats stats;
154
155
156 private long maxSize;
157
158
159 private long blockSize;
160
161
162 private float acceptableFactor;
163
164
165 private float minFactor;
166
167
168 private float singleFactor;
169
170
171 private float multiFactor;
172
173
174 private float memoryFactor;
175
176
177 private long overhead;
178
179
180 private BucketCache victimHandler = null;
181
182
183
184
185
186
187
188
189
190
191 public LruBlockCache(long maxSize, long blockSize) {
192 this(maxSize, blockSize, true);
193 }
194
195
196
197
198 public LruBlockCache(long maxSize, long blockSize, boolean evictionThread) {
199 this(maxSize, blockSize, evictionThread,
200 (int)Math.ceil(1.2*maxSize/blockSize),
201 DEFAULT_LOAD_FACTOR, DEFAULT_CONCURRENCY_LEVEL,
202 DEFAULT_MIN_FACTOR, DEFAULT_ACCEPTABLE_FACTOR,
203 DEFAULT_SINGLE_FACTOR, DEFAULT_MULTI_FACTOR,
204 DEFAULT_MEMORY_FACTOR);
205 }
206
207 public LruBlockCache(long maxSize, long blockSize, boolean evictionThread, Configuration conf) {
208 this(maxSize, blockSize, evictionThread,
209 (int)Math.ceil(1.2*maxSize/blockSize),
210 DEFAULT_LOAD_FACTOR,
211 DEFAULT_CONCURRENCY_LEVEL,
212 conf.getFloat(LRU_MIN_FACTOR_CONFIG_NAME, DEFAULT_MIN_FACTOR),
213 conf.getFloat(LRU_ACCEPTABLE_FACTOR_CONFIG_NAME, DEFAULT_ACCEPTABLE_FACTOR),
214 DEFAULT_SINGLE_FACTOR,
215 DEFAULT_MULTI_FACTOR,
216 DEFAULT_MEMORY_FACTOR);
217 }
218
219 public LruBlockCache(long maxSize, long blockSize, Configuration conf) {
220 this(maxSize, blockSize, true, conf);
221 }
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237 public LruBlockCache(long maxSize, long blockSize, boolean evictionThread,
238 int mapInitialSize, float mapLoadFactor, int mapConcurrencyLevel,
239 float minFactor, float acceptableFactor,
240 float singleFactor, float multiFactor, float memoryFactor) {
241 if(singleFactor + multiFactor + memoryFactor != 1) {
242 throw new IllegalArgumentException("Single, multi, and memory factors " +
243 " should total 1.0");
244 }
245 if(minFactor >= acceptableFactor) {
246 throw new IllegalArgumentException("minFactor must be smaller than acceptableFactor");
247 }
248 if(minFactor >= 1.0f || acceptableFactor >= 1.0f) {
249 throw new IllegalArgumentException("all factors must be < 1");
250 }
251 this.maxSize = maxSize;
252 this.blockSize = blockSize;
253 map = new ConcurrentHashMap<BlockCacheKey,CachedBlock>(mapInitialSize,
254 mapLoadFactor, mapConcurrencyLevel);
255 this.minFactor = minFactor;
256 this.acceptableFactor = acceptableFactor;
257 this.singleFactor = singleFactor;
258 this.multiFactor = multiFactor;
259 this.memoryFactor = memoryFactor;
260 this.stats = new CacheStats();
261 this.count = new AtomicLong(0);
262 this.elements = new AtomicLong(0);
263 this.overhead = calculateOverhead(maxSize, blockSize, mapConcurrencyLevel);
264 this.size = new AtomicLong(this.overhead);
265 if(evictionThread) {
266 this.evictionThread = new EvictionThread(this);
267 this.evictionThread.start();
268 } else {
269 this.evictionThread = null;
270 }
271 this.scheduleThreadPool.scheduleAtFixedRate(new StatisticsThread(this),
272 statThreadPeriod, statThreadPeriod, TimeUnit.SECONDS);
273 }
274
275 public void setMaxSize(long maxSize) {
276 this.maxSize = maxSize;
277 if(this.size.get() > acceptableSize() && !evictionInProgress) {
278 runEviction();
279 }
280 }
281
282
283
284
285
286
287
288
289
290
291
292
293 @Override
294 public void cacheBlock(BlockCacheKey cacheKey, Cacheable buf, boolean inMemory) {
295 CachedBlock cb = map.get(cacheKey);
296 if(cb != null) {
297
298 if (compare(buf, cb.getBuffer()) != 0) {
299 throw new RuntimeException("Cached block contents differ, which should not have happened."
300 + "cacheKey:" + cacheKey);
301 }
302 String msg = "Cached an already cached block: " + cacheKey + " cb:" + cb.getCacheKey();
303 msg += ". This is harmless and can happen in rare cases (see HBASE-8547)";
304 LOG.warn(msg);
305 return;
306 }
307 cb = new CachedBlock(cacheKey, buf, count.incrementAndGet(), inMemory);
308 long newSize = updateSizeMetrics(cb, false);
309 map.put(cacheKey, cb);
310 elements.incrementAndGet();
311 if(newSize > acceptableSize() && !evictionInProgress) {
312 runEviction();
313 }
314 }
315
316 private int compare(Cacheable left, Cacheable right) {
317 ByteBuffer l = ByteBuffer.allocate(left.getSerializedLength());
318 left.serialize(l);
319 ByteBuffer r = ByteBuffer.allocate(right.getSerializedLength());
320 right.serialize(r);
321 return Bytes.compareTo(l.array(), l.arrayOffset(), l.limit(),
322 r.array(), r.arrayOffset(), r.limit());
323 }
324
325
326
327
328
329
330
331
332
333
334
335 public void cacheBlock(BlockCacheKey cacheKey, Cacheable buf) {
336 cacheBlock(cacheKey, buf, false);
337 }
338
339
340
341
342
343
344
345
346
347 protected long updateSizeMetrics(CachedBlock cb, boolean evict) {
348 long heapsize = cb.heapSize();
349 if (evict) {
350 heapsize *= -1;
351 }
352 return size.addAndGet(heapsize);
353 }
354
355
356
357
358
359
360
361
362
363
364 @Override
365 public Cacheable getBlock(BlockCacheKey cacheKey, boolean caching, boolean repeat) {
366 CachedBlock cb = map.get(cacheKey);
367 if(cb == null) {
368 if (!repeat) stats.miss(caching);
369 if (victimHandler != null)
370 return victimHandler.getBlock(cacheKey, caching, repeat);
371 return null;
372 }
373 stats.hit(caching);
374 cb.access(count.incrementAndGet());
375 return cb.getBuffer();
376 }
377
378
379
380
381
382
383 public boolean containsBlock(BlockCacheKey cacheKey) {
384 return map.containsKey(cacheKey);
385 }
386
387 @Override
388 public boolean evictBlock(BlockCacheKey cacheKey) {
389 CachedBlock cb = map.get(cacheKey);
390 if (cb == null) return false;
391 evictBlock(cb, false);
392 return true;
393 }
394
395
396
397
398
399
400
401
402
403
404
405 @Override
406 public int evictBlocksByHfileName(String hfileName) {
407 int numEvicted = 0;
408 for (BlockCacheKey key : map.keySet()) {
409 if (key.getHfileName().equals(hfileName)) {
410 if (evictBlock(key))
411 ++numEvicted;
412 }
413 }
414 if (victimHandler != null) {
415 numEvicted += victimHandler.evictBlocksByHfileName(hfileName);
416 }
417 return numEvicted;
418 }
419
420
421
422
423
424
425
426
427
428 protected long evictBlock(CachedBlock block, boolean evictedByEvictionProcess) {
429 map.remove(block.getCacheKey());
430 updateSizeMetrics(block, true);
431 elements.decrementAndGet();
432 stats.evicted();
433 if (evictedByEvictionProcess && victimHandler != null) {
434 boolean wait = getCurrentSize() < acceptableSize();
435 boolean inMemory = block.getPriority() == BlockPriority.MEMORY;
436 victimHandler.cacheBlockWithWait(block.getCacheKey(), block.getBuffer(),
437 inMemory, wait);
438 }
439 return block.heapSize();
440 }
441
442
443
444
445 private void runEviction() {
446 if(evictionThread == null) {
447 evict();
448 } else {
449 evictionThread.evict();
450 }
451 }
452
453
454
455
456 void evict() {
457
458
459 if(!evictionLock.tryLock()) return;
460
461 try {
462 evictionInProgress = true;
463 long currentSize = this.size.get();
464 long bytesToFree = currentSize - minSize();
465
466 if (LOG.isTraceEnabled()) {
467 LOG.trace("Block cache LRU eviction started; Attempting to free " +
468 StringUtils.byteDesc(bytesToFree) + " of total=" +
469 StringUtils.byteDesc(currentSize));
470 }
471
472 if(bytesToFree <= 0) return;
473
474
475 BlockBucket bucketSingle = new BlockBucket(bytesToFree, blockSize,
476 singleSize());
477 BlockBucket bucketMulti = new BlockBucket(bytesToFree, blockSize,
478 multiSize());
479 BlockBucket bucketMemory = new BlockBucket(bytesToFree, blockSize,
480 memorySize());
481
482
483 for(CachedBlock cachedBlock : map.values()) {
484 switch(cachedBlock.getPriority()) {
485 case SINGLE: {
486 bucketSingle.add(cachedBlock);
487 break;
488 }
489 case MULTI: {
490 bucketMulti.add(cachedBlock);
491 break;
492 }
493 case MEMORY: {
494 bucketMemory.add(cachedBlock);
495 break;
496 }
497 }
498 }
499
500 PriorityQueue<BlockBucket> bucketQueue =
501 new PriorityQueue<BlockBucket>(3);
502
503 bucketQueue.add(bucketSingle);
504 bucketQueue.add(bucketMulti);
505 bucketQueue.add(bucketMemory);
506
507 int remainingBuckets = 3;
508 long bytesFreed = 0;
509
510 BlockBucket bucket;
511 while((bucket = bucketQueue.poll()) != null) {
512 long overflow = bucket.overflow();
513 if(overflow > 0) {
514 long bucketBytesToFree = Math.min(overflow,
515 (bytesToFree - bytesFreed) / remainingBuckets);
516 bytesFreed += bucket.free(bucketBytesToFree);
517 }
518 remainingBuckets--;
519 }
520
521 if (LOG.isTraceEnabled()) {
522 long single = bucketSingle.totalSize();
523 long multi = bucketMulti.totalSize();
524 long memory = bucketMemory.totalSize();
525 LOG.trace("Block cache LRU eviction completed; " +
526 "freed=" + StringUtils.byteDesc(bytesFreed) + ", " +
527 "total=" + StringUtils.byteDesc(this.size.get()) + ", " +
528 "single=" + StringUtils.byteDesc(single) + ", " +
529 "multi=" + StringUtils.byteDesc(multi) + ", " +
530 "memory=" + StringUtils.byteDesc(memory));
531 }
532 } finally {
533 stats.evict();
534 evictionInProgress = false;
535 evictionLock.unlock();
536 }
537 }
538
539
540
541
542
543
544
545 private class BlockBucket implements Comparable<BlockBucket> {
546
547 private CachedBlockQueue queue;
548 private long totalSize = 0;
549 private long bucketSize;
550
551 public BlockBucket(long bytesToFree, long blockSize, long bucketSize) {
552 this.bucketSize = bucketSize;
553 queue = new CachedBlockQueue(bytesToFree, blockSize);
554 totalSize = 0;
555 }
556
557 public void add(CachedBlock block) {
558 totalSize += block.heapSize();
559 queue.add(block);
560 }
561
562 public long free(long toFree) {
563 CachedBlock cb;
564 long freedBytes = 0;
565 while ((cb = queue.pollLast()) != null) {
566 freedBytes += evictBlock(cb, true);
567 if (freedBytes >= toFree) {
568 return freedBytes;
569 }
570 }
571 return freedBytes;
572 }
573
574 public long overflow() {
575 return totalSize - bucketSize;
576 }
577
578 public long totalSize() {
579 return totalSize;
580 }
581
582 public int compareTo(BlockBucket that) {
583 if(this.overflow() == that.overflow()) return 0;
584 return this.overflow() > that.overflow() ? 1 : -1;
585 }
586
587 @Override
588 public boolean equals(Object that) {
589 if (that == null || !(that instanceof BlockBucket)){
590 return false;
591 }
592
593 return compareTo(( BlockBucket)that) == 0;
594 }
595
596 }
597
598
599
600
601
602 public long getMaxSize() {
603 return this.maxSize;
604 }
605
606
607
608
609
610 public long getCurrentSize() {
611 return this.size.get();
612 }
613
614
615
616
617
618 public long getFreeSize() {
619 return getMaxSize() - getCurrentSize();
620 }
621
622
623
624
625
626 public long size() {
627 return this.elements.get();
628 }
629
630 @Override
631 public long getBlockCount() {
632 return this.elements.get();
633 }
634
635
636
637
638 public long getEvictionCount() {
639 return this.stats.getEvictionCount();
640 }
641
642
643
644
645
646 public long getEvictedCount() {
647 return this.stats.getEvictedCount();
648 }
649
650 EvictionThread getEvictionThread() {
651 return this.evictionThread;
652 }
653
654
655
656
657
658
659
660 static class EvictionThread extends HasThread {
661 private WeakReference<LruBlockCache> cache;
662 private boolean go = true;
663
664 private boolean enteringRun = false;
665
666 public EvictionThread(LruBlockCache cache) {
667 super(Thread.currentThread().getName() + ".LruBlockCache.EvictionThread");
668 setDaemon(true);
669 this.cache = new WeakReference<LruBlockCache>(cache);
670 }
671
672 @Override
673 public void run() {
674 enteringRun = true;
675 while (this.go) {
676 synchronized(this) {
677 try {
678 this.wait();
679 } catch(InterruptedException e) {}
680 }
681 LruBlockCache cache = this.cache.get();
682 if(cache == null) break;
683 cache.evict();
684 }
685 }
686
687 public void evict() {
688 synchronized(this) {
689 this.notifyAll();
690 }
691 }
692
693 synchronized void shutdown() {
694 this.go = false;
695 this.notifyAll();
696 }
697
698
699
700
701 boolean isEnteringRun() {
702 return this.enteringRun;
703 }
704 }
705
706
707
708
709 static class StatisticsThread extends Thread {
710 LruBlockCache lru;
711
712 public StatisticsThread(LruBlockCache lru) {
713 super("LruBlockCache.StatisticsThread");
714 setDaemon(true);
715 this.lru = lru;
716 }
717 @Override
718 public void run() {
719 lru.logStats();
720 }
721 }
722
723 public void logStats() {
724 if (!LOG.isDebugEnabled()) return;
725
726 long totalSize = heapSize();
727 long freeSize = maxSize - totalSize;
728 LruBlockCache.LOG.debug("Stats: " +
729 "total=" + StringUtils.byteDesc(totalSize) + ", " +
730 "free=" + StringUtils.byteDesc(freeSize) + ", " +
731 "max=" + StringUtils.byteDesc(this.maxSize) + ", " +
732 "blocks=" + size() +", " +
733 "accesses=" + stats.getRequestCount() + ", " +
734 "hits=" + stats.getHitCount() + ", " +
735 "hitRatio=" +
736 (stats.getHitCount() == 0 ? "0" : (StringUtils.formatPercent(stats.getHitRatio(), 2)+ ", ")) + ", " +
737 "cachingAccesses=" + stats.getRequestCachingCount() + ", " +
738 "cachingHits=" + stats.getHitCachingCount() + ", " +
739 "cachingHitsRatio=" +
740 (stats.getHitCachingCount() == 0 ? "0,": (StringUtils.formatPercent(stats.getHitCachingRatio(), 2) + ", ")) +
741 "evictions=" + stats.getEvictionCount() + ", " +
742 "evicted=" + stats.getEvictedCount() + ", " +
743 "evictedPerRun=" + stats.evictedPerEviction());
744 }
745
746
747
748
749
750
751
752 public CacheStats getStats() {
753 return this.stats;
754 }
755
756 public final static long CACHE_FIXED_OVERHEAD = ClassSize.align(
757 (3 * Bytes.SIZEOF_LONG) + (9 * ClassSize.REFERENCE) +
758 (5 * Bytes.SIZEOF_FLOAT) + Bytes.SIZEOF_BOOLEAN
759 + ClassSize.OBJECT);
760
761
762 public long heapSize() {
763 return getCurrentSize();
764 }
765
766 public static long calculateOverhead(long maxSize, long blockSize, int concurrency){
767
768 return CACHE_FIXED_OVERHEAD + ClassSize.CONCURRENT_HASHMAP +
769 ((long)Math.ceil(maxSize*1.2/blockSize)
770 * ClassSize.CONCURRENT_HASHMAP_ENTRY) +
771 ((long)concurrency * ClassSize.CONCURRENT_HASHMAP_SEGMENT);
772 }
773
774 @Override
775 public List<BlockCacheColumnFamilySummary> getBlockCacheColumnFamilySummaries(Configuration conf) throws IOException {
776
777 Map<String, Path> sfMap = FSUtils.getTableStoreFilePathMap(
778 FileSystem.get(conf),
779 FSUtils.getRootDir(conf));
780
781
782
783 Map<BlockCacheColumnFamilySummary, BlockCacheColumnFamilySummary> bcs =
784 new HashMap<BlockCacheColumnFamilySummary, BlockCacheColumnFamilySummary>();
785
786 for (CachedBlock cb : map.values()) {
787 String sf = cb.getCacheKey().getHfileName();
788 Path path = sfMap.get(sf);
789 if ( path != null) {
790 BlockCacheColumnFamilySummary lookup =
791 BlockCacheColumnFamilySummary.createFromStoreFilePath(path);
792 BlockCacheColumnFamilySummary bcse = bcs.get(lookup);
793 if (bcse == null) {
794 bcse = BlockCacheColumnFamilySummary.create(lookup);
795 bcs.put(lookup,bcse);
796 }
797 bcse.incrementBlocks();
798 bcse.incrementHeapSize(cb.heapSize());
799 }
800 }
801 List<BlockCacheColumnFamilySummary> list =
802 new ArrayList<BlockCacheColumnFamilySummary>(bcs.values());
803 Collections.sort( list );
804 return list;
805 }
806
807
808
809 private long acceptableSize() {
810 return (long)Math.floor(this.maxSize * this.acceptableFactor);
811 }
812 private long minSize() {
813 return (long)Math.floor(this.maxSize * this.minFactor);
814 }
815 private long singleSize() {
816 return (long)Math.floor(this.maxSize * this.singleFactor * this.minFactor);
817 }
818 private long multiSize() {
819 return (long)Math.floor(this.maxSize * this.multiFactor * this.minFactor);
820 }
821 private long memorySize() {
822 return (long)Math.floor(this.maxSize * this.memoryFactor * this.minFactor);
823 }
824
825 public void shutdown() {
826 if (victimHandler != null)
827 victimHandler.shutdown();
828 this.scheduleThreadPool.shutdown();
829 for (int i = 0; i < 10; i++) {
830 if (!this.scheduleThreadPool.isShutdown()) Threads.sleep(10);
831 }
832 if (!this.scheduleThreadPool.isShutdown()) {
833 List<Runnable> runnables = this.scheduleThreadPool.shutdownNow();
834 LOG.debug("Still running " + runnables);
835 }
836 this.evictionThread.shutdown();
837 }
838
839
840 public void clearCache() {
841 map.clear();
842 }
843
844
845
846
847
848 SortedSet<String> getCachedFileNamesForTest() {
849 SortedSet<String> fileNames = new TreeSet<String>();
850 for (BlockCacheKey cacheKey : map.keySet()) {
851 fileNames.add(cacheKey.getHfileName());
852 }
853 return fileNames;
854 }
855
856 Map<BlockType, Integer> getBlockTypeCountsForTest() {
857 Map<BlockType, Integer> counts =
858 new EnumMap<BlockType, Integer>(BlockType.class);
859 for (CachedBlock cb : map.values()) {
860 BlockType blockType = ((HFileBlock) cb.getBuffer()).getBlockType();
861 Integer count = counts.get(blockType);
862 counts.put(blockType, (count == null ? 0 : count) + 1);
863 }
864 return counts;
865 }
866
867 public Map<DataBlockEncoding, Integer> getEncodingCountsForTest() {
868 Map<DataBlockEncoding, Integer> counts =
869 new EnumMap<DataBlockEncoding, Integer>(DataBlockEncoding.class);
870 for (BlockCacheKey cacheKey : map.keySet()) {
871 DataBlockEncoding encoding = cacheKey.getDataBlockEncoding();
872 Integer count = counts.get(encoding);
873 counts.put(encoding, (count == null ? 0 : count) + 1);
874 }
875 return counts;
876 }
877
878 public void setVictimCache(BucketCache handler) {
879 assert victimHandler == null;
880 victimHandler = handler;
881 }
882
883 }