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