1   /**
2    * Copyright 2009 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
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.KeyValue;
28  import org.apache.hadoop.hbase.util.Bytes;
29  
30  /**
31   * Test {@link HFileScanner#seekTo(byte[])} and its variants.
32   */
33  public class TestSeekTo extends HBaseTestCase {
34  
35    static KeyValue toKV(String row) {
36      return new KeyValue(Bytes.toBytes(row), Bytes.toBytes("family"), Bytes
37          .toBytes("qualifier"), Bytes.toBytes("value"));
38    }
39  
40    static String toRowStr(KeyValue kv) {
41      return Bytes.toString(kv.getRow());
42    }
43  
44    Path makeNewFile() throws IOException {
45      Path ncTFile = new Path(this.testDir, "basic.hfile");
46      FSDataOutputStream fout = this.fs.create(ncTFile);
47      int blocksize = toKV("a").getLength() * 3;
48      HFile.Writer writer = new HFile.Writer(fout, blocksize, "none", null);
49      // 4 bytes * 3 * 2 for each key/value +
50      // 3 for keys, 15 for values = 42 (woot)
51      writer.append(toKV("c"));
52      writer.append(toKV("e"));
53      writer.append(toKV("g"));
54      // block transition
55      writer.append(toKV("i"));
56      writer.append(toKV("k"));
57      writer.close();
58      fout.close();
59      return ncTFile;
60    }
61    public void testSeekBefore() throws Exception {
62      Path p = makeNewFile();
63      HFile.Reader reader = new HFile.Reader(fs, p, null, false);
64      reader.loadFileInfo();
65      HFileScanner scanner = reader.getScanner(false, true);
66      assertEquals(false, scanner.seekBefore(toKV("a").getKey()));
67  
68      assertEquals(false, scanner.seekBefore(toKV("c").getKey()));
69  
70      assertEquals(true, scanner.seekBefore(toKV("d").getKey()));
71      assertEquals("c", toRowStr(scanner.getKeyValue()));
72  
73      assertEquals(true, scanner.seekBefore(toKV("e").getKey()));
74      assertEquals("c", toRowStr(scanner.getKeyValue()));
75  
76      assertEquals(true, scanner.seekBefore(toKV("f").getKey()));
77      assertEquals("e", toRowStr(scanner.getKeyValue()));
78  
79      assertEquals(true, scanner.seekBefore(toKV("g").getKey()));
80      assertEquals("e", toRowStr(scanner.getKeyValue()));
81  
82      assertEquals(true, scanner.seekBefore(toKV("h").getKey()));
83      assertEquals("g", toRowStr(scanner.getKeyValue()));
84      assertEquals(true, scanner.seekBefore(toKV("i").getKey()));
85      assertEquals("g", toRowStr(scanner.getKeyValue()));
86      assertEquals(true, scanner.seekBefore(toKV("j").getKey()));
87      assertEquals("i", toRowStr(scanner.getKeyValue()));
88      assertEquals(true, scanner.seekBefore(toKV("k").getKey()));
89      assertEquals("i", toRowStr(scanner.getKeyValue()));
90      assertEquals(true, scanner.seekBefore(toKV("l").getKey()));
91      assertEquals("k", toRowStr(scanner.getKeyValue()));
92    }
93  
94    public void testSeekTo() throws Exception {
95      Path p = makeNewFile();
96      HFile.Reader reader = new HFile.Reader(fs, p, null, false);
97      reader.loadFileInfo();
98      assertEquals(2, reader.blockIndex.count);
99      HFileScanner scanner = reader.getScanner(false, true);
100     // lies before the start of the file.
101     assertEquals(-1, scanner.seekTo(toKV("a").getKey()));
102 
103     assertEquals(1, scanner.seekTo(toKV("d").getKey()));
104     assertEquals("c", toRowStr(scanner.getKeyValue()));
105 
106     // Across a block boundary now.
107     assertEquals(1, scanner.seekTo(toKV("h").getKey()));
108     assertEquals("g", toRowStr(scanner.getKeyValue()));
109 
110     assertEquals(1, scanner.seekTo(toKV("l").getKey()));
111     assertEquals("k", toRowStr(scanner.getKeyValue()));
112   }
113 
114   public void testBlockContainingKey() throws Exception {
115     Path p = makeNewFile();
116     HFile.Reader reader = new HFile.Reader(fs, p, null, false);
117     reader.loadFileInfo();
118     System.out.println(reader.blockIndex.toString());
119     int klen = toKV("a").getKey().length;
120     // falls before the start of the file.
121     assertEquals(-1, reader.blockIndex.blockContainingKey(toKV("a").getKey(),
122         0, klen));
123     assertEquals(0, reader.blockIndex.blockContainingKey(toKV("c").getKey(), 0,
124         klen));
125     assertEquals(0, reader.blockIndex.blockContainingKey(toKV("d").getKey(), 0,
126         klen));
127     assertEquals(0, reader.blockIndex.blockContainingKey(toKV("e").getKey(), 0,
128         klen));
129     assertEquals(0, reader.blockIndex.blockContainingKey(toKV("g").getKey(), 0,
130         klen));
131     assertEquals(0, reader.blockIndex.blockContainingKey(toKV("h").getKey(), 0,
132         klen));
133     assertEquals(1, reader.blockIndex.blockContainingKey(toKV("i").getKey(), 0,
134         klen));
135     assertEquals(1, reader.blockIndex.blockContainingKey(toKV("j").getKey(), 0,
136         klen));
137     assertEquals(1, reader.blockIndex.blockContainingKey(toKV("k").getKey(), 0,
138         klen));
139     assertEquals(1, reader.blockIndex.blockContainingKey(toKV("l").getKey(), 0,
140         klen));
141   }
142 }