HADOOP-3315. Add a new, binary file foramt, TFile. Contributed by Hong Tang.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@787913 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Christopher Douglas 2009-06-24 05:48:25 +00:00
parent 84541f6f91
commit 8246aa28ff
37 changed files with 9945 additions and 0 deletions

View File

@ -155,6 +155,8 @@ Trunk (unreleased changes)
HADOOP-5897. Add name-node metrics to capture java heap usage.
(Suresh Srinivas via shv)
HADOOP-3315. Add a new, binary file foramt, TFile. (Hong Tang via cdouglas)
IMPROVEMENTS
HADOOP-4565. Added CombineFileInputFormat to use data locality information

View File

@ -507,6 +507,8 @@
<sysproperty key="java.library.path"
value="${build.native}/lib:${lib.dir}/native/${build.platform}"/>
<sysproperty key="install.c++.examples" value="${install.c++.examples}"/>
<sysproperty key="io.compression.codec.lzo.class"
value="${io.compression.codec.lzo.class}"/>
<!-- set compile.c++ in the child jvm only if it is set -->
<syspropertyset dynamic="no">
<propertyref name="compile.c++"/>

View File

@ -0,0 +1,979 @@
/**
* 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.Closeable;
import java.io.DataInput;
import java.io.DataInputStream;
import java.io.DataOutput;
import java.io.DataOutputStream;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Map;
import java.util.TreeMap;
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.compress.Compressor;
import org.apache.hadoop.io.compress.Decompressor;
import org.apache.hadoop.io.file.tfile.CompareUtils.Scalar;
import org.apache.hadoop.io.file.tfile.CompareUtils.ScalarComparator;
import org.apache.hadoop.io.file.tfile.CompareUtils.ScalarLong;
import org.apache.hadoop.io.file.tfile.Compression.Algorithm;
import org.apache.hadoop.io.file.tfile.Utils.Version;
/**
* Block Compressed file, the underlying physical storage layer for TFile.
* BCFile provides the basic block level compression for the data block and meta
* blocks. It is separated from TFile as it may be used for other
* block-compressed file implementation.
*/
final class BCFile {
// the current version of BCFile impl, increment them (major or minor) made
// enough changes
static final Version API_VERSION = new Version((short) 1, (short) 0);
static final Log LOG = LogFactory.getLog(BCFile.class);
/**
* Prevent the instantiation of BCFile objects.
*/
private BCFile() {
// nothing
}
/**
* BCFile writer, the entry point for creating a new BCFile.
*/
static public class Writer implements Closeable {
private final FSDataOutputStream out;
private final Configuration conf;
// the single meta block containing index of compressed data blocks
final DataIndex dataIndex;
// index for meta blocks
final MetaIndex metaIndex;
boolean blkInProgress = false;
private boolean metaBlkSeen = false;
private boolean closed = false;
long errorCount = 0;
// reusable buffers.
private BytesWritable fsOutputBuffer;
/**
* Call-back interface to register a block after a block is closed.
*/
private static interface BlockRegister {
/**
* Register a block that is fully closed.
*
* @param raw
* The size of block in terms of uncompressed bytes.
* @param offsetStart
* The start offset of the block.
* @param offsetEnd
* One byte after the end of the block. Compressed block size is
* offsetEnd - offsetStart.
*/
public void register(long raw, long offsetStart, long offsetEnd);
}
/**
* Intermediate class that maintain the state of a Writable Compression
* Block.
*/
private static final class WBlockState {
private final Algorithm compressAlgo;
private Compressor compressor; // !null only if using native
// Hadoop compression
private final FSDataOutputStream fsOut;
private final long posStart;
private final SimpleBufferedOutputStream fsBufferedOutput;
private OutputStream out;
/**
* @param compressionAlgo
* The compression algorithm to be used to for compression.
* @throws IOException
*/
public WBlockState(Algorithm compressionAlgo, FSDataOutputStream fsOut,
BytesWritable fsOutputBuffer, Configuration conf) throws IOException {
this.compressAlgo = compressionAlgo;
this.fsOut = fsOut;
this.posStart = fsOut.getPos();
fsOutputBuffer.setCapacity(TFile.getFSOutputBufferSize(conf));
this.fsBufferedOutput =
new SimpleBufferedOutputStream(this.fsOut, fsOutputBuffer.get());
this.compressor = compressAlgo.getCompressor();
try {
this.out =
compressionAlgo.createCompressionStream(fsBufferedOutput,
compressor, 0);
} catch (IOException e) {
compressAlgo.returnCompressor(compressor);
throw e;
}
}
/**
* Get the output stream for BlockAppender's consumption.
*
* @return the output stream suitable for writing block data.
*/
OutputStream getOutputStream() {
return out;
}
/**
* Get the current position in file.
*
* @return The current byte offset in underlying file.
* @throws IOException
*/
long getCurrentPos() throws IOException {
return fsOut.getPos() + fsBufferedOutput.size();
}
long getStartPos() {
return posStart;
}
/**
* Current size of compressed data.
*
* @return
* @throws IOException
*/
long getCompressedSize() throws IOException {
long ret = getCurrentPos() - posStart;
return ret;
}
/**
* Finishing up the current block.
*/
public void finish() throws IOException {
try {
if (out != null) {
out.flush();
out = null;
}
} finally {
compressAlgo.returnCompressor(compressor);
compressor = null;
}
}
}
/**
* Access point to stuff data into a block.
*
* TODO: Change DataOutputStream to something else that tracks the size as
* long instead of int. Currently, we will wrap around if the row block size
* is greater than 4GB.
*/
public class BlockAppender extends DataOutputStream {
private final BlockRegister blockRegister;
private final WBlockState wBlkState;
@SuppressWarnings("hiding")
private boolean closed = false;
/**
* Constructor
*
* @param register
* the block register, which is called when the block is closed.
* @param wbs
* The writable compression block state.
*/
BlockAppender(BlockRegister register, WBlockState wbs) {
super(wbs.getOutputStream());
this.blockRegister = register;
this.wBlkState = wbs;
}
/**
* Get the raw size of the block.
*
* @return the number of uncompressed bytes written through the
* BlockAppender so far.
* @throws IOException
*/
public long getRawSize() throws IOException {
/**
* Expecting the size() of a block not exceeding 4GB. Assuming the
* size() will wrap to negative integer if it exceeds 2GB.
*/
return size() & 0x00000000ffffffffL;
}
/**
* Get the compressed size of the block in progress.
*
* @return the number of compressed bytes written to the underlying FS
* file. The size may be smaller than actual need to compress the
* all data written due to internal buffering inside the
* compressor.
* @throws IOException
*/
public long getCompressedSize() throws IOException {
return wBlkState.getCompressedSize();
}
@Override
public void flush() {
// The down stream is a special kind of stream that finishes a
// compression block upon flush. So we disable flush() here.
}
/**
* Signaling the end of write to the block. The block register will be
* called for registering the finished block.
*/
@Override
public void close() throws IOException {
if (closed == true) {
return;
}
try {
++errorCount;
wBlkState.finish();
blockRegister.register(getRawSize(), wBlkState.getStartPos(),
wBlkState.getCurrentPos());
--errorCount;
} finally {
closed = true;
blkInProgress = false;
}
}
}
/**
* Constructor
*
* @param fout
* FS output stream.
* @param compressionName
* Name of the compression algorithm, which will be used for all
* data blocks.
* @throws IOException
* @see Compression#getSupportedAlgorithms
*/
public Writer(FSDataOutputStream fout, String compressionName,
Configuration conf) throws IOException {
if (fout.getPos() != 0) {
throw new IOException("Output file not at zero offset.");
}
this.out = fout;
this.conf = conf;
dataIndex = new DataIndex(compressionName);
metaIndex = new MetaIndex();
fsOutputBuffer = new BytesWritable();
Magic.write(fout);
}
/**
* Close the BCFile Writer. Attempting to use the Writer after calling
* <code>close</code> is not allowed and may lead to undetermined results.
*/
public void close() throws IOException {
if (closed == true) {
return;
}
try {
if (errorCount == 0) {
if (blkInProgress == true) {
throw new IllegalStateException(
"Close() called with active block appender.");
}
// add metaBCFileIndex to metaIndex as the last meta block
BlockAppender appender =
prepareMetaBlock(DataIndex.BLOCK_NAME,
getDefaultCompressionAlgorithm());
try {
dataIndex.write(appender);
} finally {
appender.close();
}
long offsetIndexMeta = out.getPos();
metaIndex.write(out);
// Meta Index and the trailing section are written out directly.
out.writeLong(offsetIndexMeta);
API_VERSION.write(out);
Magic.write(out);
out.flush();
}
} finally {
closed = true;
}
}
private Algorithm getDefaultCompressionAlgorithm() {
return dataIndex.getDefaultCompressionAlgorithm();
}
private BlockAppender prepareMetaBlock(String name, Algorithm compressAlgo)
throws IOException, MetaBlockAlreadyExists {
if (blkInProgress == true) {
throw new IllegalStateException(
"Cannot create Meta Block until previous block is closed.");
}
if (metaIndex.getMetaByName(name) != null) {
throw new MetaBlockAlreadyExists("name=" + name);
}
MetaBlockRegister mbr = new MetaBlockRegister(name, compressAlgo);
WBlockState wbs =
new WBlockState(compressAlgo, out, fsOutputBuffer, conf);
BlockAppender ba = new BlockAppender(mbr, wbs);
blkInProgress = true;
metaBlkSeen = true;
return ba;
}
/**
* Create a Meta Block and obtain an output stream for adding data into the
* block. There can only be one BlockAppender stream active at any time.
* Regular Blocks may not be created after the first Meta Blocks. The caller
* must call BlockAppender.close() to conclude the block creation.
*
* @param name
* The name of the Meta Block. The name must not conflict with
* existing Meta Blocks.
* @param compressionName
* The name of the compression algorithm to be used.
* @return The BlockAppender stream
* @throws IOException
* @throws MetaBlockAlreadyExists
* If the meta block with the name already exists.
*/
public BlockAppender prepareMetaBlock(String name, String compressionName)
throws IOException, MetaBlockAlreadyExists {
return prepareMetaBlock(name, Compression
.getCompressionAlgorithmByName(compressionName));
}
/**
* Create a Meta Block and obtain an output stream for adding data into the
* block. The Meta Block will be compressed with the same compression
* algorithm as data blocks. There can only be one BlockAppender stream
* active at any time. Regular Blocks may not be created after the first
* Meta Blocks. The caller must call BlockAppender.close() to conclude the
* block creation.
*
* @param name
* The name of the Meta Block. The name must not conflict with
* existing Meta Blocks.
* @return The BlockAppender stream
* @throws MetaBlockAlreadyExists
* If the meta block with the name already exists.
* @throws IOException
*/
public BlockAppender prepareMetaBlock(String name) throws IOException,
MetaBlockAlreadyExists {
return prepareMetaBlock(name, getDefaultCompressionAlgorithm());
}
/**
* Create a Data Block and obtain an output stream for adding data into the
* block. There can only be one BlockAppender stream active at any time.
* Data Blocks may not be created after the first Meta Blocks. The caller
* must call BlockAppender.close() to conclude the block creation.
*
* @return The BlockAppender stream
* @throws IOException
*/
public BlockAppender prepareDataBlock() throws IOException {
if (blkInProgress == true) {
throw new IllegalStateException(
"Cannot create Data Block until previous block is closed.");
}
if (metaBlkSeen == true) {
throw new IllegalStateException(
"Cannot create Data Block after Meta Blocks.");
}
DataBlockRegister dbr = new DataBlockRegister();
WBlockState wbs =
new WBlockState(getDefaultCompressionAlgorithm(), out,
fsOutputBuffer, conf);
BlockAppender ba = new BlockAppender(dbr, wbs);
blkInProgress = true;
return ba;
}
/**
* Callback to make sure a meta block is added to the internal list when its
* stream is closed.
*/
private class MetaBlockRegister implements BlockRegister {
private final String name;
private final Algorithm compressAlgo;
MetaBlockRegister(String name, Algorithm compressAlgo) {
this.name = name;
this.compressAlgo = compressAlgo;
}
public void register(long raw, long begin, long end) {
metaIndex.addEntry(new MetaIndexEntry(name, compressAlgo,
new BlockRegion(begin, end - begin, raw)));
}
}
/**
* Callback to make sure a data block is added to the internal list when
* it's being closed.
*
*/
private class DataBlockRegister implements BlockRegister {
DataBlockRegister() {
// do nothing
}
public void register(long raw, long begin, long end) {
dataIndex.addBlockRegion(new BlockRegion(begin, end - begin, raw));
}
}
}
/**
* BCFile Reader, interface to read the file's data and meta blocks.
*/
static public class Reader implements Closeable {
private final FSDataInputStream in;
private final Configuration conf;
final DataIndex dataIndex;
// Index for meta blocks
final MetaIndex metaIndex;
final Version version;
/**
* Intermediate class that maintain the state of a Readable Compression
* Block.
*/
static private final class RBlockState {
private final Algorithm compressAlgo;
private Decompressor decompressor;
private final BlockRegion region;
private final InputStream in;
public RBlockState(Algorithm compressionAlgo, FSDataInputStream fsin,
BlockRegion region, Configuration conf) throws IOException {
this.compressAlgo = compressionAlgo;
this.region = region;
this.decompressor = compressionAlgo.getDecompressor();
try {
this.in =
compressAlgo
.createDecompressionStream(new BoundedRangeFileInputStream(
fsin, this.region.getOffset(), this.region
.getCompressedSize()), decompressor, TFile
.getFSInputBufferSize(conf));
} catch (IOException e) {
compressAlgo.returnDecompressor(decompressor);
throw e;
}
}
/**
* Get the output stream for BlockAppender's consumption.
*
* @return the output stream suitable for writing block data.
*/
public InputStream getInputStream() {
return in;
}
public String getCompressionName() {
return compressAlgo.getName();
}
public BlockRegion getBlockRegion() {
return region;
}
public void finish() throws IOException {
try {
in.close();
} finally {
compressAlgo.returnDecompressor(decompressor);
decompressor = null;
}
}
}
/**
* Access point to read a block.
*/
public static class BlockReader extends DataInputStream {
private final RBlockState rBlkState;
private boolean closed = false;
BlockReader(RBlockState rbs) {
super(rbs.getInputStream());
rBlkState = rbs;
}
/**
* Finishing reading the block. Release all resources.
*/
@Override
public void close() throws IOException {
if (closed == true) {
return;
}
try {
// Do not set rBlkState to null. People may access stats after calling
// close().
rBlkState.finish();
} finally {
closed = true;
}
}
/**
* Get the name of the compression algorithm used to compress the block.
*
* @return name of the compression algorithm.
*/
public String getCompressionName() {
return rBlkState.getCompressionName();
}
/**
* Get the uncompressed size of the block.
*
* @return uncompressed size of the block.
*/
public long getRawSize() {
return rBlkState.getBlockRegion().getRawSize();
}
/**
* Get the compressed size of the block.
*
* @return compressed size of the block.
*/
public long getCompressedSize() {
return rBlkState.getBlockRegion().getCompressedSize();
}
/**
* Get the starting position of the block in the file.
*
* @return the starting position of the block in the file.
*/
public long getStartPos() {
return rBlkState.getBlockRegion().getOffset();
}
}
/**
* Constructor
*
* @param fin
* FS input stream.
* @param fileLength
* Length of the corresponding file
* @throws IOException
*/
public Reader(FSDataInputStream fin, long fileLength, Configuration conf)
throws IOException {
this.in = fin;
this.conf = conf;
// move the cursor to the beginning of the tail, containing: offset to the
// meta block index, version and magic
fin.seek(fileLength - Magic.size() - Version.size() - Long.SIZE
/ Byte.SIZE);
long offsetIndexMeta = fin.readLong();
version = new Version(fin);
Magic.readAndVerify(fin);
if (!version.compatibleWith(BCFile.API_VERSION)) {
throw new RuntimeException("Incompatible BCFile fileBCFileVersion.");
}
// read meta index
fin.seek(offsetIndexMeta);
metaIndex = new MetaIndex(fin);
// read data:BCFile.index, the data block index
BlockReader blockR = getMetaBlock(DataIndex.BLOCK_NAME);
try {
dataIndex = new DataIndex(blockR);
} finally {
blockR.close();
}
}
/**
* Get the name of the default compression algorithm.
*
* @return the name of the default compression algorithm.
*/
public String getDefaultCompressionName() {
return dataIndex.getDefaultCompressionAlgorithm().getName();
}
/**
* Get version of BCFile file being read.
*
* @return version of BCFile file being read.
*/
public Version getBCFileVersion() {
return version;
}
/**
* Get version of BCFile API.
*
* @return version of BCFile API.
*/
public Version getAPIVersion() {
return API_VERSION;
}
/**
* Finishing reading the BCFile. Release all resources.
*/
public void close() {
// nothing to be done now
}
/**
* Get the number of data blocks.
*
* @return the number of data blocks.
*/
public int getBlockCount() {
return dataIndex.getBlockRegionList().size();
}
/**
* Stream access to a Meta Block.
*
* @param name
* meta block name
* @return BlockReader input stream for reading the meta block.
* @throws IOException
* @throws MetaBlockDoesNotExist
* The Meta Block with the given name does not exist.
*/
public BlockReader getMetaBlock(String name) throws IOException,
MetaBlockDoesNotExist {
MetaIndexEntry imeBCIndex = metaIndex.getMetaByName(name);
if (imeBCIndex == null) {
throw new MetaBlockDoesNotExist("name=" + name);
}
BlockRegion region = imeBCIndex.getRegion();
return createReader(imeBCIndex.getCompressionAlgorithm(), region);
}
/**
* Stream access to a Data Block.
*
* @param blockIndex
* 0-based data block index.
* @return BlockReader input stream for reading the data block.
* @throws IOException
*/
public BlockReader getDataBlock(int blockIndex) throws IOException {
if (blockIndex < 0 || blockIndex >= getBlockCount()) {
throw new IndexOutOfBoundsException(String.format(
"blockIndex=%d, numBlocks=%d", blockIndex, getBlockCount()));
}
BlockRegion region = dataIndex.getBlockRegionList().get(blockIndex);
return createReader(dataIndex.getDefaultCompressionAlgorithm(), region);
}
private BlockReader createReader(Algorithm compressAlgo, BlockRegion region)
throws IOException {
RBlockState rbs = new RBlockState(compressAlgo, in, region, conf);
return new BlockReader(rbs);
}
/**
* Find the smallest Block index whose starting offset is greater than or
* equal to the specified offset.
*
* @param offset
* User-specific offset.
* @return the index to the data Block if such block exists; or -1
* otherwise.
*/
public int getBlockIndexNear(long offset) {
ArrayList<BlockRegion> list = dataIndex.getBlockRegionList();
int idx =
Utils
.lowerBound(list, new ScalarLong(offset), new ScalarComparator());
if (idx == list.size()) {
return -1;
}
return idx;
}
}
/**
* Index for all Meta blocks.
*/
static class MetaIndex {
// use a tree map, for getting a meta block entry by name
final Map<String, MetaIndexEntry> index;
// for write
public MetaIndex() {
index = new TreeMap<String, MetaIndexEntry>();
}
// for read, construct the map from the file
public MetaIndex(DataInput in) throws IOException {
int count = Utils.readVInt(in);
index = new TreeMap<String, MetaIndexEntry>();
for (int nx = 0; nx < count; nx++) {
MetaIndexEntry indexEntry = new MetaIndexEntry(in);
index.put(indexEntry.getMetaName(), indexEntry);
}
}
public void addEntry(MetaIndexEntry indexEntry) {
index.put(indexEntry.getMetaName(), indexEntry);
}
public MetaIndexEntry getMetaByName(String name) {
return index.get(name);
}
public void write(DataOutput out) throws IOException {
Utils.writeVInt(out, index.size());
for (MetaIndexEntry indexEntry : index.values()) {
indexEntry.write(out);
}
}
}
/**
* An entry describes a meta block in the MetaIndex.
*/
static final class MetaIndexEntry {
private final String metaName;
private final Algorithm compressionAlgorithm;
private final static String defaultPrefix = "data:";
private final BlockRegion region;
public MetaIndexEntry(DataInput in) throws IOException {
String fullMetaName = Utils.readString(in);
if (fullMetaName.startsWith(defaultPrefix)) {
metaName =
fullMetaName.substring(defaultPrefix.length(), fullMetaName
.length());
} else {
throw new IOException("Corrupted Meta region Index");
}
compressionAlgorithm =
Compression.getCompressionAlgorithmByName(Utils.readString(in));
region = new BlockRegion(in);
}
public MetaIndexEntry(String metaName, Algorithm compressionAlgorithm,
BlockRegion region) {
this.metaName = metaName;
this.compressionAlgorithm = compressionAlgorithm;
this.region = region;
}
public String getMetaName() {
return metaName;
}
public Algorithm getCompressionAlgorithm() {
return compressionAlgorithm;
}
public BlockRegion getRegion() {
return region;
}
public void write(DataOutput out) throws IOException {
Utils.writeString(out, defaultPrefix + metaName);
Utils.writeString(out, compressionAlgorithm.getName());
region.write(out);
}
}
/**
* Index of all compressed data blocks.
*/
static class DataIndex {
final static String BLOCK_NAME = "BCFile.index";
private final Algorithm defaultCompressionAlgorithm;
// for data blocks, each entry specifies a block's offset, compressed size
// and raw size
private final ArrayList<BlockRegion> listRegions;
// for read, deserialized from a file
public DataIndex(DataInput in) throws IOException {
defaultCompressionAlgorithm =
Compression.getCompressionAlgorithmByName(Utils.readString(in));
int n = Utils.readVInt(in);
listRegions = new ArrayList<BlockRegion>(n);
for (int i = 0; i < n; i++) {
BlockRegion region = new BlockRegion(in);
listRegions.add(region);
}
}
// for write
public DataIndex(String defaultCompressionAlgorithmName) {
this.defaultCompressionAlgorithm =
Compression
.getCompressionAlgorithmByName(defaultCompressionAlgorithmName);
listRegions = new ArrayList<BlockRegion>();
}
public Algorithm getDefaultCompressionAlgorithm() {
return defaultCompressionAlgorithm;
}
public ArrayList<BlockRegion> getBlockRegionList() {
return listRegions;
}
public void addBlockRegion(BlockRegion region) {
listRegions.add(region);
}
public void write(DataOutput out) throws IOException {
Utils.writeString(out, defaultCompressionAlgorithm.getName());
Utils.writeVInt(out, listRegions.size());
for (BlockRegion region : listRegions) {
region.write(out);
}
}
}
/**
* Magic number uniquely identifying a BCFile in the header/footer.
*/
static final class Magic {
private final static byte[] AB_MAGIC_BCFILE =
{
// ... total of 16 bytes
(byte) 0xd1, (byte) 0x11, (byte) 0xd3, (byte) 0x68, (byte) 0x91,
(byte) 0xb5, (byte) 0xd7, (byte) 0xb6, (byte) 0x39, (byte) 0xdf,
(byte) 0x41, (byte) 0x40, (byte) 0x92, (byte) 0xba, (byte) 0xe1,
(byte) 0x50 };
public static void readAndVerify(DataInput in) throws IOException {
byte[] abMagic = new byte[size()];
in.readFully(abMagic);
// check against AB_MAGIC_BCFILE, if not matching, throw an
// Exception
if (!Arrays.equals(abMagic, AB_MAGIC_BCFILE)) {
throw new IOException("Not a valid BCFile.");
}
}
public static void write(DataOutput out) throws IOException {
out.write(AB_MAGIC_BCFILE);
}
public static int size() {
return AB_MAGIC_BCFILE.length;
}
}
/**
* Block region.
*/
static final class BlockRegion implements Scalar {
private final long offset;
private final long compressedSize;
private final long rawSize;
public BlockRegion(DataInput in) throws IOException {
offset = Utils.readVLong(in);
compressedSize = Utils.readVLong(in);
rawSize = Utils.readVLong(in);
}
public BlockRegion(long offset, long compressedSize, long rawSize) {
this.offset = offset;
this.compressedSize = compressedSize;
this.rawSize = rawSize;
}
public void write(DataOutput out) throws IOException {
Utils.writeVLong(out, offset);
Utils.writeVLong(out, compressedSize);
Utils.writeVLong(out, rawSize);
}
public long getOffset() {
return offset;
}
public long getCompressedSize() {
return compressedSize;
}
public long getRawSize() {
return rawSize;
}
@Override
public long magnitude() {
return offset;
}
}
}

View File

@ -0,0 +1,96 @@
/**
* 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.EOFException;
import java.io.IOException;
import java.io.OutputStream;
/**
* A byte array backed output stream with a limit. The limit should be smaller
* than the buffer capacity. The object can be reused through <code>reset</code>
* API and choose different limits in each round.
*/
class BoundedByteArrayOutputStream extends OutputStream {
private final byte[] buffer;
private int limit;
private int count;
public BoundedByteArrayOutputStream(int capacity) {
this(capacity, capacity);
}
public BoundedByteArrayOutputStream(int capacity, int limit) {
if ((capacity < limit) || (capacity | limit) < 0) {
throw new IllegalArgumentException("Invalid capacity/limit");
}
this.buffer = new byte[capacity];
this.limit = limit;
this.count = 0;
}
@Override
public void write(int b) throws IOException {
if (count >= limit) {
throw new EOFException("Reaching the limit of the buffer.");
}
buffer[count++] = (byte) b;
}
@Override
public void write(byte b[], int off, int len) throws IOException {
if ((off < 0) || (off > b.length) || (len < 0) || ((off + len) > b.length)
|| ((off + len) < 0)) {
throw new IndexOutOfBoundsException();
} else if (len == 0) {
return;
}
if (count + len > limit) {
throw new EOFException("Reach the limit of the buffer");
}
System.arraycopy(b, off, buffer, count, len);
count += len;
}
public void reset(int newlim) {
if (newlim > buffer.length) {
throw new IndexOutOfBoundsException("Limit exceeds buffer size");
}
this.limit = newlim;
this.count = 0;
}
public void reset() {
this.limit = buffer.length;
this.count = 0;
}
public int getLimit() {
return limit;
}
public byte[] getBuffer() {
return buffer;
}
public int size() {
return count;
}
}

View File

@ -0,0 +1,141 @@
/**
* 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.IOException;
import java.io.InputStream;
import org.apache.hadoop.fs.FSDataInputStream;
/**
* BoundedRangeFIleInputStream abstracts a contiguous region of a Hadoop
* FSDataInputStream as a regular input stream. One can create multiple
* BoundedRangeFileInputStream on top of the same FSDataInputStream and they
* would not interfere with each other.
*/
class BoundedRangeFileInputStream extends InputStream {
private FSDataInputStream in;
private long pos;
private long end;
private long mark;
private final byte[] oneByte = new byte[1];
/**
* Constructor
*
* @param in
* The FSDataInputStream we connect to.
* @param offset
* Begining offset of the region.
* @param length
* Length of the region.
*
* The actual length of the region may be smaller if (off_begin +
* length) goes beyond the end of FS input stream.
*/
public BoundedRangeFileInputStream(FSDataInputStream in, long offset,
long length) {
if (offset < 0 || length < 0) {
throw new IndexOutOfBoundsException("Invalid offset/length: " + offset
+ "/" + length);
}
this.in = in;
this.pos = offset;
this.end = offset + length;
this.mark = -1;
}
@Override
public int available() throws IOException {
int avail = in.available();
if (pos + avail > end) {
avail = (int) (end - pos);
}
return avail;
}
@Override
public int read() throws IOException {
int ret = read(oneByte);
if (ret == 1) return oneByte[0] & 0xff;
return -1;
}
@Override
public int read(byte[] b) throws IOException {
return read(b, 0, b.length);
}
@Override
public int read(byte[] b, int off, int len) throws IOException {
if ((off | len | (off + len) | (b.length - (off + len))) < 0) {
throw new IndexOutOfBoundsException();
}
int n = (int) Math.min(Integer.MAX_VALUE, Math.min(len, (end - pos)));
if (n == 0) return -1;
int ret = 0;
synchronized (in) {
in.seek(pos);
ret = in.read(b, off, n);
}
if (ret < 0) {
end = pos;
return -1;
}
pos += ret;
return ret;
}
@Override
/*
* We may skip beyond the end of the file.
*/
public long skip(long n) throws IOException {
long len = Math.min(n, end - pos);
pos += len;
return len;
}
@Override
public void mark(int readlimit) {
mark = pos;
}
@Override
public void reset() throws IOException {
if (mark < 0) throw new IOException("Resetting to invalid mark");
pos = mark;
}
@Override
public boolean markSupported() {
return true;
}
@Override
public void close() {
// Invalidate the state of the stream.
in = null;
pos = end;
mark = -1;
}
}

View File

@ -0,0 +1,92 @@
/**
* 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 org.apache.hadoop.io.BytesWritable;
/**
* Adaptor class to wrap byte-array backed objects (including java byte array)
* as RawComparable objects.
*/
public final class ByteArray implements RawComparable {
private final byte[] buffer;
private final int offset;
private final int len;
/**
* Constructing a ByteArray from a {@link BytesWritable}.
*
* @param other
*/
public ByteArray(BytesWritable other) {
this(other.get(), 0, other.getSize());
}
/**
* Wrap a whole byte array as a RawComparable.
*
* @param buffer
* the byte array buffer.
*/
public ByteArray(byte[] buffer) {
this(buffer, 0, buffer.length);
}
/**
* Wrap a partial byte array as a RawComparable.
*
* @param buffer
* the byte array buffer.
* @param offset
* the starting offset
* @param len
* the length of the consecutive bytes to be wrapped.
*/
public ByteArray(byte[] buffer, int offset, int len) {
if ((offset | len | (buffer.length - offset - len)) < 0) {
throw new IndexOutOfBoundsException();
}
this.buffer = buffer;
this.offset = offset;
this.len = len;
}
/**
* @return the underlying buffer.
*/
@Override
public byte[] buffer() {
return buffer;
}
/**
* @return the offset in the buffer.
*/
@Override
public int offset() {
return offset;
}
/**
* @return the size of the byte array.
*/
@Override
public int size() {
return len;
}
}

View File

@ -0,0 +1,429 @@
/**
* 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.DataInputStream;
import java.io.DataOutputStream;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
/**
* Several related classes to support chunk-encoded sub-streams on top of a
* regular stream.
*/
final class Chunk {
/**
* Prevent the instantiation of class.
*/
private Chunk() {
// nothing
}
/**
* Decoding a chain of chunks encoded through ChunkEncoder or
* SingleChunkEncoder.
*/
static public class ChunkDecoder extends InputStream {
private DataInputStream in = null;
private boolean lastChunk;
private int remain = 0;
private boolean closed;
public ChunkDecoder() {
lastChunk = true;
closed = true;
}
public void reset(DataInputStream downStream) {
// no need to wind forward the old input.
in = downStream;
lastChunk = false;
remain = 0;
closed = false;
}
/**
* Constructor
*
* @param in
* The source input stream which contains chunk-encoded data
* stream.
*/
public ChunkDecoder(DataInputStream in) {
this.in = in;
lastChunk = false;
closed = false;
}
/**
* Have we reached the last chunk.
*
* @return true if we have reached the last chunk.
* @throws java.io.IOException
*/
public boolean isLastChunk() throws IOException {
checkEOF();
return lastChunk;
}
/**
* How many bytes remain in the current chunk?
*
* @return remaining bytes left in the current chunk.
* @throws java.io.IOException
*/
public int getRemain() throws IOException {
checkEOF();
return remain;
}
/**
* Reading the length of next chunk.
*
* @throws java.io.IOException
* when no more data is available.
*/
private void readLength() throws IOException {
remain = Utils.readVInt(in);
if (remain >= 0) {
lastChunk = true;
} else {
remain = -remain;
}
}
/**
* Check whether we reach the end of the stream.
*
* @return false if the chunk encoded stream has more data to read (in which
* case available() will be greater than 0); true otherwise.
* @throws java.io.IOException
* on I/O errors.
*/
private boolean checkEOF() throws IOException {
if (isClosed()) return true;
while (true) {
if (remain > 0) return false;
if (lastChunk) return true;
readLength();
}
}
@Override
/*
* This method never blocks the caller. Returning 0 does not mean we reach
* the end of the stream.
*/
public int available() {
return remain;
}
@Override
public int read() throws IOException {
if (checkEOF()) return -1;
int ret = in.read();
if (ret < 0) throw new IOException("Corrupted chunk encoding stream");
--remain;
return ret;
}
@Override
public int read(byte[] b) throws IOException {
return read(b, 0, b.length);
}
@Override
public int read(byte[] b, int off, int len) throws IOException {
if ((off | len | (off + len) | (b.length - (off + len))) < 0) {
throw new IndexOutOfBoundsException();
}
if (!checkEOF()) {
int n = Math.min(remain, len);
int ret = in.read(b, off, n);
if (ret < 0) throw new IOException("Corrupted chunk encoding stream");
remain -= ret;
return ret;
}
return -1;
}
@Override
public long skip(long n) throws IOException {
if (!checkEOF()) {
long ret = in.skip(Math.min(remain, n));
remain -= ret;
return ret;
}
return 0;
}
@Override
public boolean markSupported() {
return false;
}
public boolean isClosed() {
return closed;
}
@Override
public void close() throws IOException {
if (closed == false) {
try {
while (!checkEOF()) {
skip(Integer.MAX_VALUE);
}
} finally {
closed = true;
}
}
}
}
/**
* Chunk Encoder. Encoding the output data into a chain of chunks in the
* following sequences: -len1, byte[len1], -len2, byte[len2], ... len_n,
* byte[len_n]. Where len1, len2, ..., len_n are the lengths of the data
* chunks. Non-terminal chunks have their lengths negated. Non-terminal chunks
* cannot have length 0. All lengths are in the range of 0 to
* Integer.MAX_VALUE and are encoded in Utils.VInt format.
*/
static public class ChunkEncoder extends OutputStream {
/**
* The data output stream it connects to.
*/
private DataOutputStream out;
/**
* The internal buffer that is only used when we do not know the advertised
* size.
*/
private byte buf[];
/**
* The number of valid bytes in the buffer. This value is always in the
* range <tt>0</tt> through <tt>buf.length</tt>; elements <tt>buf[0]</tt>
* through <tt>buf[count-1]</tt> contain valid byte data.
*/
private int count;
/**
* Constructor.
*
* @param out
* the underlying output stream.
* @param buf
* user-supplied buffer. The buffer would be used exclusively by
* the ChunkEncoder during its life cycle.
*/
public ChunkEncoder(DataOutputStream out, byte[] buf) {
this.out = out;
this.buf = buf;
this.count = 0;
}
/**
* Write out a chunk.
*
* @param chunk
* The chunk buffer.
* @param offset
* Offset to chunk buffer for the beginning of chunk.
* @param len
* @param last
* Is this the last call to flushBuffer?
*/
private void writeChunk(byte[] chunk, int offset, int len, boolean last)
throws IOException {
if (last) { // always write out the length for the last chunk.
Utils.writeVInt(out, len);
if (len > 0) {
out.write(chunk, offset, len);
}
} else {
if (len > 0) {
Utils.writeVInt(out, -len);
out.write(chunk, offset, len);
}
}
}
/**
* Write out a chunk that is a concatenation of the internal buffer plus
* user supplied data. This will never be the last block.
*
* @param data
* User supplied data buffer.
* @param offset
* Offset to user data buffer.
* @param len
* User data buffer size.
*/
private void writeBufData(byte[] data, int offset, int len)
throws IOException {
if (count + len > 0) {
Utils.writeVInt(out, -(count + len));
out.write(buf, 0, count);
count = 0;
out.write(data, offset, len);
}
}
/**
* Flush the internal buffer.
*
* Is this the last call to flushBuffer?
*
* @throws java.io.IOException
*/
private void flushBuffer() throws IOException {
if (count > 0) {
writeChunk(buf, 0, count, false);
count = 0;
}
}
@Override
public void write(int b) throws IOException {
if (count >= buf.length) {
flushBuffer();
}
buf[count++] = (byte) b;
}
@Override
public void write(byte b[]) throws IOException {
write(b, 0, b.length);
}
@Override
public void write(byte b[], int off, int len) throws IOException {
if ((len + count) >= buf.length) {
/*
* If the input data do not fit in buffer, flush the output buffer and
* then write the data directly. In this way buffered streams will
* cascade harmlessly.
*/
writeBufData(b, off, len);
return;
}
System.arraycopy(b, off, buf, count, len);
count += len;
}
@Override
public void flush() throws IOException {
flushBuffer();
out.flush();
}
@Override
public void close() throws IOException {
if (buf != null) {
try {
writeChunk(buf, 0, count, true);
} finally {
buf = null;
out = null;
}
}
}
}
/**
* Encode the whole stream as a single chunk. Expecting to know the size of
* the chunk up-front.
*/
static public class SingleChunkEncoder extends OutputStream {
/**
* The data output stream it connects to.
*/
private final DataOutputStream out;
/**
* The remaining bytes to be written.
*/
private int remain;
private boolean closed = false;
/**
* Constructor.
*
* @param out
* the underlying output stream.
* @param size
* The total # of bytes to be written as a single chunk.
* @throws java.io.IOException
* if an I/O error occurs.
*/
public SingleChunkEncoder(DataOutputStream out, int size)
throws IOException {
this.out = out;
this.remain = size;
Utils.writeVInt(out, size);
}
@Override
public void write(int b) throws IOException {
if (remain > 0) {
out.write(b);
--remain;
} else {
throw new IOException("Writing more bytes than advertised size.");
}
}
@Override
public void write(byte b[]) throws IOException {
write(b, 0, b.length);
}
@Override
public void write(byte b[], int off, int len) throws IOException {
if (remain >= len) {
out.write(b, off, len);
remain -= len;
} else {
throw new IOException("Writing more bytes than advertised size.");
}
}
@Override
public void flush() throws IOException {
out.flush();
}
@Override
public void close() throws IOException {
if (closed == true) {
return;
}
try {
if (remain > 0) {
throw new IOException("Writing less bytes than advertised size.");
}
} finally {
closed = true;
}
}
}
}

View File

@ -0,0 +1,97 @@
/**
* 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.util.Comparator;
import org.apache.hadoop.io.RawComparator;
import org.apache.hadoop.io.WritableComparator;
class CompareUtils {
/**
* Prevent the instantiation of class.
*/
private CompareUtils() {
// nothing
}
/**
* A comparator to compare anything that implements {@link RawComparable}
* using a customized comparator.
*/
public static final class BytesComparator implements
Comparator<RawComparable> {
private RawComparator<Object> cmp;
public BytesComparator(RawComparator<Object> cmp) {
this.cmp = cmp;
}
@Override
public int compare(RawComparable o1, RawComparable o2) {
return compare(o1.buffer(), o1.offset(), o1.size(), o2.buffer(), o2
.offset(), o2.size());
}
public int compare(byte[] a, int off1, int len1, byte[] b, int off2,
int len2) {
return cmp.compare(a, off1, len1, b, off2, len2);
}
}
/**
* Interface for all objects that has a single integer magnitude.
*/
static interface Scalar {
long magnitude();
}
static final class ScalarLong implements Scalar {
private long magnitude;
public ScalarLong(long m) {
magnitude = m;
}
public long magnitude() {
return magnitude;
}
}
public static final class ScalarComparator implements Comparator<Scalar> {
@Override
public int compare(Scalar o1, Scalar o2) {
long diff = o1.magnitude() - o2.magnitude();
if (diff < 0) return -1;
if (diff > 0) return 1;
return 0;
}
}
public static final class MemcmpRawComparator implements
RawComparator<Object> {
@Override
public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
return WritableComparator.compareBytes(b1, s1, l1, b2, s2, l2);
}
@Override
public int compare(Object o1, Object o2) {
throw new RuntimeException("Object comparison not supported");
}
}
}

View File

@ -0,0 +1,361 @@
/**
* 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.BufferedInputStream;
import java.io.BufferedOutputStream;
import java.io.FilterOutputStream;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.util.ArrayList;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.io.compress.CodecPool;
import org.apache.hadoop.io.compress.CompressionCodec;
import org.apache.hadoop.io.compress.CompressionInputStream;
import org.apache.hadoop.io.compress.CompressionOutputStream;
import org.apache.hadoop.io.compress.Compressor;
import org.apache.hadoop.io.compress.Decompressor;
import org.apache.hadoop.io.compress.DefaultCodec;
import org.apache.hadoop.util.ReflectionUtils;
/**
* Compression related stuff.
*/
final class Compression {
static final Log LOG = LogFactory.getLog(Compression.class);
/**
* Prevent the instantiation of class.
*/
private Compression() {
// nothing
}
static class FinishOnFlushCompressionStream extends FilterOutputStream {
public FinishOnFlushCompressionStream(CompressionOutputStream cout) {
super(cout);
}
@Override
public void write(byte b[], int off, int len) throws IOException {
out.write(b, off, len);
}
@Override
public void flush() throws IOException {
CompressionOutputStream cout = (CompressionOutputStream) out;
cout.finish();
cout.flush();
cout.resetState();
}
}
/**
* Compression algorithms.
*/
static enum Algorithm {
LZO(TFile.COMPRESSION_LZO) {
private transient boolean checked = false;
private static final String defaultClazz =
"org.apache.hadoop.io.compress.LzoCodec";
private transient CompressionCodec codec = null;
@Override
public synchronized boolean isSupported() {
if (!checked) {
checked = true;
String extClazz =
(conf.get(CONF_LZO_CLASS) == null ? System
.getProperty(CONF_LZO_CLASS) : null);
String clazz = (extClazz != null) ? extClazz : defaultClazz;
try {
LOG.info("Trying to load Lzo codec class: " + clazz);
codec =
(CompressionCodec) ReflectionUtils.newInstance(Class
.forName(clazz), conf);
} catch (ClassNotFoundException e) {
// that is okay
}
}
return codec != null;
}
@Override
CompressionCodec getCodec() throws IOException {
if (!isSupported()) {
throw new IOException(
"LZO codec class not specified. Did you forget to set property "
+ CONF_LZO_CLASS + "?");
}
return codec;
}
@Override
public synchronized InputStream createDecompressionStream(
InputStream downStream, Decompressor decompressor,
int downStreamBufferSize) throws IOException {
if (!isSupported()) {
throw new IOException(
"LZO codec class not specified. Did you forget to set property "
+ CONF_LZO_CLASS + "?");
}
InputStream bis1 = null;
if (downStreamBufferSize > 0) {
bis1 = new BufferedInputStream(downStream, downStreamBufferSize);
} else {
bis1 = downStream;
}
conf.setInt("io.compression.codec.lzo.buffersize", 64 * 1024);
CompressionInputStream cis =
codec.createInputStream(bis1, decompressor);
BufferedInputStream bis2 = new BufferedInputStream(cis, DATA_IBUF_SIZE);
return bis2;
}
@Override
public synchronized OutputStream createCompressionStream(
OutputStream downStream, Compressor compressor,
int downStreamBufferSize) throws IOException {
if (!isSupported()) {
throw new IOException(
"LZO codec class not specified. Did you forget to set property "
+ CONF_LZO_CLASS + "?");
}
OutputStream bos1 = null;
if (downStreamBufferSize > 0) {
bos1 = new BufferedOutputStream(downStream, downStreamBufferSize);
} else {
bos1 = downStream;
}
conf.setInt("io.compression.codec.lzo.buffersize", 64 * 1024);
CompressionOutputStream cos =
codec.createOutputStream(bos1, compressor);
BufferedOutputStream bos2 =
new BufferedOutputStream(new FinishOnFlushCompressionStream(cos),
DATA_OBUF_SIZE);
return bos2;
}
},
GZ(TFile.COMPRESSION_GZ) {
private transient DefaultCodec codec;
@Override
CompressionCodec getCodec() {
if (codec == null) {
codec = new DefaultCodec();
codec.setConf(conf);
}
return codec;
}
@Override
public synchronized InputStream createDecompressionStream(
InputStream downStream, Decompressor decompressor,
int downStreamBufferSize) throws IOException {
// Set the internal buffer size to read from down stream.
if (downStreamBufferSize > 0) {
codec.getConf().setInt("io.file.buffer.size", downStreamBufferSize);
}
CompressionInputStream cis =
codec.createInputStream(downStream, decompressor);
BufferedInputStream bis2 = new BufferedInputStream(cis, DATA_IBUF_SIZE);
return bis2;
}
@Override
public synchronized OutputStream createCompressionStream(
OutputStream downStream, Compressor compressor,
int downStreamBufferSize) throws IOException {
OutputStream bos1 = null;
if (downStreamBufferSize > 0) {
bos1 = new BufferedOutputStream(downStream, downStreamBufferSize);
} else {
bos1 = downStream;
}
codec.getConf().setInt("io.file.buffer.size", 32 * 1024);
CompressionOutputStream cos =
codec.createOutputStream(bos1, compressor);
BufferedOutputStream bos2 =
new BufferedOutputStream(new FinishOnFlushCompressionStream(cos),
DATA_OBUF_SIZE);
return bos2;
}
@Override
public boolean isSupported() {
return true;
}
},
NONE(TFile.COMPRESSION_NONE) {
@Override
CompressionCodec getCodec() {
return null;
}
@Override
public synchronized InputStream createDecompressionStream(
InputStream downStream, Decompressor decompressor,
int downStreamBufferSize) throws IOException {
if (downStreamBufferSize > 0) {
return new BufferedInputStream(downStream, downStreamBufferSize);
}
return downStream;
}
@Override
public synchronized OutputStream createCompressionStream(
OutputStream downStream, Compressor compressor,
int downStreamBufferSize) throws IOException {
if (downStreamBufferSize > 0) {
return new BufferedOutputStream(downStream, downStreamBufferSize);
}
return downStream;
}
@Override
public boolean isSupported() {
return true;
}
};
// We require that all compression related settings are configured
// statically in the Configuration object.
protected static final Configuration conf = new Configuration();
private final String compressName;
// data input buffer size to absorb small reads from application.
private static final int DATA_IBUF_SIZE = 1 * 1024;
// data output buffer size to absorb small writes from application.
private static final int DATA_OBUF_SIZE = 4 * 1024;
public static final String CONF_LZO_CLASS =
"io.compression.codec.lzo.class";
Algorithm(String name) {
this.compressName = name;
}
abstract CompressionCodec getCodec() throws IOException;
public abstract InputStream createDecompressionStream(
InputStream downStream, Decompressor decompressor,
int downStreamBufferSize) throws IOException;
public abstract OutputStream createCompressionStream(
OutputStream downStream, Compressor compressor, int downStreamBufferSize)
throws IOException;
public abstract boolean isSupported();
public Compressor getCompressor() throws IOException {
CompressionCodec codec = getCodec();
if (codec != null) {
Compressor compressor = CodecPool.getCompressor(codec);
if (compressor != null) {
if (compressor.finished()) {
// Somebody returns the compressor to CodecPool but is still using
// it.
LOG.warn("Compressor obtained from CodecPool already finished()");
} else {
LOG.debug("Got a compressor: " + compressor.hashCode());
}
/**
* Following statement is necessary to get around bugs in 0.18 where a
* compressor is referenced after returned back to the codec pool.
*/
compressor.reset();
}
return compressor;
}
return null;
}
public void returnCompressor(Compressor compressor) {
if (compressor != null) {
LOG.debug("Return a compressor: " + compressor.hashCode());
CodecPool.returnCompressor(compressor);
}
}
public Decompressor getDecompressor() throws IOException {
CompressionCodec codec = getCodec();
if (codec != null) {
Decompressor decompressor = CodecPool.getDecompressor(codec);
if (decompressor != null) {
if (decompressor.finished()) {
// Somebody returns the decompressor to CodecPool but is still using
// it.
LOG.warn("Deompressor obtained from CodecPool already finished()");
} else {
LOG.debug("Got a decompressor: " + decompressor.hashCode());
}
/**
* Following statement is necessary to get around bugs in 0.18 where a
* decompressor is referenced after returned back to the codec pool.
*/
decompressor.reset();
}
return decompressor;
}
return null;
}
public void returnDecompressor(Decompressor decompressor) {
if (decompressor != null) {
LOG.debug("Returned a decompressor: " + decompressor.hashCode());
CodecPool.returnDecompressor(decompressor);
}
}
public String getName() {
return compressName;
}
}
static Algorithm getCompressionAlgorithmByName(String compressName) {
Algorithm[] algos = Algorithm.class.getEnumConstants();
for (Algorithm a : algos) {
if (a.getName().equals(compressName)) {
return a;
}
}
throw new IllegalArgumentException(
"Unsupported compression algorithm name: " + compressName);
}
static String[] getSupportedAlgorithms() {
Algorithm[] algos = Algorithm.class.getEnumConstants();
ArrayList<String> ret = new ArrayList<String>();
for (Algorithm a : algos) {
if (a.isSupported()) {
ret.add(a.getName());
}
}
return ret.toArray(new String[ret.size()]);
}
}

View File

@ -0,0 +1,36 @@
/**
* 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.IOException;
/**
* Exception - Meta Block with the same name already exists.
*/
@SuppressWarnings("serial")
public class MetaBlockAlreadyExists extends IOException {
/**
* Constructor
*
* @param s
* message.
*/
MetaBlockAlreadyExists(String s) {
super(s);
}
}

View File

@ -0,0 +1,36 @@
/**
* 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.IOException;
/**
* Exception - No such Meta Block with the given name.
*/
@SuppressWarnings("serial")
public class MetaBlockDoesNotExist extends IOException {
/**
* Constructor
*
* @param s
* message.
*/
MetaBlockDoesNotExist(String s) {
super(s);
}
}

View File

@ -0,0 +1,57 @@
/**
* 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.util.Collections;
import java.util.Comparator;
import org.apache.hadoop.io.RawComparator;
/**
* Interface for objects that can be compared through {@link RawComparator}.
* This is useful in places where we need a single object reference to specify a
* range of bytes in a byte array, such as {@link Comparable} or
* {@link Collections#binarySearch(java.util.List, Object, Comparator)}
*
* The actual comparison among RawComparable's requires an external
* RawComparator and it is applications' responsibility to ensure two
* RawComparable are supposed to be semantically comparable with the same
* RawComparator.
*/
public interface RawComparable {
/**
* Get the underlying byte array.
*
* @return The underlying byte array.
*/
abstract byte[] buffer();
/**
* Get the offset of the first byte in the byte array.
*
* @return The offset of the first byte in the byte array.
*/
abstract int offset();
/**
* Get the size of the byte range in the byte array.
*
* @return The size of the byte range in the byte array.
*/
abstract int size();
}

View File

@ -0,0 +1,77 @@
/**
* 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.FilterOutputStream;
import java.io.IOException;
import java.io.OutputStream;
/**
* A simplified BufferedOutputStream with borrowed buffer, and allow users to
* see how much data have been buffered.
*/
class SimpleBufferedOutputStream extends FilterOutputStream {
protected byte buf[]; // the borrowed buffer
protected int count = 0; // bytes used in buffer.
// Constructor
public SimpleBufferedOutputStream(OutputStream out, byte[] buf) {
super(out);
this.buf = buf;
}
private void flushBuffer() throws IOException {
if (count > 0) {
out.write(buf, 0, count);
count = 0;
}
}
@Override
public void write(int b) throws IOException {
if (count >= buf.length) {
flushBuffer();
}
buf[count++] = (byte) b;
}
@Override
public void write(byte b[], int off, int len) throws IOException {
if (len >= buf.length) {
flushBuffer();
out.write(b, off, len);
return;
}
if (len > buf.length - count) {
flushBuffer();
}
System.arraycopy(b, off, buf, count, len);
count += len;
}
@Override
public synchronized void flush() throws IOException {
flushBuffer();
out.flush();
}
// Get the size of internal buffer being used.
public int size() {
return count;
}
}

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,295 @@
/**
* 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.IOException;
import java.io.PrintStream;
import java.util.Collection;
import java.util.Iterator;
import java.util.LinkedHashMap;
import java.util.Map;
import java.util.Set;
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.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.file.tfile.BCFile.BlockRegion;
import org.apache.hadoop.io.file.tfile.BCFile.MetaIndexEntry;
import org.apache.hadoop.io.file.tfile.TFile.TFileIndexEntry;
import org.apache.hadoop.io.file.tfile.Utils.Version;
/**
* Dumping the information of a TFile.
*/
class TFileDumper {
static final Log LOG = LogFactory.getLog(TFileDumper.class);
private TFileDumper() {
// namespace object not constructable.
}
private enum Align {
LEFT, CENTER, RIGHT, ZERO_PADDED;
static String format(String s, int width, Align align) {
if (s.length() >= width) return s;
int room = width - s.length();
Align alignAdjusted = align;
if (room == 1) {
alignAdjusted = LEFT;
}
if (alignAdjusted == LEFT) {
return s + String.format("%" + room + "s", "");
}
if (alignAdjusted == RIGHT) {
return String.format("%" + room + "s", "") + s;
}
if (alignAdjusted == CENTER) {
int half = room / 2;
return String.format("%" + half + "s", "") + s
+ String.format("%" + (room - half) + "s", "");
}
throw new IllegalArgumentException("Unsupported alignment");
}
static String format(long l, int width, Align align) {
if (align == ZERO_PADDED) {
return String.format("%0" + width + "d", l);
}
return format(Long.toString(l), width, align);
}
static int calculateWidth(String caption, long max) {
return Math.max(caption.length(), Long.toString(max).length());
}
}
/**
* Dump information about TFile.
*
* @param file
* Path string of the TFile
* @param out
* PrintStream to output the information.
* @param conf
* The configuration object.
* @throws IOException
*/
static public void dumpInfo(String file, PrintStream out, Configuration conf)
throws IOException {
final int maxKeySampleLen = 16;
Path path = new Path(file);
FileSystem fs = path.getFileSystem(conf);
long length = fs.getFileStatus(path).getLen();
FSDataInputStream fsdis = fs.open(path);
TFile.Reader reader = new TFile.Reader(fsdis, length, conf);
try {
LinkedHashMap<String, String> properties =
new LinkedHashMap<String, String>();
int blockCnt = reader.readerBCF.getBlockCount();
int metaBlkCnt = reader.readerBCF.metaIndex.index.size();
properties.put("BCFile Version", reader.readerBCF.version.toString());
properties.put("TFile Version", reader.tfileMeta.version.toString());
properties.put("File Length", Long.toString(length));
properties.put("Data Compression", reader.readerBCF
.getDefaultCompressionName());
properties.put("Record Count", Long.toString(reader.getEntryCount()));
properties.put("Sorted", Boolean.toString(reader.isSorted()));
if (reader.isSorted()) {
properties.put("Comparator", reader.getComparatorName());
}
properties.put("Data Block Count", Integer.toString(blockCnt));
long dataSize = 0, dataSizeUncompressed = 0;
if (blockCnt > 0) {
for (int i = 0; i < blockCnt; ++i) {
BlockRegion region =
reader.readerBCF.dataIndex.getBlockRegionList().get(i);
dataSize += region.getCompressedSize();
dataSizeUncompressed += region.getRawSize();
}
properties.put("Data Block Bytes", Long.toString(dataSize));
if (reader.readerBCF.getDefaultCompressionName() != "none") {
properties.put("Data Block Uncompressed Bytes", Long
.toString(dataSizeUncompressed));
properties.put("Data Block Compression Ratio", String.format(
"1:%.1f", (double) dataSizeUncompressed / dataSize));
}
}
properties.put("Meta Block Count", Integer.toString(metaBlkCnt));
long metaSize = 0, metaSizeUncompressed = 0;
if (metaBlkCnt > 0) {
Collection<MetaIndexEntry> metaBlks =
reader.readerBCF.metaIndex.index.values();
boolean calculateCompression = false;
for (Iterator<MetaIndexEntry> it = metaBlks.iterator(); it.hasNext();) {
MetaIndexEntry e = it.next();
metaSize += e.getRegion().getCompressedSize();
metaSizeUncompressed += e.getRegion().getRawSize();
if (e.getCompressionAlgorithm() != Compression.Algorithm.NONE) {
calculateCompression = true;
}
}
properties.put("Meta Block Bytes", Long.toString(metaSize));
if (calculateCompression) {
properties.put("Meta Block Uncompressed Bytes", Long
.toString(metaSizeUncompressed));
properties.put("Meta Block Compression Ratio", String.format(
"1:%.1f", (double) metaSizeUncompressed / metaSize));
}
}
properties.put("Meta-Data Size Ratio", String.format("1:%.1f",
(double) dataSize / metaSize));
long leftOverBytes = length - dataSize - metaSize;
long miscSize =
BCFile.Magic.size() * 2 + Long.SIZE / Byte.SIZE + Version.size();
long metaIndexSize = leftOverBytes - miscSize;
properties.put("Meta Block Index Bytes", Long.toString(metaIndexSize));
properties.put("Headers Etc Bytes", Long.toString(miscSize));
// Now output the properties table.
int maxKeyLength = 0;
Set<Map.Entry<String, String>> entrySet = properties.entrySet();
for (Iterator<Map.Entry<String, String>> it = entrySet.iterator(); it
.hasNext();) {
Map.Entry<String, String> e = it.next();
if (e.getKey().length() > maxKeyLength) {
maxKeyLength = e.getKey().length();
}
}
for (Iterator<Map.Entry<String, String>> it = entrySet.iterator(); it
.hasNext();) {
Map.Entry<String, String> e = it.next();
out.printf("%s : %s\n", Align.format(e.getKey(), maxKeyLength,
Align.LEFT), e.getValue());
}
out.println();
reader.checkTFileDataIndex();
if (blockCnt > 0) {
String blkID = "Data-Block";
int blkIDWidth = Align.calculateWidth(blkID, blockCnt);
int blkIDWidth2 = Align.calculateWidth("", blockCnt);
String offset = "Offset";
int offsetWidth = Align.calculateWidth(offset, length);
String blkLen = "Length";
int blkLenWidth =
Align.calculateWidth(blkLen, dataSize / blockCnt * 10);
String rawSize = "Raw-Size";
int rawSizeWidth =
Align.calculateWidth(rawSize, dataSizeUncompressed / blockCnt * 10);
String records = "Records";
int recordsWidth =
Align.calculateWidth(records, reader.getEntryCount() / blockCnt
* 10);
String endKey = "End-Key";
int endKeyWidth = Math.max(endKey.length(), maxKeySampleLen * 2 + 5);
out.printf("%s %s %s %s %s %s\n", Align.format(blkID, blkIDWidth,
Align.CENTER), Align.format(offset, offsetWidth, Align.CENTER),
Align.format(blkLen, blkLenWidth, Align.CENTER), Align.format(
rawSize, rawSizeWidth, Align.CENTER), Align.format(records,
recordsWidth, Align.CENTER), Align.format(endKey, endKeyWidth,
Align.LEFT));
for (int i = 0; i < blockCnt; ++i) {
BlockRegion region =
reader.readerBCF.dataIndex.getBlockRegionList().get(i);
TFileIndexEntry indexEntry = reader.tfileIndex.getEntry(i);
out.printf("%s %s %s %s %s ", Align.format(Align.format(i,
blkIDWidth2, Align.ZERO_PADDED), blkIDWidth, Align.LEFT), Align
.format(region.getOffset(), offsetWidth, Align.LEFT), Align
.format(region.getCompressedSize(), blkLenWidth, Align.LEFT),
Align.format(region.getRawSize(), rawSizeWidth, Align.LEFT),
Align.format(indexEntry.kvEntries, recordsWidth, Align.LEFT));
byte[] key = indexEntry.key;
boolean asAscii = true;
int sampleLen = Math.min(maxKeySampleLen, key.length);
for (int j = 0; j < sampleLen; ++j) {
byte b = key[j];
if ((b < 32 && b != 9) || (b == 127)) {
asAscii = false;
}
}
if (!asAscii) {
out.print("0X");
for (int j = 0; j < sampleLen; ++j) {
byte b = key[i];
out.printf("%X", b);
}
} else {
out.print(new String(key, 0, sampleLen));
}
if (sampleLen < key.length) {
out.print("...");
}
out.println();
}
}
out.println();
if (metaBlkCnt > 0) {
String name = "Meta-Block";
int maxNameLen = 0;
Set<Map.Entry<String, MetaIndexEntry>> metaBlkEntrySet =
reader.readerBCF.metaIndex.index.entrySet();
for (Iterator<Map.Entry<String, MetaIndexEntry>> it =
metaBlkEntrySet.iterator(); it.hasNext();) {
Map.Entry<String, MetaIndexEntry> e = it.next();
if (e.getKey().length() > maxNameLen) {
maxNameLen = e.getKey().length();
}
}
int nameWidth = Math.max(name.length(), maxNameLen);
String offset = "Offset";
int offsetWidth = Align.calculateWidth(offset, length);
String blkLen = "Length";
int blkLenWidth =
Align.calculateWidth(blkLen, metaSize / metaBlkCnt * 10);
String rawSize = "Raw-Size";
int rawSizeWidth =
Align.calculateWidth(rawSize, metaSizeUncompressed / metaBlkCnt
* 10);
String compression = "Compression";
int compressionWidth = compression.length();
out.printf("%s %s %s %s %s\n", Align.format(name, nameWidth,
Align.CENTER), Align.format(offset, offsetWidth, Align.CENTER),
Align.format(blkLen, blkLenWidth, Align.CENTER), Align.format(
rawSize, rawSizeWidth, Align.CENTER), Align.format(compression,
compressionWidth, Align.LEFT));
for (Iterator<Map.Entry<String, MetaIndexEntry>> it =
metaBlkEntrySet.iterator(); it.hasNext();) {
Map.Entry<String, MetaIndexEntry> e = it.next();
String blkName = e.getValue().getMetaName();
BlockRegion region = e.getValue().getRegion();
String blkCompression =
e.getValue().getCompressionAlgorithm().getName();
out.printf("%s %s %s %s %s\n", Align.format(blkName, nameWidth,
Align.LEFT), Align.format(region.getOffset(), offsetWidth,
Align.LEFT), Align.format(region.getCompressedSize(),
blkLenWidth, Align.LEFT), Align.format(region.getRawSize(),
rawSizeWidth, Align.LEFT), Align.format(blkCompression,
compressionWidth, Align.LEFT));
}
}
} finally {
IOUtils.cleanup(LOG, reader, fsdis);
}
}
}

View File

@ -0,0 +1,516 @@
/**
* 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.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import java.util.Comparator;
import java.util.List;
import org.apache.hadoop.io.Text;
/**
* Supporting Utility classes used by TFile, and shared by users of TFile.
*/
public final class Utils {
/**
* Prevent the instantiation of Utils.
*/
private Utils() {
// nothing
}
/**
* Encoding an integer into a variable-length encoding format. Synonymous to
* <code>Utils#writeVLong(out, n)</code>.
*
* @param out
* output stream
* @param n
* The integer to be encoded
* @throws IOException
* @see Utils#writeVLong(DataOutput, long)
*/
public static void writeVInt(DataOutput out, int n) throws IOException {
writeVLong(out, n);
}
/**
* Encoding a Long integer into a variable-length encoding format.
* <ul>
* <li>if n in [-32, 127): encode in one byte with the actual value.
* Otherwise,
* <li>if n in [-20*2^8, 20*2^8): encode in two bytes: byte[0] = n/256 - 52;
* byte[1]=n&0xff. Otherwise,
* <li>if n IN [-16*2^16, 16*2^16): encode in three bytes: byte[0]=n/2^16 -
* 88; byte[1]=(n>>8)&0xff; byte[2]=n&0xff. Otherwise,
* <li>if n in [-8*2^24, 8*2^24): encode in four bytes: byte[0]=n/2^24 - 112;
* byte[1] = (n>>16)&0xff; byte[2] = (n>>8)&0xff; byte[3]=n&0xff. Otherwise:
* <li>if n in [-2^31, 2^31): encode in five bytes: byte[0]=-125; byte[1] =
* (n>>24)&0xff; byte[2]=(n>>16)&0xff; byte[3]=(n>>8)&0xff; byte[4]=n&0xff;
* <li>if n in [-2^39, 2^39): encode in six bytes: byte[0]=-124; byte[1] =
* (n>>32)&0xff; byte[2]=(n>>24)&0xff; byte[3]=(n>>16)&0xff;
* byte[4]=(n>>8)&0xff; byte[5]=n&0xff
* <li>if n in [-2^47, 2^47): encode in seven bytes: byte[0]=-123; byte[1] =
* (n>>40)&0xff; byte[2]=(n>>32)&0xff; byte[3]=(n>>24)&0xff;
* byte[4]=(n>>16)&0xff; byte[5]=(n>>8)&0xff; byte[6]=n&0xff;
* <li>if n in [-2^55, 2^55): encode in eight bytes: byte[0]=-122; byte[1] =
* (n>>48)&0xff; byte[2] = (n>>40)&0xff; byte[3]=(n>>32)&0xff;
* byte[4]=(n>>24)&0xff; byte[5]=(n>>16)&0xff; byte[6]=(n>>8)&0xff;
* byte[7]=n&0xff;
* <li>if n in [-2^63, 2^63): encode in nine bytes: byte[0]=-121; byte[1] =
* (n>>54)&0xff; byte[2] = (n>>48)&0xff; byte[3] = (n>>40)&0xff;
* byte[4]=(n>>32)&0xff; byte[5]=(n>>24)&0xff; byte[6]=(n>>16)&0xff;
* byte[7]=(n>>8)&0xff; byte[8]=n&0xff;
* </ul>
*
* @param out
* output stream
* @param n
* the integer number
* @throws IOException
*/
@SuppressWarnings("fallthrough")
public static void writeVLong(DataOutput out, long n) throws IOException {
if ((n < 128) && (n >= -32)) {
out.writeByte((int) n);
return;
}
long un = (n < 0) ? ~n : n;
// how many bytes do we need to represent the number with sign bit?
int len = (Long.SIZE - Long.numberOfLeadingZeros(un)) / 8 + 1;
int firstByte = (int) (n >> ((len - 1) * 8));
switch (len) {
case 1:
// fall it through to firstByte==-1, len=2.
firstByte >>= 8;
case 2:
if ((firstByte < 20) && (firstByte >= -20)) {
out.writeByte(firstByte - 52);
out.writeByte((int) n);
return;
}
// fall it through to firstByte==0/-1, len=3.
firstByte >>= 8;
case 3:
if ((firstByte < 16) && (firstByte >= -16)) {
out.writeByte(firstByte - 88);
out.writeShort((int) n);
return;
}
// fall it through to firstByte==0/-1, len=4.
firstByte >>= 8;
case 4:
if ((firstByte < 8) && (firstByte >= -8)) {
out.writeByte(firstByte - 112);
out.writeShort(((int) n) >>> 8);
out.writeByte((int) n);
return;
}
out.writeByte(len - 129);
out.writeInt((int) n);
return;
case 5:
out.writeByte(len - 129);
out.writeInt((int) (n >>> 8));
out.writeByte((int) n);
return;
case 6:
out.writeByte(len - 129);
out.writeInt((int) (n >>> 16));
out.writeShort((int) n);
return;
case 7:
out.writeByte(len - 129);
out.writeInt((int) (n >>> 24));
out.writeShort((int) (n >>> 8));
out.writeByte((int) n);
return;
case 8:
out.writeByte(len - 129);
out.writeLong(n);
return;
default:
throw new RuntimeException("Internel error");
}
}
/**
* Decoding the variable-length integer. Synonymous to
* <code>(int)Utils#readVLong(in)</code>.
*
* @param in
* input stream
* @return the decoded integer
* @throws IOException
*
* @see Utils#readVLong(DataInput)
*/
public static int readVInt(DataInput in) throws IOException {
long ret = readVLong(in);
if ((ret > Integer.MAX_VALUE) || (ret < Integer.MIN_VALUE)) {
throw new RuntimeException(
"Number too large to be represented as Integer");
}
return (int) ret;
}
/**
* Decoding the variable-length integer. Suppose the value of the first byte
* is FB, and the following bytes are NB[*].
* <ul>
* <li>if (FB >= -32), return (long)FB;
* <li>if (FB in [-72, -33]), return (FB+52)<<8 + NB[0]&0xff;
* <li>if (FB in [-104, -73]), return (FB+88)<<16 + (NB[0]&0xff)<<8 +
* NB[1]&0xff;
* <li>if (FB in [-120, -105]), return (FB+112)<<24 + (NB[0]&0xff)<<16 +
* (NB[1]&0xff)<<8 + NB[2]&0xff;
* <li>if (FB in [-128, -121]), return interpret NB[FB+129] as a signed
* big-endian integer.
*
* @param in
* input stream
* @return the decoded long integer.
* @throws IOException
*/
public static long readVLong(DataInput in) throws IOException {
int firstByte = in.readByte();
if (firstByte >= -32) {
return firstByte;
}
switch ((firstByte + 128) / 8) {
case 11:
case 10:
case 9:
case 8:
case 7:
return ((firstByte + 52) << 8) | in.readUnsignedByte();
case 6:
case 5:
case 4:
case 3:
return ((firstByte + 88) << 16) | in.readUnsignedShort();
case 2:
case 1:
return ((firstByte + 112) << 24) | (in.readUnsignedShort() << 8)
| in.readUnsignedByte();
case 0:
int len = firstByte + 129;
switch (len) {
case 4:
return in.readInt();
case 5:
return ((long) in.readInt()) << 8 | in.readUnsignedByte();
case 6:
return ((long) in.readInt()) << 16 | in.readUnsignedShort();
case 7:
return ((long) in.readInt()) << 24 | (in.readUnsignedShort() << 8)
| in.readUnsignedByte();
case 8:
return in.readLong();
default:
throw new IOException("Corrupted VLong encoding");
}
default:
throw new RuntimeException("Internal error");
}
}
/**
* Write a String as a VInt n, followed by n Bytes as in Text format.
*
* @param out
* @param s
* @throws IOException
*/
public static void writeString(DataOutput out, String s) throws IOException {
if (s != null) {
Text text = new Text(s);
byte[] buffer = text.getBytes();
int len = text.getLength();
writeVInt(out, len);
out.write(buffer, 0, len);
} else {
writeVInt(out, -1);
}
}
/**
* Read a String as a VInt n, followed by n Bytes in Text format.
*
* @param in
* The input stream.
* @return The string
* @throws IOException
*/
public static String readString(DataInput in) throws IOException {
int length = readVInt(in);
if (length == -1) return null;
byte[] buffer = new byte[length];
in.readFully(buffer);
return Text.decode(buffer);
}
/**
* A generic Version class. We suggest applications built on top of TFile use
* this class to maintain version information in their meta blocks.
*
* A version number consists of a major version and a minor version. The
* suggested usage of major and minor version number is to increment major
* version number when the new storage format is not backward compatible, and
* increment the minor version otherwise.
*/
public static final class Version implements Comparable<Version> {
private final short major;
private final short minor;
/**
* Construct the Version object by reading from the input stream.
*
* @param in
* input stream
* @throws IOException
*/
public Version(DataInput in) throws IOException {
major = in.readShort();
minor = in.readShort();
}
/**
* Constructor.
*
* @param major
* major version.
* @param minor
* minor version.
*/
public Version(short major, short minor) {
this.major = major;
this.minor = minor;
}
/**
* Write the objec to a DataOutput. The serialized format of the Version is
* major version followed by minor version, both as big-endian short
* integers.
*
* @param out
* The DataOutput object.
* @throws IOException
*/
public void write(DataOutput out) throws IOException {
out.writeShort(major);
out.writeShort(minor);
}
/**
* Get the major version.
*
* @return Major version.
*/
public int getMajor() {
return major;
}
/**
* Get the minor version.
*
* @return The minor version.
*/
public int getMinor() {
return minor;
}
/**
* Get the size of the serialized Version object.
*
* @return serialized size of the version object.
*/
public static int size() {
return (Short.SIZE + Short.SIZE) / Byte.SIZE;
}
/**
* Return a string representation of the version.
*/
public String toString() {
return new StringBuilder("v").append(major).append(".").append(minor)
.toString();
}
/**
* Test compatibility.
*
* @param other
* The Version object to test compatibility with.
* @return true if both versions have the same major version number; false
* otherwise.
*/
public boolean compatibleWith(Version other) {
return major == other.major;
}
/**
* Compare this version with another version.
*/
@Override
public int compareTo(Version that) {
if (major != that.major) {
return major - that.major;
}
return minor - that.minor;
}
@Override
public boolean equals(Object other) {
if (this == other) return true;
if (!(other instanceof Version)) return false;
return compareTo((Version) other) == 0;
}
@Override
public int hashCode() {
return (major << 16 + minor);
}
}
/**
* Lower bound binary search. Find the index to the first element in the list
* that compares greater than or equal to key.
*
* @param <T>
* Type of the input key.
* @param list
* The list
* @param key
* The input key.
* @param cmp
* Comparator for the key.
* @return The index to the desired element if it exists; or list.size()
* otherwise.
*/
public static <T> int lowerBound(List<? extends T> list, T key,
Comparator<? super T> cmp) {
int low = 0;
int high = list.size();
while (low < high) {
int mid = (low + high) >>> 1;
T midVal = list.get(mid);
int ret = cmp.compare(midVal, key);
if (ret < 0)
low = mid + 1;
else high = mid;
}
return low;
}
/**
* Upper bound binary search. Find the index to the first element in the list
* that compares greater than the input key.
*
* @param <T>
* Type of the input key.
* @param list
* The list
* @param key
* The input key.
* @param cmp
* Comparator for the key.
* @return The index to the desired element if it exists; or list.size()
* otherwise.
*/
public static <T> int upperBound(List<? extends T> list, T key,
Comparator<? super T> cmp) {
int low = 0;
int high = list.size();
while (low < high) {
int mid = (low + high) >>> 1;
T midVal = list.get(mid);
int ret = cmp.compare(midVal, key);
if (ret <= 0)
low = mid + 1;
else high = mid;
}
return low;
}
/**
* Lower bound binary search. Find the index to the first element in the list
* that compares greater than or equal to key.
*
* @param <T>
* Type of the input key.
* @param list
* The list
* @param key
* The input key.
* @return The index to the desired element if it exists; or list.size()
* otherwise.
*/
public static <T> int lowerBound(List<? extends Comparable<? super T>> list,
T key) {
int low = 0;
int high = list.size();
while (low < high) {
int mid = (low + high) >>> 1;
Comparable<? super T> midVal = list.get(mid);
int ret = midVal.compareTo(key);
if (ret < 0)
low = mid + 1;
else high = mid;
}
return low;
}
/**
* Upper bound binary search. Find the index to the first element in the list
* that compares greater than the input key.
*
* @param <T>
* Type of the input key.
* @param list
* The list
* @param key
* The input key.
* @return The index to the desired element if it exists; or list.size()
* otherwise.
*/
public static <T> int upperBound(List<? extends Comparable<? super T>> list,
T key) {
int low = 0;
int high = list.size();
while (low < high) {
int mid = (low + high) >>> 1;
Comparable<? super T> midVal = list.get(mid);
int ret = midVal.compareTo(key);
if (ret <= 0)
low = mid + 1;
else high = mid;
}
return low;
}
}

View File

@ -0,0 +1,105 @@
/**
* 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.util.Random;
import org.apache.hadoop.io.BytesWritable;
import org.apache.hadoop.io.WritableComparator;
import org.apache.hadoop.io.file.tfile.RandomDistribution.DiscreteRNG;
/**
* Generate random <key, value> pairs.
*/
class KVGenerator {
private final Random random;
private final byte[][] dict;
private final boolean sorted;
private final DiscreteRNG keyLenRNG, valLenRNG;
private BytesWritable lastKey;
private static final int MIN_KEY_LEN = 4;
private final byte prefix[] = new byte[MIN_KEY_LEN];
public KVGenerator(Random random, boolean sorted, DiscreteRNG keyLenRNG,
DiscreteRNG valLenRNG, DiscreteRNG wordLenRNG, int dictSize) {
this.random = random;
dict = new byte[dictSize][];
this.sorted = sorted;
this.keyLenRNG = keyLenRNG;
this.valLenRNG = valLenRNG;
for (int i = 0; i < dictSize; ++i) {
int wordLen = wordLenRNG.nextInt();
dict[i] = new byte[wordLen];
random.nextBytes(dict[i]);
}
lastKey = new BytesWritable();
fillKey(lastKey);
}
private void fillKey(BytesWritable o) {
int len = keyLenRNG.nextInt();
if (len < MIN_KEY_LEN) len = MIN_KEY_LEN;
o.setSize(len);
int n = MIN_KEY_LEN;
while (n < len) {
byte[] word = dict[random.nextInt(dict.length)];
int l = Math.min(word.length, len - n);
System.arraycopy(word, 0, o.get(), n, l);
n += l;
}
if (sorted
&& WritableComparator.compareBytes(lastKey.get(), MIN_KEY_LEN, lastKey
.getSize()
- MIN_KEY_LEN, o.get(), MIN_KEY_LEN, o.getSize() - MIN_KEY_LEN) > 0) {
incrementPrefix();
}
System.arraycopy(prefix, 0, o.get(), 0, MIN_KEY_LEN);
lastKey.set(o);
}
private void fillValue(BytesWritable o) {
int len = valLenRNG.nextInt();
o.setSize(len);
int n = 0;
while (n < len) {
byte[] word = dict[random.nextInt(dict.length)];
int l = Math.min(word.length, len - n);
System.arraycopy(word, 0, o.get(), n, l);
n += l;
}
}
private void incrementPrefix() {
for (int i = MIN_KEY_LEN - 1; i >= 0; --i) {
++prefix[i];
if (prefix[i] != 0) return;
}
throw new RuntimeException("Prefix overflown");
}
public void next(BytesWritable key, BytesWritable value, boolean dupKey) {
if (dupKey) {
key.set(lastKey);
}
else {
fillKey(key);
}
fillValue(value);
}
}

View File

@ -0,0 +1,56 @@
/**
* 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.IOException;
import java.util.Random;
import org.apache.hadoop.io.BytesWritable;
import org.apache.hadoop.io.file.tfile.RandomDistribution.DiscreteRNG;
class KeySampler {
Random random;
int min, max;
DiscreteRNG keyLenRNG;
private static final int MIN_KEY_LEN = 4;
public KeySampler(Random random, RawComparable first, RawComparable last,
DiscreteRNG keyLenRNG) throws IOException {
this.random = random;
min = keyPrefixToInt(first);
max = keyPrefixToInt(last);
this.keyLenRNG = keyLenRNG;
}
private int keyPrefixToInt(RawComparable key) throws IOException {
byte[] b = key.buffer();
int o = key.offset();
return (b[o] & 0xff) << 24 | (b[o + 1] & 0xff) << 16
| (b[o + 2] & 0xff) << 8 | (b[o + 3] & 0xff);
}
public void next(BytesWritable key) {
key.setSize(Math.max(MIN_KEY_LEN, keyLenRNG.nextInt()));
random.nextBytes(key.get());
int n = random.nextInt(max - min) + min;
byte[] b = key.get();
b[0] = (byte) (n >> 24);
b[1] = (byte) (n >> 16);
b[2] = (byte) (n >> 8);
b[3] = (byte) n;
}
}

View File

@ -0,0 +1,193 @@
/**
* 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;
/**
* A nano-second timer.
*/
public class NanoTimer {
private long last = -1;
private boolean started = false;
private long cumulate = 0;
/**
* Constructor
*
* @param start
* Start the timer upon construction.
*/
public NanoTimer(boolean start) {
if (start) this.start();
}
/**
* Start the timer.
*
* Note: No effect if timer is already started.
*/
public void start() {
if (!this.started) {
this.last = System.nanoTime();
this.started = true;
}
}
/**
* Stop the timer.
*
* Note: No effect if timer is already stopped.
*/
public void stop() {
if (this.started) {
this.started = false;
this.cumulate += System.nanoTime() - this.last;
}
}
/**
* Read the timer.
*
* @return the elapsed time in nano-seconds. Note: If the timer is never
* started before, -1 is returned.
*/
public long read() {
if (!readable()) return -1;
return this.cumulate;
}
/**
* Reset the timer.
*/
public void reset() {
this.last = -1;
this.started = false;
this.cumulate = 0;
}
/**
* Checking whether the timer is started
*
* @return true if timer is started.
*/
public boolean isStarted() {
return this.started;
}
/**
* Format the elapsed time to a human understandable string.
*
* Note: If timer is never started, "ERR" will be returned.
*/
public String toString() {
if (!readable()) {
return "ERR";
}
return NanoTimer.nanoTimeToString(this.cumulate);
}
/**
* A utility method to format a time duration in nano seconds into a human
* understandable stirng.
*
* @param t
* Time duration in nano seconds.
* @return String representation.
*/
public static String nanoTimeToString(long t) {
if (t < 0) return "ERR";
if (t == 0) return "0";
if (t < 1000) {
return t + "ns";
}
double us = (double) t / 1000;
if (us < 1000) {
return String.format("%.2fus", us);
}
double ms = us / 1000;
if (ms < 1000) {
return String.format("%.2fms", ms);
}
double ss = ms / 1000;
if (ss < 1000) {
return String.format("%.2fs", ss);
}
long mm = (long) ss / 60;
ss -= mm * 60;
long hh = mm / 60;
mm -= hh * 60;
long dd = hh / 24;
hh -= dd * 24;
if (dd > 0) {
return String.format("%dd%dh", dd, hh);
}
if (hh > 0) {
return String.format("%dh%dm", hh, mm);
}
if (mm > 0) {
return String.format("%dm%.1fs", mm, ss);
}
return String.format("%.2fs", ss);
/**
* StringBuilder sb = new StringBuilder(); String sep = "";
*
* if (dd > 0) { String unit = (dd > 1) ? "days" : "day";
* sb.append(String.format("%s%d%s", sep, dd, unit)); sep = " "; }
*
* if (hh > 0) { String unit = (hh > 1) ? "hrs" : "hr";
* sb.append(String.format("%s%d%s", sep, hh, unit)); sep = " "; }
*
* if (mm > 0) { String unit = (mm > 1) ? "mins" : "min";
* sb.append(String.format("%s%d%s", sep, mm, unit)); sep = " "; }
*
* if (ss > 0) { String unit = (ss > 1) ? "secs" : "sec";
* sb.append(String.format("%s%.3f%s", sep, ss, unit)); sep = " "; }
*
* return sb.toString();
*/
}
private boolean readable() {
return this.last != -1;
}
/**
* Simple tester.
*
* @param args
*/
public static void main(String[] args) {
long i = 7;
for (int x = 0; x < 20; ++x, i *= 7) {
System.out.println(NanoTimer.nanoTimeToString(i));
}
}
}

View File

@ -0,0 +1,266 @@
/**
* 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.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.Random;
/**
* A class that generates random numbers that follow some distribution.
*/
public class RandomDistribution {
/**
* Interface for discrete (integer) random distributions.
*/
public static interface DiscreteRNG {
/**
* Get the next random number
*
* @return the next random number.
*/
public int nextInt();
}
/**
* P(i)=1/(max-min)
*/
public static final class Flat implements DiscreteRNG {
private final Random random;
private final int min;
private final int max;
/**
* Generate random integers from min (inclusive) to max (exclusive)
* following even distribution.
*
* @param random
* The basic random number generator.
* @param min
* Minimum integer
* @param max
* maximum integer (exclusive).
*
*/
public Flat(Random random, int min, int max) {
if (min >= max) {
throw new IllegalArgumentException("Invalid range");
}
this.random = random;
this.min = min;
this.max = max;
}
/**
* @see DiscreteRNG#nextInt()
*/
@Override
public int nextInt() {
return random.nextInt(max - min) + min;
}
}
/**
* Zipf distribution. The ratio of the probabilities of integer i and j is
* defined as follows:
*
* P(i)/P(j)=((j-min+1)/(i-min+1))^sigma.
*/
public static final class Zipf implements DiscreteRNG {
private static final double DEFAULT_EPSILON = 0.001;
private final Random random;
private final ArrayList<Integer> k;
private final ArrayList<Double> v;
/**
* Constructor
*
* @param r
* The random number generator.
* @param min
* minimum integer (inclusvie)
* @param max
* maximum integer (exclusive)
* @param sigma
* parameter sigma. (sigma > 1.0)
*/
public Zipf(Random r, int min, int max, double sigma) {
this(r, min, max, sigma, DEFAULT_EPSILON);
}
/**
* Constructor.
*
* @param r
* The random number generator.
* @param min
* minimum integer (inclusvie)
* @param max
* maximum integer (exclusive)
* @param sigma
* parameter sigma. (sigma > 1.0)
* @param epsilon
* Allowable error percentage (0 < epsilon < 1.0).
*/
public Zipf(Random r, int min, int max, double sigma, double epsilon) {
if ((max <= min) || (sigma <= 1) || (epsilon <= 0)
|| (epsilon >= 0.5)) {
throw new IllegalArgumentException("Invalid arguments");
}
random = r;
k = new ArrayList<Integer>();
v = new ArrayList<Double>();
double sum = 0;
int last = -1;
for (int i = min; i < max; ++i) {
sum += Math.exp(-sigma * Math.log(i - min + 1));
if ((last == -1) || i * (1 - epsilon) > last) {
k.add(i);
v.add(sum);
last = i;
}
}
if (last != max - 1) {
k.add(max - 1);
v.add(sum);
}
v.set(v.size() - 1, 1.0);
for (int i = v.size() - 2; i >= 0; --i) {
v.set(i, v.get(i) / sum);
}
}
/**
* @see DiscreteRNG#nextInt()
*/
@Override
public int nextInt() {
double d = random.nextDouble();
int idx = Collections.binarySearch(v, d);
if (idx > 0) {
++idx;
}
else {
idx = -(idx + 1);
}
if (idx >= v.size()) {
idx = v.size() - 1;
}
if (idx == 0) {
return k.get(0);
}
int ceiling = k.get(idx);
int lower = k.get(idx - 1);
return ceiling - random.nextInt(ceiling - lower);
}
}
/**
* Binomial distribution.
*
* P(k)=select(n, k)*p^k*(1-p)^(n-k) (k = 0, 1, ..., n)
*
* P(k)=select(max-min-1, k-min)*p^(k-min)*(1-p)^(k-min)*(1-p)^(max-k-1)
*/
public static final class Binomial implements DiscreteRNG {
private final Random random;
private final int min;
private final int n;
private final double[] v;
private static double select(int n, int k) {
double ret = 1.0;
for (int i = k + 1; i <= n; ++i) {
ret *= (double) i / (i - k);
}
return ret;
}
private static double power(double p, int k) {
return Math.exp(k * Math.log(p));
}
/**
* Generate random integers from min (inclusive) to max (exclusive)
* following Binomial distribution.
*
* @param random
* The basic random number generator.
* @param min
* Minimum integer
* @param max
* maximum integer (exclusive).
* @param p
* parameter.
*
*/
public Binomial(Random random, int min, int max, double p) {
if (min >= max) {
throw new IllegalArgumentException("Invalid range");
}
this.random = random;
this.min = min;
this.n = max - min - 1;
if (n > 0) {
v = new double[n + 1];
double sum = 0.0;
for (int i = 0; i <= n; ++i) {
sum += select(n, i) * power(p, i) * power(1 - p, n - i);
v[i] = sum;
}
for (int i = 0; i <= n; ++i) {
v[i] /= sum;
}
}
else {
v = null;
}
}
/**
* @see DiscreteRNG#nextInt()
*/
@Override
public int nextInt() {
if (v == null) {
return min;
}
double d = random.nextDouble();
int idx = Arrays.binarySearch(v, d);
if (idx > 0) {
++idx;
} else {
idx = -(idx + 1);
}
if (idx >= v.length) {
idx = v.length - 1;
}
return idx + min;
}
}
}

View File

@ -0,0 +1,431 @@
/**
* 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.DataInputStream;
import java.io.DataOutputStream;
import java.io.IOException;
import java.util.Arrays;
import junit.framework.TestCase;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.file.tfile.TFile.Reader;
import org.apache.hadoop.io.file.tfile.TFile.Writer;
import org.apache.hadoop.io.file.tfile.TFile.Reader.Scanner;
/**
* test tfile features.
*
*/
public class TestTFile extends TestCase {
private static String ROOT =
System.getProperty("test.build.data", "/tmp/tfile-test");
private FileSystem fs;
private Configuration conf;
private final int minBlockSize = 512;
private final int largeVal = 3 * 1024 * 1024;
private static String localFormatter = "%010d";
@Override
public void setUp() throws IOException {
conf = new Configuration();
fs = FileSystem.get(conf);
}
@Override
public void tearDown() throws IOException {
// do nothing
}
// read a key from the scanner
public byte[] readKey(Scanner scanner) throws IOException {
int keylen = scanner.entry().getKeyLength();
byte[] read = new byte[keylen];
scanner.entry().getKey(read);
return read;
}
// read a value from the scanner
public byte[] readValue(Scanner scanner) throws IOException {
int valueLen = scanner.entry().getValueLength();
byte[] read = new byte[valueLen];
scanner.entry().getValue(read);
return read;
}
// read a long value from the scanner
public byte[] readLongValue(Scanner scanner, int len) throws IOException {
DataInputStream din = scanner.entry().getValueStream();
byte[] b = new byte[len];
din.readFully(b);
din.close();
return b;
}
// write some records into the tfile
// write them twice
private int writeSomeRecords(Writer writer, int start, int n)
throws IOException {
String value = "value";
for (int i = start; i < (start + n); i++) {
String key = String.format(localFormatter, i);
writer.append(key.getBytes(), (value + key).getBytes());
writer.append(key.getBytes(), (value + key).getBytes());
}
return (start + n);
}
// read the records and check
private int readAndCheckbytes(Scanner scanner, int start, int n)
throws IOException {
String value = "value";
for (int i = start; i < (start + n); i++) {
byte[] key = readKey(scanner);
byte[] val = readValue(scanner);
String keyStr = String.format(localFormatter, i);
String valStr = value + keyStr;
assertTrue("btyes for keys do not match " + keyStr + " "
+ new String(key), Arrays.equals(keyStr.getBytes(), key));
assertTrue("bytes for vals do not match " + valStr + " "
+ new String(val), Arrays.equals(
valStr.getBytes(), val));
assertTrue(scanner.advance());
key = readKey(scanner);
val = readValue(scanner);
assertTrue("btyes for keys do not match", Arrays.equals(
keyStr.getBytes(), key));
assertTrue("bytes for vals do not match", Arrays.equals(
valStr.getBytes(), val));
assertTrue(scanner.advance());
}
return (start + n);
}
// write some large records
// write them twice
private int writeLargeRecords(Writer writer, int start, int n)
throws IOException {
byte[] value = new byte[largeVal];
for (int i = start; i < (start + n); i++) {
String key = String.format(localFormatter, i);
writer.append(key.getBytes(), value);
writer.append(key.getBytes(), value);
}
return (start + n);
}
// read large records
// read them twice since its duplicated
private int readLargeRecords(Scanner scanner, int start, int n)
throws IOException {
for (int i = start; i < (start + n); i++) {
byte[] key = readKey(scanner);
String keyStr = String.format(localFormatter, i);
assertTrue("btyes for keys do not match", Arrays.equals(
keyStr.getBytes(), key));
scanner.advance();
key = readKey(scanner);
assertTrue("btyes for keys do not match", Arrays.equals(
keyStr.getBytes(), key));
scanner.advance();
}
return (start + n);
}
// write empty keys and values
private void writeEmptyRecords(Writer writer, int n) throws IOException {
byte[] key = new byte[0];
byte[] value = new byte[0];
for (int i = 0; i < n; i++) {
writer.append(key, value);
}
}
// read empty keys and values
private void readEmptyRecords(Scanner scanner, int n) throws IOException {
byte[] key = new byte[0];
byte[] value = new byte[0];
byte[] readKey = null;
byte[] readValue = null;
for (int i = 0; i < n; i++) {
readKey = readKey(scanner);
readValue = readValue(scanner);
assertTrue("failed to match keys", Arrays.equals(readKey, key));
assertTrue("failed to match values", Arrays.equals(readValue, value));
assertTrue("failed to advance cursor", scanner.advance());
}
}
private int writePrepWithKnownLength(Writer writer, int start, int n)
throws IOException {
// get the length of the key
String key = String.format(localFormatter, start);
int keyLen = key.getBytes().length;
String value = "value" + key;
int valueLen = value.getBytes().length;
for (int i = start; i < (start + n); i++) {
DataOutputStream out = writer.prepareAppendKey(keyLen);
String localKey = String.format(localFormatter, i);
out.write(localKey.getBytes());
out.close();
out = writer.prepareAppendValue(valueLen);
String localValue = "value" + localKey;
out.write(localValue.getBytes());
out.close();
}
return (start + n);
}
private int readPrepWithKnownLength(Scanner scanner, int start, int n)
throws IOException {
for (int i = start; i < (start + n); i++) {
String key = String.format(localFormatter, i);
byte[] read = readKey(scanner);
assertTrue("keys not equal", Arrays.equals(key.getBytes(), read));
String value = "value" + key;
read = readValue(scanner);
assertTrue("values not equal", Arrays.equals(value.getBytes(), read));
scanner.advance();
}
return (start + n);
}
private int writePrepWithUnkownLength(Writer writer, int start, int n)
throws IOException {
for (int i = start; i < (start + n); i++) {
DataOutputStream out = writer.prepareAppendKey(-1);
String localKey = String.format(localFormatter, i);
out.write(localKey.getBytes());
out.close();
String value = "value" + localKey;
out = writer.prepareAppendValue(-1);
out.write(value.getBytes());
out.close();
}
return (start + n);
}
private int readPrepWithUnknownLength(Scanner scanner, int start, int n)
throws IOException {
for (int i = start; i < start; i++) {
String key = String.format(localFormatter, i);
byte[] read = readKey(scanner);
assertTrue("keys not equal", Arrays.equals(key.getBytes(), read));
try {
read = readValue(scanner);
assertTrue(false);
}
catch (IOException ie) {
// should have thrown exception
}
String value = "value" + key;
read = readLongValue(scanner, value.getBytes().length);
assertTrue("values nto equal", Arrays.equals(read, value.getBytes()));
scanner.advance();
}
return (start + n);
}
private byte[] getSomeKey(int rowId) {
return String.format(localFormatter, rowId).getBytes();
}
private void writeRecords(Writer writer) throws IOException {
writeEmptyRecords(writer, 10);
int ret = writeSomeRecords(writer, 0, 100);
ret = writeLargeRecords(writer, ret, 1);
ret = writePrepWithKnownLength(writer, ret, 40);
ret = writePrepWithUnkownLength(writer, ret, 50);
writer.close();
}
private void readAllRecords(Scanner scanner) throws IOException {
readEmptyRecords(scanner, 10);
int ret = readAndCheckbytes(scanner, 0, 100);
ret = readLargeRecords(scanner, ret, 1);
ret = readPrepWithKnownLength(scanner, ret, 40);
ret = readPrepWithUnknownLength(scanner, ret, 50);
}
private FSDataOutputStream createFSOutput(Path name) throws IOException {
if (fs.exists(name)) fs.delete(name, true);
FSDataOutputStream fout = fs.create(name);
return fout;
}
/**
* test none codecs
*/
void basicWithSomeCodec(String codec) throws IOException {
Path ncTFile = new Path(ROOT, "basic.tfile");
FSDataOutputStream fout = createFSOutput(ncTFile);
Writer writer = new Writer(fout, minBlockSize, codec, "memcmp", conf);
writeRecords(writer);
fout.close();
FSDataInputStream fin = fs.open(ncTFile);
Reader reader =
new Reader(fs.open(ncTFile), fs.getFileStatus(ncTFile).getLen(), conf);
Scanner scanner = reader.createScanner();
readAllRecords(scanner);
scanner.seekTo(getSomeKey(50));
assertTrue("location lookup failed", scanner.seekTo(getSomeKey(50)));
// read the key and see if it matches
byte[] readKey = readKey(scanner);
assertTrue("seeked key does not match", Arrays.equals(getSomeKey(50),
readKey));
scanner.seekTo(new byte[0]);
byte[] val1 = readValue(scanner);
scanner.seekTo(new byte[0]);
byte[] val2 = readValue(scanner);
assertTrue(Arrays.equals(val1, val2));
// check for lowerBound
scanner.lowerBound(getSomeKey(50));
assertTrue("locaton lookup failed", scanner.currentLocation
.compareTo(reader.end()) < 0);
readKey = readKey(scanner);
assertTrue("seeked key does not match", Arrays.equals(readKey,
getSomeKey(50)));
// check for upper bound
scanner.upperBound(getSomeKey(50));
assertTrue("location lookup failed", scanner.currentLocation
.compareTo(reader.end()) < 0);
readKey = readKey(scanner);
assertTrue("seeked key does not match", Arrays.equals(readKey,
getSomeKey(51)));
scanner.close();
// test for a range of scanner
scanner = reader.createScanner(getSomeKey(10), getSomeKey(60));
readAndCheckbytes(scanner, 10, 50);
assertFalse(scanner.advance());
scanner.close();
reader.close();
fin.close();
fs.delete(ncTFile, true);
}
// unsorted with some codec
void unsortedWithSomeCodec(String codec) throws IOException {
Path uTfile = new Path(ROOT, "unsorted.tfile");
FSDataOutputStream fout = createFSOutput(uTfile);
Writer writer = new Writer(fout, minBlockSize, codec, null, conf);
writeRecords(writer);
writer.close();
fout.close();
FSDataInputStream fin = fs.open(uTfile);
Reader reader =
new Reader(fs.open(uTfile), fs.getFileStatus(uTfile).getLen(), conf);
Scanner scanner = reader.createScanner();
readAllRecords(scanner);
scanner.close();
reader.close();
fin.close();
fs.delete(uTfile, true);
}
public void testTFileFeatures() throws IOException {
basicWithSomeCodec("none");
basicWithSomeCodec("gz");
}
// test unsorted t files.
public void testUnsortedTFileFeatures() throws IOException {
unsortedWithSomeCodec("none");
unsortedWithSomeCodec("gz");
}
private void writeNumMetablocks(Writer writer, String compression, int n)
throws IOException {
for (int i = 0; i < n; i++) {
DataOutputStream dout =
writer.prepareMetaBlock("TfileMeta" + i, compression);
byte[] b = ("something to test" + i).getBytes();
dout.write(b);
dout.close();
}
}
private void someTestingWithMetaBlock(Writer writer, String compression)
throws IOException {
DataOutputStream dout = null;
writeNumMetablocks(writer, compression, 10);
try {
dout = writer.prepareMetaBlock("TfileMeta1", compression);
assertTrue(false);
}
catch (MetaBlockAlreadyExists me) {
// avoid this exception
}
dout = writer.prepareMetaBlock("TFileMeta100", compression);
dout.close();
}
private void readNumMetablocks(Reader reader, int n) throws IOException {
int len = ("something to test" + 0).getBytes().length;
for (int i = 0; i < n; i++) {
DataInputStream din = reader.getMetaBlock("TfileMeta" + i);
byte b[] = new byte[len];
din.readFully(b);
assertTrue("faield to match metadata", Arrays.equals(
("something to test" + i).getBytes(), b));
din.close();
}
}
private void someReadingWithMetaBlock(Reader reader) throws IOException {
DataInputStream din = null;
readNumMetablocks(reader, 10);
try {
din = reader.getMetaBlock("NO ONE");
assertTrue(false);
}
catch (MetaBlockDoesNotExist me) {
// should catch
}
din = reader.getMetaBlock("TFileMeta100");
int read = din.read();
assertTrue("check for status", (read == -1));
din.close();
}
// test meta blocks for tfiles
public void testMetaBlocks() throws IOException {
Path mFile = new Path(ROOT, "meta.tfile");
FSDataOutputStream fout = createFSOutput(mFile);
Writer writer = new Writer(fout, minBlockSize, "none", null, conf);
someTestingWithMetaBlock(writer, "none");
writer.close();
fout.close();
FSDataInputStream fin = fs.open(mFile);
Reader reader = new Reader(fin, fs.getFileStatus(mFile).getLen(), conf);
someReadingWithMetaBlock(reader);
fs.delete(mFile, true);
reader.close();
fin.close();
}
}

View File

@ -0,0 +1,790 @@
/**
* 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.DataInputStream;
import java.io.DataOutputStream;
import java.io.EOFException;
import java.io.IOException;
import java.util.Random;
import junit.framework.Assert;
import junit.framework.TestCase;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.WritableUtils;
import org.apache.hadoop.io.file.tfile.TFile.Reader;
import org.apache.hadoop.io.file.tfile.TFile.Writer;
import org.apache.hadoop.io.file.tfile.TFile.Reader.Location;
import org.apache.hadoop.io.file.tfile.TFile.Reader.Scanner;
/**
*
* Byte arrays test case class using GZ compression codec, base class of none
* and LZO compression classes.
*
*/
public class TestTFileByteArrays extends TestCase {
private static String ROOT =
System.getProperty("test.build.data", "/tmp/tfile-test");
private final static int BLOCK_SIZE = 512;
private final static int BUF_SIZE = 64;
private final static int K = 1024;
protected boolean skip = false;
private static final String KEY = "key";
private static final String VALUE = "value";
private FileSystem fs;
private Configuration conf;
private Path path;
private FSDataOutputStream out;
private Writer writer;
private String compression = Compression.Algorithm.GZ.getName();
private String comparator = "memcmp";
private String outputFile = "TFileTestByteArrays";
/*
* pre-sampled numbers of records in one block, based on the given the
* generated key and value strings
*/
// private int records1stBlock = 4314;
// private int records2ndBlock = 4108;
private int records1stBlock = 4480;
private int records2ndBlock = 4263;
public void init(String compression, String comparator, String outputFile,
int numRecords1stBlock, int numRecords2ndBlock) {
this.compression = compression;
this.comparator = comparator;
this.outputFile = outputFile;
this.records1stBlock = numRecords1stBlock;
this.records2ndBlock = numRecords2ndBlock;
}
@Override
public void setUp() throws IOException {
conf = new Configuration();
path = new Path(ROOT, outputFile);
fs = path.getFileSystem(conf);
out = fs.create(path);
writer = new Writer(out, BLOCK_SIZE, compression, comparator, conf);
}
@Override
public void tearDown() throws IOException {
if (!skip)
fs.delete(path, true);
}
public void testNoDataEntry() throws IOException {
if (skip)
return;
closeOutput();
Reader reader = new Reader(fs.open(path), fs.getFileStatus(path).getLen(), conf);
Assert.assertTrue(reader.isSorted());
Scanner scanner = reader.createScanner();
Assert.assertTrue(scanner.atEnd());
scanner.close();
reader.close();
}
public void testOneDataEntry() throws IOException {
if (skip)
return;
writeRecords(1);
readRecords(1);
checkBlockIndex(1, 0, 0);
readValueBeforeKey(1, 0);
readKeyWithoutValue(1, 0);
readValueWithoutKey(1, 0);
readKeyManyTimes(1, 0);
}
public void testTwoDataEntries() throws IOException {
if (skip)
return;
writeRecords(2);
readRecords(2);
}
/**
* Fill up exactly one block.
*
* @throws IOException
*/
public void testOneBlock() throws IOException {
if (skip)
return;
// just under one block
writeRecords(records1stBlock);
readRecords(records1stBlock);
// last key should be in the first block (block 0)
checkBlockIndex(records1stBlock, records1stBlock - 1, 0);
}
/**
* One block plus one record.
*
* @throws IOException
*/
public void testOneBlockPlusOneEntry() throws IOException {
if (skip)
return;
writeRecords(records1stBlock + 1);
readRecords(records1stBlock + 1);
checkBlockIndex(records1stBlock + 1, records1stBlock - 1, 0);
checkBlockIndex(records1stBlock + 1, records1stBlock, 1);
}
public void testTwoBlocks() throws IOException {
if (skip)
return;
writeRecords(records1stBlock + 5);
readRecords(records1stBlock + 5);
checkBlockIndex(records1stBlock + 5, records1stBlock + 4, 1);
}
public void testThreeBlocks() throws IOException {
if (skip)
return;
writeRecords(2 * records1stBlock + 5);
readRecords(2 * records1stBlock + 5);
checkBlockIndex(2 * records1stBlock + 5, 2 * records1stBlock + 4, 2);
// 1st key in file
readValueBeforeKey(2 * records1stBlock + 5, 0);
readKeyWithoutValue(2 * records1stBlock + 5, 0);
readValueWithoutKey(2 * records1stBlock + 5, 0);
readKeyManyTimes(2 * records1stBlock + 5, 0);
// last key in file
readValueBeforeKey(2 * records1stBlock + 5, 2 * records1stBlock + 4);
readKeyWithoutValue(2 * records1stBlock + 5, 2 * records1stBlock + 4);
readValueWithoutKey(2 * records1stBlock + 5, 2 * records1stBlock + 4);
readKeyManyTimes(2 * records1stBlock + 5, 2 * records1stBlock + 4);
// 1st key in mid block, verify block indexes then read
checkBlockIndex(2 * records1stBlock + 5, records1stBlock - 1, 0);
checkBlockIndex(2 * records1stBlock + 5, records1stBlock, 1);
readValueBeforeKey(2 * records1stBlock + 5, records1stBlock);
readKeyWithoutValue(2 * records1stBlock + 5, records1stBlock);
readValueWithoutKey(2 * records1stBlock + 5, records1stBlock);
readKeyManyTimes(2 * records1stBlock + 5, records1stBlock);
// last key in mid block, verify block indexes then read
checkBlockIndex(2 * records1stBlock + 5, records1stBlock + records2ndBlock
- 1, 1);
checkBlockIndex(2 * records1stBlock + 5, records1stBlock + records2ndBlock,
2);
readValueBeforeKey(2 * records1stBlock + 5, records1stBlock
+ records2ndBlock - 1);
readKeyWithoutValue(2 * records1stBlock + 5, records1stBlock
+ records2ndBlock - 1);
readValueWithoutKey(2 * records1stBlock + 5, records1stBlock
+ records2ndBlock - 1);
readKeyManyTimes(2 * records1stBlock + 5, records1stBlock + records2ndBlock
- 1);
// mid in mid block
readValueBeforeKey(2 * records1stBlock + 5, records1stBlock + 10);
readKeyWithoutValue(2 * records1stBlock + 5, records1stBlock + 10);
readValueWithoutKey(2 * records1stBlock + 5, records1stBlock + 10);
readKeyManyTimes(2 * records1stBlock + 5, records1stBlock + 10);
}
Location locate(Scanner scanner, byte[] key) throws IOException {
if (scanner.seekTo(key) == true) {
return scanner.currentLocation;
}
return scanner.endLocation;
}
public void testLocate() throws IOException {
if (skip)
return;
writeRecords(3 * records1stBlock);
Reader reader = new Reader(fs.open(path), fs.getFileStatus(path).getLen(), conf);
Scanner scanner = reader.createScanner();
Location loc2 =
locate(scanner, composeSortedKey(KEY, 3 * records1stBlock, 2)
.getBytes());
Location locLastIn1stBlock =
locate(scanner, composeSortedKey(KEY, 3 * records1stBlock,
records1stBlock - 1).getBytes());
Location locFirstIn2ndBlock =
locate(scanner, composeSortedKey(KEY, 3 * records1stBlock,
records1stBlock).getBytes());
Location locX = locate(scanner, "keyX".getBytes());
Assert.assertEquals(scanner.endLocation, locX);
scanner.close();
reader.close();
}
public void testFailureWriterNotClosed() throws IOException {
if (skip)
return;
Reader reader = null;
try {
reader = new Reader(fs.open(path), fs.getFileStatus(path).getLen(), conf);
Assert.fail("Cannot read before closing the writer.");
}
catch (IOException e) {
// noop, expecting exceptions
}
finally {
if (reader != null) {
reader.close();
}
}
}
public void testFailureWriteMetaBlocksWithSameName() throws IOException {
if (skip)
return;
writer.append("keyX".getBytes(), "valueX".getBytes());
// create a new metablock
DataOutputStream outMeta =
writer.prepareMetaBlock("testX", Compression.Algorithm.GZ.getName());
outMeta.write(123);
outMeta.write("foo".getBytes());
outMeta.close();
// add the same metablock
try {
DataOutputStream outMeta2 =
writer.prepareMetaBlock("testX", Compression.Algorithm.GZ.getName());
Assert.fail("Cannot create metablocks with the same name.");
}
catch (Exception e) {
// noop, expecting exceptions
}
closeOutput();
}
public void testFailureGetNonExistentMetaBlock() throws IOException {
if (skip)
return;
writer.append("keyX".getBytes(), "valueX".getBytes());
// create a new metablock
DataOutputStream outMeta =
writer.prepareMetaBlock("testX", Compression.Algorithm.GZ.getName());
outMeta.write(123);
outMeta.write("foo".getBytes());
outMeta.close();
closeOutput();
Reader reader = new Reader(fs.open(path), fs.getFileStatus(path).getLen(), conf);
DataInputStream mb = reader.getMetaBlock("testX");
Assert.assertNotNull(mb);
mb.close();
try {
DataInputStream mbBad = reader.getMetaBlock("testY");
Assert.assertNull(mbBad);
Assert.fail("Error on handling non-existent metablocks.");
}
catch (Exception e) {
// noop, expecting exceptions
}
reader.close();
}
public void testFailureWriteRecordAfterMetaBlock() throws IOException {
if (skip)
return;
// write a key/value first
writer.append("keyX".getBytes(), "valueX".getBytes());
// create a new metablock
DataOutputStream outMeta =
writer.prepareMetaBlock("testX", Compression.Algorithm.GZ.getName());
outMeta.write(123);
outMeta.write("dummy".getBytes());
outMeta.close();
// add more key/value
try {
writer.append("keyY".getBytes(), "valueY".getBytes());
Assert.fail("Cannot add key/value after start adding meta blocks.");
}
catch (Exception e) {
// noop, expecting exceptions
}
closeOutput();
}
public void testFailureReadValueManyTimes() throws IOException {
if (skip)
return;
writeRecords(5);
Reader reader = new Reader(fs.open(path), fs.getFileStatus(path).getLen(), conf);
Scanner scanner = reader.createScanner();
byte[] vbuf = new byte[BUF_SIZE];
int vlen = scanner.entry().getValueLength();
scanner.entry().getValue(vbuf);
Assert.assertEquals(new String(vbuf, 0, vlen), VALUE + 0);
try {
scanner.entry().getValue(vbuf);
Assert.fail("Cannot get the value mlutiple times.");
}
catch (Exception e) {
// noop, expecting exceptions
}
scanner.close();
reader.close();
}
public void testFailureBadCompressionCodec() throws IOException {
if (skip)
return;
closeOutput();
out = fs.create(path);
try {
writer = new Writer(out, BLOCK_SIZE, "BAD", comparator, conf);
Assert.fail("Error on handling invalid compression codecs.");
}
catch (Exception e) {
// noop, expecting exceptions
// e.printStackTrace();
}
}
public void testFailureOpenEmptyFile() throws IOException {
if (skip)
return;
closeOutput();
// create an absolutely empty file
path = new Path(fs.getWorkingDirectory(), outputFile);
out = fs.create(path);
out.close();
try {
Reader reader =
new Reader(fs.open(path), fs.getFileStatus(path).getLen(), conf);
Assert.fail("Error on handling empty files.");
}
catch (EOFException e) {
// noop, expecting exceptions
}
}
public void testFailureOpenRandomFile() throws IOException {
if (skip)
return;
closeOutput();
// create an random file
path = new Path(fs.getWorkingDirectory(), outputFile);
out = fs.create(path);
Random rand = new Random();
byte[] buf = new byte[K];
// fill with > 1MB data
for (int nx = 0; nx < K + 2; nx++) {
rand.nextBytes(buf);
out.write(buf);
}
out.close();
try {
Reader reader =
new Reader(fs.open(path), fs.getFileStatus(path).getLen(), conf);
Assert.fail("Error on handling random files.");
}
catch (IOException e) {
// noop, expecting exceptions
}
}
public void testFailureKeyLongerThan64K() throws IOException {
if (skip)
return;
byte[] buf = new byte[64 * K + 1];
Random rand = new Random();
rand.nextBytes(buf);
try {
writer.append(buf, "valueX".getBytes());
}
catch (IndexOutOfBoundsException e) {
// noop, expecting exceptions
}
closeOutput();
}
public void testFailureOutOfOrderKeys() throws IOException {
if (skip)
return;
try {
writer.append("keyM".getBytes(), "valueM".getBytes());
writer.append("keyA".getBytes(), "valueA".getBytes());
Assert.fail("Error on handling out of order keys.");
}
catch (Exception e) {
// noop, expecting exceptions
// e.printStackTrace();
}
closeOutput();
}
public void testFailureNegativeOffset() throws IOException {
if (skip)
return;
try {
writer.append("keyX".getBytes(), -1, 4, "valueX".getBytes(), 0, 6);
Assert.fail("Error on handling negative offset.");
}
catch (Exception e) {
// noop, expecting exceptions
}
closeOutput();
}
public void testFailureNegativeOffset_2() throws IOException {
if (skip)
return;
closeOutput();
Reader reader = new Reader(fs.open(path), fs.getFileStatus(path).getLen(), conf);
Scanner scanner = reader.createScanner();
try {
scanner.lowerBound("keyX".getBytes(), -1, 4);
Assert.fail("Error on handling negative offset.");
}
catch (Exception e) {
// noop, expecting exceptions
}
finally {
reader.close();
scanner.close();
}
closeOutput();
}
public void testFailureNegativeLength() throws IOException {
if (skip)
return;
try {
writer.append("keyX".getBytes(), 0, -1, "valueX".getBytes(), 0, 6);
Assert.fail("Error on handling negative length.");
}
catch (Exception e) {
// noop, expecting exceptions
}
closeOutput();
}
public void testFailureNegativeLength_2() throws IOException {
if (skip)
return;
closeOutput();
Reader reader = new Reader(fs.open(path), fs.getFileStatus(path).getLen(), conf);
Scanner scanner = reader.createScanner();
try {
scanner.lowerBound("keyX".getBytes(), 0, -1);
Assert.fail("Error on handling negative length.");
}
catch (Exception e) {
// noop, expecting exceptions
}
finally {
scanner.close();
reader.close();
}
closeOutput();
}
public void testFailureNegativeLength_3() throws IOException {
if (skip)
return;
writeRecords(3);
Reader reader =
new Reader(fs.open(path), fs.getFileStatus(path).getLen(), conf);
Scanner scanner = reader.createScanner();
try {
// test negative array offset
try {
scanner.seekTo("keyY".getBytes(), -1, 4);
Assert.fail("Failed to handle negative offset.");
} catch (Exception e) {
// noop, expecting exceptions
}
// test negative array length
try {
scanner.seekTo("keyY".getBytes(), 0, -2);
Assert.fail("Failed to handle negative key length.");
} catch (Exception e) {
// noop, expecting exceptions
}
} finally {
reader.close();
scanner.close();
}
}
public void testFailureCompressionNotWorking() throws IOException {
if (skip)
return;
long rawDataSize = writeRecords(10 * records1stBlock, false);
if (!compression.equalsIgnoreCase(Compression.Algorithm.NONE.getName())) {
Assert.assertTrue(out.getPos() < rawDataSize);
}
closeOutput();
}
public void testFailureFileWriteNotAt0Position() throws IOException {
if (skip)
return;
closeOutput();
out = fs.create(path);
out.write(123);
try {
writer = new Writer(out, BLOCK_SIZE, compression, comparator, conf);
Assert.fail("Failed to catch file write not at position 0.");
}
catch (Exception e) {
// noop, expecting exceptions
}
closeOutput();
}
private long writeRecords(int count) throws IOException {
return writeRecords(count, true);
}
private long writeRecords(int count, boolean close) throws IOException {
long rawDataSize = writeRecords(writer, count);
if (close) {
closeOutput();
}
return rawDataSize;
}
static long writeRecords(Writer writer, int count) throws IOException {
long rawDataSize = 0;
int nx;
for (nx = 0; nx < count; nx++) {
byte[] key = composeSortedKey(KEY, count, nx).getBytes();
byte[] value = (VALUE + nx).getBytes();
writer.append(key, value);
rawDataSize +=
WritableUtils.getVIntSize(key.length) + key.length
+ WritableUtils.getVIntSize(value.length) + value.length;
}
return rawDataSize;
}
/**
* Insert some leading 0's in front of the value, to make the keys sorted.
*
* @param prefix
* @param total
* @param value
* @return
*/
static String composeSortedKey(String prefix, int total, int value) {
return String.format("%s%010d", prefix, value);
}
/**
* Calculate how many digits are in the 10-based integer.
*
* @param value
* @return
*/
private static int numberDigits(int value) {
int digits = 0;
while ((value = value / 10) > 0) {
digits++;
}
return digits;
}
private void readRecords(int count) throws IOException {
readRecords(fs, path, count, conf);
}
static void readRecords(FileSystem fs, Path path, int count,
Configuration conf) throws IOException {
Reader reader =
new Reader(fs.open(path), fs.getFileStatus(path).getLen(), conf);
Scanner scanner = reader.createScanner();
try {
for (int nx = 0; nx < count; nx++, scanner.advance()) {
Assert.assertFalse(scanner.atEnd());
// Assert.assertTrue(scanner.next());
byte[] kbuf = new byte[BUF_SIZE];
int klen = scanner.entry().getKeyLength();
scanner.entry().getKey(kbuf);
Assert.assertEquals(new String(kbuf, 0, klen), composeSortedKey(KEY,
count, nx));
byte[] vbuf = new byte[BUF_SIZE];
int vlen = scanner.entry().getValueLength();
scanner.entry().getValue(vbuf);
Assert.assertEquals(new String(vbuf, 0, vlen), VALUE + nx);
}
Assert.assertTrue(scanner.atEnd());
Assert.assertFalse(scanner.advance());
}
finally {
scanner.close();
reader.close();
}
}
private void checkBlockIndex(int count, int recordIndex,
int blockIndexExpected) throws IOException {
Reader reader = new Reader(fs.open(path), fs.getFileStatus(path).getLen(), conf);
Scanner scanner = reader.createScanner();
scanner.seekTo(composeSortedKey(KEY, count, recordIndex).getBytes());
Assert.assertEquals(blockIndexExpected, scanner.currentLocation
.getBlockIndex());
scanner.close();
reader.close();
}
private void readValueBeforeKey(int count, int recordIndex)
throws IOException {
Reader reader =
new Reader(fs.open(path), fs.getFileStatus(path).getLen(), conf);
Scanner scanner =
reader.createScanner(composeSortedKey(KEY, count, recordIndex)
.getBytes(), null);
try {
byte[] vbuf = new byte[BUF_SIZE];
int vlen = scanner.entry().getValueLength();
scanner.entry().getValue(vbuf);
Assert.assertEquals(new String(vbuf, 0, vlen), VALUE + recordIndex);
byte[] kbuf = new byte[BUF_SIZE];
int klen = scanner.entry().getKeyLength();
scanner.entry().getKey(kbuf);
Assert.assertEquals(new String(kbuf, 0, klen), composeSortedKey(KEY,
count, recordIndex));
}
finally {
scanner.close();
reader.close();
}
}
private void readKeyWithoutValue(int count, int recordIndex)
throws IOException {
Reader reader = new Reader(fs.open(path), fs.getFileStatus(path).getLen(), conf);
Scanner scanner =
reader.createScanner(composeSortedKey(KEY, count, recordIndex)
.getBytes(), null);
try {
// read the indexed key
byte[] kbuf1 = new byte[BUF_SIZE];
int klen1 = scanner.entry().getKeyLength();
scanner.entry().getKey(kbuf1);
Assert.assertEquals(new String(kbuf1, 0, klen1), composeSortedKey(KEY,
count, recordIndex));
if (scanner.advance() && !scanner.atEnd()) {
// read the next key following the indexed
byte[] kbuf2 = new byte[BUF_SIZE];
int klen2 = scanner.entry().getKeyLength();
scanner.entry().getKey(kbuf2);
Assert.assertEquals(new String(kbuf2, 0, klen2), composeSortedKey(KEY,
count, recordIndex + 1));
}
}
finally {
scanner.close();
reader.close();
}
}
private void readValueWithoutKey(int count, int recordIndex)
throws IOException {
Reader reader = new Reader(fs.open(path), fs.getFileStatus(path).getLen(), conf);
Scanner scanner =
reader.createScanner(composeSortedKey(KEY, count, recordIndex)
.getBytes(), null);
byte[] vbuf1 = new byte[BUF_SIZE];
int vlen1 = scanner.entry().getValueLength();
scanner.entry().getValue(vbuf1);
Assert.assertEquals(new String(vbuf1, 0, vlen1), VALUE + recordIndex);
if (scanner.advance() && !scanner.atEnd()) {
byte[] vbuf2 = new byte[BUF_SIZE];
int vlen2 = scanner.entry().getValueLength();
scanner.entry().getValue(vbuf2);
Assert.assertEquals(new String(vbuf2, 0, vlen2), VALUE
+ (recordIndex + 1));
}
scanner.close();
reader.close();
}
private void readKeyManyTimes(int count, int recordIndex) throws IOException {
Reader reader = new Reader(fs.open(path), fs.getFileStatus(path).getLen(), conf);
Scanner scanner =
reader.createScanner(composeSortedKey(KEY, count, recordIndex)
.getBytes(), null);
// read the indexed key
byte[] kbuf1 = new byte[BUF_SIZE];
int klen1 = scanner.entry().getKeyLength();
scanner.entry().getKey(kbuf1);
Assert.assertEquals(new String(kbuf1, 0, klen1), composeSortedKey(KEY,
count, recordIndex));
klen1 = scanner.entry().getKeyLength();
scanner.entry().getKey(kbuf1);
Assert.assertEquals(new String(kbuf1, 0, klen1), composeSortedKey(KEY,
count, recordIndex));
klen1 = scanner.entry().getKeyLength();
scanner.entry().getKey(kbuf1);
Assert.assertEquals(new String(kbuf1, 0, klen1), composeSortedKey(KEY,
count, recordIndex));
scanner.close();
reader.close();
}
private void closeOutput() throws IOException {
if (writer != null) {
writer.close();
writer = null;
}
if (out != null) {
out.close();
out = null;
}
}
}

View File

@ -0,0 +1,122 @@
/**
* 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.IOException;
import junit.framework.Assert;
import junit.framework.TestCase;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.file.tfile.TFile.Writer;
/**
*
* Byte arrays test case class using GZ compression codec, base class of none
* and LZO compression classes.
*
*/
public class TestTFileComparators extends TestCase {
private static String ROOT =
System.getProperty("test.build.data", "/tmp/tfile-test");
private final static int BLOCK_SIZE = 512;
private FileSystem fs;
private Configuration conf;
private Path path;
private FSDataOutputStream out;
private Writer writer;
private String compression = Compression.Algorithm.GZ.getName();
private String outputFile = "TFileTestComparators";
/*
* pre-sampled numbers of records in one block, based on the given the
* generated key and value strings
*/
// private int records1stBlock = 4314;
// private int records2ndBlock = 4108;
private int records1stBlock = 4480;
private int records2ndBlock = 4263;
@Override
public void setUp() throws IOException {
conf = new Configuration();
path = new Path(ROOT, outputFile);
fs = path.getFileSystem(conf);
out = fs.create(path);
}
@Override
public void tearDown() throws IOException {
fs.delete(path, true);
}
// bad comparator format
public void testFailureBadComparatorNames() throws IOException {
try {
writer = new Writer(out, BLOCK_SIZE, compression, "badcmp", conf);
Assert.fail("Failed to catch unsupported comparator names");
}
catch (Exception e) {
// noop, expecting exceptions
e.printStackTrace();
}
}
// jclass that doesn't exist
public void testFailureBadJClassNames() throws IOException {
try {
writer =
new Writer(out, BLOCK_SIZE, compression,
"jclass: some.non.existence.clazz", conf);
Assert.fail("Failed to catch unsupported comparator names");
}
catch (Exception e) {
// noop, expecting exceptions
e.printStackTrace();
}
}
// class exists but not a RawComparator
public void testFailureBadJClasses() throws IOException {
try {
writer =
new Writer(out, BLOCK_SIZE, compression,
"jclass:org.apache.hadoop.io.file.tfile.Chunk", conf);
Assert.fail("Failed to catch unsupported comparator names");
}
catch (Exception e) {
// noop, expecting exceptions
e.printStackTrace();
}
}
private void closeOutput() throws IOException {
if (writer != null) {
writer.close();
writer = null;
}
if (out != null) {
out.close();
out = null;
}
}
}

View File

@ -0,0 +1,58 @@
/**
* 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.IOException;
import org.apache.hadoop.io.RawComparator;
import org.apache.hadoop.io.WritableComparator;
/**
*
* Byte arrays test case class using GZ compression codec, base class of none
* and LZO compression classes.
*
*/
public class TestTFileJClassComparatorByteArrays extends TestTFileByteArrays {
/**
* Test non-compression codec, using the same test cases as in the ByteArrays.
*/
@Override
public void setUp() throws IOException {
init(Compression.Algorithm.GZ.getName(),
"jclass: org.apache.hadoop.io.file.tfile.MyComparator",
"TFileTestJClassComparator", 4480, 4263);
super.setUp();
}
}
class MyComparator implements RawComparator<byte[]> {
@Override
public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
return WritableComparator.compareBytes(b1, s1, l1, b2, s2, l2);
}
@Override
public int compare(byte[] o1, byte[] o2) {
return WritableComparator.compareBytes(o1, 0, o1.length, o2, 0, o2.length);
}
}

View File

@ -0,0 +1,42 @@
/**
* 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.IOException;
import org.apache.hadoop.io.file.tfile.Compression.Algorithm;
public class TestTFileLzoCodecsByteArrays extends TestTFileByteArrays {
/**
* Test LZO compression codec, using the same test cases as in the ByteArrays.
*/
@Override
public void setUp() throws IOException {
skip = !(Algorithm.LZO.isSupported());
if (skip) {
System.out.println("Skipped");
}
// TODO: sample the generated key/value records, and put the numbers below
init(Compression.Algorithm.LZO.getName(), "memcmp", "TFileTestCodecsLzo",
2605, 2558);
if (!skip)
super.setUp();
}
}

View File

@ -0,0 +1,39 @@
/**
* 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.IOException;
import org.apache.hadoop.io.file.tfile.Compression.Algorithm;
public class TestTFileLzoCodecsStreams extends TestTFileStreams {
/**
* Test LZO compression codec, using the same test cases as in the ByteArrays.
*/
@Override
public void setUp() throws IOException {
skip = !(Algorithm.LZO.isSupported());
if (skip) {
System.out.println("Skipped");
}
init(Compression.Algorithm.LZO.getName(), "memcmp", "TFileTestCodecsLzo");
if (!skip)
super.setUp();
}
}

View File

@ -0,0 +1,32 @@
/**
* 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.IOException;
public class TestTFileNoneCodecsByteArrays extends TestTFileByteArrays {
/**
* Test non-compression codec, using the same test cases as in the ByteArrays.
*/
@Override
public void setUp() throws IOException {
init(Compression.Algorithm.NONE.getName(), "memcmp", "TFileTestCodecsNone",
24, 24);
super.setUp();
}
}

View File

@ -0,0 +1,43 @@
/**
* 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.IOException;
import org.apache.hadoop.io.RawComparator;
import org.apache.hadoop.io.WritableComparator;
/**
*
* Byte arrays test case class using GZ compression codec, base class of none
* and LZO compression classes.
*
*/
public class TestTFileNoneCodecsJClassComparatorByteArrays extends TestTFileByteArrays {
/**
* Test non-compression codec, using the same test cases as in the ByteArrays.
*/
@Override
public void setUp() throws IOException {
init(Compression.Algorithm.NONE.getName(),
"jclass: org.apache.hadoop.io.file.tfile.MyComparator",
"TestTFileNoneCodecsJClassComparatorByteArrays", 24, 24);
super.setUp();
}
}

View File

@ -0,0 +1,32 @@
/**
* 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.IOException;
public class TestTFileNoneCodecsStreams extends TestTFileStreams {
/**
* Test non-compression codec, using the same test cases as in the ByteArrays.
*/
@Override
public void setUp() throws IOException {
init(Compression.Algorithm.NONE.getName(), "memcmp", "TFileTestCodecsNone");
super.setUp();
}
}

View File

@ -0,0 +1,504 @@
/**
* 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.IOException;
import java.util.Random;
import java.util.StringTokenizer;
import junit.framework.TestCase;
import org.apache.commons.cli.CommandLine;
import org.apache.commons.cli.CommandLineParser;
import org.apache.commons.cli.GnuParser;
import org.apache.commons.cli.HelpFormatter;
import org.apache.commons.cli.Option;
import org.apache.commons.cli.OptionBuilder;
import org.apache.commons.cli.Options;
import org.apache.commons.cli.ParseException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.BytesWritable;
import org.apache.hadoop.io.file.tfile.RandomDistribution.DiscreteRNG;
import org.apache.hadoop.io.file.tfile.TFile.Reader;
import org.apache.hadoop.io.file.tfile.TFile.Writer;
import org.apache.hadoop.io.file.tfile.TFile.Reader.Scanner;
/**
* test the performance for seek.
*
*/
public class TestTFileSeek extends TestCase {
private MyOptions options;
private Configuration conf;
private Path path;
private FileSystem fs;
private NanoTimer timer;
private Random rng;
private DiscreteRNG keyLenGen;
private KVGenerator kvGen;
@Override
public void setUp() throws IOException {
if (options == null) {
options = new MyOptions(new String[0]);
}
conf = new Configuration();
conf.setInt("tfile.fs.input.buffer.size", options.fsInputBufferSize);
conf.setInt("tfile.fs.output.buffer.size", options.fsOutputBufferSize);
path = new Path(new Path(options.rootDir), options.file);
fs = path.getFileSystem(conf);
timer = new NanoTimer(false);
rng = new Random(options.seed);
keyLenGen =
new RandomDistribution.Zipf(new Random(rng.nextLong()),
options.minKeyLen, options.maxKeyLen, 1.2);
DiscreteRNG valLenGen =
new RandomDistribution.Flat(new Random(rng.nextLong()),
options.minValLength, options.maxValLength);
DiscreteRNG wordLenGen =
new RandomDistribution.Flat(new Random(rng.nextLong()),
options.minWordLen, options.maxWordLen);
kvGen =
new KVGenerator(rng, true, keyLenGen, valLenGen, wordLenGen,
options.dictSize);
}
@Override
public void tearDown() throws IOException {
fs.delete(path, true);
}
private static FSDataOutputStream createFSOutput(Path name, FileSystem fs)
throws IOException {
if (fs.exists(name)) {
fs.delete(name, true);
}
FSDataOutputStream fout = fs.create(name);
return fout;
}
private void createTFile() throws IOException {
long totalBytes = 0;
FSDataOutputStream fout = createFSOutput(path, fs);
try {
Writer writer =
new Writer(fout, options.minBlockSize, options.compress, "memcmp",
conf);
try {
BytesWritable key = new BytesWritable();
BytesWritable val = new BytesWritable();
timer.start();
for (long i = 0; true; ++i) {
if (i % 1000 == 0) { // test the size for every 1000 rows.
if (fs.getFileStatus(path).getLen() >= options.fileSize) {
break;
}
}
kvGen.next(key, val, false);
writer.append(key.get(), 0, key.getSize(), val.get(), 0, val
.getSize());
totalBytes += key.getSize();
totalBytes += val.getSize();
}
timer.stop();
}
finally {
writer.close();
}
}
finally {
fout.close();
}
double duration = (double)timer.read()/1000; // in us.
long fsize = fs.getFileStatus(path).getLen();
System.out.printf(
"time: %s...uncompressed: %.2fMB...raw thrpt: %.2fMB/s\n",
timer.toString(), (double) totalBytes / 1024 / 1024, totalBytes
/ duration);
System.out.printf("time: %s...file size: %.2fMB...disk thrpt: %.2fMB/s\n",
timer.toString(), (double) fsize / 1024 / 1024, fsize / duration);
}
public void seekTFile() throws IOException {
int miss = 0;
long totalBytes = 0;
FSDataInputStream fsdis = fs.open(path);
Reader reader =
new Reader(fsdis, fs.getFileStatus(path).getLen(), conf);
KeySampler kSampler =
new KeySampler(rng, reader.getFirstKey(), reader.getLastKey(),
keyLenGen);
Scanner scanner = reader.createScanner();
BytesWritable key = new BytesWritable();
BytesWritable val = new BytesWritable();
timer.reset();
timer.start();
for (int i = 0; i < options.seekCount; ++i) {
kSampler.next(key);
scanner.lowerBound(key.get(), 0, key.getSize());
if (!scanner.atEnd()) {
scanner.entry().get(key, val);
totalBytes += key.getSize();
totalBytes += val.getSize();
}
else {
++miss;
}
}
timer.stop();
double duration = (double) timer.read() / 1000; // in us.
System.out.printf(
"time: %s...avg seek: %s...%d hit...%d miss...avg I/O size: %.2fKB\n",
timer.toString(), NanoTimer.nanoTimeToString(timer.read()
/ options.seekCount), options.seekCount - miss, miss,
(double) totalBytes / 1024 / (options.seekCount - miss));
}
public void testSeeks() throws IOException {
String[] supported = TFile.getSupportedCompressionAlgorithms();
boolean proceed = false;
for (String c : supported) {
if (c.equals(options.compress)) {
proceed = true;
break;
}
}
if (!proceed) {
System.out.println("Skipped for " + options.compress);
return;
}
if (options.doCreate()) {
createTFile();
}
if (options.doRead()) {
seekTFile();
}
}
private static class IntegerRange {
private final int from, to;
public IntegerRange(int from, int to) {
this.from = from;
this.to = to;
}
public static IntegerRange parse(String s) throws ParseException {
StringTokenizer st = new StringTokenizer(s, " \t,");
if (st.countTokens() != 2) {
throw new ParseException("Bad integer specification: " + s);
}
int from = Integer.parseInt(st.nextToken());
int to = Integer.parseInt(st.nextToken());
return new IntegerRange(from, to);
}
public int from() {
return from;
}
public int to() {
return to;
}
}
private static class MyOptions {
// hard coded constants
int dictSize = 1000;
int minWordLen = 5;
int maxWordLen = 20;
int osInputBufferSize = 64 * 1024;
int osOutputBufferSize = 64 * 1024;
int fsInputBufferSizeNone = 0;
int fsInputBufferSizeLzo = 0;
int fsInputBufferSizeGz = 0;
int fsOutputBufferSizeNone = 1;
int fsOutputBufferSizeLzo = 1;
int fsOutputBufferSizeGz = 1;
String rootDir =
System.getProperty("test.build.data", "/tmp/tfile-test");
String file = "TestTFileSeek";
String compress = "gz";
int minKeyLen = 10;
int maxKeyLen = 50;
int minValLength = 100;
int maxValLength = 200;
int minBlockSize = 64 * 1024;
int fsOutputBufferSize = 1;
int fsInputBufferSize = 0;
long fileSize = 3 * 1024 * 1024;
long seekCount = 1000;
long seed;
static final int OP_CREATE = 1;
static final int OP_READ = 2;
int op = OP_CREATE | OP_READ;
boolean proceed = false;
public MyOptions(String[] args) {
seed = System.nanoTime();
try {
Options opts = buildOptions();
CommandLineParser parser = new GnuParser();
CommandLine line = parser.parse(opts, args, true);
processOptions(line, opts);
validateOptions();
}
catch (ParseException e) {
System.out.println(e.getMessage());
System.out.println("Try \"--help\" option for details.");
setStopProceed();
}
}
public boolean proceed() {
return proceed;
}
private Options buildOptions() {
Option compress =
OptionBuilder.withLongOpt("compress").withArgName("[none|lzo|gz]")
.hasArg().withDescription("compression scheme").create('c');
Option fileSize =
OptionBuilder.withLongOpt("file-size").withArgName("size-in-MB")
.hasArg().withDescription("target size of the file (in MB).")
.create('s');
Option fsInputBufferSz =
OptionBuilder.withLongOpt("fs-input-buffer").withArgName("size")
.hasArg().withDescription(
"size of the file system input buffer (in bytes).").create(
'i');
Option fsOutputBufferSize =
OptionBuilder.withLongOpt("fs-output-buffer").withArgName("size")
.hasArg().withDescription(
"size of the file system output buffer (in bytes).").create(
'o');
Option keyLen =
OptionBuilder
.withLongOpt("key-length")
.withArgName("min,max")
.hasArg()
.withDescription(
"the length range of the key (in bytes)")
.create('k');
Option valueLen =
OptionBuilder
.withLongOpt("value-length")
.withArgName("min,max")
.hasArg()
.withDescription(
"the length range of the value (in bytes)")
.create('v');
Option blockSz =
OptionBuilder.withLongOpt("block").withArgName("size-in-KB").hasArg()
.withDescription("minimum block size (in KB)").create('b');
Option seed =
OptionBuilder.withLongOpt("seed").withArgName("long-int").hasArg()
.withDescription("specify the seed").create('S');
Option operation =
OptionBuilder.withLongOpt("operation").withArgName("r|w|rw").hasArg()
.withDescription(
"action: seek-only, create-only, seek-after-create").create(
'x');
Option rootDir =
OptionBuilder.withLongOpt("root-dir").withArgName("path").hasArg()
.withDescription(
"specify root directory where files will be created.")
.create('r');
Option file =
OptionBuilder.withLongOpt("file").withArgName("name").hasArg()
.withDescription("specify the file name to be created or read.")
.create('f');
Option seekCount =
OptionBuilder
.withLongOpt("seek")
.withArgName("count")
.hasArg()
.withDescription(
"specify how many seek operations we perform (requires -x r or -x rw.")
.create('n');
Option help =
OptionBuilder.withLongOpt("help").hasArg(false).withDescription(
"show this screen").create("h");
return new Options().addOption(compress).addOption(fileSize).addOption(
fsInputBufferSz).addOption(fsOutputBufferSize).addOption(keyLen)
.addOption(blockSz).addOption(rootDir).addOption(valueLen).addOption(
operation).addOption(seekCount).addOption(file).addOption(help);
}
private void processOptions(CommandLine line, Options opts)
throws ParseException {
// --help -h and --version -V must be processed first.
if (line.hasOption('h')) {
HelpFormatter formatter = new HelpFormatter();
System.out.println("TFile and SeqFile benchmark.");
System.out.println();
formatter.printHelp(100,
"java ... TestTFileSeqFileComparison [options]",
"\nSupported options:", opts, "");
return;
}
if (line.hasOption('c')) {
compress = line.getOptionValue('c');
}
if (line.hasOption('d')) {
dictSize = Integer.parseInt(line.getOptionValue('d'));
}
if (line.hasOption('s')) {
fileSize = Long.parseLong(line.getOptionValue('s')) * 1024 * 1024;
}
if (line.hasOption('i')) {
fsInputBufferSize = Integer.parseInt(line.getOptionValue('i'));
}
if (line.hasOption('o')) {
fsOutputBufferSize = Integer.parseInt(line.getOptionValue('o'));
}
if (line.hasOption('n')) {
seekCount = Integer.parseInt(line.getOptionValue('n'));
}
if (line.hasOption('k')) {
IntegerRange ir = IntegerRange.parse(line.getOptionValue('k'));
minKeyLen = ir.from();
maxKeyLen = ir.to();
}
if (line.hasOption('v')) {
IntegerRange ir = IntegerRange.parse(line.getOptionValue('v'));
minValLength = ir.from();
maxValLength = ir.to();
}
if (line.hasOption('b')) {
minBlockSize = Integer.parseInt(line.getOptionValue('b')) * 1024;
}
if (line.hasOption('r')) {
rootDir = line.getOptionValue('r');
}
if (line.hasOption('f')) {
file = line.getOptionValue('f');
}
if (line.hasOption('S')) {
seed = Long.parseLong(line.getOptionValue('S'));
}
if (line.hasOption('x')) {
String strOp = line.getOptionValue('x');
if (strOp.equals("r")) {
op = OP_READ;
}
else if (strOp.equals("w")) {
op = OP_CREATE;
}
else if (strOp.equals("rw")) {
op = OP_CREATE | OP_READ;
}
else {
throw new ParseException("Unknown action specifier: " + strOp);
}
}
proceed = true;
}
private void validateOptions() throws ParseException {
if (!compress.equals("none") && !compress.equals("lzo")
&& !compress.equals("gz")) {
throw new ParseException("Unknown compression scheme: " + compress);
}
if (minKeyLen >= maxKeyLen) {
throw new ParseException(
"Max key length must be greater than min key length.");
}
if (minValLength >= maxValLength) {
throw new ParseException(
"Max value length must be greater than min value length.");
}
if (minWordLen >= maxWordLen) {
throw new ParseException(
"Max word length must be greater than min word length.");
}
return;
}
private void setStopProceed() {
proceed = false;
}
public boolean doCreate() {
return (op & OP_CREATE) != 0;
}
public boolean doRead() {
return (op & OP_READ) != 0;
}
}
public static void main(String[] argv) throws IOException {
TestTFileSeek testCase = new TestTFileSeek();
MyOptions options = new MyOptions(argv);
if (options.proceed == false) {
return;
}
testCase.options = options;
testCase.setUp();
testCase.testSeeks();
testCase.tearDown();
}
}

View File

@ -0,0 +1,782 @@
/**
* 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.IOException;
import java.text.DateFormat;
import java.text.SimpleDateFormat;
import java.util.Random;
import java.util.StringTokenizer;
import junit.framework.TestCase;
import org.apache.commons.cli.CommandLine;
import org.apache.commons.cli.CommandLineParser;
import org.apache.commons.cli.GnuParser;
import org.apache.commons.cli.HelpFormatter;
import org.apache.commons.cli.Option;
import org.apache.commons.cli.OptionBuilder;
import org.apache.commons.cli.Options;
import org.apache.commons.cli.ParseException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.BytesWritable;
import org.apache.hadoop.io.SequenceFile;
import org.apache.hadoop.io.compress.CompressionCodec;
import org.apache.hadoop.io.file.tfile.TFile.Reader.Scanner.Entry;
public class TestTFileSeqFileComparison extends TestCase {
MyOptions options;
private FileSystem fs;
private Configuration conf;
private long startTimeEpoch;
private long finishTimeEpoch;
private DateFormat formatter;
byte[][] dictionary;
@Override
public void setUp() throws IOException {
if (options == null) {
options = new MyOptions(new String[0]);
}
conf = new Configuration();
conf.setInt("tfile.fs.input.buffer.size", options.fsInputBufferSize);
conf.setInt("tfile.fs.output.buffer.size", options.fsOutputBufferSize);
Path path = new Path(options.rootDir);
fs = path.getFileSystem(conf);
formatter = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
setUpDictionary();
}
private void setUpDictionary() {
Random rng = new Random();
dictionary = new byte[options.dictSize][];
for (int i = 0; i < options.dictSize; ++i) {
int len =
rng.nextInt(options.maxWordLen - options.minWordLen)
+ options.minWordLen;
dictionary[i] = new byte[len];
rng.nextBytes(dictionary[i]);
}
}
@Override
public void tearDown() throws IOException {
// do nothing
}
public void startTime() throws IOException {
startTimeEpoch = System.currentTimeMillis();
System.out.println(formatTime() + " Started timing.");
}
public void stopTime() throws IOException {
finishTimeEpoch = System.currentTimeMillis();
System.out.println(formatTime() + " Stopped timing.");
}
public long getIntervalMillis() throws IOException {
return finishTimeEpoch - startTimeEpoch;
}
public void printlnWithTimestamp(String message) throws IOException {
System.out.println(formatTime() + " " + message);
}
/*
* Format millis into minutes and seconds.
*/
public String formatTime(long milis) {
return formatter.format(milis);
}
public String formatTime() {
return formatTime(System.currentTimeMillis());
}
private interface KVAppendable {
public void append(BytesWritable key, BytesWritable value)
throws IOException;
public void close() throws IOException;
}
private interface KVReadable {
public byte[] getKey();
public byte[] getValue();
public int getKeyLength();
public int getValueLength();
public boolean next() throws IOException;
public void close() throws IOException;
}
static class TFileAppendable implements KVAppendable {
private FSDataOutputStream fsdos;
private TFile.Writer writer;
public TFileAppendable(FileSystem fs, Path path, String compress,
int minBlkSize, int osBufferSize, Configuration conf)
throws IOException {
this.fsdos = fs.create(path, true, osBufferSize);
this.writer = new TFile.Writer(fsdos, minBlkSize, compress, null, conf);
}
public void append(BytesWritable key, BytesWritable value)
throws IOException {
writer.append(key.get(), 0, key.getSize(), value.get(), 0, value
.getSize());
}
public void close() throws IOException {
writer.close();
fsdos.close();
}
}
static class TFileReadable implements KVReadable {
private FSDataInputStream fsdis;
private TFile.Reader reader;
private TFile.Reader.Scanner scanner;
private byte[] keyBuffer;
private int keyLength;
private byte[] valueBuffer;
private int valueLength;
public TFileReadable(FileSystem fs, Path path, int osBufferSize,
Configuration conf) throws IOException {
this.fsdis = fs.open(path, osBufferSize);
this.reader =
new TFile.Reader(fsdis, fs.getFileStatus(path).getLen(), conf);
this.scanner = reader.createScanner();
keyBuffer = new byte[32];
valueBuffer = new byte[32];
}
private void checkKeyBuffer(int size) {
if (size <= keyBuffer.length) {
return;
}
keyBuffer =
new byte[Math.max(2 * keyBuffer.length, 2 * size - keyBuffer.length)];
}
private void checkValueBuffer(int size) {
if (size <= valueBuffer.length) {
return;
}
valueBuffer =
new byte[Math.max(2 * valueBuffer.length, 2 * size
- valueBuffer.length)];
}
public byte[] getKey() {
return keyBuffer;
}
public int getKeyLength() {
return keyLength;
}
public byte[] getValue() {
return valueBuffer;
}
public int getValueLength() {
return valueLength;
}
public boolean next() throws IOException {
if (scanner.atEnd()) return false;
Entry entry = scanner.entry();
keyLength = entry.getKeyLength();
checkKeyBuffer(keyLength);
entry.getKey(keyBuffer);
valueLength = entry.getValueLength();
checkValueBuffer(valueLength);
entry.getValue(valueBuffer);
scanner.advance();
return true;
}
public void close() throws IOException {
scanner.close();
reader.close();
fsdis.close();
}
}
static class SeqFileAppendable implements KVAppendable {
private FSDataOutputStream fsdos;
private SequenceFile.Writer writer;
public SeqFileAppendable(FileSystem fs, Path path, int osBufferSize,
String compress, int minBlkSize) throws IOException {
Configuration conf = new Configuration();
conf.setBoolean("hadoop.native.lib", true);
CompressionCodec codec = null;
if ("lzo".equals(compress)) {
codec = Compression.Algorithm.LZO.getCodec();
}
else if ("gz".equals(compress)) {
codec = Compression.Algorithm.GZ.getCodec();
}
else if (!"none".equals(compress))
throw new IOException("Codec not supported.");
this.fsdos = fs.create(path, true, osBufferSize);
if (!"none".equals(compress)) {
writer =
SequenceFile.createWriter(conf, fsdos, BytesWritable.class,
BytesWritable.class, SequenceFile.CompressionType.BLOCK, codec);
}
else {
writer =
SequenceFile.createWriter(conf, fsdos, BytesWritable.class,
BytesWritable.class, SequenceFile.CompressionType.NONE, null);
}
}
public void append(BytesWritable key, BytesWritable value)
throws IOException {
writer.append(key, value);
}
public void close() throws IOException {
writer.close();
fsdos.close();
}
}
static class SeqFileReadable implements KVReadable {
private SequenceFile.Reader reader;
private BytesWritable key;
private BytesWritable value;
public SeqFileReadable(FileSystem fs, Path path, int osBufferSize)
throws IOException {
Configuration conf = new Configuration();
conf.setInt("io.file.buffer.size", osBufferSize);
reader = new SequenceFile.Reader(fs, path, conf);
key = new BytesWritable();
value = new BytesWritable();
}
public byte[] getKey() {
return key.get();
}
public int getKeyLength() {
return key.getSize();
}
public byte[] getValue() {
return value.get();
}
public int getValueLength() {
return value.getSize();
}
public boolean next() throws IOException {
return reader.next(key, value);
}
public void close() throws IOException {
reader.close();
}
}
private void reportStats(Path path, long totalBytes) throws IOException {
long duration = getIntervalMillis();
long fsize = fs.getFileStatus(path).getLen();
printlnWithTimestamp(String.format(
"Duration: %dms...total size: %.2fMB...raw thrpt: %.2fMB/s", duration,
(double) totalBytes / 1024 / 1024, (double) totalBytes / duration
* 1000 / 1024 / 1024));
printlnWithTimestamp(String.format(
"Compressed size: %.2fMB...compressed thrpt: %.2fMB/s.",
(double) fsize / 1024 / 1024, (double) fsize / duration * 1000 / 1024
/ 1024));
}
private void fillBuffer(Random rng, BytesWritable bw, byte[] tmp, int len) {
int n = 0;
while (n < len) {
byte[] word = dictionary[rng.nextInt(dictionary.length)];
int l = Math.min(word.length, len - n);
System.arraycopy(word, 0, tmp, n, l);
n += l;
}
bw.set(tmp, 0, len);
}
private void timeWrite(Path path, KVAppendable appendable, int baseKlen,
int baseVlen, long fileSize) throws IOException {
int maxKlen = baseKlen * 2;
int maxVlen = baseVlen * 2;
BytesWritable key = new BytesWritable();
BytesWritable value = new BytesWritable();
byte[] keyBuffer = new byte[maxKlen];
byte[] valueBuffer = new byte[maxVlen];
Random rng = new Random(options.seed);
long totalBytes = 0;
printlnWithTimestamp("Start writing: " + path.getName() + "...");
startTime();
for (long i = 0; true; ++i) {
if (i % 1000 == 0) { // test the size for every 1000 rows.
if (fs.getFileStatus(path).getLen() >= fileSize) {
break;
}
}
int klen = rng.nextInt(baseKlen) + baseKlen;
int vlen = rng.nextInt(baseVlen) + baseVlen;
fillBuffer(rng, key, keyBuffer, klen);
fillBuffer(rng, value, valueBuffer, vlen);
key.set(keyBuffer, 0, klen);
value.set(valueBuffer, 0, vlen);
appendable.append(key, value);
totalBytes += klen;
totalBytes += vlen;
}
stopTime();
appendable.close();
reportStats(path, totalBytes);
}
private void timeRead(Path path, KVReadable readable) throws IOException {
printlnWithTimestamp("Start reading: " + path.getName() + "...");
long totalBytes = 0;
startTime();
for (; readable.next();) {
totalBytes += readable.getKeyLength();
totalBytes += readable.getValueLength();
}
stopTime();
readable.close();
reportStats(path, totalBytes);
}
private void createTFile(String parameters, String compress)
throws IOException {
System.out.println("=== TFile: Creation (" + parameters + ") === ");
Path path = new Path(options.rootDir, "TFile.Performance");
KVAppendable appendable =
new TFileAppendable(fs, path, compress, options.minBlockSize,
options.osOutputBufferSize, conf);
timeWrite(path, appendable, options.keyLength, options.valueLength,
options.fileSize);
}
private void readTFile(String parameters, boolean delFile) throws IOException {
System.out.println("=== TFile: Reading (" + parameters + ") === ");
{
Path path = new Path(options.rootDir, "TFile.Performance");
KVReadable readable =
new TFileReadable(fs, path, options.osInputBufferSize, conf);
timeRead(path, readable);
if (delFile) {
if (fs.exists(path)) {
fs.delete(path, true);
}
}
}
}
private void createSeqFile(String parameters, String compress)
throws IOException {
System.out.println("=== SeqFile: Creation (" + parameters + ") === ");
Path path = new Path(options.rootDir, "SeqFile.Performance");
KVAppendable appendable =
new SeqFileAppendable(fs, path, options.osOutputBufferSize, compress,
options.minBlockSize);
timeWrite(path, appendable, options.keyLength, options.valueLength,
options.fileSize);
}
private void readSeqFile(String parameters, boolean delFile)
throws IOException {
System.out.println("=== SeqFile: Reading (" + parameters + ") === ");
Path path = new Path(options.rootDir, "SeqFile.Performance");
KVReadable readable =
new SeqFileReadable(fs, path, options.osInputBufferSize);
timeRead(path, readable);
if (delFile) {
if (fs.exists(path)) {
fs.delete(path, true);
}
}
}
private void compareRun(String compress) throws IOException {
String[] supported = TFile.getSupportedCompressionAlgorithms();
boolean proceed = false;
for (String c : supported) {
if (c.equals(compress)) {
proceed = true;
break;
}
}
if (!proceed) {
System.out.println("Skipped for " + compress);
return;
}
options.compress = compress;
String parameters = parameters2String(options);
createSeqFile(parameters, compress);
readSeqFile(parameters, true);
createTFile(parameters, compress);
readTFile(parameters, true);
createTFile(parameters, compress);
readTFile(parameters, true);
createSeqFile(parameters, compress);
readSeqFile(parameters, true);
}
public void testRunComparisons() throws IOException {
String[] compresses = new String[] { "none", "lzo", "gz" };
for (String compress : compresses) {
if (compress.equals("none")) {
conf
.setInt("tfile.fs.input.buffer.size", options.fsInputBufferSizeNone);
conf.setInt("tfile.fs.output.buffer.size",
options.fsOutputBufferSizeNone);
}
else if (compress.equals("lzo")) {
conf.setInt("tfile.fs.input.buffer.size", options.fsInputBufferSizeLzo);
conf.setInt("tfile.fs.output.buffer.size",
options.fsOutputBufferSizeLzo);
}
else {
conf.setInt("tfile.fs.input.buffer.size", options.fsInputBufferSizeGz);
conf
.setInt("tfile.fs.output.buffer.size", options.fsOutputBufferSizeGz);
}
compareRun(compress);
}
}
private static String parameters2String(MyOptions options) {
return String
.format(
"KLEN: %d-%d... VLEN: %d-%d...MinBlkSize: %.2fKB...Target Size: %.2fMB...Compression: ...%s",
options.keyLength, options.keyLength * 2, options.valueLength,
options.valueLength * 2, (double) options.minBlockSize / 1024,
(double) options.fileSize / 1024 / 1024, options.compress);
}
private static class MyOptions {
String rootDir =
System
.getProperty("test.build.data", "/tmp/tfile-test");
String compress = "gz";
String format = "tfile";
int dictSize = 1000;
int minWordLen = 5;
int maxWordLen = 20;
int keyLength = 50;
int valueLength = 100;
int minBlockSize = 256 * 1024;
int fsOutputBufferSize = 1;
int fsInputBufferSize = 0;
// special variable only for unit testing.
int fsInputBufferSizeNone = 0;
int fsInputBufferSizeGz = 0;
int fsInputBufferSizeLzo = 0;
int fsOutputBufferSizeNone = 1;
int fsOutputBufferSizeGz = 1;
int fsOutputBufferSizeLzo = 1;
// un-exposed parameters.
int osInputBufferSize = 64 * 1024;
int osOutputBufferSize = 64 * 1024;
long fileSize = 3 * 1024 * 1024;
long seed;
static final int OP_CREATE = 1;
static final int OP_READ = 2;
int op = OP_READ;
boolean proceed = false;
public MyOptions(String[] args) {
seed = System.nanoTime();
try {
Options opts = buildOptions();
CommandLineParser parser = new GnuParser();
CommandLine line = parser.parse(opts, args, true);
processOptions(line, opts);
validateOptions();
}
catch (ParseException e) {
System.out.println(e.getMessage());
System.out.println("Try \"--help\" option for details.");
setStopProceed();
}
}
public boolean proceed() {
return proceed;
}
private Options buildOptions() {
Option compress =
OptionBuilder.withLongOpt("compress").withArgName("[none|lzo|gz]")
.hasArg().withDescription("compression scheme").create('c');
Option ditSize =
OptionBuilder.withLongOpt("dict").withArgName("size").hasArg()
.withDescription("number of dictionary entries").create('d');
Option fileSize =
OptionBuilder.withLongOpt("file-size").withArgName("size-in-MB")
.hasArg().withDescription("target size of the file (in MB).")
.create('s');
Option format =
OptionBuilder.withLongOpt("format").withArgName("[tfile|seqfile]")
.hasArg().withDescription("choose TFile or SeqFile").create('f');
Option fsInputBufferSz =
OptionBuilder.withLongOpt("fs-input-buffer").withArgName("size")
.hasArg().withDescription(
"size of the file system input buffer (in bytes).").create(
'i');
Option fsOutputBufferSize =
OptionBuilder.withLongOpt("fs-output-buffer").withArgName("size")
.hasArg().withDescription(
"size of the file system output buffer (in bytes).").create(
'o');
Option keyLen =
OptionBuilder
.withLongOpt("key-length")
.withArgName("length")
.hasArg()
.withDescription(
"base length of the key (in bytes), actual length varies in [base, 2*base)")
.create('k');
Option valueLen =
OptionBuilder
.withLongOpt("value-length")
.withArgName("length")
.hasArg()
.withDescription(
"base length of the value (in bytes), actual length varies in [base, 2*base)")
.create('v');
Option wordLen =
OptionBuilder.withLongOpt("word-length").withArgName("min,max")
.hasArg().withDescription(
"range of dictionary word length (in bytes)").create('w');
Option blockSz =
OptionBuilder.withLongOpt("block").withArgName("size-in-KB").hasArg()
.withDescription("minimum block size (in KB)").create('b');
Option seed =
OptionBuilder.withLongOpt("seed").withArgName("long-int").hasArg()
.withDescription("specify the seed").create('S');
Option operation =
OptionBuilder.withLongOpt("operation").withArgName("r|w|rw").hasArg()
.withDescription(
"action: read-only, create-only, read-after-create").create(
'x');
Option rootDir =
OptionBuilder.withLongOpt("root-dir").withArgName("path").hasArg()
.withDescription(
"specify root directory where files will be created.")
.create('r');
Option help =
OptionBuilder.withLongOpt("help").hasArg(false).withDescription(
"show this screen").create("h");
return new Options().addOption(compress).addOption(ditSize).addOption(
fileSize).addOption(format).addOption(fsInputBufferSz).addOption(
fsOutputBufferSize).addOption(keyLen).addOption(wordLen).addOption(
blockSz).addOption(rootDir).addOption(valueLen).addOption(operation)
.addOption(help);
}
private void processOptions(CommandLine line, Options opts)
throws ParseException {
// --help -h and --version -V must be processed first.
if (line.hasOption('h')) {
HelpFormatter formatter = new HelpFormatter();
System.out.println("TFile and SeqFile benchmark.");
System.out.println();
formatter.printHelp(100,
"java ... TestTFileSeqFileComparison [options]",
"\nSupported options:", opts, "");
return;
}
if (line.hasOption('c')) {
compress = line.getOptionValue('c');
}
if (line.hasOption('d')) {
dictSize = Integer.parseInt(line.getOptionValue('d'));
}
if (line.hasOption('s')) {
fileSize = Long.parseLong(line.getOptionValue('s')) * 1024 * 1024;
}
if (line.hasOption('f')) {
format = line.getOptionValue('f');
}
if (line.hasOption('i')) {
fsInputBufferSize = Integer.parseInt(line.getOptionValue('i'));
}
if (line.hasOption('o')) {
fsOutputBufferSize = Integer.parseInt(line.getOptionValue('o'));
}
if (line.hasOption('k')) {
keyLength = Integer.parseInt(line.getOptionValue('k'));
}
if (line.hasOption('v')) {
valueLength = Integer.parseInt(line.getOptionValue('v'));
}
if (line.hasOption('b')) {
minBlockSize = Integer.parseInt(line.getOptionValue('b')) * 1024;
}
if (line.hasOption('r')) {
rootDir = line.getOptionValue('r');
}
if (line.hasOption('S')) {
seed = Long.parseLong(line.getOptionValue('S'));
}
if (line.hasOption('w')) {
String min_max = line.getOptionValue('w');
StringTokenizer st = new StringTokenizer(min_max, " \t,");
if (st.countTokens() != 2) {
throw new ParseException("Bad word length specification: " + min_max);
}
minWordLen = Integer.parseInt(st.nextToken());
maxWordLen = Integer.parseInt(st.nextToken());
}
if (line.hasOption('x')) {
String strOp = line.getOptionValue('x');
if (strOp.equals("r")) {
op = OP_READ;
}
else if (strOp.equals("w")) {
op = OP_CREATE;
}
else if (strOp.equals("rw")) {
op = OP_CREATE | OP_READ;
}
else {
throw new ParseException("Unknown action specifier: " + strOp);
}
}
proceed = true;
}
private void validateOptions() throws ParseException {
if (!compress.equals("none") && !compress.equals("lzo")
&& !compress.equals("gz")) {
throw new ParseException("Unknown compression scheme: " + compress);
}
if (!format.equals("tfile") && !format.equals("seqfile")) {
throw new ParseException("Unknown file format: " + format);
}
if (minWordLen >= maxWordLen) {
throw new ParseException(
"Max word length must be greater than min word length.");
}
return;
}
private void setStopProceed() {
proceed = false;
}
public boolean doCreate() {
return (op & OP_CREATE) != 0;
}
public boolean doRead() {
return (op & OP_READ) != 0;
}
}
public static void main(String[] args) throws IOException {
TestTFileSeqFileComparison testCase = new TestTFileSeqFileComparison();
MyOptions options = new MyOptions(args);
if (options.proceed == false) {
return;
}
testCase.options = options;
String parameters = parameters2String(options);
testCase.setUp();
if (testCase.options.format.equals("tfile")) {
if (options.doCreate()) {
testCase.createTFile(parameters, options.compress);
}
if (options.doRead()) {
testCase.readTFile(parameters, options.doCreate());
}
}
else {
if (options.doCreate()) {
testCase.createSeqFile(parameters, options.compress);
}
if (options.doRead()) {
testCase.readSeqFile(parameters, options.doCreate());
}
}
testCase.tearDown();
}
}

View File

@ -0,0 +1,107 @@
/**
* 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.IOException;
import junit.framework.Assert;
import junit.framework.TestCase;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.BytesWritable;
import org.apache.hadoop.io.file.tfile.TFile.Reader;
import org.apache.hadoop.io.file.tfile.TFile.Writer;
import org.apache.hadoop.io.file.tfile.TFile.Reader.Scanner;
public class TestTFileSplit extends TestCase {
private static String ROOT =
System.getProperty("test.build.data", "/tmp/tfile-test");
private final static int BLOCK_SIZE = 64 * 1024;
private static final String KEY = "key";
private static final String VALUE = "value";
private FileSystem fs;
private Configuration conf;
private Path path;
private String comparator = "memcmp";
private String outputFile = "TestTFileSplit";
void createFile(int count, String compress) throws IOException {
conf = new Configuration();
path = new Path(ROOT, outputFile + "." + compress);
fs = path.getFileSystem(conf);
FSDataOutputStream out = fs.create(path);
Writer writer = new Writer(out, BLOCK_SIZE, compress, comparator, conf);
int nx;
for (nx = 0; nx < count; nx++) {
byte[] key = composeSortedKey(KEY, count, nx).getBytes();
byte[] value = (VALUE + nx).getBytes();
writer.append(key, value);
}
writer.close();
out.close();
}
void readFile() throws IOException {
long fileLength = fs.getFileStatus(path).getLen();
int numSplit = 10;
long splitSize = fileLength / numSplit + 1;
Reader reader =
new Reader(fs.open(path), fs.getFileStatus(path).getLen(), conf);
long offset = 0;
long rowCount = 0;
BytesWritable key, value;
for (int i = 0; i < numSplit; ++i, offset += splitSize) {
Scanner scanner = reader.createScanner(offset, splitSize);
int count = 0;
key = new BytesWritable();
value = new BytesWritable();
while (!scanner.atEnd()) {
scanner.entry().get(key, value);
++count;
scanner.advance();
}
scanner.close();
Assert.assertTrue(count > 0);
rowCount += count;
}
Assert.assertEquals(rowCount, reader.getEntryCount());
reader.close();
}
static String composeSortedKey(String prefix, int total, int value) {
return String.format("%s%010d", prefix, value);
}
public void testSplit() throws IOException {
System.out.println("testSplit");
createFile(100000, Compression.Algorithm.NONE.getName());
readFile();
fs.delete(path, true);
createFile(500000, Compression.Algorithm.GZ.getName());
readFile();
fs.delete(path, true);
}
}

View File

@ -0,0 +1,423 @@
/**
* 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.DataOutputStream;
import java.io.EOFException;
import java.io.IOException;
import java.util.Random;
import junit.framework.Assert;
import junit.framework.TestCase;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.WritableUtils;
import org.apache.hadoop.io.file.tfile.TFile.Reader;
import org.apache.hadoop.io.file.tfile.TFile.Writer;
import org.apache.hadoop.io.file.tfile.TFile.Reader.Scanner;
/**
*
* Streaming interfaces test case class using GZ compression codec, base class
* of none and LZO compression classes.
*
*/
public class TestTFileStreams extends TestCase {
private static String ROOT =
System.getProperty("test.build.data", "/tmp/tfile-test");
private final static int BLOCK_SIZE = 512;
private final static int K = 1024;
private final static int M = K * K;
protected boolean skip = false;
private FileSystem fs;
private Configuration conf;
private Path path;
private FSDataOutputStream out;
Writer writer;
private String compression = Compression.Algorithm.GZ.getName();
private String comparator = "memcmp";
private String outputFile = "TFileTestStreams";
public void init(String compression, String comparator, String outputFile) {
this.compression = compression;
this.comparator = comparator;
this.outputFile = outputFile;
}
@Override
public void setUp() throws IOException {
conf = new Configuration();
path = new Path(ROOT, outputFile);
fs = path.getFileSystem(conf);
out = fs.create(path);
writer = new Writer(out, BLOCK_SIZE, compression, comparator, conf);
}
@Override
public void tearDown() throws IOException {
if (!skip) {
try {
closeOutput();
} catch (Exception e) {
// no-op
}
fs.delete(path, true);
}
}
public void testNoEntry() throws IOException {
if (skip)
return;
closeOutput();
TestTFileByteArrays.readRecords(fs, path, 0, conf);
}
public void testOneEntryKnownLength() throws IOException {
if (skip)
return;
writeRecords(1, true, true);
TestTFileByteArrays.readRecords(fs, path, 1, conf);
}
public void testOneEntryUnknownLength() throws IOException {
if (skip)
return;
writeRecords(1, false, false);
// TODO: will throw exception at getValueLength, it's inconsistent though;
// getKeyLength returns a value correctly, though initial length is -1
TestTFileByteArrays.readRecords(fs, path, 1, conf);
}
// known key length, unknown value length
public void testOneEntryMixedLengths1() throws IOException {
if (skip)
return;
writeRecords(1, true, false);
TestTFileByteArrays.readRecords(fs, path, 1, conf);
}
// unknown key length, known value length
public void testOneEntryMixedLengths2() throws IOException {
if (skip)
return;
writeRecords(1, false, true);
TestTFileByteArrays.readRecords(fs, path, 1, conf);
}
public void testTwoEntriesKnownLength() throws IOException {
if (skip)
return;
writeRecords(2, true, true);
TestTFileByteArrays.readRecords(fs, path, 2, conf);
}
// Negative test
public void testFailureAddKeyWithoutValue() throws IOException {
if (skip)
return;
DataOutputStream dos = writer.prepareAppendKey(-1);
dos.write("key0".getBytes());
try {
closeOutput();
fail("Cannot add only a key without a value. ");
}
catch (IllegalStateException e) {
// noop, expecting an exception
}
}
public void testFailureAddValueWithoutKey() throws IOException {
if (skip)
return;
DataOutputStream outValue = null;
try {
outValue = writer.prepareAppendValue(6);
outValue.write("value0".getBytes());
fail("Cannot add a value without adding key first. ");
}
catch (Exception e) {
// noop, expecting an exception
}
finally {
if (outValue != null) {
outValue.close();
}
}
}
public void testFailureOneEntryKnownLength() throws IOException {
if (skip)
return;
DataOutputStream outKey = writer.prepareAppendKey(2);
try {
outKey.write("key0".getBytes());
fail("Specified key length mismatched the actual key length.");
}
catch (IOException e) {
// noop, expecting an exception
}
DataOutputStream outValue = null;
try {
outValue = writer.prepareAppendValue(6);
outValue.write("value0".getBytes());
}
catch (Exception e) {
// noop, expecting an exception
}
}
public void testFailureKeyTooLong() throws IOException {
if (skip)
return;
DataOutputStream outKey = writer.prepareAppendKey(2);
try {
outKey.write("key0".getBytes());
outKey.close();
Assert.fail("Key is longer than requested.");
}
catch (Exception e) {
// noop, expecting an exception
}
finally {
}
}
public void testFailureKeyTooShort() throws IOException {
if (skip)
return;
DataOutputStream outKey = writer.prepareAppendKey(4);
outKey.write("key0".getBytes());
outKey.close();
DataOutputStream outValue = writer.prepareAppendValue(15);
try {
outValue.write("value0".getBytes());
outValue.close();
Assert.fail("Value is shorter than expected.");
}
catch (Exception e) {
// noop, expecting an exception
}
finally {
}
}
public void testFailureValueTooLong() throws IOException {
if (skip)
return;
DataOutputStream outKey = writer.prepareAppendKey(4);
outKey.write("key0".getBytes());
outKey.close();
DataOutputStream outValue = writer.prepareAppendValue(3);
try {
outValue.write("value0".getBytes());
outValue.close();
Assert.fail("Value is longer than expected.");
}
catch (Exception e) {
// noop, expecting an exception
}
try {
outKey.close();
outKey.close();
}
catch (Exception e) {
Assert.fail("Second or more close() should have no effect.");
}
}
public void testFailureValueTooShort() throws IOException {
if (skip)
return;
DataOutputStream outKey = writer.prepareAppendKey(8);
try {
outKey.write("key0".getBytes());
outKey.close();
Assert.fail("Key is shorter than expected.");
}
catch (Exception e) {
// noop, expecting an exception
}
finally {
}
}
public void testFailureCloseKeyStreamManyTimesInWriter() throws IOException {
if (skip)
return;
DataOutputStream outKey = writer.prepareAppendKey(4);
try {
outKey.write("key0".getBytes());
outKey.close();
}
catch (Exception e) {
// noop, expecting an exception
}
finally {
try {
outKey.close();
}
catch (Exception e) {
// no-op
}
}
outKey.close();
outKey.close();
Assert.assertTrue("Multiple close should have no effect.", true);
}
public void testFailureKeyLongerThan64K() throws IOException {
if (skip)
return;
try {
DataOutputStream outKey = writer.prepareAppendKey(64 * K + 1);
Assert.fail("Failed to handle key longer than 64K.");
}
catch (IndexOutOfBoundsException e) {
// noop, expecting exceptions
}
closeOutput();
}
public void testFailureKeyLongerThan64K_2() throws IOException {
if (skip)
return;
DataOutputStream outKey = writer.prepareAppendKey(-1);
try {
byte[] buf = new byte[K];
Random rand = new Random();
for (int nx = 0; nx < K + 2; nx++) {
rand.nextBytes(buf);
outKey.write(buf);
}
outKey.close();
Assert.fail("Failed to handle key longer than 64K.");
}
catch (EOFException e) {
// noop, expecting exceptions
}
finally {
try {
closeOutput();
}
catch (Exception e) {
// no-op
}
}
}
public void testFailureNegativeOffset() throws IOException {
if (skip)
return;
writeRecords(2, true, true);
Reader reader = new Reader(fs.open(path), fs.getFileStatus(path).getLen(), conf);
Scanner scanner = reader.createScanner();
byte[] buf = new byte[K];
try {
scanner.entry().getKey(buf, -1);
Assert.fail("Failed to handle key negative offset.");
}
catch (Exception e) {
// noop, expecting exceptions
}
finally {
}
scanner.close();
reader.close();
}
/**
* Verify that the compressed data size is less than raw data size.
*
* @throws IOException
*/
public void testFailureCompressionNotWorking() throws IOException {
if (skip)
return;
long rawDataSize = writeRecords(10000, false, false, false);
if (!compression.equalsIgnoreCase(Compression.Algorithm.NONE.getName())) {
Assert.assertTrue(out.getPos() < rawDataSize);
}
closeOutput();
}
public void testFailureCompressionNotWorking2() throws IOException {
if (skip)
return;
long rawDataSize = writeRecords(10000, true, true, false);
if (!compression.equalsIgnoreCase(Compression.Algorithm.NONE.getName())) {
Assert.assertTrue(out.getPos() < rawDataSize);
}
closeOutput();
}
private long writeRecords(int count, boolean knownKeyLength,
boolean knownValueLength, boolean close) throws IOException {
long rawDataSize = 0;
for (int nx = 0; nx < count; nx++) {
String key = TestTFileByteArrays.composeSortedKey("key", count, nx);
DataOutputStream outKey =
writer.prepareAppendKey(knownKeyLength ? key.length() : -1);
outKey.write(key.getBytes());
outKey.close();
String value = "value" + nx;
DataOutputStream outValue =
writer.prepareAppendValue(knownValueLength ? value.length() : -1);
outValue.write(value.getBytes());
outValue.close();
rawDataSize +=
WritableUtils.getVIntSize(key.getBytes().length)
+ key.getBytes().length
+ WritableUtils.getVIntSize(value.getBytes().length)
+ value.getBytes().length;
}
if (close) {
closeOutput();
}
return rawDataSize;
}
private long writeRecords(int count, boolean knownKeyLength,
boolean knownValueLength) throws IOException {
return writeRecords(count, knownKeyLength, knownValueLength, true);
}
private void closeOutput() throws IOException {
if (writer != null) {
writer.close();
writer = null;
}
if (out != null) {
out.close();
out = null;
}
}
}

View File

@ -0,0 +1,238 @@
/**
* 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.IOException;
import junit.framework.Assert;
import junit.framework.TestCase;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.file.tfile.TFile.Reader;
import org.apache.hadoop.io.file.tfile.TFile.Writer;
import org.apache.hadoop.io.file.tfile.TFile.Reader.Scanner;
public class TestTFileUnsortedByteArrays extends TestCase {
private static String ROOT =
System.getProperty("test.build.data", "/tmp/tfile-test");
private final static int BLOCK_SIZE = 512;
private final static int BUF_SIZE = 64;
private FileSystem fs;
private Configuration conf;
private Path path;
private FSDataOutputStream out;
private Writer writer;
private String compression = Compression.Algorithm.GZ.getName();
private String outputFile = "TFileTestUnsorted";
/*
* pre-sampled numbers of records in one block, based on the given the
* generated key and value strings
*/
private int records1stBlock = 4314;
private int records2ndBlock = 4108;
public void init(String compression, String outputFile,
int numRecords1stBlock, int numRecords2ndBlock) {
this.compression = compression;
this.outputFile = outputFile;
this.records1stBlock = numRecords1stBlock;
this.records2ndBlock = numRecords2ndBlock;
}
@Override
public void setUp() throws IOException {
conf = new Configuration();
path = new Path(ROOT, outputFile);
fs = path.getFileSystem(conf);
out = fs.create(path);
writer = new Writer(out, BLOCK_SIZE, compression, null, conf);
writer.append("keyZ".getBytes(), "valueZ".getBytes());
writer.append("keyM".getBytes(), "valueM".getBytes());
writer.append("keyN".getBytes(), "valueN".getBytes());
writer.append("keyA".getBytes(), "valueA".getBytes());
closeOutput();
}
@Override
public void tearDown() throws IOException {
fs.delete(path, true);
}
// we still can scan records in an unsorted TFile
public void testFailureScannerWithKeys() throws IOException {
Reader reader =
new Reader(fs.open(path), fs.getFileStatus(path).getLen(), conf);
Assert.assertFalse(reader.isSorted());
Assert.assertEquals((int) reader.getEntryCount(), 4);
try {
Scanner scanner =
reader.createScanner("aaa".getBytes(), "zzz".getBytes());
Assert
.fail("Failed to catch creating scanner with keys on unsorted file.");
}
catch (RuntimeException e) {
}
finally {
reader.close();
}
}
// we still can scan records in an unsorted TFile
public void testScan() throws IOException {
Reader reader =
new Reader(fs.open(path), fs.getFileStatus(path).getLen(), conf);
Assert.assertFalse(reader.isSorted());
Assert.assertEquals((int) reader.getEntryCount(), 4);
Scanner scanner = reader.createScanner();
try {
// read key and value
byte[] kbuf = new byte[BUF_SIZE];
int klen = scanner.entry().getKeyLength();
scanner.entry().getKey(kbuf);
Assert.assertEquals(new String(kbuf, 0, klen), "keyZ");
byte[] vbuf = new byte[BUF_SIZE];
int vlen = scanner.entry().getValueLength();
scanner.entry().getValue(vbuf);
Assert.assertEquals(new String(vbuf, 0, vlen), "valueZ");
scanner.advance();
// now try get value first
vbuf = new byte[BUF_SIZE];
vlen = scanner.entry().getValueLength();
scanner.entry().getValue(vbuf);
Assert.assertEquals(new String(vbuf, 0, vlen), "valueM");
kbuf = new byte[BUF_SIZE];
klen = scanner.entry().getKeyLength();
scanner.entry().getKey(kbuf);
Assert.assertEquals(new String(kbuf, 0, klen), "keyM");
}
finally {
scanner.close();
reader.close();
}
}
// we still can scan records in an unsorted TFile
public void testScanRange() throws IOException {
Reader reader =
new Reader(fs.open(path), fs.getFileStatus(path).getLen(), conf);
Assert.assertFalse(reader.isSorted());
Assert.assertEquals((int) reader.getEntryCount(), 4);
Scanner scanner = reader.createScanner();
try {
// read key and value
byte[] kbuf = new byte[BUF_SIZE];
int klen = scanner.entry().getKeyLength();
scanner.entry().getKey(kbuf);
Assert.assertEquals(new String(kbuf, 0, klen), "keyZ");
byte[] vbuf = new byte[BUF_SIZE];
int vlen = scanner.entry().getValueLength();
scanner.entry().getValue(vbuf);
Assert.assertEquals(new String(vbuf, 0, vlen), "valueZ");
scanner.advance();
// now try get value first
vbuf = new byte[BUF_SIZE];
vlen = scanner.entry().getValueLength();
scanner.entry().getValue(vbuf);
Assert.assertEquals(new String(vbuf, 0, vlen), "valueM");
kbuf = new byte[BUF_SIZE];
klen = scanner.entry().getKeyLength();
scanner.entry().getKey(kbuf);
Assert.assertEquals(new String(kbuf, 0, klen), "keyM");
}
finally {
scanner.close();
reader.close();
}
}
public void testFailureSeek() throws IOException {
Reader reader =
new Reader(fs.open(path), fs.getFileStatus(path).getLen(), conf);
Scanner scanner = reader.createScanner();
try {
// can't find ceil
try {
scanner.lowerBound("keyN".getBytes());
Assert.fail("Cannot search in a unsorted TFile!");
}
catch (Exception e) {
// noop, expecting excetions
}
finally {
}
// can't find higher
try {
scanner.upperBound("keyA".getBytes());
Assert.fail("Cannot search higher in a unsorted TFile!");
}
catch (Exception e) {
// noop, expecting excetions
}
finally {
}
// can't seek
try {
scanner.seekTo("keyM".getBytes());
Assert.fail("Cannot search a unsorted TFile!");
}
catch (Exception e) {
// noop, expecting excetions
}
finally {
}
}
finally {
scanner.close();
reader.close();
}
}
private void closeOutput() throws IOException {
if (writer != null) {
writer.close();
writer = null;
out.close();
out = null;
}
}
}

View File

@ -0,0 +1,161 @@
/**
* 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.IOException;
import java.util.Random;
import junit.framework.Assert;
import junit.framework.TestCase;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
public class TestVLong extends TestCase {
private static String ROOT =
System.getProperty("test.build.data", "/tmp/tfile-test");
private Configuration conf;
private FileSystem fs;
private Path path;
private String outputFile = "TestVLong";
@Override
public void setUp() throws IOException {
conf = new Configuration();
path = new Path(ROOT, outputFile);
fs = path.getFileSystem(conf);
if (fs.exists(path)) {
fs.delete(path, false);
}
}
@Override
public void tearDown() throws IOException {
if (fs.exists(path)) {
fs.delete(path, false);
}
}
public void testVLongByte() throws IOException {
FSDataOutputStream out = fs.create(path);
for (int i = Byte.MIN_VALUE; i <= Byte.MAX_VALUE; ++i) {
Utils.writeVLong(out, i);
}
out.close();
Assert.assertEquals("Incorrect encoded size", (1 << Byte.SIZE) + 96, fs
.getFileStatus(
path).getLen());
FSDataInputStream in = fs.open(path);
for (int i = Byte.MIN_VALUE; i <= Byte.MAX_VALUE; ++i) {
long n = Utils.readVLong(in);
Assert.assertEquals(n, i);
}
in.close();
fs.delete(path, false);
}
private long writeAndVerify(int shift) throws IOException {
FSDataOutputStream out = fs.create(path);
for (int i = Short.MIN_VALUE; i <= Short.MAX_VALUE; ++i) {
Utils.writeVLong(out, ((long) i) << shift);
}
out.close();
FSDataInputStream in = fs.open(path);
for (int i = Short.MIN_VALUE; i <= Short.MAX_VALUE; ++i) {
long n = Utils.readVLong(in);
Assert.assertEquals(n, ((long) i) << shift);
}
in.close();
long ret = fs.getFileStatus(path).getLen();
fs.delete(path, false);
return ret;
}
public void testVLongShort() throws IOException {
long size = writeAndVerify(0);
Assert.assertEquals("Incorrect encoded size", (1 << Short.SIZE) * 2
+ ((1 << Byte.SIZE) - 40)
* (1 << Byte.SIZE) - 128 - 32, size);
}
public void testVLong3Bytes() throws IOException {
long size = writeAndVerify(Byte.SIZE);
Assert.assertEquals("Incorrect encoded size", (1 << Short.SIZE) * 3
+ ((1 << Byte.SIZE) - 32) * (1 << Byte.SIZE) - 40 - 1, size);
}
public void testVLong4Bytes() throws IOException {
long size = writeAndVerify(Byte.SIZE * 2);
Assert.assertEquals("Incorrect encoded size", (1 << Short.SIZE) * 4
+ ((1 << Byte.SIZE) - 16) * (1 << Byte.SIZE) - 32 - 2, size);
}
public void testVLong5Bytes() throws IOException {
long size = writeAndVerify(Byte.SIZE * 3);
Assert.assertEquals("Incorrect encoded size", (1 << Short.SIZE) * 6 - 256
- 16 - 3, size);
}
private void verifySixOrMoreBytes(int bytes) throws IOException {
long size = writeAndVerify(Byte.SIZE * (bytes - 2));
Assert.assertEquals("Incorrect encoded size", (1 << Short.SIZE)
* (bytes + 1) - 256 - bytes + 1, size);
}
public void testVLong6Bytes() throws IOException {
verifySixOrMoreBytes(6);
}
public void testVLong7Bytes() throws IOException {
verifySixOrMoreBytes(7);
}
public void testVLong8Bytes() throws IOException {
verifySixOrMoreBytes(8);
}
public void testVLongRandom() throws IOException {
int count = 1024 * 1024;
long data[] = new long[count];
Random rng = new Random();
for (int i = 0; i < data.length; ++i) {
int shift = rng.nextInt(Long.SIZE) + 1;
long mask = (1L << shift) - 1;
long a = rng.nextInt() << 32;
long b = ((long) rng.nextInt()) & 0xffffffff;
data[i] = (a + b) & mask;
}
FSDataOutputStream out = fs.create(path);
for (int i = 0; i < data.length; ++i) {
Utils.writeVLong(out, data[i]);
}
out.close();
FSDataInputStream in = fs.open(path);
for (int i = 0; i < data.length; ++i) {
Assert.assertEquals(Utils.readVLong(in), data[i]);
}
in.close();
fs.delete(path, false);
}
}

View File

@ -0,0 +1,63 @@
/**
* 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.IOException;
import java.text.DateFormat;
import java.text.SimpleDateFormat;
/**
* this class is a time class to
* measure to measure the time
* taken for some event.
*/
public class Timer {
long startTimeEpoch;
long finishTimeEpoch;
private DateFormat formatter = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
public void startTime() throws IOException {
startTimeEpoch = System.currentTimeMillis();
}
public void stopTime() throws IOException {
finishTimeEpoch = System.currentTimeMillis();
}
public long getIntervalMillis() throws IOException {
return finishTimeEpoch - startTimeEpoch;
}
public void printlnWithTimestamp(String message) throws IOException {
System.out.println(formatCurrentTime() + " " + message);
}
public String formatTime(long millis) {
return formatter.format(millis);
}
public String getIntervalString() throws IOException {
long time = getIntervalMillis();
return formatTime(time);
}
public String formatCurrentTime() {
return formatTime(System.currentTimeMillis());
}
}

View File

@ -213,4 +213,26 @@
<Bug pattern="OBL_UNSATISFIED_OBLIGATION" />
</Match>
<!--
TFile
-->
<Match>
<Class name="org.apache.hadoop.io.file.tfile.Chunk$ChunkDecoder" />
<Method name="close" />
<Bug pattern="SR_NOT_CHECKED" />
</Match>
<!--
The purpose of skip() is to drain remaining bytes of the chunk-encoded
stream (one chunk at a time). The termination condition is checked by
checkEOF().
-->
<Match>
<Class name="org.apache.hadoop.io.file.tfile.Utils" />
<Method name="writeVLong" />
<Bug pattern="SF_SWITCH_FALLTHROUGH" />
</Match>
<!--
The switch condition fall through is intentional and for performance
purposes.
-->
</FindBugsFilter>