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