View Javadoc

1   /*
2    * Licensed to the Apache Software Foundation (ASF) under one or more
3    * contributor license agreements. See the NOTICE file distributed with this
4    * work for additional information regarding copyright ownership. The ASF
5    * licenses this file to you under the Apache License, Version 2.0 (the
6    * "License"); you may not use this file except in compliance with the License.
7    * You may obtain a copy of the License at
8    *
9    * http://www.apache.org/licenses/LICENSE-2.0
10   *
11   * Unless required by applicable law or agreed to in writing, software
12   * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
13   * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
14   * License for the specific language governing permissions and limitations
15   * under the License.
16   */
17  package org.apache.hadoop.hbase.io.encoding;
18  
19  import static org.junit.Assert.assertEquals;
20  
21  import java.io.IOException;
22  import java.nio.ByteBuffer;
23  import java.util.ArrayList;
24  import java.util.List;
25  
26  import org.apache.hadoop.hbase.Cell;
27  import org.apache.hadoop.hbase.HConstants;
28  import org.apache.hadoop.hbase.KeyValue;
29  import org.apache.hadoop.hbase.KeyValueUtil;
30  import org.apache.hadoop.hbase.testclassification.SmallTests;
31  import org.apache.hadoop.hbase.io.compress.Compression;
32  import org.apache.hadoop.hbase.io.hfile.HFileContext;
33  import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
34  import org.apache.hadoop.hbase.util.Bytes;
35  import org.junit.Test;
36  import org.junit.experimental.categories.Category;
37  
38  @Category(SmallTests.class)
39  public class TestSeekToBlockWithEncoders {
40    static final byte[] HFILEBLOCK_DUMMY_HEADER = new byte[HConstants.HFILEBLOCK_HEADER_SIZE];
41  
42    /**
43     * Test seeking while file is encoded.
44     */
45    @Test
46    public void testSeekToBlockWithNonMatchingSeekKey() throws IOException {
47      List<KeyValue> sampleKv = new ArrayList<KeyValue>();
48      KeyValue kv1 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
49          Bytes.toBytes("val"));
50      sampleKv.add(kv1);
51      KeyValue kv2 = new KeyValue(Bytes.toBytes("aab"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
52          Bytes.toBytes("val"));
53      sampleKv.add(kv2);
54      KeyValue kv3 = new KeyValue(Bytes.toBytes("aac"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
55          Bytes.toBytes("val"));
56      sampleKv.add(kv3);
57      KeyValue kv4 = new KeyValue(Bytes.toBytes("aad"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
58          Bytes.toBytes("val"));
59      sampleKv.add(kv4);
60      KeyValue kv5 = new KeyValue(Bytes.toBytes("bba"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
61          Bytes.toBytes("val"));
62      sampleKv.add(kv5);
63      KeyValue toSeek = new KeyValue(Bytes.toBytes("aae"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
64          Bytes.toBytes("val"));
65      seekToTheKey(kv4, sampleKv, toSeek);
66    }
67  
68    /**
69     * Test seeking while file is encoded.
70     */
71    @Test
72    public void testSeekingToBlockWithBiggerNonLength1() throws IOException {
73      List<KeyValue> sampleKv = new ArrayList<KeyValue>();
74      KeyValue kv1 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
75          Bytes.toBytes("val"));
76      sampleKv.add(kv1);
77      KeyValue kv2 = new KeyValue(Bytes.toBytes("aab"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
78          Bytes.toBytes("val"));
79      sampleKv.add(kv2);
80      KeyValue kv3 = new KeyValue(Bytes.toBytes("aac"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
81          Bytes.toBytes("val"));
82      sampleKv.add(kv3);
83      KeyValue kv4 = new KeyValue(Bytes.toBytes("aad"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
84          Bytes.toBytes("val"));
85      sampleKv.add(kv4);
86      KeyValue kv5 = new KeyValue(Bytes.toBytes("aaddd"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
87          Bytes.toBytes("val"));
88      sampleKv.add(kv5);
89      KeyValue toSeek = new KeyValue(Bytes.toBytes("aaaa"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
90          Bytes.toBytes("val"));
91      seekToTheKey(kv1, sampleKv, toSeek);
92    }
93  
94    /**
95     * Test seeking while file is encoded.
96     */
97    @Test
98    public void testSeekingToBlockToANotAvailableKey() throws IOException {
99      List<KeyValue> sampleKv = new ArrayList<KeyValue>();
100     KeyValue kv1 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
101         Bytes.toBytes("val"));
102     sampleKv.add(kv1);
103     KeyValue kv2 = new KeyValue(Bytes.toBytes("aab"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
104         Bytes.toBytes("val"));
105     sampleKv.add(kv2);
106     KeyValue kv3 = new KeyValue(Bytes.toBytes("aac"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
107         Bytes.toBytes("val"));
108     sampleKv.add(kv3);
109     KeyValue kv4 = new KeyValue(Bytes.toBytes("aade"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
110         Bytes.toBytes("val"));
111     sampleKv.add(kv4);
112     KeyValue kv5 = new KeyValue(Bytes.toBytes("bbbcd"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
113         Bytes.toBytes("val"));
114     sampleKv.add(kv5);
115     KeyValue toSeek = new KeyValue(Bytes.toBytes("bbbce"), Bytes.toBytes("f1"),
116         Bytes.toBytes("q1"), Bytes.toBytes("val"));
117     seekToTheKey(kv5, sampleKv, toSeek);
118   }
119 
120   /**
121    * Test seeking while file is encoded.
122    */
123   @Test
124   public void testSeekToBlockWithDecreasingCommonPrefix() throws IOException {
125     List<KeyValue> sampleKv = new ArrayList<KeyValue>();
126     KeyValue kv1 = new KeyValue(Bytes.toBytes("row10aaa"), Bytes.toBytes("f1"),
127         Bytes.toBytes("q1"), Bytes.toBytes("val"));
128     sampleKv.add(kv1);
129     KeyValue kv2 = new KeyValue(Bytes.toBytes("row10aaa"), Bytes.toBytes("f1"),
130         Bytes.toBytes("q2"), Bytes.toBytes("val"));
131     sampleKv.add(kv2);
132     KeyValue kv3 = new KeyValue(Bytes.toBytes("row10aaa"), Bytes.toBytes("f1"),
133         Bytes.toBytes("q3"), Bytes.toBytes("val"));
134     sampleKv.add(kv3);
135     KeyValue kv4 = new KeyValue(Bytes.toBytes("row11baa"), Bytes.toBytes("f1"),
136         Bytes.toBytes("q1"), Bytes.toBytes("val"));
137     sampleKv.add(kv4);
138     KeyValue toSeek = KeyValueUtil.createLastOnRow(kv3.getRowArray(), kv3.getRowOffset(),
139         kv3.getRowLength(), null, 0, 0, null, 0, 0);
140     seekToTheKey(kv3, sampleKv, toSeek);
141   }
142 
143   @Test
144   public void testSeekToBlockWithDiffQualifer() throws IOException {
145     List<KeyValue> sampleKv = new ArrayList<KeyValue>();
146     KeyValue kv1 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
147         Bytes.toBytes("val"));
148     sampleKv.add(kv1);
149     KeyValue kv2 = new KeyValue(Bytes.toBytes("aab"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
150         Bytes.toBytes("val"));
151     sampleKv.add(kv2);
152     KeyValue kv4 = new KeyValue(Bytes.toBytes("aac"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
153         Bytes.toBytes("val"));
154     sampleKv.add(kv4);
155     KeyValue kv5 = new KeyValue(Bytes.toBytes("aac"), Bytes.toBytes("f1"), Bytes.toBytes("q2"),
156         Bytes.toBytes("val"));
157     sampleKv.add(kv5);
158     KeyValue toSeek = new KeyValue(Bytes.toBytes("aac"), Bytes.toBytes("f1"), Bytes.toBytes("q2"),
159         Bytes.toBytes("val"));
160     seekToTheKey(kv5, sampleKv, toSeek);
161   }
162 
163   @Test
164   public void testSeekToBlockWithDiffQualiferOnSameRow() throws IOException {
165     List<KeyValue> sampleKv = new ArrayList<KeyValue>();
166     KeyValue kv1 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
167         Bytes.toBytes("val"));
168     sampleKv.add(kv1);
169     KeyValue kv2 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q2"),
170         Bytes.toBytes("val"));
171     sampleKv.add(kv2);
172     KeyValue kv4 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q3"),
173         Bytes.toBytes("val"));
174     sampleKv.add(kv4);
175     KeyValue kv5 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q4"),
176         Bytes.toBytes("val"));
177     sampleKv.add(kv5);
178     KeyValue kv6 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q5"),
179         Bytes.toBytes("val"));
180     sampleKv.add(kv6);
181     KeyValue toSeek = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q5"),
182         Bytes.toBytes("val"));
183     seekToTheKey(kv6, sampleKv, toSeek);
184   }
185 
186   @Test
187   public void testSeekToBlockWithDiffQualiferOnSameRow1() throws IOException {
188     List<KeyValue> sampleKv = new ArrayList<KeyValue>();
189     KeyValue kv1 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
190         Bytes.toBytes("val"));
191     sampleKv.add(kv1);
192     KeyValue kv2 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q2"),
193         Bytes.toBytes("val"));
194     sampleKv.add(kv2);
195     KeyValue kv4 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q3"),
196         Bytes.toBytes("val"));
197     sampleKv.add(kv4);
198     KeyValue kv5 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q4"),
199         Bytes.toBytes("val"));
200     sampleKv.add(kv5);
201     KeyValue kv6 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("z5"),
202         Bytes.toBytes("val"));
203     sampleKv.add(kv6);
204     KeyValue toSeek = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q5"),
205         Bytes.toBytes("val"));
206     seekToTheKey(kv5, sampleKv, toSeek);
207   }
208 
209   @Test
210   public void testSeekToBlockWithDiffQualiferOnSameRowButDescendingInSize() throws IOException {
211     List<KeyValue> sampleKv = new ArrayList<KeyValue>();
212     KeyValue kv1 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("qual1"),
213         Bytes.toBytes("val"));
214     sampleKv.add(kv1);
215     KeyValue kv2 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("qual2"),
216         Bytes.toBytes("val"));
217     sampleKv.add(kv2);
218     KeyValue kv4 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("qual3"),
219         Bytes.toBytes("val"));
220     sampleKv.add(kv4);
221     KeyValue kv5 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("qual4"),
222         Bytes.toBytes("val"));
223     sampleKv.add(kv5);
224     KeyValue kv6 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("qz"),
225         Bytes.toBytes("val"));
226     sampleKv.add(kv6);
227     KeyValue toSeek = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("qz"),
228         Bytes.toBytes("val"));
229     seekToTheKey(kv6, sampleKv, toSeek);
230   }
231 
232   @Test
233   public void testSeekToBlockWithDiffFamilyAndQualifer() throws IOException {
234     List<KeyValue> sampleKv = new ArrayList<KeyValue>();
235     KeyValue kv1 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("fam1"), Bytes.toBytes("q1"),
236         Bytes.toBytes("val"));
237     sampleKv.add(kv1);
238     KeyValue kv2 = new KeyValue(Bytes.toBytes("aab"), Bytes.toBytes("fam1"), Bytes.toBytes("q1"),
239         Bytes.toBytes("val"));
240     sampleKv.add(kv2);
241     KeyValue kv4 = new KeyValue(Bytes.toBytes("aac"), Bytes.toBytes("fam1"), Bytes.toBytes("q1"),
242         Bytes.toBytes("val"));
243     sampleKv.add(kv4);
244     KeyValue kv5 = new KeyValue(Bytes.toBytes("aac"), Bytes.toBytes("fam1"), Bytes.toBytes("q2"),
245         Bytes.toBytes("val"));
246     sampleKv.add(kv5);
247     KeyValue toSeek = new KeyValue(Bytes.toBytes("aac"), Bytes.toBytes("fam2"),
248         Bytes.toBytes("q2"), Bytes.toBytes("val"));
249     seekToTheKey(kv5, sampleKv, toSeek);
250   }
251 
252   private void seekToTheKey(KeyValue expected, List<KeyValue> kvs, KeyValue toSeek)
253       throws IOException {
254     // create all seekers
255     List<DataBlockEncoder.EncodedSeeker> encodedSeekers = new ArrayList<DataBlockEncoder.EncodedSeeker>();
256     for (DataBlockEncoding encoding : DataBlockEncoding.values()) {
257       if (encoding.getEncoder() == null || encoding == DataBlockEncoding.PREFIX_TREE) {
258         continue;
259       }
260 
261       DataBlockEncoder encoder = encoding.getEncoder();
262       HFileContext meta = new HFileContextBuilder().withHBaseCheckSum(false)
263           .withIncludesMvcc(false).withIncludesTags(false)
264           .withCompression(Compression.Algorithm.NONE).build();
265       HFileBlockEncodingContext encodingContext = encoder.newDataBlockEncodingContext(encoding, HFILEBLOCK_DUMMY_HEADER, meta);
266       ByteBuffer encodedBuffer = TestDataBlockEncoders.encodeKeyValues(encoding, kvs,
267           encodingContext);
268       DataBlockEncoder.EncodedSeeker seeker = encoder.createSeeker(KeyValue.COMPARATOR,
269           encoder.newDataBlockDecodingContext(meta));
270       seeker.setCurrentBuffer(encodedBuffer);
271       encodedSeekers.add(seeker);
272     }
273     // test it!
274     // try a few random seeks
275     checkSeekingConsistency(encodedSeekers, toSeek, expected);
276   }
277 
278   private void checkSeekingConsistency(List<DataBlockEncoder.EncodedSeeker> encodedSeekers,
279       KeyValue keyValue, KeyValue expected) {
280     for (DataBlockEncoder.EncodedSeeker seeker : encodedSeekers) {
281       seeker.seekToKeyInBlock(
282           new KeyValue.KeyOnlyKeyValue(keyValue.getBuffer(), keyValue.getKeyOffset(), keyValue
283               .getKeyLength()), false);
284       Cell keyValue2 = seeker.getKeyValue();
285       assertEquals(expected, keyValue2);
286       seeker.rewind();
287     }
288   }
289 }