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}