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 java.io.IOException;
23
24 import org.apache.hadoop.fs.FSDataOutputStream;
25 import org.apache.hadoop.fs.Path;
26 import org.apache.hadoop.hbase.HBaseTestCase;
27 import org.apache.hadoop.hbase.util.Bytes;
28
29
30
31
32 public class TestSeekTo extends HBaseTestCase {
33
34 Path makeNewFile() throws IOException {
35 Path ncTFile = new Path(this.testDir, "basic.hfile");
36 FSDataOutputStream fout = this.fs.create(ncTFile);
37 HFile.Writer writer = new HFile.Writer(fout, 40, "none", null);
38
39
40 writer.append(Bytes.toBytes("c"), Bytes.toBytes("value"));
41 writer.append(Bytes.toBytes("e"), Bytes.toBytes("value"));
42 writer.append(Bytes.toBytes("g"), Bytes.toBytes("value"));
43
44 writer.append(Bytes.toBytes("i"), Bytes.toBytes("value"));
45 writer.append(Bytes.toBytes("k"), Bytes.toBytes("value"));
46 writer.close();
47 fout.close();
48 return ncTFile;
49 }
50 public void testSeekBefore() throws Exception {
51 Path p = makeNewFile();
52 HFile.Reader reader = new HFile.Reader(fs, p, null, false);
53 reader.loadFileInfo();
54 HFileScanner scanner = reader.getScanner(false, true);
55 assertEquals(false, scanner.seekBefore(Bytes.toBytes("a")));
56
57 assertEquals(false, scanner.seekBefore(Bytes.toBytes("c")));
58
59 assertEquals(true, scanner.seekBefore(Bytes.toBytes("d")));
60 assertEquals("c", scanner.getKeyString());
61
62 assertEquals(true, scanner.seekBefore(Bytes.toBytes("e")));
63 assertEquals("c", scanner.getKeyString());
64
65 assertEquals(true, scanner.seekBefore(Bytes.toBytes("f")));
66 assertEquals("e", scanner.getKeyString());
67
68 assertEquals(true, scanner.seekBefore(Bytes.toBytes("g")));
69 assertEquals("e", scanner.getKeyString());
70
71 assertEquals(true, scanner.seekBefore(Bytes.toBytes("h")));
72 assertEquals("g", scanner.getKeyString());
73 assertEquals(true, scanner.seekBefore(Bytes.toBytes("i")));
74 assertEquals("g", scanner.getKeyString());
75 assertEquals(true, scanner.seekBefore(Bytes.toBytes("j")));
76 assertEquals("i", scanner.getKeyString());
77 assertEquals(true, scanner.seekBefore(Bytes.toBytes("k")));
78 assertEquals("i", scanner.getKeyString());
79 assertEquals(true, scanner.seekBefore(Bytes.toBytes("l")));
80 assertEquals("k", scanner.getKeyString());
81 }
82
83 public void testSeekTo() throws Exception {
84 Path p = makeNewFile();
85 HFile.Reader reader = new HFile.Reader(fs, p, null, false);
86 reader.loadFileInfo();
87 assertEquals(2, reader.blockIndex.count);
88 HFileScanner scanner = reader.getScanner(false, true);
89
90 assertEquals(-1, scanner.seekTo(Bytes.toBytes("a")));
91
92 assertEquals(1, scanner.seekTo(Bytes.toBytes("d")));
93 assertEquals("c", scanner.getKeyString());
94
95
96 assertEquals(1, scanner.seekTo(Bytes.toBytes("h")));
97 assertEquals("g", scanner.getKeyString());
98
99 assertEquals(1, scanner.seekTo(Bytes.toBytes("l")));
100 assertEquals("k", scanner.getKeyString());
101 }
102
103 public void testBlockContainingKey() throws Exception {
104 Path p = makeNewFile();
105 HFile.Reader reader = new HFile.Reader(fs, p, null, false);
106 reader.loadFileInfo();
107 System.out.println(reader.blockIndex.toString());
108
109 assertEquals(-1, reader.blockIndex.blockContainingKey(Bytes.toBytes("a"), 0, 1));
110 assertEquals(0, reader.blockIndex.blockContainingKey(Bytes.toBytes("c"), 0, 1));
111 assertEquals(0, reader.blockIndex.blockContainingKey(Bytes.toBytes("d"), 0, 1));
112 assertEquals(0, reader.blockIndex.blockContainingKey(Bytes.toBytes("e"), 0, 1));
113 assertEquals(0, reader.blockIndex.blockContainingKey(Bytes.toBytes("g"), 0, 1));
114 assertEquals(0, reader.blockIndex.blockContainingKey(Bytes.toBytes("h"), 0, 1));
115 assertEquals(1, reader.blockIndex.blockContainingKey(Bytes.toBytes("i"), 0, 1));
116 assertEquals(1, reader.blockIndex.blockContainingKey(Bytes.toBytes("j"), 0, 1));
117 assertEquals(1, reader.blockIndex.blockContainingKey(Bytes.toBytes("k"), 0, 1));
118 assertEquals(1, reader.blockIndex.blockContainingKey(Bytes.toBytes("l"), 0, 1));
119
120
121
122 }
123 }