A TFile is a container of key-value pairs. Both keys and values are type-less
bytes. Keys are restricted to 64KB, value length is not restricted
(practically limited to the available disk storage). TFile further provides
the following features:
- Block Compression.
- Named meta data blocks.
- Sorted or unsorted keys.
- Seek by key or by file offset.
The memory footprint of a TFile includes the following:
- Some constant overhead of reading or writing a compressed block.
- Each compressed block requires one compression/decompression codec for
I/O.
- Temporary space to buffer the key.
- Temporary space to buffer the value (for TFile.Writer only). Values are
chunk encoded, so that we buffer at most one chunk of user data. By default,
the chunk buffer is 1MB. Reading chunked value does not require additional
memory.
- TFile index, which is proportional to the total number of Data Blocks.
The total amount of memory needed to hold the index can be estimated as
(56+AvgKeySize)*NumBlocks.
- MetaBlock index, which is proportional to the total number of Meta
Blocks.The total amount of memory needed to hold the index for Meta Blocks
can be estimated as (40+AvgMetaBlockName)*NumMetaBlock.
The behavior of TFile can be customized by the following variables through
Configuration:
- tfile.io.chunk.size: Value chunk size. Integer (in bytes). Default
to 1MB. Values of the length less than the chunk size is guaranteed to have
known value length in read time (See
TFile.Reader.Scanner.Entry.isValueLengthKnown()
).
- tfile.fs.output.buffer.size: Buffer size used for
FSDataOutputStream. Integer (in bytes). Default to 256KB.
- tfile.fs.input.buffer.size: Buffer size used for
FSDataInputStream. Integer (in bytes). Default to 256KB.
Suggestions on performance optimization.
- Minimum block size. We recommend a setting of minimum block size between
256KB to 1MB for general usage. Larger block size is preferred if files are
primarily for sequential access. However, it would lead to inefficient random
access (because there are more data to decompress). Smaller blocks are good
for random access, but require more memory to hold the block index, and may
be slower to create (because we must flush the compressor stream at the
conclusion of each data block, which leads to an FS I/O flush). Further, due
to the internal caching in Compression codec, the smallest possible block
size would be around 20KB-30KB.
- The current implementation does not offer true multi-threading for
reading. The implementation uses FSDataInputStream seek()+read(), which is
shown to be much faster than positioned-read call in single thread mode.
However, it also means that if multiple threads attempt to access the same
TFile (using multiple scanners) simultaneously, the actual I/O is carried out
sequentially even if they access different DFS blocks.
- Compression codec. Use "none" if the data is not very compressable (by
compressable, I mean a compression ratio at least 2:1). Generally, use "lzo"
as the starting point for experimenting. "gz" overs slightly better
compression ratio over "lzo" but requires 4x CPU to compress and 2x CPU to
decompress, comparing to "lzo".
- File system buffering, if the underlying FSDataInputStream and
FSDataOutputStream is already adequately buffered; or if applications
reads/writes keys and values in large buffers, we can reduce the sizes of
input/output buffering in TFile layer by setting the configuration parameters
"tfile.fs.input.buffer.size" and "tfile.fs.output.buffer.size".
Some design rationale behind TFile can be found at
Hadoop-3315.