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.math.BigDecimal;
23 import java.util.List;
24 import java.util.Map.Entry;
25 import java.util.TreeMap;
26 import java.util.concurrent.ConcurrentHashMap;
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.classification.InterfaceAudience;
35 import org.apache.hadoop.conf.Configuration;
36 import org.apache.hadoop.hbase.io.HeapSize;
37 import org.apache.hadoop.hbase.io.hfile.BlockCache;
38 import org.apache.hadoop.hbase.io.hfile.BlockCacheColumnFamilySummary;
39 import org.apache.hadoop.hbase.io.hfile.BlockCacheKey;
40 import org.apache.hadoop.hbase.io.hfile.CacheStats;
41 import org.apache.hadoop.hbase.io.hfile.Cacheable;
42 import org.apache.hadoop.hbase.util.ClassSize;
43 import org.apache.hadoop.hbase.util.HasThread;
44 import org.apache.hadoop.util.StringUtils;
45
46 import com.google.common.base.Preconditions;
47 import com.google.common.util.concurrent.ThreadFactoryBuilder;
48
49
50
51
52
53
54
55 @InterfaceAudience.Private
56 public class SlabCache implements SlabItemActionWatcher, BlockCache, HeapSize {
57
58 private final ConcurrentHashMap<BlockCacheKey, SingleSizeCache> backingStore;
59 private final TreeMap<Integer, SingleSizeCache> sizer;
60 static final Log LOG = LogFactory.getLog(SlabCache.class);
61 static final int STAT_THREAD_PERIOD_SECS = 60 * 5;
62
63 private final ScheduledExecutorService scheduleThreadPool = Executors.newScheduledThreadPool(1,
64 new ThreadFactoryBuilder().setDaemon(true).setNameFormat("Slab Statistics #%d").build());
65
66 long size;
67 private final CacheStats stats;
68 final SlabStats requestStats;
69 final SlabStats successfullyCachedStats;
70 private final long avgBlockSize;
71 private static final long CACHE_FIXED_OVERHEAD = ClassSize.estimateBase(
72 SlabCache.class, false);
73
74
75
76
77
78
79
80
81 public SlabCache(long size, long avgBlockSize) {
82 this.avgBlockSize = avgBlockSize;
83 this.size = size;
84 this.stats = new CacheStats();
85 this.requestStats = new SlabStats();
86 this.successfullyCachedStats = new SlabStats();
87
88 backingStore = new ConcurrentHashMap<BlockCacheKey, SingleSizeCache>();
89 sizer = new TreeMap<Integer, SingleSizeCache>();
90 this.scheduleThreadPool.scheduleAtFixedRate(new StatisticsThread(this),
91 STAT_THREAD_PERIOD_SECS, STAT_THREAD_PERIOD_SECS, TimeUnit.SECONDS);
92
93 }
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109 public void addSlabByConf(Configuration conf) {
110
111 String[] porportions = conf.getStrings(
112 "hbase.offheapcache.slab.proportions", "0.80", "0.20");
113 String[] sizes = conf.getStrings("hbase.offheapcache.slab.sizes",
114 Long.valueOf(avgBlockSize * 11 / 10).toString(),
115 Long.valueOf(avgBlockSize * 21 / 10).toString());
116
117 if (porportions.length != sizes.length) {
118 throw new IllegalArgumentException(
119 "SlabCache conf not "
120 + "initialized, error in configuration. hbase.offheap.slab.proportions specifies "
121 + porportions.length
122 + " slabs while hbase.offheap.slab.sizes specifies "
123 + sizes.length + " slabs "
124 + "offheapslabporportions and offheapslabsizes");
125 }
126
127
128
129
130 BigDecimal[] parsedProportions = stringArrayToBigDecimalArray(porportions);
131 BigDecimal[] parsedSizes = stringArrayToBigDecimalArray(sizes);
132
133 BigDecimal sumProportions = new BigDecimal(0);
134 for (BigDecimal b : parsedProportions) {
135
136 Preconditions
137 .checkArgument(b.compareTo(BigDecimal.ZERO) == 1,
138 "Proportions in hbase.offheap.slab.proportions must be greater than 0!");
139 sumProportions = sumProportions.add(b);
140 }
141
142
143 Preconditions
144 .checkArgument(sumProportions.compareTo(BigDecimal.ONE) != 1,
145 "Sum of all proportions in hbase.offheap.slab.proportions must be less than 1");
146
147
148 if (sumProportions.compareTo(new BigDecimal("0.99")) == -1) {
149 LOG.warn("Sum of hbase.offheap.slab.proportions is less than 0.99! Memory is being wasted");
150 }
151 for (int i = 0; i < parsedProportions.length; i++) {
152 int blockSize = parsedSizes[i].intValue();
153 int numBlocks = new BigDecimal(this.size).multiply(parsedProportions[i])
154 .divide(parsedSizes[i], BigDecimal.ROUND_DOWN).intValue();
155 addSlab(blockSize, numBlocks);
156 }
157 }
158
159
160
161
162
163
164
165
166
167
168 Entry<Integer, SingleSizeCache> getHigherBlock(int size) {
169 return sizer.higherEntry(size - 1);
170 }
171
172 private BigDecimal[] stringArrayToBigDecimalArray(String[] parsee) {
173 BigDecimal[] parsed = new BigDecimal[parsee.length];
174 for (int i = 0; i < parsee.length; i++) {
175 parsed[i] = new BigDecimal(parsee[i].trim());
176 }
177 return parsed;
178 }
179
180 private void addSlab(int blockSize, int numBlocks) {
181 LOG.info("Creating a slab of blockSize " + blockSize + " with " + numBlocks
182 + " blocks, " + StringUtils.humanReadableInt(blockSize * (long) numBlocks) + "bytes.");
183 sizer.put(blockSize, new SingleSizeCache(blockSize, numBlocks, this));
184 }
185
186
187
188
189
190
191
192
193
194
195
196
197
198 public void cacheBlock(BlockCacheKey cacheKey, Cacheable cachedItem) {
199 Entry<Integer, SingleSizeCache> scacheEntry = getHigherBlock(cachedItem
200 .getSerializedLength());
201
202 this.requestStats.addin(cachedItem.getSerializedLength());
203
204 if (scacheEntry == null) {
205 return;
206 }
207
208 this.successfullyCachedStats.addin(cachedItem.getSerializedLength());
209 SingleSizeCache scache = scacheEntry.getValue();
210
211
212
213
214
215 scache.cacheBlock(cacheKey, cachedItem);
216 }
217
218
219
220
221
222 public void cacheBlock(BlockCacheKey cacheKey, Cacheable buf, boolean inMemory) {
223 cacheBlock(cacheKey, buf);
224 }
225
226 public CacheStats getStats() {
227 return this.stats;
228 }
229
230
231
232
233
234
235 public Cacheable getBlock(BlockCacheKey key, boolean caching, boolean repeat,
236 boolean updateCacheMetrics) {
237 SingleSizeCache cachedBlock = backingStore.get(key);
238 if (cachedBlock == null) {
239 if (!repeat) stats.miss(caching);
240 return null;
241 }
242
243 Cacheable contentBlock = cachedBlock.getBlock(key, caching, false, updateCacheMetrics);
244
245 if (contentBlock != null) {
246 if (updateCacheMetrics) stats.hit(caching);
247 } else if (!repeat) {
248 if (updateCacheMetrics) stats.miss(caching);
249 }
250 return contentBlock;
251 }
252
253
254
255
256
257 public boolean evictBlock(BlockCacheKey cacheKey) {
258 SingleSizeCache cacheEntry = backingStore.get(cacheKey);
259 if (cacheEntry == null) {
260 return false;
261 } else {
262 cacheEntry.evictBlock(cacheKey);
263 return true;
264 }
265 }
266
267 @Override
268 public void onEviction(BlockCacheKey key, SingleSizeCache notifier) {
269 stats.evicted();
270 backingStore.remove(key);
271 }
272
273 @Override
274 public void onInsertion(BlockCacheKey key, SingleSizeCache notifier) {
275 backingStore.put(key, notifier);
276 }
277
278
279
280
281
282
283 public void shutdown() {
284 for (SingleSizeCache s : sizer.values()) {
285 s.shutdown();
286 }
287 this.scheduleThreadPool.shutdown();
288 }
289
290 public long heapSize() {
291 long childCacheSize = 0;
292 for (SingleSizeCache s : sizer.values()) {
293 childCacheSize += s.heapSize();
294 }
295 return SlabCache.CACHE_FIXED_OVERHEAD + childCacheSize;
296 }
297
298 public long size() {
299 return this.size;
300 }
301
302 public long getFreeSize() {
303 long childFreeSize = 0;
304 for (SingleSizeCache s : sizer.values()) {
305 childFreeSize += s.getFreeSize();
306 }
307 return childFreeSize;
308 }
309
310 @Override
311 public long getBlockCount() {
312 long count = 0;
313 for (SingleSizeCache cache : sizer.values()) {
314 count += cache.getBlockCount();
315 }
316 return count;
317 }
318
319 public long getCurrentSize() {
320 return size;
321 }
322
323 public long getEvictedCount() {
324 return stats.getEvictedCount();
325 }
326
327
328
329
330 static class StatisticsThread extends HasThread {
331 SlabCache ourcache;
332
333 public StatisticsThread(SlabCache slabCache) {
334 super("SlabCache.StatisticsThread");
335 setDaemon(true);
336 this.ourcache = slabCache;
337 }
338
339 @Override
340 public void run() {
341 for (SingleSizeCache s : ourcache.sizer.values()) {
342 s.logStats();
343 }
344
345 SlabCache.LOG.info("Current heap size is: "
346 + StringUtils.humanReadableInt(ourcache.heapSize()));
347
348 LOG.info("Request Stats");
349 ourcache.requestStats.logStats();
350 LOG.info("Successfully Cached Stats");
351 ourcache.successfullyCachedStats.logStats();
352 }
353
354 }
355
356
357
358
359
360
361 static class SlabStats {
362
363
364
365 static final int MULTIPLIER = 10;
366 final int NUMDIVISIONS = (int) (Math.log(Integer.MAX_VALUE) * MULTIPLIER);
367 private final AtomicLong[] counts = new AtomicLong[NUMDIVISIONS];
368
369 public SlabStats() {
370 for (int i = 0; i < NUMDIVISIONS; i++) {
371 counts[i] = new AtomicLong();
372 }
373 }
374
375 public void addin(int size) {
376 int index = (int) (Math.log(size) * MULTIPLIER);
377 counts[index].incrementAndGet();
378 }
379
380 public AtomicLong[] getUsage() {
381 return counts;
382 }
383
384 double getUpperBound(int index) {
385 return Math.pow(Math.E, ((index + 0.5) / MULTIPLIER));
386 }
387
388 double getLowerBound(int index) {
389 return Math.pow(Math.E, ((index - 0.5) / MULTIPLIER));
390 }
391
392 public void logStats() {
393 AtomicLong[] fineGrainedStats = getUsage();
394 for (int i = 0; i < fineGrainedStats.length; i++) {
395
396 if (fineGrainedStats[i].get() > 0) {
397 SlabCache.LOG.info("From "
398 + StringUtils.humanReadableInt((long) getLowerBound(i)) + "- "
399 + StringUtils.humanReadableInt((long) getUpperBound(i)) + ": "
400 + StringUtils.humanReadableInt(fineGrainedStats[i].get())
401 + " requests");
402
403 }
404 }
405 }
406 }
407
408 public int evictBlocksByHfileName(String hfileName) {
409 int numEvicted = 0;
410 for (BlockCacheKey key : backingStore.keySet()) {
411 if (key.getHfileName().equals(hfileName)) {
412 if (evictBlock(key))
413 ++numEvicted;
414 }
415 }
416 return numEvicted;
417 }
418
419
420
421
422
423 @Override
424 public List<BlockCacheColumnFamilySummary> getBlockCacheColumnFamilySummaries(
425 Configuration conf) {
426 throw new UnsupportedOperationException();
427 }
428
429 }