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.assertArrayEquals;
22 import static org.junit.Assert.assertEquals;
23 import static org.junit.Assert.assertNull;
24 import static org.junit.Assert.assertTrue;
25 import static org.junit.Assert.fail;
26
27 import java.io.IOException;
28 import java.nio.ByteBuffer;
29 import java.util.Arrays;
30 import java.util.HashSet;
31 import java.util.Random;
32 import java.util.concurrent.ConcurrentLinkedQueue;
33 import java.util.concurrent.atomic.AtomicInteger;
34
35 import org.apache.hadoop.conf.Configuration;
36 import org.apache.hadoop.hbase.HConstants;
37 import org.apache.hadoop.hbase.MultithreadedTestUtil;
38 import org.apache.hadoop.hbase.MultithreadedTestUtil.TestThread;
39 import org.apache.hadoop.hbase.classification.InterfaceAudience;
40 import org.apache.hadoop.hbase.io.HeapSize;
41 import org.apache.hadoop.hbase.io.compress.Compression;
42 import org.apache.hadoop.hbase.io.hfile.bucket.BucketCache;
43 import org.apache.hadoop.hbase.util.ChecksumType;
44
45 public class CacheTestUtils {
46
47 private static final boolean includesMemstoreTS = true;
48
49
50
51
52
53
54 public static void testHeapSizeChanges(final BlockCache toBeTested,
55 final int blockSize) {
56 HFileBlockPair[] blocks = generateHFileBlocks(blockSize, 1);
57 long heapSize = ((HeapSize) toBeTested).heapSize();
58 toBeTested.cacheBlock(blocks[0].blockName, blocks[0].block);
59
60
61 assertTrue(heapSize < ((HeapSize) toBeTested).heapSize());
62
63 toBeTested.evictBlock(blocks[0].blockName);
64
65
66 assertEquals(heapSize, ((HeapSize) toBeTested).heapSize());
67 }
68
69 public static void testCacheMultiThreaded(final BlockCache toBeTested,
70 final int blockSize, final int numThreads, final int numQueries,
71 final double passingScore) throws Exception {
72
73 Configuration conf = new Configuration();
74 MultithreadedTestUtil.TestContext ctx = new MultithreadedTestUtil.TestContext(
75 conf);
76
77 final AtomicInteger totalQueries = new AtomicInteger();
78 final ConcurrentLinkedQueue<HFileBlockPair> blocksToTest = new ConcurrentLinkedQueue<HFileBlockPair>();
79 final AtomicInteger hits = new AtomicInteger();
80 final AtomicInteger miss = new AtomicInteger();
81
82 HFileBlockPair[] blocks = generateHFileBlocks(numQueries, blockSize);
83 blocksToTest.addAll(Arrays.asList(blocks));
84
85 for (int i = 0; i < numThreads; i++) {
86 TestThread t = new MultithreadedTestUtil.RepeatingTestThread(ctx) {
87 @Override
88 public void doAnAction() throws Exception {
89 if (!blocksToTest.isEmpty()) {
90 HFileBlockPair ourBlock = blocksToTest.poll();
91
92 if (ourBlock == null) {
93 ctx.setStopFlag(true);
94 return;
95 }
96 toBeTested.cacheBlock(ourBlock.blockName, ourBlock.block);
97 Cacheable retrievedBlock = toBeTested.getBlock(ourBlock.blockName,
98 false, false, true);
99 if (retrievedBlock != null) {
100 assertEquals(ourBlock.block, retrievedBlock);
101 toBeTested.evictBlock(ourBlock.blockName);
102 hits.incrementAndGet();
103 assertNull(toBeTested.getBlock(ourBlock.blockName, false, false, true));
104 } else {
105 miss.incrementAndGet();
106 }
107 totalQueries.incrementAndGet();
108 }
109 }
110 };
111 t.setDaemon(true);
112 ctx.addThread(t);
113 }
114 ctx.startThreads();
115 while (!blocksToTest.isEmpty() && ctx.shouldRun()) {
116 Thread.sleep(10);
117 }
118 ctx.stop();
119 if (hits.get() / ((double) hits.get() + (double) miss.get()) < passingScore) {
120 fail("Too many nulls returned. Hits: " + hits.get() + " Misses: "
121 + miss.get());
122 }
123 }
124
125 public static void testCacheSimple(BlockCache toBeTested, int blockSize,
126 int numBlocks) throws Exception {
127
128 HFileBlockPair[] blocks = generateHFileBlocks(numBlocks, blockSize);
129
130 for (HFileBlockPair block : blocks) {
131 assertNull(toBeTested.getBlock(block.blockName, true, false, true));
132 }
133
134
135 for (HFileBlockPair block : blocks) {
136 toBeTested.cacheBlock(block.blockName, block.block);
137 }
138
139
140
141
142
143 for (HFileBlockPair block : blocks) {
144 HFileBlock buf = (HFileBlock) toBeTested.getBlock(block.blockName, true, false, true);
145 if (buf != null) {
146 assertEquals(block.block, buf);
147 }
148
149 }
150
151
152
153 for (HFileBlockPair block : blocks) {
154 try {
155 if (toBeTested.getBlock(block.blockName, true, false, true) != null) {
156 toBeTested.cacheBlock(block.blockName, block.block);
157 if (!(toBeTested instanceof BucketCache)) {
158
159
160 fail("Cache should not allow re-caching a block");
161 }
162 }
163 } catch (RuntimeException re) {
164
165 }
166 }
167
168 }
169
170 public static void hammerSingleKey(final BlockCache toBeTested,
171 int BlockSize, int numThreads, int numQueries) throws Exception {
172 final BlockCacheKey key = new BlockCacheKey("key", 0);
173 final byte[] buf = new byte[5 * 1024];
174 Arrays.fill(buf, (byte) 5);
175
176 final ByteArrayCacheable bac = new ByteArrayCacheable(buf);
177 Configuration conf = new Configuration();
178 MultithreadedTestUtil.TestContext ctx = new MultithreadedTestUtil.TestContext(
179 conf);
180
181 final AtomicInteger totalQueries = new AtomicInteger();
182 toBeTested.cacheBlock(key, bac);
183
184 for (int i = 0; i < numThreads; i++) {
185 TestThread t = new MultithreadedTestUtil.RepeatingTestThread(ctx) {
186 @Override
187 public void doAnAction() throws Exception {
188 ByteArrayCacheable returned = (ByteArrayCacheable) toBeTested
189 .getBlock(key, false, false, true);
190 if (returned != null) {
191 assertArrayEquals(buf, returned.buf);
192 } else {
193 Thread.sleep(10);
194 }
195 totalQueries.incrementAndGet();
196 }
197 };
198
199 t.setDaemon(true);
200 ctx.addThread(t);
201 }
202
203
204 final long blockEvictPeriod = 50;
205 TestThread t = new MultithreadedTestUtil.RepeatingTestThread(ctx) {
206 @Override
207 public void doAnAction() throws Exception {
208 toBeTested.evictBlock(key);
209 toBeTested.cacheBlock(key, bac);
210 Thread.sleep(blockEvictPeriod);
211 }
212 };
213 t.setDaemon(true);
214 ctx.addThread(t);
215
216 ctx.startThreads();
217 while (totalQueries.get() < numQueries && ctx.shouldRun()) {
218 Thread.sleep(10);
219 }
220 ctx.stop();
221 }
222
223 public static void hammerEviction(final BlockCache toBeTested, int BlockSize,
224 int numThreads, int numQueries) throws Exception {
225
226 Configuration conf = new Configuration();
227 MultithreadedTestUtil.TestContext ctx = new MultithreadedTestUtil.TestContext(
228 conf);
229
230 final AtomicInteger totalQueries = new AtomicInteger();
231
232 for (int i = 0; i < numThreads; i++) {
233 final int finalI = i;
234
235 final byte[] buf = new byte[5 * 1024];
236 TestThread t = new MultithreadedTestUtil.RepeatingTestThread(ctx) {
237 @Override
238 public void doAnAction() throws Exception {
239 for (int j = 0; j < 100; j++) {
240 BlockCacheKey key = new BlockCacheKey("key_" + finalI + "_" + j, 0);
241 Arrays.fill(buf, (byte) (finalI * j));
242 final ByteArrayCacheable bac = new ByteArrayCacheable(buf);
243
244 ByteArrayCacheable gotBack = (ByteArrayCacheable) toBeTested
245 .getBlock(key, true, false, true);
246 if (gotBack != null) {
247 assertArrayEquals(gotBack.buf, bac.buf);
248 } else {
249 toBeTested.cacheBlock(key, bac);
250 }
251 }
252 totalQueries.incrementAndGet();
253 }
254 };
255
256 t.setDaemon(true);
257 ctx.addThread(t);
258 }
259
260 ctx.startThreads();
261 while (totalQueries.get() < numQueries && ctx.shouldRun()) {
262 Thread.sleep(10);
263 }
264 ctx.stop();
265
266 assertTrue(toBeTested.getStats().getEvictedCount() > 0);
267 }
268
269 public static class ByteArrayCacheable implements Cacheable {
270
271 static final CacheableDeserializer<Cacheable> blockDeserializer =
272 new CacheableDeserializer<Cacheable>() {
273
274 @Override
275 public Cacheable deserialize(ByteBuffer b) throws IOException {
276 int len = b.getInt();
277 Thread.yield();
278 byte buf[] = new byte[len];
279 b.get(buf);
280 return new ByteArrayCacheable(buf);
281 }
282
283 @Override
284 public int getDeserialiserIdentifier() {
285 return deserializerIdentifier;
286 }
287
288 @Override
289 public Cacheable deserialize(ByteBuffer b, boolean reuse)
290 throws IOException {
291 return deserialize(b);
292 }
293 };
294
295 final byte[] buf;
296
297 public ByteArrayCacheable(byte[] buf) {
298 this.buf = buf;
299 }
300
301 @Override
302 public long heapSize() {
303 return 4 + buf.length;
304 }
305
306 @Override
307 public int getSerializedLength() {
308 return 4 + buf.length;
309 }
310
311 @Override
312 public void serialize(ByteBuffer destination, boolean includeNextBlockOnDiskSize) {
313 destination.putInt(buf.length);
314 Thread.yield();
315 destination.put(buf);
316 destination.rewind();
317 }
318
319 @Override
320 public CacheableDeserializer<Cacheable> getDeserializer() {
321 return blockDeserializer;
322 }
323
324 private static final int deserializerIdentifier;
325 static {
326 deserializerIdentifier = CacheableDeserializerIdManager
327 .registerDeserializer(blockDeserializer);
328 }
329
330 @Override
331 public BlockType getBlockType() {
332 return BlockType.DATA;
333 }
334 }
335
336
337 public static HFileBlockPair[] generateHFileBlocks(int blockSize, int numBlocks) {
338 HFileBlockPair[] returnedBlocks = new HFileBlockPair[numBlocks];
339 Random rand = new Random();
340 HashSet<String> usedStrings = new HashSet<String>();
341 for (int i = 0; i < numBlocks; i++) {
342 ByteBuffer cachedBuffer = ByteBuffer.allocate(blockSize);
343 rand.nextBytes(cachedBuffer.array());
344 cachedBuffer.rewind();
345 int onDiskSizeWithoutHeader = blockSize;
346 int uncompressedSizeWithoutHeader = blockSize;
347 long prevBlockOffset = rand.nextLong();
348 BlockType.DATA.write(cachedBuffer);
349 cachedBuffer.putInt(onDiskSizeWithoutHeader);
350 cachedBuffer.putInt(uncompressedSizeWithoutHeader);
351 cachedBuffer.putLong(prevBlockOffset);
352 cachedBuffer.rewind();
353 HFileContext meta = new HFileContextBuilder()
354 .withHBaseCheckSum(false)
355 .withIncludesMvcc(includesMemstoreTS)
356 .withIncludesTags(false)
357 .withCompression(Compression.Algorithm.NONE)
358 .withBytesPerCheckSum(0)
359 .withChecksumType(ChecksumType.NULL)
360 .build();
361 HFileBlock generated = new HFileBlock(BlockType.DATA,
362 onDiskSizeWithoutHeader, uncompressedSizeWithoutHeader,
363 prevBlockOffset, cachedBuffer, HFileBlock.DONT_FILL_HEADER,
364 blockSize,
365 onDiskSizeWithoutHeader + HConstants.HFILEBLOCK_HEADER_SIZE, -1, meta);
366
367 String strKey;
368
369 for (strKey = new Long(rand.nextLong()).toString(); !usedStrings
370 .add(strKey); strKey = new Long(rand.nextLong()).toString())
371 ;
372
373 returnedBlocks[i] = new HFileBlockPair();
374 returnedBlocks[i].blockName = new BlockCacheKey(strKey, 0);
375 returnedBlocks[i].block = generated;
376 }
377 return returnedBlocks;
378 }
379
380 @InterfaceAudience.Private
381 public static class HFileBlockPair {
382 BlockCacheKey blockName;
383 HFileBlock block;
384
385 public BlockCacheKey getBlockName() {
386 return this.blockName;
387 }
388
389 public HFileBlock getBlock() {
390 return this.block;
391 }
392 }
393
394 public static void getBlockAndAssertEquals(BlockCache cache, BlockCacheKey key,
395 Cacheable blockToCache, ByteBuffer destBuffer,
396 ByteBuffer expectedBuffer) {
397 destBuffer.clear();
398 cache.cacheBlock(key, blockToCache);
399 Cacheable actualBlock = cache.getBlock(key, false, false, false);
400 actualBlock.serialize(destBuffer, true);
401 assertEquals(expectedBuffer, destBuffer);
402 }
403 }