1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20 package org.apache.hadoop.hbase.io.hfile;
21
22 import static org.junit.Assert.assertEquals;
23 import static org.junit.Assert.assertTrue;
24
25 import java.io.IOException;
26 import java.nio.ByteBuffer;
27 import java.util.Random;
28
29 import org.apache.commons.logging.Log;
30 import org.apache.commons.logging.LogFactory;
31 import org.apache.hadoop.conf.Configuration;
32 import org.apache.hadoop.fs.FileSystem;
33 import org.apache.hadoop.fs.Path;
34 import org.apache.hadoop.hbase.HBaseTestingUtility;
35 import org.apache.hadoop.hbase.HConstants;
36 import org.apache.hadoop.hbase.KeyValue;
37 import org.apache.hadoop.hbase.fs.HFileSystem;
38 import org.apache.hadoop.hbase.regionserver.BloomType;
39 import org.apache.hadoop.hbase.regionserver.StoreFile;
40 import org.apache.hadoop.hbase.testclassification.MediumTests;
41 import org.apache.hadoop.hbase.util.BloomFilterFactory;
42 import org.apache.hadoop.hbase.util.Bytes;
43 import org.junit.Test;
44 import org.junit.experimental.categories.Category;
45
46 @Category(MediumTests.class)
47 public class TestSeekBeforeWithInlineBlocks {
48
49 private static final Log LOG = LogFactory.getLog(TestSeekBeforeWithInlineBlocks.class);
50
51 private static final HBaseTestingUtility TEST_UTIL =
52 new HBaseTestingUtility();
53
54 private static final int NUM_KV = 10000;
55
56 private static final int DATA_BLOCK_SIZE = 4096;
57 private static final int BLOOM_BLOCK_SIZE = 1024;
58 private static final int[] INDEX_CHUNK_SIZES = { HFileBlockIndex.DEFAULT_MAX_CHUNK_SIZE, 65536, 4096, 1024 };
59 private static final int[] EXPECTED_NUM_LEVELS = { 1, 1, 2, 3 };
60
61 private static final Random RAND = new Random(192537);
62 private static final byte[] FAM = Bytes.toBytes("family");
63
64 private FileSystem fs;
65 private Configuration conf;
66
67
68
69
70
71
72
73
74
75 @Test
76 public void testMultiIndexLevelRandomHFileWithBlooms() throws IOException {
77 conf = TEST_UTIL.getConfiguration();
78
79
80 for (int hfileVersion = HFile.MIN_FORMAT_VERSION;
81 hfileVersion <= HFile.MAX_FORMAT_VERSION; hfileVersion++) {
82
83 conf.setInt(HFile.FORMAT_VERSION_KEY, hfileVersion);
84 fs = HFileSystem.get(conf);
85
86
87 for (BloomType bloomType : BloomType.values()) {
88
89
90 for (int testI = 0; testI < INDEX_CHUNK_SIZES.length; testI++) {
91 int indexBlockSize = INDEX_CHUNK_SIZES[testI];
92 int expectedNumLevels = EXPECTED_NUM_LEVELS[testI];
93
94 LOG.info(String.format("Testing HFileVersion: %s, BloomType: %s, Index Levels: %s",
95 hfileVersion, bloomType, expectedNumLevels));
96
97 conf.setInt(HFileBlockIndex.MAX_CHUNK_SIZE_KEY, indexBlockSize);
98 conf.setInt(BloomFilterFactory.IO_STOREFILE_BLOOM_BLOCK_SIZE, BLOOM_BLOCK_SIZE);
99
100 byte[][] keys = new byte[NUM_KV][];
101 byte[][] values = new byte[NUM_KV][];
102
103 Path hfilePath = new Path(TEST_UTIL.getDataTestDir(),
104 String.format("testMultiIndexLevelRandomHFileWithBlooms-%s-%s-%s",
105 hfileVersion, bloomType, testI));
106
107
108 conf.setFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, 0.0f);
109 CacheConfig cacheConf = new CacheConfig(conf);
110
111
112 {
113 HFileContext meta = new HFileContextBuilder()
114 .withBlockSize(DATA_BLOCK_SIZE)
115 .build();
116
117 StoreFile.Writer storeFileWriter = new StoreFile.WriterBuilder(conf, cacheConf, fs)
118 .withFilePath(hfilePath)
119 .withFileContext(meta)
120 .withBloomType(bloomType)
121 .build();
122
123 for (int i = 0; i < NUM_KV; i++) {
124 byte[] row = TestHFileWriterV2.randomOrderedKey(RAND, i);
125 byte[] qual = TestHFileWriterV2.randomRowOrQualifier(RAND);
126 byte[] value = TestHFileWriterV2.randomValue(RAND);
127 KeyValue kv = new KeyValue(row, FAM, qual, value);
128
129 storeFileWriter.append(kv);
130 keys[i] = kv.getKey();
131 values[i] = value;
132 }
133
134 storeFileWriter.close();
135 }
136
137
138 HFile.Reader reader = HFile.createReader(fs, hfilePath, cacheConf, conf);
139
140
141 assertEquals(expectedNumLevels, reader.getTrailer().getNumDataIndexLevels());
142
143
144
145 for (boolean pread : new boolean[] { false, true }) {
146 HFileScanner scanner = reader.getScanner(true, pread);
147 checkNoSeekBefore(keys, scanner, 0);
148 for (int i = 1; i < NUM_KV; i++) {
149 checkSeekBefore(keys, scanner, i);
150 checkKeyValue("i=" + i, keys[i-1], values[i-1],
151 scanner.getKey(),
152 scanner.getValue());
153 }
154 assertTrue(scanner.seekTo());
155 for (int i = NUM_KV - 1; i >= 1; i--) {
156 checkSeekBefore(keys, scanner, i);
157 checkKeyValue("i=" + i, keys[i-1], values[i-1],
158 scanner.getKey(),
159 scanner.getValue());
160 }
161 checkNoSeekBefore(keys, scanner, 0);
162 }
163
164 reader.close();
165 }
166 }
167 }
168 }
169
170 private void checkSeekBefore(byte[][] keys, HFileScanner scanner, int i)
171 throws IOException {
172 assertEquals("Failed to seek to the key before #" + i + " ("
173 + Bytes.toStringBinary(keys[i]) + ")", true,
174 scanner.seekBefore(keys[i]));
175 }
176
177 private void checkNoSeekBefore(byte[][] keys, HFileScanner scanner, int i)
178 throws IOException {
179 assertEquals("Incorrectly succeeded in seeking to before first key ("
180 + Bytes.toStringBinary(keys[i]) + ")", false,
181 scanner.seekBefore(keys[i]));
182 }
183
184 private void assertArrayEqualsBuffer(String msgPrefix, byte[] arr,
185 ByteBuffer buf) {
186 assertEquals(msgPrefix + ": expected " + Bytes.toStringBinary(arr)
187 + ", actual " + Bytes.toStringBinary(buf), 0, Bytes.compareTo(arr, 0,
188 arr.length, buf.array(), buf.arrayOffset(), buf.limit()));
189 }
190
191
192 private void checkKeyValue(String msgPrefix, byte[] expectedKey,
193 byte[] expectedValue, ByteBuffer keyRead, ByteBuffer valueRead) {
194 if (!msgPrefix.isEmpty())
195 msgPrefix += ". ";
196
197 assertArrayEqualsBuffer(msgPrefix + "Invalid key", expectedKey, keyRead);
198 assertArrayEqualsBuffer(msgPrefix + "Invalid value", expectedValue,
199 valueRead);
200 }
201 }