@InterfaceAudience.Private public class FastDiffDeltaEncoder extends java.lang.Object
DiffKeyDeltaEncoder but supposedly faster.
Compress using:
- store size of common prefix
- save column family once in the first KeyValue
- use integer compression for key, value and prefix (7-bit encoding)
- use bits to avoid duplication key length, value length
and type if it same as previous
- store in 3 bits length of prefix timestamp
with previous KeyValue's timestamp
- one bit which allow to omit value if it is the same
Format:
- 1 byte: flag
- 1-5 bytes: key length (only if FLAG_SAME_KEY_LENGTH is not set in flag)
- 1-5 bytes: value length (only if FLAG_SAME_VALUE_LENGTH is not set in flag)
- 1-5 bytes: prefix length
- ... bytes: rest of the row (if prefix length is small enough)
- ... bytes: qualifier (or suffix depending on prefix length)
- 1-8 bytes: timestamp suffix
- 1 byte: type (only if FLAG_SAME_TYPE is not set in the flag)
- ... bytes: value (only if FLAG_SAME_VALUE is not set in the flag)| Modifier and Type | Class and Description |
|---|---|
protected static class |
org.apache.hadoop.hbase.io.encoding.BufferedDataBlockEncoder.BufferedEncodedSeeker<STATE extends org.apache.hadoop.hbase.io.encoding.BufferedDataBlockEncoder.SeekerState> |
protected static class |
org.apache.hadoop.hbase.io.encoding.BufferedDataBlockEncoder.ClonedSeekerState
Copies only the key part of the keybuffer by doing a deep copy and passes the
seeker state members for taking a clone.
|
protected static class |
org.apache.hadoop.hbase.io.encoding.BufferedDataBlockEncoder.SeekerState |
protected static class |
FastDiffDeltaEncoder.FastDiffSeekerState |
DataBlockEncoder.EncodedSeeker| Constructor and Description |
|---|
FastDiffDeltaEncoder() |
| Modifier and Type | Method and Description |
|---|---|
protected void |
afterDecodingKeyValue(java.io.DataInputStream source,
java.nio.ByteBuffer dest,
HFileBlockDefaultDecodingContext decodingCtx) |
protected int |
afterEncodingKeyValue(Cell cell,
java.io.DataOutputStream out,
HFileBlockDefaultEncodingContext encodingCtx) |
DataBlockEncoder.EncodedSeeker |
createSeeker(KeyValue.KVComparator comparator,
HFileBlockDecodingContext decodingCtx)
Create a HFileBlock seeker which find KeyValues within a block.
|
java.nio.ByteBuffer |
decodeKeyValues(java.io.DataInputStream source,
HFileBlockDecodingContext blkDecodingCtx)
Decode.
|
int |
encode(Cell cell,
HFileBlockEncodingContext encodingCtx,
java.io.DataOutputStream out)
Encodes a KeyValue.
|
void |
endBlockEncoding(HFileBlockEncodingContext encodingCtx,
java.io.DataOutputStream out,
byte[] uncompressedBytesWithHeader)
Ends encoding for a block of KeyValues.
|
protected static void |
ensureSpace(java.nio.ByteBuffer out,
int length)
Asserts that there is at least the given amount of unfilled space
remaining in the given buffer.
|
java.nio.ByteBuffer |
getFirstKeyInBlock(java.nio.ByteBuffer block)
Return first key in block.
|
protected java.nio.ByteBuffer |
internalDecodeKeyValues(java.io.DataInputStream source,
int allocateHeaderLength,
int skipLastBytes,
HFileBlockDefaultDecodingContext decodingCtx) |
int |
internalEncode(Cell cell,
HFileBlockDefaultEncodingContext encodingContext,
java.io.DataOutputStream out) |
HFileBlockDecodingContext |
newDataBlockDecodingContext(HFileContext meta)
Creates an encoder specific decoding context, which will prepare the data
before actual decoding
|
HFileBlockEncodingContext |
newDataBlockEncodingContext(DataBlockEncoding encoding,
byte[] header,
HFileContext meta)
Creates a encoder specific encoding context
|
void |
startBlockEncoding(HFileBlockEncodingContext blkEncodingCtx,
java.io.DataOutputStream out)
Starts encoding for a block of KeyValues.
|
java.lang.String |
toString() |
public int internalEncode(Cell cell, HFileBlockDefaultEncodingContext encodingContext, java.io.DataOutputStream out) throws java.io.IOException
java.io.IOExceptionprotected java.nio.ByteBuffer internalDecodeKeyValues(java.io.DataInputStream source,
int allocateHeaderLength,
int skipLastBytes,
HFileBlockDefaultDecodingContext decodingCtx)
throws java.io.IOException
java.io.IOExceptionpublic java.nio.ByteBuffer getFirstKeyInBlock(java.nio.ByteBuffer block)
DataBlockEncoderblock - encoded block we want index, the position will not changepublic java.lang.String toString()
toString in class java.lang.Objectpublic DataBlockEncoder.EncodedSeeker createSeeker(KeyValue.KVComparator comparator, HFileBlockDecodingContext decodingCtx)
DataBlockEncodercomparator - what kind of comparison should be usedpublic java.nio.ByteBuffer decodeKeyValues(java.io.DataInputStream source,
HFileBlockDecodingContext blkDecodingCtx)
throws java.io.IOException
DataBlockEncoderdecodeKeyValues in interface DataBlockEncodersource - Compressed stream of KeyValues.java.io.IOException - If there is an error in source.protected final int afterEncodingKeyValue(Cell cell, java.io.DataOutputStream out, HFileBlockDefaultEncodingContext encodingCtx) throws java.io.IOException
cell - out - encodingCtx - java.io.IOExceptionprotected final void afterDecodingKeyValue(java.io.DataInputStream source,
java.nio.ByteBuffer dest,
HFileBlockDefaultDecodingContext decodingCtx)
throws java.io.IOException
java.io.IOExceptionpublic HFileBlockEncodingContext newDataBlockEncodingContext(DataBlockEncoding encoding, byte[] header, HFileContext meta)
DataBlockEncodernewDataBlockEncodingContext in interface DataBlockEncoderencoding - encoding strategy usedheader - header bytes to be written, put a dummy header here if the header
is unknownmeta - HFile meta datapublic HFileBlockDecodingContext newDataBlockDecodingContext(HFileContext meta)
DataBlockEncodernewDataBlockDecodingContext in interface DataBlockEncodermeta - HFile meta dataprotected static void ensureSpace(java.nio.ByteBuffer out,
int length)
throws EncoderBufferTooSmallException
out - typically, the buffer we are writing tolength - the required space in the bufferEncoderBufferTooSmallException - If there are no enough bytes.public void startBlockEncoding(HFileBlockEncodingContext blkEncodingCtx, java.io.DataOutputStream out) throws java.io.IOException
DataBlockEncoderDataBlockEncoder.endBlockEncoding(HFileBlockEncodingContext, DataOutputStream, byte[]) to finish
encoding of a block.startBlockEncoding in interface DataBlockEncoderjava.io.IOExceptionpublic int encode(Cell cell, HFileBlockEncodingContext encodingCtx, java.io.DataOutputStream out) throws java.io.IOException
DataBlockEncoderencode in interface DataBlockEncoderjava.io.IOExceptionpublic void endBlockEncoding(HFileBlockEncodingContext encodingCtx, java.io.DataOutputStream out, byte[] uncompressedBytesWithHeader) throws java.io.IOException
DataBlockEncoderendBlockEncoding in interface DataBlockEncoderjava.io.IOException