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.");
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
236
237 public Cacheable getBlock(BlockCacheKey key, boolean caching, boolean repeat) {
238 SingleSizeCache cachedBlock = backingStore.get(key);
239 if (cachedBlock == null) {
240 if (!repeat) stats.miss(caching);
241 return null;
242 }
243
244 Cacheable contentBlock = cachedBlock.getBlock(key, caching, false);
245
246 if (contentBlock != null) {
247 stats.hit(caching);
248 } else if (!repeat) {
249 stats.miss(caching);
250 }
251 return contentBlock;
252 }
253
254
255
256
257
258 public boolean evictBlock(BlockCacheKey cacheKey) {
259 SingleSizeCache cacheEntry = backingStore.get(cacheKey);
260 if (cacheEntry == null) {
261 return false;
262 } else {
263 cacheEntry.evictBlock(cacheKey);
264 return true;
265 }
266 }
267
268 @Override
269 public void onEviction(BlockCacheKey key, SingleSizeCache notifier) {
270 stats.evicted();
271 backingStore.remove(key);
272 }
273
274 @Override
275 public void onInsertion(BlockCacheKey key, SingleSizeCache notifier) {
276 backingStore.put(key, notifier);
277 }
278
279
280
281
282
283
284 public void shutdown() {
285 for (SingleSizeCache s : sizer.values()) {
286 s.shutdown();
287 }
288 this.scheduleThreadPool.shutdown();
289 }
290
291 public long heapSize() {
292 long childCacheSize = 0;
293 for (SingleSizeCache s : sizer.values()) {
294 childCacheSize += s.heapSize();
295 }
296 return SlabCache.CACHE_FIXED_OVERHEAD + childCacheSize;
297 }
298
299 public long size() {
300 return this.size;
301 }
302
303 public long getFreeSize() {
304 return 0;
305 }
306
307 @Override
308 public long getBlockCount() {
309 long count = 0;
310 for (SingleSizeCache cache : backingStore.values()) {
311 count += cache.getBlockCount();
312 }
313 return count;
314 }
315
316 public long getCurrentSize() {
317 return size;
318 }
319
320 public long getEvictedCount() {
321 return stats.getEvictedCount();
322 }
323
324
325
326
327 static class StatisticsThread extends HasThread {
328 SlabCache ourcache;
329
330 public StatisticsThread(SlabCache slabCache) {
331 super("SlabCache.StatisticsThread");
332 setDaemon(true);
333 this.ourcache = slabCache;
334 }
335
336 @Override
337 public void run() {
338 for (SingleSizeCache s : ourcache.sizer.values()) {
339 s.logStats();
340 }
341
342 SlabCache.LOG.info("Current heap size is: "
343 + StringUtils.humanReadableInt(ourcache.heapSize()));
344
345 LOG.info("Request Stats");
346 ourcache.requestStats.logStats();
347 LOG.info("Successfully Cached Stats");
348 ourcache.successfullyCachedStats.logStats();
349 }
350
351 }
352
353
354
355
356
357
358 static class SlabStats {
359
360
361
362 static final int MULTIPLIER = 10;
363 final int NUMDIVISIONS = (int) (Math.log(Integer.MAX_VALUE) * MULTIPLIER);
364 private final AtomicLong[] counts = new AtomicLong[NUMDIVISIONS];
365
366 public SlabStats() {
367 for (int i = 0; i < NUMDIVISIONS; i++) {
368 counts[i] = new AtomicLong();
369 }
370 }
371
372 public void addin(int size) {
373 int index = (int) (Math.log(size) * MULTIPLIER);
374 counts[index].incrementAndGet();
375 }
376
377 public AtomicLong[] getUsage() {
378 return counts;
379 }
380
381 double getUpperBound(int index) {
382 return Math.pow(Math.E, ((index + 0.5) / MULTIPLIER));
383 }
384
385 double getLowerBound(int index) {
386 return Math.pow(Math.E, ((index - 0.5) / MULTIPLIER));
387 }
388
389 public void logStats() {
390 AtomicLong[] fineGrainedStats = getUsage();
391 for (int i = 0; i < fineGrainedStats.length; i++) {
392
393 if (fineGrainedStats[i].get() > 0) {
394 SlabCache.LOG.info("From "
395 + StringUtils.humanReadableInt((long) getLowerBound(i)) + "- "
396 + StringUtils.humanReadableInt((long) getUpperBound(i)) + ": "
397 + StringUtils.humanReadableInt(fineGrainedStats[i].get())
398 + " requests");
399
400 }
401 }
402 }
403 }
404
405 public int evictBlocksByHfileName(String hfileName) {
406 int numEvicted = 0;
407 for (BlockCacheKey key : backingStore.keySet()) {
408 if (key.getHfileName().equals(hfileName)) {
409 if (evictBlock(key))
410 ++numEvicted;
411 }
412 }
413 return numEvicted;
414 }
415
416
417
418
419
420 @Override
421 public List<BlockCacheColumnFamilySummary> getBlockCacheColumnFamilySummaries(
422 Configuration conf) {
423 throw new UnsupportedOperationException();
424 }
425
426 }