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 blocksByHFile.add(cacheKey);
471 }
472 }
473
474
475
476
477
478
479
480
481
482 @Override
483 public Cacheable getBlock(BlockCacheKey key, boolean caching, boolean repeat,
484 boolean updateCacheMetrics) {
485 if (!cacheEnabled) {
486 return null;
487 }
488 RAMQueueEntry re = ramCache.get(key);
489 if (re != null) {
490 if (updateCacheMetrics) {
491 cacheStats.hit(caching, key.isPrimary(), key.getBlockType());
492 }
493 re.access(accessCount.incrementAndGet());
494 return re.getData();
495 }
496 BucketEntry bucketEntry = backingMap.get(key);
497 if (bucketEntry != null) {
498 long start = System.nanoTime();
499 ReentrantReadWriteLock lock = offsetLock.getLock(bucketEntry.offset());
500 try {
501 lock.readLock().lock();
502
503
504
505 if (bucketEntry.equals(backingMap.get(key))) {
506 int len = bucketEntry.getLength();
507 if (LOG.isTraceEnabled()) {
508 LOG.trace("Read offset=" + bucketEntry.offset() + ", len=" + len);
509 }
510 ByteBuffer bb = ByteBuffer.allocate(len);
511 int lenRead = ioEngine.read(bb, bucketEntry.offset());
512 if (lenRead != len) {
513 throw new RuntimeException("Only " + lenRead + " bytes read, " + len + " expected");
514 }
515 CacheableDeserializer<Cacheable> deserializer =
516 bucketEntry.deserializerReference(this.deserialiserMap);
517 Cacheable cachedBlock = deserializer.deserialize(bb, true);
518 long timeTaken = System.nanoTime() - start;
519 if (updateCacheMetrics) {
520 cacheStats.hit(caching, key.isPrimary(), key.getBlockType());
521 cacheStats.ioHit(timeTaken);
522 }
523 bucketEntry.access(accessCount.incrementAndGet());
524 if (this.ioErrorStartTime > 0) {
525 ioErrorStartTime = -1;
526 }
527 return cachedBlock;
528 }
529 } catch (IOException ioex) {
530 LOG.error("Failed reading block " + key + " from bucket cache", ioex);
531 checkIOErrorIsTolerated();
532 } finally {
533 lock.readLock().unlock();
534 }
535 }
536 if (!repeat && updateCacheMetrics) {
537 cacheStats.miss(caching, key.isPrimary(), key.getBlockType());
538 }
539 return null;
540 }
541
542 void blockEvicted(BlockCacheKey cacheKey, BucketEntry bucketEntry, boolean decrementBlockNumber) {
543 bucketAllocator.freeBlock(bucketEntry.offset());
544 realCacheSize.addAndGet(-1 * bucketEntry.getLength());
545 blocksByHFile.remove(cacheKey);
546 if (decrementBlockNumber) {
547 this.blockNumber.decrementAndGet();
548 }
549 }
550
551 @Override
552 public boolean evictBlock(BlockCacheKey cacheKey) {
553 if (!cacheEnabled) {
554 return false;
555 }
556 RAMQueueEntry removedBlock = ramCache.remove(cacheKey);
557 if (removedBlock != null) {
558 this.blockNumber.decrementAndGet();
559 this.heapSize.addAndGet(-1 * removedBlock.getData().heapSize());
560 }
561 BucketEntry bucketEntry = backingMap.get(cacheKey);
562 if (bucketEntry == null) {
563 if (removedBlock != null) {
564 cacheStats.evicted(0, cacheKey.isPrimary());
565 return true;
566 } else {
567 return false;
568 }
569 }
570 ReentrantReadWriteLock lock = offsetLock.getLock(bucketEntry.offset());
571 try {
572 lock.writeLock().lock();
573 if (backingMap.remove(cacheKey, bucketEntry)) {
574 blockEvicted(cacheKey, bucketEntry, removedBlock == null);
575 } else {
576 return false;
577 }
578 } finally {
579 lock.writeLock().unlock();
580 }
581 cacheStats.evicted(bucketEntry.getCachedTime(), cacheKey.isPrimary());
582 return true;
583 }
584
585
586
587
588 private static class StatisticsThread extends Thread {
589 private final BucketCache bucketCache;
590
591 public StatisticsThread(BucketCache bucketCache) {
592 super("BucketCacheStatsThread");
593 setDaemon(true);
594 this.bucketCache = bucketCache;
595 }
596
597 @Override
598 public void run() {
599 bucketCache.logStats();
600 }
601 }
602
603 public void logStats() {
604 long totalSize = bucketAllocator.getTotalSize();
605 long usedSize = bucketAllocator.getUsedSize();
606 long freeSize = totalSize - usedSize;
607 long cacheSize = getRealCacheSize();
608 LOG.info("failedBlockAdditions=" + cacheStats.getFailedInserts() + ", " +
609 "totalSize=" + StringUtils.byteDesc(totalSize) + ", " +
610 "freeSize=" + StringUtils.byteDesc(freeSize) + ", " +
611 "usedSize=" + StringUtils.byteDesc(usedSize) +", " +
612 "cacheSize=" + StringUtils.byteDesc(cacheSize) +", " +
613 "accesses=" + cacheStats.getRequestCount() + ", " +
614 "hits=" + cacheStats.getHitCount() + ", " +
615 "IOhitsPerSecond=" + cacheStats.getIOHitsPerSecond() + ", " +
616 "IOTimePerHit=" + String.format("%.2f", cacheStats.getIOTimePerHit())+ ", " +
617 "hitRatio=" + (cacheStats.getHitCount() == 0 ? "0," :
618 (StringUtils.formatPercent(cacheStats.getHitRatio(), 2)+ ", ")) +
619 "cachingAccesses=" + cacheStats.getRequestCachingCount() + ", " +
620 "cachingHits=" + cacheStats.getHitCachingCount() + ", " +
621 "cachingHitsRatio=" +(cacheStats.getHitCachingCount() == 0 ? "0," :
622 (StringUtils.formatPercent(cacheStats.getHitCachingRatio(), 2)+ ", ")) +
623 "evictions=" + cacheStats.getEvictionCount() + ", " +
624 "evicted=" + cacheStats.getEvictedCount() + ", " +
625 "evictedPerRun=" + cacheStats.evictedPerEviction());
626 cacheStats.reset();
627 }
628
629 public long getRealCacheSize() {
630 return this.realCacheSize.get();
631 }
632
633 private long acceptableSize() {
634 return (long) Math.floor(bucketAllocator.getTotalSize() * acceptableFactor);
635 }
636
637 long getPartitionSize(float partitionFactor) {
638 return (long) Math.floor(bucketAllocator.getTotalSize() * partitionFactor * minFactor);
639 }
640
641
642
643
644 private int bucketSizesAboveThresholdCount(float minFactor) {
645 BucketAllocator.IndexStatistics[] stats = bucketAllocator.getIndexStatistics();
646 int fullCount = 0;
647 for (int i = 0; i < stats.length; i++) {
648 long freeGoal = (long) Math.floor(stats[i].totalCount() * (1 - minFactor));
649 freeGoal = Math.max(freeGoal, 1);
650 if (stats[i].freeCount() < freeGoal) {
651 fullCount++;
652 }
653 }
654 return fullCount;
655 }
656
657
658
659
660
661
662
663
664
665
666 private void freeEntireBuckets(int completelyFreeBucketsNeeded) {
667 if (completelyFreeBucketsNeeded != 0) {
668
669
670 Set<Integer> inUseBuckets = new HashSet<Integer>();
671 for (BucketEntry entry : backingMap.values()) {
672 inUseBuckets.add(bucketAllocator.getBucketIndex(entry.offset()));
673 }
674
675 Set<Integer> candidateBuckets = bucketAllocator.getLeastFilledBuckets(
676 inUseBuckets, completelyFreeBucketsNeeded);
677 for (Map.Entry<BlockCacheKey, BucketEntry> entry : backingMap.entrySet()) {
678 if (candidateBuckets.contains(bucketAllocator
679 .getBucketIndex(entry.getValue().offset()))) {
680 evictBlock(entry.getKey());
681 }
682 }
683 }
684 }
685
686
687
688
689
690
691
692 private void freeSpace(final String why) {
693
694 if (!freeSpaceLock.tryLock()) {
695 return;
696 }
697 try {
698 freeInProgress = true;
699 long bytesToFreeWithoutExtra = 0;
700
701 StringBuffer msgBuffer = LOG.isDebugEnabled()? new StringBuffer(): null;
702 BucketAllocator.IndexStatistics[] stats = bucketAllocator.getIndexStatistics();
703 long[] bytesToFreeForBucket = new long[stats.length];
704 for (int i = 0; i < stats.length; i++) {
705 bytesToFreeForBucket[i] = 0;
706 long freeGoal = (long) Math.floor(stats[i].totalCount() * (1 - minFactor));
707 freeGoal = Math.max(freeGoal, 1);
708 if (stats[i].freeCount() < freeGoal) {
709 bytesToFreeForBucket[i] = stats[i].itemSize() * (freeGoal - stats[i].freeCount());
710 bytesToFreeWithoutExtra += bytesToFreeForBucket[i];
711 if (msgBuffer != null) {
712 msgBuffer.append("Free for bucketSize(" + stats[i].itemSize() + ")="
713 + StringUtils.byteDesc(bytesToFreeForBucket[i]) + ", ");
714 }
715 }
716 }
717 if (msgBuffer != null) {
718 msgBuffer.append("Free for total=" + StringUtils.byteDesc(bytesToFreeWithoutExtra) + ", ");
719 }
720
721 if (bytesToFreeWithoutExtra <= 0) {
722 return;
723 }
724 long currentSize = bucketAllocator.getUsedSize();
725 long totalSize = bucketAllocator.getTotalSize();
726 if (LOG.isDebugEnabled() && msgBuffer != null) {
727 LOG.debug("Free started because \"" + why + "\"; " + msgBuffer.toString() +
728 " of current used=" + StringUtils.byteDesc(currentSize) + ", actual cacheSize=" +
729 StringUtils.byteDesc(realCacheSize.get()) + ", total=" + StringUtils.byteDesc(totalSize));
730 }
731
732 long bytesToFreeWithExtra = (long) Math.floor(bytesToFreeWithoutExtra
733 * (1 + extraFreeFactor));
734
735
736 BucketEntryGroup bucketSingle = new BucketEntryGroup(bytesToFreeWithExtra,
737 blockSize, getPartitionSize(singleFactor));
738 BucketEntryGroup bucketMulti = new BucketEntryGroup(bytesToFreeWithExtra,
739 blockSize, getPartitionSize(multiFactor));
740 BucketEntryGroup bucketMemory = new BucketEntryGroup(bytesToFreeWithExtra,
741 blockSize, getPartitionSize(memoryFactor));
742
743
744
745 for (Map.Entry<BlockCacheKey, BucketEntry> bucketEntryWithKey : backingMap.entrySet()) {
746 switch (bucketEntryWithKey.getValue().getPriority()) {
747 case SINGLE: {
748 bucketSingle.add(bucketEntryWithKey);
749 break;
750 }
751 case MULTI: {
752 bucketMulti.add(bucketEntryWithKey);
753 break;
754 }
755 case MEMORY: {
756 bucketMemory.add(bucketEntryWithKey);
757 break;
758 }
759 }
760 }
761
762 PriorityQueue<BucketEntryGroup> bucketQueue = new PriorityQueue<BucketEntryGroup>(3);
763
764 bucketQueue.add(bucketSingle);
765 bucketQueue.add(bucketMulti);
766 bucketQueue.add(bucketMemory);
767
768 int remainingBuckets = bucketQueue.size();
769 long bytesFreed = 0;
770
771 BucketEntryGroup bucketGroup;
772 while ((bucketGroup = bucketQueue.poll()) != null) {
773 long overflow = bucketGroup.overflow();
774 if (overflow > 0) {
775 long bucketBytesToFree = Math.min(overflow,
776 (bytesToFreeWithoutExtra - bytesFreed) / remainingBuckets);
777 bytesFreed += bucketGroup.free(bucketBytesToFree);
778 }
779 remainingBuckets--;
780 }
781
782
783 if (bucketSizesAboveThresholdCount(minFactor) > 0) {
784 bucketQueue.clear();
785 remainingBuckets = 3;
786
787 bucketQueue.add(bucketSingle);
788 bucketQueue.add(bucketMulti);
789 bucketQueue.add(bucketMemory);
790
791 while ((bucketGroup = bucketQueue.poll()) != null) {
792 long bucketBytesToFree = (bytesToFreeWithExtra - bytesFreed) / remainingBuckets;
793 bytesFreed += bucketGroup.free(bucketBytesToFree);
794 remainingBuckets--;
795 }
796 }
797
798
799
800
801
802
803 freeEntireBuckets(DEFAULT_FREE_ENTIRE_BLOCK_FACTOR *
804 bucketSizesAboveThresholdCount(1.0f));
805
806 if (LOG.isDebugEnabled()) {
807 long single = bucketSingle.totalSize();
808 long multi = bucketMulti.totalSize();
809 long memory = bucketMemory.totalSize();
810 if (LOG.isDebugEnabled()) {
811 LOG.debug("Bucket cache free space completed; " + "freed="
812 + StringUtils.byteDesc(bytesFreed) + ", " + "total="
813 + StringUtils.byteDesc(totalSize) + ", " + "single="
814 + StringUtils.byteDesc(single) + ", " + "multi="
815 + StringUtils.byteDesc(multi) + ", " + "memory="
816 + StringUtils.byteDesc(memory));
817 }
818 }
819
820 } catch (Throwable t) {
821 LOG.warn("Failed freeing space", t);
822 } finally {
823 cacheStats.evict();
824 freeInProgress = false;
825 freeSpaceLock.unlock();
826 }
827 }
828
829
830 class WriterThread extends HasThread {
831 private final BlockingQueue<RAMQueueEntry> inputQueue;
832 private volatile boolean writerEnabled = true;
833
834 WriterThread(BlockingQueue<RAMQueueEntry> queue) {
835 super("BucketCacheWriterThread");
836 this.inputQueue = queue;
837 }
838
839
840 void disableWriter() {
841 this.writerEnabled = false;
842 }
843
844 @Override
845 public void run() {
846 List<RAMQueueEntry> entries = new ArrayList<RAMQueueEntry>();
847 try {
848 while (cacheEnabled && writerEnabled) {
849 try {
850 try {
851
852 entries = getRAMQueueEntries(inputQueue, entries);
853 } catch (InterruptedException ie) {
854 if (!cacheEnabled) break;
855 }
856 doDrain(entries);
857 } catch (Exception ioe) {
858 LOG.error("WriterThread encountered error", ioe);
859 }
860 }
861 } catch (Throwable t) {
862 LOG.warn("Failed doing drain", t);
863 }
864 LOG.info(this.getName() + " exiting, cacheEnabled=" + cacheEnabled);
865 }
866
867
868
869
870
871
872
873
874
875
876
877
878
879 private void putIntoBackingMap(BlockCacheKey key, BucketEntry bucketEntry) {
880 BucketEntry previousEntry = backingMap.put(key, bucketEntry);
881 if (previousEntry != null && previousEntry != bucketEntry) {
882 ReentrantReadWriteLock lock = offsetLock.getLock(previousEntry.offset());
883 lock.writeLock().lock();
884 try {
885 blockEvicted(key, previousEntry, false);
886 } finally {
887 lock.writeLock().unlock();
888 }
889 }
890 }
891
892
893
894
895
896
897
898
899
900 void doDrain(final List<RAMQueueEntry> entries) throws InterruptedException {
901 if (entries.isEmpty()) {
902 return;
903 }
904
905
906
907
908
909
910 final int size = entries.size();
911 BucketEntry[] bucketEntries = new BucketEntry[size];
912
913
914 int index = 0;
915 while (cacheEnabled && index < size) {
916 RAMQueueEntry re = null;
917 try {
918 re = entries.get(index);
919 if (re == null) {
920 LOG.warn("Couldn't get entry or changed on us; who else is messing with it?");
921 index++;
922 continue;
923 }
924 BucketEntry bucketEntry =
925 re.writeToCache(ioEngine, bucketAllocator, deserialiserMap, realCacheSize);
926
927 bucketEntries[index] = bucketEntry;
928 if (ioErrorStartTime > 0) {
929 ioErrorStartTime = -1;
930 }
931 index++;
932 } catch (BucketAllocatorException fle) {
933 LOG.warn("Failed allocation for " + (re == null ? "" : re.getKey()) + "; " + fle);
934
935 bucketEntries[index] = null;
936 index++;
937 } catch (CacheFullException cfe) {
938
939 if (!freeInProgress) {
940 freeSpace("Full!");
941 } else {
942 Thread.sleep(50);
943 }
944 } catch (IOException ioex) {
945
946 LOG.error("Failed writing to bucket cache", ioex);
947 checkIOErrorIsTolerated();
948 }
949 }
950
951
952 try {
953 ioEngine.sync();
954 } catch (IOException ioex) {
955 LOG.error("Failed syncing IO engine", ioex);
956 checkIOErrorIsTolerated();
957
958 for (int i = 0; i < entries.size(); ++i) {
959 if (bucketEntries[i] != null) {
960 bucketAllocator.freeBlock(bucketEntries[i].offset());
961 bucketEntries[i] = null;
962 }
963 }
964 }
965
966
967
968 for (int i = 0; i < size; ++i) {
969 BlockCacheKey key = entries.get(i).getKey();
970
971 if (bucketEntries[i] != null) {
972 putIntoBackingMap(key, bucketEntries[i]);
973 }
974
975 RAMQueueEntry ramCacheEntry = ramCache.remove(key);
976 if (ramCacheEntry != null) {
977 heapSize.addAndGet(-1 * entries.get(i).getData().heapSize());
978 } else if (bucketEntries[i] != null){
979
980 ReentrantReadWriteLock lock = offsetLock.getLock(bucketEntries[i].offset());
981 try {
982 lock.writeLock().lock();
983 if (backingMap.remove(key, bucketEntries[i])) {
984 blockEvicted(key, bucketEntries[i], false);
985 }
986 } finally {
987 lock.writeLock().unlock();
988 }
989 }
990 }
991
992 long used = bucketAllocator.getUsedSize();
993 if (used > acceptableSize()) {
994 freeSpace("Used=" + used + " > acceptable=" + acceptableSize());
995 }
996 return;
997 }
998 }
999
1000
1001
1002
1003
1004
1005
1006
1007
1008 static List<RAMQueueEntry> getRAMQueueEntries(final BlockingQueue<RAMQueueEntry> q,
1009 final List<RAMQueueEntry> receptical)
1010 throws InterruptedException {
1011
1012
1013 receptical.clear();
1014 receptical.add(q.take());
1015 q.drainTo(receptical);
1016 return receptical;
1017 }
1018
1019 private void persistToFile() throws IOException {
1020 assert !cacheEnabled;
1021 try (ObjectOutputStream oos = new ObjectOutputStream(
1022 new FileOutputStream(persistencePath, false))){
1023 if (!ioEngine.isPersistent()) {
1024 throw new IOException("Attempt to persist non-persistent cache mappings!");
1025 }
1026 byte[] checksum = ((PersistentIOEngine) ioEngine).calculateChecksum(algorithm);
1027 if (checksum != null) {
1028 oos.write(ProtobufUtil.PB_MAGIC);
1029 oos.writeInt(checksum.length);
1030 oos.write(checksum);
1031 }
1032 oos.writeLong(cacheCapacity);
1033 oos.writeUTF(ioEngine.getClass().getName());
1034 oos.writeUTF(backingMap.getClass().getName());
1035 oos.writeObject(deserialiserMap);
1036 oos.writeObject(backingMap);
1037 }
1038 }
1039
1040 @SuppressWarnings("unchecked")
1041 private void retrieveFromFile(int[] bucketSizes) throws IOException,
1042 ClassNotFoundException {
1043 File persistenceFile = new File(persistencePath);
1044 if (!persistenceFile.exists()) {
1045 return;
1046 }
1047 assert !cacheEnabled;
1048 ObjectInputStream ois = null;
1049 try {
1050 if (!ioEngine.isPersistent())
1051 throw new IOException(
1052 "Attempt to restore non-persistent cache mappings!");
1053 ois = new ObjectInputStream(new FileInputStream(persistencePath));
1054 int pblen = ProtobufUtil.lengthOfPBMagic();
1055 byte[] pbuf = new byte[pblen];
1056 int read = ois.read(pbuf);
1057 if (read != pblen) {
1058 LOG.warn("Can't restore from file because of incorrect number of bytes read while " +
1059 "checking for protobuf magic number. Requested=" + pblen + ", but received= " +
1060 read + ".");
1061 return;
1062 }
1063 if (Bytes.equals(ProtobufUtil.PB_MAGIC, pbuf)) {
1064 int length = ois.readInt();
1065 byte[] persistentChecksum = new byte[length];
1066 int readLen = ois.read(persistentChecksum);
1067 if (readLen != length) {
1068 LOG.warn("Can't restore from file because of incorrect number of bytes read while " +
1069 "checking for persistent checksum. Requested=" + length + ", but received=" +
1070 readLen + ". ");
1071 return;
1072 }
1073 if (!((PersistentIOEngine) ioEngine).verifyFileIntegrity(
1074 persistentChecksum, algorithm)) {
1075 LOG.warn("Can't restore from file because of verification failed.");
1076 return;
1077 }
1078 } else {
1079
1080
1081 ois.close();
1082 ois = new ObjectInputStream(new FileInputStream(persistencePath));
1083 }
1084 long capacitySize = ois.readLong();
1085 if (capacitySize != cacheCapacity)
1086 throw new IOException("Mismatched cache capacity:"
1087 + StringUtils.byteDesc(capacitySize) + ", expected: "
1088 + StringUtils.byteDesc(cacheCapacity));
1089 String ioclass = ois.readUTF();
1090 String mapclass = ois.readUTF();
1091 if (!ioEngine.getClass().getName().equals(ioclass))
1092 throw new IOException("Class name for IO engine mismatch: " + ioclass
1093 + ", expected:" + ioEngine.getClass().getName());
1094 if (!backingMap.getClass().getName().equals(mapclass))
1095 throw new IOException("Class name for cache map mismatch: " + mapclass
1096 + ", expected:" + backingMap.getClass().getName());
1097 UniqueIndexMap<Integer> deserMap = (UniqueIndexMap<Integer>) ois
1098 .readObject();
1099 ConcurrentHashMap<BlockCacheKey, BucketEntry> backingMapFromFile =
1100 (ConcurrentHashMap<BlockCacheKey, BucketEntry>) ois.readObject();
1101 BucketAllocator allocator = new BucketAllocator(cacheCapacity, bucketSizes,
1102 backingMapFromFile, realCacheSize);
1103 bucketAllocator = allocator;
1104 deserialiserMap = deserMap;
1105 backingMap = backingMapFromFile;
1106 blockNumber.set(backingMap.size());
1107 } finally {
1108 if (ois != null) {
1109 ois.close();
1110 }
1111 if (!persistenceFile.delete()) {
1112 throw new IOException("Failed deleting persistence file "
1113 + persistenceFile.getAbsolutePath());
1114 }
1115 }
1116 }
1117
1118
1119
1120
1121
1122
1123 private void checkIOErrorIsTolerated() {
1124 long now = EnvironmentEdgeManager.currentTime();
1125
1126 long ioErrorStartTimeTmp = this.ioErrorStartTime;
1127 if (ioErrorStartTimeTmp > 0) {
1128 if (cacheEnabled && (now - ioErrorStartTimeTmp) > this.ioErrorsTolerationDuration) {
1129 LOG.error("IO errors duration time has exceeded " + ioErrorsTolerationDuration +
1130 "ms, disabling cache, please check your IOEngine");
1131 disableCache();
1132 }
1133 } else {
1134 this.ioErrorStartTime = now;
1135 }
1136 }
1137
1138
1139
1140
1141 private void disableCache() {
1142 if (!cacheEnabled) return;
1143 cacheEnabled = false;
1144 ioEngine.shutdown();
1145 this.scheduleThreadPool.shutdown();
1146 for (int i = 0; i < writerThreads.length; ++i) writerThreads[i].interrupt();
1147 this.ramCache.clear();
1148 if (!ioEngine.isPersistent() || persistencePath == null) {
1149
1150 this.backingMap.clear();
1151 }
1152 }
1153
1154 private void join() throws InterruptedException {
1155 for (int i = 0; i < writerThreads.length; ++i)
1156 writerThreads[i].join();
1157 }
1158
1159 @Override
1160 public void shutdown() {
1161 disableCache();
1162 LOG.info("Shutdown bucket cache: IO persistent=" + ioEngine.isPersistent()
1163 + "; path to write=" + persistencePath);
1164 if (ioEngine.isPersistent() && persistencePath != null) {
1165 try {
1166 join();
1167 persistToFile();
1168 } catch (IOException ex) {
1169 LOG.error("Unable to persist data on exit: " + ex.toString(), ex);
1170 } catch (InterruptedException e) {
1171 LOG.warn("Failed to persist data on exit", e);
1172 }
1173 }
1174 }
1175
1176 @Override
1177 public CacheStats getStats() {
1178 return cacheStats;
1179 }
1180
1181 public BucketAllocator getAllocator() {
1182 return this.bucketAllocator;
1183 }
1184
1185 @Override
1186 public long heapSize() {
1187 return this.heapSize.get();
1188 }
1189
1190 @Override
1191 public long size() {
1192 return this.realCacheSize.get();
1193 }
1194
1195 @Override
1196 public long getCurrentDataSize() {
1197 return size();
1198 }
1199
1200 @Override
1201 public long getFreeSize() {
1202 return this.bucketAllocator.getFreeSize();
1203 }
1204
1205 @Override
1206 public long getBlockCount() {
1207 return this.blockNumber.get();
1208 }
1209
1210 @Override
1211 public long getDataBlockCount() {
1212 return getBlockCount();
1213 }
1214
1215 @Override
1216 public long getCurrentSize() {
1217 return this.bucketAllocator.getUsedSize();
1218 }
1219
1220
1221
1222
1223
1224
1225
1226
1227 @Override
1228 public int evictBlocksByHfileName(String hfileName) {
1229 Set<BlockCacheKey> keySet = blocksByHFile.subSet(
1230 new BlockCacheKey(hfileName, Long.MIN_VALUE), true,
1231 new BlockCacheKey(hfileName, Long.MAX_VALUE), true);
1232
1233 int numEvicted = 0;
1234 for (BlockCacheKey key : keySet) {
1235 if (evictBlock(key)) {
1236 ++numEvicted;
1237 }
1238 }
1239
1240 return numEvicted;
1241 }
1242
1243
1244
1245
1246
1247
1248
1249
1250
1251 static class BucketEntry implements Serializable {
1252 private static final long serialVersionUID = -6741504807982257534L;
1253
1254
1255 static final Comparator<BucketEntry> COMPARATOR = new Comparator<BucketCache.BucketEntry>() {
1256
1257 @Override
1258 public int compare(BucketEntry o1, BucketEntry o2) {
1259 return Long.compare(o2.accessCounter, o1.accessCounter);
1260 }
1261 };
1262
1263 private int offsetBase;
1264 private int length;
1265 private byte offset1;
1266 byte deserialiserIndex;
1267 private volatile long accessCounter;
1268 private BlockPriority priority;
1269
1270
1271
1272 private final long cachedTime = System.nanoTime();
1273
1274 BucketEntry(long offset, int length, long accessCounter, boolean inMemory) {
1275 setOffset(offset);
1276 this.length = length;
1277 this.accessCounter = accessCounter;
1278 if (inMemory) {
1279 this.priority = BlockPriority.MEMORY;
1280 } else {
1281 this.priority = BlockPriority.SINGLE;
1282 }
1283 }
1284
1285 long offset() {
1286 long o = ((long) offsetBase) & 0xFFFFFFFFL;
1287 o += (((long) (offset1)) & 0xFF) << 32;
1288 return o << 8;
1289 }
1290
1291 private void setOffset(long value) {
1292 assert (value & 0xFF) == 0;
1293 value >>= 8;
1294 offsetBase = (int) value;
1295 offset1 = (byte) (value >> 32);
1296 }
1297
1298 public int getLength() {
1299 return length;
1300 }
1301
1302 protected CacheableDeserializer<Cacheable> deserializerReference(
1303 UniqueIndexMap<Integer> deserialiserMap) {
1304 return CacheableDeserializerIdManager.getDeserializer(deserialiserMap
1305 .unmap(deserialiserIndex));
1306 }
1307
1308 protected void setDeserialiserReference(
1309 CacheableDeserializer<Cacheable> deserializer,
1310 UniqueIndexMap<Integer> deserialiserMap) {
1311 this.deserialiserIndex = ((byte) deserialiserMap.map(deserializer
1312 .getDeserialiserIdentifier()));
1313 }
1314
1315
1316
1317
1318 public void access(long accessCounter) {
1319 this.accessCounter = accessCounter;
1320 if (this.priority == BlockPriority.SINGLE) {
1321 this.priority = BlockPriority.MULTI;
1322 }
1323 }
1324
1325 public BlockPriority getPriority() {
1326 return this.priority;
1327 }
1328
1329 public long getCachedTime() {
1330 return cachedTime;
1331 }
1332 }
1333
1334
1335
1336
1337
1338
1339
1340 private class BucketEntryGroup implements Comparable<BucketEntryGroup> {
1341
1342 private CachedEntryQueue queue;
1343 private long totalSize = 0;
1344 private long bucketSize;
1345
1346 public BucketEntryGroup(long bytesToFree, long blockSize, long bucketSize) {
1347 this.bucketSize = bucketSize;
1348 queue = new CachedEntryQueue(bytesToFree, blockSize);
1349 totalSize = 0;
1350 }
1351
1352 public void add(Map.Entry<BlockCacheKey, BucketEntry> block) {
1353 totalSize += block.getValue().getLength();
1354 queue.add(block);
1355 }
1356
1357 public long free(long toFree) {
1358 Map.Entry<BlockCacheKey, BucketEntry> entry;
1359 long freedBytes = 0;
1360 while ((entry = queue.pollLast()) != null) {
1361 evictBlock(entry.getKey());
1362 freedBytes += entry.getValue().getLength();
1363 if (freedBytes >= toFree) {
1364 return freedBytes;
1365 }
1366 }
1367 return freedBytes;
1368 }
1369
1370 public long overflow() {
1371 return totalSize - bucketSize;
1372 }
1373
1374 public long totalSize() {
1375 return totalSize;
1376 }
1377
1378 @Override
1379 public int compareTo(BucketEntryGroup that) {
1380 return Long.compare(this.overflow(), that.overflow());
1381 }
1382
1383 @Override
1384 public int hashCode() {
1385 final int prime = 31;
1386 int result = 1;
1387 result = prime * result + getOuterType().hashCode();
1388 result = prime * result + (int) (bucketSize ^ (bucketSize >>> 32));
1389 result = prime * result + ((queue == null) ? 0 : queue.hashCode());
1390 result = prime * result + (int) (totalSize ^ (totalSize >>> 32));
1391 return result;
1392 }
1393
1394 @Override
1395 public boolean equals(Object obj) {
1396 if (this == obj) {
1397 return true;
1398 }
1399 if (obj == null) {
1400 return false;
1401 }
1402 if (getClass() != obj.getClass()) {
1403 return false;
1404 }
1405 BucketEntryGroup other = (BucketEntryGroup) obj;
1406 if (!getOuterType().equals(other.getOuterType())) {
1407 return false;
1408 }
1409 if (bucketSize != other.bucketSize) {
1410 return false;
1411 }
1412 if (queue == null) {
1413 if (other.queue != null) {
1414 return false;
1415 }
1416 } else if (!queue.equals(other.queue)) {
1417 return false;
1418 }
1419 if (totalSize != other.totalSize) {
1420 return false;
1421 }
1422 return true;
1423 }
1424
1425 private BucketCache getOuterType() {
1426 return BucketCache.this;
1427 }
1428
1429 }
1430
1431
1432
1433
1434 static class RAMQueueEntry {
1435 private BlockCacheKey key;
1436 private Cacheable data;
1437 private long accessCounter;
1438 private boolean inMemory;
1439
1440 public RAMQueueEntry(BlockCacheKey bck, Cacheable data, long accessCounter,
1441 boolean inMemory) {
1442 this.key = bck;
1443 this.data = data;
1444 this.accessCounter = accessCounter;
1445 this.inMemory = inMemory;
1446 }
1447
1448 public Cacheable getData() {
1449 return data;
1450 }
1451
1452 public BlockCacheKey getKey() {
1453 return key;
1454 }
1455
1456 public void access(long accessCounter) {
1457 this.accessCounter = accessCounter;
1458 }
1459
1460 public BucketEntry writeToCache(final IOEngine ioEngine, final BucketAllocator bucketAllocator,
1461 final UniqueIndexMap<Integer> deserialiserMap, final AtomicLong realCacheSize)
1462 throws IOException {
1463 int len = data.getSerializedLength();
1464
1465 if (len == 0) {
1466 return null;
1467 }
1468 long offset = bucketAllocator.allocateBlock(len);
1469 BucketEntry bucketEntry;
1470 boolean succ = false;
1471 try {
1472 bucketEntry = new BucketEntry(offset, len, accessCounter, inMemory);
1473 bucketEntry.setDeserialiserReference(data.getDeserializer(), deserialiserMap);
1474 if (data instanceof HFileBlock) {
1475
1476 HFileBlock block = (HFileBlock) data;
1477 ByteBuffer sliceBuf = block.getBufferReadOnly();
1478 ByteBuffer metadata = block.getMetaData();
1479 ioEngine.write(sliceBuf, offset);
1480 ioEngine.write(metadata, offset + len - metadata.limit());
1481 } else {
1482 ByteBuffer bb = ByteBuffer.allocate(len);
1483 data.serialize(bb, true);
1484 ioEngine.write(bb, offset);
1485 }
1486 succ = true;
1487 } finally {
1488 if (!succ) {
1489 bucketAllocator.freeBlock(offset);
1490 }
1491 }
1492 realCacheSize.addAndGet(len);
1493 return bucketEntry;
1494 }
1495 }
1496
1497
1498
1499
1500
1501 void stopWriterThreads() throws InterruptedException {
1502 for (WriterThread writerThread : writerThreads) {
1503 writerThread.disableWriter();
1504 writerThread.interrupt();
1505 writerThread.join();
1506 }
1507 }
1508
1509 @Override
1510 public Iterator<CachedBlock> iterator() {
1511
1512 final Iterator<Map.Entry<BlockCacheKey, BucketEntry>> i =
1513 this.backingMap.entrySet().iterator();
1514 return new Iterator<CachedBlock>() {
1515 private final long now = System.nanoTime();
1516
1517 @Override
1518 public boolean hasNext() {
1519 return i.hasNext();
1520 }
1521
1522 @Override
1523 public CachedBlock next() {
1524 final Map.Entry<BlockCacheKey, BucketEntry> e = i.next();
1525 return new CachedBlock() {
1526 @Override
1527 public String toString() {
1528 return BlockCacheUtil.toString(this, now);
1529 }
1530
1531 @Override
1532 public BlockPriority getBlockPriority() {
1533 return e.getValue().getPriority();
1534 }
1535
1536 @Override
1537 public BlockType getBlockType() {
1538
1539 return null;
1540 }
1541
1542 @Override
1543 public long getOffset() {
1544 return e.getKey().getOffset();
1545 }
1546
1547 @Override
1548 public long getSize() {
1549 return e.getValue().getLength();
1550 }
1551
1552 @Override
1553 public long getCachedTime() {
1554 return e.getValue().getCachedTime();
1555 }
1556
1557 @Override
1558 public String getFilename() {
1559 return e.getKey().getHfileName();
1560 }
1561
1562 @Override
1563 public int compareTo(CachedBlock other) {
1564 int diff = this.getFilename().compareTo(other.getFilename());
1565 if (diff != 0) return diff;
1566
1567 diff = Long.compare(this.getOffset(), other.getOffset());
1568 if (diff != 0) return diff;
1569 if (other.getCachedTime() < 0 || this.getCachedTime() < 0) {
1570 throw new IllegalStateException("" + this.getCachedTime() + ", " +
1571 other.getCachedTime());
1572 }
1573 return Long.compare(other.getCachedTime(), this.getCachedTime());
1574 }
1575
1576 @Override
1577 public int hashCode() {
1578 return e.getKey().hashCode();
1579 }
1580
1581 @Override
1582 public boolean equals(Object obj) {
1583 if (obj instanceof CachedBlock) {
1584 CachedBlock cb = (CachedBlock)obj;
1585 return compareTo(cb) == 0;
1586 } else {
1587 return false;
1588 }
1589 }
1590 };
1591 }
1592
1593 @Override
1594 public void remove() {
1595 throw new UnsupportedOperationException();
1596 }
1597 };
1598 }
1599
1600 @Override
1601 public BlockCache[] getBlockCaches() {
1602 return null;
1603 }
1604
1605 float getAcceptableFactor() {
1606 return acceptableFactor;
1607 }
1608
1609 float getMinFactor() {
1610 return minFactor;
1611 }
1612
1613 float getExtraFreeFactor() {
1614 return extraFreeFactor;
1615 }
1616
1617 float getSingleFactor() {
1618 return singleFactor;
1619 }
1620
1621 float getMultiFactor() {
1622 return multiFactor;
1623 }
1624
1625 float getMemoryFactor() {
1626 return memoryFactor;
1627 }
1628
1629 public UniqueIndexMap<Integer> getDeserialiserMap() {
1630 return deserialiserMap;
1631 }
1632 }