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