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