1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17 package org.apache.hadoop.hbase.io.encoding;
18
19 import static org.junit.Assert.assertEquals;
20 import static org.junit.Assert.fail;
21
22 import java.io.ByteArrayInputStream;
23 import java.io.DataInputStream;
24 import java.io.DataOutputStream;
25 import java.io.IOException;
26 import java.nio.ByteBuffer;
27 import java.util.ArrayList;
28 import java.util.Collection;
29 import java.util.List;
30 import java.util.Random;
31
32 import org.apache.commons.logging.Log;
33 import org.apache.commons.logging.LogFactory;
34 import org.apache.hadoop.hbase.CategoryBasedTimeout;
35 import org.apache.hadoop.hbase.Cell;
36 import org.apache.hadoop.hbase.HBaseTestingUtility;
37 import org.apache.hadoop.hbase.HConstants;
38 import org.apache.hadoop.hbase.KeyValue;
39 import org.apache.hadoop.hbase.KeyValue.Type;
40 import org.apache.hadoop.hbase.KeyValueUtil;
41 import org.apache.hadoop.hbase.Tag;
42 import org.apache.hadoop.hbase.io.ByteArrayOutputStream;
43 import org.apache.hadoop.hbase.io.compress.Compression;
44 import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
45 import org.apache.hadoop.hbase.io.hfile.HFileContext;
46 import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
47 import org.apache.hadoop.hbase.testclassification.LargeTests;
48 import org.apache.hadoop.hbase.util.Bytes;
49 import org.apache.hadoop.hbase.util.RedundantKVGenerator;
50 import org.junit.Assert;
51 import org.junit.Rule;
52 import org.junit.Test;
53 import org.junit.experimental.categories.Category;
54 import org.junit.rules.TestRule;
55 import org.junit.runner.RunWith;
56 import org.junit.runners.Parameterized;
57 import org.junit.runners.Parameterized.Parameters;
58
59
60
61
62
63 @Category(LargeTests.class)
64 @RunWith(Parameterized.class)
65 public class TestDataBlockEncoders {
66
67 private static final Log LOG = LogFactory.getLog(TestDataBlockEncoders.class);
68
69 @Rule public final TestRule timeout = CategoryBasedTimeout.builder().
70 withTimeout(this.getClass()).withLookingForStuckThread(true).build();
71
72 private static int NUMBER_OF_KV = 10000;
73 private static int NUM_RANDOM_SEEKS = 1000;
74
75 private static int ENCODED_DATA_OFFSET = HConstants.HFILEBLOCK_HEADER_SIZE
76 + DataBlockEncoding.ID_SIZE;
77 static final byte[] HFILEBLOCK_DUMMY_HEADER = new byte[HConstants.HFILEBLOCK_HEADER_SIZE];
78
79 private RedundantKVGenerator generator = new RedundantKVGenerator();
80 private Random randomizer = new Random(42l);
81
82 private final boolean includesMemstoreTS;
83 private final boolean includesTags;
84
85 @Parameters
86 public static Collection<Object[]> parameters() {
87 return HBaseTestingUtility.MEMSTORETS_TAGS_PARAMETRIZED;
88 }
89 public TestDataBlockEncoders(boolean includesMemstoreTS, boolean includesTag) {
90 this.includesMemstoreTS = includesMemstoreTS;
91 this.includesTags = includesTag;
92 }
93
94 private HFileBlockEncodingContext getEncodingContext(Compression.Algorithm algo,
95 DataBlockEncoding encoding) {
96 DataBlockEncoder encoder = encoding.getEncoder();
97 HFileContext meta = new HFileContextBuilder()
98 .withHBaseCheckSum(false)
99 .withIncludesMvcc(includesMemstoreTS)
100 .withIncludesTags(includesTags)
101 .withCompression(algo).build();
102 if (encoder != null) {
103 return encoder.newDataBlockEncodingContext(encoding, HFILEBLOCK_DUMMY_HEADER, meta);
104 } else {
105 return new HFileBlockDefaultEncodingContext(encoding, HFILEBLOCK_DUMMY_HEADER, meta);
106 }
107 }
108
109
110
111
112
113
114
115 @Test
116 public void testEmptyKeyValues() throws IOException {
117 List<KeyValue> kvList = new ArrayList<KeyValue>();
118 byte[] row = new byte[0];
119 byte[] family = new byte[0];
120 byte[] qualifier = new byte[0];
121 byte[] value = new byte[0];
122 if (!includesTags) {
123 kvList.add(new KeyValue(row, family, qualifier, 0l, value));
124 kvList.add(new KeyValue(row, family, qualifier, 0l, value));
125 } else {
126 byte[] metaValue1 = Bytes.toBytes("metaValue1");
127 byte[] metaValue2 = Bytes.toBytes("metaValue2");
128 kvList.add(new KeyValue(row, family, qualifier, 0l, value, new Tag[] { new Tag((byte) 1,
129 metaValue1) }));
130 kvList.add(new KeyValue(row, family, qualifier, 0l, value, new Tag[] { new Tag((byte) 1,
131 metaValue2) }));
132 }
133 testEncodersOnDataset(kvList, includesMemstoreTS, includesTags);
134 }
135
136
137
138
139
140
141 @Test
142 public void testExecutionOnSample() throws IOException {
143 List<KeyValue> kvList = generator.generateTestKeyValues(NUMBER_OF_KV, includesTags);
144 testEncodersOnDataset(kvList, includesMemstoreTS, includesTags);
145 }
146
147
148
149
150 @Test
151 public void testSeekingOnSample() throws IOException {
152 List<KeyValue> sampleKv = generator.generateTestKeyValues(NUMBER_OF_KV, includesTags);
153
154
155 List<DataBlockEncoder.EncodedSeeker> encodedSeekers =
156 new ArrayList<DataBlockEncoder.EncodedSeeker>();
157 for (DataBlockEncoding encoding : DataBlockEncoding.values()) {
158 LOG.info("Encoding: " + encoding);
159
160
161 if (encoding == DataBlockEncoding.PREFIX_TREE) continue;
162 DataBlockEncoder encoder = encoding.getEncoder();
163 if (encoder == null) {
164 continue;
165 }
166 LOG.info("Encoder: " + encoder);
167 ByteBuffer encodedBuffer = encodeKeyValues(encoding, sampleKv,
168 getEncodingContext(Compression.Algorithm.NONE, encoding));
169 HFileContext meta = new HFileContextBuilder()
170 .withHBaseCheckSum(false)
171 .withIncludesMvcc(includesMemstoreTS)
172 .withIncludesTags(includesTags)
173 .withCompression(Compression.Algorithm.NONE)
174 .build();
175 DataBlockEncoder.EncodedSeeker seeker = encoder.createSeeker(KeyValue.COMPARATOR,
176 encoder.newDataBlockDecodingContext(meta));
177 seeker.setCurrentBuffer(encodedBuffer);
178 encodedSeekers.add(seeker);
179 }
180 LOG.info("Testing it!");
181
182
183 for (boolean seekBefore : new boolean[] { false, true }) {
184 for (int i = 0; i < NUM_RANDOM_SEEKS; ++i) {
185 int keyValueId;
186 if (!seekBefore) {
187 keyValueId = randomizer.nextInt(sampleKv.size());
188 } else {
189 keyValueId = randomizer.nextInt(sampleKv.size() - 1) + 1;
190 }
191
192 KeyValue keyValue = sampleKv.get(keyValueId);
193 checkSeekingConsistency(encodedSeekers, seekBefore, keyValue);
194 }
195 }
196
197
198 LOG.info("Checking edge cases");
199 checkSeekingConsistency(encodedSeekers, false, sampleKv.get(0));
200 for (boolean seekBefore : new boolean[] { false, true }) {
201 checkSeekingConsistency(encodedSeekers, seekBefore, sampleKv.get(sampleKv.size() - 1));
202 KeyValue midKv = sampleKv.get(sampleKv.size() / 2);
203 KeyValue lastMidKv =KeyValueUtil.createLastOnRowCol(midKv);
204 checkSeekingConsistency(encodedSeekers, seekBefore, lastMidKv);
205 }
206 LOG.info("Done");
207 }
208
209 static ByteBuffer encodeKeyValues(DataBlockEncoding encoding, List<KeyValue> kvs,
210 HFileBlockEncodingContext encodingContext) throws IOException {
211 DataBlockEncoder encoder = encoding.getEncoder();
212 ByteArrayOutputStream baos = new ByteArrayOutputStream();
213 baos.write(HFILEBLOCK_DUMMY_HEADER);
214 DataOutputStream dos = new DataOutputStream(baos);
215 encoder.startBlockEncoding(encodingContext, dos);
216 for (KeyValue kv : kvs) {
217 encoder.encode(kv, encodingContext, dos);
218 }
219 encoder.endBlockEncoding(encodingContext, dos, baos.getBuffer());
220 byte[] encodedData = new byte[baos.size() - ENCODED_DATA_OFFSET];
221 System.arraycopy(baos.toByteArray(), ENCODED_DATA_OFFSET, encodedData, 0, encodedData.length);
222 return ByteBuffer.wrap(encodedData);
223 }
224
225 @Test
226 public void testNextOnSample() throws IOException {
227 List<KeyValue> sampleKv = generator.generateTestKeyValues(NUMBER_OF_KV, includesTags);
228
229 for (DataBlockEncoding encoding : DataBlockEncoding.values()) {
230 if (encoding.getEncoder() == null) {
231 continue;
232 }
233 DataBlockEncoder encoder = encoding.getEncoder();
234 ByteBuffer encodedBuffer = encodeKeyValues(encoding, sampleKv,
235 getEncodingContext(Compression.Algorithm.NONE, encoding));
236 HFileContext meta = new HFileContextBuilder()
237 .withHBaseCheckSum(false)
238 .withIncludesMvcc(includesMemstoreTS)
239 .withIncludesTags(includesTags)
240 .withCompression(Compression.Algorithm.NONE)
241 .build();
242 DataBlockEncoder.EncodedSeeker seeker = encoder.createSeeker(KeyValue.COMPARATOR,
243 encoder.newDataBlockDecodingContext(meta));
244 seeker.setCurrentBuffer(encodedBuffer);
245 int i = 0;
246 do {
247 KeyValue expectedKeyValue = sampleKv.get(i);
248 KeyValue keyValue = KeyValueUtil.copyToNewKeyValue(seeker.getKeyValue());
249 if (0 != Bytes.compareTo(keyValue.getBuffer(), keyValue.getOffset(), keyValue.getLength(),
250 expectedKeyValue.getBuffer(), expectedKeyValue.getOffset(),
251 expectedKeyValue.getLength())) {
252
253 int commonPrefix = 0;
254 byte[] left = keyValue.getBuffer();
255 byte[] right = expectedKeyValue.getBuffer();
256 int leftOff = keyValue.getOffset();
257 int rightOff = expectedKeyValue.getOffset();
258 int length = Math.min(keyValue.getLength(), expectedKeyValue.getLength());
259 while (commonPrefix < length
260 && left[commonPrefix + leftOff] == right[commonPrefix + rightOff]) {
261 commonPrefix++;
262 }
263
264 fail(String.format("next() produces wrong results "
265 + "encoder: %s i: %d commonPrefix: %d" + "\n expected %s\n actual %s", encoder
266 .toString(), i, commonPrefix, Bytes.toStringBinary(expectedKeyValue.getBuffer(),
267 expectedKeyValue.getOffset(), expectedKeyValue.getLength()), Bytes
268 .toStringBinary(keyValue.getBuffer())));
269 }
270 i++;
271 } while (seeker.next());
272 }
273 }
274
275
276
277
278
279 @Test
280 public void testFirstKeyInBlockOnSample() throws IOException {
281 List<KeyValue> sampleKv = generator.generateTestKeyValues(NUMBER_OF_KV, includesTags);
282
283 for (DataBlockEncoding encoding : DataBlockEncoding.values()) {
284 if (encoding.getEncoder() == null) {
285 continue;
286 }
287 DataBlockEncoder encoder = encoding.getEncoder();
288 ByteBuffer encodedBuffer = encodeKeyValues(encoding, sampleKv,
289 getEncodingContext(Compression.Algorithm.NONE, encoding));
290 ByteBuffer keyBuffer = encoder.getFirstKeyInBlock(encodedBuffer);
291 KeyValue firstKv = sampleKv.get(0);
292 if (0 != Bytes.compareTo(keyBuffer.array(), keyBuffer.arrayOffset(), keyBuffer.limit(),
293 firstKv.getBuffer(), firstKv.getKeyOffset(), firstKv.getKeyLength())) {
294
295 int commonPrefix = 0;
296 int length = Math.min(keyBuffer.limit(), firstKv.getKeyLength());
297 while (commonPrefix < length
298 && keyBuffer.array()[keyBuffer.arrayOffset() + commonPrefix] == firstKv.getBuffer()[firstKv
299 .getKeyOffset() + commonPrefix]) {
300 commonPrefix++;
301 }
302 fail(String.format("Bug in '%s' commonPrefix %d", encoder.toString(), commonPrefix));
303 }
304 }
305 }
306
307 private void checkSeekingConsistency(List<DataBlockEncoder.EncodedSeeker> encodedSeekers,
308 boolean seekBefore, KeyValue keyValue) {
309 ByteBuffer expectedKeyValue = null;
310 ByteBuffer expectedKey = null;
311 ByteBuffer expectedValue = null;
312 for (DataBlockEncoder.EncodedSeeker seeker : encodedSeekers) {
313 seeker.seekToKeyInBlock(keyValue, seekBefore);
314 seeker.rewind();
315
316 ByteBuffer actualKeyValue = KeyValueUtil.copyKeyToNewByteBuffer(seeker.getKeyValue());
317 ByteBuffer actualKey = seeker.getKeyDeepCopy();
318 ByteBuffer actualValue = seeker.getValueShallowCopy();
319
320 if (expectedKeyValue != null) {
321 assertEquals(expectedKeyValue, actualKeyValue);
322 } else {
323 expectedKeyValue = actualKeyValue;
324 }
325
326 if (expectedKey != null) {
327 assertEquals(expectedKey, actualKey);
328 } else {
329 expectedKey = actualKey;
330 }
331
332 if (expectedValue != null) {
333 assertEquals(expectedValue, actualValue);
334 } else {
335 expectedValue = actualValue;
336 }
337 }
338 }
339
340 private void testEncodersOnDataset(List<KeyValue> kvList, boolean includesMemstoreTS,
341 boolean includesTags) throws IOException {
342 ByteBuffer unencodedDataBuf = RedundantKVGenerator.convertKvToByteBuffer(kvList,
343 includesMemstoreTS);
344 HFileContext fileContext = new HFileContextBuilder().withIncludesMvcc(includesMemstoreTS)
345 .withIncludesTags(includesTags).build();
346 for (DataBlockEncoding encoding : DataBlockEncoding.values()) {
347 DataBlockEncoder encoder = encoding.getEncoder();
348 if (encoder == null) {
349 continue;
350 }
351 HFileBlockEncodingContext encodingContext = new HFileBlockDefaultEncodingContext(encoding,
352 HFILEBLOCK_DUMMY_HEADER, fileContext);
353
354 ByteArrayOutputStream baos = new ByteArrayOutputStream();
355 baos.write(HFILEBLOCK_DUMMY_HEADER);
356 DataOutputStream dos = new DataOutputStream(baos);
357 encoder.startBlockEncoding(encodingContext, dos);
358 for (KeyValue kv : kvList) {
359 encoder.encode(kv, encodingContext, dos);
360 }
361 encoder.endBlockEncoding(encodingContext, dos, baos.getBuffer());
362 byte[] encodedData = baos.toByteArray();
363
364 testAlgorithm(encodedData, unencodedDataBuf, encoder);
365 }
366 }
367
368 @Test
369 public void testZeroByte() throws IOException {
370 List<KeyValue> kvList = new ArrayList<KeyValue>();
371 byte[] row = Bytes.toBytes("abcd");
372 byte[] family = new byte[] { 'f' };
373 byte[] qualifier0 = new byte[] { 'b' };
374 byte[] qualifier1 = new byte[] { 'c' };
375 byte[] value0 = new byte[] { 'd' };
376 byte[] value1 = new byte[] { 0x00 };
377 if (includesTags) {
378 kvList.add(new KeyValue(row, family, qualifier0, 0, value0, new Tag[] { new Tag((byte) 1,
379 "value1") }));
380 kvList.add(new KeyValue(row, family, qualifier1, 0, value1, new Tag[] { new Tag((byte) 1,
381 "value1") }));
382 } else {
383 kvList.add(new KeyValue(row, family, qualifier0, 0, Type.Put, value0));
384 kvList.add(new KeyValue(row, family, qualifier1, 0, Type.Put, value1));
385 }
386 testEncodersOnDataset(kvList, includesMemstoreTS, includesTags);
387 }
388
389 private void testAlgorithm(byte[] encodedData, ByteBuffer unencodedDataBuf,
390 DataBlockEncoder encoder) throws IOException {
391
392 ByteArrayInputStream bais = new ByteArrayInputStream(encodedData, ENCODED_DATA_OFFSET,
393 encodedData.length - ENCODED_DATA_OFFSET);
394 DataInputStream dis = new DataInputStream(bais);
395 ByteBuffer actualDataset;
396 HFileContext meta = new HFileContextBuilder().withHBaseCheckSum(false)
397 .withIncludesMvcc(includesMemstoreTS).withIncludesTags(includesTags)
398 .withCompression(Compression.Algorithm.NONE).build();
399 actualDataset = encoder.decodeKeyValues(dis, encoder.newDataBlockDecodingContext(meta));
400 actualDataset.rewind();
401
402
403
404
405 assertEquals("Encoding -> decoding gives different results for " + encoder,
406 Bytes.toStringBinary(unencodedDataBuf), Bytes.toStringBinary(actualDataset));
407 }
408
409 @Test
410 public void testRowIndexWithTagsButNoTagsInCell() throws IOException {
411 List<KeyValue> kvList = new ArrayList<>();
412 byte[] row = new byte[0];
413 byte[] family = new byte[0];
414 byte[] qualifier = new byte[0];
415 byte[] value = new byte[0];
416 KeyValue expectedKV = new KeyValue(row, family, qualifier, 1L, Type.Put, value);
417 kvList.add(expectedKV);
418 DataBlockEncoding encoding = DataBlockEncoding.ROW_INDEX_V1;
419 DataBlockEncoder encoder = encoding.getEncoder();
420 ByteBuffer encodedBuffer =
421 encodeKeyValues(encoding, kvList, getEncodingContext(Algorithm.NONE, encoding));
422 HFileContext meta =
423 new HFileContextBuilder().withHBaseCheckSum(false).withIncludesMvcc(includesMemstoreTS)
424 .withIncludesTags(includesTags).withCompression(Compression.Algorithm.NONE).build();
425 DataBlockEncoder.EncodedSeeker seeker =
426 encoder.createSeeker(KeyValue.COMPARATOR, encoder.newDataBlockDecodingContext(meta));
427 seeker.setCurrentBuffer(encodedBuffer);
428 Cell cell = seeker.getKeyValue();
429 Assert.assertEquals(expectedKV.getLength(), ((KeyValue) cell).getLength());
430 }
431
432 }