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 writerThreadNum,
201 int writerQLen, String persistencePath) throws FileNotFoundException,
202 IOException {
203 this(ioEngineName, capacity, writerThreadNum, writerQLen, persistencePath,
204 DEFAULT_ERROR_TOLERATION_DURATION);
205 }
206
207 public BucketCache(String ioEngineName, long capacity, 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 / 16384;
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 = StoreFile.DEFAULT_BLOCKSIZE_SMALL;
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 @Override
352 public Cacheable getBlock(BlockCacheKey key, boolean caching, boolean repeat) {
353 if (!cacheEnabled)
354 return null;
355 RAMQueueEntry re = ramCache.get(key);
356 if (re != null) {
357 cacheStats.hit(caching);
358 re.access(accessCount.incrementAndGet());
359 return re.getData();
360 }
361 BucketEntry bucketEntry = backingMap.get(key);
362 if(bucketEntry!=null) {
363 long start = System.nanoTime();
364 IdLock.Entry lockEntry = null;
365 try {
366 lockEntry = offsetLock.getLockEntry(bucketEntry.offset());
367 if (bucketEntry.equals(backingMap.get(key))) {
368 int len = bucketEntry.getLength();
369 ByteBuffer bb = ByteBuffer.allocate(len);
370 ioEngine.read(bb, bucketEntry.offset());
371 Cacheable cachedBlock = bucketEntry.deserializerReference(
372 deserialiserMap).deserialize(bb, true);
373 long timeTaken = System.nanoTime() - start;
374 cacheStats.hit(caching);
375 cacheStats.ioHit(timeTaken);
376 bucketEntry.access(accessCount.incrementAndGet());
377 if (this.ioErrorStartTime > 0) {
378 ioErrorStartTime = -1;
379 }
380 return cachedBlock;
381 }
382 } catch (IOException ioex) {
383 LOG.error("Failed reading block " + key + " from bucket cache", ioex);
384 checkIOErrorIsTolerated();
385 } finally {
386 if (lockEntry != null) {
387 offsetLock.releaseLockEntry(lockEntry);
388 }
389 }
390 }
391 if(!repeat)cacheStats.miss(caching);
392 return null;
393 }
394
395 @Override
396 public boolean evictBlock(BlockCacheKey cacheKey) {
397 if (!cacheEnabled) return false;
398 RAMQueueEntry removedBlock = ramCache.remove(cacheKey);
399 if (removedBlock != null) {
400 this.blockNumber.decrementAndGet();
401 this.heapSize.addAndGet(-1 * removedBlock.getData().heapSize());
402 }
403 BucketEntry bucketEntry = backingMap.get(cacheKey);
404 if (bucketEntry != null) {
405 IdLock.Entry lockEntry = null;
406 try {
407 lockEntry = offsetLock.getLockEntry(bucketEntry.offset());
408 if (bucketEntry.equals(backingMap.remove(cacheKey))) {
409 bucketAllocator.freeBlock(bucketEntry.offset());
410 realCacheSize.addAndGet(-1 * bucketEntry.getLength());
411 blocksByHFile.remove(cacheKey.getHfileName(), cacheKey);
412 if (removedBlock == null) {
413 this.blockNumber.decrementAndGet();
414 }
415 } else {
416 return false;
417 }
418 } catch (IOException ie) {
419 LOG.warn("Failed evicting block " + cacheKey);
420 return false;
421 } finally {
422 if (lockEntry != null) {
423 offsetLock.releaseLockEntry(lockEntry);
424 }
425 }
426 }
427 cacheStats.evicted();
428 return true;
429 }
430
431
432
433
434 private static class StatisticsThread extends Thread {
435 BucketCache bucketCache;
436
437 public StatisticsThread(BucketCache bucketCache) {
438 super("BucketCache.StatisticsThread");
439 setDaemon(true);
440 this.bucketCache = bucketCache;
441 }
442 @Override
443 public void run() {
444 bucketCache.logStats();
445 }
446 }
447
448 public void logStats() {
449 if (!LOG.isDebugEnabled()) return;
450
451 long totalSize = bucketAllocator.getTotalSize();
452 long usedSize = bucketAllocator.getUsedSize();
453 long freeSize = totalSize - usedSize;
454 long cacheSize = this.realCacheSize.get();
455 LOG.debug("BucketCache Stats: " +
456 "failedBlockAdditions=" + this.failedBlockAdditions.get() + ", " +
457 "total=" + StringUtils.byteDesc(totalSize) + ", " +
458 "free=" + StringUtils.byteDesc(freeSize) + ", " +
459 "usedSize=" + StringUtils.byteDesc(usedSize) +", " +
460 "cacheSize=" + StringUtils.byteDesc(cacheSize) +", " +
461 "accesses=" + cacheStats.getRequestCount() + ", " +
462 "hits=" + cacheStats.getHitCount() + ", " +
463 "IOhitsPerSecond=" + cacheStats.getIOHitsPerSecond() + ", " +
464 "IOTimePerHit=" + String.format("%.2f", cacheStats.getIOTimePerHit())+ ", " +
465 "hitRatio=" + (cacheStats.getHitCount() == 0 ? "0," :
466 (StringUtils.formatPercent(cacheStats.getHitRatio(), 2)+ ", ")) +
467 "cachingAccesses=" + cacheStats.getRequestCachingCount() + ", " +
468 "cachingHits=" + cacheStats.getHitCachingCount() + ", " +
469 "cachingHitsRatio=" +(cacheStats.getHitCachingCount() == 0 ? "0," :
470 (StringUtils.formatPercent(cacheStats.getHitCachingRatio(), 2)+ ", ")) +
471 "evictions=" + cacheStats.getEvictionCount() + ", " +
472 "evicted=" + cacheStats.getEvictedCount() + ", " +
473 "evictedPerRun=" + cacheStats.evictedPerEviction());
474 cacheStats.reset();
475 }
476
477 private long acceptableSize() {
478 return (long) Math.floor(bucketAllocator.getTotalSize() * DEFAULT_ACCEPT_FACTOR);
479 }
480
481 private long minSize() {
482 return (long) Math.floor(bucketAllocator.getTotalSize() * DEFAULT_MIN_FACTOR);
483 }
484
485 private long singleSize() {
486 return (long) Math.floor(bucketAllocator.getTotalSize()
487 * DEFAULT_SINGLE_FACTOR * DEFAULT_MIN_FACTOR);
488 }
489
490 private long multiSize() {
491 return (long) Math.floor(bucketAllocator.getTotalSize() * DEFAULT_MULTI_FACTOR
492 * DEFAULT_MIN_FACTOR);
493 }
494
495 private long memorySize() {
496 return (long) Math.floor(bucketAllocator.getTotalSize() * DEFAULT_MEMORY_FACTOR
497 * DEFAULT_MIN_FACTOR);
498 }
499
500
501
502
503
504
505 private void freeSpace() {
506
507 if (!freeSpaceLock.tryLock()) return;
508 try {
509 freeInProgress = true;
510 long bytesToFreeWithoutExtra = 0;
511
512
513
514 StringBuffer msgBuffer = new StringBuffer();
515 BucketAllocator.IndexStatistics[] stats = bucketAllocator.getIndexStatistics();
516 long[] bytesToFreeForBucket = new long[stats.length];
517 for (int i = 0; i < stats.length; i++) {
518 bytesToFreeForBucket[i] = 0;
519 long freeGoal = (long) Math.floor(stats[i].totalCount()
520 * (1 - DEFAULT_MIN_FACTOR));
521 freeGoal = Math.max(freeGoal, 1);
522 if (stats[i].freeCount() < freeGoal) {
523 bytesToFreeForBucket[i] = stats[i].itemSize()
524 * (freeGoal - stats[i].freeCount());
525 bytesToFreeWithoutExtra += bytesToFreeForBucket[i];
526 msgBuffer.append("Free for bucketSize(" + stats[i].itemSize() + ")="
527 + StringUtils.byteDesc(bytesToFreeForBucket[i]) + ", ");
528 }
529 }
530 msgBuffer.append("Free for total="
531 + StringUtils.byteDesc(bytesToFreeWithoutExtra) + ", ");
532
533 if (bytesToFreeWithoutExtra <= 0) {
534 return;
535 }
536 long currentSize = bucketAllocator.getUsedSize();
537 long totalSize=bucketAllocator.getTotalSize();
538 LOG.debug("Bucket cache free space started; Attempting to " + msgBuffer.toString()
539 + " of current used=" + StringUtils.byteDesc(currentSize)
540 + ",actual cacheSize=" + StringUtils.byteDesc(realCacheSize.get())
541 + ",total=" + StringUtils.byteDesc(totalSize));
542
543 long bytesToFreeWithExtra = (long) Math.floor(bytesToFreeWithoutExtra
544 * (1 + DEFAULT_EXTRA_FREE_FACTOR));
545
546
547 BucketEntryGroup bucketSingle = new BucketEntryGroup(bytesToFreeWithExtra,
548 blockSize, singleSize());
549 BucketEntryGroup bucketMulti = new BucketEntryGroup(bytesToFreeWithExtra,
550 blockSize, multiSize());
551 BucketEntryGroup bucketMemory = new BucketEntryGroup(bytesToFreeWithExtra,
552 blockSize, memorySize());
553
554
555
556 for (Map.Entry<BlockCacheKey, BucketEntry> bucketEntryWithKey : backingMap.entrySet()) {
557 switch (bucketEntryWithKey.getValue().getPriority()) {
558 case SINGLE: {
559 bucketSingle.add(bucketEntryWithKey);
560 break;
561 }
562 case MULTI: {
563 bucketMulti.add(bucketEntryWithKey);
564 break;
565 }
566 case MEMORY: {
567 bucketMemory.add(bucketEntryWithKey);
568 break;
569 }
570 }
571 }
572
573 PriorityQueue<BucketEntryGroup> bucketQueue = new PriorityQueue<BucketEntryGroup>(3);
574
575 bucketQueue.add(bucketSingle);
576 bucketQueue.add(bucketMulti);
577 bucketQueue.add(bucketMemory);
578
579 int remainingBuckets = 3;
580 long bytesFreed = 0;
581
582 BucketEntryGroup bucketGroup;
583 while ((bucketGroup = bucketQueue.poll()) != null) {
584 long overflow = bucketGroup.overflow();
585 if (overflow > 0) {
586 long bucketBytesToFree = Math.min(overflow,
587 (bytesToFreeWithoutExtra - bytesFreed) / remainingBuckets);
588 bytesFreed += bucketGroup.free(bucketBytesToFree);
589 }
590 remainingBuckets--;
591 }
592
593
594
595
596
597 stats = bucketAllocator.getIndexStatistics();
598 boolean needFreeForExtra = false;
599 for (int i = 0; i < stats.length; i++) {
600 long freeGoal = (long) Math.floor(stats[i].totalCount()
601 * (1 - DEFAULT_MIN_FACTOR));
602 freeGoal = Math.max(freeGoal, 1);
603 if (stats[i].freeCount() < freeGoal) {
604 needFreeForExtra = true;
605 break;
606 }
607 }
608
609 if (needFreeForExtra) {
610 bucketQueue.clear();
611 remainingBuckets = 2;
612
613 bucketQueue.add(bucketSingle);
614 bucketQueue.add(bucketMulti);
615
616 while ((bucketGroup = bucketQueue.poll()) != null) {
617 long bucketBytesToFree = (bytesToFreeWithExtra - bytesFreed)
618 / remainingBuckets;
619 bytesFreed += bucketGroup.free(bucketBytesToFree);
620 remainingBuckets--;
621 }
622 }
623
624 if (LOG.isDebugEnabled()) {
625 long single = bucketSingle.totalSize();
626 long multi = bucketMulti.totalSize();
627 long memory = bucketMemory.totalSize();
628 LOG.debug("Bucket cache free space completed; " + "freed="
629 + StringUtils.byteDesc(bytesFreed) + ", " + "total="
630 + StringUtils.byteDesc(totalSize) + ", " + "single="
631 + StringUtils.byteDesc(single) + ", " + "multi="
632 + StringUtils.byteDesc(multi) + ", " + "memory="
633 + StringUtils.byteDesc(memory));
634 }
635
636 } finally {
637 cacheStats.evict();
638 freeInProgress = false;
639 freeSpaceLock.unlock();
640 }
641 }
642
643
644 private class WriterThread extends HasThread {
645 BlockingQueue<RAMQueueEntry> inputQueue;
646 final int threadNO;
647 boolean writerEnabled = true;
648
649 WriterThread(BlockingQueue<RAMQueueEntry> queue, int threadNO) {
650 super();
651 this.inputQueue = queue;
652 this.threadNO = threadNO;
653 setDaemon(true);
654 }
655
656
657 void disableWriter() {
658 this.writerEnabled = false;
659 }
660
661 public void run() {
662 List<RAMQueueEntry> entries = new ArrayList<RAMQueueEntry>();
663 try {
664 while (cacheEnabled && writerEnabled) {
665 try {
666
667 entries.add(inputQueue.take());
668 inputQueue.drainTo(entries);
669 synchronized (cacheWaitSignals[threadNO]) {
670 cacheWaitSignals[threadNO].notifyAll();
671 }
672 } catch (InterruptedException ie) {
673 if (!cacheEnabled) break;
674 }
675 doDrain(entries);
676 }
677 } catch (Throwable t) {
678 LOG.warn("Failed doing drain", t);
679 }
680 LOG.info(this.getName() + " exiting, cacheEnabled=" + cacheEnabled);
681 }
682
683
684
685
686
687
688
689 private void doDrain(List<RAMQueueEntry> entries)
690 throws InterruptedException {
691 BucketEntry[] bucketEntries = new BucketEntry[entries.size()];
692 RAMQueueEntry[] ramEntries = new RAMQueueEntry[entries.size()];
693 int done = 0;
694 while (entries.size() > 0 && cacheEnabled) {
695
696 RAMQueueEntry ramEntry = null;
697 try {
698 ramEntry = entries.remove(entries.size() - 1);
699 if (ramEntry == null) {
700 LOG.warn("Couldn't get the entry from RAM queue, who steals it?");
701 continue;
702 }
703 BucketEntry bucketEntry = ramEntry.writeToCache(ioEngine,
704 bucketAllocator, deserialiserMap, realCacheSize);
705 ramEntries[done] = ramEntry;
706 bucketEntries[done++] = bucketEntry;
707 if (ioErrorStartTime > 0) {
708 ioErrorStartTime = -1;
709 }
710 } catch (BucketAllocatorException fle) {
711 LOG.warn("Failed allocating for block "
712 + (ramEntry == null ? "" : ramEntry.getKey()), fle);
713 } catch (CacheFullException cfe) {
714 if (!freeInProgress) {
715 freeSpace();
716 } else {
717 Thread.sleep(50);
718 }
719 } catch (IOException ioex) {
720 LOG.error("Failed writing to bucket cache", ioex);
721 checkIOErrorIsTolerated();
722 }
723 }
724
725
726
727 try {
728 ioEngine.sync();
729 } catch (IOException ioex) {
730 LOG.error("Faild syncing IO engine", ioex);
731 checkIOErrorIsTolerated();
732
733 for (int i = 0; i < done; ++i) {
734 if (bucketEntries[i] != null) {
735 bucketAllocator.freeBlock(bucketEntries[i].offset());
736 }
737 }
738 done = 0;
739 }
740
741 for (int i = 0; i < done; ++i) {
742 if (bucketEntries[i] != null) {
743 backingMap.put(ramEntries[i].getKey(), bucketEntries[i]);
744 }
745 RAMQueueEntry ramCacheEntry = ramCache.remove(ramEntries[i].getKey());
746 if (ramCacheEntry != null) {
747 heapSize.addAndGet(-1 * ramEntries[i].getData().heapSize());
748 }
749 }
750
751 if (bucketAllocator.getUsedSize() > acceptableSize()) {
752 freeSpace();
753 }
754 }
755 }
756
757
758
759 private void persistToFile() throws IOException {
760 assert !cacheEnabled;
761 FileOutputStream fos = null;
762 ObjectOutputStream oos = null;
763 try {
764 if (!ioEngine.isPersistent())
765 throw new IOException(
766 "Attempt to persist non-persistent cache mappings!");
767 fos = new FileOutputStream(persistencePath, false);
768 oos = new ObjectOutputStream(fos);
769 oos.writeLong(cacheCapacity);
770 oos.writeUTF(ioEngine.getClass().getName());
771 oos.writeUTF(backingMap.getClass().getName());
772 oos.writeObject(deserialiserMap);
773 oos.writeObject(backingMap);
774 } finally {
775 if (oos != null) oos.close();
776 if (fos != null) fos.close();
777 }
778 }
779
780 @SuppressWarnings("unchecked")
781 private void retrieveFromFile() throws IOException, BucketAllocatorException,
782 ClassNotFoundException {
783 File persistenceFile = new File(persistencePath);
784 if (!persistenceFile.exists()) {
785 return;
786 }
787 assert !cacheEnabled;
788 FileInputStream fis = null;
789 ObjectInputStream ois = null;
790 try {
791 if (!ioEngine.isPersistent())
792 throw new IOException(
793 "Attempt to restore non-persistent cache mappings!");
794 fis = new FileInputStream(persistencePath);
795 ois = new ObjectInputStream(fis);
796 long capacitySize = ois.readLong();
797 if (capacitySize != cacheCapacity)
798 throw new IOException("Mismatched cache capacity:"
799 + StringUtils.byteDesc(capacitySize) + ", expected: "
800 + StringUtils.byteDesc(cacheCapacity));
801 String ioclass = ois.readUTF();
802 String mapclass = ois.readUTF();
803 if (!ioEngine.getClass().getName().equals(ioclass))
804 throw new IOException("Class name for IO engine mismatch: " + ioclass
805 + ", expected:" + ioEngine.getClass().getName());
806 if (!backingMap.getClass().getName().equals(mapclass))
807 throw new IOException("Class name for cache map mismatch: " + mapclass
808 + ", expected:" + backingMap.getClass().getName());
809 UniqueIndexMap<Integer> deserMap = (UniqueIndexMap<Integer>) ois
810 .readObject();
811 BucketAllocator allocator = new BucketAllocator(cacheCapacity,
812 backingMap, this.realCacheSize);
813 backingMap = (ConcurrentHashMap<BlockCacheKey, BucketEntry>) ois
814 .readObject();
815 bucketAllocator = allocator;
816 deserialiserMap = deserMap;
817 } finally {
818 if (ois != null) ois.close();
819 if (fis != null) fis.close();
820 if (!persistenceFile.delete()) {
821 throw new IOException("Failed deleting persistence file "
822 + persistenceFile.getAbsolutePath());
823 }
824 }
825 }
826
827
828
829
830
831
832 private void checkIOErrorIsTolerated() {
833 long now = EnvironmentEdgeManager.currentTimeMillis();
834 if (this.ioErrorStartTime > 0) {
835 if (cacheEnabled
836 && (now - ioErrorStartTime) > this.ioErrorsTolerationDuration) {
837 LOG.error("IO errors duration time has exceeded "
838 + ioErrorsTolerationDuration
839 + "ms, disabing cache, please check your IOEngine");
840 disableCache();
841 }
842 } else {
843 this.ioErrorStartTime = now;
844 }
845 }
846
847
848
849
850
851 private void disableCache() {
852 if (!cacheEnabled)
853 return;
854 cacheEnabled = false;
855 ioEngine.shutdown();
856 this.scheduleThreadPool.shutdown();
857 for (int i = 0; i < writerThreads.length; ++i)
858 writerThreads[i].interrupt();
859 this.ramCache.clear();
860 if (!ioEngine.isPersistent() || persistencePath == null) {
861 this.backingMap.clear();
862 }
863 }
864
865 private void join() throws InterruptedException {
866 for (int i = 0; i < writerThreads.length; ++i)
867 writerThreads[i].join();
868 }
869
870 @Override
871 public void shutdown() {
872 disableCache();
873 LOG.info("Shutdown bucket cache: IO persistent=" + ioEngine.isPersistent()
874 + "; path to write=" + persistencePath);
875 if (ioEngine.isPersistent() && persistencePath != null) {
876 try {
877 join();
878 persistToFile();
879 } catch (IOException ex) {
880 LOG.error("Unable to persist data on exit: " + ex.toString(), ex);
881 } catch (InterruptedException e) {
882 LOG.warn("Failed to persist data on exit", e);
883 }
884 }
885 }
886
887 @Override
888 public CacheStats getStats() {
889 return cacheStats;
890 }
891
892 BucketAllocator getAllocator() {
893 return this.bucketAllocator;
894 }
895
896 @Override
897 public long heapSize() {
898 return this.heapSize.get();
899 }
900
901 @Override
902 public long size() {
903 return this.realCacheSize.get();
904 }
905
906 @Override
907 public long getFreeSize() {
908 return this.bucketAllocator.getFreeSize();
909 }
910
911 @Override
912 public long getBlockCount() {
913 return this.blockNumber.get();
914 }
915
916 @Override
917 public long getCurrentSize() {
918 return this.bucketAllocator.getUsedSize();
919 }
920
921 @Override
922 public long getEvictedCount() {
923 return cacheStats.getEvictedCount();
924 }
925
926
927
928
929
930
931
932
933 @Override
934 public int evictBlocksByHfileName(String hfileName) {
935
936
937 Set<BlockCacheKey> keySet = blocksByHFile.values(hfileName);
938 if (keySet == null) {
939 return 0;
940 }
941 int numEvicted = 0;
942 List<BlockCacheKey> keysForHFile = ImmutableList.copyOf(keySet);
943 for (BlockCacheKey key : keysForHFile) {
944 if (evictBlock(key)) {
945 ++numEvicted;
946 }
947 }
948
949 return numEvicted;
950 }
951
952
953 @Override
954 public List<BlockCacheColumnFamilySummary> getBlockCacheColumnFamilySummaries(
955 Configuration conf) {
956 throw new UnsupportedOperationException();
957 }
958
959 static enum BlockPriority {
960
961
962
963 SINGLE,
964
965
966
967 MULTI,
968
969
970
971 MEMORY
972 };
973
974
975
976
977
978
979
980
981
982 static class BucketEntry implements Serializable, Comparable<BucketEntry> {
983 private static final long serialVersionUID = -6741504807982257534L;
984 private int offsetBase;
985 private int length;
986 private byte offset1;
987 byte deserialiserIndex;
988 private volatile long accessTime;
989 private BlockPriority priority;
990
991 BucketEntry(long offset, int length, long accessTime, boolean inMemory) {
992 setOffset(offset);
993 this.length = length;
994 this.accessTime = accessTime;
995 if (inMemory) {
996 this.priority = BlockPriority.MEMORY;
997 } else {
998 this.priority = BlockPriority.SINGLE;
999 }
1000 }
1001
1002 long offset() {
1003 long o = ((long) offsetBase) & 0xFFFFFFFF;
1004 o += (((long) (offset1)) & 0xFF) << 32;
1005 return o << 8;
1006 }
1007
1008 private void setOffset(long value) {
1009 assert (value & 0xFF) == 0;
1010 value >>= 8;
1011 offsetBase = (int) value;
1012 offset1 = (byte) (value >> 32);
1013 }
1014
1015 public int getLength() {
1016 return length;
1017 }
1018
1019 protected CacheableDeserializer<Cacheable> deserializerReference(
1020 UniqueIndexMap<Integer> deserialiserMap) {
1021 return CacheableDeserializerIdManager.getDeserializer(deserialiserMap
1022 .unmap(deserialiserIndex));
1023 }
1024
1025 protected void setDeserialiserReference(
1026 CacheableDeserializer<Cacheable> deserializer,
1027 UniqueIndexMap<Integer> deserialiserMap) {
1028 this.deserialiserIndex = ((byte) deserialiserMap.map(deserializer
1029 .getDeserialiserIdentifier()));
1030 }
1031
1032
1033
1034
1035 public void access(long accessTime) {
1036 this.accessTime = accessTime;
1037 if (this.priority == BlockPriority.SINGLE) {
1038 this.priority = BlockPriority.MULTI;
1039 }
1040 }
1041
1042 public BlockPriority getPriority() {
1043 return this.priority;
1044 }
1045
1046 @Override
1047 public int compareTo(BucketEntry that) {
1048 if(this.accessTime == that.accessTime) return 0;
1049 return this.accessTime < that.accessTime ? 1 : -1;
1050 }
1051
1052 @Override
1053 public boolean equals(Object that) {
1054 return this == that;
1055 }
1056 }
1057
1058
1059
1060
1061
1062
1063
1064 private class BucketEntryGroup implements Comparable<BucketEntryGroup> {
1065
1066 private CachedEntryQueue queue;
1067 private long totalSize = 0;
1068 private long bucketSize;
1069
1070 public BucketEntryGroup(long bytesToFree, long blockSize, long bucketSize) {
1071 this.bucketSize = bucketSize;
1072 queue = new CachedEntryQueue(bytesToFree, blockSize);
1073 totalSize = 0;
1074 }
1075
1076 public void add(Map.Entry<BlockCacheKey, BucketEntry> block) {
1077 totalSize += block.getValue().getLength();
1078 queue.add(block);
1079 }
1080
1081 public long free(long toFree) {
1082 Map.Entry<BlockCacheKey, BucketEntry> entry;
1083 long freedBytes = 0;
1084 while ((entry = queue.pollLast()) != null) {
1085 evictBlock(entry.getKey());
1086 freedBytes += entry.getValue().getLength();
1087 if (freedBytes >= toFree) {
1088 return freedBytes;
1089 }
1090 }
1091 return freedBytes;
1092 }
1093
1094 public long overflow() {
1095 return totalSize - bucketSize;
1096 }
1097
1098 public long totalSize() {
1099 return totalSize;
1100 }
1101
1102 @Override
1103 public int compareTo(BucketEntryGroup that) {
1104 if (this.overflow() == that.overflow())
1105 return 0;
1106 return this.overflow() > that.overflow() ? 1 : -1;
1107 }
1108
1109 @Override
1110 public boolean equals(Object that) {
1111 return this == that;
1112 }
1113
1114 }
1115
1116
1117
1118
1119 private static class RAMQueueEntry {
1120 private BlockCacheKey key;
1121 private Cacheable data;
1122 private long accessTime;
1123 private boolean inMemory;
1124
1125 public RAMQueueEntry(BlockCacheKey bck, Cacheable data, long accessTime,
1126 boolean inMemory) {
1127 this.key = bck;
1128 this.data = data;
1129 this.accessTime = accessTime;
1130 this.inMemory = inMemory;
1131 }
1132
1133 public Cacheable getData() {
1134 return data;
1135 }
1136
1137 public BlockCacheKey getKey() {
1138 return key;
1139 }
1140
1141 public void access(long accessTime) {
1142 this.accessTime = accessTime;
1143 }
1144
1145 public BucketEntry writeToCache(final IOEngine ioEngine,
1146 final BucketAllocator bucketAllocator,
1147 final UniqueIndexMap<Integer> deserialiserMap,
1148 final AtomicLong realCacheSize) throws CacheFullException, IOException,
1149 BucketAllocatorException {
1150 int len = data.getSerializedLength();
1151
1152 if (len == 0) return null;
1153 long offset = bucketAllocator.allocateBlock(len);
1154 BucketEntry bucketEntry = new BucketEntry(offset, len, accessTime,
1155 inMemory);
1156 bucketEntry.setDeserialiserReference(data.getDeserializer(), deserialiserMap);
1157 try {
1158 if (data instanceof HFileBlock) {
1159 ByteBuffer sliceBuf = ((HFileBlock) data).getBufferReadOnlyWithHeader();
1160 sliceBuf.rewind();
1161 assert len == sliceBuf.limit() + HFileBlock.EXTRA_SERIALIZATION_SPACE;
1162 ByteBuffer extraInfoBuffer = ByteBuffer.allocate(HFileBlock.EXTRA_SERIALIZATION_SPACE);
1163 ((HFileBlock) data).serializeExtraInfo(extraInfoBuffer);
1164 ioEngine.write(sliceBuf, offset);
1165 ioEngine.write(extraInfoBuffer, offset + len - HFileBlock.EXTRA_SERIALIZATION_SPACE);
1166 } else {
1167 ByteBuffer bb = ByteBuffer.allocate(len);
1168 data.serialize(bb);
1169 ioEngine.write(bb, offset);
1170 }
1171 } catch (IOException ioe) {
1172
1173 bucketAllocator.freeBlock(offset);
1174 throw ioe;
1175 }
1176
1177 realCacheSize.addAndGet(len);
1178 return bucketEntry;
1179 }
1180 }
1181
1182
1183
1184
1185
1186 void stopWriterThreads() throws InterruptedException {
1187 for (WriterThread writerThread : writerThreads) {
1188 writerThread.disableWriter();
1189 writerThread.interrupt();
1190 writerThread.join();
1191 }
1192 }
1193
1194 }