1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21 package org.apache.hadoop.hbase.io.hfile.bucket;
22
23 import java.io.File;
24 import java.io.FileInputStream;
25 import java.io.FileNotFoundException;
26 import java.io.FileOutputStream;
27 import java.io.IOException;
28 import java.io.ObjectInputStream;
29 import java.io.ObjectOutputStream;
30 import java.io.Serializable;
31 import java.nio.ByteBuffer;
32 import java.util.ArrayList;
33 import java.util.Comparator;
34 import java.util.HashSet;
35 import java.util.Iterator;
36 import java.util.List;
37 import java.util.Map;
38 import java.util.PriorityQueue;
39 import java.util.Set;
40 import java.util.concurrent.ArrayBlockingQueue;
41 import java.util.concurrent.BlockingQueue;
42 import java.util.concurrent.ConcurrentHashMap;
43 import java.util.concurrent.ConcurrentMap;
44 import java.util.concurrent.Executors;
45 import java.util.concurrent.ScheduledExecutorService;
46 import java.util.concurrent.TimeUnit;
47 import java.util.concurrent.atomic.AtomicLong;
48 import java.util.concurrent.locks.Lock;
49 import java.util.concurrent.locks.ReentrantLock;
50 import java.util.concurrent.locks.ReentrantReadWriteLock;
51
52 import org.apache.commons.logging.Log;
53 import org.apache.commons.logging.LogFactory;
54 import org.apache.hadoop.hbase.classification.InterfaceAudience;
55 import org.apache.hadoop.hbase.io.HeapSize;
56 import org.apache.hadoop.hbase.io.hfile.BlockCache;
57 import org.apache.hadoop.hbase.io.hfile.BlockCacheKey;
58 import org.apache.hadoop.hbase.io.hfile.BlockCacheUtil;
59 import org.apache.hadoop.hbase.io.hfile.BlockPriority;
60 import org.apache.hadoop.hbase.io.hfile.BlockType;
61 import org.apache.hadoop.hbase.io.hfile.CacheStats;
62 import org.apache.hadoop.hbase.io.hfile.Cacheable;
63 import org.apache.hadoop.hbase.io.hfile.CacheableDeserializer;
64 import org.apache.hadoop.hbase.io.hfile.CacheableDeserializerIdManager;
65 import org.apache.hadoop.hbase.io.hfile.CachedBlock;
66 import org.apache.hadoop.hbase.io.hfile.HFileBlock;
67 import org.apache.hadoop.hbase.util.ConcurrentIndex;
68 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
69 import org.apache.hadoop.hbase.util.HasThread;
70 import org.apache.hadoop.hbase.util.IdReadWriteLock;
71 import org.apache.hadoop.util.StringUtils;
72
73 import com.google.common.annotations.VisibleForTesting;
74 import com.google.common.collect.ImmutableList;
75 import com.google.common.util.concurrent.ThreadFactoryBuilder;
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95 @InterfaceAudience.Private
96 public class BucketCache implements BlockCache, HeapSize {
97 static final Log LOG = LogFactory.getLog(BucketCache.class);
98
99
100 private static final float DEFAULT_SINGLE_FACTOR = 0.25f;
101 private static final float DEFAULT_MULTI_FACTOR = 0.50f;
102 private static final float DEFAULT_MEMORY_FACTOR = 0.25f;
103 private static final float DEFAULT_EXTRA_FREE_FACTOR = 0.10f;
104
105 private static final float DEFAULT_ACCEPT_FACTOR = 0.95f;
106 private static final float DEFAULT_MIN_FACTOR = 0.85f;
107
108
109 private static final int DEFAULT_FREE_ENTIRE_BLOCK_FACTOR = 2;
110
111
112 private static final int statThreadPeriod = 5 * 60;
113
114 final static int DEFAULT_WRITER_THREADS = 3;
115 final static int DEFAULT_WRITER_QUEUE_ITEMS = 64;
116
117
118 final IOEngine ioEngine;
119
120
121 @VisibleForTesting
122 final ConcurrentMap<BlockCacheKey, RAMQueueEntry> ramCache;
123
124 @VisibleForTesting
125 ConcurrentMap<BlockCacheKey, BucketEntry> backingMap;
126
127
128
129
130
131
132 private volatile boolean cacheEnabled;
133
134
135
136
137
138
139
140
141 @VisibleForTesting
142 final ArrayList<BlockingQueue<RAMQueueEntry>> writerQueues =
143 new ArrayList<BlockingQueue<RAMQueueEntry>>();
144 @VisibleForTesting
145 final WriterThread[] writerThreads;
146
147
148 private volatile boolean freeInProgress = false;
149 private final Lock freeSpaceLock = new ReentrantLock();
150
151 private UniqueIndexMap<Integer> deserialiserMap = new UniqueIndexMap<Integer>();
152
153 private final AtomicLong realCacheSize = new AtomicLong(0);
154 private final AtomicLong heapSize = new AtomicLong(0);
155
156 private final AtomicLong blockNumber = new AtomicLong(0);
157 private final AtomicLong failedBlockAdditions = new AtomicLong(0);
158
159
160 private final AtomicLong accessCount = new AtomicLong(0);
161
162 private static final int DEFAULT_CACHE_WAIT_TIME = 50;
163
164
165
166 boolean wait_when_cache = false;
167
168 private final BucketCacheStats cacheStats = new BucketCacheStats();
169
170 private final String persistencePath;
171 private final long cacheCapacity;
172
173 private final long blockSize;
174
175
176 private final int ioErrorsTolerationDuration;
177
178 public static final int DEFAULT_ERROR_TOLERATION_DURATION = 60 * 1000;
179
180
181
182 private volatile long ioErrorStartTime = -1;
183
184
185
186
187
188 @VisibleForTesting
189 final IdReadWriteLock offsetLock = new IdReadWriteLock();
190
191 private final ConcurrentIndex<String, BlockCacheKey> blocksByHFile =
192 new ConcurrentIndex<String, BlockCacheKey>(new Comparator<BlockCacheKey>() {
193 @Override
194 public int compare(BlockCacheKey a, BlockCacheKey b) {
195 if (a.getOffset() == b.getOffset()) {
196 return 0;
197 } else if (a.getOffset() < b.getOffset()) {
198 return -1;
199 }
200 return 1;
201 }
202 });
203
204
205 private final ScheduledExecutorService scheduleThreadPool = Executors.newScheduledThreadPool(1,
206 new ThreadFactoryBuilder().setNameFormat("BucketCacheStatsExecutor").setDaemon(true).build());
207
208
209 private BucketAllocator bucketAllocator;
210
211 public BucketCache(String ioEngineName, long capacity, int blockSize, int[] bucketSizes,
212 int writerThreadNum, int writerQLen, String persistencePath) throws FileNotFoundException,
213 IOException {
214 this(ioEngineName, capacity, blockSize, bucketSizes, writerThreadNum, writerQLen,
215 persistencePath, DEFAULT_ERROR_TOLERATION_DURATION);
216 }
217
218 public BucketCache(String ioEngineName, long capacity, int blockSize, int[] bucketSizes,
219 int writerThreadNum, int writerQLen, String persistencePath, int ioErrorsTolerationDuration)
220 throws FileNotFoundException, IOException {
221 this.ioEngine = getIOEngineFromName(ioEngineName, capacity);
222 this.writerThreads = new WriterThread[writerThreadNum];
223 long blockNumCapacity = capacity / blockSize;
224 if (blockNumCapacity >= Integer.MAX_VALUE) {
225
226 throw new IllegalArgumentException("Cache capacity is too large, only support 32TB now");
227 }
228
229 this.cacheCapacity = capacity;
230 this.persistencePath = persistencePath;
231 this.blockSize = blockSize;
232 this.ioErrorsTolerationDuration = ioErrorsTolerationDuration;
233
234 bucketAllocator = new BucketAllocator(capacity, bucketSizes);
235 for (int i = 0; i < writerThreads.length; ++i) {
236 writerQueues.add(new ArrayBlockingQueue<RAMQueueEntry>(writerQLen));
237 }
238
239 assert writerQueues.size() == writerThreads.length;
240 this.ramCache = new ConcurrentHashMap<BlockCacheKey, RAMQueueEntry>();
241
242 this.backingMap = new ConcurrentHashMap<BlockCacheKey, BucketEntry>((int) blockNumCapacity);
243
244 if (ioEngine.isPersistent() && persistencePath != null) {
245 try {
246 retrieveFromFile(bucketSizes);
247 } catch (IOException ioex) {
248 LOG.error("Can't restore from file because of", ioex);
249 } catch (ClassNotFoundException cnfe) {
250 LOG.error("Can't restore from file in rebuild because can't deserialise",cnfe);
251 throw new RuntimeException(cnfe);
252 }
253 }
254 final String threadName = Thread.currentThread().getName();
255 this.cacheEnabled = true;
256 for (int i = 0; i < writerThreads.length; ++i) {
257 writerThreads[i] = new WriterThread(writerQueues.get(i));
258 writerThreads[i].setName(threadName + "-BucketCacheWriter-" + i);
259 writerThreads[i].setDaemon(true);
260 }
261 startWriterThreads();
262
263
264
265
266 this.scheduleThreadPool.scheduleAtFixedRate(new StatisticsThread(this),
267 statThreadPeriod, statThreadPeriod, TimeUnit.SECONDS);
268 LOG.info("Started bucket cache; ioengine=" + ioEngineName +
269 ", capacity=" + StringUtils.byteDesc(capacity) +
270 ", blockSize=" + StringUtils.byteDesc(blockSize) + ", writerThreadNum=" +
271 writerThreadNum + ", writerQLen=" + writerQLen + ", persistencePath=" +
272 persistencePath + ", bucketAllocator=" + this.bucketAllocator.getClass().getName());
273 }
274
275
276
277
278
279 @VisibleForTesting
280 protected void startWriterThreads() {
281 for (WriterThread thread : writerThreads) {
282 thread.start();
283 }
284 }
285
286 @VisibleForTesting
287 boolean isCacheEnabled() {
288 return this.cacheEnabled;
289 }
290
291 public long getMaxSize() {
292 return this.cacheCapacity;
293 }
294
295 public String getIoEngine() {
296 return ioEngine.toString();
297 }
298
299
300
301
302
303
304
305
306 private IOEngine getIOEngineFromName(String ioEngineName, long capacity)
307 throws IOException {
308 if (ioEngineName.startsWith("file:"))
309 return new FileIOEngine(ioEngineName.substring(5), capacity);
310 else if (ioEngineName.startsWith("offheap"))
311 return new ByteBufferIOEngine(capacity, true);
312 else if (ioEngineName.startsWith("heap"))
313 return new ByteBufferIOEngine(capacity, false);
314 else
315 throw new IllegalArgumentException(
316 "Don't understand io engine name for cache - prefix with file:, heap or offheap");
317 }
318
319
320
321
322
323
324 @Override
325 public void cacheBlock(BlockCacheKey cacheKey, Cacheable buf) {
326 cacheBlock(cacheKey, buf, false, false);
327 }
328
329
330
331
332
333
334
335
336 @Override
337 public void cacheBlock(BlockCacheKey cacheKey, Cacheable cachedItem, boolean inMemory,
338 final boolean cacheDataInL1) {
339 cacheBlockWithWait(cacheKey, cachedItem, inMemory, wait_when_cache);
340 }
341
342
343
344
345
346
347
348
349 public void cacheBlockWithWait(BlockCacheKey cacheKey, Cacheable cachedItem, boolean inMemory,
350 boolean wait) {
351 if (!cacheEnabled) {
352 return;
353 }
354
355 if (backingMap.containsKey(cacheKey)) {
356 return;
357 }
358
359
360
361
362 RAMQueueEntry re =
363 new RAMQueueEntry(cacheKey, cachedItem, accessCount.incrementAndGet(), inMemory);
364 if (ramCache.putIfAbsent(cacheKey, re) != null) {
365 return;
366 }
367 int queueNum = (cacheKey.hashCode() & 0x7FFFFFFF) % writerQueues.size();
368 BlockingQueue<RAMQueueEntry> bq = writerQueues.get(queueNum);
369 boolean successfulAddition = false;
370 if (wait) {
371 try {
372 successfulAddition = bq.offer(re, DEFAULT_CACHE_WAIT_TIME, TimeUnit.MILLISECONDS);
373 } catch (InterruptedException e) {
374 Thread.currentThread().interrupt();
375 }
376 } else {
377 successfulAddition = bq.offer(re);
378 }
379 if (!successfulAddition) {
380 ramCache.remove(cacheKey);
381 failedBlockAdditions.incrementAndGet();
382 } else {
383 this.blockNumber.incrementAndGet();
384 this.heapSize.addAndGet(cachedItem.heapSize());
385 blocksByHFile.put(cacheKey.getHfileName(), cacheKey);
386 }
387 }
388
389
390
391
392
393
394
395
396
397 @Override
398 public Cacheable getBlock(BlockCacheKey key, boolean caching, boolean repeat,
399 boolean updateCacheMetrics) {
400 if (!cacheEnabled) {
401 return null;
402 }
403 RAMQueueEntry re = ramCache.get(key);
404 if (re != null) {
405 if (updateCacheMetrics) {
406 cacheStats.hit(caching, key.isPrimary(), key.getBlockType());
407 }
408 re.access(accessCount.incrementAndGet());
409 return re.getData();
410 }
411 BucketEntry bucketEntry = backingMap.get(key);
412 if (bucketEntry != null) {
413 long start = System.nanoTime();
414 ReentrantReadWriteLock lock = offsetLock.getLock(bucketEntry.offset());
415 try {
416 lock.readLock().lock();
417
418
419
420 if (bucketEntry.equals(backingMap.get(key))) {
421 int len = bucketEntry.getLength();
422 ByteBuffer bb = ByteBuffer.allocate(len);
423 int lenRead = ioEngine.read(bb, bucketEntry.offset());
424 if (lenRead != len) {
425 throw new RuntimeException("Only " + lenRead + " bytes read, " + len + " expected");
426 }
427 CacheableDeserializer<Cacheable> deserializer =
428 bucketEntry.deserializerReference(this.deserialiserMap);
429 Cacheable cachedBlock = deserializer.deserialize(bb, true);
430 long timeTaken = System.nanoTime() - start;
431 if (updateCacheMetrics) {
432 cacheStats.hit(caching, key.isPrimary(), key.getBlockType());
433 cacheStats.ioHit(timeTaken);
434 }
435 bucketEntry.access(accessCount.incrementAndGet());
436 if (this.ioErrorStartTime > 0) {
437 ioErrorStartTime = -1;
438 }
439 return cachedBlock;
440 }
441 } catch (IOException ioex) {
442 LOG.error("Failed reading block " + key + " from bucket cache", ioex);
443 checkIOErrorIsTolerated();
444 } finally {
445 lock.readLock().unlock();
446 }
447 }
448 if (!repeat && updateCacheMetrics) {
449 cacheStats.miss(caching, key.isPrimary(), key.getBlockType());
450 }
451 return null;
452 }
453
454 @VisibleForTesting
455 void blockEvicted(BlockCacheKey cacheKey, BucketEntry bucketEntry, boolean decrementBlockNumber) {
456 bucketAllocator.freeBlock(bucketEntry.offset());
457 realCacheSize.addAndGet(-1 * bucketEntry.getLength());
458 blocksByHFile.remove(cacheKey.getHfileName(), cacheKey);
459 if (decrementBlockNumber) {
460 this.blockNumber.decrementAndGet();
461 }
462 }
463
464 @Override
465 public boolean evictBlock(BlockCacheKey cacheKey) {
466 if (!cacheEnabled) {
467 return false;
468 }
469 RAMQueueEntry removedBlock = ramCache.remove(cacheKey);
470 if (removedBlock != null) {
471 this.blockNumber.decrementAndGet();
472 this.heapSize.addAndGet(-1 * removedBlock.getData().heapSize());
473 }
474 BucketEntry bucketEntry = backingMap.get(cacheKey);
475 if (bucketEntry == null) {
476 if (removedBlock != null) {
477 cacheStats.evicted(0, cacheKey.isPrimary());
478 return true;
479 } else {
480 return false;
481 }
482 }
483 ReentrantReadWriteLock lock = offsetLock.getLock(bucketEntry.offset());
484 try {
485 lock.writeLock().lock();
486 if (backingMap.remove(cacheKey, bucketEntry)) {
487 blockEvicted(cacheKey, bucketEntry, removedBlock == null);
488 } else {
489 return false;
490 }
491 } finally {
492 lock.writeLock().unlock();
493 }
494 cacheStats.evicted(bucketEntry.getCachedTime(), cacheKey.isPrimary());
495 return true;
496 }
497
498
499
500
501 private static class StatisticsThread extends Thread {
502 private final BucketCache bucketCache;
503
504 public StatisticsThread(BucketCache bucketCache) {
505 super("BucketCacheStatsThread");
506 setDaemon(true);
507 this.bucketCache = bucketCache;
508 }
509
510 @Override
511 public void run() {
512 bucketCache.logStats();
513 }
514 }
515
516 public void logStats() {
517 long totalSize = bucketAllocator.getTotalSize();
518 long usedSize = bucketAllocator.getUsedSize();
519 long freeSize = totalSize - usedSize;
520 long cacheSize = getRealCacheSize();
521 LOG.info("failedBlockAdditions=" + getFailedBlockAdditions() + ", " +
522 "totalSize=" + StringUtils.byteDesc(totalSize) + ", " +
523 "freeSize=" + StringUtils.byteDesc(freeSize) + ", " +
524 "usedSize=" + StringUtils.byteDesc(usedSize) +", " +
525 "cacheSize=" + StringUtils.byteDesc(cacheSize) +", " +
526 "accesses=" + cacheStats.getRequestCount() + ", " +
527 "hits=" + cacheStats.getHitCount() + ", " +
528 "IOhitsPerSecond=" + cacheStats.getIOHitsPerSecond() + ", " +
529 "IOTimePerHit=" + String.format("%.2f", cacheStats.getIOTimePerHit())+ ", " +
530 "hitRatio=" + (cacheStats.getHitCount() == 0 ? "0," :
531 (StringUtils.formatPercent(cacheStats.getHitRatio(), 2)+ ", ")) +
532 "cachingAccesses=" + cacheStats.getRequestCachingCount() + ", " +
533 "cachingHits=" + cacheStats.getHitCachingCount() + ", " +
534 "cachingHitsRatio=" +(cacheStats.getHitCachingCount() == 0 ? "0," :
535 (StringUtils.formatPercent(cacheStats.getHitCachingRatio(), 2)+ ", ")) +
536 "evictions=" + cacheStats.getEvictionCount() + ", " +
537 "evicted=" + cacheStats.getEvictedCount() + ", " +
538 "evictedPerRun=" + cacheStats.evictedPerEviction());
539 cacheStats.reset();
540 }
541
542 public long getFailedBlockAdditions() {
543 return this.failedBlockAdditions.get();
544 }
545
546 public long getRealCacheSize() {
547 return this.realCacheSize.get();
548 }
549
550 private long acceptableSize() {
551 return (long) Math.floor(bucketAllocator.getTotalSize() * DEFAULT_ACCEPT_FACTOR);
552 }
553
554 private long singleSize() {
555 return (long) Math.floor(bucketAllocator.getTotalSize()
556 * DEFAULT_SINGLE_FACTOR * DEFAULT_MIN_FACTOR);
557 }
558
559 private long multiSize() {
560 return (long) Math.floor(bucketAllocator.getTotalSize() * DEFAULT_MULTI_FACTOR
561 * DEFAULT_MIN_FACTOR);
562 }
563
564 private long memorySize() {
565 return (long) Math.floor(bucketAllocator.getTotalSize() * DEFAULT_MEMORY_FACTOR
566 * DEFAULT_MIN_FACTOR);
567 }
568
569
570
571
572 private int bucketSizesAboveThresholdCount(float minFactor) {
573 BucketAllocator.IndexStatistics[] stats = bucketAllocator.getIndexStatistics();
574 int fullCount = 0;
575 for (int i = 0; i < stats.length; i++) {
576 long freeGoal = (long) Math.floor(stats[i].totalCount() * (1 - minFactor));
577 freeGoal = Math.max(freeGoal, 1);
578 if (stats[i].freeCount() < freeGoal) {
579 fullCount++;
580 }
581 }
582 return fullCount;
583 }
584
585
586
587
588
589
590
591
592
593
594 private void freeEntireBuckets(int completelyFreeBucketsNeeded) {
595 if (completelyFreeBucketsNeeded != 0) {
596
597
598 Set<Integer> inUseBuckets = new HashSet<Integer>();
599 for (BucketEntry entry : backingMap.values()) {
600 inUseBuckets.add(bucketAllocator.getBucketIndex(entry.offset()));
601 }
602
603 Set<Integer> candidateBuckets = bucketAllocator.getLeastFilledBuckets(
604 inUseBuckets, completelyFreeBucketsNeeded);
605 for (Map.Entry<BlockCacheKey, BucketEntry> entry : backingMap.entrySet()) {
606 if (candidateBuckets.contains(bucketAllocator
607 .getBucketIndex(entry.getValue().offset()))) {
608 evictBlock(entry.getKey());
609 }
610 }
611 }
612 }
613
614
615
616
617
618
619
620 private void freeSpace(final String why) {
621
622 if (!freeSpaceLock.tryLock()) return;
623 try {
624 freeInProgress = true;
625 long bytesToFreeWithoutExtra = 0;
626
627 StringBuffer msgBuffer = LOG.isDebugEnabled()? new StringBuffer(): null;
628 BucketAllocator.IndexStatistics[] stats = bucketAllocator.getIndexStatistics();
629 long[] bytesToFreeForBucket = new long[stats.length];
630 for (int i = 0; i < stats.length; i++) {
631 bytesToFreeForBucket[i] = 0;
632 long freeGoal = (long) Math.floor(stats[i].totalCount() * (1 - DEFAULT_MIN_FACTOR));
633 freeGoal = Math.max(freeGoal, 1);
634 if (stats[i].freeCount() < freeGoal) {
635 bytesToFreeForBucket[i] = stats[i].itemSize() * (freeGoal - stats[i].freeCount());
636 bytesToFreeWithoutExtra += bytesToFreeForBucket[i];
637 if (msgBuffer != null) {
638 msgBuffer.append("Free for bucketSize(" + stats[i].itemSize() + ")="
639 + StringUtils.byteDesc(bytesToFreeForBucket[i]) + ", ");
640 }
641 }
642 }
643 if (msgBuffer != null) {
644 msgBuffer.append("Free for total=" + StringUtils.byteDesc(bytesToFreeWithoutExtra) + ", ");
645 }
646
647 if (bytesToFreeWithoutExtra <= 0) {
648 return;
649 }
650 long currentSize = bucketAllocator.getUsedSize();
651 long totalSize=bucketAllocator.getTotalSize();
652 if (LOG.isDebugEnabled() && msgBuffer != null) {
653 LOG.debug("Free started because \"" + why + "\"; " + msgBuffer.toString() +
654 " of current used=" + StringUtils.byteDesc(currentSize) + ", actual cacheSize=" +
655 StringUtils.byteDesc(realCacheSize.get()) + ", total=" + StringUtils.byteDesc(totalSize));
656 }
657
658 long bytesToFreeWithExtra = (long) Math.floor(bytesToFreeWithoutExtra
659 * (1 + DEFAULT_EXTRA_FREE_FACTOR));
660
661
662 BucketEntryGroup bucketSingle = new BucketEntryGroup(bytesToFreeWithExtra,
663 blockSize, singleSize());
664 BucketEntryGroup bucketMulti = new BucketEntryGroup(bytesToFreeWithExtra,
665 blockSize, multiSize());
666 BucketEntryGroup bucketMemory = new BucketEntryGroup(bytesToFreeWithExtra,
667 blockSize, memorySize());
668
669
670
671 for (Map.Entry<BlockCacheKey, BucketEntry> bucketEntryWithKey : backingMap.entrySet()) {
672 switch (bucketEntryWithKey.getValue().getPriority()) {
673 case SINGLE: {
674 bucketSingle.add(bucketEntryWithKey);
675 break;
676 }
677 case MULTI: {
678 bucketMulti.add(bucketEntryWithKey);
679 break;
680 }
681 case MEMORY: {
682 bucketMemory.add(bucketEntryWithKey);
683 break;
684 }
685 }
686 }
687
688 PriorityQueue<BucketEntryGroup> bucketQueue = new PriorityQueue<BucketEntryGroup>(3);
689
690 bucketQueue.add(bucketSingle);
691 bucketQueue.add(bucketMulti);
692 bucketQueue.add(bucketMemory);
693
694 int remainingBuckets = 3;
695 long bytesFreed = 0;
696
697 BucketEntryGroup bucketGroup;
698 while ((bucketGroup = bucketQueue.poll()) != null) {
699 long overflow = bucketGroup.overflow();
700 if (overflow > 0) {
701 long bucketBytesToFree = Math.min(overflow,
702 (bytesToFreeWithoutExtra - bytesFreed) / remainingBuckets);
703 bytesFreed += bucketGroup.free(bucketBytesToFree);
704 }
705 remainingBuckets--;
706 }
707
708
709 if (bucketSizesAboveThresholdCount(DEFAULT_MIN_FACTOR) > 0) {
710 bucketQueue.clear();
711 remainingBuckets = 3;
712
713 bucketQueue.add(bucketSingle);
714 bucketQueue.add(bucketMulti);
715 bucketQueue.add(bucketMemory);
716
717 while ((bucketGroup = bucketQueue.poll()) != null) {
718 long bucketBytesToFree = (bytesToFreeWithExtra - bytesFreed) / remainingBuckets;
719 bytesFreed += bucketGroup.free(bucketBytesToFree);
720 remainingBuckets--;
721 }
722 }
723
724
725
726
727
728
729 freeEntireBuckets(DEFAULT_FREE_ENTIRE_BLOCK_FACTOR *
730 bucketSizesAboveThresholdCount(1.0f));
731
732 if (LOG.isDebugEnabled()) {
733 long single = bucketSingle.totalSize();
734 long multi = bucketMulti.totalSize();
735 long memory = bucketMemory.totalSize();
736 if (LOG.isDebugEnabled()) {
737 LOG.debug("Bucket cache free space completed; " + "freed="
738 + StringUtils.byteDesc(bytesFreed) + ", " + "total="
739 + StringUtils.byteDesc(totalSize) + ", " + "single="
740 + StringUtils.byteDesc(single) + ", " + "multi="
741 + StringUtils.byteDesc(multi) + ", " + "memory="
742 + StringUtils.byteDesc(memory));
743 }
744 }
745
746 } catch (Throwable t) {
747 LOG.warn("Failed freeing space", t);
748 } finally {
749 cacheStats.evict();
750 freeInProgress = false;
751 freeSpaceLock.unlock();
752 }
753 }
754
755
756 @VisibleForTesting
757 class WriterThread extends HasThread {
758 private final BlockingQueue<RAMQueueEntry> inputQueue;
759 private volatile boolean writerEnabled = true;
760
761 WriterThread(BlockingQueue<RAMQueueEntry> queue) {
762 this.inputQueue = queue;
763 }
764
765
766 @VisibleForTesting
767 void disableWriter() {
768 this.writerEnabled = false;
769 }
770
771 public void run() {
772 List<RAMQueueEntry> entries = new ArrayList<RAMQueueEntry>();
773 try {
774 while (cacheEnabled && writerEnabled) {
775 try {
776 try {
777
778 entries = getRAMQueueEntries(inputQueue, entries);
779 } catch (InterruptedException ie) {
780 if (!cacheEnabled) break;
781 }
782 doDrain(entries);
783 } catch (Exception ioe) {
784 LOG.error("WriterThread encountered error", ioe);
785 }
786 }
787 } catch (Throwable t) {
788 LOG.warn("Failed doing drain", t);
789 }
790 LOG.info(this.getName() + " exiting, cacheEnabled=" + cacheEnabled);
791 }
792
793
794
795
796
797
798
799
800
801 @VisibleForTesting
802 void doDrain(final List<RAMQueueEntry> entries) throws InterruptedException {
803 if (entries.isEmpty()) {
804 return;
805 }
806
807
808
809
810
811
812 final int size = entries.size();
813 BucketEntry[] bucketEntries = new BucketEntry[size];
814
815
816 int index = 0;
817 while (cacheEnabled && index < size) {
818 RAMQueueEntry re = null;
819 try {
820 re = entries.get(index);
821 if (re == null) {
822 LOG.warn("Couldn't get entry or changed on us; who else is messing with it?");
823 index++;
824 continue;
825 }
826 BucketEntry bucketEntry =
827 re.writeToCache(ioEngine, bucketAllocator, deserialiserMap, realCacheSize);
828
829 bucketEntries[index] = bucketEntry;
830 if (ioErrorStartTime > 0) {
831 ioErrorStartTime = -1;
832 }
833 index++;
834 } catch (BucketAllocatorException fle) {
835 LOG.warn("Failed allocation for " + (re == null ? "" : re.getKey()) + "; " + fle);
836
837 bucketEntries[index] = null;
838 index++;
839 } catch (CacheFullException cfe) {
840
841 if (!freeInProgress) {
842 freeSpace("Full!");
843 } else {
844 Thread.sleep(50);
845 }
846 } catch (IOException ioex) {
847
848 LOG.error("Failed writing to bucket cache", ioex);
849 checkIOErrorIsTolerated();
850 }
851 }
852
853
854 try {
855 ioEngine.sync();
856 } catch (IOException ioex) {
857 LOG.error("Failed syncing IO engine", ioex);
858 checkIOErrorIsTolerated();
859
860 for (int i = 0; i < entries.size(); ++i) {
861 if (bucketEntries[i] != null) {
862 bucketAllocator.freeBlock(bucketEntries[i].offset());
863 bucketEntries[i] = null;
864 }
865 }
866 }
867
868
869
870 for (int i = 0; i < size; ++i) {
871 BlockCacheKey key = entries.get(i).getKey();
872
873 if (bucketEntries[i] != null) {
874 backingMap.put(key, bucketEntries[i]);
875 }
876
877 RAMQueueEntry ramCacheEntry = ramCache.remove(key);
878 if (ramCacheEntry != null) {
879 heapSize.addAndGet(-1 * entries.get(i).getData().heapSize());
880 } else if (bucketEntries[i] != null){
881
882 ReentrantReadWriteLock lock = offsetLock.getLock(bucketEntries[i].offset());
883 try {
884 lock.writeLock().lock();
885 if (backingMap.remove(key, bucketEntries[i])) {
886 blockEvicted(key, bucketEntries[i], false);
887 }
888 } finally {
889 lock.writeLock().unlock();
890 }
891 }
892 }
893
894 long used = bucketAllocator.getUsedSize();
895 if (used > acceptableSize()) {
896 freeSpace("Used=" + used + " > acceptable=" + acceptableSize());
897 }
898 return;
899 }
900 }
901
902
903
904
905
906
907
908
909
910 @VisibleForTesting
911 static List<RAMQueueEntry> getRAMQueueEntries(final BlockingQueue<RAMQueueEntry> q,
912 final List<RAMQueueEntry> receptical)
913 throws InterruptedException {
914
915
916 receptical.clear();
917 receptical.add(q.take());
918 q.drainTo(receptical);
919 return receptical;
920 }
921
922 private void persistToFile() throws IOException {
923 assert !cacheEnabled;
924 FileOutputStream fos = null;
925 ObjectOutputStream oos = null;
926 try {
927 if (!ioEngine.isPersistent())
928 throw new IOException(
929 "Attempt to persist non-persistent cache mappings!");
930 fos = new FileOutputStream(persistencePath, false);
931 oos = new ObjectOutputStream(fos);
932 oos.writeLong(cacheCapacity);
933 oos.writeUTF(ioEngine.getClass().getName());
934 oos.writeUTF(backingMap.getClass().getName());
935 oos.writeObject(deserialiserMap);
936 oos.writeObject(backingMap);
937 } finally {
938 if (oos != null) oos.close();
939 if (fos != null) fos.close();
940 }
941 }
942
943 @SuppressWarnings("unchecked")
944 private void retrieveFromFile(int[] bucketSizes) throws IOException, BucketAllocatorException,
945 ClassNotFoundException {
946 File persistenceFile = new File(persistencePath);
947 if (!persistenceFile.exists()) {
948 return;
949 }
950 assert !cacheEnabled;
951 FileInputStream fis = null;
952 ObjectInputStream ois = null;
953 try {
954 if (!ioEngine.isPersistent())
955 throw new IOException(
956 "Attempt to restore non-persistent cache mappings!");
957 fis = new FileInputStream(persistencePath);
958 ois = new ObjectInputStream(fis);
959 long capacitySize = ois.readLong();
960 if (capacitySize != cacheCapacity)
961 throw new IOException("Mismatched cache capacity:"
962 + StringUtils.byteDesc(capacitySize) + ", expected: "
963 + StringUtils.byteDesc(cacheCapacity));
964 String ioclass = ois.readUTF();
965 String mapclass = ois.readUTF();
966 if (!ioEngine.getClass().getName().equals(ioclass))
967 throw new IOException("Class name for IO engine mismatch: " + ioclass
968 + ", expected:" + ioEngine.getClass().getName());
969 if (!backingMap.getClass().getName().equals(mapclass))
970 throw new IOException("Class name for cache map mismatch: " + mapclass
971 + ", expected:" + backingMap.getClass().getName());
972 UniqueIndexMap<Integer> deserMap = (UniqueIndexMap<Integer>) ois
973 .readObject();
974 BucketAllocator allocator = new BucketAllocator(cacheCapacity, bucketSizes,
975 backingMap, realCacheSize);
976 backingMap = (ConcurrentHashMap<BlockCacheKey, BucketEntry>) ois
977 .readObject();
978 bucketAllocator = allocator;
979 deserialiserMap = deserMap;
980 } finally {
981 if (ois != null) ois.close();
982 if (fis != null) fis.close();
983 if (!persistenceFile.delete()) {
984 throw new IOException("Failed deleting persistence file "
985 + persistenceFile.getAbsolutePath());
986 }
987 }
988 }
989
990
991
992
993
994
995 private void checkIOErrorIsTolerated() {
996 long now = EnvironmentEdgeManager.currentTime();
997 if (this.ioErrorStartTime > 0) {
998 if (cacheEnabled && (now - ioErrorStartTime) > this.ioErrorsTolerationDuration) {
999 LOG.error("IO errors duration time has exceeded " + ioErrorsTolerationDuration +
1000 "ms, disabing cache, please check your IOEngine");
1001 disableCache();
1002 }
1003 } else {
1004 this.ioErrorStartTime = now;
1005 }
1006 }
1007
1008
1009
1010
1011
1012 private void disableCache() {
1013 if (!cacheEnabled)
1014 return;
1015 cacheEnabled = false;
1016 ioEngine.shutdown();
1017 this.scheduleThreadPool.shutdown();
1018 for (int i = 0; i < writerThreads.length; ++i)
1019 writerThreads[i].interrupt();
1020 this.ramCache.clear();
1021 if (!ioEngine.isPersistent() || persistencePath == null) {
1022 this.backingMap.clear();
1023 }
1024 }
1025
1026 private void join() throws InterruptedException {
1027 for (int i = 0; i < writerThreads.length; ++i)
1028 writerThreads[i].join();
1029 }
1030
1031 @Override
1032 public void shutdown() {
1033 disableCache();
1034 LOG.info("Shutdown bucket cache: IO persistent=" + ioEngine.isPersistent()
1035 + "; path to write=" + persistencePath);
1036 if (ioEngine.isPersistent() && persistencePath != null) {
1037 try {
1038 join();
1039 persistToFile();
1040 } catch (IOException ex) {
1041 LOG.error("Unable to persist data on exit: " + ex.toString(), ex);
1042 } catch (InterruptedException e) {
1043 LOG.warn("Failed to persist data on exit", e);
1044 }
1045 }
1046 }
1047
1048 @Override
1049 public CacheStats getStats() {
1050 return cacheStats;
1051 }
1052
1053 public BucketAllocator getAllocator() {
1054 return this.bucketAllocator;
1055 }
1056
1057 @Override
1058 public long heapSize() {
1059 return this.heapSize.get();
1060 }
1061
1062 @Override
1063 public long size() {
1064 return this.realCacheSize.get();
1065 }
1066
1067 @Override
1068 public long getFreeSize() {
1069 return this.bucketAllocator.getFreeSize();
1070 }
1071
1072 @Override
1073 public long getBlockCount() {
1074 return this.blockNumber.get();
1075 }
1076
1077 @Override
1078 public long getCurrentSize() {
1079 return this.bucketAllocator.getUsedSize();
1080 }
1081
1082
1083
1084
1085
1086
1087
1088
1089 @Override
1090 public int evictBlocksByHfileName(String hfileName) {
1091
1092
1093 Set<BlockCacheKey> keySet = blocksByHFile.values(hfileName);
1094 if (keySet == null) {
1095 return 0;
1096 }
1097 int numEvicted = 0;
1098 List<BlockCacheKey> keysForHFile = ImmutableList.copyOf(keySet);
1099 for (BlockCacheKey key : keysForHFile) {
1100 if (evictBlock(key)) {
1101 ++numEvicted;
1102 }
1103 }
1104
1105 return numEvicted;
1106 }
1107
1108
1109
1110
1111
1112
1113
1114
1115
1116 static class BucketEntry implements Serializable {
1117 private static final long serialVersionUID = -6741504807982257534L;
1118
1119
1120 static final Comparator<BucketEntry> COMPARATOR = new Comparator<BucketCache.BucketEntry>() {
1121
1122 @Override
1123 public int compare(BucketEntry o1, BucketEntry o2) {
1124 long accessCounter1 = o1.accessCounter;
1125 long accessCounter2 = o2.accessCounter;
1126 return accessCounter1 < accessCounter2 ? 1 : accessCounter1 == accessCounter2 ? 0 : -1;
1127 }
1128 };
1129
1130 private int offsetBase;
1131 private int length;
1132 private byte offset1;
1133 byte deserialiserIndex;
1134 private volatile long accessCounter;
1135 private BlockPriority priority;
1136
1137
1138
1139 private final long cachedTime = System.nanoTime();
1140
1141 BucketEntry(long offset, int length, long accessCounter, boolean inMemory) {
1142 setOffset(offset);
1143 this.length = length;
1144 this.accessCounter = accessCounter;
1145 if (inMemory) {
1146 this.priority = BlockPriority.MEMORY;
1147 } else {
1148 this.priority = BlockPriority.SINGLE;
1149 }
1150 }
1151
1152 long offset() {
1153 long o = ((long) offsetBase) & 0xFFFFFFFFL;
1154 o += (((long) (offset1)) & 0xFF) << 32;
1155 return o << 8;
1156 }
1157
1158 private void setOffset(long value) {
1159 assert (value & 0xFF) == 0;
1160 value >>= 8;
1161 offsetBase = (int) value;
1162 offset1 = (byte) (value >> 32);
1163 }
1164
1165 public int getLength() {
1166 return length;
1167 }
1168
1169 protected CacheableDeserializer<Cacheable> deserializerReference(
1170 UniqueIndexMap<Integer> deserialiserMap) {
1171 return CacheableDeserializerIdManager.getDeserializer(deserialiserMap
1172 .unmap(deserialiserIndex));
1173 }
1174
1175 protected void setDeserialiserReference(
1176 CacheableDeserializer<Cacheable> deserializer,
1177 UniqueIndexMap<Integer> deserialiserMap) {
1178 this.deserialiserIndex = ((byte) deserialiserMap.map(deserializer
1179 .getDeserialiserIdentifier()));
1180 }
1181
1182
1183
1184
1185 public void access(long accessCounter) {
1186 this.accessCounter = accessCounter;
1187 if (this.priority == BlockPriority.SINGLE) {
1188 this.priority = BlockPriority.MULTI;
1189 }
1190 }
1191
1192 public BlockPriority getPriority() {
1193 return this.priority;
1194 }
1195
1196 public long getCachedTime() {
1197 return cachedTime;
1198 }
1199 }
1200
1201
1202
1203
1204
1205
1206
1207 private class BucketEntryGroup implements Comparable<BucketEntryGroup> {
1208
1209 private CachedEntryQueue queue;
1210 private long totalSize = 0;
1211 private long bucketSize;
1212
1213 public BucketEntryGroup(long bytesToFree, long blockSize, long bucketSize) {
1214 this.bucketSize = bucketSize;
1215 queue = new CachedEntryQueue(bytesToFree, blockSize);
1216 totalSize = 0;
1217 }
1218
1219 public void add(Map.Entry<BlockCacheKey, BucketEntry> block) {
1220 totalSize += block.getValue().getLength();
1221 queue.add(block);
1222 }
1223
1224 public long free(long toFree) {
1225 Map.Entry<BlockCacheKey, BucketEntry> entry;
1226 long freedBytes = 0;
1227 while ((entry = queue.pollLast()) != null) {
1228 evictBlock(entry.getKey());
1229 freedBytes += entry.getValue().getLength();
1230 if (freedBytes >= toFree) {
1231 return freedBytes;
1232 }
1233 }
1234 return freedBytes;
1235 }
1236
1237 public long overflow() {
1238 return totalSize - bucketSize;
1239 }
1240
1241 public long totalSize() {
1242 return totalSize;
1243 }
1244
1245 @Override
1246 public int compareTo(BucketEntryGroup that) {
1247 if (this.overflow() == that.overflow())
1248 return 0;
1249 return this.overflow() > that.overflow() ? 1 : -1;
1250 }
1251
1252 @Override
1253 public boolean equals(Object that) {
1254 return this == that;
1255 }
1256
1257 }
1258
1259
1260
1261
1262 @VisibleForTesting
1263 static class RAMQueueEntry {
1264 private BlockCacheKey key;
1265 private Cacheable data;
1266 private long accessCounter;
1267 private boolean inMemory;
1268
1269 public RAMQueueEntry(BlockCacheKey bck, Cacheable data, long accessCounter,
1270 boolean inMemory) {
1271 this.key = bck;
1272 this.data = data;
1273 this.accessCounter = accessCounter;
1274 this.inMemory = inMemory;
1275 }
1276
1277 public Cacheable getData() {
1278 return data;
1279 }
1280
1281 public BlockCacheKey getKey() {
1282 return key;
1283 }
1284
1285 public void access(long accessCounter) {
1286 this.accessCounter = accessCounter;
1287 }
1288
1289 public BucketEntry writeToCache(final IOEngine ioEngine,
1290 final BucketAllocator bucketAllocator,
1291 final UniqueIndexMap<Integer> deserialiserMap,
1292 final AtomicLong realCacheSize) throws CacheFullException, IOException,
1293 BucketAllocatorException {
1294 int len = data.getSerializedLength();
1295
1296 if (len == 0) return null;
1297 long offset = bucketAllocator.allocateBlock(len);
1298 BucketEntry bucketEntry = new BucketEntry(offset, len, accessCounter, inMemory);
1299 bucketEntry.setDeserialiserReference(data.getDeserializer(), deserialiserMap);
1300 try {
1301 if (data instanceof HFileBlock) {
1302 HFileBlock block = (HFileBlock) data;
1303 ByteBuffer sliceBuf = block.getBufferReadOnlyWithHeader();
1304 sliceBuf.rewind();
1305 assert len == sliceBuf.limit() + HFileBlock.EXTRA_SERIALIZATION_SPACE ||
1306 len == sliceBuf.limit() + block.headerSize() + HFileBlock.EXTRA_SERIALIZATION_SPACE;
1307 ByteBuffer extraInfoBuffer = ByteBuffer.allocate(HFileBlock.EXTRA_SERIALIZATION_SPACE);
1308 block.serializeExtraInfo(extraInfoBuffer);
1309 ioEngine.write(sliceBuf, offset);
1310 ioEngine.write(extraInfoBuffer, offset + len - HFileBlock.EXTRA_SERIALIZATION_SPACE);
1311 } else {
1312 ByteBuffer bb = ByteBuffer.allocate(len);
1313 data.serialize(bb);
1314 ioEngine.write(bb, offset);
1315 }
1316 } catch (IOException ioe) {
1317
1318 bucketAllocator.freeBlock(offset);
1319 throw ioe;
1320 }
1321
1322 realCacheSize.addAndGet(len);
1323 return bucketEntry;
1324 }
1325 }
1326
1327
1328
1329
1330
1331 void stopWriterThreads() throws InterruptedException {
1332 for (WriterThread writerThread : writerThreads) {
1333 writerThread.disableWriter();
1334 writerThread.interrupt();
1335 writerThread.join();
1336 }
1337 }
1338
1339 @Override
1340 public Iterator<CachedBlock> iterator() {
1341
1342 final Iterator<Map.Entry<BlockCacheKey, BucketEntry>> i =
1343 this.backingMap.entrySet().iterator();
1344 return new Iterator<CachedBlock>() {
1345 private final long now = System.nanoTime();
1346
1347 @Override
1348 public boolean hasNext() {
1349 return i.hasNext();
1350 }
1351
1352 @Override
1353 public CachedBlock next() {
1354 final Map.Entry<BlockCacheKey, BucketEntry> e = i.next();
1355 return new CachedBlock() {
1356 @Override
1357 public String toString() {
1358 return BlockCacheUtil.toString(this, now);
1359 }
1360
1361 @Override
1362 public BlockPriority getBlockPriority() {
1363 return e.getValue().getPriority();
1364 }
1365
1366 @Override
1367 public BlockType getBlockType() {
1368
1369 return null;
1370 }
1371
1372 @Override
1373 public long getOffset() {
1374 return e.getKey().getOffset();
1375 }
1376
1377 @Override
1378 public long getSize() {
1379 return e.getValue().getLength();
1380 }
1381
1382 @Override
1383 public long getCachedTime() {
1384 return e.getValue().getCachedTime();
1385 }
1386
1387 @Override
1388 public String getFilename() {
1389 return e.getKey().getHfileName();
1390 }
1391
1392 @Override
1393 public int compareTo(CachedBlock other) {
1394 int diff = this.getFilename().compareTo(other.getFilename());
1395 if (diff != 0) return diff;
1396 diff = (int)(this.getOffset() - other.getOffset());
1397 if (diff != 0) return diff;
1398 if (other.getCachedTime() < 0 || this.getCachedTime() < 0) {
1399 throw new IllegalStateException("" + this.getCachedTime() + ", " +
1400 other.getCachedTime());
1401 }
1402 return (int)(other.getCachedTime() - this.getCachedTime());
1403 }
1404
1405 @Override
1406 public int hashCode() {
1407 return e.getKey().hashCode();
1408 }
1409
1410 @Override
1411 public boolean equals(Object obj) {
1412 if (obj instanceof CachedBlock) {
1413 CachedBlock cb = (CachedBlock)obj;
1414 return compareTo(cb) == 0;
1415 } else {
1416 return false;
1417 }
1418 }
1419 };
1420 }
1421
1422 @Override
1423 public void remove() {
1424 throw new UnsupportedOperationException();
1425 }
1426 };
1427 }
1428
1429 @Override
1430 public BlockCache[] getBlockCaches() {
1431 return null;
1432 }
1433 }