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
96 RemovalListener<BlockCacheKey, CacheablePair> listener =
97 new RemovalListener<BlockCacheKey, CacheablePair>() {
98 @Override
99 public void onRemoval(
100 RemovalNotification<BlockCacheKey, CacheablePair> notification) {
101 if (!notification.wasEvicted()) {
102
103
104 return;
105 }
106 CacheablePair value = notification.getValue();
107 timeSinceLastAccess.set(System.nanoTime()
108 - value.recentlyAccessed.get());
109 stats.evict();
110 doEviction(notification.getKey(), value);
111 }
112 };
113
114 backingMap = CacheBuilder.newBuilder()
115 .maximumSize(numBlocks - 1)
116 .removalListener(listener)
117 .<BlockCacheKey, CacheablePair>build()
118 .asMap();
119
120
121 }
122
123 @Override
124 public void cacheBlock(BlockCacheKey blockName, Cacheable toBeCached) {
125 ByteBuffer storedBlock;
126
127 try {
128 storedBlock = backingStore.alloc(toBeCached.getSerializedLength());
129 } catch (InterruptedException e) {
130 LOG.warn("SlabAllocator was interrupted while waiting for block to become available");
131 LOG.warn(e);
132 return;
133 }
134
135 CacheablePair newEntry = new CacheablePair(toBeCached.getDeserializer(),
136 storedBlock);
137 toBeCached.serialize(storedBlock);
138
139 synchronized (this) {
140 CacheablePair alreadyCached = backingMap.putIfAbsent(blockName, newEntry);
141
142
143 if (alreadyCached != null) {
144 backingStore.free(storedBlock);
145 throw new RuntimeException("already cached " + blockName);
146 }
147 if (actionWatcher != null) {
148 actionWatcher.onInsertion(blockName, this);
149 }
150 }
151 newEntry.recentlyAccessed.set(System.nanoTime());
152 this.size.addAndGet(newEntry.heapSize());
153 }
154
155 @Override
156 public Cacheable getBlock(BlockCacheKey key, boolean caching, boolean repeat) {
157 CacheablePair contentBlock = backingMap.get(key);
158 if (contentBlock == null) {
159 if (!repeat) stats.miss(caching);
160 return null;
161 }
162
163 stats.hit(caching);
164
165 try {
166 contentBlock.recentlyAccessed.set(System.nanoTime());
167 synchronized (contentBlock) {
168 if (contentBlock.serializedData == null) {
169
170 LOG.warn("Concurrent eviction of " + key);
171 return null;
172 }
173 return contentBlock.deserializer
174 .deserialize(contentBlock.serializedData.asReadOnlyBuffer());
175 }
176 } catch (Throwable t) {
177 LOG.error("Deserializer threw an exception. This may indicate a bug.", t);
178 return null;
179 }
180 }
181
182
183
184
185
186
187
188 public boolean evictBlock(BlockCacheKey key) {
189 stats.evict();
190 CacheablePair evictedBlock = backingMap.remove(key);
191
192 if (evictedBlock != null) {
193 doEviction(key, evictedBlock);
194 }
195 return evictedBlock != null;
196
197 }
198
199 private void doEviction(BlockCacheKey key, CacheablePair evictedBlock) {
200 long evictedHeap = 0;
201 synchronized (evictedBlock) {
202 if (evictedBlock.serializedData == null) {
203
204 return;
205 }
206 evictedHeap = evictedBlock.heapSize();
207 ByteBuffer bb = evictedBlock.serializedData;
208 evictedBlock.serializedData = null;
209 backingStore.free(bb);
210
211
212
213
214
215
216
217
218
219
220
221
222 if (actionWatcher != null) {
223 actionWatcher.onEviction(key, this);
224 }
225 }
226 stats.evicted();
227 size.addAndGet(-1 * evictedHeap);
228 }
229
230 public void logStats() {
231
232 long milliseconds = this.timeSinceLastAccess.get() / 1000000;
233
234 LOG.info("For Slab of size " + this.blockSize + ": "
235 + this.getOccupiedSize() / this.blockSize
236 + " occupied, out of a capacity of " + this.numBlocks
237 + " blocks. HeapSize is "
238 + StringUtils.humanReadableInt(this.heapSize()) + " bytes." + ", "
239 + "churnTime=" + StringUtils.formatTime(milliseconds));
240
241 LOG.info("Slab Stats: " + "accesses="
242 + stats.getRequestCount()
243 + ", "
244 + "hits="
245 + stats.getHitCount()
246 + ", "
247 + "hitRatio="
248 + (stats.getHitCount() == 0 ? "0" : (StringUtils.formatPercent(
249 stats.getHitRatio(), 2) + "%, "))
250 + "cachingAccesses="
251 + stats.getRequestCachingCount()
252 + ", "
253 + "cachingHits="
254 + stats.getHitCachingCount()
255 + ", "
256 + "cachingHitsRatio="
257 + (stats.getHitCachingCount() == 0 ? "0" : (StringUtils.formatPercent(
258 stats.getHitCachingRatio(), 2) + "%, ")) + "evictions="
259 + stats.getEvictionCount() + ", " + "evicted="
260 + stats.getEvictedCount() + ", " + "evictedPerRun="
261 + stats.evictedPerEviction());
262
263 }
264
265 public void shutdown() {
266 backingStore.shutdown();
267 }
268
269 public long heapSize() {
270 return this.size.get() + backingStore.heapSize();
271 }
272
273 public long size() {
274 return (long) this.blockSize * (long) this.numBlocks;
275 }
276
277 public long getFreeSize() {
278 return (long) backingStore.getBlocksRemaining() * (long) blockSize;
279 }
280
281 public long getOccupiedSize() {
282 return (long) (numBlocks - backingStore.getBlocksRemaining()) * (long) blockSize;
283 }
284
285 public long getEvictedCount() {
286 return stats.getEvictedCount();
287 }
288
289 public CacheStats getStats() {
290 return this.stats;
291 }
292
293 @Override
294 public long getBlockCount() {
295 return numBlocks - backingStore.getBlocksRemaining();
296 }
297
298
299 @Override
300 public void cacheBlock(BlockCacheKey cacheKey, Cacheable buf, boolean inMemory) {
301 this.cacheBlock(cacheKey, buf);
302 }
303
304
305
306
307
308 @Override
309 public int evictBlocksByHfileName(String hfileName) {
310 int evictedCount = 0;
311 for (BlockCacheKey e : backingMap.keySet()) {
312 if (e.getHfileName().equals(hfileName)) {
313 this.evictBlock(e);
314 }
315 }
316 return evictedCount;
317 }
318
319 @Override
320 public long getCurrentSize() {
321 return 0;
322 }
323
324
325
326
327
328 @Override
329 public List<BlockCacheColumnFamilySummary> getBlockCacheColumnFamilySummaries(
330 Configuration conf) {
331 throw new UnsupportedOperationException();
332 }
333
334
335 private static class CacheablePair implements HeapSize {
336 final CacheableDeserializer<Cacheable> deserializer;
337 ByteBuffer serializedData;
338 AtomicLong recentlyAccessed;
339
340 private CacheablePair(CacheableDeserializer<Cacheable> deserializer,
341 ByteBuffer serializedData) {
342 this.recentlyAccessed = new AtomicLong();
343 this.deserializer = deserializer;
344 this.serializedData = serializedData;
345 }
346
347
348
349
350
351
352 @Override
353 public long heapSize() {
354 return ClassSize.align(ClassSize.OBJECT + ClassSize.REFERENCE * 3
355 + ClassSize.ATOMIC_LONG);
356 }
357 }
358 }