@InterfaceAudience.Private public class HFileWriterImpl extends Object implements HFile.Writer
HFile
writers.Modifier and Type | Field and Description |
---|---|
private List<HFileBlock.BlockWritable> |
additionalLoadOnOpenData
Additional data items to be written to the "load-on-open" section.
|
protected HFileDataBlockEncoder |
blockEncoder
The data block encoding which will be used.
|
protected HFileBlock.Writer |
blockWriter
block writer
|
protected CacheConfig |
cacheConf
Cache configuration for caching data on write.
|
protected boolean |
closeOutputStream
True if we opened the
outputStream (and so will close it). |
protected CellComparator |
comparator
Key comparator.
|
private HFileBlockIndex.BlockIndexWriter |
dataBlockIndexWriter |
private int |
encodedBlockSizeLimit
Block size limit after encoding, used to unify encoded block Cache entry size
|
protected long |
entryCount
Total # of key/value entries, i.e.
|
protected HFile.FileInfo |
fileInfo
A "file info" block: a key-value map of file-wide metadata.
|
protected Cell |
firstCellInBlock
First cell in a block.
|
private long |
firstDataBlockOffset
The offset of the first data block or -1 if the file is empty.
|
protected HFileContext |
hFileContext |
private List<InlineBlockWriter> |
inlineBlockWriters
Inline block writers for multi-level block index and compound Blooms.
|
static int |
KEY_VALUE_VER_WITH_MEMSTORE
Version for KeyValue which includes memstore timestamp
|
static byte[] |
KEY_VALUE_VERSION
KeyValue version in FileInfo
|
protected Cell |
lastCell
The Cell previously appended.
|
private Cell |
lastCellOfPreviousBlock
The last(stop) Cell of the previous data block.
|
protected long |
lastDataBlockOffset
The offset of the last data block or 0 if the file is empty.
|
private static org.slf4j.Logger |
LOG |
protected long |
maxMemstoreTS |
private int |
maxTagsLength |
private HFileBlockIndex.BlockIndexWriter |
metaBlockIndexWriter |
protected List<org.apache.hadoop.io.Writable> |
metaData
Writable s representing meta block data. |
protected List<byte[]> |
metaNames
Meta block names.
|
protected String |
name
Name for this object used when logging or in toString.
|
protected org.apache.hadoop.fs.FSDataOutputStream |
outputStream
FileSystem stream to write into.
|
protected org.apache.hadoop.fs.Path |
path
May be null if we were passed a stream.
|
protected long |
totalKeyLength
Used for calculating the average key length.
|
protected long |
totalUncompressedBytes
Total uncompressed bytes, maybe calculate a compression ratio later.
|
protected long |
totalValueLength
Used for calculating the average value length.
|
static String |
UNIFIED_ENCODED_BLOCKSIZE_RATIO
if this feature is enabled, preCalculate encoded data size before real encoding happens
|
private static long |
UNSET |
MAX_MEMSTORE_TS_KEY
Constructor and Description |
---|
HFileWriterImpl(org.apache.hadoop.conf.Configuration conf,
CacheConfig cacheConf,
org.apache.hadoop.fs.Path path,
org.apache.hadoop.fs.FSDataOutputStream outputStream,
CellComparator comparator,
HFileContext fileContext) |
Modifier and Type | Method and Description |
---|---|
private void |
addBloomFilter(BloomFilterWriter bfw,
BlockType blockType) |
void |
addDeleteFamilyBloomFilter(BloomFilterWriter bfw)
Store delete family Bloom filter in the file, which is only supported in
HFile V2.
|
void |
addGeneralBloomFilter(BloomFilterWriter bfw)
Store general Bloom filter in the file.
|
void |
addInlineBlockWriter(InlineBlockWriter ibw)
Adds an inline block writer such as a multi-level block index writer or
a compound Bloom filter writer.
|
void |
append(Cell cell)
Add key/value to file.
|
void |
appendFileInfo(byte[] k,
byte[] v)
Add to the file info.
|
void |
appendMetaBlock(String metaBlockName,
org.apache.hadoop.io.Writable content)
Add a meta block to the end of the file.
|
void |
beforeShipped()
The action that needs to be performed before
Shipper.shipped() is performed |
protected void |
checkBlockBoundary()
At a block boundary, write all the inline blocks and opens new block.
|
protected boolean |
checkKey(Cell cell)
Checks that the given Cell's key does not violate the key order.
|
protected void |
checkValue(byte[] value,
int offset,
int length)
Checks the given value for validity.
|
void |
close() |
static Compression.Algorithm |
compressionByName(String algoName) |
protected static org.apache.hadoop.fs.FSDataOutputStream |
createOutputStream(org.apache.hadoop.conf.Configuration conf,
org.apache.hadoop.fs.FileSystem fs,
org.apache.hadoop.fs.Path path,
InetSocketAddress[] favoredNodes)
A helper method to create HFile output streams in constructors
|
private void |
doCacheOnWrite(long offset)
Caches the last written HFile block.
|
private void |
finishBlock()
Clean up the data block that is currently being written.
|
protected void |
finishClose(FixedFileTrailer trailer) |
protected void |
finishFileInfo() |
protected void |
finishInit(org.apache.hadoop.conf.Configuration conf)
Additional initialization steps
|
HFileContext |
getFileContext()
Return the file context for the HFile this writer belongs to
|
Cell |
getLastCell() |
protected int |
getMajorVersion() |
static Cell |
getMidpoint(CellComparator comparator,
Cell left,
Cell right)
Try to return a Cell that falls between
left and
right but that is shorter; i.e. |
private static byte[] |
getMinimumMidpointArray(byte[] leftArray,
int leftOffset,
int leftLength,
byte[] rightArray,
int rightOffset,
int rightLength) |
private static byte[] |
getMinimumMidpointArray(ByteBuffer left,
int leftOffset,
int leftLength,
ByteBuffer right,
int rightOffset,
int rightLength) |
protected int |
getMinorVersion() |
org.apache.hadoop.fs.Path |
getPath() |
protected void |
newBlock()
Ready a new block for writing.
|
String |
toString() |
protected void |
writeFileInfo(FixedFileTrailer trailer,
DataOutputStream out)
Sets the file info offset in the trailer, finishes up populating fields in
the file info, and writes the file info into the given data output.
|
private void |
writeInlineBlocks(boolean closing)
Gives inline block writers an opportunity to contribute blocks.
|
private static final org.slf4j.Logger LOG
private static final long UNSET
public static final String UNIFIED_ENCODED_BLOCKSIZE_RATIO
private final int encodedBlockSizeLimit
protected org.apache.hadoop.fs.FSDataOutputStream outputStream
protected final boolean closeOutputStream
outputStream
(and so will close it).protected HFile.FileInfo fileInfo
protected long entryCount
protected long totalKeyLength
protected long totalValueLength
protected long totalUncompressedBytes
protected final CellComparator comparator
protected List<org.apache.hadoop.io.Writable> metaData
Writable
s representing meta block data.protected Cell firstCellInBlock
protected final org.apache.hadoop.fs.Path path
protected final CacheConfig cacheConf
protected final String name
protected final HFileDataBlockEncoder blockEncoder
NoOpDataBlockEncoder.INSTANCE
if there is no encoding.protected final HFileContext hFileContext
private int maxTagsLength
public static final byte[] KEY_VALUE_VERSION
public static final int KEY_VALUE_VER_WITH_MEMSTORE
private List<InlineBlockWriter> inlineBlockWriters
protected HFileBlock.Writer blockWriter
private HFileBlockIndex.BlockIndexWriter dataBlockIndexWriter
private HFileBlockIndex.BlockIndexWriter metaBlockIndexWriter
private long firstDataBlockOffset
protected long lastDataBlockOffset
private Cell lastCellOfPreviousBlock
private List<HFileBlock.BlockWritable> additionalLoadOnOpenData
protected long maxMemstoreTS
public HFileWriterImpl(org.apache.hadoop.conf.Configuration conf, CacheConfig cacheConf, org.apache.hadoop.fs.Path path, org.apache.hadoop.fs.FSDataOutputStream outputStream, CellComparator comparator, HFileContext fileContext)
public void appendFileInfo(byte[] k, byte[] v) throws IOException
HFile.Reader.loadFileInfo()
.appendFileInfo
in interface HFile.Writer
k
- Keyv
- ValueIOException
- in case the key or the value are invalidprotected final void writeFileInfo(FixedFileTrailer trailer, DataOutputStream out) throws IOException
outputStream
is that we store
file info as a block in version 2.trailer
- fixed file trailerout
- the data output to write the file info toIOException
protected boolean checkKey(Cell cell) throws IOException
cell
- Cell whose key to check.IOException
- if the key or the key order is wrongprotected void checkValue(byte[] value, int offset, int length) throws IOException
IOException
public org.apache.hadoop.fs.Path getPath()
getPath
in interface HFile.Writer
public static Compression.Algorithm compressionByName(String algoName)
protected static org.apache.hadoop.fs.FSDataOutputStream createOutputStream(org.apache.hadoop.conf.Configuration conf, org.apache.hadoop.fs.FileSystem fs, org.apache.hadoop.fs.Path path, InetSocketAddress[] favoredNodes) throws IOException
IOException
protected void finishInit(org.apache.hadoop.conf.Configuration conf)
protected void checkBlockBoundary() throws IOException
IOException
private void finishBlock() throws IOException
IOException
public static Cell getMidpoint(CellComparator comparator, Cell left, Cell right)
left
and
right
but that is shorter; i.e. takes up less space. This
trick is used building HFile block index. Its an optimization. It does not
always work. In this case we'll just return the right
cell.comparator
- Comparator to use.left
- right
- left
and right
.private static byte[] getMinimumMidpointArray(byte[] leftArray, int leftOffset, int leftLength, byte[] rightArray, int rightOffset, int rightLength)
leftArray
- leftOffset
- leftLength
- rightArray
- rightOffset
- rightLength
- private static byte[] getMinimumMidpointArray(ByteBuffer left, int leftOffset, int leftLength, ByteBuffer right, int rightOffset, int rightLength)
private void writeInlineBlocks(boolean closing) throws IOException
IOException
private void doCacheOnWrite(long offset)
offset
- the offset of the block we want to cache. Used to determine
the cache key.protected void newBlock() throws IOException
IOException
public void appendMetaBlock(String metaBlockName, org.apache.hadoop.io.Writable content)
appendFileInfo(byte[], byte[])
appendMetaBlock
in interface HFile.Writer
metaBlockName
- name of the blockcontent
- will call readFields to get data later (DO NOT REUSE)public void close() throws IOException
close
in interface Closeable
close
in interface AutoCloseable
IOException
public void addInlineBlockWriter(InlineBlockWriter ibw)
HFile.Writer
addInlineBlockWriter
in interface HFile.Writer
public void addGeneralBloomFilter(BloomFilterWriter bfw)
HFile.Writer
addGeneralBloomFilter
in interface HFile.Writer
public void addDeleteFamilyBloomFilter(BloomFilterWriter bfw)
HFile.Writer
addDeleteFamilyBloomFilter
in interface HFile.Writer
private void addBloomFilter(BloomFilterWriter bfw, BlockType blockType)
public HFileContext getFileContext()
HFile.Writer
getFileContext
in interface HFile.Writer
public void append(Cell cell) throws IOException
append
in interface CellSink
cell
- Cell to add. Cannot be empty nor null.IOException
public void beforeShipped() throws IOException
ShipperListener
Shipper.shipped()
is performedbeforeShipped
in interface ShipperListener
IOException
public Cell getLastCell()
protected void finishFileInfo() throws IOException
IOException
protected int getMajorVersion()
protected int getMinorVersion()
protected void finishClose(FixedFileTrailer trailer) throws IOException
IOException
Copyright © 2007–2020 The Apache Software Foundation. All rights reserved.