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.jupiter.api.Assertions.assertEquals;
021
022import java.io.IOException;
023import java.nio.ByteBuffer;
024import java.util.ArrayList;
025import java.util.List;
026import java.util.stream.Stream;
027import org.apache.hadoop.conf.Configuration;
028import org.apache.hadoop.hbase.Cell;
029import org.apache.hadoop.hbase.ExtendedCell;
030import org.apache.hadoop.hbase.HBaseConfiguration;
031import org.apache.hadoop.hbase.HBaseParameterizedTestTemplate;
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.jupiter.api.Tag;
043import org.junit.jupiter.api.TestTemplate;
044import org.junit.jupiter.params.provider.Arguments;
045
046@Tag(IOTests.TAG)
047@Tag(SmallTests.TAG)
048@HBaseParameterizedTestTemplate(name = "dataEncoding={0}")
049public class TestSeekToBlockWithEncoders {
050
051  static final byte[] HFILEBLOCK_DUMMY_HEADER = new byte[HConstants.HFILEBLOCK_HEADER_SIZE];
052  private final boolean useOffheapData;
053  private final Configuration conf = HBaseConfiguration.create();
054
055  public static Stream<Arguments> parameters() {
056    return Stream.of(Arguments.of(true), Arguments.of(false));
057  }
058
059  public TestSeekToBlockWithEncoders(boolean useOffheapData) {
060    this.useOffheapData = useOffheapData;
061  }
062
063  /**
064   * Test seeking while file is encoded.
065   */
066  @TestTemplate
067  public void testSeekToBlockWithNonMatchingSeekKey() throws IOException {
068    List<KeyValue> sampleKv = new ArrayList<>();
069    KeyValue kv1 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
070      Bytes.toBytes("val"));
071    sampleKv.add(kv1);
072    KeyValue kv2 = new KeyValue(Bytes.toBytes("aab"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
073      Bytes.toBytes("val"));
074    sampleKv.add(kv2);
075    KeyValue kv3 = new KeyValue(Bytes.toBytes("aac"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
076      Bytes.toBytes("val"));
077    sampleKv.add(kv3);
078    KeyValue kv4 = new KeyValue(Bytes.toBytes("aad"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
079      Bytes.toBytes("val"));
080    sampleKv.add(kv4);
081    KeyValue kv5 = new KeyValue(Bytes.toBytes("bba"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
082      Bytes.toBytes("val"));
083    sampleKv.add(kv5);
084    KeyValue toSeek = new KeyValue(Bytes.toBytes("aae"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
085      Bytes.toBytes("val"));
086    seekToTheKey(kv4, sampleKv, toSeek);
087  }
088
089  /**
090   * Test seeking while file is encoded.
091   */
092  @TestTemplate
093  public void testSeekingToBlockWithBiggerNonLength1() throws IOException {
094    List<KeyValue> sampleKv = new ArrayList<>();
095    KeyValue kv1 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
096      Bytes.toBytes("val"));
097    sampleKv.add(kv1);
098    KeyValue kv2 = new KeyValue(Bytes.toBytes("aab"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
099      Bytes.toBytes("val"));
100    sampleKv.add(kv2);
101    KeyValue kv3 = new KeyValue(Bytes.toBytes("aac"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
102      Bytes.toBytes("val"));
103    sampleKv.add(kv3);
104    KeyValue kv4 = new KeyValue(Bytes.toBytes("aad"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
105      Bytes.toBytes("val"));
106    sampleKv.add(kv4);
107    KeyValue kv5 = new KeyValue(Bytes.toBytes("aaddd"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
108      Bytes.toBytes("val"));
109    sampleKv.add(kv5);
110    KeyValue toSeek = new KeyValue(Bytes.toBytes("aaaa"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
111      Bytes.toBytes("val"));
112    seekToTheKey(kv1, sampleKv, toSeek);
113  }
114
115  /**
116   * Test seeking while file is encoded.
117   */
118  @TestTemplate
119  public void testSeekingToBlockToANotAvailableKey() throws IOException {
120    List<KeyValue> sampleKv = new ArrayList<>();
121    KeyValue kv1 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
122      Bytes.toBytes("val"));
123    sampleKv.add(kv1);
124    KeyValue kv2 = new KeyValue(Bytes.toBytes("aab"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
125      Bytes.toBytes("val"));
126    sampleKv.add(kv2);
127    KeyValue kv3 = new KeyValue(Bytes.toBytes("aac"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
128      Bytes.toBytes("val"));
129    sampleKv.add(kv3);
130    KeyValue kv4 = new KeyValue(Bytes.toBytes("aade"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
131      Bytes.toBytes("val"));
132    sampleKv.add(kv4);
133    KeyValue kv5 = new KeyValue(Bytes.toBytes("bbbcd"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
134      Bytes.toBytes("val"));
135    sampleKv.add(kv5);
136    KeyValue toSeek = new KeyValue(Bytes.toBytes("bbbce"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
137      Bytes.toBytes("val"));
138    seekToTheKey(kv5, sampleKv, toSeek);
139  }
140
141  /**
142   * Test seeking while file is encoded.
143   */
144  @TestTemplate
145  public void testSeekToBlockWithDecreasingCommonPrefix() throws IOException {
146    List<KeyValue> sampleKv = new ArrayList<>();
147    KeyValue kv1 = new KeyValue(Bytes.toBytes("row10aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
148      Bytes.toBytes("val"));
149    sampleKv.add(kv1);
150    KeyValue kv2 = new KeyValue(Bytes.toBytes("row10aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q2"),
151      Bytes.toBytes("val"));
152    sampleKv.add(kv2);
153    KeyValue kv3 = new KeyValue(Bytes.toBytes("row10aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q3"),
154      Bytes.toBytes("val"));
155    sampleKv.add(kv3);
156    KeyValue kv4 = new KeyValue(Bytes.toBytes("row11baa"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
157      Bytes.toBytes("val"));
158    sampleKv.add(kv4);
159    ExtendedCell toSeek = PrivateCellUtil.createLastOnRow(kv3);
160    seekToTheKey(kv3, sampleKv, toSeek);
161  }
162
163  @TestTemplate
164  public void testSeekToBlockWithDiffQualifer() throws IOException {
165    List<KeyValue> sampleKv = new ArrayList<>();
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("aab"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
170      Bytes.toBytes("val"));
171    sampleKv.add(kv2);
172    KeyValue kv4 = new KeyValue(Bytes.toBytes("aac"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
173      Bytes.toBytes("val"));
174    sampleKv.add(kv4);
175    KeyValue kv5 = new KeyValue(Bytes.toBytes("aac"), Bytes.toBytes("f1"), Bytes.toBytes("q2"),
176      Bytes.toBytes("val"));
177    sampleKv.add(kv5);
178    KeyValue toSeek = new KeyValue(Bytes.toBytes("aac"), Bytes.toBytes("f1"), Bytes.toBytes("q2"),
179      Bytes.toBytes("val"));
180    seekToTheKey(kv5, sampleKv, toSeek);
181  }
182
183  @TestTemplate
184  public void testSeekToBlockWithDiffQualiferOnSameRow() throws IOException {
185    List<KeyValue> sampleKv = new ArrayList<>();
186    KeyValue kv1 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
187      Bytes.toBytes("val"));
188    sampleKv.add(kv1);
189    KeyValue kv2 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q2"),
190      Bytes.toBytes("val"));
191    sampleKv.add(kv2);
192    KeyValue kv4 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q3"),
193      Bytes.toBytes("val"));
194    sampleKv.add(kv4);
195    KeyValue kv5 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q4"),
196      Bytes.toBytes("val"));
197    sampleKv.add(kv5);
198    KeyValue kv6 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q5"),
199      Bytes.toBytes("val"));
200    sampleKv.add(kv6);
201    KeyValue toSeek = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q5"),
202      Bytes.toBytes("val"));
203    seekToTheKey(kv6, sampleKv, toSeek);
204  }
205
206  @TestTemplate
207  public void testSeekToBlockWithDiffQualiferOnSameRow1() throws IOException {
208    List<KeyValue> sampleKv = new ArrayList<>();
209    KeyValue kv1 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q1"),
210      Bytes.toBytes("val"));
211    sampleKv.add(kv1);
212    KeyValue kv2 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q2"),
213      Bytes.toBytes("val"));
214    sampleKv.add(kv2);
215    KeyValue kv4 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q3"),
216      Bytes.toBytes("val"));
217    sampleKv.add(kv4);
218    KeyValue kv5 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q4"),
219      Bytes.toBytes("val"));
220    sampleKv.add(kv5);
221    KeyValue kv6 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("z5"),
222      Bytes.toBytes("val"));
223    sampleKv.add(kv6);
224    KeyValue toSeek = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("q5"),
225      Bytes.toBytes("val"));
226    seekToTheKey(kv5, sampleKv, toSeek);
227  }
228
229  @TestTemplate
230  public void testSeekToBlockWithDiffQualiferOnSameRowButDescendingInSize() throws IOException {
231    List<KeyValue> sampleKv = new ArrayList<>();
232    KeyValue kv1 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("qual1"),
233      Bytes.toBytes("val"));
234    sampleKv.add(kv1);
235    KeyValue kv2 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("qual2"),
236      Bytes.toBytes("val"));
237    sampleKv.add(kv2);
238    KeyValue kv4 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("qual3"),
239      Bytes.toBytes("val"));
240    sampleKv.add(kv4);
241    KeyValue kv5 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("qual4"),
242      Bytes.toBytes("val"));
243    sampleKv.add(kv5);
244    KeyValue kv6 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("qz"),
245      Bytes.toBytes("val"));
246    sampleKv.add(kv6);
247    KeyValue toSeek = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("f1"), Bytes.toBytes("qz"),
248      Bytes.toBytes("val"));
249    seekToTheKey(kv6, sampleKv, toSeek);
250  }
251
252  @TestTemplate
253  public void testSeekToBlockWithDiffFamilyAndQualifer() throws IOException {
254    List<KeyValue> sampleKv = new ArrayList<>();
255    KeyValue kv1 = new KeyValue(Bytes.toBytes("aaa"), Bytes.toBytes("fam1"), Bytes.toBytes("q1"),
256      Bytes.toBytes("val"));
257    sampleKv.add(kv1);
258    KeyValue kv2 = new KeyValue(Bytes.toBytes("aab"), Bytes.toBytes("fam1"), Bytes.toBytes("q1"),
259      Bytes.toBytes("val"));
260    sampleKv.add(kv2);
261    KeyValue kv4 = new KeyValue(Bytes.toBytes("aac"), Bytes.toBytes("fam1"), Bytes.toBytes("q1"),
262      Bytes.toBytes("val"));
263    sampleKv.add(kv4);
264    KeyValue kv5 = new KeyValue(Bytes.toBytes("aac"), Bytes.toBytes("fam1"), Bytes.toBytes("q2"),
265      Bytes.toBytes("val"));
266    sampleKv.add(kv5);
267    KeyValue toSeek = new KeyValue(Bytes.toBytes("aac"), Bytes.toBytes("fam2"), Bytes.toBytes("q2"),
268      Bytes.toBytes("val"));
269    seekToTheKey(kv5, sampleKv, toSeek);
270  }
271
272  private void seekToTheKey(KeyValue expected, List<KeyValue> kvs, ExtendedCell toSeek)
273    throws IOException {
274    // create all seekers
275    List<DataBlockEncoder.EncodedSeeker> encodedSeekers = new ArrayList<>();
276    for (DataBlockEncoding encoding : DataBlockEncoding.values()) {
277      if (encoding.getEncoder() == null) {
278        continue;
279      }
280      DataBlockEncoder encoder = encoding.getEncoder();
281      HFileContext meta = new HFileContextBuilder().withHBaseCheckSum(false).withIncludesMvcc(false)
282        .withIncludesTags(false).withCompression(Compression.Algorithm.NONE).build();
283      HFileBlockEncodingContext encodingContext =
284        encoder.newDataBlockEncodingContext(conf, encoding, HFILEBLOCK_DUMMY_HEADER, meta);
285      ByteBuffer encodedBuffer =
286        TestDataBlockEncoders.encodeKeyValues(encoding, kvs, encodingContext, this.useOffheapData);
287      DataBlockEncoder.EncodedSeeker seeker =
288        encoder.createSeeker(encoder.newDataBlockDecodingContext(conf, meta));
289      seeker.setCurrentBuffer(new SingleByteBuff(encodedBuffer));
290      encodedSeekers.add(seeker);
291    }
292    // test it!
293    // try a few random seeks
294    checkSeekingConsistency(encodedSeekers, toSeek, expected);
295  }
296
297  private void checkSeekingConsistency(List<DataBlockEncoder.EncodedSeeker> encodedSeekers,
298    ExtendedCell keyValue, KeyValue expected) {
299    for (DataBlockEncoder.EncodedSeeker seeker : encodedSeekers) {
300      seeker.seekToKeyInBlock(keyValue, false);
301      Cell keyValue2 = seeker.getCell();
302      assertEquals(expected, keyValue2);
303      seeker.rewind();
304    }
305  }
306}