1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18 package org.apache.hadoop.hbase.io.hfile.bucket;
19
20 import com.google.common.cache.Cache;
21 import com.google.common.cache.CacheBuilder;
22 import com.google.common.cache.RemovalListener;
23 import com.google.common.cache.RemovalNotification;
24 import com.google.common.util.concurrent.ThreadFactoryBuilder;
25
26 import java.io.IOException;
27 import java.util.concurrent.Executors;
28 import java.util.concurrent.ScheduledExecutorService;
29 import java.util.concurrent.TimeUnit;
30 import java.util.concurrent.atomic.AtomicLong;
31
32 import org.apache.commons.logging.Log;
33 import org.apache.commons.logging.LogFactory;
34 import org.apache.hadoop.conf.Configuration;
35 import org.apache.hadoop.hbase.classification.InterfaceAudience;
36 import org.apache.hadoop.hbase.io.hfile.BlockCacheKey;
37 import org.apache.hadoop.hbase.io.hfile.Cacheable;
38
39
40
41
42 @InterfaceAudience.Private
43 public class BufferedBucketCache extends BucketCache {
44 private static final Log LOG = LogFactory.getLog(BufferedBucketCache.class);
45
46 static final String RAM_BUFFER_SIZE_RATIO = "hbase.bucketcache.rambuffer.ratio";
47 static final double RAM_BUFFER_SIZE_RATIO_DEFAULT = 0.1;
48 static final String RAM_BUFFER_TIMEOUT = "hbase.bucketcache.rambuffer.timeout";
49 static final int RAM_BUFFER_TIMEOUT_DEFAULT = 60;
50
51 private final Cache<BlockCacheKey, Cacheable> ramBuffer;
52 private final long maxBufferSize;
53
54 private final AtomicLong ramBufferEvictCount = new AtomicLong(0);
55
56 private volatile float ramBufferThreshold;
57
58 private transient final ScheduledExecutorService scheduleThreadPool =
59 Executors.newScheduledThreadPool(1,
60 new ThreadFactoryBuilder().setNameFormat("RAMBufferAdjustExecutor").setDaemon(true).build());
61
62 public BufferedBucketCache(String ioEngineName, long capacity, int blockSize, int[] bucketSizes,
63 int writerThreadNum, int writerQLen, String persistencePath, int ioErrorsTolerationDuration,
64 Configuration conf) throws IOException {
65 super(ioEngineName, capacity, blockSize, bucketSizes, writerThreadNum, writerQLen,
66 persistencePath, ioErrorsTolerationDuration, conf);
67
68 maxBufferSize = (long) ((capacity / (double) blockSize) * conf.getDouble(RAM_BUFFER_SIZE_RATIO,
69 RAM_BUFFER_SIZE_RATIO_DEFAULT));
70 int timeout = conf.getInt(RAM_BUFFER_TIMEOUT, RAM_BUFFER_TIMEOUT_DEFAULT);
71 ramBuffer = CacheBuilder.newBuilder().
72 expireAfterAccess(timeout, TimeUnit.SECONDS).
73 maximumSize(maxBufferSize).
74 removalListener(new RemovalListener<BlockCacheKey, Cacheable>() {
75 @Override
76 public void onRemoval(RemovalNotification<BlockCacheKey, Cacheable> removalNotification) {
77 ramBufferEvictCount.incrementAndGet();
78 }
79 }).build();
80
81
82 scheduleThreadPool.scheduleAtFixedRate(
83 new RAMBufferAdjustThread(this), 60, 60,TimeUnit.SECONDS);
84 }
85
86 @Override
87 public Cacheable getBlock(BlockCacheKey key, boolean caching, boolean repeat,
88 boolean updateCacheMetrics) {
89 Cacheable block = ramBuffer.getIfPresent(key);
90 if (block != null) {
91 if (updateCacheMetrics) {
92 this.getStats().hit(caching, key.isPrimary(), key.getBlockType());
93 }
94 return block;
95 }
96 block = super.getBlock(key, caching, repeat, updateCacheMetrics);
97 if (block != null && ramBuffer.size() < maxBufferSize * ramBufferThreshold) {
98 ramBuffer.put(key, block);
99 }
100 return block;
101 }
102
103 private void updateRAMBufferThreshold(final float newThreshold) {
104 this.ramBufferThreshold = Math.max(Math.min(newThreshold, 1.0f), 0.01f);
105 }
106
107 static class RAMBufferAdjustThread extends Thread {
108 private final BufferedBucketCache bucketCache;
109
110 RAMBufferAdjustThread(BufferedBucketCache bucketCache) {
111 this.bucketCache = bucketCache;
112 }
113
114 @Override
115 public void run() {
116 long currentEvictCount = bucketCache.ramBufferEvictCount.get();
117 try {
118 Thread.sleep(10000);
119 } catch (InterruptedException e) {
120 LOG.info(e);
121 return;
122 }
123 long delta = (bucketCache.ramBufferEvictCount.get() - currentEvictCount) / 10;
124 if (delta > 100) {
125 bucketCache.updateRAMBufferThreshold((float) (bucketCache.ramBufferThreshold * 0.9));
126 } else if (delta < 10) {
127 bucketCache.updateRAMBufferThreshold((float) (bucketCache.ramBufferThreshold * 1.1));
128 }
129 }
130 }
131 }