1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18 package org.apache.hadoop.hbase.io.hfile;
19
20 import static org.junit.Assert.*;
21
22 import java.io.IOException;
23 import java.util.Map;
24 import java.util.NavigableSet;
25
26 import org.apache.commons.logging.Log;
27 import org.apache.commons.logging.LogFactory;
28 import org.apache.hadoop.conf.Configuration;
29 import org.apache.hadoop.hbase.HBaseConfiguration;
30 import org.apache.hadoop.hbase.HConstants;
31 import org.apache.hadoop.hbase.testclassification.SmallTests;
32 import org.apache.hadoop.hbase.io.hfile.TestCacheConfig.DataCacheEntry;
33 import org.apache.hadoop.hbase.io.hfile.TestCacheConfig.IndexCacheEntry;
34 import org.codehaus.jackson.JsonGenerationException;
35 import org.codehaus.jackson.map.JsonMappingException;
36 import org.junit.After;
37 import org.junit.Before;
38 import org.junit.Test;
39 import org.junit.experimental.categories.Category;
40
41 @Category(SmallTests.class)
42 public class TestBlockCacheReporting {
43 private static final Log LOG = LogFactory.getLog(TestBlockCacheReporting.class);
44 private Configuration conf;
45
46 @Before
47 public void setUp() throws Exception {
48 CacheConfig.GLOBAL_BLOCK_CACHE_INSTANCE = null;
49 this.conf = HBaseConfiguration.create();
50 }
51
52 @After
53 public void tearDown() throws Exception {
54
55 CacheConfig.GLOBAL_BLOCK_CACHE_INSTANCE = null;
56 }
57
58 private void addDataAndHits(final BlockCache bc, final int count) {
59 Cacheable dce = new DataCacheEntry();
60 Cacheable ice = new IndexCacheEntry();
61 for (int i = 0; i < count; i++) {
62 BlockCacheKey bckd = new BlockCacheKey("f", i);
63 BlockCacheKey bcki = new BlockCacheKey("f", i + count);
64 bc.getBlock(bckd, true, false, true);
65 bc.cacheBlock(bckd, dce);
66 bc.cacheBlock(bcki, ice);
67 bc.getBlock(bckd, true, false, true);
68 bc.getBlock(bcki, true, false, true);
69 }
70 assertEquals(2 * count
71 BlockCacheKey bckd = new BlockCacheKey("f", 0);
72 BlockCacheKey bcki = new BlockCacheKey("f", 0 + count);
73 bc.evictBlock(bckd);
74 bc.evictBlock(bcki);
75 bc.getStats().getEvictedCount();
76 }
77
78 @Test
79 public void testBucketCache() throws JsonGenerationException, JsonMappingException, IOException {
80 this.conf.set(HConstants.BUCKET_CACHE_IOENGINE_KEY, "offheap");
81 this.conf.setInt(HConstants.BUCKET_CACHE_SIZE_KEY, 100);
82 CacheConfig cc = new CacheConfig(this.conf);
83 assertTrue(cc.getBlockCache() instanceof CombinedBlockCache);
84 logPerBlock(cc.getBlockCache());
85 final int count = 3;
86 addDataAndHits(cc.getBlockCache(), count);
87
88 LOG.info(cc.getBlockCache().getStats());
89 BlockCacheUtil.CachedBlocksByFile cbsbf = logPerBlock(cc.getBlockCache());
90 LOG.info(cbsbf);
91 logPerFile(cbsbf);
92 bucketCacheReport(cc.getBlockCache());
93 LOG.info(BlockCacheUtil.toJSON(cbsbf));
94 }
95
96 @Test
97 public void testLruBlockCache() throws JsonGenerationException, JsonMappingException, IOException {
98 CacheConfig cc = new CacheConfig(this.conf);
99 assertTrue(cc.isBlockCacheEnabled());
100 assertTrue(CacheConfig.DEFAULT_IN_MEMORY == cc.isInMemory());
101 assertTrue(cc.getBlockCache() instanceof LruBlockCache);
102 logPerBlock(cc.getBlockCache());
103 addDataAndHits(cc.getBlockCache(), 3);
104
105 BlockCache bc = cc.getBlockCache();
106 LOG.info("count=" + bc.getBlockCount() + ", currentSize=" + bc.getCurrentSize() +
107 ", freeSize=" + bc.getFreeSize() );
108 LOG.info(cc.getBlockCache().getStats());
109 BlockCacheUtil.CachedBlocksByFile cbsbf = logPerBlock(cc.getBlockCache());
110 LOG.info(cbsbf);
111 logPerFile(cbsbf);
112 bucketCacheReport(cc.getBlockCache());
113 LOG.info(BlockCacheUtil.toJSON(cbsbf));
114 }
115
116 private void bucketCacheReport(final BlockCache bc) {
117 LOG.info(bc.getClass().getSimpleName() + ": " + bc.getStats());
118 BlockCache [] bcs = bc.getBlockCaches();
119 if (bcs != null) {
120 for (BlockCache sbc: bc.getBlockCaches()) {
121 bucketCacheReport(sbc);
122 }
123 }
124 }
125
126 private void logPerFile(final BlockCacheUtil.CachedBlocksByFile cbsbf)
127 throws JsonGenerationException, JsonMappingException, IOException {
128 for (Map.Entry<String, NavigableSet<CachedBlock>> e:
129 cbsbf.getCachedBlockStatsByFile().entrySet()) {
130 int count = 0;
131 long size = 0;
132 int countData = 0;
133 long sizeData = 0;
134 for (CachedBlock cb: e.getValue()) {
135 count++;
136 size += cb.getSize();
137 BlockType bt = cb.getBlockType();
138 if (bt != null && bt.isData()) {
139 countData++;
140 sizeData += cb.getSize();
141 }
142 }
143 LOG.info("filename=" + e.getKey() + ", count=" + count + ", countData=" + countData +
144 ", size=" + size + ", sizeData=" + sizeData);
145 LOG.info(BlockCacheUtil.toJSON(e.getKey(), e.getValue()));
146 }
147 }
148
149 private BlockCacheUtil.CachedBlocksByFile logPerBlock(final BlockCache bc)
150 throws JsonGenerationException, JsonMappingException, IOException {
151 BlockCacheUtil.CachedBlocksByFile cbsbf = new BlockCacheUtil.CachedBlocksByFile();
152 for (CachedBlock cb: bc) {
153 LOG.info(cb.toString());
154 LOG.info(BlockCacheUtil.toJSON(bc));
155 cbsbf.update(cb);
156 }
157 return cbsbf;
158 }
159 }