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 com.google.common.base.Preconditions;
24 import com.google.common.util.concurrent.ThreadFactoryBuilder;
25 import java.io.File;
26 import java.io.FileInputStream;
27 import java.io.FileNotFoundException;
28 import java.io.FileOutputStream;
29 import java.io.IOException;
30 import java.io.ObjectInputStream;
31 import java.io.ObjectOutputStream;
32 import java.io.Serializable;
33 import java.nio.ByteBuffer;
34 import java.util.ArrayList;
35 import java.util.Comparator;
36 import java.util.HashSet;
37 import java.util.Iterator;
38 import java.util.List;
39 import java.util.Map;
40 import java.util.NavigableSet;
41 import java.util.PriorityQueue;
42 import java.util.Set;
43 import java.util.concurrent.ArrayBlockingQueue;
44 import java.util.concurrent.BlockingQueue;
45 import java.util.concurrent.ConcurrentHashMap;
46 import java.util.concurrent.ConcurrentMap;
47 import java.util.concurrent.ConcurrentSkipListSet;
48 import java.util.concurrent.Executors;
49 import java.util.concurrent.ScheduledExecutorService;
50 import java.util.concurrent.TimeUnit;
51 import java.util.concurrent.atomic.AtomicLong;
52 import java.util.concurrent.locks.Lock;
53 import java.util.concurrent.locks.ReentrantLock;
54 import java.util.concurrent.locks.ReentrantReadWriteLock;
55 import org.apache.commons.logging.Log;
56 import org.apache.commons.logging.LogFactory;
57 import org.apache.hadoop.conf.Configuration;
58 import org.apache.hadoop.hbase.HBaseConfiguration;
59 import org.apache.hadoop.hbase.classification.InterfaceAudience;
60 import org.apache.hadoop.hbase.io.HeapSize;
61 import org.apache.hadoop.hbase.io.hfile.BlockCache;
62 import org.apache.hadoop.hbase.io.hfile.BlockCacheKey;
63 import org.apache.hadoop.hbase.io.hfile.BlockCacheUtil;
64 import org.apache.hadoop.hbase.io.hfile.BlockPriority;
65 import org.apache.hadoop.hbase.io.hfile.BlockType;
66 import org.apache.hadoop.hbase.io.hfile.CacheStats;
67 import org.apache.hadoop.hbase.io.hfile.Cacheable;
68 import org.apache.hadoop.hbase.io.hfile.CacheableDeserializer;
69 import org.apache.hadoop.hbase.io.hfile.CacheableDeserializerIdManager;
70 import org.apache.hadoop.hbase.io.hfile.CachedBlock;
71 import org.apache.hadoop.hbase.io.hfile.HFileBlock;
72 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
73 import org.apache.hadoop.hbase.util.Bytes;
74 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
75 import org.apache.hadoop.hbase.util.HasThread;
76 import org.apache.hadoop.hbase.util.IdReadWriteLock;
77 import org.apache.hadoop.util.StringUtils;
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97 @InterfaceAudience.Private
98 public class BucketCache implements BlockCache, HeapSize {
99 private static final Log LOG = LogFactory.getLog(BucketCache.class);
100
101
102 static final String SINGLE_FACTOR_CONFIG_NAME = "hbase.bucketcache.single.factor";
103 static final String MULTI_FACTOR_CONFIG_NAME = "hbase.bucketcache.multi.factor";
104 static final String MEMORY_FACTOR_CONFIG_NAME = "hbase.bucketcache.memory.factor";
105 static final String EXTRA_FREE_FACTOR_CONFIG_NAME = "hbase.bucketcache.extrafreefactor";
106 static final String ACCEPT_FACTOR_CONFIG_NAME = "hbase.bucketcache.acceptfactor";
107 static final String MIN_FACTOR_CONFIG_NAME = "hbase.bucketcache.minfactor";
108
109
110 static final float DEFAULT_SINGLE_FACTOR = 0.25f;
111 static final float DEFAULT_MULTI_FACTOR = 0.50f;
112 static final float DEFAULT_MEMORY_FACTOR = 0.25f;
113 static final float DEFAULT_MIN_FACTOR = 0.85f;
114
115 private static final float DEFAULT_EXTRA_FREE_FACTOR = 0.10f;
116 private static final float DEFAULT_ACCEPT_FACTOR = 0.95f;
117
118
119 private static final int DEFAULT_FREE_ENTIRE_BLOCK_FACTOR = 2;
120
121
122 private static final int statThreadPeriod = 5 * 60;
123
124 final static int DEFAULT_WRITER_THREADS = 3;
125 final static int DEFAULT_WRITER_QUEUE_ITEMS = 64;
126
127
128 transient final IOEngine ioEngine;
129
130
131 transient final ConcurrentMap<BlockCacheKey, RAMQueueEntry> ramCache;
132
133 transient ConcurrentMap<BlockCacheKey, BucketEntry> backingMap;
134
135
136
137
138
139
140 private volatile boolean cacheEnabled;
141
142
143
144
145
146
147
148
149 transient final ArrayList<BlockingQueue<RAMQueueEntry>> writerQueues =
150 new ArrayList<BlockingQueue<RAMQueueEntry>>();
151 transient final WriterThread[] writerThreads;
152
153
154 private volatile boolean freeInProgress = false;
155 private transient final Lock freeSpaceLock = new ReentrantLock();
156
157 private UniqueIndexMap<Integer> deserialiserMap = new UniqueIndexMap<Integer>();
158
159 private final AtomicLong realCacheSize = new AtomicLong(0);
160 private final AtomicLong heapSize = new AtomicLong(0);
161
162 private final AtomicLong blockNumber = new AtomicLong(0);
163
164
165 private final AtomicLong accessCount = new AtomicLong(0);
166
167 private static final int DEFAULT_CACHE_WAIT_TIME = 50;
168
169
170
171 boolean wait_when_cache = false;
172
173 private final BucketCacheStats cacheStats = new BucketCacheStats();
174
175 private final String persistencePath;
176 private final long cacheCapacity;
177
178 private final long blockSize;
179
180
181 private final int ioErrorsTolerationDuration;
182
183 public static final int DEFAULT_ERROR_TOLERATION_DURATION = 60 * 1000;
184
185
186
187 private volatile long ioErrorStartTime = -1;
188
189
190
191
192
193 transient final IdReadWriteLock offsetLock = new IdReadWriteLock();
194
195 private final NavigableSet<BlockCacheKey> blocksByHFile =
196 new ConcurrentSkipListSet<BlockCacheKey>(new Comparator<BlockCacheKey>() {
197 @Override
198 public int compare(BlockCacheKey a, BlockCacheKey b) {
199 int nameComparison = a.getHfileName().compareTo(b.getHfileName());
200 if (nameComparison != 0) {
201 return nameComparison;
202 }
203
204 if (a.getOffset() == b.getOffset()) {
205 return 0;
206 } else if (a.getOffset() < b.getOffset()) {
207 return -1;
208 }
209 return 1;
210 }
211 });
212
213
214 private transient final ScheduledExecutorService scheduleThreadPool =
215 Executors.newScheduledThreadPool(1,
216 new ThreadFactoryBuilder().setNameFormat("BucketCacheStatsExecutor").setDaemon(true).build());
217
218
219 private transient BucketAllocator bucketAllocator;
220
221
222 private float acceptableFactor;
223
224
225 private float minFactor;
226
227
228 private float extraFreeFactor;
229
230
231 private float singleFactor;
232
233
234 private float multiFactor;
235
236
237 private float memoryFactor;
238
239 private static final String FILE_VERIFY_ALGORITHM =
240 "hbase.bucketcache.persistent.file.integrity.check.algorithm";
241 private static final String DEFAULT_FILE_VERIFY_ALGORITHM = "MD5";
242
243
244
245
246
247 private String algorithm;
248
249 public BucketCache(String ioEngineName, long capacity, int blockSize, int[] bucketSizes,
250 int writerThreadNum, int writerQLen, String persistencePath) throws FileNotFoundException,
251 IOException {
252 this(ioEngineName, capacity, blockSize, bucketSizes, writerThreadNum, writerQLen,
253 persistencePath, DEFAULT_ERROR_TOLERATION_DURATION, HBaseConfiguration.create());
254 }
255
256 public BucketCache(String ioEngineName, long capacity, int blockSize, int[] bucketSizes,
257 int writerThreadNum, int writerQLen, String persistencePath, int ioErrorsTolerationDuration,
258 Configuration conf)
259 throws IOException {
260 this.algorithm = conf.get(FILE_VERIFY_ALGORITHM, DEFAULT_FILE_VERIFY_ALGORITHM);
261 ioEngine = getIOEngineFromName(ioEngineName, capacity);
262 this.writerThreads = new WriterThread[writerThreadNum];
263 long blockNumCapacity = capacity / blockSize;
264 if (blockNumCapacity >= Integer.MAX_VALUE) {
265
266 throw new IllegalArgumentException("Cache capacity is too large, only support 32TB now");
267 }
268
269 this.acceptableFactor = conf.getFloat(ACCEPT_FACTOR_CONFIG_NAME, DEFAULT_ACCEPT_FACTOR);
270 this.minFactor = conf.getFloat(MIN_FACTOR_CONFIG_NAME, DEFAULT_MIN_FACTOR);
271 this.extraFreeFactor = conf.getFloat(EXTRA_FREE_FACTOR_CONFIG_NAME, DEFAULT_EXTRA_FREE_FACTOR);
272 this.singleFactor = conf.getFloat(SINGLE_FACTOR_CONFIG_NAME, DEFAULT_SINGLE_FACTOR);
273 this.multiFactor = conf.getFloat(MULTI_FACTOR_CONFIG_NAME, DEFAULT_MULTI_FACTOR);
274 this.memoryFactor = conf.getFloat(MEMORY_FACTOR_CONFIG_NAME, DEFAULT_MEMORY_FACTOR);
275
276 sanityCheckConfigs();
277
278 LOG.info("Instantiating BucketCache with acceptableFactor: " + acceptableFactor + ", minFactor: " + minFactor +
279 ", extraFreeFactor: " + extraFreeFactor + ", singleFactor: " + singleFactor + ", multiFactor: " + multiFactor +
280 ", memoryFactor: " + memoryFactor);
281
282 this.cacheCapacity = capacity;
283 this.persistencePath = persistencePath;
284 this.blockSize = blockSize;
285 this.ioErrorsTolerationDuration = ioErrorsTolerationDuration;
286
287 bucketAllocator = new BucketAllocator(capacity, bucketSizes);
288 for (int i = 0; i < writerThreads.length; ++i) {
289 writerQueues.add(new ArrayBlockingQueue<RAMQueueEntry>(writerQLen));
290 }
291
292 assert writerQueues.size() == writerThreads.length;
293 this.ramCache = new ConcurrentHashMap<BlockCacheKey, RAMQueueEntry>();
294
295 this.backingMap = new ConcurrentHashMap<BlockCacheKey, BucketEntry>((int) blockNumCapacity);
296
297 if (ioEngine.isPersistent() && persistencePath != null) {
298 try {
299 retrieveFromFile(bucketSizes);
300 } catch (IOException ioex) {
301 LOG.error("Can't restore from file because of", ioex);
302 } catch (ClassNotFoundException cnfe) {
303 LOG.error("Can't restore from file in rebuild because can't deserialise", cnfe);
304 throw new RuntimeException(cnfe);
305 }
306 }
307 final String threadName = Thread.currentThread().getName();
308 this.cacheEnabled = true;
309 for (int i = 0; i < writerThreads.length; ++i) {
310 writerThreads[i] = new WriterThread(writerQueues.get(i));
311 writerThreads[i].setName(threadName + "-BucketCacheWriter-" + i);
312 writerThreads[i].setDaemon(true);
313 }
314 startWriterThreads();
315
316
317
318
319 this.scheduleThreadPool.scheduleAtFixedRate(new StatisticsThread(this),
320 statThreadPeriod, statThreadPeriod, TimeUnit.SECONDS);
321 LOG.info("Started bucket cache; ioengine=" + ioEngineName +
322 ", capacity=" + StringUtils.byteDesc(capacity) +
323 ", blockSize=" + StringUtils.byteDesc(blockSize) + ", writerThreadNum=" +
324 writerThreadNum + ", writerQLen=" + writerQLen + ", persistencePath=" +
325 persistencePath + ", bucketAllocator=" + this.bucketAllocator.getClass().getName());
326 }
327
328 private void sanityCheckConfigs() {
329 Preconditions.checkArgument(acceptableFactor <= 1 && acceptableFactor >= 0, ACCEPT_FACTOR_CONFIG_NAME + " must be between 0.0 and 1.0");
330 Preconditions.checkArgument(minFactor <= 1 && minFactor >= 0, MIN_FACTOR_CONFIG_NAME + " must be between 0.0 and 1.0");
331 Preconditions.checkArgument(minFactor <= acceptableFactor, MIN_FACTOR_CONFIG_NAME + " must be <= " + ACCEPT_FACTOR_CONFIG_NAME);
332 Preconditions.checkArgument(extraFreeFactor >= 0, EXTRA_FREE_FACTOR_CONFIG_NAME + " must be greater than 0.0");
333 Preconditions.checkArgument(singleFactor <= 1 && singleFactor >= 0, SINGLE_FACTOR_CONFIG_NAME + " must be between 0.0 and 1.0");
334 Preconditions.checkArgument(multiFactor <= 1 && multiFactor >= 0, MULTI_FACTOR_CONFIG_NAME + " must be between 0.0 and 1.0");
335 Preconditions.checkArgument(memoryFactor <= 1 && memoryFactor >= 0, MEMORY_FACTOR_CONFIG_NAME + " must be between 0.0 and 1.0");
336 Preconditions.checkArgument((singleFactor + multiFactor + memoryFactor) == 1, SINGLE_FACTOR_CONFIG_NAME + ", " +
337 MULTI_FACTOR_CONFIG_NAME + ", and " + MEMORY_FACTOR_CONFIG_NAME + " segments must add up to 1.0");
338 }
339
340
341
342
343
344 protected void startWriterThreads() {
345 for (WriterThread thread : writerThreads) {
346 thread.start();
347 }
348 }
349
350 boolean isCacheEnabled() {
351 return this.cacheEnabled;
352 }
353
354 @Override
355 public long getMaxSize() {
356 return this.cacheCapacity;
357 }
358
359 public String getIoEngine() {
360 return ioEngine.toString();
361 }
362
363
364
365
366
367
368
369
370 private IOEngine getIOEngineFromName(String ioEngineName, long capacity)
371 throws IOException {
372 if (ioEngineName.startsWith("file:") || ioEngineName.startsWith("files:")) {
373
374
375
376 String[] filePaths =
377 ioEngineName.substring(ioEngineName.indexOf(":") + 1).split(FileIOEngine.FILE_DELIMITER);
378 return new FileIOEngine(capacity, filePaths);
379 } else if (ioEngineName.startsWith("offheap"))
380 return new ByteBufferIOEngine(capacity, true);
381 else if (ioEngineName.startsWith("heap"))
382 return new ByteBufferIOEngine(capacity, false);
383 else
384 throw new IllegalArgumentException(
385 "Don't understand io engine name for cache - prefix with file:, heap or offheap");
386 }
387
388
389
390
391
392
393 @Override
394 public void cacheBlock(BlockCacheKey cacheKey, Cacheable buf) {
395 cacheBlock(cacheKey, buf, false, false);
396 }
397
398
399
400
401
402
403
404
405 @Override
406 public void cacheBlock(BlockCacheKey cacheKey, Cacheable cachedItem, boolean inMemory,
407 final boolean cacheDataInL1) {
408 cacheBlockWithWait(cacheKey, cachedItem, inMemory, cacheDataInL1, wait_when_cache);
409 }
410
411
412
413
414
415
416
417
418 public void cacheBlockWithWait(BlockCacheKey cacheKey, Cacheable cachedItem, boolean inMemory,
419 boolean cacheDataInL1, boolean wait) {
420 if (cacheEnabled) {
421 if (backingMap.containsKey(cacheKey) || ramCache.containsKey(cacheKey)) {
422 if (!cacheDataInL1
423 && BlockCacheUtil.shouldReplaceExistingCacheBlock(this, cacheKey, cachedItem)) {
424 cacheBlockWithWaitInternal(cacheKey, cachedItem, inMemory, cacheDataInL1, wait);
425 }
426 } else {
427 cacheBlockWithWaitInternal(cacheKey, cachedItem, inMemory, cacheDataInL1, wait);
428 }
429 }
430 }
431
432 private void cacheBlockWithWaitInternal(BlockCacheKey cacheKey, Cacheable cachedItem,
433 boolean inMemory, boolean cacheDataInL1, boolean wait) {
434 if (LOG.isTraceEnabled()) LOG.trace("Caching key=" + cacheKey + ", item=" + cachedItem);
435 if (!cacheEnabled) {
436 return;
437 }
438
439
440 RAMQueueEntry re =
441 new RAMQueueEntry(cacheKey, cachedItem, accessCount.incrementAndGet(), inMemory);
442
443
444
445
446
447
448
449 if (ramCache.putIfAbsent(cacheKey, re) != null) {
450 return;
451 }
452 int queueNum = (cacheKey.hashCode() & 0x7FFFFFFF) % writerQueues.size();
453 BlockingQueue<RAMQueueEntry> bq = writerQueues.get(queueNum);
454 boolean successfulAddition = false;
455 if (wait) {
456 try {
457 successfulAddition = bq.offer(re, DEFAULT_CACHE_WAIT_TIME, TimeUnit.MILLISECONDS);
458 } catch (InterruptedException e) {
459 Thread.currentThread().interrupt();
460 }
461 } else {
462 successfulAddition = bq.offer(re);
463 }
464 if (!successfulAddition) {
465 ramCache.remove(cacheKey);
466 cacheStats.failInsert();
467 } else {
468 this.blockNumber.incrementAndGet();
469 this.heapSize.addAndGet(cachedItem.heapSize());
470 }
471 }
472
473
474
475
476
477
478
479
480
481 @Override
482 public Cacheable getBlock(BlockCacheKey key, boolean caching, boolean repeat,
483 boolean updateCacheMetrics) {
484 if (!cacheEnabled) {
485 return null;
486 }
487 RAMQueueEntry re = ramCache.get(key);
488 if (re != null) {
489 if (updateCacheMetrics) {
490 cacheStats.hit(caching, key.isPrimary(), key.getBlockType());
491 }
492 re.access(accessCount.incrementAndGet());
493 return re.getData();
494 }
495 BucketEntry bucketEntry = backingMap.get(key);
496 if (bucketEntry != null) {
497 long start = System.nanoTime();
498 ReentrantReadWriteLock lock = offsetLock.getLock(bucketEntry.offset());
499 try {
500 lock.readLock().lock();
501
502
503
504 if (bucketEntry.equals(backingMap.get(key))) {
505 int len = bucketEntry.getLength();
506 if (LOG.isTraceEnabled()) {
507 LOG.trace("Read offset=" + bucketEntry.offset() + ", len=" + len);
508 }
509 ByteBuffer bb = ByteBuffer.allocate(len);
510 int lenRead = ioEngine.read(bb, bucketEntry.offset());
511 if (lenRead != len) {
512 throw new RuntimeException("Only " + lenRead + " bytes read, " + len + " expected");
513 }
514 CacheableDeserializer<Cacheable> deserializer =
515 bucketEntry.deserializerReference(this.deserialiserMap);
516 Cacheable cachedBlock = deserializer.deserialize(bb, true);
517 long timeTaken = System.nanoTime() - start;
518 if (updateCacheMetrics) {
519 cacheStats.hit(caching, key.isPrimary(), key.getBlockType());
520 cacheStats.ioHit(timeTaken);
521 }
522 bucketEntry.access(accessCount.incrementAndGet());
523 if (this.ioErrorStartTime > 0) {
524 ioErrorStartTime = -1;
525 }
526 return cachedBlock;
527 }
528 } catch (IOException ioex) {
529 LOG.error("Failed reading block " + key + " from bucket cache", ioex);
530 checkIOErrorIsTolerated();
531 } finally {
532 lock.readLock().unlock();
533 }
534 }
535 if (!repeat && updateCacheMetrics) {
536 cacheStats.miss(caching, key.isPrimary(), key.getBlockType());
537 }
538 return null;
539 }
540
541 void blockEvicted(BlockCacheKey cacheKey, BucketEntry bucketEntry, boolean decrementBlockNumber) {
542 bucketAllocator.freeBlock(bucketEntry.offset());
543 realCacheSize.addAndGet(-1 * bucketEntry.getLength());
544 blocksByHFile.remove(cacheKey);
545 if (decrementBlockNumber) {
546 this.blockNumber.decrementAndGet();
547 }
548 }
549
550 @Override
551 public boolean evictBlock(BlockCacheKey cacheKey) {
552 if (!cacheEnabled) {
553 return false;
554 }
555 RAMQueueEntry removedBlock = ramCache.remove(cacheKey);
556 if (removedBlock != null) {
557 this.blockNumber.decrementAndGet();
558 this.heapSize.addAndGet(-1 * removedBlock.getData().heapSize());
559 }
560 BucketEntry bucketEntry = backingMap.get(cacheKey);
561 if (bucketEntry == null) {
562 if (removedBlock != null) {
563 cacheStats.evicted(0, cacheKey.isPrimary());
564 return true;
565 } else {
566 return false;
567 }
568 }
569 ReentrantReadWriteLock lock = offsetLock.getLock(bucketEntry.offset());
570 try {
571 lock.writeLock().lock();
572 if (backingMap.remove(cacheKey, bucketEntry)) {
573 blockEvicted(cacheKey, bucketEntry, removedBlock == null);
574 } else {
575 return false;
576 }
577 } finally {
578 lock.writeLock().unlock();
579 }
580 cacheStats.evicted(bucketEntry.getCachedTime(), cacheKey.isPrimary());
581 return true;
582 }
583
584
585
586
587 private static class StatisticsThread extends Thread {
588 private final BucketCache bucketCache;
589
590 public StatisticsThread(BucketCache bucketCache) {
591 super("BucketCacheStatsThread");
592 setDaemon(true);
593 this.bucketCache = bucketCache;
594 }
595
596 @Override
597 public void run() {
598 bucketCache.logStats();
599 }
600 }
601
602 public void logStats() {
603 long totalSize = bucketAllocator.getTotalSize();
604 long usedSize = bucketAllocator.getUsedSize();
605 long freeSize = totalSize - usedSize;
606 long cacheSize = getRealCacheSize();
607 LOG.info("failedBlockAdditions=" + cacheStats.getFailedInserts() + ", " +
608 "totalSize=" + StringUtils.byteDesc(totalSize) + ", " +
609 "freeSize=" + StringUtils.byteDesc(freeSize) + ", " +
610 "usedSize=" + StringUtils.byteDesc(usedSize) +", " +
611 "cacheSize=" + StringUtils.byteDesc(cacheSize) +", " +
612 "accesses=" + cacheStats.getRequestCount() + ", " +
613 "hits=" + cacheStats.getHitCount() + ", " +
614 "IOhitsPerSecond=" + cacheStats.getIOHitsPerSecond() + ", " +
615 "IOTimePerHit=" + String.format("%.2f", cacheStats.getIOTimePerHit())+ ", " +
616 "hitRatio=" + (cacheStats.getHitCount() == 0 ? "0," :
617 (StringUtils.formatPercent(cacheStats.getHitRatio(), 2)+ ", ")) +
618 "cachingAccesses=" + cacheStats.getRequestCachingCount() + ", " +
619 "cachingHits=" + cacheStats.getHitCachingCount() + ", " +
620 "cachingHitsRatio=" +(cacheStats.getHitCachingCount() == 0 ? "0," :
621 (StringUtils.formatPercent(cacheStats.getHitCachingRatio(), 2)+ ", ")) +
622 "evictions=" + cacheStats.getEvictionCount() + ", " +
623 "evicted=" + cacheStats.getEvictedCount() + ", " +
624 "evictedPerRun=" + cacheStats.evictedPerEviction());
625 cacheStats.reset();
626 }
627
628 public long getRealCacheSize() {
629 return this.realCacheSize.get();
630 }
631
632 private long acceptableSize() {
633 return (long) Math.floor(bucketAllocator.getTotalSize() * acceptableFactor);
634 }
635
636 long getPartitionSize(float partitionFactor) {
637 return (long) Math.floor(bucketAllocator.getTotalSize() * partitionFactor * minFactor);
638 }
639
640
641
642
643 private int bucketSizesAboveThresholdCount(float minFactor) {
644 BucketAllocator.IndexStatistics[] stats = bucketAllocator.getIndexStatistics();
645 int fullCount = 0;
646 for (int i = 0; i < stats.length; i++) {
647 long freeGoal = (long) Math.floor(stats[i].totalCount() * (1 - minFactor));
648 freeGoal = Math.max(freeGoal, 1);
649 if (stats[i].freeCount() < freeGoal) {
650 fullCount++;
651 }
652 }
653 return fullCount;
654 }
655
656
657
658
659
660
661
662
663
664
665 private void freeEntireBuckets(int completelyFreeBucketsNeeded) {
666 if (completelyFreeBucketsNeeded != 0) {
667
668
669 Set<Integer> inUseBuckets = new HashSet<Integer>();
670 for (BucketEntry entry : backingMap.values()) {
671 inUseBuckets.add(bucketAllocator.getBucketIndex(entry.offset()));
672 }
673
674 Set<Integer> candidateBuckets = bucketAllocator.getLeastFilledBuckets(
675 inUseBuckets, completelyFreeBucketsNeeded);
676 for (Map.Entry<BlockCacheKey, BucketEntry> entry : backingMap.entrySet()) {
677 if (candidateBuckets.contains(bucketAllocator
678 .getBucketIndex(entry.getValue().offset()))) {
679 evictBlock(entry.getKey());
680 }
681 }
682 }
683 }
684
685
686
687
688
689
690
691 private void freeSpace(final String why) {
692
693 if (!freeSpaceLock.tryLock()) {
694 return;
695 }
696 try {
697 freeInProgress = true;
698 long bytesToFreeWithoutExtra = 0;
699
700 StringBuffer msgBuffer = LOG.isDebugEnabled()? new StringBuffer(): null;
701 BucketAllocator.IndexStatistics[] stats = bucketAllocator.getIndexStatistics();
702 long[] bytesToFreeForBucket = new long[stats.length];
703 for (int i = 0; i < stats.length; i++) {
704 bytesToFreeForBucket[i] = 0;
705 long freeGoal = (long) Math.floor(stats[i].totalCount() * (1 - minFactor));
706 freeGoal = Math.max(freeGoal, 1);
707 if (stats[i].freeCount() < freeGoal) {
708 bytesToFreeForBucket[i] = stats[i].itemSize() * (freeGoal - stats[i].freeCount());
709 bytesToFreeWithoutExtra += bytesToFreeForBucket[i];
710 if (msgBuffer != null) {
711 msgBuffer.append("Free for bucketSize(" + stats[i].itemSize() + ")="
712 + StringUtils.byteDesc(bytesToFreeForBucket[i]) + ", ");
713 }
714 }
715 }
716 if (msgBuffer != null) {
717 msgBuffer.append("Free for total=" + StringUtils.byteDesc(bytesToFreeWithoutExtra) + ", ");
718 }
719
720 if (bytesToFreeWithoutExtra <= 0) {
721 return;
722 }
723 long currentSize = bucketAllocator.getUsedSize();
724 long totalSize = bucketAllocator.getTotalSize();
725 if (LOG.isDebugEnabled() && msgBuffer != null) {
726 LOG.debug("Free started because \"" + why + "\"; " + msgBuffer.toString() +
727 " of current used=" + StringUtils.byteDesc(currentSize) + ", actual cacheSize=" +
728 StringUtils.byteDesc(realCacheSize.get()) + ", total=" + StringUtils.byteDesc(totalSize));
729 }
730
731 long bytesToFreeWithExtra = (long) Math.floor(bytesToFreeWithoutExtra
732 * (1 + extraFreeFactor));
733
734
735 BucketEntryGroup bucketSingle = new BucketEntryGroup(bytesToFreeWithExtra,
736 blockSize, getPartitionSize(singleFactor));
737 BucketEntryGroup bucketMulti = new BucketEntryGroup(bytesToFreeWithExtra,
738 blockSize, getPartitionSize(multiFactor));
739 BucketEntryGroup bucketMemory = new BucketEntryGroup(bytesToFreeWithExtra,
740 blockSize, getPartitionSize(memoryFactor));
741
742
743
744 for (Map.Entry<BlockCacheKey, BucketEntry> bucketEntryWithKey : backingMap.entrySet()) {
745 switch (bucketEntryWithKey.getValue().getPriority()) {
746 case SINGLE: {
747 bucketSingle.add(bucketEntryWithKey);
748 break;
749 }
750 case MULTI: {
751 bucketMulti.add(bucketEntryWithKey);
752 break;
753 }
754 case MEMORY: {
755 bucketMemory.add(bucketEntryWithKey);
756 break;
757 }
758 }
759 }
760
761 PriorityQueue<BucketEntryGroup> bucketQueue = new PriorityQueue<BucketEntryGroup>(3);
762
763 bucketQueue.add(bucketSingle);
764 bucketQueue.add(bucketMulti);
765 bucketQueue.add(bucketMemory);
766
767 int remainingBuckets = bucketQueue.size();
768 long bytesFreed = 0;
769
770 BucketEntryGroup bucketGroup;
771 while ((bucketGroup = bucketQueue.poll()) != null) {
772 long overflow = bucketGroup.overflow();
773 if (overflow > 0) {
774 long bucketBytesToFree = Math.min(overflow,
775 (bytesToFreeWithoutExtra - bytesFreed) / remainingBuckets);
776 bytesFreed += bucketGroup.free(bucketBytesToFree);
777 }
778 remainingBuckets--;
779 }
780
781
782 if (bucketSizesAboveThresholdCount(minFactor) > 0) {
783 bucketQueue.clear();
784 remainingBuckets = 3;
785
786 bucketQueue.add(bucketSingle);
787 bucketQueue.add(bucketMulti);
788 bucketQueue.add(bucketMemory);
789
790 while ((bucketGroup = bucketQueue.poll()) != null) {
791 long bucketBytesToFree = (bytesToFreeWithExtra - bytesFreed) / remainingBuckets;
792 bytesFreed += bucketGroup.free(bucketBytesToFree);
793 remainingBuckets--;
794 }
795 }
796
797
798
799
800
801
802 freeEntireBuckets(DEFAULT_FREE_ENTIRE_BLOCK_FACTOR *
803 bucketSizesAboveThresholdCount(1.0f));
804
805 if (LOG.isDebugEnabled()) {
806 long single = bucketSingle.totalSize();
807 long multi = bucketMulti.totalSize();
808 long memory = bucketMemory.totalSize();
809 if (LOG.isDebugEnabled()) {
810 LOG.debug("Bucket cache free space completed; " + "freed="
811 + StringUtils.byteDesc(bytesFreed) + ", " + "total="
812 + StringUtils.byteDesc(totalSize) + ", " + "single="
813 + StringUtils.byteDesc(single) + ", " + "multi="
814 + StringUtils.byteDesc(multi) + ", " + "memory="
815 + StringUtils.byteDesc(memory));
816 }
817 }
818
819 } catch (Throwable t) {
820 LOG.warn("Failed freeing space", t);
821 } finally {
822 cacheStats.evict();
823 freeInProgress = false;
824 freeSpaceLock.unlock();
825 }
826 }
827
828
829 class WriterThread extends HasThread {
830 private final BlockingQueue<RAMQueueEntry> inputQueue;
831 private volatile boolean writerEnabled = true;
832
833 WriterThread(BlockingQueue<RAMQueueEntry> queue) {
834 super("BucketCacheWriterThread");
835 this.inputQueue = queue;
836 }
837
838
839 void disableWriter() {
840 this.writerEnabled = false;
841 }
842
843 @Override
844 public void run() {
845 List<RAMQueueEntry> entries = new ArrayList<RAMQueueEntry>();
846 try {
847 while (cacheEnabled && writerEnabled) {
848 try {
849 try {
850
851 entries = getRAMQueueEntries(inputQueue, entries);
852 } catch (InterruptedException ie) {
853 if (!cacheEnabled) break;
854 }
855 doDrain(entries);
856 } catch (Exception ioe) {
857 LOG.error("WriterThread encountered error", ioe);
858 }
859 }
860 } catch (Throwable t) {
861 LOG.warn("Failed doing drain", t);
862 }
863 LOG.info(this.getName() + " exiting, cacheEnabled=" + cacheEnabled);
864 }
865
866
867
868
869
870
871
872
873
874
875
876
877
878 private void putIntoBackingMap(BlockCacheKey key, BucketEntry bucketEntry) {
879 BucketEntry previousEntry = backingMap.put(key, bucketEntry);
880 if (previousEntry != null && previousEntry != bucketEntry) {
881 ReentrantReadWriteLock lock = offsetLock.getLock(previousEntry.offset());
882 lock.writeLock().lock();
883 try {
884 blockEvicted(key, previousEntry, false);
885 } finally {
886 lock.writeLock().unlock();
887 }
888 }
889 }
890
891
892
893
894
895
896
897
898
899 void doDrain(final List<RAMQueueEntry> entries) throws InterruptedException {
900 if (entries.isEmpty()) {
901 return;
902 }
903
904
905
906
907
908
909 final int size = entries.size();
910 BucketEntry[] bucketEntries = new BucketEntry[size];
911
912
913 int index = 0;
914 while (cacheEnabled && index < size) {
915 RAMQueueEntry re = null;
916 try {
917 re = entries.get(index);
918 if (re == null) {
919 LOG.warn("Couldn't get entry or changed on us; who else is messing with it?");
920 index++;
921 continue;
922 }
923
924 BlockCacheKey cacheKey = re.getKey();
925 if (ramCache.containsKey(cacheKey)) {
926 blocksByHFile.add(cacheKey);
927 }
928
929 BucketEntry bucketEntry =
930 re.writeToCache(ioEngine, bucketAllocator, deserialiserMap, realCacheSize);
931
932 bucketEntries[index] = bucketEntry;
933 if (ioErrorStartTime > 0) {
934 ioErrorStartTime = -1;
935 }
936 index++;
937 } catch (BucketAllocatorException fle) {
938 LOG.warn("Failed allocation for " + (re == null ? "" : re.getKey()) + "; " + fle);
939
940 bucketEntries[index] = null;
941 index++;
942 } catch (CacheFullException cfe) {
943
944 if (!freeInProgress) {
945 freeSpace("Full!");
946 } else {
947 Thread.sleep(50);
948 }
949 } catch (IOException ioex) {
950
951 LOG.error("Failed writing to bucket cache", ioex);
952 checkIOErrorIsTolerated();
953 }
954 }
955
956
957 try {
958 ioEngine.sync();
959 } catch (IOException ioex) {
960 LOG.error("Failed syncing IO engine", ioex);
961 checkIOErrorIsTolerated();
962
963 for (int i = 0; i < entries.size(); ++i) {
964 if (bucketEntries[i] != null) {
965 bucketAllocator.freeBlock(bucketEntries[i].offset());
966 bucketEntries[i] = null;
967 }
968 }
969 }
970
971
972
973 for (int i = 0; i < size; ++i) {
974 BlockCacheKey key = entries.get(i).getKey();
975
976 if (bucketEntries[i] != null) {
977 putIntoBackingMap(key, bucketEntries[i]);
978 }
979
980 RAMQueueEntry ramCacheEntry = ramCache.remove(key);
981 if (ramCacheEntry != null) {
982 heapSize.addAndGet(-1 * entries.get(i).getData().heapSize());
983 } else if (bucketEntries[i] != null){
984
985 ReentrantReadWriteLock lock = offsetLock.getLock(bucketEntries[i].offset());
986 try {
987 lock.writeLock().lock();
988 if (backingMap.remove(key, bucketEntries[i])) {
989 blockEvicted(key, bucketEntries[i], false);
990 }
991 } finally {
992 lock.writeLock().unlock();
993 }
994 }
995 }
996
997 long used = bucketAllocator.getUsedSize();
998 if (used > acceptableSize()) {
999 freeSpace("Used=" + used + " > acceptable=" + acceptableSize());
1000 }
1001 return;
1002 }
1003 }
1004
1005
1006
1007
1008
1009
1010
1011
1012
1013 static List<RAMQueueEntry> getRAMQueueEntries(final BlockingQueue<RAMQueueEntry> q,
1014 final List<RAMQueueEntry> receptical)
1015 throws InterruptedException {
1016
1017
1018 receptical.clear();
1019 receptical.add(q.take());
1020 q.drainTo(receptical);
1021 return receptical;
1022 }
1023
1024 private void persistToFile() throws IOException {
1025 assert !cacheEnabled;
1026 try (ObjectOutputStream oos = new ObjectOutputStream(
1027 new FileOutputStream(persistencePath, false))){
1028 if (!ioEngine.isPersistent()) {
1029 throw new IOException("Attempt to persist non-persistent cache mappings!");
1030 }
1031 byte[] checksum = ((PersistentIOEngine) ioEngine).calculateChecksum(algorithm);
1032 if (checksum != null) {
1033 oos.write(ProtobufUtil.PB_MAGIC);
1034 oos.writeInt(checksum.length);
1035 oos.write(checksum);
1036 }
1037 oos.writeLong(cacheCapacity);
1038 oos.writeUTF(ioEngine.getClass().getName());
1039 oos.writeUTF(backingMap.getClass().getName());
1040 oos.writeObject(deserialiserMap);
1041 oos.writeObject(backingMap);
1042 }
1043 }
1044
1045 @SuppressWarnings("unchecked")
1046 private void retrieveFromFile(int[] bucketSizes) throws IOException,
1047 ClassNotFoundException {
1048 File persistenceFile = new File(persistencePath);
1049 if (!persistenceFile.exists()) {
1050 return;
1051 }
1052 assert !cacheEnabled;
1053 ObjectInputStream ois = null;
1054 try {
1055 if (!ioEngine.isPersistent())
1056 throw new IOException(
1057 "Attempt to restore non-persistent cache mappings!");
1058 ois = new ObjectInputStream(new FileInputStream(persistencePath));
1059 int pblen = ProtobufUtil.lengthOfPBMagic();
1060 byte[] pbuf = new byte[pblen];
1061 int read = ois.read(pbuf);
1062 if (read != pblen) {
1063 LOG.warn("Can't restore from file because of incorrect number of bytes read while " +
1064 "checking for protobuf magic number. Requested=" + pblen + ", but received= " +
1065 read + ".");
1066 return;
1067 }
1068 if (Bytes.equals(ProtobufUtil.PB_MAGIC, pbuf)) {
1069 int length = ois.readInt();
1070 byte[] persistentChecksum = new byte[length];
1071 int readLen = ois.read(persistentChecksum);
1072 if (readLen != length) {
1073 LOG.warn("Can't restore from file because of incorrect number of bytes read while " +
1074 "checking for persistent checksum. Requested=" + length + ", but received=" +
1075 readLen + ". ");
1076 return;
1077 }
1078 if (!((PersistentIOEngine) ioEngine).verifyFileIntegrity(
1079 persistentChecksum, algorithm)) {
1080 LOG.warn("Can't restore from file because of verification failed.");
1081 return;
1082 }
1083 } else {
1084
1085
1086 ois.close();
1087 ois = new ObjectInputStream(new FileInputStream(persistencePath));
1088 }
1089 long capacitySize = ois.readLong();
1090 if (capacitySize != cacheCapacity)
1091 throw new IOException("Mismatched cache capacity:"
1092 + StringUtils.byteDesc(capacitySize) + ", expected: "
1093 + StringUtils.byteDesc(cacheCapacity));
1094 String ioclass = ois.readUTF();
1095 String mapclass = ois.readUTF();
1096 if (!ioEngine.getClass().getName().equals(ioclass))
1097 throw new IOException("Class name for IO engine mismatch: " + ioclass
1098 + ", expected:" + ioEngine.getClass().getName());
1099 if (!backingMap.getClass().getName().equals(mapclass))
1100 throw new IOException("Class name for cache map mismatch: " + mapclass
1101 + ", expected:" + backingMap.getClass().getName());
1102 UniqueIndexMap<Integer> deserMap = (UniqueIndexMap<Integer>) ois
1103 .readObject();
1104 ConcurrentHashMap<BlockCacheKey, BucketEntry> backingMapFromFile =
1105 (ConcurrentHashMap<BlockCacheKey, BucketEntry>) ois.readObject();
1106 BucketAllocator allocator = new BucketAllocator(cacheCapacity, bucketSizes,
1107 backingMapFromFile, realCacheSize);
1108 bucketAllocator = allocator;
1109 deserialiserMap = deserMap;
1110 backingMap = backingMapFromFile;
1111 blockNumber.set(backingMap.size());
1112 } finally {
1113 if (ois != null) {
1114 ois.close();
1115 }
1116 if (!persistenceFile.delete()) {
1117 throw new IOException("Failed deleting persistence file "
1118 + persistenceFile.getAbsolutePath());
1119 }
1120 }
1121 }
1122
1123
1124
1125
1126
1127
1128 private void checkIOErrorIsTolerated() {
1129 long now = EnvironmentEdgeManager.currentTime();
1130
1131 long ioErrorStartTimeTmp = this.ioErrorStartTime;
1132 if (ioErrorStartTimeTmp > 0) {
1133 if (cacheEnabled && (now - ioErrorStartTimeTmp) > this.ioErrorsTolerationDuration) {
1134 LOG.error("IO errors duration time has exceeded " + ioErrorsTolerationDuration +
1135 "ms, disabling cache, please check your IOEngine");
1136 disableCache();
1137 }
1138 } else {
1139 this.ioErrorStartTime = now;
1140 }
1141 }
1142
1143
1144
1145
1146 private void disableCache() {
1147 if (!cacheEnabled) return;
1148 cacheEnabled = false;
1149 ioEngine.shutdown();
1150 this.scheduleThreadPool.shutdown();
1151 for (int i = 0; i < writerThreads.length; ++i) writerThreads[i].interrupt();
1152 this.ramCache.clear();
1153 if (!ioEngine.isPersistent() || persistencePath == null) {
1154
1155 this.backingMap.clear();
1156 }
1157 }
1158
1159 private void join() throws InterruptedException {
1160 for (int i = 0; i < writerThreads.length; ++i)
1161 writerThreads[i].join();
1162 }
1163
1164 @Override
1165 public void shutdown() {
1166 disableCache();
1167 LOG.info("Shutdown bucket cache: IO persistent=" + ioEngine.isPersistent()
1168 + "; path to write=" + persistencePath);
1169 if (ioEngine.isPersistent() && persistencePath != null) {
1170 try {
1171 join();
1172 persistToFile();
1173 } catch (IOException ex) {
1174 LOG.error("Unable to persist data on exit: " + ex.toString(), ex);
1175 } catch (InterruptedException e) {
1176 LOG.warn("Failed to persist data on exit", e);
1177 }
1178 }
1179 }
1180
1181 @Override
1182 public CacheStats getStats() {
1183 return cacheStats;
1184 }
1185
1186 public BucketAllocator getAllocator() {
1187 return this.bucketAllocator;
1188 }
1189
1190 @Override
1191 public long heapSize() {
1192 return this.heapSize.get();
1193 }
1194
1195 @Override
1196 public long size() {
1197 return this.realCacheSize.get();
1198 }
1199
1200 @Override
1201 public long getCurrentDataSize() {
1202 return size();
1203 }
1204
1205 @Override
1206 public long getFreeSize() {
1207 return this.bucketAllocator.getFreeSize();
1208 }
1209
1210 @Override
1211 public long getBlockCount() {
1212 return this.blockNumber.get();
1213 }
1214
1215 @Override
1216 public long getDataBlockCount() {
1217 return getBlockCount();
1218 }
1219
1220 @Override
1221 public long getCurrentSize() {
1222 return this.bucketAllocator.getUsedSize();
1223 }
1224
1225
1226
1227
1228
1229
1230
1231
1232 @Override
1233 public int evictBlocksByHfileName(String hfileName) {
1234 Set<BlockCacheKey> keySet = blocksByHFile.subSet(
1235 new BlockCacheKey(hfileName, Long.MIN_VALUE), true,
1236 new BlockCacheKey(hfileName, Long.MAX_VALUE), true);
1237
1238 int numEvicted = 0;
1239 for (BlockCacheKey key : keySet) {
1240 if (evictBlock(key)) {
1241 ++numEvicted;
1242 }
1243 }
1244
1245 return numEvicted;
1246 }
1247
1248
1249
1250
1251
1252
1253
1254
1255
1256 static class BucketEntry implements Serializable {
1257 private static final long serialVersionUID = -6741504807982257534L;
1258
1259
1260 static final Comparator<BucketEntry> COMPARATOR = new Comparator<BucketCache.BucketEntry>() {
1261
1262 @Override
1263 public int compare(BucketEntry o1, BucketEntry o2) {
1264 return Long.compare(o2.accessCounter, o1.accessCounter);
1265 }
1266 };
1267
1268 private int offsetBase;
1269 private int length;
1270 private byte offset1;
1271 byte deserialiserIndex;
1272 private volatile long accessCounter;
1273 private BlockPriority priority;
1274
1275
1276
1277 private final long cachedTime = System.nanoTime();
1278
1279 BucketEntry(long offset, int length, long accessCounter, boolean inMemory) {
1280 setOffset(offset);
1281 this.length = length;
1282 this.accessCounter = accessCounter;
1283 if (inMemory) {
1284 this.priority = BlockPriority.MEMORY;
1285 } else {
1286 this.priority = BlockPriority.SINGLE;
1287 }
1288 }
1289
1290 long offset() {
1291 long o = ((long) offsetBase) & 0xFFFFFFFFL;
1292 o += (((long) (offset1)) & 0xFF) << 32;
1293 return o << 8;
1294 }
1295
1296 private void setOffset(long value) {
1297 assert (value & 0xFF) == 0;
1298 value >>= 8;
1299 offsetBase = (int) value;
1300 offset1 = (byte) (value >> 32);
1301 }
1302
1303 public int getLength() {
1304 return length;
1305 }
1306
1307 protected CacheableDeserializer<Cacheable> deserializerReference(
1308 UniqueIndexMap<Integer> deserialiserMap) {
1309 return CacheableDeserializerIdManager.getDeserializer(deserialiserMap
1310 .unmap(deserialiserIndex));
1311 }
1312
1313 protected void setDeserialiserReference(
1314 CacheableDeserializer<Cacheable> deserializer,
1315 UniqueIndexMap<Integer> deserialiserMap) {
1316 this.deserialiserIndex = ((byte) deserialiserMap.map(deserializer
1317 .getDeserialiserIdentifier()));
1318 }
1319
1320
1321
1322
1323 public void access(long accessCounter) {
1324 this.accessCounter = accessCounter;
1325 if (this.priority == BlockPriority.SINGLE) {
1326 this.priority = BlockPriority.MULTI;
1327 }
1328 }
1329
1330 public BlockPriority getPriority() {
1331 return this.priority;
1332 }
1333
1334 public long getCachedTime() {
1335 return cachedTime;
1336 }
1337 }
1338
1339
1340
1341
1342
1343
1344
1345 private class BucketEntryGroup implements Comparable<BucketEntryGroup> {
1346
1347 private CachedEntryQueue queue;
1348 private long totalSize = 0;
1349 private long bucketSize;
1350
1351 public BucketEntryGroup(long bytesToFree, long blockSize, long bucketSize) {
1352 this.bucketSize = bucketSize;
1353 queue = new CachedEntryQueue(bytesToFree, blockSize);
1354 totalSize = 0;
1355 }
1356
1357 public void add(Map.Entry<BlockCacheKey, BucketEntry> block) {
1358 totalSize += block.getValue().getLength();
1359 queue.add(block);
1360 }
1361
1362 public long free(long toFree) {
1363 Map.Entry<BlockCacheKey, BucketEntry> entry;
1364 long freedBytes = 0;
1365 while ((entry = queue.pollLast()) != null) {
1366 evictBlock(entry.getKey());
1367 freedBytes += entry.getValue().getLength();
1368 if (freedBytes >= toFree) {
1369 return freedBytes;
1370 }
1371 }
1372 return freedBytes;
1373 }
1374
1375 public long overflow() {
1376 return totalSize - bucketSize;
1377 }
1378
1379 public long totalSize() {
1380 return totalSize;
1381 }
1382
1383 @Override
1384 public int compareTo(BucketEntryGroup that) {
1385 return Long.compare(this.overflow(), that.overflow());
1386 }
1387
1388 @Override
1389 public int hashCode() {
1390 final int prime = 31;
1391 int result = 1;
1392 result = prime * result + getOuterType().hashCode();
1393 result = prime * result + (int) (bucketSize ^ (bucketSize >>> 32));
1394 result = prime * result + ((queue == null) ? 0 : queue.hashCode());
1395 result = prime * result + (int) (totalSize ^ (totalSize >>> 32));
1396 return result;
1397 }
1398
1399 @Override
1400 public boolean equals(Object obj) {
1401 if (this == obj) {
1402 return true;
1403 }
1404 if (obj == null) {
1405 return false;
1406 }
1407 if (getClass() != obj.getClass()) {
1408 return false;
1409 }
1410 BucketEntryGroup other = (BucketEntryGroup) obj;
1411 if (!getOuterType().equals(other.getOuterType())) {
1412 return false;
1413 }
1414 if (bucketSize != other.bucketSize) {
1415 return false;
1416 }
1417 if (queue == null) {
1418 if (other.queue != null) {
1419 return false;
1420 }
1421 } else if (!queue.equals(other.queue)) {
1422 return false;
1423 }
1424 if (totalSize != other.totalSize) {
1425 return false;
1426 }
1427 return true;
1428 }
1429
1430 private BucketCache getOuterType() {
1431 return BucketCache.this;
1432 }
1433
1434 }
1435
1436
1437
1438
1439 static class RAMQueueEntry {
1440 private BlockCacheKey key;
1441 private Cacheable data;
1442 private long accessCounter;
1443 private boolean inMemory;
1444
1445 public RAMQueueEntry(BlockCacheKey bck, Cacheable data, long accessCounter,
1446 boolean inMemory) {
1447 this.key = bck;
1448 this.data = data;
1449 this.accessCounter = accessCounter;
1450 this.inMemory = inMemory;
1451 }
1452
1453 public Cacheable getData() {
1454 return data;
1455 }
1456
1457 public BlockCacheKey getKey() {
1458 return key;
1459 }
1460
1461 public void access(long accessCounter) {
1462 this.accessCounter = accessCounter;
1463 }
1464
1465 public BucketEntry writeToCache(final IOEngine ioEngine, final BucketAllocator bucketAllocator,
1466 final UniqueIndexMap<Integer> deserialiserMap, final AtomicLong realCacheSize)
1467 throws IOException {
1468 int len = data.getSerializedLength();
1469
1470 if (len == 0) {
1471 return null;
1472 }
1473 long offset = bucketAllocator.allocateBlock(len);
1474 BucketEntry bucketEntry;
1475 boolean succ = false;
1476 try {
1477 bucketEntry = new BucketEntry(offset, len, accessCounter, inMemory);
1478 bucketEntry.setDeserialiserReference(data.getDeserializer(), deserialiserMap);
1479 if (data instanceof HFileBlock) {
1480
1481 HFileBlock block = (HFileBlock) data;
1482 ByteBuffer sliceBuf = block.getBufferReadOnly();
1483 ByteBuffer metadata = block.getMetaData();
1484 ioEngine.write(sliceBuf, offset);
1485 ioEngine.write(metadata, offset + len - metadata.limit());
1486 } else {
1487 ByteBuffer bb = ByteBuffer.allocate(len);
1488 data.serialize(bb, true);
1489 ioEngine.write(bb, offset);
1490 }
1491 succ = true;
1492 } finally {
1493 if (!succ) {
1494 bucketAllocator.freeBlock(offset);
1495 }
1496 }
1497 realCacheSize.addAndGet(len);
1498 return bucketEntry;
1499 }
1500 }
1501
1502
1503
1504
1505
1506 void stopWriterThreads() throws InterruptedException {
1507 for (WriterThread writerThread : writerThreads) {
1508 writerThread.disableWriter();
1509 writerThread.interrupt();
1510 writerThread.join();
1511 }
1512 }
1513
1514 @Override
1515 public Iterator<CachedBlock> iterator() {
1516
1517 final Iterator<Map.Entry<BlockCacheKey, BucketEntry>> i =
1518 this.backingMap.entrySet().iterator();
1519 return new Iterator<CachedBlock>() {
1520 private final long now = System.nanoTime();
1521
1522 @Override
1523 public boolean hasNext() {
1524 return i.hasNext();
1525 }
1526
1527 @Override
1528 public CachedBlock next() {
1529 final Map.Entry<BlockCacheKey, BucketEntry> e = i.next();
1530 return new CachedBlock() {
1531 @Override
1532 public String toString() {
1533 return BlockCacheUtil.toString(this, now);
1534 }
1535
1536 @Override
1537 public BlockPriority getBlockPriority() {
1538 return e.getValue().getPriority();
1539 }
1540
1541 @Override
1542 public BlockType getBlockType() {
1543
1544 return null;
1545 }
1546
1547 @Override
1548 public long getOffset() {
1549 return e.getKey().getOffset();
1550 }
1551
1552 @Override
1553 public long getSize() {
1554 return e.getValue().getLength();
1555 }
1556
1557 @Override
1558 public long getCachedTime() {
1559 return e.getValue().getCachedTime();
1560 }
1561
1562 @Override
1563 public String getFilename() {
1564 return e.getKey().getHfileName();
1565 }
1566
1567 @Override
1568 public int compareTo(CachedBlock other) {
1569 int diff = this.getFilename().compareTo(other.getFilename());
1570 if (diff != 0) return diff;
1571
1572 diff = Long.compare(this.getOffset(), other.getOffset());
1573 if (diff != 0) return diff;
1574 if (other.getCachedTime() < 0 || this.getCachedTime() < 0) {
1575 throw new IllegalStateException("" + this.getCachedTime() + ", " +
1576 other.getCachedTime());
1577 }
1578 return Long.compare(other.getCachedTime(), this.getCachedTime());
1579 }
1580
1581 @Override
1582 public int hashCode() {
1583 return e.getKey().hashCode();
1584 }
1585
1586 @Override
1587 public boolean equals(Object obj) {
1588 if (obj instanceof CachedBlock) {
1589 CachedBlock cb = (CachedBlock)obj;
1590 return compareTo(cb) == 0;
1591 } else {
1592 return false;
1593 }
1594 }
1595 };
1596 }
1597
1598 @Override
1599 public void remove() {
1600 throw new UnsupportedOperationException();
1601 }
1602 };
1603 }
1604
1605 @Override
1606 public BlockCache[] getBlockCaches() {
1607 return null;
1608 }
1609
1610 float getAcceptableFactor() {
1611 return acceptableFactor;
1612 }
1613
1614 float getMinFactor() {
1615 return minFactor;
1616 }
1617
1618 float getExtraFreeFactor() {
1619 return extraFreeFactor;
1620 }
1621
1622 float getSingleFactor() {
1623 return singleFactor;
1624 }
1625
1626 float getMultiFactor() {
1627 return multiFactor;
1628 }
1629
1630 float getMemoryFactor() {
1631 return memoryFactor;
1632 }
1633
1634 public UniqueIndexMap<Integer> getDeserialiserMap() {
1635 return deserialiserMap;
1636 }
1637 }