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