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