001/*
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.io.encoding;
019
020import static org.junit.Assert.assertEquals;
021
022import java.io.IOException;
023import java.nio.ByteBuffer;
024import java.util.ArrayList;
025import java.util.Collection;
026import java.util.List;
027import org.apache.hadoop.conf.Configuration;
028import org.apache.hadoop.hbase.Cell;
029import org.apache.hadoop.hbase.HBaseClassTestRule;
030import org.apache.hadoop.hbase.HBaseCommonTestingUtility;
031import org.apache.hadoop.hbase.HBaseConfiguration;
032import org.apache.hadoop.hbase.HConstants;
033import org.apache.hadoop.hbase.KeyValue;
034import org.apache.hadoop.hbase.PrivateCellUtil;
035import org.apache.hadoop.hbase.io.compress.Compression;
036import org.apache.hadoop.hbase.io.hfile.HFileContext;
037import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
038import org.apache.hadoop.hbase.nio.SingleByteBuff;
039import org.apache.hadoop.hbase.testclassification.IOTests;
040import org.apache.hadoop.hbase.testclassification.SmallTests;
041import org.apache.hadoop.hbase.util.Bytes;
042import org.junit.ClassRule;
043import org.junit.Test;
044import org.junit.experimental.categories.Category;
045import org.junit.runner.RunWith;
046import org.junit.runners.Parameterized;
047import org.junit.runners.Parameterized.Parameters;
048
049@Category({ IOTests.class, SmallTests.class })
050@RunWith(Parameterized.class)
051public class TestSeekToBlockWithEncoders {
052
053  @ClassRule
054  public static final HBaseClassTestRule CLASS_RULE =
055    HBaseClassTestRule.forClass(TestSeekToBlockWithEncoders.class);
056
057  static final byte[] HFILEBLOCK_DUMMY_HEADER = new byte[HConstants.HFILEBLOCK_HEADER_SIZE];
058  private final boolean useOffheapData;
059  private final Configuration conf = HBaseConfiguration.create();
060
061  @Parameters
062  public static Collection<Object[]> parameters() {
063    return HBaseCommonTestingUtility.BOOLEAN_PARAMETERIZED;
064  }
065
066  public TestSeekToBlockWithEncoders(boolean useOffheapData) {
067    this.useOffheapData = useOffheapData;
068  }
069
070  /**
071   * Test seeking while file is encoded.
072   */
073  @Test
074  public void testSeekToBlockWithNonMatchingSeekKey() throws IOException {
075    List<KeyValue> sampleKv = new ArrayList<>();
076    KeyValue kv1 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
077      Bytes.toBytes("val"));
078    sampleKv.add(kv1);
079    KeyValue kv2 = new KeyValue(Bytes.toBytes("aab"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
080      Bytes.toBytes("val"));
081    sampleKv.add(kv2);
082    KeyValue kv3 = new KeyValue(Bytes.toBytes("aac"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
083      Bytes.toBytes("val"));
084    sampleKv.add(kv3);
085    KeyValue kv4 = new KeyValue(Bytes.toBytes("aad"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
086      Bytes.toBytes("val"));
087    sampleKv.add(kv4);
088    KeyValue kv5 = new KeyValue(Bytes.toBytes("bba"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
089      Bytes.toBytes("val"));
090    sampleKv.add(kv5);
091    KeyValue toSeek = new KeyValue(Bytes.toBytes("aae"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
092      Bytes.toBytes("val"));
093    seekToTheKey(kv4, sampleKv, toSeek);
094  }
095
096  /**
097   * Test seeking while file is encoded.
098   */
099  @Test
100  public void testSeekingToBlockWithBiggerNonLength1() throws IOException {
101    List<KeyValue> sampleKv = new ArrayList<>();
102    KeyValue kv1 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
103      Bytes.toBytes("val"));
104    sampleKv.add(kv1);
105    KeyValue kv2 = new KeyValue(Bytes.toBytes("aab"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
106      Bytes.toBytes("val"));
107    sampleKv.add(kv2);
108    KeyValue kv3 = new KeyValue(Bytes.toBytes("aac"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
109      Bytes.toBytes("val"));
110    sampleKv.add(kv3);
111    KeyValue kv4 = new KeyValue(Bytes.toBytes("aad"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
112      Bytes.toBytes("val"));
113    sampleKv.add(kv4);
114    KeyValue kv5 = new KeyValue(Bytes.toBytes("aaddd"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
115      Bytes.toBytes("val"));
116    sampleKv.add(kv5);
117    KeyValue toSeek = new KeyValue(Bytes.toBytes("aaaa"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
118      Bytes.toBytes("val"));
119    seekToTheKey(kv1, sampleKv, toSeek);
120  }
121
122  /**
123   * Test seeking while file is encoded.
124   */
125  @Test
126  public void testSeekingToBlockToANotAvailableKey() throws IOException {
127    List<KeyValue> sampleKv = new ArrayList<>();
128    KeyValue kv1 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
129      Bytes.toBytes("val"));
130    sampleKv.add(kv1);
131    KeyValue kv2 = new KeyValue(Bytes.toBytes("aab"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
132      Bytes.toBytes("val"));
133    sampleKv.add(kv2);
134    KeyValue kv3 = new KeyValue(Bytes.toBytes("aac"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
135      Bytes.toBytes("val"));
136    sampleKv.add(kv3);
137    KeyValue kv4 = new KeyValue(Bytes.toBytes("aade"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
138      Bytes.toBytes("val"));
139    sampleKv.add(kv4);
140    KeyValue kv5 = new KeyValue(Bytes.toBytes("bbbcd"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
141      Bytes.toBytes("val"));
142    sampleKv.add(kv5);
143    KeyValue toSeek = new KeyValue(Bytes.toBytes("bbbce"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
144      Bytes.toBytes("val"));
145    seekToTheKey(kv5, sampleKv, toSeek);
146  }
147
148  /**
149   * Test seeking while file is encoded.
150   */
151  @Test
152  public void testSeekToBlockWithDecreasingCommonPrefix() throws IOException {
153    List<KeyValue> sampleKv = new ArrayList<>();
154    KeyValue kv1 = new KeyValue(Bytes.toBytes("row10aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
155      Bytes.toBytes("val"));
156    sampleKv.add(kv1);
157    KeyValue kv2 = new KeyValue(Bytes.toBytes("row10aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q2"),
158      Bytes.toBytes("val"));
159    sampleKv.add(kv2);
160    KeyValue kv3 = new KeyValue(Bytes.toBytes("row10aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q3"),
161      Bytes.toBytes("val"));
162    sampleKv.add(kv3);
163    KeyValue kv4 = new KeyValue(Bytes.toBytes("row11baa"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
164      Bytes.toBytes("val"));
165    sampleKv.add(kv4);
166    Cell toSeek = PrivateCellUtil.createLastOnRow(kv3);
167    seekToTheKey(kv3, sampleKv, toSeek);
168  }
169
170  @Test
171  public void testSeekToBlockWithDiffQualifer() throws IOException {
172    List<KeyValue> sampleKv = new ArrayList<>();
173    KeyValue kv1 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
174      Bytes.toBytes("val"));
175    sampleKv.add(kv1);
176    KeyValue kv2 = new KeyValue(Bytes.toBytes("aab"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
177      Bytes.toBytes("val"));
178    sampleKv.add(kv2);
179    KeyValue kv4 = new KeyValue(Bytes.toBytes("aac"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
180      Bytes.toBytes("val"));
181    sampleKv.add(kv4);
182    KeyValue kv5 = new KeyValue(Bytes.toBytes("aac"), Bytes.toBytes("f1"), Bytes.toBytes("q2"),
183      Bytes.toBytes("val"));
184    sampleKv.add(kv5);
185    KeyValue toSeek = new KeyValue(Bytes.toBytes("aac"), Bytes.toBytes("f1"), Bytes.toBytes("q2"),
186      Bytes.toBytes("val"));
187    seekToTheKey(kv5, sampleKv, toSeek);
188  }
189
190  @Test
191  public void testSeekToBlockWithDiffQualiferOnSameRow() throws IOException {
192    List<KeyValue> sampleKv = new ArrayList<>();
193    KeyValue kv1 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
194      Bytes.toBytes("val"));
195    sampleKv.add(kv1);
196    KeyValue kv2 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q2"),
197      Bytes.toBytes("val"));
198    sampleKv.add(kv2);
199    KeyValue kv4 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q3"),
200      Bytes.toBytes("val"));
201    sampleKv.add(kv4);
202    KeyValue kv5 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q4"),
203      Bytes.toBytes("val"));
204    sampleKv.add(kv5);
205    KeyValue kv6 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q5"),
206      Bytes.toBytes("val"));
207    sampleKv.add(kv6);
208    KeyValue toSeek = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q5"),
209      Bytes.toBytes("val"));
210    seekToTheKey(kv6, sampleKv, toSeek);
211  }
212
213  @Test
214  public void testSeekToBlockWithDiffQualiferOnSameRow1() throws IOException {
215    List<KeyValue> sampleKv = new ArrayList<>();
216    KeyValue kv1 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
217      Bytes.toBytes("val"));
218    sampleKv.add(kv1);
219    KeyValue kv2 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q2"),
220      Bytes.toBytes("val"));
221    sampleKv.add(kv2);
222    KeyValue kv4 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q3"),
223      Bytes.toBytes("val"));
224    sampleKv.add(kv4);
225    KeyValue kv5 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q4"),
226      Bytes.toBytes("val"));
227    sampleKv.add(kv5);
228    KeyValue kv6 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("z5"),
229      Bytes.toBytes("val"));
230    sampleKv.add(kv6);
231    KeyValue toSeek = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q5"),
232      Bytes.toBytes("val"));
233    seekToTheKey(kv5, sampleKv, toSeek);
234  }
235
236  @Test
237  public void testSeekToBlockWithDiffQualiferOnSameRowButDescendingInSize() throws IOException {
238    List<KeyValue> sampleKv = new ArrayList<>();
239    KeyValue kv1 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("qual1"),
240      Bytes.toBytes("val"));
241    sampleKv.add(kv1);
242    KeyValue kv2 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("qual2"),
243      Bytes.toBytes("val"));
244    sampleKv.add(kv2);
245    KeyValue kv4 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("qual3"),
246      Bytes.toBytes("val"));
247    sampleKv.add(kv4);
248    KeyValue kv5 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("qual4"),
249      Bytes.toBytes("val"));
250    sampleKv.add(kv5);
251    KeyValue kv6 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("qz"),
252      Bytes.toBytes("val"));
253    sampleKv.add(kv6);
254    KeyValue toSeek = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("qz"),
255      Bytes.toBytes("val"));
256    seekToTheKey(kv6, sampleKv, toSeek);
257  }
258
259  @Test
260  public void testSeekToBlockWithDiffFamilyAndQualifer() throws IOException {
261    List<KeyValue> sampleKv = new ArrayList<>();
262    KeyValue kv1 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("fam1"), Bytes.toBytes("q1"),
263      Bytes.toBytes("val"));
264    sampleKv.add(kv1);
265    KeyValue kv2 = new KeyValue(Bytes.toBytes("aab"), Bytes.toBytes("fam1"), Bytes.toBytes("q1"),
266      Bytes.toBytes("val"));
267    sampleKv.add(kv2);
268    KeyValue kv4 = new KeyValue(Bytes.toBytes("aac"), Bytes.toBytes("fam1"), Bytes.toBytes("q1"),
269      Bytes.toBytes("val"));
270    sampleKv.add(kv4);
271    KeyValue kv5 = new KeyValue(Bytes.toBytes("aac"), Bytes.toBytes("fam1"), Bytes.toBytes("q2"),
272      Bytes.toBytes("val"));
273    sampleKv.add(kv5);
274    KeyValue toSeek = new KeyValue(Bytes.toBytes("aac"), Bytes.toBytes("fam2"), Bytes.toBytes("q2"),
275      Bytes.toBytes("val"));
276    seekToTheKey(kv5, sampleKv, toSeek);
277  }
278
279  private void seekToTheKey(KeyValue expected, List<KeyValue> kvs, Cell toSeek) throws IOException {
280    // create all seekers
281    List<DataBlockEncoder.EncodedSeeker> encodedSeekers = new ArrayList<>();
282    for (DataBlockEncoding encoding : DataBlockEncoding.values()) {
283      if (encoding.getEncoder() == null) {
284        continue;
285      }
286      DataBlockEncoder encoder = encoding.getEncoder();
287      HFileContext meta = new HFileContextBuilder().withHBaseCheckSum(false).withIncludesMvcc(false)
288        .withIncludesTags(false).withCompression(Compression.Algorithm.NONE).build();
289      HFileBlockEncodingContext encodingContext =
290        encoder.newDataBlockEncodingContext(conf, encoding, HFILEBLOCK_DUMMY_HEADER, meta);
291      ByteBuffer encodedBuffer =
292        TestDataBlockEncoders.encodeKeyValues(encoding, kvs, encodingContext, this.useOffheapData);
293      DataBlockEncoder.EncodedSeeker seeker =
294        encoder.createSeeker(encoder.newDataBlockDecodingContext(conf, meta));
295      seeker.setCurrentBuffer(new SingleByteBuff(encodedBuffer));
296      encodedSeekers.add(seeker);
297    }
298    // test it!
299    // try a few random seeks
300    checkSeekingConsistency(encodedSeekers, toSeek, expected);
301  }
302
303  private void checkSeekingConsistency(List<DataBlockEncoder.EncodedSeeker> encodedSeekers,
304    Cell keyValue, KeyValue expected) {
305    for (DataBlockEncoder.EncodedSeeker seeker : encodedSeekers) {
306      seeker.seekToKeyInBlock(keyValue, false);
307      Cell keyValue2 = seeker.getCell();
308      assertEquals(expected, keyValue2);
309      seeker.rewind();
310    }
311  }
312}