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 java.io.IOException;
22
23 import org.apache.hadoop.fs.FSDataOutputStream;
24 import org.apache.hadoop.fs.Path;
25 import org.apache.hadoop.hbase.HBaseTestCase;
26 import org.apache.hadoop.hbase.HConstants;
27 import org.apache.hadoop.hbase.KeyValue;
28 import org.apache.hadoop.hbase.Tag;
29 import org.apache.hadoop.hbase.testclassification.SmallTests;
30 import org.apache.hadoop.hbase.util.Bytes;
31 import org.junit.experimental.categories.Category;
32
33
34
35
36 @Category(SmallTests.class)
37 public class TestSeekTo extends HBaseTestCase {
38
39 static boolean switchKVs = false;
40
41 static KeyValue toKV(String row, TagUsage tagUsage) {
42 if (tagUsage == TagUsage.NO_TAG) {
43 return new KeyValue(Bytes.toBytes(row), Bytes.toBytes("family"), Bytes.toBytes("qualifier"),
44 Bytes.toBytes("value"));
45 } else if (tagUsage == TagUsage.ONLY_TAG) {
46 Tag t = new Tag((byte) 1, "myTag1");
47 Tag[] tags = new Tag[1];
48 tags[0] = t;
49 return new KeyValue(Bytes.toBytes(row), Bytes.toBytes("family"), Bytes.toBytes("qualifier"),
50 HConstants.LATEST_TIMESTAMP, Bytes.toBytes("value"), tags);
51 } else {
52 if (!switchKVs) {
53 switchKVs = true;
54 return new KeyValue(Bytes.toBytes(row), Bytes.toBytes("family"),
55 Bytes.toBytes("qualifier"), Bytes.toBytes("value"));
56 } else {
57 switchKVs = false;
58 Tag t = new Tag((byte) 1, "myTag1");
59 Tag[] tags = new Tag[1];
60 tags[0] = t;
61 return new KeyValue(Bytes.toBytes(row), Bytes.toBytes("family"),
62 Bytes.toBytes("qualifier"), HConstants.LATEST_TIMESTAMP, Bytes.toBytes("value"), tags);
63 }
64 }
65 }
66 static String toRowStr(KeyValue kv) {
67 return Bytes.toString(kv.getRow());
68 }
69
70 Path makeNewFile(TagUsage tagUsage) throws IOException {
71 Path ncTFile = new Path(this.testDir, "basic.hfile");
72 if (tagUsage != TagUsage.NO_TAG) {
73 conf.setInt("hfile.format.version", 3);
74 } else {
75 conf.setInt("hfile.format.version", 2);
76 }
77 FSDataOutputStream fout = this.fs.create(ncTFile);
78 int blocksize = toKV("a", tagUsage).getLength() * 3;
79 HFileContext context = new HFileContextBuilder().withBlockSize(blocksize)
80 .withIncludesTags(true).build();
81 HFile.Writer writer = HFile.getWriterFactoryNoCache(conf).withOutputStream(fout)
82 .withFileContext(context)
83
84
85 .withComparator(KeyValue.RAW_COMPARATOR).create();
86
87
88 writer.append(toKV("c", tagUsage));
89 writer.append(toKV("e", tagUsage));
90 writer.append(toKV("g", tagUsage));
91
92 writer.append(toKV("i", tagUsage));
93 writer.append(toKV("k", tagUsage));
94 writer.close();
95 fout.close();
96 return ncTFile;
97 }
98
99 public void testSeekBefore() throws Exception {
100 testSeekBeforeInternals(TagUsage.NO_TAG);
101 testSeekBeforeInternals(TagUsage.ONLY_TAG);
102 testSeekBeforeInternals(TagUsage.PARTIAL_TAG);
103 }
104
105 protected void testSeekBeforeInternals(TagUsage tagUsage) throws IOException {
106 Path p = makeNewFile(tagUsage);
107 HFile.Reader reader = HFile.createReader(fs, p, new CacheConfig(conf), conf);
108 reader.loadFileInfo();
109 HFileScanner scanner = reader.getScanner(false, true);
110 assertEquals(false, scanner.seekBefore(toKV("a", tagUsage).getKey()));
111
112 assertEquals(false, scanner.seekBefore(toKV("c", tagUsage).getKey()));
113
114 assertEquals(true, scanner.seekBefore(toKV("d", tagUsage).getKey()));
115 assertEquals("c", toRowStr(scanner.getKeyValue()));
116
117 assertEquals(true, scanner.seekBefore(toKV("e", tagUsage).getKey()));
118 assertEquals("c", toRowStr(scanner.getKeyValue()));
119
120 assertEquals(true, scanner.seekBefore(toKV("f", tagUsage).getKey()));
121 assertEquals("e", toRowStr(scanner.getKeyValue()));
122
123 assertEquals(true, scanner.seekBefore(toKV("g", tagUsage).getKey()));
124 assertEquals("e", toRowStr(scanner.getKeyValue()));
125
126 assertEquals(true, scanner.seekBefore(toKV("h", tagUsage).getKey()));
127 assertEquals("g", toRowStr(scanner.getKeyValue()));
128 assertEquals(true, scanner.seekBefore(toKV("i", tagUsage).getKey()));
129 assertEquals("g", toRowStr(scanner.getKeyValue()));
130 assertEquals(true, scanner.seekBefore(toKV("j", tagUsage).getKey()));
131 assertEquals("i", toRowStr(scanner.getKeyValue()));
132 assertEquals(true, scanner.seekBefore(toKV("k", tagUsage).getKey()));
133 assertEquals("i", toRowStr(scanner.getKeyValue()));
134 assertEquals(true, scanner.seekBefore(toKV("l", tagUsage).getKey()));
135 assertEquals("k", toRowStr(scanner.getKeyValue()));
136
137 reader.close();
138 }
139
140 public void testSeekBeforeWithReSeekTo() throws Exception {
141 testSeekBeforeWithReSeekToInternals(TagUsage.NO_TAG);
142 testSeekBeforeWithReSeekToInternals(TagUsage.ONLY_TAG);
143 testSeekBeforeWithReSeekToInternals(TagUsage.PARTIAL_TAG);
144 }
145
146 protected void testSeekBeforeWithReSeekToInternals(TagUsage tagUsage) throws IOException {
147 Path p = makeNewFile(tagUsage);
148 HFile.Reader reader = HFile.createReader(fs, p, new CacheConfig(conf), conf);
149 reader.loadFileInfo();
150 HFileScanner scanner = reader.getScanner(false, true);
151 assertEquals(false, scanner.seekBefore(toKV("a", tagUsage).getKey()));
152 assertEquals(false, scanner.seekBefore(toKV("b", tagUsage).getKey()));
153 assertEquals(false, scanner.seekBefore(toKV("c", tagUsage).getKey()));
154
155
156 assertEquals(true, scanner.seekBefore(toKV("d", tagUsage).getKey()));
157 assertEquals("c", toRowStr(scanner.getKeyValue()));
158
159 assertEquals(0, scanner.reseekTo(toKV("c", tagUsage).getKey()));
160 assertEquals("c", toRowStr(scanner.getKeyValue()));
161 assertEquals(0, scanner.reseekTo(toKV("g", tagUsage).getKey()));
162 assertEquals("g", toRowStr(scanner.getKeyValue()));
163
164
165 assertEquals(true, scanner.seekBefore(toKV("e", tagUsage).getKey()));
166 assertEquals("c", toRowStr(scanner.getKeyValue()));
167
168 assertEquals(0, scanner.reseekTo(toKV("e", tagUsage).getKey()));
169 assertEquals("e", toRowStr(scanner.getKeyValue()));
170 assertEquals(0, scanner.reseekTo(toKV("g", tagUsage).getKey()));
171 assertEquals("g", toRowStr(scanner.getKeyValue()));
172
173
174 assertEquals(true, scanner.seekBefore(toKV("f", tagUsage).getKey()));
175 assertEquals("e", toRowStr(scanner.getKeyValue()));
176
177 assertEquals(0, scanner.reseekTo(toKV("e", tagUsage).getKey()));
178 assertEquals("e", toRowStr(scanner.getKeyValue()));
179 assertEquals(0, scanner.reseekTo(toKV("g", tagUsage).getKey()));
180 assertEquals("g", toRowStr(scanner.getKeyValue()));
181
182
183 assertEquals(true, scanner.seekBefore(toKV("g", tagUsage).getKey()));
184 assertEquals("e", toRowStr(scanner.getKeyValue()));
185
186 assertEquals(0, scanner.reseekTo(toKV("e", tagUsage).getKey()));
187 assertEquals("e", toRowStr(scanner.getKeyValue()));
188 assertEquals(0, scanner.reseekTo(toKV("g", tagUsage).getKey()));
189 assertEquals("g", toRowStr(scanner.getKeyValue()));
190
191
192 assertEquals(true, scanner.seekBefore(toKV("h", tagUsage).getKey()));
193 assertEquals("g", toRowStr(scanner.getKeyValue()));
194
195 assertEquals(0, scanner.reseekTo(toKV("g", tagUsage).getKey()));
196 assertEquals("g", toRowStr(scanner.getKeyValue()));
197
198
199 assertEquals(true, scanner.seekBefore(toKV("i", tagUsage).getKey()));
200 assertEquals("g", toRowStr(scanner.getKeyValue()));
201
202 assertEquals(0, scanner.reseekTo(toKV("g", tagUsage).getKey()));
203 assertEquals("g", toRowStr(scanner.getKeyValue()));
204
205
206 assertEquals(true, scanner.seekBefore(toKV("j", tagUsage).getKey()));
207 assertEquals("i", toRowStr(scanner.getKeyValue()));
208
209 assertEquals(0, scanner.reseekTo(toKV("i", tagUsage).getKey()));
210 assertEquals("i", toRowStr(scanner.getKeyValue()));
211
212
213 assertEquals(true, scanner.seekBefore(toKV("k", tagUsage).getKey()));
214 assertEquals("i", toRowStr(scanner.getKeyValue()));
215
216 assertEquals(0, scanner.reseekTo(toKV("i", tagUsage).getKey()));
217 assertEquals("i", toRowStr(scanner.getKeyValue()));
218 assertEquals(0, scanner.reseekTo(toKV("k", tagUsage).getKey()));
219 assertEquals("k", toRowStr(scanner.getKeyValue()));
220
221
222 assertEquals(true, scanner.seekBefore(toKV("l", tagUsage).getKey()));
223 assertEquals("k", toRowStr(scanner.getKeyValue()));
224
225 assertEquals(0, scanner.reseekTo(toKV("k", tagUsage).getKey()));
226 assertEquals("k", toRowStr(scanner.getKeyValue()));
227 }
228
229 public void testSeekTo() throws Exception {
230 testSeekToInternals(TagUsage.NO_TAG);
231 testSeekToInternals(TagUsage.ONLY_TAG);
232 testSeekToInternals(TagUsage.PARTIAL_TAG);
233 }
234
235 protected void testSeekToInternals(TagUsage tagUsage) throws IOException {
236 Path p = makeNewFile(tagUsage);
237 HFile.Reader reader = HFile.createReader(fs, p, new CacheConfig(conf), conf);
238 reader.loadFileInfo();
239 assertEquals(2, reader.getDataBlockIndexReader().getRootBlockCount());
240 HFileScanner scanner = reader.getScanner(false, true);
241
242 assertEquals(-1, scanner.seekTo(toKV("a", tagUsage).getKey()));
243
244 assertEquals(1, scanner.seekTo(toKV("d", tagUsage).getKey()));
245 assertEquals("c", toRowStr(scanner.getKeyValue()));
246
247
248 assertEquals(1, scanner.seekTo(toKV("h", tagUsage).getKey()));
249 assertEquals("g", toRowStr(scanner.getKeyValue()));
250
251 assertEquals(1, scanner.seekTo(toKV("l", tagUsage).getKey()));
252 assertEquals("k", toRowStr(scanner.getKeyValue()));
253
254 reader.close();
255 }
256 public void testBlockContainingKey() throws Exception {
257 testBlockContainingKeyInternals(TagUsage.NO_TAG);
258 testBlockContainingKeyInternals(TagUsage.ONLY_TAG);
259 testBlockContainingKeyInternals(TagUsage.PARTIAL_TAG);
260 }
261
262 protected void testBlockContainingKeyInternals(TagUsage tagUsage) throws IOException {
263 Path p = makeNewFile(tagUsage);
264 HFile.Reader reader = HFile.createReader(fs, p, new CacheConfig(conf), conf);
265 reader.loadFileInfo();
266 HFileBlockIndex.BlockIndexReader blockIndexReader =
267 reader.getDataBlockIndexReader();
268 System.out.println(blockIndexReader.toString());
269 int klen = toKV("a", tagUsage).getKey().length;
270
271 assertEquals(-1, blockIndexReader.rootBlockContainingKey(
272 toKV("a", tagUsage).getKey(), 0, klen));
273 assertEquals(0, blockIndexReader.rootBlockContainingKey(
274 toKV("c", tagUsage).getKey(), 0, klen));
275 assertEquals(0, blockIndexReader.rootBlockContainingKey(
276 toKV("d", tagUsage).getKey(), 0, klen));
277 assertEquals(0, blockIndexReader.rootBlockContainingKey(
278 toKV("e", tagUsage).getKey(), 0, klen));
279 assertEquals(0, blockIndexReader.rootBlockContainingKey(
280 toKV("g", tagUsage).getKey(), 0, klen));
281 assertEquals(0, blockIndexReader.rootBlockContainingKey(
282 toKV("h", tagUsage).getKey(), 0, klen));
283 assertEquals(1, blockIndexReader.rootBlockContainingKey(
284 toKV("i", tagUsage).getKey(), 0, klen));
285 assertEquals(1, blockIndexReader.rootBlockContainingKey(
286 toKV("j", tagUsage).getKey(), 0, klen));
287 assertEquals(1, blockIndexReader.rootBlockContainingKey(
288 toKV("k", tagUsage).getKey(), 0, klen));
289 assertEquals(1, blockIndexReader.rootBlockContainingKey(
290 toKV("l", tagUsage).getKey(), 0, klen));
291
292 reader.close();
293 }
294
295
296 }
297