/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with this
* work for additional information regarding copyright ownership. The ASF
* licenses this file to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
* License for the specific language governing permissions and limitations under
* the License.
*/
package org.apache.hadoop.io.file.tfile;
import java.io.ByteArrayInputStream;
import java.io.Closeable;
import java.io.DataInput;
import java.io.DataInputStream;
import java.io.DataOutput;
import java.io.DataOutputStream;
import java.io.EOFException;
import java.io.IOException;
import java.io.OutputStream;
import java.util.ArrayList;
import java.util.Comparator;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.io.BoundedByteArrayOutputStream;
import org.apache.hadoop.io.BytesWritable;
import org.apache.hadoop.io.DataInputBuffer;
import org.apache.hadoop.io.DataOutputBuffer;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.RawComparator;
import org.apache.hadoop.io.WritableComparator;
import org.apache.hadoop.io.file.tfile.BCFile.Reader.BlockReader;
import org.apache.hadoop.io.file.tfile.BCFile.Writer.BlockAppender;
import org.apache.hadoop.io.file.tfile.Chunk.ChunkDecoder;
import org.apache.hadoop.io.file.tfile.Chunk.ChunkEncoder;
import org.apache.hadoop.io.file.tfile.CompareUtils.BytesComparator;
import org.apache.hadoop.io.file.tfile.CompareUtils.MemcmpRawComparator;
import org.apache.hadoop.io.file.tfile.Utils.Version;
import org.apache.hadoop.io.serializer.JavaSerializationComparator;
/**
* 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:
* <ul>
* <li>Block Compression.
* <li>Named meta data blocks.
* <li>Sorted or unsorted keys.
* <li>Seek by key or by file offset.
* </ul>
* The memory footprint of a TFile includes the following:
* <ul>
* <li>Some constant overhead of reading or writing a compressed block.
* <ul>
* <li>Each compressed block requires one compression/decompression codec for
* I/O.
* <li>Temporary space to buffer the key.
* <li>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.
* </ul>
* <li>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.
* <li>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.
* </ul>
* <p>
* The behavior of TFile can be customized by the following variables through
* Configuration:
* <ul>
* <li><b>tfile.io.chunk.size</b>: 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
* {@link TFile.Reader.Scanner.Entry#isValueLengthKnown()}).
* <li><b>tfile.fs.output.buffer.size</b>: Buffer size used for
* FSDataOutputStream. Integer (in bytes). Default to 256KB.
* <li><b>tfile.fs.input.buffer.size</b>: Buffer size used for
* FSDataInputStream. Integer (in bytes). Default to 256KB.
* </ul>
* <p>
* Suggestions on performance optimization.
* <ul>
* <li>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.
* <li>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.
* <li>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".
* <li>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".
* </ul>
*
* Some design rationale behind TFile can be found at <a
* href=https://issues.apache.org/jira/browse/HADOOP-3315>Hadoop-3315</a>.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class TFile {
static final Log LOG = LogFactory.getLog(TFile.class);
private static final String CHUNK_BUF_SIZE_ATTR = "tfile.io.chunk.size";
private static final String FS_INPUT_BUF_SIZE_ATTR =
"tfile.fs.input.buffer.size";
private static final String FS_OUTPUT_BUF_SIZE_ATTR =
"tfile.fs.output.buffer.size";
static int getChunkBufferSize(Configuration conf) {
int ret = conf.getInt(CHUNK_BUF_SIZE_ATTR, 1024 * 1024);
return (ret > 0) ? ret : 1024 * 1024;
}
static int getFSInputBufferSize(Configuration conf) {
return conf.getInt(FS_INPUT_BUF_SIZE_ATTR, 256 * 1024);
}
static int getFSOutputBufferSize(Configuration conf) {
return conf.getInt(FS_OUTPUT_BUF_SIZE_ATTR, 256 * 1024);
}
private static final int MAX_KEY_SIZE = 64 * 1024; // 64KB
static final Version API_VERSION = new Version((short) 1, (short) 0);
/** compression: gzip */
public static final String COMPRESSION_GZ = "gz";
/** compression: lzo */
public static final String COMPRESSION_LZO = "lzo";
/** compression: none */
public static final String COMPRESSION_NONE = "none";
/** comparator: memcmp */
public static final String COMPARATOR_MEMCMP = "memcmp";
/** comparator prefix: java class */
public static final String COMPARATOR_JCLASS = "jclass:";
/**
* Make a raw comparator from a string name.
*
* @param name
* Comparator name
* @return A RawComparable comparator.
*/
static public Comparator<RawComparable> makeComparator(String name) {
return TFileMeta.makeComparator(name);
}
// Prevent the instantiation of TFiles
private TFile() {
// nothing
}
/**
* Get names of supported compression algorithms. The names are acceptable by
* TFile.Writer.
*
* @return Array of strings, each represents a supported compression
* algorithm. Currently, the following compression algorithms are
* supported.
* <ul>
* <li>"none" - No compression.
* <li>"lzo" - LZO compression.
* <li>"gz" - GZIP compression.
* </ul>
*/
public static String[] getSupportedCompressionAlgorithms() {
return Compression.getSupportedAlgorithms();
}
/**
* TFile Writer.
*/
@InterfaceStability.Evolving
public static class Writer implements Closeable {
// minimum compressed size for a block.
private final int sizeMinBlock;
// Meta blocks.
final TFileIndex tfileIndex;
final TFileMeta tfileMeta;
// reference to the underlying BCFile.
private BCFile.Writer writerBCF;
// current data block appender.
BlockAppender blkAppender;
long blkRecordCount;
// buffers for caching the key.
BoundedByteArrayOutputStream currentKeyBufferOS;
BoundedByteArrayOutputStream lastKeyBufferOS;
// buffer used by chunk codec
private byte[] valueBuffer;
/**
* Writer states. The state always transits in circles: READY -> IN_KEY ->
* END_KEY -> IN_VALUE -> READY.
*/
private enum State {
READY, // Ready to start a new key-value pair insertion.
IN_KEY, // In the middle of key insertion.
END_KEY, // Key insertion complete, ready to insert value.
IN_VALUE, // In value insertion.
// ERROR, // Error encountered, cannot continue.
CLOSED, // TFile already closed.
};
// current state of Writer.
State state = State.READY;
Configuration conf;
long errorCount = 0;
/**
* Constructor
*
* @param fsdos
* output stream for writing. Must be at position 0.
* @param minBlockSize
* Minimum compressed block size in bytes. A compression block will
* not be closed until it reaches this size except for the last
* block.
* @param compressName
* Name of the compression algorithm. Must be one of the strings
* returned by {@link TFile#getSupportedCompressionAlgorithms()}.
* @param comparator
* Leave comparator as null or empty string if TFile is not sorted.
* Otherwise, provide the string name for the comparison algorithm
* for keys. Two kinds of comparators are supported.
* <ul>
* <li>Algorithmic comparator: binary comparators that is language
* independent. Currently, only "memcmp" is supported.
* <li>Language-specific comparator: binary comparators that can
* only be constructed in specific language. For Java, the syntax
* is "jclass:", followed by the class name of the RawComparator.
* Currently, we only support RawComparators that can be
* constructed through the default constructor (with no
* parameters). Parameterized RawComparators such as
* {@link WritableComparator} or
* {@link JavaSerializationComparator} may not be directly used.
* One should write a wrapper class that inherits from such classes
* and use its default constructor to perform proper
* initialization.
* </ul>
* @param conf
* The configuration object.
* @throws IOException
*/
public Writer(FSDataOutputStream fsdos, int minBlockSize,
String compressName, String comparator, Configuration conf)
throws IOException {
sizeMinBlock = minBlockSize;
tfileMeta = new TFileMeta(comparator);
tfileIndex = new TFileIndex(tfileMeta.getComparator());
writerBCF = new BCFile.Writer(fsdos, compressName, conf);
currentKeyBufferOS = new BoundedByteArrayOutputStream(MAX_KEY_SIZE);
lastKeyBufferOS = new BoundedByteArrayOutputStream(MAX_KEY_SIZE);
this.conf = conf;
}
/**
* Close the Writer. Resources will be released regardless of the exceptions
* being thrown. Future close calls will have no effect.
*
* The underlying FSDataOutputStream is not closed.
*/
public void close() throws IOException {
if ((state == State.CLOSED)) {
return;
}
try {
// First try the normal finish.
// Terminate upon the first Exception.
if (errorCount == 0) {
if (state != State.READY) {
throw new IllegalStateException(
"Cannot close TFile in the middle of key-value insertion.");
}
finishDataBlock(true);
// first, write out data:TFile.meta
BlockAppender outMeta =
writerBCF
.prepareMetaBlock(TFileMeta.BLOCK_NAME, COMPRESSION_NONE);
try {
tfileMeta.write(outMeta);
} finally {
outMeta.close();
}
// second, write out data:TFile.index
BlockAppender outIndex =
writerBCF.prepareMetaBlock(TFileIndex.BLOCK_NAME);
try {
tfileIndex.write(outIndex);
} finally {
outIndex.close();
}
writerBCF.close();
}
} finally {
IOUtils.cleanup(LOG, blkAppender, writerBCF);
blkAppender = null;
writerBCF = null;
state = State.CLOSED;
}
}
/**
* Adding a new key-value pair to the TFile. This is synonymous to
* append(key, 0, key.length, value, 0, value.length)
*
* @param key
* Buffer for key.
* @param value
* Buffer for value.
* @throws IOException
*/
public void append(byte[] key, byte[] value) throws IOException {
append(key, 0, key.length, value, 0, value.length);
}
/**
* Adding a new key-value pair to TFile.
*
* @param key
* buffer for key.
* @param koff
* offset in key buffer.
* @param klen
* length of key.
* @param value
* buffer for value.
* @param voff
* offset in value buffer.
* @param vlen
* length of value.
* @throws IOException
* Upon IO errors.
* <p>
* If an exception is thrown, the TFile will be in an inconsistent
* state. The only legitimate call after that would be close
*/
public void append(byte[] key, int koff, int klen, byte[] value, int voff,
int vlen) throws IOException {
if ((koff | klen | (koff + klen) | (key.length - (koff + klen))) < 0) {
throw new IndexOutOfBoundsException(
"Bad key buffer offset-length combination.");
}
if ((voff | vlen | (voff + vlen) | (value.length - (voff + vlen))) < 0) {
throw new IndexOutOfBoundsException(
"Bad value buffer offset-length combination.");
}
try {
DataOutputStream dosKey = prepareAppendKey(klen);
try {
++errorCount;
dosKey.write(key, koff, klen);
--errorCount;
} finally {
dosKey.close();
}
DataOutputStream dosValue = prepareAppendValue(vlen);
try {
++errorCount;
dosValue.write(value, voff, vlen);
--errorCount;
} finally {
dosValue.close();
}
} finally {
state = State.READY;
}
}
/**
* Helper class to register key after close call on key append stream.
*/
private class KeyRegister extends DataOutputStream {
private final int expectedLength;
private boolean closed = false;
public KeyRegister(int len) {
super(currentKeyBufferOS);
if (len >= 0) {
currentKeyBufferOS.reset(len);
} else {
currentKeyBufferOS.reset();
}
expectedLength = len;
}
@Override
public void close() throws IOException {
if (closed == true) {
return;
}
try {
++errorCount;
byte[] key = currentKeyBufferOS.getBuffer();
int len = currentKeyBufferOS.size();
/**
* verify length.
*/
if (expectedLength >= 0 && expectedLength != len) {
throw new IOException("Incorrect key length: expected="
+ expectedLength + " actual=" + len);
}
Utils.writeVInt(blkAppender, len);
blkAppender.write(key, 0, len);
if (tfileIndex.getFirstKey() == null) {
tfileIndex.setFirstKey(key, 0, len);
}
if (tfileMeta.isSorted() && tfileMeta.getRecordCount()>0) {
byte[] lastKey = lastKeyBufferOS.getBuffer();
int lastLen = lastKeyBufferOS.size();
if (tfileMeta.getComparator().compare(key, 0, len, lastKey, 0,
lastLen) < 0) {
throw new IOException("Keys are not added in sorted order");
}
}
BoundedByteArrayOutputStream tmp = currentKeyBufferOS;
currentKeyBufferOS = lastKeyBufferOS;
lastKeyBufferOS = tmp;
--errorCount;
} finally {
closed = true;
state = State.END_KEY;
}
}
}
/**
* Helper class to register value after close call on value append stream.
*/
private class ValueRegister extends DataOutputStream {
private boolean closed = false;
public ValueRegister(OutputStream os) {
super(os);
}
// Avoiding flushing call to down stream.
@Override
public void flush() {
// do nothing
}
@Override
public void close() throws IOException {
if (closed == true) {
return;
}
try {
++errorCount;
super.close();
blkRecordCount++;
// bump up the total record count in the whole file
tfileMeta.incRecordCount();
finishDataBlock(false);
--errorCount;
} finally {
closed = true;
state = State.READY;
}
}
}
/**
* Obtain an output stream for writing a key into TFile. This may only be
* called when there is no active Key appending stream or value appending
* stream.
*
* @param length
* The expected length of the key. If length of the key is not
* known, set length = -1. Otherwise, the application must write
* exactly as many bytes as specified here before calling close on
* the returned output stream.
* @return The key appending output stream.
* @throws IOException
*
*/
public DataOutputStream prepareAppendKey(int length) throws IOException {
if (state != State.READY) {
throw new IllegalStateException("Incorrect state to start a new key: "
+ state.name());
}
initDataBlock();
DataOutputStream ret = new KeyRegister(length);
state = State.IN_KEY;
return ret;
}
/**
* Obtain an output stream for writing a value into TFile. This may only be
* called right after a key appending operation (the key append stream must
* be closed).
*
* @param length
* The expected length of the value. If length of the value is not
* known, set length = -1. Otherwise, the application must write
* exactly as many bytes as specified here before calling close on
* the returned output stream. Advertising the value size up-front
* guarantees that the value is encoded in one chunk, and avoids
* intermediate chunk buffering.
* @throws IOException
*
*/
public DataOutputStream prepareAppendValue(int length) throws IOException {
if (state != State.END_KEY) {
throw new IllegalStateException(
"Incorrect state to start a new value: " + state.name());
}
DataOutputStream ret;
// unknown length
if (length < 0) {
if (valueBuffer == null) {
valueBuffer = new byte[getChunkBufferSize(conf)];
}
ret = new ValueRegister(new ChunkEncoder(blkAppender, valueBuffer));
} else {
ret =
new ValueRegister(new Chunk.SingleChunkEncoder(blkAppender, length));
}
state = State.IN_VALUE;
return ret;
}
/**
* Obtain an output stream for creating a meta block. This function may not
* be called when there is a key append stream or value append stream
* active. No more key-value insertion is allowed after a meta data block
* has been added to TFile.
*
* @param name
* Name of the meta block.
* @param compressName
* Name of the compression algorithm to be used. Must be one of the
* strings returned by
* {@link TFile#getSupportedCompressionAlgorithms()}.
* @return A DataOutputStream that can be used to write Meta Block data.
* Closing the stream would signal the ending of the block.
* @throws IOException
* @throws MetaBlockAlreadyExists
* the Meta Block with the same name already exists.
*/
public DataOutputStream prepareMetaBlock(String name, String compressName)
throws IOException, MetaBlockAlreadyExists {
if (state != State.READY) {
throw new IllegalStateException(
"Incorrect state to start a Meta Block: " + state.name());
}
finishDataBlock(true);
DataOutputStream outputStream =
writerBCF.prepareMetaBlock(name, compressName);
return outputStream;
}
/**
* Obtain an output stream for creating a meta block. This function may not
* be called when there is a key append stream or value append stream
* active. No more key-value insertion is allowed after a meta data block
* has been added to TFile. Data will be compressed using the default
* compressor as defined in Writer's constructor.
*
* @param name
* Name of the meta block.
* @return A DataOutputStream that can be used to write Meta Block data.
* Closing the stream would signal the ending of the block.
* @throws IOException
* @throws MetaBlockAlreadyExists
* the Meta Block with the same name already exists.
*/
public DataOutputStream prepareMetaBlock(String name) throws IOException,
MetaBlockAlreadyExists {
if (state != State.READY) {
throw new IllegalStateException(
"Incorrect state to start a Meta Block: " + state.name());
}
finishDataBlock(true);
return writerBCF.prepareMetaBlock(name);
}
/**
* Check if we need to start a new data block.
*
* @throws IOException
*/
private void initDataBlock() throws IOException {
// for each new block, get a new appender
if (blkAppender == null) {
blkAppender = writerBCF.prepareDataBlock();
}
}
/**
* Close the current data block if necessary.
*
* @param bForceFinish
* Force the closure regardless of the block size.
* @throws IOException
*/
void finishDataBlock(boolean bForceFinish) throws IOException {
if (blkAppender == null) {
return;
}
// exceeded the size limit, do the compression and finish the block
if (bForceFinish || blkAppender.getCompressedSize() >= sizeMinBlock) {
// keep tracks of the last key of each data block, no padding
// for now
TFileIndexEntry keyLast =
new TFileIndexEntry(lastKeyBufferOS.getBuffer(), 0, lastKeyBufferOS
.size(), blkRecordCount);
tfileIndex.addEntry(keyLast);
// close the appender
blkAppender.close();
blkAppender = null;
blkRecordCount = 0;
}
}
}
/**
* TFile Reader. Users may only read TFiles by creating TFile.Reader.Scanner.
* objects. A scanner may scan the whole TFile ({@link Reader#createScanner()}
* ) , a portion of TFile based on byte offsets (
* {@link Reader#createScannerByByteRange(long, long)}), or a porti