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