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  
41    /**
42     * Test seeking while file is encoded.
43     */
44    @Test
45    public void testSeekToBlockWithNonMatchingSeekKey() throws IOException {
46      List<KeyValue> sampleKv = new ArrayList<KeyValue>();
47      KeyValue kv1 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
48          Bytes.toBytes("val"));
49      sampleKv.add(kv1);
50      KeyValue kv2 = new KeyValue(Bytes.toBytes("aab"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
51          Bytes.toBytes("val"));
52      sampleKv.add(kv2);
53      KeyValue kv3 = new KeyValue(Bytes.toBytes("aac"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
54          Bytes.toBytes("val"));
55      sampleKv.add(kv3);
56      KeyValue kv4 = new KeyValue(Bytes.toBytes("aad"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
57          Bytes.toBytes("val"));
58      sampleKv.add(kv4);
59      KeyValue kv5 = new KeyValue(Bytes.toBytes("bba"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
60          Bytes.toBytes("val"));
61      sampleKv.add(kv5);
62      KeyValue toSeek = new KeyValue(Bytes.toBytes("aae"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
63          Bytes.toBytes("val"));
64      seekToTheKey(kv4, sampleKv, toSeek);
65    }
66  
67    /**
68     * Test seeking while file is encoded.
69     */
70    @Test
71    public void testSeekingToBlockWithBiggerNonLength1() throws IOException {
72      List<KeyValue> sampleKv = new ArrayList<KeyValue>();
73      KeyValue kv1 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
74          Bytes.toBytes("val"));
75      sampleKv.add(kv1);
76      KeyValue kv2 = new KeyValue(Bytes.toBytes("aab"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
77          Bytes.toBytes("val"));
78      sampleKv.add(kv2);
79      KeyValue kv3 = new KeyValue(Bytes.toBytes("aac"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
80          Bytes.toBytes("val"));
81      sampleKv.add(kv3);
82      KeyValue kv4 = new KeyValue(Bytes.toBytes("aad"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
83          Bytes.toBytes("val"));
84      sampleKv.add(kv4);
85      KeyValue kv5 = new KeyValue(Bytes.toBytes("aaaad"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
86          Bytes.toBytes("val"));
87      sampleKv.add(kv5);
88      KeyValue toSeek = new KeyValue(Bytes.toBytes("aaaa"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
89          Bytes.toBytes("val"));
90      seekToTheKey(kv1, sampleKv, toSeek);
91    }
92  
93    /**
94     * Test seeking while file is encoded.
95     */
96    @Test
97    public void testSeekingToBlockToANotAvailableKey() throws IOException {
98      List<KeyValue> sampleKv = new ArrayList<KeyValue>();
99      KeyValue kv1 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
100         Bytes.toBytes("val"));
101     sampleKv.add(kv1);
102     KeyValue kv2 = new KeyValue(Bytes.toBytes("aab"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
103         Bytes.toBytes("val"));
104     sampleKv.add(kv2);
105     KeyValue kv3 = new KeyValue(Bytes.toBytes("aac"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
106         Bytes.toBytes("val"));
107     sampleKv.add(kv3);
108     KeyValue kv4 = new KeyValue(Bytes.toBytes("aaae"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
109         Bytes.toBytes("val"));
110     sampleKv.add(kv4);
111     KeyValue kv5 = new KeyValue(Bytes.toBytes("bbbcd"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
112         Bytes.toBytes("val"));
113     sampleKv.add(kv5);
114     KeyValue toSeek = new KeyValue(Bytes.toBytes("bbbce"), Bytes.toBytes("f1"),
115         Bytes.toBytes("q1"), Bytes.toBytes("val"));
116     seekToTheKey(kv5, sampleKv, toSeek);
117   }
118 
119   /**
120    * Test seeking while file is encoded.
121    */
122   @Test
123   public void testSeekToBlockWithDecreasingCommonPrefix() throws IOException {
124     List<KeyValue> sampleKv = new ArrayList<KeyValue>();
125     KeyValue kv1 = new KeyValue(Bytes.toBytes("row10aaa"), Bytes.toBytes("f1"),
126         Bytes.toBytes("q1"), Bytes.toBytes("val"));
127     sampleKv.add(kv1);
128     KeyValue kv2 = new KeyValue(Bytes.toBytes("row10aaa"), Bytes.toBytes("f1"),
129         Bytes.toBytes("q2"), Bytes.toBytes("val"));
130     sampleKv.add(kv2);
131     KeyValue kv3 = new KeyValue(Bytes.toBytes("row10aaa"), Bytes.toBytes("f1"),
132         Bytes.toBytes("q3"), Bytes.toBytes("val"));
133     sampleKv.add(kv3);
134     KeyValue kv4 = new KeyValue(Bytes.toBytes("row11baa"), Bytes.toBytes("f1"),
135         Bytes.toBytes("q1"), Bytes.toBytes("val"));
136     sampleKv.add(kv4);
137     KeyValue toSeek = KeyValueUtil.createLastOnRow(kv3.getRowArray(), kv3.getRowOffset(),
138         kv3.getRowLength(), null, 0, 0, null, 0, 0);
139     seekToTheKey(kv3, sampleKv, toSeek);
140   }
141 
142   @Test
143   public void testSeekToBlockWithDiffQualifer() throws IOException {
144     List<KeyValue> sampleKv = new ArrayList<KeyValue>();
145     KeyValue kv1 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
146         Bytes.toBytes("val"));
147     sampleKv.add(kv1);
148     KeyValue kv2 = new KeyValue(Bytes.toBytes("aab"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
149         Bytes.toBytes("val"));
150     sampleKv.add(kv2);
151     KeyValue kv4 = new KeyValue(Bytes.toBytes("aac"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
152         Bytes.toBytes("val"));
153     sampleKv.add(kv4);
154     KeyValue kv5 = new KeyValue(Bytes.toBytes("aac"), Bytes.toBytes("f1"), Bytes.toBytes("q2"),
155         Bytes.toBytes("val"));
156     sampleKv.add(kv5);
157     KeyValue toSeek = new KeyValue(Bytes.toBytes("aac"), Bytes.toBytes("f1"), Bytes.toBytes("q2"),
158         Bytes.toBytes("val"));
159     seekToTheKey(kv5, sampleKv, toSeek);
160   }
161 
162   @Test
163   public void testSeekToBlockWithDiffQualiferOnSameRow() throws IOException {
164     List<KeyValue> sampleKv = new ArrayList<KeyValue>();
165     KeyValue kv1 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
166         Bytes.toBytes("val"));
167     sampleKv.add(kv1);
168     KeyValue kv2 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q2"),
169         Bytes.toBytes("val"));
170     sampleKv.add(kv2);
171     KeyValue kv4 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q3"),
172         Bytes.toBytes("val"));
173     sampleKv.add(kv4);
174     KeyValue kv5 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q4"),
175         Bytes.toBytes("val"));
176     sampleKv.add(kv5);
177     KeyValue kv6 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q5"),
178         Bytes.toBytes("val"));
179     sampleKv.add(kv6);
180     KeyValue toSeek = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q5"),
181         Bytes.toBytes("val"));
182     seekToTheKey(kv6, sampleKv, toSeek);
183   }
184 
185   @Test
186   public void testSeekToBlockWithDiffQualiferOnSameRow1() throws IOException {
187     List<KeyValue> sampleKv = new ArrayList<KeyValue>();
188     KeyValue kv1 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
189         Bytes.toBytes("val"));
190     sampleKv.add(kv1);
191     KeyValue kv2 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q2"),
192         Bytes.toBytes("val"));
193     sampleKv.add(kv2);
194     KeyValue kv4 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q3"),
195         Bytes.toBytes("val"));
196     sampleKv.add(kv4);
197     KeyValue kv5 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q4"),
198         Bytes.toBytes("val"));
199     sampleKv.add(kv5);
200     KeyValue kv6 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("z5"),
201         Bytes.toBytes("val"));
202     sampleKv.add(kv6);
203     KeyValue toSeek = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q5"),
204         Bytes.toBytes("val"));
205     seekToTheKey(kv5, sampleKv, toSeek);
206   }
207 
208   @Test
209   public void testSeekToBlockWithDiffQualiferOnSameRowButDescendingInSize() throws IOException {
210     List<KeyValue> sampleKv = new ArrayList<KeyValue>();
211     KeyValue kv1 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("qual1"),
212         Bytes.toBytes("val"));
213     sampleKv.add(kv1);
214     KeyValue kv2 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("qual2"),
215         Bytes.toBytes("val"));
216     sampleKv.add(kv2);
217     KeyValue kv4 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("qual3"),
218         Bytes.toBytes("val"));
219     sampleKv.add(kv4);
220     KeyValue kv5 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("qual4"),
221         Bytes.toBytes("val"));
222     sampleKv.add(kv5);
223     KeyValue kv6 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("qz"),
224         Bytes.toBytes("val"));
225     sampleKv.add(kv6);
226     KeyValue toSeek = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("qz"),
227         Bytes.toBytes("val"));
228     seekToTheKey(kv6, sampleKv, toSeek);
229   }
230 
231   @Test
232   public void testSeekToBlockWithDiffFamilyAndQualifer() throws IOException {
233     List<KeyValue> sampleKv = new ArrayList<KeyValue>();
234     KeyValue kv1 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("fam1"), Bytes.toBytes("q1"),
235         Bytes.toBytes("val"));
236     sampleKv.add(kv1);
237     KeyValue kv2 = new KeyValue(Bytes.toBytes("aab"), Bytes.toBytes("fam1"), Bytes.toBytes("q1"),
238         Bytes.toBytes("val"));
239     sampleKv.add(kv2);
240     KeyValue kv4 = new KeyValue(Bytes.toBytes("aac"), Bytes.toBytes("fam1"), Bytes.toBytes("q1"),
241         Bytes.toBytes("val"));
242     sampleKv.add(kv4);
243     KeyValue kv5 = new KeyValue(Bytes.toBytes("aac"), Bytes.toBytes("fam1"), Bytes.toBytes("q2"),
244         Bytes.toBytes("val"));
245     sampleKv.add(kv5);
246     KeyValue toSeek = new KeyValue(Bytes.toBytes("aac"), Bytes.toBytes("fam2"),
247         Bytes.toBytes("q2"), Bytes.toBytes("val"));
248     seekToTheKey(kv5, sampleKv, toSeek);
249   }
250 
251   private void seekToTheKey(KeyValue expected, List<KeyValue> kvs, KeyValue toSeek)
252       throws IOException {
253     // create all seekers
254     List<DataBlockEncoder.EncodedSeeker> encodedSeekers = new ArrayList<DataBlockEncoder.EncodedSeeker>();
255     for (DataBlockEncoding encoding : DataBlockEncoding.values()) {
256       if (encoding.getEncoder() == null || encoding == DataBlockEncoding.PREFIX_TREE) {
257         continue;
258       }
259 
260       DataBlockEncoder encoder = encoding.getEncoder();
261       HFileContext meta = new HFileContextBuilder().withHBaseCheckSum(false)
262           .withIncludesMvcc(false).withIncludesTags(false)
263           .withCompression(Compression.Algorithm.NONE).build();
264       HFileBlockEncodingContext encodingContext = encoder.newDataBlockEncodingContext(encoding,
265           HConstants.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 }