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