/**
 * 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.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
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.