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