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;
021import static org.junit.Assert.assertTrue;
022import static org.junit.Assert.fail;
023
024import java.io.ByteArrayInputStream;
025import java.io.DataInputStream;
026import java.io.DataOutputStream;
027import java.io.IOException;
028import java.nio.ByteBuffer;
029import java.util.ArrayList;
030import java.util.Collection;
031import java.util.List;
032import java.util.Random;
033import org.apache.hadoop.hbase.ArrayBackedTag;
034import org.apache.hadoop.hbase.Cell;
035import org.apache.hadoop.hbase.CellComparatorImpl;
036import org.apache.hadoop.hbase.CellUtil;
037import org.apache.hadoop.hbase.HBaseClassTestRule;
038import org.apache.hadoop.hbase.HBaseTestingUtility;
039import org.apache.hadoop.hbase.HConstants;
040import org.apache.hadoop.hbase.KeyValue;
041import org.apache.hadoop.hbase.KeyValue.Type;
042import org.apache.hadoop.hbase.PrivateCellUtil;
043import org.apache.hadoop.hbase.Tag;
044import org.apache.hadoop.hbase.io.ByteArrayOutputStream;
045import org.apache.hadoop.hbase.io.compress.Compression;
046import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
047import org.apache.hadoop.hbase.io.hfile.HFileContext;
048import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
049import org.apache.hadoop.hbase.nio.SingleByteBuff;
050import org.apache.hadoop.hbase.testclassification.IOTests;
051import org.apache.hadoop.hbase.testclassification.LargeTests;
052import org.apache.hadoop.hbase.util.Bytes;
053import org.apache.hadoop.hbase.util.RedundantKVGenerator;
054import org.junit.Assert;
055import org.junit.ClassRule;
056import org.junit.Test;
057import org.junit.experimental.categories.Category;
058import org.junit.runner.RunWith;
059import org.junit.runners.Parameterized;
060import org.junit.runners.Parameterized.Parameters;
061import org.slf4j.Logger;
062import org.slf4j.LoggerFactory;
063
064/**
065 * Test all of the data block encoding algorithms for correctness. Most of the
066 * class generate data which will test different branches in code.
067 */
068@Category({IOTests.class, LargeTests.class})
069@RunWith(Parameterized.class)
070public class TestDataBlockEncoders {
071
072  @ClassRule
073  public static final HBaseClassTestRule CLASS_RULE =
074      HBaseClassTestRule.forClass(TestDataBlockEncoders.class);
075
076  private static final Logger LOG = LoggerFactory.getLogger(TestDataBlockEncoders.class);
077
078  private static int NUMBER_OF_KV = 10000;
079  private static int NUM_RANDOM_SEEKS = 1000;
080
081  private static int ENCODED_DATA_OFFSET = HConstants.HFILEBLOCK_HEADER_SIZE
082      + DataBlockEncoding.ID_SIZE;
083  static final byte[] HFILEBLOCK_DUMMY_HEADER = new byte[HConstants.HFILEBLOCK_HEADER_SIZE];
084
085  private RedundantKVGenerator generator = new RedundantKVGenerator();
086  private Random randomizer = new Random(42L);
087
088  private final boolean includesMemstoreTS;
089  private final boolean includesTags;
090  private final boolean useOffheapData;
091
092  @Parameters
093  public static Collection<Object[]> parameters() {
094    return HBaseTestingUtility.memStoreTSTagsAndOffheapCombination();
095  }
096
097  public TestDataBlockEncoders(boolean includesMemstoreTS, boolean includesTag,
098      boolean useOffheapData) {
099    this.includesMemstoreTS = includesMemstoreTS;
100    this.includesTags = includesTag;
101    this.useOffheapData = useOffheapData;
102  }
103
104  private HFileBlockEncodingContext getEncodingContext(Compression.Algorithm algo,
105      DataBlockEncoding encoding) {
106    DataBlockEncoder encoder = encoding.getEncoder();
107    HFileContext meta = new HFileContextBuilder()
108                        .withHBaseCheckSum(false)
109                        .withIncludesMvcc(includesMemstoreTS)
110                        .withIncludesTags(includesTags)
111                        .withCompression(algo).build();
112    if (encoder != null) {
113      return encoder.newDataBlockEncodingContext(encoding, HFILEBLOCK_DUMMY_HEADER, meta);
114    } else {
115      return new HFileBlockDefaultEncodingContext(encoding, HFILEBLOCK_DUMMY_HEADER, meta);
116    }
117  }
118
119  /**
120   * Test data block encoding of empty KeyValue.
121   *
122   * @throws IOException
123   *           On test failure.
124   */
125  @Test
126  public void testEmptyKeyValues() throws IOException {
127    List<KeyValue> kvList = new ArrayList<>();
128    byte[] row = new byte[0];
129    byte[] family = new byte[0];
130    byte[] qualifier = new byte[0];
131    byte[] value = new byte[0];
132    if (!includesTags) {
133      kvList.add(new KeyValue(row, family, qualifier, 0L, value));
134      kvList.add(new KeyValue(row, family, qualifier, 0L, value));
135    } else {
136      byte[] metaValue1 = Bytes.toBytes("metaValue1");
137      byte[] metaValue2 = Bytes.toBytes("metaValue2");
138      kvList.add(new KeyValue(row, family, qualifier, 0L, value,
139          new Tag[] { new ArrayBackedTag((byte) 1, metaValue1) }));
140      kvList.add(new KeyValue(row, family, qualifier, 0L, value,
141          new Tag[] { new ArrayBackedTag((byte) 1, metaValue2) }));
142    }
143    testEncodersOnDataset(kvList, includesMemstoreTS, includesTags);
144  }
145
146  /**
147   * Test KeyValues with negative timestamp.
148   *
149   * @throws IOException
150   *           On test failure.
151   */
152  @Test
153  public void testNegativeTimestamps() throws IOException {
154    List<KeyValue> kvList = new ArrayList<>();
155    byte[] row = new byte[0];
156    byte[] family = new byte[0];
157    byte[] qualifier = new byte[0];
158    byte[] value = new byte[0];
159    if (includesTags) {
160      byte[] metaValue1 = Bytes.toBytes("metaValue1");
161      byte[] metaValue2 = Bytes.toBytes("metaValue2");
162      kvList.add(new KeyValue(row, family, qualifier, 0L, value,
163          new Tag[] { new ArrayBackedTag((byte) 1, metaValue1) }));
164      kvList.add(new KeyValue(row, family, qualifier, 0L, value,
165          new Tag[] { new ArrayBackedTag((byte) 1, metaValue2) }));
166    } else {
167      kvList.add(new KeyValue(row, family, qualifier, -1L, Type.Put, value));
168      kvList.add(new KeyValue(row, family, qualifier, -2L, Type.Put, value));
169    }
170    testEncodersOnDataset(kvList, includesMemstoreTS, includesTags);
171  }
172
173
174  /**
175   * Test whether compression -> decompression gives the consistent results on
176   * pseudorandom sample.
177   * @throws IOException On test failure.
178   */
179  @Test
180  public void testExecutionOnSample() throws IOException {
181    List<KeyValue> kvList = generator.generateTestKeyValues(NUMBER_OF_KV, includesTags);
182    testEncodersOnDataset(kvList, includesMemstoreTS, includesTags);
183  }
184
185  /**
186   * Test seeking while file is encoded.
187   */
188  @Test
189  public void testSeekingOnSample() throws IOException {
190    List<KeyValue> sampleKv = generator.generateTestKeyValues(NUMBER_OF_KV, includesTags);
191
192    // create all seekers
193    List<DataBlockEncoder.EncodedSeeker> encodedSeekers = new ArrayList<>();
194    for (DataBlockEncoding encoding : DataBlockEncoding.values()) {
195      LOG.info("Encoding: " + encoding);
196      DataBlockEncoder encoder = encoding.getEncoder();
197      if (encoder == null) {
198        continue;
199      }
200      LOG.info("Encoder: " + encoder);
201      ByteBuffer encodedBuffer = encodeKeyValues(encoding, sampleKv,
202          getEncodingContext(Compression.Algorithm.NONE, encoding), this.useOffheapData);
203      HFileContext meta = new HFileContextBuilder()
204                          .withHBaseCheckSum(false)
205                          .withIncludesMvcc(includesMemstoreTS)
206                          .withIncludesTags(includesTags)
207                          .withCompression(Compression.Algorithm.NONE)
208                          .build();
209      DataBlockEncoder.EncodedSeeker seeker = encoder.createSeeker(CellComparatorImpl.COMPARATOR,
210          encoder.newDataBlockDecodingContext(meta));
211      seeker.setCurrentBuffer(new SingleByteBuff(encodedBuffer));
212      encodedSeekers.add(seeker);
213    }
214    LOG.info("Testing it!");
215    // test it!
216    // try a few random seeks
217    for (boolean seekBefore : new boolean[] { false, true }) {
218      for (int i = 0; i < NUM_RANDOM_SEEKS; ++i) {
219        int keyValueId;
220        if (!seekBefore) {
221          keyValueId = randomizer.nextInt(sampleKv.size());
222        } else {
223          keyValueId = randomizer.nextInt(sampleKv.size() - 1) + 1;
224        }
225
226        KeyValue keyValue = sampleKv.get(keyValueId);
227        checkSeekingConsistency(encodedSeekers, seekBefore, keyValue);
228      }
229    }
230
231    // check edge cases
232    LOG.info("Checking edge cases");
233    checkSeekingConsistency(encodedSeekers, false, sampleKv.get(0));
234    for (boolean seekBefore : new boolean[] { false, true }) {
235      checkSeekingConsistency(encodedSeekers, seekBefore, sampleKv.get(sampleKv.size() - 1));
236      KeyValue midKv = sampleKv.get(sampleKv.size() / 2);
237      Cell lastMidKv =PrivateCellUtil.createLastOnRowCol(midKv);
238      checkSeekingConsistency(encodedSeekers, seekBefore, lastMidKv);
239    }
240    LOG.info("Done");
241  }
242
243  static ByteBuffer encodeKeyValues(DataBlockEncoding encoding, List<KeyValue> kvs,
244      HFileBlockEncodingContext encodingContext, boolean useOffheapData) throws IOException {
245    DataBlockEncoder encoder = encoding.getEncoder();
246    ByteArrayOutputStream baos = new ByteArrayOutputStream();
247    baos.write(HFILEBLOCK_DUMMY_HEADER);
248    DataOutputStream dos = new DataOutputStream(baos);
249    encoder.startBlockEncoding(encodingContext, dos);
250    for (KeyValue kv : kvs) {
251      encoder.encode(kv, encodingContext, dos);
252    }
253    encoder.endBlockEncoding(encodingContext, dos, baos.getBuffer());
254    byte[] encodedData = new byte[baos.size() - ENCODED_DATA_OFFSET];
255    System.arraycopy(baos.toByteArray(), ENCODED_DATA_OFFSET, encodedData, 0, encodedData.length);
256    if (useOffheapData) {
257      ByteBuffer bb = ByteBuffer.allocateDirect(encodedData.length);
258      bb.put(encodedData);
259      bb.rewind();
260      return bb;
261    }
262    return ByteBuffer.wrap(encodedData);
263  }
264
265  @Test
266  public void testNextOnSample() throws IOException {
267    List<KeyValue> sampleKv = generator.generateTestKeyValues(NUMBER_OF_KV, includesTags);
268
269    for (DataBlockEncoding encoding : DataBlockEncoding.values()) {
270      if (encoding.getEncoder() == null) {
271        continue;
272      }
273      DataBlockEncoder encoder = encoding.getEncoder();
274      ByteBuffer encodedBuffer = encodeKeyValues(encoding, sampleKv,
275          getEncodingContext(Compression.Algorithm.NONE, encoding), this.useOffheapData);
276      HFileContext meta = new HFileContextBuilder()
277                          .withHBaseCheckSum(false)
278                          .withIncludesMvcc(includesMemstoreTS)
279                          .withIncludesTags(includesTags)
280                          .withCompression(Compression.Algorithm.NONE)
281                          .build();
282      DataBlockEncoder.EncodedSeeker seeker = encoder.createSeeker(CellComparatorImpl.COMPARATOR,
283          encoder.newDataBlockDecodingContext(meta));
284      seeker.setCurrentBuffer(new SingleByteBuff(encodedBuffer));
285      int i = 0;
286      do {
287        KeyValue expectedKeyValue = sampleKv.get(i);
288        Cell cell = seeker.getCell();
289        if (PrivateCellUtil.compareKeyIgnoresMvcc(CellComparatorImpl.COMPARATOR, expectedKeyValue,
290          cell) != 0) {
291          int commonPrefix = PrivateCellUtil
292              .findCommonPrefixInFlatKey(expectedKeyValue, cell, false, true);
293          fail(String.format("next() produces wrong results "
294              + "encoder: %s i: %d commonPrefix: %d" + "\n expected %s\n actual      %s", encoder
295              .toString(), i, commonPrefix, Bytes.toStringBinary(expectedKeyValue.getBuffer(),
296              expectedKeyValue.getKeyOffset(), expectedKeyValue.getKeyLength()), CellUtil.toString(
297              cell, false)));
298        }
299        i++;
300      } while (seeker.next());
301    }
302  }
303
304  /**
305   * Test whether the decompression of first key is implemented correctly.
306   * @throws IOException
307   */
308  @Test
309  public void testFirstKeyInBlockOnSample() throws IOException {
310    List<KeyValue> sampleKv = generator.generateTestKeyValues(NUMBER_OF_KV, includesTags);
311
312    for (DataBlockEncoding encoding : DataBlockEncoding.values()) {
313      if (encoding.getEncoder() == null) {
314        continue;
315      }
316      DataBlockEncoder encoder = encoding.getEncoder();
317      ByteBuffer encodedBuffer = encodeKeyValues(encoding, sampleKv,
318          getEncodingContext(Compression.Algorithm.NONE, encoding), this.useOffheapData);
319      Cell key = encoder.getFirstKeyCellInBlock(new SingleByteBuff(encodedBuffer));
320      KeyValue firstKv = sampleKv.get(0);
321      if (0 != PrivateCellUtil.compareKeyIgnoresMvcc(CellComparatorImpl.COMPARATOR, key, firstKv)) {
322        int commonPrefix = PrivateCellUtil.findCommonPrefixInFlatKey(key, firstKv, false, true);
323        fail(String.format("Bug in '%s' commonPrefix %d", encoder.toString(), commonPrefix));
324      }
325    }
326  }
327
328  @Test
329  public void testRowIndexWithTagsButNoTagsInCell() throws IOException {
330    List<KeyValue> kvList = new ArrayList<>();
331    byte[] row = new byte[0];
332    byte[] family = new byte[0];
333    byte[] qualifier = new byte[0];
334    byte[] value = new byte[0];
335    KeyValue expectedKV = new KeyValue(row, family, qualifier, 1L, Type.Put, value);
336    kvList.add(expectedKV);
337    DataBlockEncoding encoding = DataBlockEncoding.ROW_INDEX_V1;
338    DataBlockEncoder encoder = encoding.getEncoder();
339    ByteBuffer encodedBuffer =
340        encodeKeyValues(encoding, kvList, getEncodingContext(Algorithm.NONE, encoding), false);
341    HFileContext meta =
342        new HFileContextBuilder().withHBaseCheckSum(false).withIncludesMvcc(includesMemstoreTS)
343            .withIncludesTags(includesTags).withCompression(Compression.Algorithm.NONE).build();
344    DataBlockEncoder.EncodedSeeker seeker = encoder.createSeeker(CellComparatorImpl.COMPARATOR,
345      encoder.newDataBlockDecodingContext(meta));
346    seeker.setCurrentBuffer(new SingleByteBuff(encodedBuffer));
347    Cell cell = seeker.getCell();
348    Assert.assertEquals(expectedKV.getLength(), ((KeyValue) cell).getLength());
349  }
350
351  private void checkSeekingConsistency(List<DataBlockEncoder.EncodedSeeker> encodedSeekers,
352      boolean seekBefore, Cell keyValue) {
353    Cell expectedKeyValue = null;
354    ByteBuffer expectedKey = null;
355    ByteBuffer expectedValue = null;
356    for (DataBlockEncoder.EncodedSeeker seeker : encodedSeekers) {
357      seeker.seekToKeyInBlock(keyValue, seekBefore);
358      seeker.rewind();
359
360      Cell actualKeyValue = seeker.getCell();
361      ByteBuffer actualKey = null;
362      actualKey = ByteBuffer.wrap(((KeyValue) seeker.getKey()).getKey());
363      ByteBuffer actualValue = seeker.getValueShallowCopy();
364
365      if (expectedKeyValue != null) {
366        assertTrue(CellUtil.equals(expectedKeyValue, actualKeyValue));
367      } else {
368        expectedKeyValue = actualKeyValue;
369      }
370
371      if (expectedKey != null) {
372        assertEquals(expectedKey, actualKey);
373      } else {
374        expectedKey = actualKey;
375      }
376
377      if (expectedValue != null) {
378        assertEquals(expectedValue, actualValue);
379      } else {
380        expectedValue = actualValue;
381      }
382    }
383  }
384
385  private void testEncodersOnDataset(List<KeyValue> kvList, boolean includesMemstoreTS,
386      boolean includesTags) throws IOException {
387    ByteBuffer unencodedDataBuf = RedundantKVGenerator.convertKvToByteBuffer(kvList,
388        includesMemstoreTS);
389    HFileContext fileContext = new HFileContextBuilder().withIncludesMvcc(includesMemstoreTS)
390        .withIncludesTags(includesTags).build();
391    for (DataBlockEncoding encoding : DataBlockEncoding.values()) {
392      DataBlockEncoder encoder = encoding.getEncoder();
393      if (encoder == null) {
394        continue;
395      }
396      HFileBlockEncodingContext encodingContext = new HFileBlockDefaultEncodingContext(encoding,
397          HFILEBLOCK_DUMMY_HEADER, fileContext);
398
399      ByteArrayOutputStream baos = new ByteArrayOutputStream();
400      baos.write(HFILEBLOCK_DUMMY_HEADER);
401      DataOutputStream dos = new DataOutputStream(baos);
402      encoder.startBlockEncoding(encodingContext, dos);
403      for (KeyValue kv : kvList) {
404        encoder.encode(kv, encodingContext, dos);
405      }
406      encoder.endBlockEncoding(encodingContext, dos, baos.getBuffer());
407      byte[] encodedData = baos.toByteArray();
408
409      testAlgorithm(encodedData, unencodedDataBuf, encoder);
410    }
411  }
412
413  @Test
414  public void testZeroByte() throws IOException {
415    List<KeyValue> kvList = new ArrayList<>();
416    byte[] row = Bytes.toBytes("abcd");
417    byte[] family = new byte[] { 'f' };
418    byte[] qualifier0 = new byte[] { 'b' };
419    byte[] qualifier1 = new byte[] { 'c' };
420    byte[] value0 = new byte[] { 'd' };
421    byte[] value1 = new byte[] { 0x00 };
422    if (includesTags) {
423      kvList.add(new KeyValue(row, family, qualifier0, 0, value0,
424          new Tag[] { new ArrayBackedTag((byte) 1, "value1") }));
425      kvList.add(new KeyValue(row, family, qualifier1, 0, value1,
426          new Tag[] { new ArrayBackedTag((byte) 1, "value1") }));
427    } else {
428      kvList.add(new KeyValue(row, family, qualifier0, 0, Type.Put, value0));
429      kvList.add(new KeyValue(row, family, qualifier1, 0, Type.Put, value1));
430    }
431    testEncodersOnDataset(kvList, includesMemstoreTS, includesTags);
432  }
433
434  private void testAlgorithm(byte[] encodedData, ByteBuffer unencodedDataBuf,
435      DataBlockEncoder encoder) throws IOException {
436    // decode
437    ByteArrayInputStream bais = new ByteArrayInputStream(encodedData, ENCODED_DATA_OFFSET,
438        encodedData.length - ENCODED_DATA_OFFSET);
439    DataInputStream dis = new DataInputStream(bais);
440    ByteBuffer actualDataset;
441    HFileContext meta = new HFileContextBuilder().withHBaseCheckSum(false)
442        .withIncludesMvcc(includesMemstoreTS).withIncludesTags(includesTags)
443        .withCompression(Compression.Algorithm.NONE).build();
444    actualDataset = encoder.decodeKeyValues(dis, encoder.newDataBlockDecodingContext(meta));
445    actualDataset.rewind();
446
447    // this is because in case of prefix tree the decoded stream will not have
448    // the
449    // mvcc in it.
450    assertEquals("Encoding -> decoding gives different results for " + encoder,
451        Bytes.toStringBinary(unencodedDataBuf), Bytes.toStringBinary(actualDataset));
452  }
453}