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