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