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