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