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 ByteBuffer getKeyValueBuffer() {
226 ByteBuffer kvBuffer = createKVBuffer();
227 kvBuffer.putInt(current.keyLength);
228 kvBuffer.putInt(current.valueLength);
229 kvBuffer.put(current.keyBuffer, 0, current.keyLength);
230 kvBuffer.put(currentBuffer.array(),
231 currentBuffer.arrayOffset() + current.valueOffset,
232 current.valueLength);
233 if (current.tagsLength > 0) {
234
235 kvBuffer.put((byte)(current.tagsLength >> 8 & 0xff));
236 kvBuffer.put((byte)(current.tagsLength & 0xff));
237 if (current.tagsOffset != -1) {
238
239
240 kvBuffer.put(currentBuffer.array(), currentBuffer.arrayOffset() + current.tagsOffset,
241 current.tagsLength);
242 } else {
243
244
245 kvBuffer.put(current.tagsBuffer, 0, current.tagsLength);
246 }
247 }
248 kvBuffer.rewind();
249 return kvBuffer;
250 }
251
252 protected ByteBuffer createKVBuffer() {
253 int kvBufSize = (int) KeyValue.getKeyValueDataStructureSize(current.keyLength,
254 current.valueLength, current.tagsLength);
255 ByteBuffer kvBuffer = ByteBuffer.allocate(kvBufSize);
256 return kvBuffer;
257 }
258
259 @Override
260 public KeyValue getKeyValue() {
261 ByteBuffer kvBuf = getKeyValueBuffer();
262 KeyValue kv;
263 if (current.tagsLength == 0) {
264 kv = new NoTagsKeyValue(kvBuf.array(), kvBuf.arrayOffset(), kvBuf.array().length
265 - kvBuf.arrayOffset());
266 } else {
267 kv = new KeyValue(kvBuf.array(), kvBuf.arrayOffset(), kvBuf.array().length
268 - kvBuf.arrayOffset());
269 }
270 kv.setMvccVersion(current.memstoreTS);
271 return kv;
272 }
273
274 @Override
275 public void rewind() {
276 currentBuffer.rewind();
277 if (tagCompressionContext != null) {
278 tagCompressionContext.clear();
279 }
280 decodeFirst();
281 previous.invalidate();
282 }
283
284 @Override
285 public boolean next() {
286 if (!currentBuffer.hasRemaining()) {
287 return false;
288 }
289 decodeNext();
290 previous.invalidate();
291 return true;
292 }
293
294 protected void decodeTags() {
295 current.tagsLength = ByteBufferUtils.readCompressedInt(currentBuffer);
296 if (tagCompressionContext != null) {
297 if (current.uncompressTags) {
298
299 current.ensureSpaceForTags();
300 try {
301 current.tagsCompressedLength = tagCompressionContext.uncompressTags(currentBuffer,
302 current.tagsBuffer, 0, current.tagsLength);
303 } catch (IOException e) {
304 throw new RuntimeException("Exception while uncompressing tags", e);
305 }
306 } else {
307 ByteBufferUtils.skip(currentBuffer, current.tagsCompressedLength);
308 current.uncompressTags = true;
309 }
310 current.tagsOffset = -1;
311 } else {
312
313
314 current.tagsOffset = currentBuffer.position();
315 ByteBufferUtils.skip(currentBuffer, current.tagsLength);
316 }
317 }
318
319 @Override
320 public int seekToKeyInBlock(byte[] key, int offset, int length,
321 boolean seekBefore) {
322 int commonPrefix = 0;
323 previous.invalidate();
324 do {
325 int comp;
326 if (samePrefixComparator != null) {
327 commonPrefix = Math.min(commonPrefix, current.lastCommonPrefix);
328
329
330 commonPrefix += ByteBufferUtils.findCommonPrefix(
331 key, offset + commonPrefix, length - commonPrefix,
332 current.keyBuffer, commonPrefix,
333 current.keyLength - commonPrefix);
334
335 comp = samePrefixComparator.compareIgnoringPrefix(commonPrefix, key,
336 offset, length, current.keyBuffer, 0, current.keyLength);
337 } else {
338 comp = comparator.compareFlatKey(key, offset, length,
339 current.keyBuffer, 0, current.keyLength);
340 }
341
342 if (comp == 0) {
343 if (seekBefore) {
344 if (!previous.isValid()) {
345
346
347 throw new IllegalStateException("Cannot seekBefore if " +
348 "positioned at the first key in the block: key=" +
349 Bytes.toStringBinary(key, offset, length));
350 }
351 moveToPrevious();
352 return 1;
353 }
354 return 0;
355 }
356
357 if (comp < 0) {
358 if (previous.isValid()) {
359 moveToPrevious();
360 } else {
361 return HConstants.INDEX_KEY_MAGIC;
362 }
363 return 1;
364 }
365
366
367 if (currentBuffer.hasRemaining()) {
368 previous.copyFromNext(current);
369 decodeNext();
370 } else {
371 break;
372 }
373 } while (true);
374
375
376 return 1;
377 }
378
379 private void moveToPrevious() {
380 if (!previous.isValid()) {
381 throw new IllegalStateException(
382 "Can move back only once and not in first key in the block.");
383 }
384
385 STATE tmp = previous;
386 previous = current;
387 current = tmp;
388
389
390 currentBuffer.position(current.nextKvOffset);
391
392
393
394
395
396
397 current.tagsBuffer = previous.tagsBuffer;
398 current.tagsCompressedLength = previous.tagsCompressedLength;
399 current.uncompressTags = false;
400 previous.invalidate();
401 }
402
403 @SuppressWarnings("unchecked")
404 protected STATE createSeekerState() {
405
406
407 return (STATE) new SeekerState();
408 }
409
410 abstract protected void decodeFirst();
411 abstract protected void decodeNext();
412 }
413
414 protected final void afterEncodingKeyValue(ByteBuffer in,
415 DataOutputStream out, HFileBlockDefaultEncodingContext encodingCtx) throws IOException {
416 if (encodingCtx.getHFileContext().isIncludesTags()) {
417
418 int tagsLength = ((in.get() & 0xff) << 8) ^ (in.get() & 0xff);
419 ByteBufferUtils.putCompressedInt(out, tagsLength);
420
421 if (tagsLength > 0) {
422 TagCompressionContext tagCompressionContext = encodingCtx.getTagCompressionContext();
423
424
425 if (tagCompressionContext != null) {
426 tagCompressionContext.compressTags(out, in, tagsLength);
427 } else {
428 ByteBufferUtils.moveBufferToStream(out, in, tagsLength);
429 }
430 }
431 }
432 if (encodingCtx.getHFileContext().isIncludesMvcc()) {
433
434 long memstoreTS = -1;
435 try {
436 memstoreTS = ByteBufferUtils.readVLong(in);
437 WritableUtils.writeVLong(out, memstoreTS);
438 } catch (IOException ex) {
439 throw new RuntimeException("Unable to copy memstore timestamp " +
440 memstoreTS + " after encoding a key/value");
441 }
442 }
443 }
444
445 protected final void afterDecodingKeyValue(DataInputStream source,
446 ByteBuffer dest, HFileBlockDefaultDecodingContext decodingCtx) throws IOException {
447 if (decodingCtx.getHFileContext().isIncludesTags()) {
448 int tagsLength = ByteBufferUtils.readCompressedInt(source);
449
450 dest.put((byte)((tagsLength >> 8) & 0xff));
451 dest.put((byte)(tagsLength & 0xff));
452 if (tagsLength > 0) {
453 TagCompressionContext tagCompressionContext = decodingCtx.getTagCompressionContext();
454
455
456 if (tagCompressionContext != null) {
457 tagCompressionContext.uncompressTags(source, dest, tagsLength);
458 } else {
459 ByteBufferUtils.copyFromStreamToBuffer(dest, source, tagsLength);
460 }
461 }
462 }
463 if (decodingCtx.getHFileContext().isIncludesMvcc()) {
464 long memstoreTS = -1;
465 try {
466
467
468 memstoreTS = WritableUtils.readVLong(source);
469 ByteBufferUtils.writeVLong(dest, memstoreTS);
470 } catch (IOException ex) {
471 throw new RuntimeException("Unable to copy memstore timestamp " +
472 memstoreTS + " after decoding a key/value");
473 }
474 }
475 }
476
477 @Override
478 public HFileBlockEncodingContext newDataBlockEncodingContext(DataBlockEncoding encoding,
479 byte[] header, HFileContext meta) {
480 return new HFileBlockDefaultEncodingContext(encoding, header, meta);
481 }
482
483 @Override
484 public HFileBlockDecodingContext newDataBlockDecodingContext(HFileContext meta) {
485 return new HFileBlockDefaultDecodingContext(meta);
486 }
487
488
489
490
491
492
493
494
495 public abstract void internalEncodeKeyValues(DataOutputStream out,
496 ByteBuffer in, HFileBlockDefaultEncodingContext encodingCtx) throws IOException;
497
498 protected abstract ByteBuffer internalDecodeKeyValues(DataInputStream source,
499 int allocateHeaderLength, int skipLastBytes, HFileBlockDefaultDecodingContext decodingCtx)
500 throws IOException;
501
502 @Override
503 public void encodeKeyValues(ByteBuffer in,
504 HFileBlockEncodingContext blkEncodingCtx) throws IOException {
505 if (blkEncodingCtx.getClass() != HFileBlockDefaultEncodingContext.class) {
506 throw new IOException (this.getClass().getName() + " only accepts "
507 + HFileBlockDefaultEncodingContext.class.getName() + " as the " +
508 "encoding context.");
509 }
510
511 HFileBlockDefaultEncodingContext encodingCtx =
512 (HFileBlockDefaultEncodingContext) blkEncodingCtx;
513 encodingCtx.prepareEncoding();
514 DataOutputStream dataOut = encodingCtx.getOutputStreamForEncoder();
515 if (encodingCtx.getHFileContext().isIncludesTags()
516 && encodingCtx.getHFileContext().isCompressTags()) {
517 if (encodingCtx.getTagCompressionContext() != null) {
518
519
520 encodingCtx.getTagCompressionContext().clear();
521 } else {
522 try {
523 TagCompressionContext tagCompressionContext = new TagCompressionContext(
524 LRUDictionary.class, Byte.MAX_VALUE);
525 encodingCtx.setTagCompressionContext(tagCompressionContext);
526 } catch (Exception e) {
527 throw new IOException("Failed to initialize TagCompressionContext", e);
528 }
529 }
530 }
531 internalEncodeKeyValues(dataOut, in, encodingCtx);
532 if (encodingCtx.getDataBlockEncoding() != DataBlockEncoding.NONE) {
533 encodingCtx.postEncoding(BlockType.ENCODED_DATA);
534 } else {
535 encodingCtx.postEncoding(BlockType.DATA);
536 }
537 }
538
539
540
541
542
543
544
545
546 protected static void ensureSpace(ByteBuffer out, int length)
547 throws EncoderBufferTooSmallException {
548 if (out.position() + length > out.limit()) {
549 throw new EncoderBufferTooSmallException(
550 "Buffer position=" + out.position() +
551 ", buffer limit=" + out.limit() +
552 ", length to be written=" + length);
553 }
554 }
555
556 }