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