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.io.hfile;
21
22 import java.io.IOException;
23 import java.lang.ref.WeakReference;
24 import java.util.ArrayList;
25 import java.util.Collections;
26 import java.util.HashMap;
27 import java.util.LinkedList;
28 import java.util.List;
29 import java.util.Map;
30 import java.util.PriorityQueue;
31 import java.util.concurrent.ConcurrentHashMap;
32 import java.util.concurrent.Executors;
33 import java.util.concurrent.ScheduledExecutorService;
34 import java.util.concurrent.TimeUnit;
35 import java.util.concurrent.atomic.AtomicLong;
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.fs.FileSystem;
42 import org.apache.hadoop.fs.Path;
43 import org.apache.hadoop.hbase.io.HeapSize;
44 import org.apache.hadoop.hbase.util.Bytes;
45 import org.apache.hadoop.hbase.util.ClassSize;
46 import org.apache.hadoop.hbase.util.FSUtils;
47 import org.apache.hadoop.hbase.util.HasThread;
48 import org.apache.hadoop.util.StringUtils;
49
50 import com.google.common.util.concurrent.ThreadFactoryBuilder;
51
52
53
54
55
56
57
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 public class LruBlockCache implements BlockCache, HeapSize {
91
92 static final Log LOG = LogFactory.getLog(LruBlockCache.class);
93
94
95
96
97 static final float DEFAULT_LOAD_FACTOR = 0.75f;
98 static final int DEFAULT_CONCURRENCY_LEVEL = 16;
99
100
101 static final float DEFAULT_MIN_FACTOR = 0.75f;
102 static final float DEFAULT_ACCEPTABLE_FACTOR = 0.85f;
103
104
105 static final float DEFAULT_SINGLE_FACTOR = 0.25f;
106 static final float DEFAULT_MULTI_FACTOR = 0.50f;
107 static final float DEFAULT_MEMORY_FACTOR = 0.25f;
108
109
110 static final int statThreadPeriod = 60 * 5;
111
112
113 private final ConcurrentHashMap<String,CachedBlock> map;
114
115
116 private final ReentrantLock evictionLock = new ReentrantLock(true);
117
118
119 private volatile boolean evictionInProgress = false;
120
121
122 private final EvictionThread evictionThread;
123
124
125 private final ScheduledExecutorService scheduleThreadPool =
126 Executors.newScheduledThreadPool(1,
127 new ThreadFactoryBuilder()
128 .setNameFormat("LRU Statistics #%d")
129 .setDaemon(true)
130 .build());
131
132
133 private final AtomicLong size;
134
135
136 private final AtomicLong elements;
137
138
139 private final AtomicLong count;
140
141
142 private final CacheStats stats;
143
144
145 private long maxSize;
146
147
148 private long blockSize;
149
150
151 private float acceptableFactor;
152
153
154 private float minFactor;
155
156
157 private float singleFactor;
158
159
160 private float multiFactor;
161
162
163 private float memoryFactor;
164
165
166 private long overhead;
167
168
169
170
171
172
173
174
175
176
177 public LruBlockCache(long maxSize, long blockSize) {
178 this(maxSize, blockSize, true);
179 }
180
181
182
183
184 public LruBlockCache(long maxSize, long blockSize, boolean evictionThread) {
185 this(maxSize, blockSize, evictionThread,
186 (int)Math.ceil(1.2*maxSize/blockSize),
187 DEFAULT_LOAD_FACTOR, DEFAULT_CONCURRENCY_LEVEL,
188 DEFAULT_MIN_FACTOR, DEFAULT_ACCEPTABLE_FACTOR,
189 DEFAULT_SINGLE_FACTOR, DEFAULT_MULTI_FACTOR,
190 DEFAULT_MEMORY_FACTOR);
191 }
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207 public LruBlockCache(long maxSize, long blockSize, boolean evictionThread,
208 int mapInitialSize, float mapLoadFactor, int mapConcurrencyLevel,
209 float minFactor, float acceptableFactor,
210 float singleFactor, float multiFactor, float memoryFactor) {
211 if(singleFactor + multiFactor + memoryFactor != 1) {
212 throw new IllegalArgumentException("Single, multi, and memory factors " +
213 " should total 1.0");
214 }
215 if(minFactor >= acceptableFactor) {
216 throw new IllegalArgumentException("minFactor must be smaller than acceptableFactor");
217 }
218 if(minFactor >= 1.0f || acceptableFactor >= 1.0f) {
219 throw new IllegalArgumentException("all factors must be < 1");
220 }
221 this.maxSize = maxSize;
222 this.blockSize = blockSize;
223 map = new ConcurrentHashMap<String,CachedBlock>(mapInitialSize,
224 mapLoadFactor, mapConcurrencyLevel);
225 this.minFactor = minFactor;
226 this.acceptableFactor = acceptableFactor;
227 this.singleFactor = singleFactor;
228 this.multiFactor = multiFactor;
229 this.memoryFactor = memoryFactor;
230 this.stats = new CacheStats();
231 this.count = new AtomicLong(0);
232 this.elements = new AtomicLong(0);
233 this.overhead = calculateOverhead(maxSize, blockSize, mapConcurrencyLevel);
234 this.size = new AtomicLong(this.overhead);
235 if(evictionThread) {
236 this.evictionThread = new EvictionThread(this);
237 this.evictionThread.start();
238 } else {
239 this.evictionThread = null;
240 }
241 this.scheduleThreadPool.scheduleAtFixedRate(new StatisticsThread(this),
242 statThreadPeriod, statThreadPeriod, TimeUnit.SECONDS);
243 }
244
245 public void setMaxSize(long maxSize) {
246 this.maxSize = maxSize;
247 if(this.size.get() > acceptableSize() && !evictionInProgress) {
248 runEviction();
249 }
250 }
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265 public void cacheBlock(String blockName, Cacheable buf, boolean inMemory) {
266 CachedBlock cb = map.get(blockName);
267 if(cb != null) {
268 throw new RuntimeException("Cached an already cached block");
269 }
270 cb = new CachedBlock(blockName, buf, count.incrementAndGet(), inMemory);
271 long newSize = size.addAndGet(cb.heapSize());
272 map.put(blockName, cb);
273 elements.incrementAndGet();
274 if(newSize > acceptableSize() && !evictionInProgress) {
275 runEviction();
276 }
277 }
278
279
280
281
282
283
284
285
286
287
288
289 public void cacheBlock(String blockName, Cacheable buf) {
290 cacheBlock(blockName, buf, false);
291 }
292
293
294
295
296
297
298 public Cacheable getBlock(String blockName, boolean caching) {
299 CachedBlock cb = map.get(blockName);
300 if(cb == null) {
301 stats.miss(caching);
302 return null;
303 }
304 stats.hit(caching);
305 cb.access(count.incrementAndGet());
306 return cb.getBuffer();
307 }
308
309
310 @Override
311 public boolean evictBlock(String blockName) {
312 CachedBlock cb = map.get(blockName);
313 if (cb == null) return false;
314 evictBlock(cb);
315 return true;
316 }
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331 @Override
332 public int evictBlocksByPrefix(String prefix) {
333 int numEvicted = 0;
334 for (String key : map.keySet()) {
335 if (key.startsWith(prefix)) {
336 if (evictBlock(key))
337 ++numEvicted;
338 }
339 }
340 return numEvicted;
341 }
342
343 protected long evictBlock(CachedBlock block) {
344 map.remove(block.getName());
345 size.addAndGet(-1 * block.heapSize());
346 elements.decrementAndGet();
347 stats.evicted();
348 return block.heapSize();
349 }
350
351
352
353
354 private void runEviction() {
355 if(evictionThread == null) {
356 evict();
357 } else {
358 evictionThread.evict();
359 }
360 }
361
362
363
364
365 void evict() {
366
367
368 if(!evictionLock.tryLock()) return;
369
370 try {
371 evictionInProgress = true;
372 long currentSize = this.size.get();
373 long bytesToFree = currentSize - minSize();
374
375 if (LOG.isDebugEnabled()) {
376 LOG.debug("Block cache LRU eviction started; Attempting to free " +
377 StringUtils.byteDesc(bytesToFree) + " of total=" +
378 StringUtils.byteDesc(currentSize));
379 }
380
381 if(bytesToFree <= 0) return;
382
383
384 BlockBucket bucketSingle = new BlockBucket(bytesToFree, blockSize,
385 singleSize());
386 BlockBucket bucketMulti = new BlockBucket(bytesToFree, blockSize,
387 multiSize());
388 BlockBucket bucketMemory = new BlockBucket(bytesToFree, blockSize,
389 memorySize());
390
391
392 for(CachedBlock cachedBlock : map.values()) {
393 switch(cachedBlock.getPriority()) {
394 case SINGLE: {
395 bucketSingle.add(cachedBlock);
396 break;
397 }
398 case MULTI: {
399 bucketMulti.add(cachedBlock);
400 break;
401 }
402 case MEMORY: {
403 bucketMemory.add(cachedBlock);
404 break;
405 }
406 }
407 }
408
409 PriorityQueue<BlockBucket> bucketQueue =
410 new PriorityQueue<BlockBucket>(3);
411
412 bucketQueue.add(bucketSingle);
413 bucketQueue.add(bucketMulti);
414 bucketQueue.add(bucketMemory);
415
416 int remainingBuckets = 3;
417 long bytesFreed = 0;
418
419 BlockBucket bucket;
420 while((bucket = bucketQueue.poll()) != null) {
421 long overflow = bucket.overflow();
422 if(overflow > 0) {
423 long bucketBytesToFree = Math.min(overflow,
424 (bytesToFree - bytesFreed) / remainingBuckets);
425 bytesFreed += bucket.free(bucketBytesToFree);
426 }
427 remainingBuckets--;
428 }
429
430 if (LOG.isDebugEnabled()) {
431 long single = bucketSingle.totalSize();
432 long multi = bucketMulti.totalSize();
433 long memory = bucketMemory.totalSize();
434 LOG.debug("Block cache LRU eviction completed; " +
435 "freed=" + StringUtils.byteDesc(bytesFreed) + ", " +
436 "total=" + StringUtils.byteDesc(this.size.get()) + ", " +
437 "single=" + StringUtils.byteDesc(single) + ", " +
438 "multi=" + StringUtils.byteDesc(multi) + ", " +
439 "memory=" + StringUtils.byteDesc(memory));
440 }
441 } finally {
442 stats.evict();
443 evictionInProgress = false;
444 evictionLock.unlock();
445 }
446 }
447
448
449
450
451
452
453
454 private class BlockBucket implements Comparable<BlockBucket> {
455
456 private CachedBlockQueue queue;
457 private long totalSize = 0;
458 private long bucketSize;
459
460 public BlockBucket(long bytesToFree, long blockSize, long bucketSize) {
461 this.bucketSize = bucketSize;
462 queue = new CachedBlockQueue(bytesToFree, blockSize);
463 totalSize = 0;
464 }
465
466 public void add(CachedBlock block) {
467 totalSize += block.heapSize();
468 queue.add(block);
469 }
470
471 public long free(long toFree) {
472 LinkedList<CachedBlock> blocks = queue.get();
473 long freedBytes = 0;
474 for(CachedBlock cb: blocks) {
475 freedBytes += evictBlock(cb);
476 if(freedBytes >= toFree) {
477 return freedBytes;
478 }
479 }
480 return freedBytes;
481 }
482
483 public long overflow() {
484 return totalSize - bucketSize;
485 }
486
487 public long totalSize() {
488 return totalSize;
489 }
490
491 public int compareTo(BlockBucket that) {
492 if(this.overflow() == that.overflow()) return 0;
493 return this.overflow() > that.overflow() ? 1 : -1;
494 }
495 }
496
497
498
499
500
501 public long getMaxSize() {
502 return this.maxSize;
503 }
504
505
506
507
508
509 public long getCurrentSize() {
510 return this.size.get();
511 }
512
513
514
515
516
517 public long getFreeSize() {
518 return getMaxSize() - getCurrentSize();
519 }
520
521
522
523
524
525 public long size() {
526 return this.elements.get();
527 }
528
529 @Override
530 public long getBlockCount() {
531 return this.elements.get();
532 }
533
534
535
536
537 public long getEvictionCount() {
538 return this.stats.getEvictionCount();
539 }
540
541
542
543
544
545 public long getEvictedCount() {
546 return this.stats.getEvictedCount();
547 }
548
549
550
551
552
553
554
555 private static class EvictionThread extends HasThread {
556 private WeakReference<LruBlockCache> cache;
557
558 public EvictionThread(LruBlockCache cache) {
559 super("LruBlockCache.EvictionThread");
560 setDaemon(true);
561 this.cache = new WeakReference<LruBlockCache>(cache);
562 }
563
564 @Override
565 public void run() {
566 while(true) {
567 synchronized(this) {
568 try {
569 this.wait();
570 } catch(InterruptedException e) {}
571 }
572 LruBlockCache cache = this.cache.get();
573 if(cache == null) break;
574 cache.evict();
575 }
576 }
577 public void evict() {
578 synchronized(this) {
579 this.notify();
580 }
581 }
582 }
583
584
585
586
587 static class StatisticsThread extends Thread {
588 LruBlockCache lru;
589
590 public StatisticsThread(LruBlockCache lru) {
591 super("LruBlockCache.StatisticsThread");
592 setDaemon(true);
593 this.lru = lru;
594 }
595 @Override
596 public void run() {
597 lru.logStats();
598 }
599 }
600
601 public void logStats() {
602 if (!LOG.isDebugEnabled()) return;
603
604 long totalSize = heapSize();
605 long freeSize = maxSize - totalSize;
606 LruBlockCache.LOG.debug("LRU Stats: " +
607 "total=" + StringUtils.byteDesc(totalSize) + ", " +
608 "free=" + StringUtils.byteDesc(freeSize) + ", " +
609 "max=" + StringUtils.byteDesc(this.maxSize) + ", " +
610 "blocks=" + size() +", " +
611 "accesses=" + stats.getRequestCount() + ", " +
612 "hits=" + stats.getHitCount() + ", " +
613 "hitRatio=" +
614 (stats.getHitCount() == 0 ? "0" : (StringUtils.formatPercent(stats.getHitRatio(), 2)+ ", ")) +
615 "cachingAccesses=" + stats.getRequestCachingCount() + ", " +
616 "cachingHits=" + stats.getHitCachingCount() + ", " +
617 "cachingHitsRatio=" +
618 (stats.getHitCachingCount() == 0 ? "0" : (StringUtils.formatPercent(stats.getHitCachingRatio(), 2)+ ", ")) +
619 "evictions=" + stats.getEvictionCount() + ", " +
620 "evicted=" + stats.getEvictedCount() + ", " +
621 "evictedPerRun=" + stats.evictedPerEviction());
622 }
623
624
625
626
627
628
629
630 public CacheStats getStats() {
631 return this.stats;
632 }
633
634 public final static long CACHE_FIXED_OVERHEAD = ClassSize.align(
635 (3 * Bytes.SIZEOF_LONG) + (8 * ClassSize.REFERENCE) +
636 (5 * Bytes.SIZEOF_FLOAT) + Bytes.SIZEOF_BOOLEAN
637 + ClassSize.OBJECT);
638
639
640 public long heapSize() {
641 return getCurrentSize();
642 }
643
644 public static long calculateOverhead(long maxSize, long blockSize, int concurrency){
645
646 return CACHE_FIXED_OVERHEAD + ClassSize.CONCURRENT_HASHMAP +
647 ((long)Math.ceil(maxSize*1.2/blockSize)
648 * ClassSize.CONCURRENT_HASHMAP_ENTRY) +
649 (concurrency * ClassSize.CONCURRENT_HASHMAP_SEGMENT);
650 }
651
652 @Override
653 public List<BlockCacheColumnFamilySummary> getBlockCacheColumnFamilySummaries(Configuration conf) throws IOException {
654
655 Map<String, Path> sfMap = FSUtils.getTableStoreFilePathMap(
656 FileSystem.get(conf),
657 FSUtils.getRootDir(conf));
658
659
660
661 Map<BlockCacheColumnFamilySummary, BlockCacheColumnFamilySummary> bcs =
662 new HashMap<BlockCacheColumnFamilySummary, BlockCacheColumnFamilySummary>();
663
664 final String pattern = "\\" + HFile.CACHE_KEY_SEPARATOR;
665
666 for (CachedBlock cb : map.values()) {
667
668
669 String s[] = cb.getName().split(pattern);
670 if (s.length > 0) {
671 String sf = s[0];
672 Path path = sfMap.get(sf);
673 if ( path != null) {
674 BlockCacheColumnFamilySummary lookup =
675 BlockCacheColumnFamilySummary.createFromStoreFilePath(path);
676 BlockCacheColumnFamilySummary bcse = bcs.get(lookup);
677 if (bcse == null) {
678 bcse = BlockCacheColumnFamilySummary.create(lookup);
679 bcs.put(lookup,bcse);
680 }
681 bcse.incrementBlocks();
682 bcse.incrementHeapSize(cb.heapSize());
683 }
684 }
685 }
686 List<BlockCacheColumnFamilySummary> list =
687 new ArrayList<BlockCacheColumnFamilySummary>(bcs.values());
688 Collections.sort( list );
689 return list;
690 }
691
692
693
694 private long acceptableSize() {
695 return (long)Math.floor(this.maxSize * this.acceptableFactor);
696 }
697 private long minSize() {
698 return (long)Math.floor(this.maxSize * this.minFactor);
699 }
700 private long singleSize() {
701 return (long)Math.floor(this.maxSize * this.singleFactor * this.minFactor);
702 }
703 private long multiSize() {
704 return (long)Math.floor(this.maxSize * this.multiFactor * this.minFactor);
705 }
706 private long memorySize() {
707 return (long)Math.floor(this.maxSize * this.memoryFactor * this.minFactor);
708 }
709
710 public void shutdown() {
711 this.scheduleThreadPool.shutdown();
712 }
713 }