From 8246aa28ff72e3ae81eb6ce59852abd5828fadc6 Mon Sep 17 00:00:00 2001 From: Christopher Douglas Date: Wed, 24 Jun 2009 05:48:25 +0000 Subject: [PATCH] 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 --- CHANGES.txt | 2 + build.xml | 2 + .../apache/hadoop/io/file/tfile/BCFile.java | 979 ++++++++ .../tfile/BoundedByteArrayOutputStream.java | 96 + .../tfile/BoundedRangeFileInputStream.java | 141 ++ .../hadoop/io/file/tfile/ByteArray.java | 92 + .../apache/hadoop/io/file/tfile/Chunk.java | 429 ++++ .../hadoop/io/file/tfile/CompareUtils.java | 97 + .../hadoop/io/file/tfile/Compression.java | 361 +++ .../io/file/tfile/MetaBlockAlreadyExists.java | 36 + .../io/file/tfile/MetaBlockDoesNotExist.java | 36 + .../hadoop/io/file/tfile/RawComparable.java | 57 + .../tfile/SimpleBufferedOutputStream.java | 77 + .../apache/hadoop/io/file/tfile/TFile.java | 2220 +++++++++++++++++ .../hadoop/io/file/tfile/TFileDumper.java | 295 +++ .../apache/hadoop/io/file/tfile/Utils.java | 516 ++++ .../hadoop/io/file/tfile/KVGenerator.java | 105 + .../hadoop/io/file/tfile/KeySampler.java | 56 + .../hadoop/io/file/tfile/NanoTimer.java | 193 ++ .../io/file/tfile/RandomDistribution.java | 266 ++ .../hadoop/io/file/tfile/TestTFile.java | 431 ++++ .../io/file/tfile/TestTFileByteArrays.java | 790 ++++++ .../io/file/tfile/TestTFileComparators.java | 122 + .../TestTFileJClassComparatorByteArrays.java | 58 + .../tfile/TestTFileLzoCodecsByteArrays.java | 42 + .../file/tfile/TestTFileLzoCodecsStreams.java | 39 + .../tfile/TestTFileNoneCodecsByteArrays.java | 32 + ...eNoneCodecsJClassComparatorByteArrays.java | 43 + .../tfile/TestTFileNoneCodecsStreams.java | 32 + .../hadoop/io/file/tfile/TestTFileSeek.java | 504 ++++ .../tfile/TestTFileSeqFileComparison.java | 782 ++++++ .../hadoop/io/file/tfile/TestTFileSplit.java | 107 + .../io/file/tfile/TestTFileStreams.java | 423 ++++ .../tfile/TestTFileUnsortedByteArrays.java | 238 ++ .../hadoop/io/file/tfile/TestVLong.java | 161 ++ .../apache/hadoop/io/file/tfile/Timer.java | 63 + src/test/findbugsExcludeFile.xml | 22 + 37 files changed, 9945 insertions(+) create mode 100644 src/java/org/apache/hadoop/io/file/tfile/BCFile.java create mode 100644 src/java/org/apache/hadoop/io/file/tfile/BoundedByteArrayOutputStream.java create mode 100644 src/java/org/apache/hadoop/io/file/tfile/BoundedRangeFileInputStream.java create mode 100644 src/java/org/apache/hadoop/io/file/tfile/ByteArray.java create mode 100644 src/java/org/apache/hadoop/io/file/tfile/Chunk.java create mode 100644 src/java/org/apache/hadoop/io/file/tfile/CompareUtils.java create mode 100644 src/java/org/apache/hadoop/io/file/tfile/Compression.java create mode 100644 src/java/org/apache/hadoop/io/file/tfile/MetaBlockAlreadyExists.java create mode 100644 src/java/org/apache/hadoop/io/file/tfile/MetaBlockDoesNotExist.java create mode 100644 src/java/org/apache/hadoop/io/file/tfile/RawComparable.java create mode 100644 src/java/org/apache/hadoop/io/file/tfile/SimpleBufferedOutputStream.java create mode 100644 src/java/org/apache/hadoop/io/file/tfile/TFile.java create mode 100644 src/java/org/apache/hadoop/io/file/tfile/TFileDumper.java create mode 100644 src/java/org/apache/hadoop/io/file/tfile/Utils.java create mode 100644 src/test/core/org/apache/hadoop/io/file/tfile/KVGenerator.java create mode 100644 src/test/core/org/apache/hadoop/io/file/tfile/KeySampler.java create mode 100644 src/test/core/org/apache/hadoop/io/file/tfile/NanoTimer.java create mode 100644 src/test/core/org/apache/hadoop/io/file/tfile/RandomDistribution.java create mode 100644 src/test/core/org/apache/hadoop/io/file/tfile/TestTFile.java create mode 100644 src/test/core/org/apache/hadoop/io/file/tfile/TestTFileByteArrays.java create mode 100644 src/test/core/org/apache/hadoop/io/file/tfile/TestTFileComparators.java create mode 100644 src/test/core/org/apache/hadoop/io/file/tfile/TestTFileJClassComparatorByteArrays.java create mode 100644 src/test/core/org/apache/hadoop/io/file/tfile/TestTFileLzoCodecsByteArrays.java create mode 100644 src/test/core/org/apache/hadoop/io/file/tfile/TestTFileLzoCodecsStreams.java create mode 100644 src/test/core/org/apache/hadoop/io/file/tfile/TestTFileNoneCodecsByteArrays.java create mode 100644 src/test/core/org/apache/hadoop/io/file/tfile/TestTFileNoneCodecsJClassComparatorByteArrays.java create mode 100644 src/test/core/org/apache/hadoop/io/file/tfile/TestTFileNoneCodecsStreams.java create mode 100644 src/test/core/org/apache/hadoop/io/file/tfile/TestTFileSeek.java create mode 100644 src/test/core/org/apache/hadoop/io/file/tfile/TestTFileSeqFileComparison.java create mode 100644 src/test/core/org/apache/hadoop/io/file/tfile/TestTFileSplit.java create mode 100644 src/test/core/org/apache/hadoop/io/file/tfile/TestTFileStreams.java create mode 100644 src/test/core/org/apache/hadoop/io/file/tfile/TestTFileUnsortedByteArrays.java create mode 100644 src/test/core/org/apache/hadoop/io/file/tfile/TestVLong.java create mode 100644 src/test/core/org/apache/hadoop/io/file/tfile/Timer.java diff --git a/CHANGES.txt b/CHANGES.txt index cea18f691c..ed0248638f 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -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 diff --git a/build.xml b/build.xml index 5d272236de..8d97576603 100644 --- a/build.xml +++ b/build.xml @@ -507,6 +507,8 @@ + diff --git a/src/java/org/apache/hadoop/io/file/tfile/BCFile.java b/src/java/org/apache/hadoop/io/file/tfile/BCFile.java new file mode 100644 index 0000000000..6c7ba1e831 --- /dev/null +++ b/src/java/org/apache/hadoop/io/file/tfile/BCFile.java @@ -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 + * close 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 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 index; + + // for write + public MetaIndex() { + index = new TreeMap(); + } + + // for read, construct the map from the file + public MetaIndex(DataInput in) throws IOException { + int count = Utils.readVInt(in); + index = new TreeMap(); + + 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 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(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(); + } + + public Algorithm getDefaultCompressionAlgorithm() { + return defaultCompressionAlgorithm; + } + + public ArrayList 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; + } + } +} diff --git a/src/java/org/apache/hadoop/io/file/tfile/BoundedByteArrayOutputStream.java b/src/java/org/apache/hadoop/io/file/tfile/BoundedByteArrayOutputStream.java new file mode 100644 index 0000000000..b0302f050a --- /dev/null +++ b/src/java/org/apache/hadoop/io/file/tfile/BoundedByteArrayOutputStream.java @@ -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 reset + * 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; + } +} diff --git a/src/java/org/apache/hadoop/io/file/tfile/BoundedRangeFileInputStream.java b/src/java/org/apache/hadoop/io/file/tfile/BoundedRangeFileInputStream.java new file mode 100644 index 0000000000..95af4ea547 --- /dev/null +++ b/src/java/org/apache/hadoop/io/file/tfile/BoundedRangeFileInputStream.java @@ -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; + } +} diff --git a/src/java/org/apache/hadoop/io/file/tfile/ByteArray.java b/src/java/org/apache/hadoop/io/file/tfile/ByteArray.java new file mode 100644 index 0000000000..4e05777cda --- /dev/null +++ b/src/java/org/apache/hadoop/io/file/tfile/ByteArray.java @@ -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; + } +} diff --git a/src/java/org/apache/hadoop/io/file/tfile/Chunk.java b/src/java/org/apache/hadoop/io/file/tfile/Chunk.java new file mode 100644 index 0000000000..05e3d48a46 --- /dev/null +++ b/src/java/org/apache/hadoop/io/file/tfile/Chunk.java @@ -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 0 through buf.length; elements buf[0] + * through buf[count-1] 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; + } + } + } +} diff --git a/src/java/org/apache/hadoop/io/file/tfile/CompareUtils.java b/src/java/org/apache/hadoop/io/file/tfile/CompareUtils.java new file mode 100644 index 0000000000..616c6c5945 --- /dev/null +++ b/src/java/org/apache/hadoop/io/file/tfile/CompareUtils.java @@ -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 { + private RawComparator cmp; + + public BytesComparator(RawComparator 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 { + @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 { + @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"); + } + } +} diff --git a/src/java/org/apache/hadoop/io/file/tfile/Compression.java b/src/java/org/apache/hadoop/io/file/tfile/Compression.java new file mode 100644 index 0000000000..a8371ca798 --- /dev/null +++ b/src/java/org/apache/hadoop/io/file/tfile/Compression.java @@ -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 ret = new ArrayList(); + for (Algorithm a : algos) { + if (a.isSupported()) { + ret.add(a.getName()); + } + } + return ret.toArray(new String[ret.size()]); + } +} diff --git a/src/java/org/apache/hadoop/io/file/tfile/MetaBlockAlreadyExists.java b/src/java/org/apache/hadoop/io/file/tfile/MetaBlockAlreadyExists.java new file mode 100644 index 0000000000..f381b1f9ee --- /dev/null +++ b/src/java/org/apache/hadoop/io/file/tfile/MetaBlockAlreadyExists.java @@ -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); + } +} diff --git a/src/java/org/apache/hadoop/io/file/tfile/MetaBlockDoesNotExist.java b/src/java/org/apache/hadoop/io/file/tfile/MetaBlockDoesNotExist.java new file mode 100644 index 0000000000..1a408a8cd6 --- /dev/null +++ b/src/java/org/apache/hadoop/io/file/tfile/MetaBlockDoesNotExist.java @@ -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); + } +} diff --git a/src/java/org/apache/hadoop/io/file/tfile/RawComparable.java b/src/java/org/apache/hadoop/io/file/tfile/RawComparable.java new file mode 100644 index 0000000000..837bfe70f5 --- /dev/null +++ b/src/java/org/apache/hadoop/io/file/tfile/RawComparable.java @@ -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(); +} diff --git a/src/java/org/apache/hadoop/io/file/tfile/SimpleBufferedOutputStream.java b/src/java/org/apache/hadoop/io/file/tfile/SimpleBufferedOutputStream.java new file mode 100644 index 0000000000..a26a02d576 --- /dev/null +++ b/src/java/org/apache/hadoop/io/file/tfile/SimpleBufferedOutputStream.java @@ -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; + } +} diff --git a/src/java/org/apache/hadoop/io/file/tfile/TFile.java b/src/java/org/apache/hadoop/io/file/tfile/TFile.java new file mode 100644 index 0000000000..a62b594789 --- /dev/null +++ b/src/java/org/apache/hadoop/io/file/tfile/TFile.java @@ -0,0 +1,2220 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with this + * work for additional information regarding copyright ownership. The ASF + * licenses this file to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package org.apache.hadoop.io.file.tfile; + +import java.io.ByteArrayInputStream; +import java.io.Closeable; +import java.io.DataInput; +import java.io.DataInputStream; +import java.io.DataOutput; +import java.io.DataOutputStream; +import java.io.EOFException; +import java.io.IOException; +import java.io.OutputStream; +import java.util.ArrayList; +import java.util.Comparator; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.io.DataInputBuffer; +import org.apache.hadoop.io.DataOutputBuffer; +import org.apache.hadoop.io.IOUtils; +import org.apache.hadoop.io.RawComparator; +import org.apache.hadoop.io.WritableComparator; +import org.apache.hadoop.io.file.tfile.BCFile.Reader.BlockReader; +import org.apache.hadoop.io.file.tfile.BCFile.Writer.BlockAppender; +import org.apache.hadoop.io.file.tfile.Chunk.ChunkDecoder; +import org.apache.hadoop.io.file.tfile.Chunk.ChunkEncoder; +import org.apache.hadoop.io.file.tfile.CompareUtils.BytesComparator; +import org.apache.hadoop.io.file.tfile.CompareUtils.MemcmpRawComparator; +import org.apache.hadoop.io.file.tfile.Utils.Version; +import org.apache.hadoop.io.serializer.JavaSerializationComparator; + +/** + * A TFile is a container of key-value pairs. Both keys and values are type-less + * bytes. Keys are restricted to 64KB, value length is not restricted + * (practically limited to the available disk storage). TFile further provides + * the following features: + *
    + *
  • Block Compression. + *
  • Named meta data blocks. + *
  • Sorted or unsorted keys. + *
  • Seek by key or by file offset. + *
+ * The memory footprint of a TFile includes the following: + *
    + *
  • Some constant overhead of reading or writing a compressed block. + *
      + *
    • Each compressed block requires one compression/decompression codec for + * I/O. + *
    • Temporary space to buffer the key. + *
    • Temporary space to buffer the value (for TFile.Writer only). Values are + * chunk encoded, so that we buffer at most one chunk of user data. By default, + * the chunk buffer is 1MB. Reading chunked value does not require additional + * memory. + *
    + *
  • TFile index, which is proportional to the total number of Data Blocks. + * The total amount of memory needed to hold the index can be estimated as + * (56+AvgKeySize)*NumBlocks. + *
  • MetaBlock index, which is proportional to the total number of Meta + * Blocks.The total amount of memory needed to hold the index for Meta Blocks + * can be estimated as (40+AvgMetaBlockName)*NumMetaBlock. + *
+ *

+ * The behavior of TFile can be customized by the following variables through + * Configuration: + *

    + *
  • tfile.io.chunk.size: Value chunk size. Integer (in bytes). Default + * to 1MB. Values of the length less than the chunk size is guaranteed to have + * known value length in read time (See + * {@link TFile.Reader.Scanner.Entry#isValueLengthKnown()}). + *
  • tfile.fs.output.buffer.size: Buffer size used for + * FSDataOutputStream. Integer (in bytes). Default to 256KB. + *
  • tfile.fs.input.buffer.size: Buffer size used for + * FSDataInputStream. Integer (in bytes). Default to 256KB. + *
+ *

+ * Suggestions on performance optimization. + *

    + *
  • Minimum block size. We recommend a setting of minimum block size between + * 256KB to 1MB for general usage. Larger block size is preferred if files are + * primarily for sequential access. However, it would lead to inefficient random + * access (because there are more data to decompress). Smaller blocks are good + * for random access, but require more memory to hold the block index, and may + * be slower to create (because we must flush the compressor stream at the + * conclusion of each data block, which leads to an FS I/O flush). Further, due + * to the internal caching in Compression codec, the smallest possible block + * size would be around 20KB-30KB. + *
  • The current implementation does not offer true multi-threading for + * reading. The implementation uses FSDataInputStream seek()+read(), which is + * shown to be much faster than positioned-read call in single thread mode. + * However, it also means that if multiple threads attempt to access the same + * TFile (using multiple scanners) simultaneously, the actual I/O is carried out + * sequentially even if they access different DFS blocks. + *
  • Compression codec. Use "none" if the data is not very compressable (by + * compressable, I mean a compression ratio at least 2:1). Generally, use "lzo" + * as the starting point for experimenting. "gz" overs slightly better + * compression ratio over "lzo" but requires 4x CPU to compress and 2x CPU to + * decompress, comparing to "lzo". + *
  • File system buffering, if the underlying FSDataInputStream and + * FSDataOutputStream is already adequately buffered; or if applications + * reads/writes keys and values in large buffers, we can reduce the sizes of + * input/output buffering in TFile layer by setting the configuration parameters + * "tfile.fs.input.buffer.size" and "tfile.fs.output.buffer.size". + *
+ * + * Some design rationale behind TFile can be found at Hadoop-3315. + */ +public class TFile { + static final Log LOG = LogFactory.getLog(TFile.class); + + private static final String CHUNK_BUF_SIZE_ATTR = "tfile.io.chunk.size"; + private static final String FS_INPUT_BUF_SIZE_ATTR = + "tfile.fs.input.buffer.size"; + private static final String FS_OUTPUT_BUF_SIZE_ATTR = + "tfile.fs.output.buffer.size"; + + static int getChunkBufferSize(Configuration conf) { + int ret = conf.getInt(CHUNK_BUF_SIZE_ATTR, 1024 * 1024); + return (ret > 0) ? ret : 1024 * 1024; + } + + static int getFSInputBufferSize(Configuration conf) { + return conf.getInt(FS_INPUT_BUF_SIZE_ATTR, 256 * 1024); + } + + static int getFSOutputBufferSize(Configuration conf) { + return conf.getInt(FS_OUTPUT_BUF_SIZE_ATTR, 256 * 1024); + } + + private static final int MAX_KEY_SIZE = 64 * 1024; // 64KB + static final Version API_VERSION = new Version((short) 1, (short) 0); + + /** compression: gzip */ + public static final String COMPRESSION_GZ = "gz"; + /** compression: lzo */ + public static final String COMPRESSION_LZO = "lzo"; + /** compression: none */ + public static final String COMPRESSION_NONE = "none"; + /** comparator: memcmp */ + public static final String COMPARATOR_MEMCMP = "memcmp"; + /** comparator prefix: java class */ + public static final String COMPARATOR_JCLASS = "jclass:"; + + // Prevent the instantiation of TFiles + private TFile() { + // nothing + } + + /** + * Get names of supported compression algorithms. The names are acceptable by + * TFile.Writer. + * + * @return Array of strings, each represents a supported compression + * algorithm. Currently, the following compression algorithms are + * supported. + *
    + *
  • "none" - No compression. + *
  • "lzo" - LZO compression. + *
  • "gz" - GZIP compression. + *
+ */ + public static String[] getSupportedCompressionAlgorithms() { + return Compression.getSupportedAlgorithms(); + } + + /** + * TFile Writer. + */ + public static class Writer implements Closeable { + // minimum compressed size for a block. + private final int sizeMinBlock; + + // Meta blocks. + final TFileIndex tfileIndex; + final TFileMeta tfileMeta; + + // reference to the underlying BCFile. + private BCFile.Writer writerBCF; + + // current data block appender. + BlockAppender blkAppender; + long blkRecordCount; + + // buffers for caching the key. + BoundedByteArrayOutputStream currentKeyBufferOS; + BoundedByteArrayOutputStream lastKeyBufferOS; + + // buffer used by chunk codec + private byte[] valueBuffer; + + /** + * Writer states. The state always transits in circles: READY -> IN_KEY -> + * END_KEY -> IN_VALUE -> READY. + */ + private enum State { + READY, // Ready to start a new key-value pair insertion. + IN_KEY, // In the middle of key insertion. + END_KEY, // Key insertion complete, ready to insert value. + IN_VALUE, // In value insertion. + // ERROR, // Error encountered, cannot continue. + CLOSED, // TFile already closed. + }; + + // current state of Writer. + State state = State.READY; + Configuration conf; + long errorCount = 0; + + /** + * Constructor + * + * @param fsdos + * output stream for writing. Must be at position 0. + * @param minBlockSize + * Minimum compressed block size in bytes. A compression block will + * not be closed until it reaches this size except for the last + * block. + * @param compressName + * Name of the compression algorithm. Must be one of the strings + * returned by {@link TFile#getSupportedCompressionAlgorithms()}. + * @param comparator + * Leave comparator as null or empty string if TFile is not sorted. + * Otherwise, provide the string name for the comparison algorithm + * for keys. Two kinds of comparators are supported. + *
    + *
  • Algorithmic comparator: binary comparators that is language + * independent. Currently, only "memcmp" is supported. + *
  • Language-specific comparator: binary comparators that can + * only be constructed in specific language. For Java, the syntax + * is "jclass:", followed by the class name of the RawComparator. + * Currently, we only support RawComparators that can be + * constructed through the default constructor (with no + * parameters). Parameterized RawComparators such as + * {@link WritableComparator} or + * {@link JavaSerializationComparator} may not be directly used. + * One should write a wrapper class that inherits from such classes + * and use its default constructor to perform proper + * initialization. + *
+ * @param conf + * The configuration object. + * @throws IOException + */ + public Writer(FSDataOutputStream fsdos, int minBlockSize, + String compressName, String comparator, Configuration conf) + throws IOException { + sizeMinBlock = minBlockSize; + tfileMeta = new TFileMeta(comparator); + tfileIndex = new TFileIndex(tfileMeta.getComparator()); + + writerBCF = new BCFile.Writer(fsdos, compressName, conf); + currentKeyBufferOS = new BoundedByteArrayOutputStream(MAX_KEY_SIZE); + lastKeyBufferOS = new BoundedByteArrayOutputStream(MAX_KEY_SIZE); + this.conf = conf; + } + + /** + * Close the Writer. Resources will be released regardless of the exceptions + * being thrown. Future close calls will have no effect. + * + * The underlying FSDataOutputStream is not closed. + */ + public void close() throws IOException { + if ((state == State.CLOSED)) { + return; + } + try { + // First try the normal finish. + // Terminate upon the first Exception. + if (errorCount == 0) { + if (state != State.READY) { + throw new IllegalStateException( + "Cannot close TFile in the middle of key-value insertion."); + } + + finishDataBlock(true); + + // first, write out data:TFile.meta + BlockAppender outMeta = + writerBCF + .prepareMetaBlock(TFileMeta.BLOCK_NAME, COMPRESSION_NONE); + try { + tfileMeta.write(outMeta); + } finally { + outMeta.close(); + } + + // second, write out data:TFile.index + BlockAppender outIndex = + writerBCF.prepareMetaBlock(TFileIndex.BLOCK_NAME); + try { + tfileIndex.write(outIndex); + } finally { + outIndex.close(); + } + + if (writerBCF != null) { + writerBCF.close(); + writerBCF = null; + } + } + } finally { + IOUtils.cleanup(LOG, blkAppender, writerBCF); + blkAppender = null; + writerBCF = null; + state = State.CLOSED; + } + } + + /** + * Adding a new key-value pair to the TFile. This is synonymous to + * append(key, 0, key.length, value, 0, value.length) + * + * @param key + * Buffer for key. + * @param value + * Buffer for value. + * @throws IOException + */ + public void append(byte[] key, byte[] value) throws IOException { + append(key, 0, key.length, value, 0, value.length); + } + + /** + * Adding a new key-value pair to TFile. + * + * @param key + * buffer for key. + * @param koff + * offset in key buffer. + * @param klen + * length of key. + * @param value + * buffer for value. + * @param voff + * offset in value buffer. + * @param vlen + * length of value. + * @throws IOException + * Upon IO errors. + *

+ * If an exception is thrown, the TFile will be in an inconsistent + * state. The only legitimate call after that would be close + */ + public void append(byte[] key, int koff, int klen, byte[] value, int voff, + int vlen) throws IOException { + if ((koff | klen | (koff + klen) | (key.length - (koff + klen))) < 0) { + throw new IndexOutOfBoundsException( + "Bad key buffer offset-length combination."); + } + + if ((voff | vlen | (voff + vlen) | (value.length - (voff + vlen))) < 0) { + throw new IndexOutOfBoundsException( + "Bad value buffer offset-length combination."); + } + + try { + DataOutputStream dosKey = prepareAppendKey(klen); + try { + ++errorCount; + dosKey.write(key, koff, klen); + --errorCount; + } finally { + dosKey.close(); + } + + DataOutputStream dosValue = prepareAppendValue(vlen); + try { + ++errorCount; + dosValue.write(value, voff, vlen); + --errorCount; + } finally { + dosValue.close(); + } + } finally { + state = State.READY; + } + } + + /** + * Helper class to register key after close call on key append stream. + */ + private class KeyRegister extends DataOutputStream { + private final int expectedLength; + private boolean closed = false; + + public KeyRegister(int len) { + super(currentKeyBufferOS); + if (len >= 0) { + currentKeyBufferOS.reset(len); + } else { + currentKeyBufferOS.reset(); + } + expectedLength = len; + } + + @Override + public void close() throws IOException { + if (closed == true) { + return; + } + + try { + ++errorCount; + byte[] key = currentKeyBufferOS.getBuffer(); + int len = currentKeyBufferOS.size(); + /** + * verify length. + */ + if (expectedLength >= 0 && expectedLength != len) { + throw new IOException("Incorrect key length: expected=" + + expectedLength + " actual=" + len); + } + + Utils.writeVInt(blkAppender, len); + blkAppender.write(key, 0, len); + if (tfileIndex.getFirstKey() == null) { + tfileIndex.setFirstKey(key, 0, len); + } + + if (tfileMeta.isSorted()) { + byte[] lastKey = lastKeyBufferOS.getBuffer(); + int lastLen = lastKeyBufferOS.size(); + if (tfileMeta.getComparator().compare(key, 0, len, lastKey, 0, + lastLen) < 0) { + throw new IOException("Keys are not added in sorted order"); + } + } + + BoundedByteArrayOutputStream tmp = currentKeyBufferOS; + currentKeyBufferOS = lastKeyBufferOS; + lastKeyBufferOS = tmp; + --errorCount; + } finally { + closed = true; + state = State.END_KEY; + } + } + } + + /** + * Helper class to register value after close call on value append stream. + */ + private class ValueRegister extends DataOutputStream { + private boolean closed = false; + + public ValueRegister(OutputStream os) { + super(os); + } + + // Avoiding flushing call to down stream. + @Override + public void flush() { + // do nothing + } + + @Override + public void close() throws IOException { + if (closed == true) { + return; + } + + try { + ++errorCount; + super.close(); + blkRecordCount++; + // bump up the total record count in the whole file + tfileMeta.incRecordCount(); + finishDataBlock(false); + --errorCount; + } finally { + closed = true; + state = State.READY; + } + } + } + + /** + * Obtain an output stream for writing a key into TFile. This may only be + * called when there is no active Key appending stream or value appending + * stream. + * + * @param length + * The expected length of the key. If length of the key is not + * known, set length = -1. Otherwise, the application must write + * exactly as many bytes as specified here before calling close on + * the returned output stream. + * @return The key appending output stream. + * @throws IOException + * + */ + public DataOutputStream prepareAppendKey(int length) throws IOException { + if (state != State.READY) { + throw new IllegalStateException("Incorrect state to start a new key: " + + state.name()); + } + + initDataBlock(); + DataOutputStream ret = new KeyRegister(length); + state = State.IN_KEY; + return ret; + } + + /** + * Obtain an output stream for writing a value into TFile. This may only be + * called right after a key appending operation (the key append stream must + * be closed). + * + * @param length + * The expected length of the value. If length of the value is not + * known, set length = -1. Otherwise, the application must write + * exactly as many bytes as specified here before calling close on + * the returned output stream. Advertising the value size up-front + * guarantees that the value is encoded in one chunk, and avoids + * intermediate chunk buffering. + * @throws IOException + * + */ + public DataOutputStream prepareAppendValue(int length) throws IOException { + if (state != State.END_KEY) { + throw new IllegalStateException( + "Incorrect state to start a new value: " + state.name()); + } + + DataOutputStream ret; + + // unknown length + if (length < 0) { + if (valueBuffer == null) { + valueBuffer = new byte[getChunkBufferSize(conf)]; + } + ret = new ValueRegister(new ChunkEncoder(blkAppender, valueBuffer)); + } else { + ret = + new ValueRegister(new Chunk.SingleChunkEncoder(blkAppender, length)); + } + + state = State.IN_VALUE; + return ret; + } + + /** + * Obtain an output stream for creating a meta block. This function may not + * be called when there is a key append stream or value append stream + * active. No more key-value insertion is allowed after a meta data block + * has been added to TFile. + * + * @param name + * Name of the meta block. + * @param compressName + * Name of the compression algorithm to be used. Must be one of the + * strings returned by + * {@link TFile#getSupportedCompressionAlgorithms()}. + * @return A DataOutputStream that can be used to write Meta Block data. + * Closing the stream would signal the ending of the block. + * @throws IOException + * @throws MetaBlockAlreadyExists + * the Meta Block with the same name already exists. + */ + public DataOutputStream prepareMetaBlock(String name, String compressName) + throws IOException, MetaBlockAlreadyExists { + if (state != State.READY) { + throw new IllegalStateException( + "Incorrect state to start a Meta Block: " + state.name()); + } + + finishDataBlock(true); + DataOutputStream outputStream = + writerBCF.prepareMetaBlock(name, compressName); + return outputStream; + } + + /** + * Obtain an output stream for creating a meta block. This function may not + * be called when there is a key append stream or value append stream + * active. No more key-value insertion is allowed after a meta data block + * has been added to TFile. Data will be compressed using the default + * compressor as defined in Writer's constructor. + * + * @param name + * Name of the meta block. + * @return A DataOutputStream that can be used to write Meta Block data. + * Closing the stream would signal the ending of the block. + * @throws IOException + * @throws MetaBlockAlreadyExists + * the Meta Block with the same name already exists. + */ + public DataOutputStream prepareMetaBlock(String name) throws IOException, + MetaBlockAlreadyExists { + if (state != State.READY) { + throw new IllegalStateException( + "Incorrect state to start a Meta Block: " + state.name()); + } + + finishDataBlock(true); + return writerBCF.prepareMetaBlock(name); + } + + /** + * Check if we need to start a new data block. + * + * @throws IOException + */ + private void initDataBlock() throws IOException { + // for each new block, get a new appender + if (blkAppender == null) { + blkAppender = writerBCF.prepareDataBlock(); + } + } + + /** + * Close the current data block if necessary. + * + * @param bForceFinish + * Force the closure regardless of the block size. + * @throws IOException + */ + void finishDataBlock(boolean bForceFinish) throws IOException { + if (blkAppender == null) { + return; + } + + // exceeded the size limit, do the compression and finish the block + if (bForceFinish || blkAppender.getCompressedSize() >= sizeMinBlock) { + // keep tracks of the last key of each data block, no padding + // for now + TFileIndexEntry keyLast = + new TFileIndexEntry(lastKeyBufferOS.getBuffer(), 0, lastKeyBufferOS + .size(), blkRecordCount); + tfileIndex.addEntry(keyLast); + // close the appender + blkAppender.close(); + blkAppender = null; + blkRecordCount = 0; + } + } + } + + /** + * TFile Reader. Users may only read TFiles by creating TFile.Reader.Scanner. + * objects. A scanner may scan the whole TFile ({@link Reader#createScanner()} + * ) , a portion of TFile based on byte offsets ( + * {@link Reader#createScanner(long, long)}), or a portion of TFile with keys + * fall in a certain key range (for sorted TFile only, + * {@link Reader#createScanner(byte[], byte[])} or + * {@link Reader#createScanner(RawComparable, RawComparable)}). + */ + public static class Reader implements Closeable { + // The underlying BCFile reader. + final BCFile.Reader readerBCF; + + // TFile index, it is loaded lazily. + TFileIndex tfileIndex = null; + final TFileMeta tfileMeta; + final BytesComparator comparator; + + // global begin and end locations. + private final Location begin; + private final Location end; + + /** + * Location representing a virtual position in the TFile. + */ + static final class Location implements Comparable, Cloneable { + private int blockIndex; + // distance/offset from the beginning of the block + private long recordIndex; + + Location(int blockIndex, long recordIndex) { + set(blockIndex, recordIndex); + } + + void incRecordIndex() { + ++recordIndex; + } + + Location(Location other) { + set(other); + } + + int getBlockIndex() { + return blockIndex; + } + + long getRecordIndex() { + return recordIndex; + } + + void set(int blockIndex, long recordIndex) { + if ((blockIndex | recordIndex) < 0) { + throw new IllegalArgumentException( + "Illegal parameter for BlockLocation."); + } + this.blockIndex = blockIndex; + this.recordIndex = recordIndex; + } + + void set(Location other) { + set(other.blockIndex, other.recordIndex); + } + + /** + * @see java.lang.Comparable#compareTo(java.lang.Object) + */ + @Override + public int compareTo(Location other) { + return compareTo(other.blockIndex, other.recordIndex); + } + + int compareTo(int bid, long rid) { + if (this.blockIndex == bid) { + long ret = this.recordIndex - rid; + if (ret > 0) return 1; + if (ret < 0) return -1; + return 0; + } + return this.blockIndex - bid; + } + + /** + * @see java.lang.Object#clone() + */ + @Override + protected Location clone() { + return new Location(blockIndex, recordIndex); + } + + /** + * @see java.lang.Object#hashCode() + */ + @Override + public int hashCode() { + final int prime = 31; + int result = prime + blockIndex; + result = (int) (prime * result + recordIndex); + return result; + } + + /** + * @see java.lang.Object#equals(java.lang.Object) + */ + @Override + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null) return false; + if (getClass() != obj.getClass()) return false; + Location other = (Location) obj; + if (blockIndex != other.blockIndex) return false; + if (recordIndex != other.recordIndex) return false; + return true; + } + } + + /** + * Constructor + * + * @param fsdis + * FS input stream of the TFile. + * @param fileLength + * The length of TFile. This is required because we have no easy + * way of knowing the actual size of the input file through the + * File input stream. + * @param conf + * @throws IOException + */ + public Reader(FSDataInputStream fsdis, long fileLength, Configuration conf) + throws IOException { + readerBCF = new BCFile.Reader(fsdis, fileLength, conf); + + // first, read TFile meta + BlockReader brMeta = readerBCF.getMetaBlock(TFileMeta.BLOCK_NAME); + try { + tfileMeta = new TFileMeta(brMeta); + } finally { + brMeta.close(); + } + + comparator = tfileMeta.getComparator(); + // Set begin and end locations. + begin = new Location(0, 0); + end = new Location(readerBCF.getBlockCount(), 0); + } + + /** + * Close the reader. The state of the Reader object is undefined after + * close. Calling close() for multiple times has no effect. + */ + public void close() throws IOException { + readerBCF.close(); + } + + /** + * Get the begin location of the TFile. + * + * @return If TFile is not empty, the location of the first key-value pair. + * Otherwise, it returns end(). + */ + Location begin() { + return begin; + } + + /** + * Get the end location of the TFile. + * + * @return The location right after the last key-value pair in TFile. + */ + Location end() { + return end; + } + + /** + * Get the string representation of the comparator. + * + * @return If the TFile is not sorted by keys, an empty string will be + * returned. Otherwise, the actual comparator string that is + * provided during the TFile creation time will be returned. + */ + public String getComparatorName() { + return tfileMeta.getComparatorString(); + } + + /** + * Is the TFile sorted? + * + * @return true if TFile is sorted. + */ + public boolean isSorted() { + return tfileMeta.isSorted(); + } + + /** + * Get the number of key-value pair entries in TFile. + * + * @return the number of key-value pairs in TFile + */ + public long getEntryCount() { + return tfileMeta.getRecordCount(); + } + + /** + * Lazily loading the TFile index. + * + * @throws IOException + */ + synchronized void checkTFileDataIndex() throws IOException { + if (tfileIndex == null) { + BlockReader brIndex = readerBCF.getMetaBlock(TFileIndex.BLOCK_NAME); + try { + tfileIndex = + new TFileIndex(readerBCF.getBlockCount(), brIndex, tfileMeta + .getComparator()); + } finally { + brIndex.close(); + } + } + } + + /** + * Get the first key in the TFile. + * + * @return The first key in the TFile. + * @throws IOException + */ + public RawComparable getFirstKey() throws IOException { + checkTFileDataIndex(); + return tfileIndex.getFirstKey(); + } + + /** + * Get the last key in the TFile. + * + * @return The last key in the TFile. + * @throws IOException + */ + public RawComparable getLastKey() throws IOException { + checkTFileDataIndex(); + return tfileIndex.getLastKey(); + } + + /** + * Get a Comparator object to compare Entries. It is useful when you want + * stores the entries in a collection (such as PriorityQueue) and perform + * sorting or comparison among entries based on the keys without copying out + * the key. + * + * @return An Entry Comparator.. + */ + public Comparator getEntryComparator() { + if (!isSorted()) { + throw new RuntimeException( + "Entries are not comparable for unsorted TFiles"); + } + + return new Comparator() { + /** + * Provide a customized comparator for Entries. This is useful if we + * have a collection of Entry objects. However, if the Entry objects + * come from different TFiles, users must ensure that those TFiles share + * the same RawComparator. + */ + @Override + public int compare(Scanner.Entry o1, Scanner.Entry o2) { + return comparator.compare(o1.getKeyBuffer(), 0, o1.getKeyLength(), o2 + .getKeyBuffer(), 0, o2.getKeyLength()); + } + }; + } + + /** + * Get an instance of the RawComparator that is constructed based on the + * string comparator representation. + * + * @return a Comparator that can compare RawComparable's. + */ + public Comparator getComparator() { + return comparator; + } + + /** + * Stream access to a meta block.`` + * + * @param name + * The name of the meta block. + * @return The input stream. + * @throws IOException + * on I/O error. + * @throws MetaBlockDoesNotExist + * If the meta block with the name does not exist. + */ + public DataInputStream getMetaBlock(String name) throws IOException, + MetaBlockDoesNotExist { + return readerBCF.getMetaBlock(name); + } + + /** + * if greater is true then returns the beginning location of the block + * containing the key strictly greater than input key. if greater is false + * then returns the beginning location of the block greater than equal to + * the input key + * + * @param key + * the input key + * @param greater + * boolean flag + * @return + * @throws IOException + */ + Location getBlockContainsKey(RawComparable key, boolean greater) + throws IOException { + if (!isSorted()) { + throw new RuntimeException("Seeking in unsorted TFile"); + } + checkTFileDataIndex(); + int blkIndex = + (greater) ? tfileIndex.upperBound(key) : tfileIndex.lowerBound(key); + if (blkIndex < 0) return end; + return new Location(blkIndex, 0); + } + + int compareKeys(byte[] a, int o1, int l1, byte[] b, int o2, int l2) { + if (!isSorted()) { + throw new RuntimeException("Cannot compare keys for unsorted TFiles."); + } + return comparator.compare(a, o1, l1, b, o2, l2); + } + + int compareKeys(RawComparable a, RawComparable b) { + if (!isSorted()) { + throw new RuntimeException("Cannot compare keys for unsorted TFiles."); + } + return comparator.compare(a, b); + } + + /** + * Get the location pointing to the beginning of the first key-value pair in + * a compressed block whose byte offset in the TFile is greater than or + * equal to the specified offset. + * + * @param offset + * the user supplied offset. + * @return the location to the corresponding entry; or end() if no such + * entry exists. + */ + Location getLocationNear(long offset) { + int blockIndex = readerBCF.getBlockIndexNear(offset); + if (blockIndex == -1) return end; + return new Location(blockIndex, 0); + } + + /** + * Get a sample key that is within a block whose starting offset is greater + * than or equal to the specified offset. + * + * @param offset + * The file offset. + * @return the key that fits the requirement; or null if no such key exists + * (which could happen if the offset is close to the end of the + * TFile). + * @throws IOException + */ + public RawComparable getKeyNear(long offset) throws IOException { + int blockIndex = readerBCF.getBlockIndexNear(offset); + if (blockIndex == -1) return null; + checkTFileDataIndex(); + return new ByteArray(tfileIndex.getEntry(blockIndex).key); + } + + /** + * Get a scanner than can scan the whole TFile. + * + * @return The scanner object. A valid Scanner is always returned even if + * the TFile is empty. + * @throws IOException + */ + public Scanner createScanner() throws IOException { + return new Scanner(this, begin, end); + } + + /** + * Get a scanner that covers a portion of TFile based on byte offsets. + * + * @param offset + * The beginning byte offset in the TFile. + * @param length + * The length of the region. + * @return The actual coverage of the returned scanner tries to match the + * specified byte-region but always round up to the compression + * block boundaries. It is possible that the returned scanner + * contains zero key-value pairs even if length is positive. + * @throws IOException + */ + public Scanner createScanner(long offset, long length) throws IOException { + return new Scanner(this, offset, offset + length); + } + + /** + * Get a scanner that covers a portion of TFile based on keys. + * + * @param beginKey + * Begin key of the scan (inclusive). If null, scan from the first + * key-value entry of the TFile. + * @param endKey + * End key of the scan (exclusive). If null, scan up to the last + * key-value entry of the TFile. + * @return The actual coverage of the returned scanner will cover all keys + * greater than or equal to the beginKey and less than the endKey. + * @throws IOException + */ + public Scanner createScanner(byte[] beginKey, byte[] endKey) + throws IOException { + return createScanner((beginKey == null) ? null : new ByteArray(beginKey, + 0, beginKey.length), (endKey == null) ? null : new ByteArray(endKey, + 0, endKey.length)); + } + + /** + * Get a scanner that covers a specific key range. + * + * @param beginKey + * Begin key of the scan (inclusive). If null, scan from the first + * key-value entry of the TFile. + * @param endKey + * End key of the scan (exclusive). If null, scan up to the last + * key-value entry of the TFile. + * @return The actual coverage of the returned scanner will cover all keys + * greater than or equal to the beginKey and less than the endKey. + * @throws IOException + */ + public Scanner createScanner(RawComparable beginKey, RawComparable endKey) + throws IOException { + if ((beginKey != null) && (endKey != null) + && (compareKeys(beginKey, endKey) >= 0)) { + return new Scanner(this, beginKey, beginKey); + } + return new Scanner(this, beginKey, endKey); + } + + /** + * The TFile Scanner. The Scanner has an implicit cursor, which, upon + * creation, points to the first key-value pair in the scan range. If the + * scan range is empty, the cursor will point to the end of the scan range. + *

+ * Use {@link Scanner#atEnd()} to test whether the cursor is at the end + * location of the scanner. + *

+ * Use {@link Scanner#advance()} to move the cursor to the next key-value + * pair (or end if none exists). Use seekTo methods ( + * {@link Scanner#seekTo(byte[])} or + * {@link Scanner#seekTo(byte[], int, int)}) to seek to any arbitrary + * location in the covered range (including backward seeking). Use + * {@link Scanner#rewind()} to seek back to the beginning of the scanner. + * Use {@link Scanner#seekToEnd()} to seek to the end of the scanner. + *

+ * Actual keys and values may be obtained through {@link Scanner.Entry} + * object, which is obtained through {@link Scanner#entry()}. + */ + public static class Scanner implements Closeable { + // The underlying TFile reader. + final Reader reader; + // current block (null if reaching end) + private BlockReader blkReader; + + Location beginLocation; + Location endLocation; + Location currentLocation; + + // flag to ensure value is only examined once. + boolean valueChecked = false; + // reusable buffer for keys. + final byte[] keyBuffer; + // length of key, -1 means key is invalid. + int klen = -1; + + static final int MAX_VAL_TRANSFER_BUF_SIZE = 128 * 1024; + BytesWritable valTransferBuffer; + + DataInputBuffer keyDataInputStream; + ChunkDecoder valueBufferInputStream; + DataInputStream valueDataInputStream; + // vlen == -1 if unknown. + int vlen; + + /** + * Constructor + * + * @param reader + * The TFile reader object. + * @param offBegin + * Begin byte-offset of the scan. + * @param offEnd + * End byte-offset of the scan. + * @throws IOException + * + * The offsets will be rounded to the beginning of a compressed + * block whose offset is greater than or equal to the specified + * offset. + */ + protected Scanner(Reader reader, long offBegin, long offEnd) + throws IOException { + this(reader, reader.getLocationNear(offBegin), reader + .getLocationNear(offEnd)); + } + + /** + * Constructor + * + * @param reader + * The TFile reader object. + * @param begin + * Begin location of the scan. + * @param end + * End location of the scan. + * @throws IOException + */ + Scanner(Reader reader, Location begin, Location end) throws IOException { + this.reader = reader; + // ensure the TFile index is loaded throughout the life of scanner. + reader.checkTFileDataIndex(); + beginLocation = begin; + endLocation = end; + + valTransferBuffer = new BytesWritable(); + // TODO: remember the longest key in a TFile, and use it to replace + // MAX_KEY_SIZE. + keyBuffer = new byte[MAX_KEY_SIZE]; + keyDataInputStream = new DataInputBuffer(); + valueBufferInputStream = new ChunkDecoder(); + valueDataInputStream = new DataInputStream(valueBufferInputStream); + + if (beginLocation.compareTo(endLocation) >= 0) { + currentLocation = new Location(endLocation); + } else { + currentLocation = new Location(0, 0); + initBlock(beginLocation.getBlockIndex()); + inBlockAdvance(beginLocation.getRecordIndex()); + } + } + + /** + * Constructor + * + * @param reader + * The TFile reader object. + * @param beginKey + * Begin key of the scan. If null, scan from the first + * entry of the TFile. + * @param endKey + * End key of the scan. If null, scan up to the last entry + * of the TFile. + * @throws IOException + */ + protected Scanner(Reader reader, RawComparable beginKey, + RawComparable endKey) throws IOException { + this(reader, (beginKey == null) ? reader.begin() : reader + .getBlockContainsKey(beginKey, false), reader.end()); + if (beginKey != null) { + inBlockAdvance(beginKey, false); + beginLocation.set(currentLocation); + } + if (endKey != null) { + seekTo(endKey, false); + endLocation.set(currentLocation); + seekTo(beginLocation); + } + } + + /** + * Move the cursor to the first entry whose key is greater than or equal + * to the input key. Synonymous to seekTo(key, 0, key.length). The entry + * returned by the previous entry() call will be invalid. + * + * @param key + * The input key + * @return true if we find an equal key. + * @throws IOException + */ + public boolean seekTo(byte[] key) throws IOException { + return seekTo(key, 0, key.length); + } + + /** + * Move the cursor to the first entry whose key is greater than or equal + * to the input key. The entry returned by the previous entry() call will + * be invalid. + * + * @param key + * The input key + * @param keyOffset + * offset in the key buffer. + * @param keyLen + * key buffer length. + * @return true if we find an equal key; false otherwise. + * @throws IOException + */ + public boolean seekTo(byte[] key, int keyOffset, int keyLen) + throws IOException { + return seekTo(new ByteArray(key, keyOffset, keyLen), false); + } + + private boolean seekTo(RawComparable key, boolean beyond) + throws IOException { + Location l = reader.getBlockContainsKey(key, beyond); + if (l.compareTo(beginLocation) < 0) { + l = beginLocation; + } else if (l.compareTo(endLocation) >= 0) { + seekTo(endLocation); + return false; + } + + // check if what we are seeking is in the later part of the current + // block. + if (atEnd() || (l.getBlockIndex() != currentLocation.getBlockIndex()) + || (compareCursorKeyTo(key) >= 0)) { + // sorry, we must seek to a different location first. + seekTo(l); + } + + return inBlockAdvance(key, beyond); + } + + /** + * Move the cursor to the new location. The entry returned by the previous + * entry() call will be invalid. + * + * @param l + * new cursor location. It must fall between the begin and end + * location of the scanner. + * @throws IOException + */ + private void seekTo(Location l) throws IOException { + if (l.compareTo(beginLocation) < 0) { + throw new IllegalArgumentException( + "Attempt to seek before the begin location."); + } + + if (l.compareTo(endLocation) > 0) { + throw new IllegalArgumentException( + "Attempt to seek after the end location."); + } + + if (l.compareTo(endLocation) == 0) { + parkCursorAtEnd(); + return; + } + + if (l.getBlockIndex() != currentLocation.getBlockIndex()) { + // going to a totally different block + initBlock(l.getBlockIndex()); + } else { + if (valueChecked) { + // may temporarily go beyond the last record in the block (in which + // case the next if loop will always be true). + inBlockAdvance(1); + } + if (l.getRecordIndex() < currentLocation.getRecordIndex()) { + initBlock(l.getBlockIndex()); + } + } + + inBlockAdvance(l.getRecordIndex() - currentLocation.getRecordIndex()); + + return; + } + + /** + * Rewind to the first entry in the scanner. The entry returned by the + * previous entry() call will be invalid. + * + * @throws IOException + */ + public void rewind() throws IOException { + seekTo(beginLocation); + } + + /** + * Seek to the end of the scanner. The entry returned by the previous + * entry() call will be invalid. + * + * @throws IOException + */ + public void seekToEnd() throws IOException { + parkCursorAtEnd(); + } + + /** + * Move the cursor to the first entry whose key is greater than or equal + * to the input key. Synonymous to lowerBound(key, 0, key.length). The + * entry returned by the previous entry() call will be invalid. + * + * @param key + * The input key + * @throws IOException + */ + public void lowerBound(byte[] key) throws IOException { + lowerBound(key, 0, key.length); + } + + /** + * Move the cursor to the first entry whose key is greater than or equal + * to the input key. The entry returned by the previous entry() call will + * be invalid. + * + * @param key + * The input key + * @param keyOffset + * offset in the key buffer. + * @param keyLen + * key buffer length. + * @throws IOException + */ + public void lowerBound(byte[] key, int keyOffset, int keyLen) + throws IOException { + seekTo(new ByteArray(key, keyOffset, keyLen), false); + } + + /** + * Move the cursor to the first entry whose key is strictly greater than + * the input key. Synonymous to upperBound(key, 0, key.length). The entry + * returned by the previous entry() call will be invalid. + * + * @param key + * The input key + * @throws IOException + */ + public void upperBound(byte[] key) throws IOException { + upperBound(key, 0, key.length); + } + + /** + * Move the cursor to the first entry whose key is strictly greater than + * the input key. The entry returned by the previous entry() call will be + * invalid. + * + * @param key + * The input key + * @param keyOffset + * offset in the key buffer. + * @param keyLen + * key buffer length. + * @throws IOException + */ + public void upperBound(byte[] key, int keyOffset, int keyLen) + throws IOException { + seekTo(new ByteArray(key, keyOffset, keyLen), true); + } + + /** + * Move the cursor to the next key-value pair. The entry returned by the + * previous entry() call will be invalid. + * + * @return true if the cursor successfully moves. False when cursor is + * already at the end location and cannot be advanced. + * @throws IOException + */ + public boolean advance() throws IOException { + if (atEnd()) { + return false; + } + + int curBid = currentLocation.getBlockIndex(); + long curRid = currentLocation.getRecordIndex(); + long entriesInBlock = reader.getBlockEntryCount(curBid); + if (curRid + 1 >= entriesInBlock) { + if (endLocation.compareTo(curBid + 1, 0) <= 0) { + // last entry in TFile. + parkCursorAtEnd(); + } else { + // last entry in Block. + initBlock(curBid + 1); + } + } else { + inBlockAdvance(1); + } + return true; + } + + /** + * Load a compressed block for reading. Expecting blockIndex is valid. + * + * @throws IOException + */ + private void initBlock(int blockIndex) throws IOException { + klen = -1; + if (blkReader != null) { + try { + blkReader.close(); + } finally { + blkReader = null; + } + } + blkReader = reader.getBlockReader(blockIndex); + currentLocation.set(blockIndex, 0); + } + + private void parkCursorAtEnd() throws IOException { + klen = -1; + currentLocation.set(endLocation); + if (blkReader != null) { + try { + blkReader.close(); + } finally { + blkReader = null; + } + } + } + + /** + * Close the scanner. Release all resources. The behavior of using the + * scanner after calling close is not defined. The entry returned by the + * previous entry() call will be invalid. + */ + public void close() throws IOException { + parkCursorAtEnd(); + } + + /** + * Is cursor at the end location? + * + * @return true if the cursor is at the end location. + */ + public boolean atEnd() { + return (currentLocation.compareTo(endLocation) >= 0); + } + + /** + * check whether we have already successfully obtained the key. It also + * initializes the valueInputStream. + */ + void checkKey() throws IOException { + if (klen >= 0) return; + if (atEnd()) { + throw new EOFException("No key-value to read"); + } + klen = -1; + vlen = -1; + valueChecked = false; + + klen = Utils.readVInt(blkReader); + blkReader.readFully(keyBuffer, 0, klen); + valueBufferInputStream.reset(blkReader); + if (valueBufferInputStream.isLastChunk()) { + vlen = valueBufferInputStream.getRemain(); + } + } + + /** + * Get an entry to access the key and value. + * + * @return The Entry object to access the key and value. + * @throws IOException + */ + public Entry entry() throws IOException { + checkKey(); + return new Entry(); + } + + /** + * Internal API. Comparing the key at cursor to user-specified key. + * + * @param other + * user-specified key. + * @return negative if key at cursor is smaller than user key; 0 if equal; + * and positive if key at cursor greater than user key. + * @throws IOException + */ + int compareCursorKeyTo(RawComparable other) throws IOException { + checkKey(); + return reader.compareKeys(keyBuffer, 0, klen, other.buffer(), other + .offset(), other.size()); + } + + /** + * Entry to a <Key, Value> pair. + */ + public class Entry implements Comparable { + /** + * Get the length of the key. + * + * @return the length of the key. + */ + public int getKeyLength() { + return klen; + } + + byte[] getKeyBuffer() { + return keyBuffer; + } + + /** + * Copy the key and value in one shot into BytesWritables. This is + * equivalent to getKey(key); getValue(value); + * + * @param key + * BytesWritable to hold key. + * @param value + * BytesWritable to hold value + * @throws IOException + */ + public void get(BytesWritable key, BytesWritable value) + throws IOException { + getKey(key); + getValue(value); + } + + /** + * Copy the key into BytesWritable. The input BytesWritable will be + * automatically resized to the actual key size. + * + * @param key + * BytesWritable to hold the key. + * @throws IOException + */ + public int getKey(BytesWritable key) throws IOException { + key.setSize(getKeyLength()); + getKey(key.get()); + return key.getSize(); + } + + /** + * Copy the value into BytesWritable. The input BytesWritable will be + * automatically resized to the actual value size. The implementation + * directly uses the buffer inside BytesWritable for storing the value. + * The call does not require the value length to be known. + * + * @param value + * @throws IOException + */ + public long getValue(BytesWritable value) throws IOException { + DataInputStream dis = getValueStream(); + int size = 0; + try { + int remain; + while ((remain = valueBufferInputStream.getRemain()) > 0) { + value.setSize(size + remain); + dis.readFully(value.get(), size, remain); + size += remain; + } + return value.getSize(); + } finally { + dis.close(); + } + } + + /** + * Writing the key to the output stream. This method avoids copying key + * buffer from Scanner into user buffer, then writing to the output + * stream. + * + * @param out + * The output stream + * @return the length of the key. + * @throws IOException + */ + public int writeKey(OutputStream out) throws IOException { + out.write(keyBuffer, 0, klen); + return klen; + } + + /** + * Writing the value to the output stream. This method avoids copying + * value data from Scanner into user buffer, then writing to the output + * stream. It does not require the value length to be known. + * + * @param out + * The output stream + * @return the length of the value + * @throws IOException + */ + public long writeValue(OutputStream out) throws IOException { + DataInputStream dis = getValueStream(); + long size = 0; + try { + int chunkSize; + while ((chunkSize = valueBufferInputStream.getRemain()) > 0) { + chunkSize = Math.min(chunkSize, MAX_VAL_TRANSFER_BUF_SIZE); + valTransferBuffer.setSize(chunkSize); + dis.readFully(valTransferBuffer.get(), 0, chunkSize); + out.write(valTransferBuffer.get(), 0, chunkSize); + size += chunkSize; + } + return size; + } finally { + dis.close(); + } + } + + /** + * Copy the key into user supplied buffer. + * + * @param buf + * The buffer supplied by user. The length of the buffer must + * not be shorter than the key length. + * @return The length of the key. + * + * @throws IOException + */ + public int getKey(byte[] buf) throws IOException { + return getKey(buf, 0); + } + + /** + * Copy the key into user supplied buffer. + * + * @param buf + * The buffer supplied by user. + * @param offset + * The starting offset of the user buffer where we should copy + * the key into. Requiring the key-length + offset no greater + * than the buffer length. + * @return The length of the key. + * @throws IOException + */ + public int getKey(byte[] buf, int offset) throws IOException { + if ((offset | (buf.length - offset - klen)) < 0) { + throw new IndexOutOfBoundsException( + "Bufer not enough to store the key"); + } + System.arraycopy(keyBuffer, 0, buf, offset, klen); + return klen; + } + + /** + * Streaming access to the key. Useful for desrializing the key into + * user objects. + * + * @return The input stream. + */ + public DataInputStream getKeyStream() { + keyDataInputStream.reset(keyBuffer, klen); + return keyDataInputStream; + } + + /** + * Get the length of the value. isValueLengthKnown() must be tested + * true. + * + * @return the length of the value. + */ + public int getValueLength() { + if (vlen >= 0) { + return vlen; + } + + throw new RuntimeException("Value length unknown."); + } + + /** + * Copy value into user-supplied buffer. User supplied buffer must be + * large enough to hold the whole value. The value part of the key-value + * pair pointed by the current cursor is not cached and can only be + * examined once. Calling any of the following functions more than once + * without moving the cursor will result in exception: + * {@link #getValue(byte[])}, {@link #getValue(byte[], int)}, + * {@link #getValueStream}. + * + * @return the length of the value. Does not require + * isValueLengthKnown() to be true. + * @throws IOException + * + */ + public int getValue(byte[] buf) throws IOException { + return getValue(buf, 0); + } + + /** + * Copy value into user-supplied buffer. User supplied buffer must be + * large enough to hold the whole value (starting from the offset). The + * value part of the key-value pair pointed by the current cursor is not + * cached and can only be examined once. Calling any of the following + * functions more than once without moving the cursor will result in + * exception: {@link #getValue(byte[])}, {@link #getValue(byte[], int)}, + * {@link #getValueStream}. + * + * @return the length of the value. Does not require + * isValueLengthKnown() to be true. + * @throws IOException + */ + public int getValue(byte[] buf, int offset) throws IOException { + DataInputStream dis = getValueStream(); + try { + if (isValueLengthKnown()) { + if ((offset | (buf.length - offset - vlen)) < 0) { + throw new IndexOutOfBoundsException( + "Buffer too small to hold value"); + } + dis.readFully(buf, offset, vlen); + return vlen; + } + + int nextOffset = offset; + while (nextOffset < buf.length) { + int n = dis.read(buf, nextOffset, buf.length - nextOffset); + if (n < 0) { + break; + } + nextOffset += n; + } + if (dis.read() >= 0) { + // attempt to read one more byte to determine whether we reached + // the + // end or not. + throw new IndexOutOfBoundsException( + "Buffer too small to hold value"); + } + return nextOffset - offset; + } finally { + dis.close(); + } + } + + /** + * Stream access to value. The value part of the key-value pair pointed + * by the current cursor is not cached and can only be examined once. + * Calling any of the following functions more than once without moving + * the cursor will result in exception: {@link #getValue(byte[])}, + * {@link #getValue(byte[], int)}, {@link #getValueStream}. + * + * @return The input stream for reading the value. + * @throws IOException + */ + public DataInputStream getValueStream() throws IOException { + if (valueChecked == true) { + throw new IllegalStateException( + "Attempt to examine value multiple times."); + } + valueChecked = true; + return valueDataInputStream; + } + + /** + * Check whether it is safe to call getValueLength(). + * + * @return true if value length is known before hand. Values less than + * the chunk size will always have their lengths known before + * hand. Values that are written out as a whole (with advertised + * length up-front) will always have their lengths known in + * read. + */ + public boolean isValueLengthKnown() { + return (vlen >= 0); + } + + /** + * Compare the entry key to another key. Synonymous to compareTo(key, 0, + * key.length). + * + * @param buf + * The key buffer. + * @return comparison result between the entry key with the input key. + */ + public int compareTo(byte[] buf) { + return compareTo(buf, 0, buf.length); + } + + /** + * Compare the entry key to another key. Synonymous to compareTo(new + * ByteArray(buf, offset, length) + * + * @param buf + * The key buffer + * @param offset + * offset into the key buffer. + * @param length + * the length of the key. + * @return comparison result between the entry key with the input key. + */ + public int compareTo(byte[] buf, int offset, int length) { + return compareTo(new ByteArray(buf, offset, length)); + } + + /** + * Compare an entry with a RawComparable object. This is useful when + * Entries are stored in a collection, and we want to compare a user + * supplied key. + */ + @Override + public int compareTo(RawComparable key) { + return reader.compareKeys(keyBuffer, 0, getKeyLength(), key.buffer(), + key.offset(), key.size()); + } + + /** + * Compare whether this and other points to the same key value. + */ + @Override + public boolean equals(Object other) { + if (this == other) return true; + if (!(other instanceof Entry)) return false; + return ((Entry) other).compareTo(keyBuffer, 0, getKeyLength()) == 0; + } + + @Override + public int hashCode() { + return WritableComparator.hashBytes(keyBuffer, 0, getKeyLength()); + } + } + + /** + * Advance cursor by n positions within the block. + * + * @param n + * Number of key-value pairs to skip in block. + * @throws IOException + */ + private void inBlockAdvance(long n) throws IOException { + for (long i = 0; i < n; ++i) { + checkKey(); + if (!valueBufferInputStream.isClosed()) { + valueBufferInputStream.close(); + } + klen = -1; + currentLocation.incRecordIndex(); + } + } + + /** + * Advance cursor in block until we find a key that is greater than or + * equal to the input key. + * + * @param key + * Key to compare. + * @param greater + * advance until we find a key greater than the input key. + * @return true if we find a equal key. + * @throws IOException + */ + private boolean inBlockAdvance(RawComparable key, boolean greater) + throws IOException { + int curBid = currentLocation.getBlockIndex(); + long entryInBlock = reader.getBlockEntryCount(curBid); + if (curBid == endLocation.getBlockIndex()) { + entryInBlock = endLocation.getRecordIndex(); + } + + while (currentLocation.getRecordIndex() < entryInBlock) { + int cmp = compareCursorKeyTo(key); + if (cmp > 0) return false; + if (cmp == 0 && !greater) return true; + if (!valueBufferInputStream.isClosed()) { + valueBufferInputStream.close(); + } + klen = -1; + currentLocation.incRecordIndex(); + } + + throw new RuntimeException("Cannot find matching key in block."); + } + } + + long getBlockEntryCount(int curBid) { + return tfileIndex.getEntry(curBid).entries(); + } + + BlockReader getBlockReader(int blockIndex) throws IOException { + return readerBCF.getDataBlock(blockIndex); + } + } + + /** + * Data structure representing "TFile.meta" meta block. + */ + static final class TFileMeta { + final static String BLOCK_NAME = "TFile.meta"; + final Version version; + private long recordCount; + private final String strComparator; + private final BytesComparator comparator; + + // ctor for writes + public TFileMeta(String comparator) { + // set fileVersion to API version when we create it. + version = TFile.API_VERSION; + recordCount = 0; + strComparator = (comparator == null) ? "" : comparator; + this.comparator = makeComparator(strComparator); + } + + // ctor for reads + public TFileMeta(DataInput in) throws IOException { + version = new Version(in); + if (!version.compatibleWith(TFile.API_VERSION)) { + throw new RuntimeException("Incompatible TFile fileVersion."); + } + recordCount = Utils.readVLong(in); + strComparator = Utils.readString(in); + comparator = makeComparator(strComparator); + } + + @SuppressWarnings("unchecked") + private static BytesComparator makeComparator(String comparator) { + if (comparator.length() == 0) { + // unsorted keys + return null; + } + if (comparator.equals(COMPARATOR_MEMCMP)) { + // default comparator + return new BytesComparator(new MemcmpRawComparator()); + } else if (comparator.startsWith(COMPARATOR_JCLASS)) { + String compClassName = + comparator.substring(COMPARATOR_JCLASS.length()).trim(); + try { + Class compClass = Class.forName(compClassName); + // use its default ctor to create an instance + return new BytesComparator((RawComparator) compClass + .newInstance()); + } catch (Exception e) { + throw new IllegalArgumentException( + "Failed to instantiate comparator: " + comparator + "(" + + e.toString() + ")"); + } + } else { + throw new IllegalArgumentException("Unsupported comparator: " + + comparator); + } + } + + public void write(DataOutput out) throws IOException { + TFile.API_VERSION.write(out); + Utils.writeVLong(out, recordCount); + Utils.writeString(out, strComparator); + } + + public long getRecordCount() { + return recordCount; + } + + public void incRecordCount() { + ++recordCount; + } + + public boolean isSorted() { + return !strComparator.equals(""); + } + + public String getComparatorString() { + return strComparator; + } + + public BytesComparator getComparator() { + return comparator; + } + + public Version getVersion() { + return version; + } + } // END: class MetaTFileMeta + + /** + * Data structure representing "TFile.index" meta block. + */ + static class TFileIndex { + final static String BLOCK_NAME = "TFile.index"; + private ByteArray firstKey; + private final ArrayList index; + private final BytesComparator comparator; + + /** + * For reading from file. + * + * @throws IOException + */ + public TFileIndex(int entryCount, DataInput in, BytesComparator comparator) + throws IOException { + index = new ArrayList(entryCount); + int size = Utils.readVInt(in); // size for the first key entry. + if (size > 0) { + byte[] buffer = new byte[size]; + in.readFully(buffer); + DataInputStream firstKeyInputStream = + new DataInputStream(new ByteArrayInputStream(buffer, 0, size)); + + int firstKeyLength = Utils.readVInt(firstKeyInputStream); + firstKey = new ByteArray(new byte[firstKeyLength]); + firstKeyInputStream.readFully(firstKey.buffer()); + + for (int i = 0; i < entryCount; i++) { + size = Utils.readVInt(in); + if (buffer.length < size) { + buffer = new byte[size]; + } + in.readFully(buffer, 0, size); + TFileIndexEntry idx = + new TFileIndexEntry(new DataInputStream(new ByteArrayInputStream( + buffer, 0, size))); + index.add(idx); + } + } else { + if (entryCount != 0) { + throw new RuntimeException("Internal error"); + } + } + this.comparator = comparator; + } + + /** + * @param key + * input key. + * @return the ID of the first block that contains key >= input key. Or -1 + * if no such block exists. + */ + public int lowerBound(RawComparable key) { + if (comparator == null) { + throw new RuntimeException("Cannot search in unsorted TFile"); + } + + if (firstKey == null) { + return -1; // not found + } + + int ret = Utils.lowerBound(index, key, comparator); + if (ret == index.size()) { + return -1; + } + return ret; + } + + public int upperBound(RawComparable key) { + if (comparator == null) { + throw new RuntimeException("Cannot search in unsorted TFile"); + } + + if (firstKey == null) { + return -1; // not found + } + + int ret = Utils.upperBound(index, key, comparator); + if (ret == index.size()) { + return -1; + } + return ret; + } + + /** + * For writing to file. + */ + public TFileIndex(BytesComparator comparator) { + index = new ArrayList(); + this.comparator = comparator; + } + + public RawComparable getFirstKey() { + return firstKey; + } + + public void setFirstKey(byte[] key, int offset, int length) { + firstKey = new ByteArray(new byte[length]); + System.arraycopy(key, offset, firstKey.buffer(), 0, length); + } + + public RawComparable getLastKey() { + if (index.size() == 0) { + return null; + } + return new ByteArray(index.get(index.size() - 1).buffer()); + } + + public void addEntry(TFileIndexEntry keyEntry) { + index.add(keyEntry); + } + + public TFileIndexEntry getEntry(int bid) { + return index.get(bid); + } + + public void write(DataOutput out) throws IOException { + if (firstKey == null) { + Utils.writeVInt(out, 0); + return; + } + + DataOutputBuffer dob = new DataOutputBuffer(); + Utils.writeVInt(dob, firstKey.size()); + dob.write(firstKey.buffer()); + Utils.writeVInt(out, dob.size()); + out.write(dob.getData(), 0, dob.getLength()); + + for (TFileIndexEntry entry : index) { + dob.reset(); + entry.write(dob); + Utils.writeVInt(out, dob.getLength()); + out.write(dob.getData(), 0, dob.getLength()); + } + } + } + + /** + * TFile Data Index entry. We should try to make the memory footprint of each + * index entry as small as possible. + */ + static final class TFileIndexEntry implements RawComparable { + final byte[] key; + // count of entries in the block. + final long kvEntries; + + public TFileIndexEntry(DataInput in) throws IOException { + int len = Utils.readVInt(in); + key = new byte[len]; + in.readFully(key, 0, len); + kvEntries = Utils.readVLong(in); + } + + // default entry, without any padding + public TFileIndexEntry(byte[] newkey, int offset, int len, long entries) { + key = new byte[len]; + System.arraycopy(newkey, offset, key, 0, len); + this.kvEntries = entries; + } + + @Override + public byte[] buffer() { + return key; + } + + @Override + public int offset() { + return 0; + } + + @Override + public int size() { + return key.length; + } + + long entries() { + return kvEntries; + } + + public void write(DataOutput out) throws IOException { + Utils.writeVInt(out, key.length); + out.write(key, 0, key.length); + Utils.writeVLong(out, kvEntries); + } + } + + /** + * Dumping the TFile information. + * + * @param args + * A list of TFile paths. + */ + public static void main(String[] args) { + System.out.printf("TFile Dumper (TFile %s, BCFile %s)\n", TFile.API_VERSION + .toString(), BCFile.API_VERSION.toString()); + if (args.length == 0) { + System.out + .println("Usage: java ... org.apache.hadoop.io.file.tfile.TFile tfile-path [tfile-path ...]"); + System.exit(0); + } + Configuration conf = new Configuration(); + + for (String file : args) { + System.out.println("===" + file + "==="); + try { + TFileDumper.dumpInfo(file, System.out, conf); + } catch (IOException e) { + e.printStackTrace(System.err); + } + } + } +} diff --git a/src/java/org/apache/hadoop/io/file/tfile/TFileDumper.java b/src/java/org/apache/hadoop/io/file/tfile/TFileDumper.java new file mode 100644 index 0000000000..f065e62d0f --- /dev/null +++ b/src/java/org/apache/hadoop/io/file/tfile/TFileDumper.java @@ -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 properties = + new LinkedHashMap(); + 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 metaBlks = + reader.readerBCF.metaIndex.index.values(); + boolean calculateCompression = false; + for (Iterator 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> entrySet = properties.entrySet(); + for (Iterator> it = entrySet.iterator(); it + .hasNext();) { + Map.Entry e = it.next(); + if (e.getKey().length() > maxKeyLength) { + maxKeyLength = e.getKey().length(); + } + } + for (Iterator> it = entrySet.iterator(); it + .hasNext();) { + Map.Entry 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> metaBlkEntrySet = + reader.readerBCF.metaIndex.index.entrySet(); + for (Iterator> it = + metaBlkEntrySet.iterator(); it.hasNext();) { + Map.Entry 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> it = + metaBlkEntrySet.iterator(); it.hasNext();) { + Map.Entry 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); + } + } +} diff --git a/src/java/org/apache/hadoop/io/file/tfile/Utils.java b/src/java/org/apache/hadoop/io/file/tfile/Utils.java new file mode 100644 index 0000000000..74c7504bad --- /dev/null +++ b/src/java/org/apache/hadoop/io/file/tfile/Utils.java @@ -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 + * Utils#writeVLong(out, n). + * + * @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. + *
    + *
  • if n in [-32, 127): encode in one byte with the actual value. + * Otherwise, + *
  • if n in [-20*2^8, 20*2^8): encode in two bytes: byte[0] = n/256 - 52; + * byte[1]=n&0xff. Otherwise, + *
  • 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, + *
  • 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: + *
  • 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; + *
  • 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 + *
  • 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; + *
  • 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; + *
  • 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; + *
+ * + * @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 + * (int)Utils#readVLong(in). + * + * @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[*]. + *
    + *
  • if (FB >= -32), return (long)FB; + *
  • if (FB in [-72, -33]), return (FB+52)<<8 + NB[0]&0xff; + *
  • if (FB in [-104, -73]), return (FB+88)<<16 + (NB[0]&0xff)<<8 + + * NB[1]&0xff; + *
  • if (FB in [-120, -105]), return (FB+112)<<24 + (NB[0]&0xff)<<16 + + * (NB[1]&0xff)<<8 + NB[2]&0xff; + *
  • 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 { + 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 + * 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 int lowerBound(List list, T key, + Comparator 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 + * 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 int upperBound(List list, T key, + Comparator 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 + * 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 int lowerBound(List> list, + T key) { + int low = 0; + int high = list.size(); + + while (low < high) { + int mid = (low + high) >>> 1; + Comparable 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 + * 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 int upperBound(List> list, + T key) { + int low = 0; + int high = list.size(); + + while (low < high) { + int mid = (low + high) >>> 1; + Comparable midVal = list.get(mid); + int ret = midVal.compareTo(key); + if (ret <= 0) + low = mid + 1; + else high = mid; + } + return low; + } +} diff --git a/src/test/core/org/apache/hadoop/io/file/tfile/KVGenerator.java b/src/test/core/org/apache/hadoop/io/file/tfile/KVGenerator.java new file mode 100644 index 0000000000..86671ab994 --- /dev/null +++ b/src/test/core/org/apache/hadoop/io/file/tfile/KVGenerator.java @@ -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 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); + } +} diff --git a/src/test/core/org/apache/hadoop/io/file/tfile/KeySampler.java b/src/test/core/org/apache/hadoop/io/file/tfile/KeySampler.java new file mode 100644 index 0000000000..92c9c52471 --- /dev/null +++ b/src/test/core/org/apache/hadoop/io/file/tfile/KeySampler.java @@ -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; + } +} diff --git a/src/test/core/org/apache/hadoop/io/file/tfile/NanoTimer.java b/src/test/core/org/apache/hadoop/io/file/tfile/NanoTimer.java new file mode 100644 index 0000000000..1584895407 --- /dev/null +++ b/src/test/core/org/apache/hadoop/io/file/tfile/NanoTimer.java @@ -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)); + } + } +} + diff --git a/src/test/core/org/apache/hadoop/io/file/tfile/RandomDistribution.java b/src/test/core/org/apache/hadoop/io/file/tfile/RandomDistribution.java new file mode 100644 index 0000000000..d019c5253a --- /dev/null +++ b/src/test/core/org/apache/hadoop/io/file/tfile/RandomDistribution.java @@ -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 k; + private final ArrayList 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(); + v = new ArrayList(); + + 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; + } + } +} diff --git a/src/test/core/org/apache/hadoop/io/file/tfile/TestTFile.java b/src/test/core/org/apache/hadoop/io/file/tfile/TestTFile.java new file mode 100644 index 0000000000..4957fc3a6d --- /dev/null +++ b/src/test/core/org/apache/hadoop/io/file/tfile/TestTFile.java @@ -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(); + } +} diff --git a/src/test/core/org/apache/hadoop/io/file/tfile/TestTFileByteArrays.java b/src/test/core/org/apache/hadoop/io/file/tfile/TestTFileByteArrays.java new file mode 100644 index 0000000000..2d397c7e3a --- /dev/null +++ b/src/test/core/org/apache/hadoop/io/file/tfile/TestTFileByteArrays.java @@ -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; + } + } +} diff --git a/src/test/core/org/apache/hadoop/io/file/tfile/TestTFileComparators.java b/src/test/core/org/apache/hadoop/io/file/tfile/TestTFileComparators.java new file mode 100644 index 0000000000..5bb191854d --- /dev/null +++ b/src/test/core/org/apache/hadoop/io/file/tfile/TestTFileComparators.java @@ -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; + } + } +} diff --git a/src/test/core/org/apache/hadoop/io/file/tfile/TestTFileJClassComparatorByteArrays.java b/src/test/core/org/apache/hadoop/io/file/tfile/TestTFileJClassComparatorByteArrays.java new file mode 100644 index 0000000000..89b992db89 --- /dev/null +++ b/src/test/core/org/apache/hadoop/io/file/tfile/TestTFileJClassComparatorByteArrays.java @@ -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 { + + @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); + } + +} + diff --git a/src/test/core/org/apache/hadoop/io/file/tfile/TestTFileLzoCodecsByteArrays.java b/src/test/core/org/apache/hadoop/io/file/tfile/TestTFileLzoCodecsByteArrays.java new file mode 100644 index 0000000000..c3f0e4e694 --- /dev/null +++ b/src/test/core/org/apache/hadoop/io/file/tfile/TestTFileLzoCodecsByteArrays.java @@ -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(); + } +} diff --git a/src/test/core/org/apache/hadoop/io/file/tfile/TestTFileLzoCodecsStreams.java b/src/test/core/org/apache/hadoop/io/file/tfile/TestTFileLzoCodecsStreams.java new file mode 100644 index 0000000000..8ac911be9d --- /dev/null +++ b/src/test/core/org/apache/hadoop/io/file/tfile/TestTFileLzoCodecsStreams.java @@ -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(); + } +} diff --git a/src/test/core/org/apache/hadoop/io/file/tfile/TestTFileNoneCodecsByteArrays.java b/src/test/core/org/apache/hadoop/io/file/tfile/TestTFileNoneCodecsByteArrays.java new file mode 100644 index 0000000000..6fde1d8c1d --- /dev/null +++ b/src/test/core/org/apache/hadoop/io/file/tfile/TestTFileNoneCodecsByteArrays.java @@ -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(); + } +} diff --git a/src/test/core/org/apache/hadoop/io/file/tfile/TestTFileNoneCodecsJClassComparatorByteArrays.java b/src/test/core/org/apache/hadoop/io/file/tfile/TestTFileNoneCodecsJClassComparatorByteArrays.java new file mode 100644 index 0000000000..bd56d44965 --- /dev/null +++ b/src/test/core/org/apache/hadoop/io/file/tfile/TestTFileNoneCodecsJClassComparatorByteArrays.java @@ -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(); + } +} diff --git a/src/test/core/org/apache/hadoop/io/file/tfile/TestTFileNoneCodecsStreams.java b/src/test/core/org/apache/hadoop/io/file/tfile/TestTFileNoneCodecsStreams.java new file mode 100644 index 0000000000..54621f3c5b --- /dev/null +++ b/src/test/core/org/apache/hadoop/io/file/tfile/TestTFileNoneCodecsStreams.java @@ -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(); + } +} diff --git a/src/test/core/org/apache/hadoop/io/file/tfile/TestTFileSeek.java b/src/test/core/org/apache/hadoop/io/file/tfile/TestTFileSeek.java new file mode 100644 index 0000000000..d6bbcad359 --- /dev/null +++ b/src/test/core/org/apache/hadoop/io/file/tfile/TestTFileSeek.java @@ -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(); + } +} diff --git a/src/test/core/org/apache/hadoop/io/file/tfile/TestTFileSeqFileComparison.java b/src/test/core/org/apache/hadoop/io/file/tfile/TestTFileSeqFileComparison.java new file mode 100644 index 0000000000..a9df13752d --- /dev/null +++ b/src/test/core/org/apache/hadoop/io/file/tfile/TestTFileSeqFileComparison.java @@ -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(); + } +} diff --git a/src/test/core/org/apache/hadoop/io/file/tfile/TestTFileSplit.java b/src/test/core/org/apache/hadoop/io/file/tfile/TestTFileSplit.java new file mode 100644 index 0000000000..78ec270ec7 --- /dev/null +++ b/src/test/core/org/apache/hadoop/io/file/tfile/TestTFileSplit.java @@ -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); + } +} diff --git a/src/test/core/org/apache/hadoop/io/file/tfile/TestTFileStreams.java b/src/test/core/org/apache/hadoop/io/file/tfile/TestTFileStreams.java new file mode 100644 index 0000000000..d7055406d7 --- /dev/null +++ b/src/test/core/org/apache/hadoop/io/file/tfile/TestTFileStreams.java @@ -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; + } + } +} diff --git a/src/test/core/org/apache/hadoop/io/file/tfile/TestTFileUnsortedByteArrays.java b/src/test/core/org/apache/hadoop/io/file/tfile/TestTFileUnsortedByteArrays.java new file mode 100644 index 0000000000..2825647857 --- /dev/null +++ b/src/test/core/org/apache/hadoop/io/file/tfile/TestTFileUnsortedByteArrays.java @@ -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; + } + } +} diff --git a/src/test/core/org/apache/hadoop/io/file/tfile/TestVLong.java b/src/test/core/org/apache/hadoop/io/file/tfile/TestVLong.java new file mode 100644 index 0000000000..1132117117 --- /dev/null +++ b/src/test/core/org/apache/hadoop/io/file/tfile/TestVLong.java @@ -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); + } +} diff --git a/src/test/core/org/apache/hadoop/io/file/tfile/Timer.java b/src/test/core/org/apache/hadoop/io/file/tfile/Timer.java new file mode 100644 index 0000000000..c0084e181b --- /dev/null +++ b/src/test/core/org/apache/hadoop/io/file/tfile/Timer.java @@ -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()); + } + +} + diff --git a/src/test/findbugsExcludeFile.xml b/src/test/findbugsExcludeFile.xml index 4ff3500db6..d5852856d8 100644 --- a/src/test/findbugsExcludeFile.xml +++ b/src/test/findbugsExcludeFile.xml @@ -213,4 +213,26 @@ + + + + + + + + + + + + +