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.Random;
24
25 import org.apache.hadoop.conf.Configuration;
26 import org.apache.hadoop.fs.FileSystem;
27 import org.apache.hadoop.fs.Path;
28 import org.apache.hadoop.hbase.HBaseTestingUtility;
29 import org.apache.hadoop.hbase.KeyValue;
30 import org.apache.hadoop.hbase.fs.HFileSystem;
31 import org.apache.hadoop.hbase.regionserver.StoreFile;
32
33 import org.apache.hadoop.hbase.testclassification.SmallTests;
34 import org.junit.Before;
35 import org.junit.Test;
36 import org.junit.experimental.categories.Category;
37
38 @Category(SmallTests.class)
39 public class TestPrefetch {
40
41 private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
42
43 private static final int NUM_VALID_KEY_TYPES = KeyValue.Type.values().length - 2;
44 private static final int DATA_BLOCK_SIZE = 2048;
45 private static final int NUM_KV = 1000;
46 private static final Random RNG = new Random();
47
48 private Configuration conf;
49 private CacheConfig cacheConf;
50 private FileSystem fs;
51
52 @Before
53 public void setUp() throws IOException {
54 conf = TEST_UTIL.getConfiguration();
55 conf.setInt(HFile.FORMAT_VERSION_KEY, 3);
56 conf.setBoolean(CacheConfig.PREFETCH_BLOCKS_ON_OPEN_KEY, true);
57 fs = HFileSystem.get(conf);
58 cacheConf = new CacheConfig(conf);
59 }
60
61 @Test(timeout=60000)
62 public void testPrefetch() throws Exception {
63 Path storeFile = writeStoreFile();
64 readStoreFile(storeFile);
65 }
66
67 private void readStoreFile(Path storeFilePath) throws Exception {
68
69 HFileReaderV2 reader = (HFileReaderV2) HFile.createReader(fs,
70 storeFilePath, cacheConf, conf);
71
72 while (!((HFileReaderV3)reader).prefetchComplete()) {
73
74 Thread.sleep(1000);
75 }
76
77
78 BlockCache blockCache = cacheConf.getBlockCache();
79 long offset = 0;
80 HFileBlock prevBlock = null;
81 while (offset < reader.getTrailer().getLoadOnOpenDataOffset()) {
82 long onDiskSize = -1;
83 if (prevBlock != null) {
84 onDiskSize = prevBlock.getNextBlockOnDiskSizeWithHeader();
85 }
86 HFileBlock block = reader.readBlock(offset, onDiskSize, false, true, false, true, null);
87 BlockCacheKey blockCacheKey = new BlockCacheKey(reader.getName(), offset);
88 boolean isCached = blockCache.getBlock(blockCacheKey, true, false, true) != null;
89 if (block.getBlockType() == BlockType.DATA ||
90 block.getBlockType() == BlockType.ROOT_INDEX ||
91 block.getBlockType() == BlockType.INTERMEDIATE_INDEX) {
92 assertTrue(isCached);
93 }
94 prevBlock = block;
95 offset += block.getOnDiskSizeWithHeader();
96 }
97 }
98
99 private Path writeStoreFile() throws IOException {
100 Path storeFileParentDir = new Path(TEST_UTIL.getDataTestDir(), "TestPrefetch");
101 HFileContext meta = new HFileContextBuilder()
102 .withBlockSize(DATA_BLOCK_SIZE)
103 .build();
104 StoreFile.Writer sfw = new StoreFile.WriterBuilder(conf, cacheConf, fs)
105 .withOutputDir(storeFileParentDir)
106 .withComparator(KeyValue.COMPARATOR)
107 .withFileContext(meta)
108 .build();
109
110 final int rowLen = 32;
111 for (int i = 0; i < NUM_KV; ++i) {
112 byte[] k = TestHFileWriterV2.randomOrderedKey(RNG, i);
113 byte[] v = TestHFileWriterV2.randomValue(RNG);
114 int cfLen = RNG.nextInt(k.length - rowLen + 1);
115 KeyValue kv = new KeyValue(
116 k, 0, rowLen,
117 k, rowLen, cfLen,
118 k, rowLen + cfLen, k.length - rowLen - cfLen,
119 RNG.nextLong(),
120 generateKeyType(RNG),
121 v, 0, v.length);
122 sfw.append(kv);
123 }
124
125 sfw.close();
126 return sfw.getPath();
127 }
128
129 public static KeyValue.Type generateKeyType(Random rand) {
130 if (rand.nextBoolean()) {
131
132 return KeyValue.Type.Put;
133 } else {
134 KeyValue.Type keyType =
135 KeyValue.Type.values()[1 + rand.nextInt(NUM_VALID_KEY_TYPES)];
136 if (keyType == KeyValue.Type.Minimum || keyType == KeyValue.Type.Maximum)
137 {
138 throw new RuntimeException("Generated an invalid key type: " + keyType
139 + ". " + "Probably the layout of KeyValue.Type has changed.");
140 }
141 return keyType;
142 }
143 }
144
145 }