@InterfaceAudience.Private public class FastDiffDeltaEncoder extends AbstractDataBlockEncoder
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 |
BufferedDataBlockEncoder.BufferedEncodedSeeker<STATE extends BufferedDataBlockEncoder.SeekerState> |
protected static class |
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 |
BufferedDataBlockEncoder.SeekerState |
protected static class |
FastDiffDeltaEncoder.FastDiffSeekerState |
AbstractDataBlockEncoder.AbstractEncodedSeeker
DataBlockEncoder.EncodedSeeker
Constructor and Description |
---|
FastDiffDeltaEncoder() |
Modifier and Type | Method and Description |
---|---|
protected void |
afterDecodingKeyValue(DataInputStream source,
ByteBuffer dest,
HFileBlockDefaultDecodingContext decodingCtx) |
protected int |
afterEncodingKeyValue(Cell cell,
DataOutputStream out,
HFileBlockDefaultEncodingContext encodingCtx) |
DataBlockEncoder.EncodedSeeker |
createSeeker(KeyValue.KVComparator comparator,
HFileBlockDecodingContext decodingCtx)
Create a HFileBlock seeker which find KeyValues within a block.
|
ByteBuffer |
decodeKeyValues(DataInputStream source,
HFileBlockDecodingContext blkDecodingCtx)
Decode.
|
int |
encode(Cell cell,
HFileBlockEncodingContext encodingCtx,
DataOutputStream out)
Encodes a KeyValue.
|
void |
endBlockEncoding(HFileBlockEncodingContext encodingCtx,
DataOutputStream out,
byte[] uncompressedBytesWithHeader)
Ends encoding for a block of KeyValues.
|
protected static void |
ensureSpace(ByteBuffer out,
int length)
Asserts that there is at least the given amount of unfilled space
remaining in the given buffer.
|
ByteBuffer |
getFirstKeyInBlock(ByteBuffer block)
Return first key in block.
|
protected ByteBuffer |
internalDecodeKeyValues(DataInputStream source,
int allocateHeaderLength,
int skipLastBytes,
HFileBlockDefaultDecodingContext decodingCtx) |
int |
internalEncode(Cell cell,
HFileBlockDefaultEncodingContext encodingContext,
DataOutputStream out) |
void |
startBlockEncoding(HFileBlockEncodingContext blkEncodingCtx,
DataOutputStream out)
Starts encoding for a block of KeyValues.
|
String |
toString() |
newDataBlockDecodingContext, newDataBlockEncodingContext, postEncoding
public int internalEncode(Cell cell, HFileBlockDefaultEncodingContext encodingContext, DataOutputStream out) throws IOException
IOException
protected ByteBuffer internalDecodeKeyValues(DataInputStream source, int allocateHeaderLength, int skipLastBytes, HFileBlockDefaultDecodingContext decodingCtx) throws IOException
IOException
public ByteBuffer getFirstKeyInBlock(ByteBuffer block)
DataBlockEncoder
block
- encoded block we want index, the position will not changepublic DataBlockEncoder.EncodedSeeker createSeeker(KeyValue.KVComparator comparator, HFileBlockDecodingContext decodingCtx)
DataBlockEncoder
comparator
- what kind of comparison should be usedpublic ByteBuffer decodeKeyValues(DataInputStream source, HFileBlockDecodingContext blkDecodingCtx) throws IOException
DataBlockEncoder
source
- Compressed stream of KeyValues.IOException
- If there is an error in source.protected final int afterEncodingKeyValue(Cell cell, DataOutputStream out, HFileBlockDefaultEncodingContext encodingCtx) throws IOException
cell
- out
- encodingCtx
- IOException
protected final void afterDecodingKeyValue(DataInputStream source, ByteBuffer dest, HFileBlockDefaultDecodingContext decodingCtx) throws IOException
IOException
protected static void ensureSpace(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, DataOutputStream out) throws IOException
DataBlockEncoder
DataBlockEncoder.endBlockEncoding(HFileBlockEncodingContext, DataOutputStream, byte[])
to finish
encoding of a block.IOException
public int encode(Cell cell, HFileBlockEncodingContext encodingCtx, DataOutputStream out) throws IOException
DataBlockEncoder
IOException
public void endBlockEncoding(HFileBlockEncodingContext encodingCtx, DataOutputStream out, byte[] uncompressedBytesWithHeader) throws IOException
DataBlockEncoder
IOException
Copyright © 2007–2019 The Apache Software Foundation. All rights reserved.