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