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;
20
21 import static org.junit.Assert.assertEquals;
22 import static org.junit.Assert.assertTrue;
23
24 import java.io.IOException;
25 import java.lang.management.ManagementFactory;
26 import java.lang.management.MemoryUsage;
27 import java.nio.ByteBuffer;
28 import java.util.Map;
29
30 import org.apache.commons.logging.Log;
31 import org.apache.commons.logging.LogFactory;
32 import org.apache.hadoop.conf.Configuration;
33 import org.apache.hadoop.fs.FileSystem;
34 import org.apache.hadoop.fs.Path;
35 import org.apache.hadoop.hbase.HBaseConfiguration;
36 import org.apache.hadoop.hbase.HBaseTestingUtility;
37 import org.apache.hadoop.hbase.HConstants;
38 import org.apache.hadoop.hbase.testclassification.LargeTests;
39 import org.apache.hadoop.hbase.io.hfile.bucket.BucketCache;
40 import org.apache.hadoop.hbase.util.Threads;
41 import org.junit.After;
42 import org.junit.Before;
43 import org.junit.Test;
44 import org.junit.experimental.categories.Category;
45
46
47
48
49
50
51
52
53 @Category(LargeTests.class)
54 public class TestCacheConfig {
55 private static final Log LOG = LogFactory.getLog(TestCacheConfig.class);
56 private Configuration conf;
57
58 static class Deserializer implements CacheableDeserializer<Cacheable> {
59 private final Cacheable cacheable;
60 private int deserializedIdentifier = 0;
61
62 Deserializer(final Cacheable c) {
63 deserializedIdentifier = CacheableDeserializerIdManager.registerDeserializer(this);
64 this.cacheable = c;
65 }
66
67 @Override
68 public int getDeserialiserIdentifier() {
69 return deserializedIdentifier;
70 }
71
72 @Override
73 public Cacheable deserialize(ByteBuffer b, boolean reuse) throws IOException {
74 LOG.info("Deserialized " + b + ", reuse=" + reuse);
75 return cacheable;
76 }
77
78 @Override
79 public Cacheable deserialize(ByteBuffer b) throws IOException {
80 LOG.info("Deserialized " + b);
81 return cacheable;
82 }
83 };
84
85 static class IndexCacheEntry extends DataCacheEntry {
86 private static IndexCacheEntry SINGLETON = new IndexCacheEntry();
87
88 public IndexCacheEntry() {
89 super(SINGLETON);
90 }
91
92 @Override
93 public BlockType getBlockType() {
94 return BlockType.ROOT_INDEX;
95 }
96 }
97
98 static class DataCacheEntry implements Cacheable {
99 private static final int SIZE = 1;
100 private static DataCacheEntry SINGLETON = new DataCacheEntry();
101 final CacheableDeserializer<Cacheable> deserializer;
102
103 DataCacheEntry() {
104 this(SINGLETON);
105 }
106
107 DataCacheEntry(final Cacheable c) {
108 this.deserializer = new Deserializer(c);
109 }
110
111 @Override
112 public String toString() {
113 return "size=" + SIZE + ", type=" + getBlockType();
114 };
115
116 @Override
117 public long heapSize() {
118 return SIZE;
119 }
120
121 @Override
122 public int getSerializedLength() {
123 return SIZE;
124 }
125
126 @Override
127 public void serialize(ByteBuffer destination) {
128 LOG.info("Serialized " + this + " to " + destination);
129 }
130
131 @Override
132 public CacheableDeserializer<Cacheable> getDeserializer() {
133 return this.deserializer;
134 }
135
136 @Override
137 public BlockType getBlockType() {
138 return BlockType.DATA;
139 }
140 };
141
142 static class MetaCacheEntry extends DataCacheEntry {
143 @Override
144 public BlockType getBlockType() {
145 return BlockType.INTERMEDIATE_INDEX;
146 }
147 }
148
149 @Before
150 public void setUp() throws Exception {
151 CacheConfig.GLOBAL_BLOCK_CACHE_INSTANCE = null;
152 this.conf = HBaseConfiguration.create();
153 }
154
155 @After
156 public void tearDown() throws Exception {
157
158 CacheConfig.GLOBAL_BLOCK_CACHE_INSTANCE = null;
159 }
160
161
162
163
164
165
166
167 void basicBlockCacheOps(final CacheConfig cc, final boolean doubling,
168 final boolean sizing) {
169 assertTrue(cc.isBlockCacheEnabled());
170 assertTrue(CacheConfig.DEFAULT_IN_MEMORY == cc.isInMemory());
171 BlockCache bc = cc.getBlockCache();
172 BlockCacheKey bck = new BlockCacheKey("f", 0);
173 Cacheable c = new DataCacheEntry();
174
175 long initialBlockCount = bc.getBlockCount();
176 bc.cacheBlock(bck, c, cc.isInMemory(), cc.isCacheDataInL1());
177 assertEquals(doubling? 2: 1, bc.getBlockCount() - initialBlockCount);
178 bc.evictBlock(bck);
179 assertEquals(initialBlockCount, bc.getBlockCount());
180
181
182 if (sizing) {
183 long originalSize = bc.getCurrentSize();
184 bc.cacheBlock(bck, c, cc.isInMemory(), cc.isCacheDataInL1());
185 assertTrue(bc.getCurrentSize() > originalSize);
186 bc.evictBlock(bck);
187 long size = bc.getCurrentSize();
188 assertEquals(originalSize, size);
189 }
190 }
191
192
193
194
195
196
197 private long cacheDataBlock(final CacheConfig cc, final String filename) {
198 BlockCacheKey bck = new BlockCacheKey(filename, 0);
199 Cacheable c = new DataCacheEntry();
200
201 cc.getBlockCache().cacheBlock(bck, c, cc.isInMemory(), cc.isCacheDataInL1());
202 return cc.getBlockCache().getBlockCount();
203 }
204
205 @Test
206 public void testCacheConfigDefaultLRUBlockCache() {
207 CacheConfig cc = new CacheConfig(this.conf);
208 assertTrue(cc.isBlockCacheEnabled());
209 assertTrue(CacheConfig.DEFAULT_IN_MEMORY == cc.isInMemory());
210 basicBlockCacheOps(cc, false, true);
211 assertTrue(cc.getBlockCache() instanceof LruBlockCache);
212 }
213
214
215
216
217 @Test
218 public void testOffHeapBucketCacheConfig() {
219 this.conf.set(HConstants.BUCKET_CACHE_IOENGINE_KEY, "offheap");
220 doBucketCacheConfigTest();
221 }
222
223 @Test
224 public void testOnHeapBucketCacheConfig() {
225 this.conf.set(HConstants.BUCKET_CACHE_IOENGINE_KEY, "heap");
226 doBucketCacheConfigTest();
227 }
228
229 @Test
230 public void testFileBucketCacheConfig() throws IOException {
231 HBaseTestingUtility htu = new HBaseTestingUtility(this.conf);
232 try {
233 Path p = new Path(htu.getDataTestDir(), "bc.txt");
234 FileSystem fs = FileSystem.get(this.conf);
235 fs.create(p).close();
236 this.conf.set(HConstants.BUCKET_CACHE_IOENGINE_KEY, "file:" + p);
237 doBucketCacheConfigTest();
238 } finally {
239 htu.cleanupTestDir();
240 }
241 }
242
243 private void doBucketCacheConfigTest() {
244 final int bcSize = 100;
245 this.conf.setInt(HConstants.BUCKET_CACHE_SIZE_KEY, bcSize);
246 CacheConfig cc = new CacheConfig(this.conf);
247 basicBlockCacheOps(cc, false, false);
248 assertTrue(cc.getBlockCache() instanceof CombinedBlockCache);
249
250 CombinedBlockCache cbc = (CombinedBlockCache)cc.getBlockCache();
251 BlockCache [] bcs = cbc.getBlockCaches();
252 assertTrue(bcs[0] instanceof LruBlockCache);
253 LruBlockCache lbc = (LruBlockCache)bcs[0];
254 assertEquals(CacheConfig.getLruCacheSize(this.conf,
255 ManagementFactory.getMemoryMXBean().getHeapMemoryUsage()), lbc.getMaxSize());
256 assertTrue(bcs[1] instanceof BucketCache);
257 BucketCache bc = (BucketCache)bcs[1];
258
259 assertEquals(bcSize, bc.getMaxSize() / (1024 * 1024));
260 }
261
262
263
264
265
266 @Test (timeout=10000)
267 public void testBucketCacheConfigL1L2Setup() {
268 this.conf.set(HConstants.BUCKET_CACHE_IOENGINE_KEY, "offheap");
269
270
271 this.conf.setFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, 0.001f);
272 MemoryUsage mu = ManagementFactory.getMemoryMXBean().getHeapMemoryUsage();
273 long lruExpectedSize = CacheConfig.getLruCacheSize(this.conf, mu);
274 final int bcSize = 100;
275 long bcExpectedSize = 100 * 1024 * 1024;
276 assertTrue(lruExpectedSize < bcExpectedSize);
277 this.conf.setInt(HConstants.BUCKET_CACHE_SIZE_KEY, bcSize);
278 this.conf.setBoolean(CacheConfig.BUCKET_CACHE_COMBINED_KEY, false);
279 CacheConfig cc = new CacheConfig(this.conf);
280 basicBlockCacheOps(cc, false, false);
281 assertTrue(cc.getBlockCache() instanceof LruBlockCache);
282
283 LruBlockCache lbc = (LruBlockCache)cc.getBlockCache();
284 assertEquals(lruExpectedSize, lbc.getMaxSize());
285 BlockCache bc = lbc.getVictimHandler();
286
287 assertEquals(bcExpectedSize, ((BucketCache) bc).getMaxSize());
288
289 long initialL1BlockCount = lbc.getBlockCount();
290 long initialL2BlockCount = bc.getBlockCount();
291 Cacheable c = new DataCacheEntry();
292 BlockCacheKey bck = new BlockCacheKey("bck", 0);
293 lbc.cacheBlock(bck, c, false, false);
294 assertEquals(initialL1BlockCount + 1, lbc.getBlockCount());
295 assertEquals(initialL2BlockCount, bc.getBlockCount());
296
297 final long justTooBigSize = lbc.acceptableSize() + 1;
298 lbc.cacheBlock(new BlockCacheKey("bck2", 0), new DataCacheEntry() {
299 @Override
300 public long heapSize() {
301 return justTooBigSize;
302 }
303
304 @Override
305 public int getSerializedLength() {
306 return (int)heapSize();
307 }
308 });
309
310 while (initialL1BlockCount != lbc.getBlockCount()) Threads.sleep(10);
311 assertEquals(initialL1BlockCount, lbc.getBlockCount());
312 long count = bc.getBlockCount();
313 assertTrue(initialL2BlockCount + 1 <= count);
314 }
315
316
317
318
319
320 @Test
321 public void testCacheDataInL1() {
322 this.conf.set(HConstants.BUCKET_CACHE_IOENGINE_KEY, "offheap");
323 this.conf.setInt(HConstants.BUCKET_CACHE_SIZE_KEY, 100);
324 CacheConfig cc = new CacheConfig(this.conf);
325 assertTrue(cc.getBlockCache() instanceof CombinedBlockCache);
326 CombinedBlockCache cbc = (CombinedBlockCache)cc.getBlockCache();
327
328 cacheDataBlock(cc, "1");
329 LruBlockCache lrubc = (LruBlockCache)cbc.getBlockCaches()[0];
330 assertDataBlockCount(lrubc, 0);
331
332 cc.setCacheDataInL1(true);
333 cacheDataBlock(cc, "2");
334 assertDataBlockCount(lrubc, 1);
335 cc.setCacheDataInL1(false);
336 cacheDataBlock(cc, "3");
337 assertDataBlockCount(lrubc, 1);
338 }
339
340 private void assertDataBlockCount(final LruBlockCache bc, final int expected) {
341 Map<BlockType, Integer> blocks = bc.getBlockTypeCountsForTest();
342 assertEquals(expected, blocks == null? 0:
343 blocks.get(BlockType.DATA) == null? 0:
344 blocks.get(BlockType.DATA).intValue());
345 }
346 }