1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.io.hfile.slab;
20
21 import java.nio.ByteBuffer;
22 import java.util.List;
23 import java.util.concurrent.ConcurrentMap;
24 import java.util.concurrent.atomic.AtomicLong;
25
26 import org.apache.commons.logging.Log;
27 import org.apache.commons.logging.LogFactory;
28 import org.apache.hadoop.classification.InterfaceAudience;
29 import org.apache.hadoop.conf.Configuration;
30 import org.apache.hadoop.hbase.io.HeapSize;
31 import org.apache.hadoop.hbase.io.hfile.BlockCache;
32 import org.apache.hadoop.hbase.io.hfile.BlockCacheColumnFamilySummary;
33 import org.apache.hadoop.hbase.io.hfile.BlockCacheKey;
34 import org.apache.hadoop.hbase.io.hfile.CacheStats;
35 import org.apache.hadoop.hbase.io.hfile.Cacheable;
36 import org.apache.hadoop.hbase.io.hfile.CacheableDeserializer;
37 import org.apache.hadoop.hbase.util.Bytes;
38 import org.apache.hadoop.hbase.util.ClassSize;
39 import org.apache.hadoop.util.StringUtils;
40
41 import com.google.common.cache.CacheBuilder;
42 import com.google.common.cache.RemovalListener;
43 import com.google.common.cache.RemovalNotification;
44
45
46
47
48
49
50
51
52
53
54
55
56 @InterfaceAudience.Private
57 public class SingleSizeCache implements BlockCache, HeapSize {
58 private final Slab backingStore;
59 private final ConcurrentMap<BlockCacheKey, CacheablePair> backingMap;
60 private final int numBlocks;
61 private final int blockSize;
62 private final CacheStats stats;
63 private final SlabItemActionWatcher actionWatcher;
64 private final AtomicLong size;
65 private final AtomicLong timeSinceLastAccess;
66 public final static long CACHE_FIXED_OVERHEAD = ClassSize
67 .align((2 * Bytes.SIZEOF_INT) + (5 * ClassSize.REFERENCE)
68 + +ClassSize.OBJECT);
69
70 static final Log LOG = LogFactory.getLog(SingleSizeCache.class);
71
72
73
74
75
76
77
78
79
80
81
82
83 public SingleSizeCache(int blockSize, int numBlocks,
84 SlabItemActionWatcher master) {
85 this.blockSize = blockSize;
86 this.numBlocks = numBlocks;
87 backingStore = new Slab(blockSize, numBlocks);
88 this.stats = new CacheStats();
89 this.actionWatcher = master;
90 this.size = new AtomicLong(CACHE_FIXED_OVERHEAD + backingStore.heapSize());
91 this.timeSinceLastAccess = new AtomicLong();
92
93
94
95 RemovalListener<BlockCacheKey, CacheablePair> listener =
96 new RemovalListener<BlockCacheKey, CacheablePair>() {
97 @Override
98 public void onRemoval(
99 RemovalNotification<BlockCacheKey, CacheablePair> notification) {
100 if (!notification.wasEvicted()) {
101
102
103 return;
104 }
105 CacheablePair value = notification.getValue();
106 timeSinceLastAccess.set(System.nanoTime()
107 - value.recentlyAccessed.get());
108 stats.evict();
109 doEviction(notification.getKey(), value);
110 }
111 };
112
113 backingMap = CacheBuilder.newBuilder()
114 .maximumSize(numBlocks - 1)
115 .removalListener(listener)
116 .<BlockCacheKey, CacheablePair>build()
117 .asMap();
118 }
119
120 @Override
121 public void cacheBlock(BlockCacheKey blockName, Cacheable toBeCached) {
122 ByteBuffer storedBlock;
123
124 try {
125 storedBlock = backingStore.alloc(toBeCached.getSerializedLength());
126 } catch (InterruptedException e) {
127 LOG.warn("SlabAllocator was interrupted while waiting for block to become available");
128 LOG.warn(e);
129 return;
130 }
131
132 CacheablePair newEntry = new CacheablePair(toBeCached.getDeserializer(),
133 storedBlock);
134 toBeCached.serialize(storedBlock);
135
136 synchronized (this) {
137 CacheablePair alreadyCached = backingMap.putIfAbsent(blockName, newEntry);
138
139 if (alreadyCached != null) {
140 backingStore.free(storedBlock);
141 throw new RuntimeException("already cached " + blockName);
142 }
143 if (actionWatcher != null) {
144 actionWatcher.onInsertion(blockName, this);
145 }
146 }
147 newEntry.recentlyAccessed.set(System.nanoTime());
148 this.size.addAndGet(newEntry.heapSize());
149 }
150
151 @Override
152 public Cacheable getBlock(BlockCacheKey key, boolean caching, boolean repeat) {
153 CacheablePair contentBlock = backingMap.get(key);
154 if (contentBlock == null) {
155 if (!repeat) stats.miss(caching);
156 return null;
157 }
158
159 stats.hit(caching);
160
161 try {
162 contentBlock.recentlyAccessed.set(System.nanoTime());
163 synchronized (contentBlock) {
164 if (contentBlock.serializedData == null) {
165
166 LOG.warn("Concurrent eviction of " + key);
167 return null;
168 }
169 return contentBlock.deserializer
170 .deserialize(contentBlock.serializedData.asReadOnlyBuffer());
171 }
172 } catch (Throwable t) {
173 LOG.error("Deserializer threw an exception. This may indicate a bug.", t);
174 return null;
175 }
176 }
177
178
179
180
181
182
183
184 public boolean evictBlock(BlockCacheKey key) {
185 stats.evict();
186 CacheablePair evictedBlock = backingMap.remove(key);
187
188 if (evictedBlock != null) {
189 doEviction(key, evictedBlock);
190 }
191 return evictedBlock != null;
192 }
193
194 private void doEviction(BlockCacheKey key, CacheablePair evictedBlock) {
195 long evictedHeap = 0;
196 synchronized (evictedBlock) {
197 if (evictedBlock.serializedData == null) {
198
199 return;
200 }
201 evictedHeap = evictedBlock.heapSize();
202 ByteBuffer bb = evictedBlock.serializedData;
203 evictedBlock.serializedData = null;
204 backingStore.free(bb);
205
206
207
208
209
210
211
212
213
214
215
216
217 if (actionWatcher != null) {
218 actionWatcher.onEviction(key, this);
219 }
220 }
221 stats.evicted();
222 size.addAndGet(-1 * evictedHeap);
223 }
224
225 public void logStats() {
226
227 long milliseconds = this.timeSinceLastAccess.get() / 1000000;
228
229 LOG.info("For Slab of size " + this.blockSize + ": "
230 + this.getOccupiedSize() / this.blockSize
231 + " occupied, out of a capacity of " + this.numBlocks
232 + " blocks. HeapSize is "
233 + StringUtils.humanReadableInt(this.heapSize()) + " bytes." + ", "
234 + "churnTime=" + StringUtils.formatTime(milliseconds));
235
236 LOG.info("Slab Stats: " + "accesses="
237 + stats.getRequestCount()
238 + ", "
239 + "hits="
240 + stats.getHitCount()
241 + ", "
242 + "hitRatio="
243 + (stats.getHitCount() == 0 ? "0" : (StringUtils.formatPercent(
244 stats.getHitRatio(), 2) + "%, "))
245 + "cachingAccesses="
246 + stats.getRequestCachingCount()
247 + ", "
248 + "cachingHits="
249 + stats.getHitCachingCount()
250 + ", "
251 + "cachingHitsRatio="
252 + (stats.getHitCachingCount() == 0 ? "0" : (StringUtils.formatPercent(
253 stats.getHitCachingRatio(), 2) + "%, ")) + "evictions="
254 + stats.getEvictionCount() + ", " + "evicted="
255 + stats.getEvictedCount() + ", " + "evictedPerRun="
256 + stats.evictedPerEviction());
257
258 }
259
260 public void shutdown() {
261 backingStore.shutdown();
262 }
263
264 public long heapSize() {
265 return this.size.get() + backingStore.heapSize();
266 }
267
268 public long size() {
269 return (long) this.blockSize * (long) this.numBlocks;
270 }
271
272 public long getFreeSize() {
273 return (long) backingStore.getBlocksRemaining() * (long) blockSize;
274 }
275
276 public long getOccupiedSize() {
277 return (long) (numBlocks - backingStore.getBlocksRemaining()) * (long) blockSize;
278 }
279
280 public long getEvictedCount() {
281 return stats.getEvictedCount();
282 }
283
284 public CacheStats getStats() {
285 return this.stats;
286 }
287
288 @Override
289 public long getBlockCount() {
290 return numBlocks - backingStore.getBlocksRemaining();
291 }
292
293
294 @Override
295 public void cacheBlock(BlockCacheKey cacheKey, Cacheable buf, boolean inMemory) {
296 this.cacheBlock(cacheKey, buf);
297 }
298
299
300
301
302
303 @Override
304 public int evictBlocksByHfileName(String hfileName) {
305 int evictedCount = 0;
306 for (BlockCacheKey e : backingMap.keySet()) {
307 if (e.getHfileName().equals(hfileName)) {
308 this.evictBlock(e);
309 }
310 }
311 return evictedCount;
312 }
313
314 @Override
315 public long getCurrentSize() {
316 return 0;
317 }
318
319
320
321
322
323 @Override
324 public List<BlockCacheColumnFamilySummary> getBlockCacheColumnFamilySummaries(
325 Configuration conf) {
326 throw new UnsupportedOperationException();
327 }
328
329
330 private static class CacheablePair implements HeapSize {
331 final CacheableDeserializer<Cacheable> deserializer;
332 ByteBuffer serializedData;
333 AtomicLong recentlyAccessed;
334
335 private CacheablePair(CacheableDeserializer<Cacheable> deserializer,
336 ByteBuffer serializedData) {
337 this.recentlyAccessed = new AtomicLong();
338 this.deserializer = deserializer;
339 this.serializedData = serializedData;
340 }
341
342
343
344
345
346
347 @Override
348 public long heapSize() {
349 return ClassSize.align(ClassSize.OBJECT + ClassSize.REFERENCE * 3
350 + ClassSize.ATOMIC_LONG);
351 }
352 }
353 }