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 java.io.DataInputStream;
20 import java.io.DataOutputStream;
21 import java.io.IOException;
22 import java.nio.ByteBuffer;
23
24 import org.apache.hadoop.hbase.classification.InterfaceAudience;
25 import org.apache.hadoop.hbase.HConstants;
26 import org.apache.hadoop.hbase.KeyValue;
27 import org.apache.hadoop.hbase.KeyValue.KVComparator;
28 import org.apache.hadoop.hbase.KeyValue.SamePrefixComparator;
29 import org.apache.hadoop.hbase.NoTagsKeyValue;
30 import org.apache.hadoop.hbase.io.TagCompressionContext;
31 import org.apache.hadoop.hbase.io.hfile.BlockType;
32 import org.apache.hadoop.hbase.io.hfile.HFileContext;
33 import org.apache.hadoop.hbase.io.util.LRUDictionary;
34 import org.apache.hadoop.hbase.util.ByteBufferUtils;
35 import org.apache.hadoop.hbase.util.Bytes;
36 import org.apache.hadoop.io.WritableUtils;
37
38
39
40
41 @InterfaceAudience.Private
42 abstract class BufferedDataBlockEncoder implements DataBlockEncoder {
43
44 private static int INITIAL_KEY_BUFFER_SIZE = 512;
45
46 @Override
47 public ByteBuffer decodeKeyValues(DataInputStream source,
48 HFileBlockDecodingContext blkDecodingCtx) throws IOException {
49 if (blkDecodingCtx.getClass() != HFileBlockDefaultDecodingContext.class) {
50 throw new IOException(this.getClass().getName() + " only accepts "
51 + HFileBlockDefaultDecodingContext.class.getName() + " as the decoding context.");
52 }
53
54 HFileBlockDefaultDecodingContext decodingCtx =
55 (HFileBlockDefaultDecodingContext) blkDecodingCtx;
56 if (decodingCtx.getHFileContext().isIncludesTags()
57 && decodingCtx.getHFileContext().isCompressTags()) {
58 if (decodingCtx.getTagCompressionContext() != null) {
59
60
61 decodingCtx.getTagCompressionContext().clear();
62 } else {
63 try {
64 TagCompressionContext tagCompressionContext = new TagCompressionContext(
65 LRUDictionary.class, Byte.MAX_VALUE);
66 decodingCtx.setTagCompressionContext(tagCompressionContext);
67 } catch (Exception e) {
68 throw new IOException("Failed to initialize TagCompressionContext", e);
69 }
70 }
71 }
72 return internalDecodeKeyValues(source, 0, 0, decodingCtx);
73 }
74
75 protected static class SeekerState {
76 protected int valueOffset = -1;
77 protected int keyLength;
78 protected int valueLength;
79 protected int lastCommonPrefix;
80 protected int tagsLength = 0;
81 protected int tagsOffset = -1;
82 protected int tagsCompressedLength = 0;
83 protected boolean uncompressTags = true;
84
85
86 protected byte[] keyBuffer = new byte[INITIAL_KEY_BUFFER_SIZE];
87 protected byte[] tagsBuffer = new byte[INITIAL_KEY_BUFFER_SIZE];
88
89 protected long memstoreTS;
90 protected int nextKvOffset;
91
92 protected boolean isValid() {
93 return valueOffset != -1;
94 }
95
96 protected void invalidate() {
97 valueOffset = -1;
98 tagsCompressedLength = 0;
99 uncompressTags = true;
100 }
101
102 protected void ensureSpaceForKey() {
103 if (keyLength > keyBuffer.length) {
104
105 int newKeyBufferLength = Math.max(keyBuffer.length, 1) * 2;
106 while (keyLength > newKeyBufferLength) {
107 newKeyBufferLength *= 2;
108 }
109 byte[] newKeyBuffer = new byte[newKeyBufferLength];
110 System.arraycopy(keyBuffer, 0, newKeyBuffer, 0, keyBuffer.length);
111 keyBuffer = newKeyBuffer;
112 }
113 }
114
115 protected void ensureSpaceForTags() {
116 if (tagsLength > tagsBuffer.length) {
117
118 int newTagsBufferLength = Math.max(tagsBuffer.length, 1) * 2;
119 while (tagsLength > newTagsBufferLength) {
120 newTagsBufferLength *= 2;
121 }
122 byte[] newTagsBuffer = new byte[newTagsBufferLength];
123 System.arraycopy(tagsBuffer, 0, newTagsBuffer, 0, tagsBuffer.length);
124 tagsBuffer = newTagsBuffer;
125 }
126 }
127
128
129
130
131
132
133 protected void copyFromNext(SeekerState nextState) {
134 if (keyBuffer.length != nextState.keyBuffer.length) {
135 keyBuffer = nextState.keyBuffer.clone();
136 } else if (!isValid()) {
137
138
139 System.arraycopy(nextState.keyBuffer, 0, keyBuffer, 0,
140 nextState.keyLength);
141 } else {
142
143 System.arraycopy(nextState.keyBuffer, nextState.lastCommonPrefix,
144 keyBuffer, nextState.lastCommonPrefix, nextState.keyLength
145 - nextState.lastCommonPrefix);
146 }
147
148 valueOffset = nextState.valueOffset;
149 keyLength = nextState.keyLength;
150 valueLength = nextState.valueLength;
151 lastCommonPrefix = nextState.lastCommonPrefix;
152 nextKvOffset = nextState.nextKvOffset;
153 memstoreTS = nextState.memstoreTS;
154 tagsOffset = nextState.tagsOffset;
155 tagsLength = nextState.tagsLength;
156 }
157
158 }
159
160 protected abstract static class
161 BufferedEncodedSeeker<STATE extends SeekerState>
162 implements EncodedSeeker {
163 protected HFileBlockDecodingContext decodingCtx;
164 protected final KVComparator comparator;
165 protected final SamePrefixComparator<byte[]> samePrefixComparator;
166 protected ByteBuffer currentBuffer;
167 protected STATE current = createSeekerState();
168 protected STATE previous = createSeekerState();
169 protected TagCompressionContext tagCompressionContext = null;
170
171 public BufferedEncodedSeeker(KVComparator comparator,
172 HFileBlockDecodingContext decodingCtx) {
173 this.comparator = comparator;
174 this.samePrefixComparator = comparator;
175 this.decodingCtx = decodingCtx;
176 if (decodingCtx.getHFileContext().isCompressTags()) {
177 try {
178 tagCompressionContext = new TagCompressionContext(LRUDictionary.class, Byte.MAX_VALUE);
179 } catch (Exception e) {
180 throw new RuntimeException("Failed to initialize TagCompressionContext", e);
181 }
182 }
183 }
184
185 protected boolean includesMvcc() {
186 return this.decodingCtx.getHFileContext().isIncludesMvcc();
187 }
188
189 protected boolean includesTags() {
190 return this.decodingCtx.getHFileContext().isIncludesTags();
191 }
192
193 @Override
194 public int compareKey(KVComparator comparator, byte[] key, int offset, int length) {
195 return comparator.compareFlatKey(key, offset, length,
196 current.keyBuffer, 0, current.keyLength);
197 }
198
199 @Override
200 public void setCurrentBuffer(ByteBuffer buffer) {
201 if (this.tagCompressionContext != null) {
202 this.tagCompressionContext.clear();
203 }
204 currentBuffer = buffer;
205 decodeFirst();
206 previous.invalidate();
207 }
208
209 @Override
210 public ByteBuffer getKeyDeepCopy() {
211 ByteBuffer keyBuffer = ByteBuffer.allocate(current.keyLength);
212 keyBuffer.put(current.keyBuffer, 0, current.keyLength);
213 keyBuffer.rewind();
214 return keyBuffer;
215 }
216
217 @Override
218 public ByteBuffer getValueShallowCopy() {
219 return ByteBuffer.wrap(currentBuffer.array(),
220 currentBuffer.arrayOffset() + current.valueOffset,
221 current.valueLength);
222 }
223
224 @Override
225 public KeyValue getKeyValue() {
226 byte[] kvBuf = new byte[(int)KeyValue.getKeyValueDataStructureSize(current.keyLength,
227 current.valueLength, current.tagsLength)];
228 int offset = Bytes.putInt(kvBuf, 0, current.keyLength);
229 offset = Bytes.putInt(kvBuf, offset, current.valueLength);
230 System.arraycopy(current.keyBuffer, 0, kvBuf, offset, current.keyLength);
231 offset += current.keyLength;
232 System.arraycopy(currentBuffer.array(),
233 currentBuffer.arrayOffset() + current.valueOffset, kvBuf, offset, current.valueLength);
234 offset += current.valueLength;
235 if (current.tagsLength > 0) {
236
237 offset = Bytes.putByte(kvBuf, offset, (byte)(current.tagsLength >> 8 & 0xff));
238 offset = Bytes.putByte(kvBuf, offset, (byte)(current.tagsLength & 0xff));
239 if (current.tagsOffset != -1) {
240
241
242 System.arraycopy(currentBuffer.array(), currentBuffer.arrayOffset() + current.tagsOffset, kvBuf, offset, current.tagsLength);
243 } else {
244
245
246 System.arraycopy(current.tagsBuffer, 0, kvBuf, offset, current.tagsLength);
247 }
248 }
249
250 KeyValue kv;
251 if (current.tagsLength == 0) {
252 kv = new NoTagsKeyValue(kvBuf, 0, kvBuf.length);
253 } else {
254 kv = new KeyValue(kvBuf, 0, kvBuf.length);
255 }
256 kv.setMvccVersion(current.memstoreTS);
257 return kv;
258 }
259
260 @Override
261 public void rewind() {
262 currentBuffer.rewind();
263 if (tagCompressionContext != null) {
264 tagCompressionContext.clear();
265 }
266 decodeFirst();
267 previous.invalidate();
268 }
269
270 @Override
271 public boolean next() {
272 if (!currentBuffer.hasRemaining()) {
273 return false;
274 }
275 decodeNext();
276 previous.invalidate();
277 return true;
278 }
279
280 protected void decodeTags() {
281 current.tagsLength = ByteBufferUtils.readCompressedInt(currentBuffer);
282 if (tagCompressionContext != null) {
283 if (current.uncompressTags) {
284
285 current.ensureSpaceForTags();
286 try {
287 current.tagsCompressedLength = tagCompressionContext.uncompressTags(currentBuffer,
288 current.tagsBuffer, 0, current.tagsLength);
289 } catch (IOException e) {
290 throw new RuntimeException("Exception while uncompressing tags", e);
291 }
292 } else {
293 ByteBufferUtils.skip(currentBuffer, current.tagsCompressedLength);
294 current.uncompressTags = true;
295 }
296 current.tagsOffset = -1;
297 } else {
298
299
300 current.tagsOffset = currentBuffer.position();
301 ByteBufferUtils.skip(currentBuffer, current.tagsLength);
302 }
303 }
304
305 @Override
306 public int seekToKeyInBlock(byte[] key, int offset, int length,
307 boolean seekBefore) {
308 int commonPrefix = 0;
309 previous.invalidate();
310 do {
311 int comp;
312 if (samePrefixComparator != null) {
313 commonPrefix = Math.min(commonPrefix, current.lastCommonPrefix);
314
315
316 commonPrefix += ByteBufferUtils.findCommonPrefix(
317 key, offset + commonPrefix, length - commonPrefix,
318 current.keyBuffer, commonPrefix,
319 current.keyLength - commonPrefix);
320
321 comp = samePrefixComparator.compareIgnoringPrefix(commonPrefix, key,
322 offset, length, current.keyBuffer, 0, current.keyLength);
323 } else {
324 comp = comparator.compareFlatKey(key, offset, length,
325 current.keyBuffer, 0, current.keyLength);
326 }
327
328 if (comp == 0) {
329 if (seekBefore) {
330 if (!previous.isValid()) {
331
332
333 throw new IllegalStateException("Cannot seekBefore if " +
334 "positioned at the first key in the block: key=" +
335 Bytes.toStringBinary(key, offset, length));
336 }
337 moveToPrevious();
338 return 1;
339 }
340 return 0;
341 }
342
343 if (comp < 0) {
344 if (previous.isValid()) {
345 moveToPrevious();
346 } else {
347 return HConstants.INDEX_KEY_MAGIC;
348 }
349 return 1;
350 }
351
352
353 if (currentBuffer.hasRemaining()) {
354 previous.copyFromNext(current);
355 decodeNext();
356 } else {
357 break;
358 }
359 } while (true);
360
361
362 return 1;
363 }
364
365 private void moveToPrevious() {
366 if (!previous.isValid()) {
367 throw new IllegalStateException(
368 "Can move back only once and not in first key in the block.");
369 }
370
371 STATE tmp = previous;
372 previous = current;
373 current = tmp;
374
375
376 currentBuffer.position(current.nextKvOffset);
377
378
379
380
381
382
383 current.tagsBuffer = previous.tagsBuffer;
384 current.tagsCompressedLength = previous.tagsCompressedLength;
385 current.uncompressTags = false;
386 previous.invalidate();
387 }
388
389 @SuppressWarnings("unchecked")
390 protected STATE createSeekerState() {
391
392
393 return (STATE) new SeekerState();
394 }
395
396 abstract protected void decodeFirst();
397 abstract protected void decodeNext();
398 }
399
400 protected final void afterEncodingKeyValue(ByteBuffer in,
401 DataOutputStream out, HFileBlockDefaultEncodingContext encodingCtx) throws IOException {
402 if (encodingCtx.getHFileContext().isIncludesTags()) {
403
404 int tagsLength = ((in.get() & 0xff) << 8) ^ (in.get() & 0xff);
405 ByteBufferUtils.putCompressedInt(out, tagsLength);
406
407 if (tagsLength > 0) {
408 TagCompressionContext tagCompressionContext = encodingCtx.getTagCompressionContext();
409
410
411 if (tagCompressionContext != null) {
412 tagCompressionContext.compressTags(out, in, tagsLength);
413 } else {
414 ByteBufferUtils.moveBufferToStream(out, in, tagsLength);
415 }
416 }
417 }
418 if (encodingCtx.getHFileContext().isIncludesMvcc()) {
419
420 long memstoreTS = -1;
421 try {
422 memstoreTS = ByteBufferUtils.readVLong(in);
423 WritableUtils.writeVLong(out, memstoreTS);
424 } catch (IOException ex) {
425 throw new RuntimeException("Unable to copy memstore timestamp " +
426 memstoreTS + " after encoding a key/value");
427 }
428 }
429 }
430
431 protected final void afterDecodingKeyValue(DataInputStream source,
432 ByteBuffer dest, HFileBlockDefaultDecodingContext decodingCtx) throws IOException {
433 if (decodingCtx.getHFileContext().isIncludesTags()) {
434 int tagsLength = ByteBufferUtils.readCompressedInt(source);
435
436 dest.put((byte)((tagsLength >> 8) & 0xff));
437 dest.put((byte)(tagsLength & 0xff));
438 if (tagsLength > 0) {
439 TagCompressionContext tagCompressionContext = decodingCtx.getTagCompressionContext();
440
441
442 if (tagCompressionContext != null) {
443 tagCompressionContext.uncompressTags(source, dest, tagsLength);
444 } else {
445 ByteBufferUtils.copyFromStreamToBuffer(dest, source, tagsLength);
446 }
447 }
448 }
449 if (decodingCtx.getHFileContext().isIncludesMvcc()) {
450 long memstoreTS = -1;
451 try {
452
453
454 memstoreTS = WritableUtils.readVLong(source);
455 ByteBufferUtils.writeVLong(dest, memstoreTS);
456 } catch (IOException ex) {
457 throw new RuntimeException("Unable to copy memstore timestamp " +
458 memstoreTS + " after decoding a key/value");
459 }
460 }
461 }
462
463 @Override
464 public HFileBlockEncodingContext newDataBlockEncodingContext(DataBlockEncoding encoding,
465 byte[] header, HFileContext meta) {
466 return new HFileBlockDefaultEncodingContext(encoding, header, meta);
467 }
468
469 @Override
470 public HFileBlockDecodingContext newDataBlockDecodingContext(HFileContext meta) {
471 return new HFileBlockDefaultDecodingContext(meta);
472 }
473
474
475
476
477
478
479
480
481 public abstract void internalEncodeKeyValues(DataOutputStream out,
482 ByteBuffer in, HFileBlockDefaultEncodingContext encodingCtx) throws IOException;
483
484 protected abstract ByteBuffer internalDecodeKeyValues(DataInputStream source,
485 int allocateHeaderLength, int skipLastBytes, HFileBlockDefaultDecodingContext decodingCtx)
486 throws IOException;
487
488 @Override
489 public void encodeKeyValues(ByteBuffer in,
490 HFileBlockEncodingContext blkEncodingCtx) throws IOException {
491 if (blkEncodingCtx.getClass() != HFileBlockDefaultEncodingContext.class) {
492 throw new IOException (this.getClass().getName() + " only accepts "
493 + HFileBlockDefaultEncodingContext.class.getName() + " as the " +
494 "encoding context.");
495 }
496
497 HFileBlockDefaultEncodingContext encodingCtx =
498 (HFileBlockDefaultEncodingContext) blkEncodingCtx;
499 encodingCtx.prepareEncoding();
500 DataOutputStream dataOut = encodingCtx.getOutputStreamForEncoder();
501 if (encodingCtx.getHFileContext().isIncludesTags()
502 && encodingCtx.getHFileContext().isCompressTags()) {
503 if (encodingCtx.getTagCompressionContext() != null) {
504
505
506 encodingCtx.getTagCompressionContext().clear();
507 } else {
508 try {
509 TagCompressionContext tagCompressionContext = new TagCompressionContext(
510 LRUDictionary.class, Byte.MAX_VALUE);
511 encodingCtx.setTagCompressionContext(tagCompressionContext);
512 } catch (Exception e) {
513 throw new IOException("Failed to initialize TagCompressionContext", e);
514 }
515 }
516 }
517 internalEncodeKeyValues(dataOut, in, encodingCtx);
518 if (encodingCtx.getDataBlockEncoding() != DataBlockEncoding.NONE) {
519 encodingCtx.postEncoding(BlockType.ENCODED_DATA);
520 } else {
521 encodingCtx.postEncoding(BlockType.DATA);
522 }
523 }
524
525
526
527
528
529
530
531
532 protected static void ensureSpace(ByteBuffer out, int length)
533 throws EncoderBufferTooSmallException {
534 if (out.position() + length > out.limit()) {
535 throw new EncoderBufferTooSmallException(
536 "Buffer position=" + out.position() +
537 ", buffer limit=" + out.limit() +
538 ", length to be written=" + length);
539 }
540 }
541
542 }