View Javadoc

1   /*
2    * Licensed to the Apache Software Foundation (ASF) under one or more
3    * contributor license agreements. See the NOTICE file distributed with this
4    * work for additional information regarding copyright ownership. The ASF
5    * licenses this file to you under the Apache License, Version 2.0 (the
6    * "License"); you may not use this file except in compliance with the License.
7    * You may obtain a copy of the License at
8    *
9    * http://www.apache.org/licenses/LICENSE-2.0
10   *
11   * Unless required by applicable law or agreed to in writing, software
12   * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
13   * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
14   * License for the specific language governing permissions and limitations
15   * under the License.
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.io.OutputStream;
23  import java.nio.ByteBuffer;
24  
25  import org.apache.hadoop.classification.InterfaceAudience;
26  import org.apache.hadoop.hbase.KeyValue;
27  import org.apache.hadoop.hbase.KeyValue.KVComparator;
28  import org.apache.hadoop.hbase.util.ByteBufferUtils;
29  import org.apache.hadoop.hbase.util.Bytes;
30  import org.apache.hadoop.io.RawComparator;
31  
32  /**
33   * Encoder similar to {@link DiffKeyDeltaEncoder} but supposedly faster.
34   *
35   * Compress using:
36   * - store size of common prefix
37   * - save column family once in the first KeyValue
38   * - use integer compression for key, value and prefix (7-bit encoding)
39   * - use bits to avoid duplication key length, value length
40   *   and type if it same as previous
41   * - store in 3 bits length of prefix timestamp
42   *    with previous KeyValue's timestamp
43   * - one bit which allow to omit value if it is the same
44   *
45   * Format:
46   * - 1 byte:    flag
47   * - 1-5 bytes: key length (only if FLAG_SAME_KEY_LENGTH is not set in flag)
48   * - 1-5 bytes: value length (only if FLAG_SAME_VALUE_LENGTH is not set in flag)
49   * - 1-5 bytes: prefix length
50   * - ... bytes: rest of the row (if prefix length is small enough)
51   * - ... bytes: qualifier (or suffix depending on prefix length)
52   * - 1-8 bytes: timestamp suffix
53   * - 1 byte:    type (only if FLAG_SAME_TYPE is not set in the flag)
54   * - ... bytes: value (only if FLAG_SAME_VALUE is not set in the flag)
55   *
56   */
57  @InterfaceAudience.Private
58  public class FastDiffDeltaEncoder extends BufferedDataBlockEncoder {
59    final int MASK_TIMESTAMP_LENGTH = (1 << 0) | (1 << 1) | (1 << 2);
60    final int SHIFT_TIMESTAMP_LENGTH = 0;
61    final int FLAG_SAME_KEY_LENGTH = 1 << 3;
62    final int FLAG_SAME_VALUE_LENGTH = 1 << 4;
63    final int FLAG_SAME_TYPE = 1 << 5;
64    final int FLAG_SAME_VALUE = 1 << 6;
65  
66    private static class FastDiffCompressionState extends CompressionState {
67      byte[] timestamp = new byte[KeyValue.TIMESTAMP_SIZE];
68      int prevTimestampOffset;
69  
70      @Override
71      protected void readTimestamp(ByteBuffer in) {
72        in.get(timestamp);
73      }
74  
75      @Override
76      void copyFrom(CompressionState state) {
77        super.copyFrom(state);
78        FastDiffCompressionState state2 = (FastDiffCompressionState) state;
79        System.arraycopy(state2.timestamp, 0, timestamp, 0,
80            KeyValue.TIMESTAMP_SIZE);
81        prevTimestampOffset = state2.prevTimestampOffset;
82      }
83  
84      /**
85       * Copies the first key/value from the given stream, and initializes
86       * decompression state based on it. Assumes that we have already read key
87       * and value lengths. Does not set {@link #qualifierLength} (not used by
88       * decompression) or {@link #prevOffset} (set by the calle afterwards).
89       */
90      private void decompressFirstKV(ByteBuffer out, DataInputStream in)
91          throws IOException {
92        int kvPos = out.position();
93        out.putInt(keyLength);
94        out.putInt(valueLength);
95        prevTimestampOffset = out.position() + keyLength -
96            KeyValue.TIMESTAMP_TYPE_SIZE;
97        ByteBufferUtils.copyFromStreamToBuffer(out, in, keyLength + valueLength);
98        rowLength = out.getShort(kvPos + KeyValue.ROW_OFFSET);
99        familyLength = out.get(kvPos + KeyValue.ROW_OFFSET +
100           KeyValue.ROW_LENGTH_SIZE + rowLength);
101       type = out.get(prevTimestampOffset + KeyValue.TIMESTAMP_SIZE);
102     }
103 
104   }
105 
106   private void compressSingleKeyValue(
107         FastDiffCompressionState previousState,
108         FastDiffCompressionState currentState,
109         OutputStream out, ByteBuffer in) throws IOException {
110     currentState.prevOffset = in.position();
111     int keyLength = in.getInt();
112     int valueOffset =
113         currentState.prevOffset + keyLength + KeyValue.ROW_OFFSET;
114     int valueLength = in.getInt();
115     byte flag = 0;
116 
117     if (previousState.isFirst()) {
118       // copy the key, there is no common prefix with none
119       out.write(flag);
120       ByteBufferUtils.putCompressedInt(out, keyLength);
121       ByteBufferUtils.putCompressedInt(out, valueLength);
122       ByteBufferUtils.putCompressedInt(out, 0);
123 
124       currentState.readKey(in, keyLength, valueLength);
125 
126       ByteBufferUtils.moveBufferToStream(out, in, keyLength + valueLength);
127     } else {
128       // find a common prefix and skip it
129       int commonPrefix = ByteBufferUtils.findCommonPrefix(in, in.position(),
130           previousState.prevOffset + KeyValue.ROW_OFFSET,
131           Math.min(keyLength, previousState.keyLength) -
132           KeyValue.TIMESTAMP_TYPE_SIZE);
133 
134       currentState.readKey(in, keyLength, valueLength,
135           commonPrefix, previousState);
136 
137       if (keyLength == previousState.keyLength) {
138         flag |= FLAG_SAME_KEY_LENGTH;
139       }
140       if (valueLength == previousState.valueLength) {
141         flag |= FLAG_SAME_VALUE_LENGTH;
142       }
143       if (currentState.type == previousState.type) {
144         flag |= FLAG_SAME_TYPE;
145       }
146 
147       int commonTimestampPrefix = findCommonTimestampPrefix(
148           currentState, previousState);
149       flag |= commonTimestampPrefix << SHIFT_TIMESTAMP_LENGTH;
150 
151       // Check if current and previous values are the same. Compare value
152       // length first as an optimization.
153       if (valueLength == previousState.valueLength) {
154         int previousValueOffset = previousState.prevOffset
155             + previousState.keyLength + KeyValue.ROW_OFFSET;
156         if (ByteBufferUtils.arePartsEqual(in,
157                 previousValueOffset, previousState.valueLength,
158                 valueOffset, valueLength)) {
159           flag |= FLAG_SAME_VALUE;
160         }
161       }
162 
163       out.write(flag);
164       if ((flag & FLAG_SAME_KEY_LENGTH) == 0) {
165         ByteBufferUtils.putCompressedInt(out, keyLength);
166       }
167       if ((flag & FLAG_SAME_VALUE_LENGTH) == 0) {
168         ByteBufferUtils.putCompressedInt(out, valueLength);
169       }
170       ByteBufferUtils.putCompressedInt(out, commonPrefix);
171 
172       ByteBufferUtils.skip(in, commonPrefix);
173       if (commonPrefix < currentState.rowLength + KeyValue.ROW_LENGTH_SIZE) {
174         // Previous and current rows are different. Copy the differing part of
175         // the row, skip the column family, and copy the qualifier.
176         ByteBufferUtils.moveBufferToStream(out, in,
177             currentState.rowLength + KeyValue.ROW_LENGTH_SIZE - commonPrefix);
178         ByteBufferUtils.skip(in, currentState.familyLength +
179             KeyValue.FAMILY_LENGTH_SIZE);
180         ByteBufferUtils.moveBufferToStream(out, in,
181             currentState.qualifierLength);
182       } else {
183         // The common part includes the whole row. As the column family is the
184         // same across the whole file, it will automatically be included in the
185         // common prefix, so we need not special-case it here.
186         int restKeyLength = keyLength - commonPrefix -
187             KeyValue.TIMESTAMP_TYPE_SIZE;
188         ByteBufferUtils.moveBufferToStream(out, in, restKeyLength);
189       }
190       ByteBufferUtils.skip(in, commonTimestampPrefix);
191       ByteBufferUtils.moveBufferToStream(out, in,
192           KeyValue.TIMESTAMP_SIZE - commonTimestampPrefix);
193 
194       // Write the type if it is not the same as before.
195       if ((flag & FLAG_SAME_TYPE) == 0) {
196         out.write(currentState.type);
197       }
198 
199       // Write the value if it is not the same as before.
200       if ((flag & FLAG_SAME_VALUE) == 0) {
201         ByteBufferUtils.copyBufferToStream(out, in, valueOffset, valueLength);
202       }
203 
204       // Skip key type and value in the input buffer.
205       ByteBufferUtils.skip(in, KeyValue.TYPE_SIZE + currentState.valueLength);
206     }
207   }
208 
209   private int findCommonTimestampPrefix(FastDiffCompressionState left,
210       FastDiffCompressionState right) {
211     int prefixTimestamp = 0;
212     while (prefixTimestamp < (KeyValue.TIMESTAMP_SIZE - 1) &&
213         left.timestamp[prefixTimestamp]
214             == right.timestamp[prefixTimestamp]) {
215       prefixTimestamp++;
216     }
217     return prefixTimestamp; // has to be at most 7 bytes
218   }
219 
220   private void uncompressSingleKeyValue(DataInputStream source,
221       ByteBuffer out, FastDiffCompressionState state)
222           throws IOException, EncoderBufferTooSmallException {
223     byte flag = source.readByte();
224     int prevKeyLength = state.keyLength;
225 
226     if ((flag & FLAG_SAME_KEY_LENGTH) == 0) {
227       state.keyLength = ByteBufferUtils.readCompressedInt(source);
228     }
229     if ((flag & FLAG_SAME_VALUE_LENGTH) == 0) {
230       state.valueLength = ByteBufferUtils.readCompressedInt(source);
231     }
232     int commonLength = ByteBufferUtils.readCompressedInt(source);
233 
234     ensureSpace(out, state.keyLength + state.valueLength + KeyValue.ROW_OFFSET);
235 
236     int kvPos = out.position();
237 
238     if (!state.isFirst()) {
239       // copy the prefix
240       int common;
241       int prevOffset;
242 
243       if ((flag & FLAG_SAME_VALUE_LENGTH) == 0) {
244         out.putInt(state.keyLength);
245         out.putInt(state.valueLength);
246         prevOffset = state.prevOffset + KeyValue.ROW_OFFSET;
247         common = commonLength;
248       } else {
249         if ((flag & FLAG_SAME_KEY_LENGTH) != 0) {
250           prevOffset = state.prevOffset;
251           common = commonLength + KeyValue.ROW_OFFSET;
252         } else {
253           out.putInt(state.keyLength);
254           prevOffset = state.prevOffset + KeyValue.KEY_LENGTH_SIZE;
255           common = commonLength + KeyValue.KEY_LENGTH_SIZE;
256         }
257       }
258 
259       ByteBufferUtils.copyFromBufferToBuffer(out, out, prevOffset, common);
260 
261       // copy the rest of the key from the buffer
262       int keyRestLength;
263       if (commonLength < state.rowLength + KeyValue.ROW_LENGTH_SIZE) {
264         // omit the family part of the key, it is always the same
265         int rowWithSizeLength;
266         int rowRestLength;
267 
268         // check length of row
269         if (commonLength < KeyValue.ROW_LENGTH_SIZE) {
270           // not yet copied, do it now
271           ByteBufferUtils.copyFromStreamToBuffer(out, source,
272               KeyValue.ROW_LENGTH_SIZE - commonLength);
273 
274           rowWithSizeLength = out.getShort(out.position() -
275               KeyValue.ROW_LENGTH_SIZE) + KeyValue.ROW_LENGTH_SIZE;
276           rowRestLength = rowWithSizeLength - KeyValue.ROW_LENGTH_SIZE;
277         } else {
278           // already in kvBuffer, just read it
279           rowWithSizeLength = out.getShort(kvPos + KeyValue.ROW_OFFSET) +
280               KeyValue.ROW_LENGTH_SIZE;
281           rowRestLength = rowWithSizeLength - commonLength;
282         }
283 
284         // copy the rest of row
285         ByteBufferUtils.copyFromStreamToBuffer(out, source, rowRestLength);
286 
287         // copy the column family
288         ByteBufferUtils.copyFromBufferToBuffer(out, out,
289             state.prevOffset + KeyValue.ROW_OFFSET + KeyValue.ROW_LENGTH_SIZE
290                 + state.rowLength, state.familyLength
291                 + KeyValue.FAMILY_LENGTH_SIZE);
292         state.rowLength = (short) (rowWithSizeLength -
293             KeyValue.ROW_LENGTH_SIZE);
294 
295         keyRestLength = state.keyLength - rowWithSizeLength -
296             state.familyLength -
297             (KeyValue.FAMILY_LENGTH_SIZE + KeyValue.TIMESTAMP_TYPE_SIZE);
298       } else {
299         // prevRowWithSizeLength is the same as on previous row
300         keyRestLength = state.keyLength - commonLength -
301             KeyValue.TIMESTAMP_TYPE_SIZE;
302       }
303       // copy the rest of the key, after column family == column qualifier
304       ByteBufferUtils.copyFromStreamToBuffer(out, source, keyRestLength);
305 
306       // copy timestamp
307       int prefixTimestamp =
308           (flag & MASK_TIMESTAMP_LENGTH) >>> SHIFT_TIMESTAMP_LENGTH;
309       ByteBufferUtils.copyFromBufferToBuffer(out, out,
310           state.prevTimestampOffset, prefixTimestamp);
311       state.prevTimestampOffset = out.position() - prefixTimestamp;
312       ByteBufferUtils.copyFromStreamToBuffer(out, source,
313           KeyValue.TIMESTAMP_SIZE - prefixTimestamp);
314 
315       // copy the type and value
316       if ((flag & FLAG_SAME_TYPE) != 0) {
317         out.put(state.type);
318         if ((flag & FLAG_SAME_VALUE) != 0) {
319           ByteBufferUtils.copyFromBufferToBuffer(out, out, state.prevOffset +
320               KeyValue.ROW_OFFSET + prevKeyLength, state.valueLength);
321         } else {
322           ByteBufferUtils.copyFromStreamToBuffer(out, source,
323               state.valueLength);
324         }
325       } else {
326         if ((flag & FLAG_SAME_VALUE) != 0) {
327           ByteBufferUtils.copyFromStreamToBuffer(out, source,
328               KeyValue.TYPE_SIZE);
329           ByteBufferUtils.copyFromBufferToBuffer(out, out, state.prevOffset +
330               KeyValue.ROW_OFFSET + prevKeyLength, state.valueLength);
331         } else {
332           ByteBufferUtils.copyFromStreamToBuffer(out, source,
333               state.valueLength + KeyValue.TYPE_SIZE);
334         }
335         state.type = out.get(state.prevTimestampOffset +
336             KeyValue.TIMESTAMP_SIZE);
337       }
338     } else { // this is the first element
339       state.decompressFirstKV(out, source);
340     }
341 
342     state.prevOffset = kvPos;
343   }
344 
345   @Override
346   public void internalEncodeKeyValues(DataOutputStream out,
347       ByteBuffer in, boolean includesMemstoreTS) throws IOException {
348     in.rewind();
349     ByteBufferUtils.putInt(out, in.limit());
350     FastDiffCompressionState previousState = new FastDiffCompressionState();
351     FastDiffCompressionState currentState = new FastDiffCompressionState();
352     while (in.hasRemaining()) {
353       compressSingleKeyValue(previousState, currentState,
354           out, in);
355       afterEncodingKeyValue(in, out, includesMemstoreTS);
356 
357       // swap previousState <-> currentState
358       FastDiffCompressionState tmp = previousState;
359       previousState = currentState;
360       currentState = tmp;
361     }
362   }
363 
364   @Override
365   public ByteBuffer decodeKeyValues(DataInputStream source,
366       int allocHeaderLength, int skipLastBytes, boolean includesMemstoreTS)
367           throws IOException {
368     int decompressedSize = source.readInt();
369     ByteBuffer buffer = ByteBuffer.allocate(decompressedSize +
370         allocHeaderLength);
371     buffer.position(allocHeaderLength);
372     FastDiffCompressionState state = new FastDiffCompressionState();
373     while (source.available() > skipLastBytes) {
374       uncompressSingleKeyValue(source, buffer, state);
375       afterDecodingKeyValue(source, buffer, includesMemstoreTS);
376     }
377 
378     if (source.available() != skipLastBytes) {
379       throw new IllegalStateException("Read too much bytes.");
380     }
381 
382     return buffer;
383   }
384 
385   @Override
386   public ByteBuffer getFirstKeyInBlock(ByteBuffer block) {
387     block.mark();
388     block.position(Bytes.SIZEOF_INT + Bytes.SIZEOF_BYTE);
389     int keyLength = ByteBufferUtils.readCompressedInt(block);
390     ByteBufferUtils.readCompressedInt(block); // valueLength
391     ByteBufferUtils.readCompressedInt(block); // commonLength
392     int pos = block.position();
393     block.reset();
394     return ByteBuffer.wrap(block.array(), pos, keyLength).slice();
395   }
396 
397   @Override
398   public String toString() {
399     return FastDiffDeltaEncoder.class.getSimpleName();
400   }
401 
402   protected static class FastDiffSeekerState extends SeekerState {
403     private byte[] prevTimestampAndType =
404         new byte[KeyValue.TIMESTAMP_TYPE_SIZE];
405     private int rowLengthWithSize;
406     private int familyLengthWithSize;
407 
408     @Override
409     protected void copyFromNext(SeekerState that) {
410       super.copyFromNext(that);
411       FastDiffSeekerState other = (FastDiffSeekerState) that;
412       System.arraycopy(other.prevTimestampAndType, 0,
413           prevTimestampAndType, 0,
414           KeyValue.TIMESTAMP_TYPE_SIZE);
415       rowLengthWithSize = other.rowLengthWithSize;
416       familyLengthWithSize = other.familyLengthWithSize;
417     }
418   }
419 
420   @Override
421   public EncodedSeeker createSeeker(KVComparator comparator,
422       final boolean includesMemstoreTS) {
423     return new BufferedEncodedSeeker<FastDiffSeekerState>(comparator) {
424       private void decode(boolean isFirst) {
425         byte flag = currentBuffer.get();
426         if ((flag & FLAG_SAME_KEY_LENGTH) == 0) {
427           if (!isFirst) {
428             System.arraycopy(current.keyBuffer,
429                 current.keyLength - current.prevTimestampAndType.length,
430                 current.prevTimestampAndType, 0,
431                 current.prevTimestampAndType.length);
432           }
433           current.keyLength = ByteBufferUtils.readCompressedInt(currentBuffer);
434         }
435         if ((flag & FLAG_SAME_VALUE_LENGTH) == 0) {
436           current.valueLength =
437               ByteBufferUtils.readCompressedInt(currentBuffer);
438         }
439         current.lastCommonPrefix =
440             ByteBufferUtils.readCompressedInt(currentBuffer);
441 
442         current.ensureSpaceForKey();
443 
444         if (isFirst) {
445           // copy everything
446           currentBuffer.get(current.keyBuffer, current.lastCommonPrefix,
447               current.keyLength - current.prevTimestampAndType.length);
448           current.rowLengthWithSize = Bytes.toShort(current.keyBuffer, 0) +
449               Bytes.SIZEOF_SHORT;
450           current.familyLengthWithSize =
451               current.keyBuffer[current.rowLengthWithSize] + Bytes.SIZEOF_BYTE;
452         } else if (current.lastCommonPrefix < Bytes.SIZEOF_SHORT) {
453           // length of row is different, copy everything except family
454 
455           // copy the row size
456           int oldRowLengthWithSize = current.rowLengthWithSize;
457           currentBuffer.get(current.keyBuffer, current.lastCommonPrefix,
458               Bytes.SIZEOF_SHORT - current.lastCommonPrefix);
459           current.rowLengthWithSize = Bytes.toShort(current.keyBuffer, 0) +
460               Bytes.SIZEOF_SHORT;
461 
462           // move the column family
463           System.arraycopy(current.keyBuffer, oldRowLengthWithSize,
464               current.keyBuffer, current.rowLengthWithSize,
465               current.familyLengthWithSize);
466 
467           // copy the rest of row
468           currentBuffer.get(current.keyBuffer, Bytes.SIZEOF_SHORT,
469               current.rowLengthWithSize - Bytes.SIZEOF_SHORT);
470 
471           // copy the qualifier
472           currentBuffer.get(current.keyBuffer, current.rowLengthWithSize
473               + current.familyLengthWithSize, current.keyLength
474               - current.rowLengthWithSize - current.familyLengthWithSize
475               - current.prevTimestampAndType.length);
476         } else if (current.lastCommonPrefix < current.rowLengthWithSize) {
477           // We have to copy part of row and qualifier, but the column family
478           // is in the right place.
479 
480           // before column family (rest of row)
481           currentBuffer.get(current.keyBuffer, current.lastCommonPrefix,
482               current.rowLengthWithSize - current.lastCommonPrefix);
483 
484           // after column family (qualifier)
485           currentBuffer.get(current.keyBuffer, current.rowLengthWithSize
486               + current.familyLengthWithSize, current.keyLength
487               - current.rowLengthWithSize - current.familyLengthWithSize
488               - current.prevTimestampAndType.length);
489         } else {
490           // copy just the ending
491           currentBuffer.get(current.keyBuffer, current.lastCommonPrefix,
492               current.keyLength - current.prevTimestampAndType.length
493                   - current.lastCommonPrefix);
494         }
495 
496         // timestamp
497         int pos = current.keyLength - current.prevTimestampAndType.length;
498         int commonTimestampPrefix = (flag & MASK_TIMESTAMP_LENGTH) >>>
499           SHIFT_TIMESTAMP_LENGTH;
500         if ((flag & FLAG_SAME_KEY_LENGTH) == 0) {
501           System.arraycopy(current.prevTimestampAndType, 0, current.keyBuffer,
502               pos, commonTimestampPrefix);
503         }
504         pos += commonTimestampPrefix;
505         currentBuffer.get(current.keyBuffer, pos,
506             Bytes.SIZEOF_LONG - commonTimestampPrefix);
507         pos += Bytes.SIZEOF_LONG - commonTimestampPrefix;
508 
509         // type
510         if ((flag & FLAG_SAME_TYPE) == 0) {
511           currentBuffer.get(current.keyBuffer, pos, Bytes.SIZEOF_BYTE);
512         } else if ((flag & FLAG_SAME_KEY_LENGTH) == 0) {
513           current.keyBuffer[pos] =
514               current.prevTimestampAndType[Bytes.SIZEOF_LONG];
515         }
516 
517         // handle value
518         if ((flag & FLAG_SAME_VALUE) == 0) {
519           current.valueOffset = currentBuffer.position();
520           ByteBufferUtils.skip(currentBuffer, current.valueLength);
521         }
522 
523         if (includesMemstoreTS) {
524           current.memstoreTS = ByteBufferUtils.readVLong(currentBuffer);
525         } else {
526           current.memstoreTS = 0;
527         }
528         current.nextKvOffset = currentBuffer.position();
529       }
530 
531       @Override
532       protected void decodeFirst() {
533         ByteBufferUtils.skip(currentBuffer, Bytes.SIZEOF_INT);
534         decode(true);
535       }
536 
537       @Override
538       protected void decodeNext() {
539         decode(false);
540       }
541 
542       @Override
543       protected FastDiffSeekerState createSeekerState() {
544         return new FastDiffSeekerState();
545       }
546     };
547   }
548 }