You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@apex.apache.org by hs...@apache.org on 2015/11/07 01:57:00 UTC
[1/8] incubator-apex-malhar git commit: MLHR-1877 #resolve #comment
moved DTFile implementation to from contrib to lib
Repository: incubator-apex-malhar
Updated Branches:
refs/heads/devel-3 c1ebde9c2 -> 02f48e1b2
http://git-wip-us.apache.org/repos/asf/incubator-apex-malhar/blob/02f48e1b/library/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileSeqFileComparison.java
----------------------------------------------------------------------
diff --git a/library/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileSeqFileComparison.java b/library/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileSeqFileComparison.java
new file mode 100644
index 0000000..e513ccd
--- /dev/null
+++ b/library/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileSeqFileComparison.java
@@ -0,0 +1,802 @@
+/**
+ * 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.fs.CommonConfigurationKeys;
+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.DTFile.Reader.Scanner.Entry;
+import org.apache.hadoop.util.Time;
+
+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 = Time.now();
+ System.out.println(formatTime() + " Started timing.");
+ }
+
+ public void stopTime() throws IOException {
+ finishTimeEpoch = Time.now();
+ 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(Time.now());
+ }
+
+ 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);
+ }
+
+ @Override
+ public void append(BytesWritable key, BytesWritable value)
+ throws IOException {
+ writer.append(key.get(), 0, key.getSize(), value.get(), 0, value
+ .getSize());
+ }
+
+ @Override
+ public void close() throws IOException {
+ writer.close();
+ fsdos.close();
+ }
+ }
+
+ static class TFileReadable implements KVReadable {
+ private FSDataInputStream fsdis;
+ private DTFile.Reader reader;
+ private DTFile.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 DTFile.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)];
+ }
+
+ @Override
+ public byte[] getKey() {
+ return keyBuffer;
+ }
+
+ @Override
+ public int getKeyLength() {
+ return keyLength;
+ }
+
+ @Override
+ public byte[] getValue() {
+ return valueBuffer;
+ }
+
+ @Override
+ public int getValueLength() {
+ return valueLength;
+ }
+
+ @Override
+ 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;
+ }
+
+ @Override
+ 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(CommonConfigurationKeys.IO_NATIVE_LIB_AVAILABLE_KEY,
+ 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);
+ }
+ }
+
+ @Override
+ public void append(BytesWritable key, BytesWritable value)
+ throws IOException {
+ writer.append(key, value);
+ }
+
+ @Override
+ 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();
+ }
+
+ @Override
+ public byte[] getKey() {
+ return key.get();
+ }
+
+ @Override
+ public int getKeyLength() {
+ return key.getSize();
+ }
+
+ @Override
+ public byte[] getValue() {
+ return value.get();
+ }
+
+ @Override
+ public int getValueLength() {
+ return value.getSize();
+ }
+
+ @Override
+ public boolean next() throws IOException {
+ return reader.next(key, value);
+ }
+
+ @Override
+ 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();
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-apex-malhar/blob/02f48e1b/library/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileSplit.java
----------------------------------------------------------------------
diff --git a/library/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileSplit.java b/library/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileSplit.java
new file mode 100644
index 0000000..aad563d
--- /dev/null
+++ b/library/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileSplit.java
@@ -0,0 +1,194 @@
+/**
+ * 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.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.DTFile.Reader;
+import org.apache.hadoop.io.file.tfile.DTFile.Writer;
+import org.apache.hadoop.io.file.tfile.DTFile.Reader.Scanner;
+import org.junit.Assert;
+
+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 Random random = new Random();
+
+ 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.createScannerByByteRange(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();
+ }
+
+ /* Similar to readFile(), tests the scanner created
+ * by record numbers rather than the offsets.
+ */
+ void readRowSplits(int numSplits) throws IOException {
+
+ Reader reader =
+ new Reader(fs.open(path), fs.getFileStatus(path).getLen(), conf);
+
+ long totalRecords = reader.getEntryCount();
+ for (int i=0; i<numSplits; i++) {
+ long startRec = i*totalRecords/numSplits;
+ long endRec = (i+1)*totalRecords/numSplits;
+ if (i == numSplits-1) {
+ endRec = totalRecords;
+ }
+ Scanner scanner = reader.createScannerByRecordNum(startRec, endRec);
+ int count = 0;
+ BytesWritable key = new BytesWritable();
+ BytesWritable value = new BytesWritable();
+ long x=startRec;
+ while (!scanner.atEnd()) {
+ assertEquals("Incorrect RecNum returned by scanner", scanner.getRecordNum(), x);
+ scanner.entry().get(key, value);
+ ++count;
+ assertEquals("Incorrect RecNum returned by scanner", scanner.getRecordNum(), x);
+ scanner.advance();
+ ++x;
+ }
+ scanner.close();
+ Assert.assertTrue(count == (endRec - startRec));
+ }
+ // make sure specifying range at the end gives zero records.
+ Scanner scanner = reader.createScannerByRecordNum(totalRecords, -1);
+ Assert.assertTrue(scanner.atEnd());
+ }
+
+ static String composeSortedKey(String prefix, int total, int value) {
+ return String.format("%s%010d", prefix, value);
+ }
+
+ void checkRecNums() throws IOException {
+ long fileLen = fs.getFileStatus(path).getLen();
+ Reader reader = new Reader(fs.open(path), fileLen, conf);
+ long totalRecs = reader.getEntryCount();
+ long begin = random.nextLong() % (totalRecs / 2);
+ if (begin < 0)
+ begin += (totalRecs / 2);
+ long end = random.nextLong() % (totalRecs / 2);
+ if (end < 0)
+ end += (totalRecs / 2);
+ end += (totalRecs / 2) + 1;
+
+ assertEquals("RecNum for offset=0 should be 0", 0, reader
+ .getRecordNumNear(0));
+ for (long x : new long[] { fileLen, fileLen + 1, 2 * fileLen }) {
+ assertEquals("RecNum for offset>=fileLen should be total entries",
+ totalRecs, reader.getRecordNumNear(x));
+ }
+
+ for (long i = 0; i < 100; ++i) {
+ assertEquals("Locaton to RecNum conversion not symmetric", i, reader
+ .getRecordNumByLocation(reader.getLocationByRecordNum(i)));
+ }
+
+ for (long i = 1; i < 100; ++i) {
+ long x = totalRecs - i;
+ assertEquals("Locaton to RecNum conversion not symmetric", x, reader
+ .getRecordNumByLocation(reader.getLocationByRecordNum(x)));
+ }
+
+ for (long i = begin; i < end; ++i) {
+ assertEquals("Locaton to RecNum conversion not symmetric", i, reader
+ .getRecordNumByLocation(reader.getLocationByRecordNum(i)));
+ }
+
+ for (int i = 0; i < 1000; ++i) {
+ long x = random.nextLong() % totalRecs;
+ if (x < 0) x += totalRecs;
+ assertEquals("Locaton to RecNum conversion not symmetric", x, reader
+ .getRecordNumByLocation(reader.getLocationByRecordNum(x)));
+ }
+ }
+
+ public void testSplit() throws IOException {
+ System.out.println("testSplit");
+ createFile(100000, Compression.Algorithm.NONE.getName());
+ checkRecNums();
+ readFile();
+ readRowSplits(10);
+ fs.delete(path, true);
+ createFile(500000, Compression.Algorithm.GZ.getName());
+ checkRecNums();
+ readFile();
+ readRowSplits(83);
+ fs.delete(path, true);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-apex-malhar/blob/02f48e1b/library/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileStreams.java
----------------------------------------------------------------------
diff --git a/library/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileStreams.java b/library/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileStreams.java
new file mode 100644
index 0000000..2e0506c
--- /dev/null
+++ b/library/src/test/java/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.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.DTFile.Reader;
+import org.apache.hadoop.io.file.tfile.DTFile.Writer;
+import org.apache.hadoop.io.file.tfile.DTFile.Reader.Scanner;
+import org.junit.Assert;
+
+/**
+ *
+ * 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 final String outputFile = getClass().getSimpleName();
+
+ public void init(String compression, String comparator) {
+ this.compression = compression;
+ this.comparator = comparator;
+ }
+
+ @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();
+ TestDTFileByteArrays.readRecords(fs, path, 0, conf);
+ }
+
+ public void testOneEntryKnownLength() throws IOException {
+ if (skip)
+ return;
+ writeRecords(1, true, true);
+
+ TestDTFileByteArrays.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
+ TestDTFileByteArrays.readRecords(fs, path, 1, conf);
+ }
+
+ // known key length, unknown value length
+ public void testOneEntryMixedLengths1() throws IOException {
+ if (skip)
+ return;
+ writeRecords(1, true, false);
+
+ TestDTFileByteArrays.readRecords(fs, path, 1, conf);
+ }
+
+ // unknown key length, known value length
+ public void testOneEntryMixedLengths2() throws IOException {
+ if (skip)
+ return;
+ writeRecords(1, false, true);
+
+ TestDTFileByteArrays.readRecords(fs, path, 1, conf);
+ }
+
+ public void testTwoEntriesKnownLength() throws IOException {
+ if (skip)
+ return;
+ writeRecords(2, true, true);
+
+ TestDTFileByteArrays.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 = TestDTFileByteArrays.composeSortedKey("key", 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;
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-apex-malhar/blob/02f48e1b/library/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileUnsortedByteArrays.java
----------------------------------------------------------------------
diff --git a/library/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileUnsortedByteArrays.java b/library/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileUnsortedByteArrays.java
new file mode 100644
index 0000000..a58f649
--- /dev/null
+++ b/library/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileUnsortedByteArrays.java
@@ -0,0 +1,239 @@
+/**
+ * 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.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.DTFile.Reader;
+import org.apache.hadoop.io.file.tfile.DTFile.Writer;
+import org.apache.hadoop.io.file.tfile.DTFile.Reader.Scanner;
+import org.junit.Assert;
+
+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.createScannerByKey("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;
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-apex-malhar/blob/02f48e1b/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 9419738..a5725d2 100644
--- a/pom.xml
+++ b/pom.xml
@@ -126,6 +126,7 @@
<ignoreMissingClasses>true</ignoreMissingClasses>
<excludes>
<exclude>*POJO*</exclude>
+ <exclude>org.apache.hadoop.io.file.tfile.*</exclude>
</excludes>
</parameter>
<skip>${semver.plugin.skip}</skip>
[2/8] incubator-apex-malhar git commit: MLHR-1877 #resolve #comment
moved DTFile implementation to from contrib to lib
Posted by hs...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-apex-malhar/blob/02f48e1b/library/src/test/java/org/apache/hadoop/io/file/tfile/DTFileTest.java
----------------------------------------------------------------------
diff --git a/library/src/test/java/org/apache/hadoop/io/file/tfile/DTFileTest.java b/library/src/test/java/org/apache/hadoop/io/file/tfile/DTFileTest.java
new file mode 100644
index 0000000..003d9eb
--- /dev/null
+++ b/library/src/test/java/org/apache/hadoop/io/file/tfile/DTFileTest.java
@@ -0,0 +1,220 @@
+/**
+ * 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.junit.Assert;
+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.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Random;
+
+public class DTFileTest
+{
+ private Configuration conf;
+ private Path path;
+ private FileSystem fs;
+ private NanoTimer timer;
+ private Random rng;
+ private RandomDistribution.DiscreteRNG keyLenGen;
+ private KVGenerator kvGen;
+
+
+ static class TestConf {
+ public int minWordLen = 5;
+ public int maxWordLen = 20;
+ public int dictSize = 1000;
+ int minKeyLen = 10;
+ int maxKeyLen = 50;
+ int minValLength = 100;
+ int maxValLength = 200;
+ int minBlockSize = 64 * 1024;
+ int fsOutputBufferSize = 1;
+ int fsInputBufferSize = 256 * 1024;
+ long fileSize = 3 * 1024 * 1024;
+ long seekCount = 1000;
+ String compress = "gz";
+
+ }
+
+ TestConf tconf = new TestConf();
+
+ public void setUp() throws IOException
+ {
+ conf = new Configuration();
+
+ conf.setInt("tfile.fs.input.buffer.size", tconf.fsInputBufferSize);
+ conf.setInt("tfile.fs.output.buffer.size", tconf.fsOutputBufferSize);
+ path = new Path("tmp/dtfile");
+ fs = path.getFileSystem(conf);
+ timer = new NanoTimer(false);
+ rng = new Random();
+ keyLenGen =
+ new RandomDistribution.Zipf(new Random(rng.nextLong()),
+ tconf.minKeyLen, tconf.maxKeyLen, 1.2);
+ RandomDistribution.DiscreteRNG valLenGen =
+ new RandomDistribution.Flat(new Random(rng.nextLong()),
+ tconf.minValLength, tconf.maxValLength);
+ RandomDistribution.DiscreteRNG wordLenGen =
+ new RandomDistribution.Flat(new Random(rng.nextLong()),
+ tconf.minWordLen, tconf.maxWordLen);
+ kvGen =
+ new KVGenerator(rng, true, keyLenGen, valLenGen, wordLenGen,
+ tconf.dictSize);
+ }
+
+
+ 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;
+ }
+
+ int tuples = 0;
+
+ private void writeTFile() throws IOException
+ {
+
+ FSDataOutputStream fout = createFSOutput(path, fs);
+ byte[] key = new byte[16];
+ ByteBuffer bb = ByteBuffer.wrap(key);
+ try {
+ DTFile.Writer writer =
+ new DTFile.Writer(fout, tconf.minBlockSize, tconf.compress, "memcmp",
+ conf);
+ try {
+ BytesWritable tmpKey = new BytesWritable();
+ BytesWritable val = new BytesWritable();
+ for (long i = 0; true; ++i) {
+ if (i % 1000 == 0) { // test the size for every 1000 rows.
+ if (fs.getFileStatus(path).getLen() >= tconf.fileSize) {
+ break;
+ }
+ }
+ bb.clear();
+ bb.putLong(i);
+ kvGen.next(tmpKey, val, false);
+ writer.append(key, 0, key.length, val.get(), 0, val
+ .getSize());
+ tuples++;
+ }
+ }
+ finally {
+ writer.close();
+ }
+ }
+ finally {
+ fout.close();
+ }
+
+ long fsize = fs.getFileStatus(path).getLen();
+
+ System.out.println("Total tuple wrote " + tuples + " File size " + fsize / (1024.0 * 1024));
+ }
+
+
+
+ @Test
+ public void seekDTFile() throws IOException
+ {
+ Random random = new Random();
+ int ikey = random.nextInt(tuples);
+ byte[] key = new byte[16];
+ ByteBuffer bb = ByteBuffer.wrap(key);
+ bb.putLong(ikey);
+
+ FSDataInputStream fsdis = fs.open(path);
+
+ if (CacheManager.getCache() != null) {
+ CacheManager.getCache().invalidateAll();
+ }
+ CacheManager.setEnableStats(true);
+ Assert.assertEquals("Cache Contains no block", CacheManager.getCacheSize(), 0);
+
+ DTFile.Reader reader = new DTFile.Reader(fsdis, fs.getFileStatus(path).getLen(), conf);
+ DTFile.Reader.Scanner scanner = reader.createScanner();
+
+ /* Read first key in the file */
+ long numBlocks = CacheManager.getCacheSize();
+ scanner.lowerBound(key);
+ Assert.assertEquals("Cache contains some blocks ", numBlocks + 1, CacheManager.getCacheSize());
+
+ /* Next key does not add a new block in cache, it reads directly from cache */
+ // close scanner, so that it does not use its own cache.
+ scanner.close();
+ ikey++;
+ bb.clear();
+ bb.putLong(ikey);
+
+ numBlocks = CacheManager.getCacheSize();
+ long hit = CacheManager.getCache().stats().hitCount();
+ scanner.lowerBound(key);
+ Assert.assertEquals("Cache contains some blocks ", CacheManager.getCacheSize(), numBlocks);
+ Assert.assertEquals("Cache hit ", CacheManager.getCache().stats().hitCount(), hit+1);
+
+ /* test cache miss */
+ scanner.close();
+ hit = CacheManager.getCache().stats().hitCount();
+ long oldmiss = CacheManager.getCache().stats().missCount();
+ ikey = tuples-1;
+ bb.clear();
+ bb.putLong(ikey);
+ numBlocks = CacheManager.getCacheSize();
+ scanner.lowerBound(key);
+ Assert.assertEquals("Cache contains one more blocks ", CacheManager.getCacheSize(), numBlocks + 1);
+ Assert.assertEquals("No cache hit ", CacheManager.getCache().stats().hitCount(), hit);
+ Assert.assertEquals("Cache miss", CacheManager.getCache().stats().missCount(), oldmiss + 1);
+
+ Assert.assertEquals("Reverse lookup cache and block cache has same number of entries",
+ reader.readerBCF.getCacheKeys().size(), CacheManager.getCacheSize());
+ reader.close();
+ Assert.assertEquals("Cache blocks are deleted on reader close ", CacheManager.getCacheSize(), 0);
+ Assert.assertEquals("Size of reverse lookup cache is zero ", 0, reader.readerBCF.getCacheKeys().size());
+ }
+
+ @Test
+ public void checkInvalidKeys()
+ {
+ /* invalidating non existing key do not throw exception */
+ List<String> lst = new LinkedList<String>();
+ lst.add("One");
+ lst.add("Two");
+ CacheManager.getCache().invalidateAll(lst);
+ }
+
+ @Before
+ public void createDTfile() throws IOException
+ {
+ setUp();
+ writeTFile();
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-apex-malhar/blob/02f48e1b/library/src/test/java/org/apache/hadoop/io/file/tfile/TestDTFile.java
----------------------------------------------------------------------
diff --git a/library/src/test/java/org/apache/hadoop/io/file/tfile/TestDTFile.java b/library/src/test/java/org/apache/hadoop/io/file/tfile/TestDTFile.java
new file mode 100644
index 0000000..49fedeb
--- /dev/null
+++ b/library/src/test/java/org/apache/hadoop/io/file/tfile/TestDTFile.java
@@ -0,0 +1,432 @@
+/**
+ * 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.DTFile.Reader;
+import org.apache.hadoop.io.file.tfile.DTFile.Writer;
+import org.apache.hadoop.io.file.tfile.DTFile.Reader.Scanner;
+
+/**
+ * test tfile features.
+ *
+ */
+public class TestDTFile extends TestCase {
+ private static String ROOT =
+ System.getProperty("test.build.data", "/tmp/tfile-test");
+ private FileSystem fs;
+ private Configuration conf;
+ private static final int minBlockSize = 512;
+ private static final int largeVal = 3 * 1024 * 1024;
+ private static final 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.createScannerByKey(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();
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-apex-malhar/blob/02f48e1b/library/src/test/java/org/apache/hadoop/io/file/tfile/TestDTFileByteArrays.java
----------------------------------------------------------------------
diff --git a/library/src/test/java/org/apache/hadoop/io/file/tfile/TestDTFileByteArrays.java b/library/src/test/java/org/apache/hadoop/io/file/tfile/TestDTFileByteArrays.java
new file mode 100644
index 0000000..a1fa5c8
--- /dev/null
+++ b/library/src/test/java/org/apache/hadoop/io/file/tfile/TestDTFileByteArrays.java
@@ -0,0 +1,773 @@
+/**
+ * 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 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.compress.zlib.ZlibFactory;
+import org.apache.hadoop.io.file.tfile.DTFile.Reader;
+import org.apache.hadoop.io.file.tfile.DTFile.Writer;
+import org.apache.hadoop.io.file.tfile.DTFile.Reader.Location;
+import org.apache.hadoop.io.file.tfile.DTFile.Reader.Scanner;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ *
+ * Byte arrays test case class using GZ compression codec, base class of none
+ * and LZO compression classes.
+ *
+ */
+public class TestDTFileByteArrays {
+ 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 = new Configuration();
+ private Path path;
+ private FSDataOutputStream out;
+ private Writer writer;
+
+ private String compression = Compression.Algorithm.GZ.getName();
+ private String comparator = "memcmp";
+ private final String outputFile = getClass().getSimpleName();
+
+ /*
+ * pre-sampled numbers of records in one block, based on the given the
+ * generated key and value strings. This is slightly different based on
+ * whether or not the native libs are present.
+ */
+ private boolean usingNative = ZlibFactory.isNativeZlibLoaded(conf);
+ private int records1stBlock = usingNative ? 5674 : 4480;
+ private int records2ndBlock = usingNative ? 5574 : 4263;
+
+ public void init(String compression, String comparator,
+ int numRecords1stBlock, int numRecords2ndBlock) {
+ init(compression, comparator);
+ this.records1stBlock = numRecords1stBlock;
+ this.records2ndBlock = numRecords2ndBlock;
+ }
+
+ public void init(String compression, String comparator) {
+ this.compression = compression;
+ this.comparator = comparator;
+ }
+
+ @Before
+ public void setUp() throws IOException {
+ path = new Path(ROOT, outputFile);
+ fs = path.getFileSystem(conf);
+ out = fs.create(path);
+ writer = new Writer(out, BLOCK_SIZE, compression, comparator, conf);
+ }
+
+ @After
+ public void tearDown() throws IOException {
+ if (!skip)
+ fs.delete(path, true);
+ }
+
+ @Test
+ 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();
+ }
+
+ @Test
+ public void testOneDataEntry() throws IOException {
+ if (skip)
+ return;
+ writeRecords(1);
+ readRecords(1);
+
+ checkBlockIndex(0, 0);
+ readValueBeforeKey(0);
+ readKeyWithoutValue(0);
+ readValueWithoutKey(0);
+ readKeyManyTimes(0);
+ }
+
+ @Test
+ public void testTwoDataEntries() throws IOException {
+ if (skip)
+ return;
+ writeRecords(2);
+ readRecords(2);
+ }
+
+ /**
+ * Fill up exactly one block.
+ *
+ * @throws IOException
+ */
+ @Test
+ 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 - 1, 0);
+ }
+
+ /**
+ * One block plus one record.
+ *
+ * @throws IOException
+ */
+ @Test
+ public void testOneBlockPlusOneEntry() throws IOException {
+ if (skip)
+ return;
+ writeRecords(records1stBlock + 1);
+ readRecords(records1stBlock + 1);
+ checkBlockIndex(records1stBlock - 1, 0);
+ checkBlockIndex(records1stBlock, 1);
+ }
+
+ @Test
+ public void testTwoBlocks() throws IOException {
+ if (skip)
+ return;
+ writeRecords(records1stBlock + 5);
+ readRecords(records1stBlock + 5);
+ checkBlockIndex(records1stBlock + 4, 1);
+ }
+
+ @Test
+ public void testThreeBlocks() throws IOException {
+ if (skip)
+ return;
+ writeRecords(2 * records1stBlock + 5);
+ readRecords(2 * records1stBlock + 5);
+
+ checkBlockIndex(2 * records1stBlock + 4, 2);
+ // 1st key in file
+ readValueBeforeKey(0);
+ readKeyWithoutValue(0);
+ readValueWithoutKey(0);
+ readKeyManyTimes(0);
+ // last key in file
+ readValueBeforeKey(2 * records1stBlock + 4);
+ readKeyWithoutValue(2 * records1stBlock + 4);
+ readValueWithoutKey(2 * records1stBlock + 4);
+ readKeyManyTimes(2 * records1stBlock + 4);
+
+ // 1st key in mid block, verify block indexes then read
+ checkBlockIndex(records1stBlock - 1, 0);
+ checkBlockIndex(records1stBlock, 1);
+ readValueBeforeKey(records1stBlock);
+ readKeyWithoutValue(records1stBlock);
+ readValueWithoutKey(records1stBlock);
+ readKeyManyTimes(records1stBlock);
+
+ // last key in mid block, verify block indexes then read
+ checkBlockIndex(records1stBlock + records2ndBlock
+ - 1, 1);
+ checkBlockIndex(records1stBlock + records2ndBlock, 2);
+ readValueBeforeKey(records1stBlock
+ + records2ndBlock - 1);
+ readKeyWithoutValue(records1stBlock
+ + records2ndBlock - 1);
+ readValueWithoutKey(records1stBlock
+ + records2ndBlock - 1);
+ readKeyManyTimes(records1stBlock + records2ndBlock
+ - 1);
+
+ // mid in mid block
+ readValueBeforeKey(records1stBlock + 10);
+ readKeyWithoutValue(records1stBlock + 10);
+ readValueWithoutKey(records1stBlock + 10);
+ readKeyManyTimes(records1stBlock + 10);
+ }
+
+ Location locate(Scanner scanner, byte[] key) throws IOException {
+ if (scanner.seekTo(key) == true) {
+ return scanner.currentLocation;
+ }
+ return scanner.endLocation;
+ }
+
+ @Test
+ 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();
+ locate(scanner, composeSortedKey(KEY, 2).getBytes());
+ locate(scanner, composeSortedKey(KEY, records1stBlock - 1).getBytes());
+ locate(scanner, composeSortedKey(KEY, records1stBlock).getBytes());
+ Location locX = locate(scanner, "keyX".getBytes());
+ Assert.assertEquals(scanner.endLocation, locX);
+ scanner.close();
+ reader.close();
+ }
+
+ @Test
+ 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();
+ }
+ }
+ }
+
+ @Test
+ 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 {
+ writer.prepareMetaBlock("testX", Compression.Algorithm.GZ.getName());
+ Assert.fail("Cannot create metablocks with the same name.");
+ } catch (Exception e) {
+ // noop, expecting exceptions
+ }
+ closeOutput();
+ }
+
+ @Test
+ 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.fail("Error on handling non-existent metablocks.");
+ } catch (Exception e) {
+ // noop, expecting exceptions
+ }
+ reader.close();
+ }
+
+ @Test
+ 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();
+ }
+
+ @Test
+ 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();
+ }
+
+ @Test
+ 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();
+ }
+ }
+
+ @Test
+ 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 {
+ new Reader(fs.open(path), fs.getFileStatus(path).getLen(), conf);
+ Assert.fail("Error on handling empty files.");
+ } catch (EOFException e) {
+ // noop, expecting exceptions
+ }
+ }
+
+ @Test
+ 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 {
+ new Reader(fs.open(path), fs.getFileStatus(path).getLen(), conf);
+ Assert.fail("Error on handling random files.");
+ } catch (IOException e) {
+ // noop, expecting exceptions
+ }
+ }
+
+ @Test
+ 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();
+ }
+
+ @Test
+ 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();
+ }
+
+ @Test
+ 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();
+ }
+
+ @Test
+ 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();
+ }
+
+ @Test
+ 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();
+ }
+
+ @Test
+ 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();
+ }
+
+ @Test
+ 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();
+ }
+ }
+
+ @Test
+ 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();
+ }
+
+ @Test
+ 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, 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 prefix
+ * @param value value
+ * @return sorted key
+ */
+ static String composeSortedKey(String prefix, int value) {
+ return String.format("%s%010d", prefix, value);
+ }
+
+ 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,
+ 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 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, recordIndex).getBytes());
+ Assert.assertEquals(blockIndexExpected, scanner.currentLocation
+ .getBlockIndex());
+ scanner.close();
+ reader.close();
+ }
+
+ private void readValueBeforeKey(int recordIndex)
+ throws IOException {
+ Reader reader =
+ new Reader(fs.open(path), fs.getFileStatus(path).getLen(), conf);
+ Scanner scanner =
+ reader.createScannerByKey(composeSortedKey(KEY, 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,
+ recordIndex));
+ } finally {
+ scanner.close();
+ reader.close();
+ }
+ }
+
+ private void readKeyWithoutValue(int recordIndex)
+ throws IOException {
+ Reader reader = new Reader(fs.open(path), fs.getFileStatus(path).getLen(), conf);
+ Scanner scanner =
+ reader.createScannerByKey(composeSortedKey(KEY, 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,
+ 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,
+ recordIndex + 1));
+ }
+ } finally {
+ scanner.close();
+ reader.close();
+ }
+ }
+
+ private void readValueWithoutKey(int recordIndex)
+ throws IOException {
+ Reader reader = new Reader(fs.open(path), fs.getFileStatus(path).getLen(), conf);
+
+ Scanner scanner =
+ reader.createScannerByKey(composeSortedKey(KEY, 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 recordIndex) throws IOException {
+ Reader reader = new Reader(fs.open(path), fs.getFileStatus(path).getLen(), conf);
+
+ Scanner scanner =
+ reader.createScannerByKey(composeSortedKey(KEY, 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,
+ recordIndex));
+
+ klen1 = scanner.entry().getKeyLength();
+ scanner.entry().getKey(kbuf1);
+ Assert.assertEquals(new String(kbuf1, 0, klen1), composeSortedKey(KEY,
+ recordIndex));
+
+ klen1 = scanner.entry().getKeyLength();
+ scanner.entry().getKey(kbuf1);
+ Assert.assertEquals(new String(kbuf1, 0, klen1), composeSortedKey(KEY,
+ recordIndex));
+
+ scanner.close();
+ reader.close();
+ }
+
+ private void closeOutput() throws IOException {
+ if (writer != null) {
+ writer.close();
+ writer = null;
+ }
+ if (out != null) {
+ out.close();
+ out = null;
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-apex-malhar/blob/02f48e1b/library/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileComparator2.java
----------------------------------------------------------------------
diff --git a/library/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileComparator2.java b/library/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileComparator2.java
new file mode 100644
index 0000000..c313813
--- /dev/null
+++ b/library/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileComparator2.java
@@ -0,0 +1,108 @@
+/**
+ * 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.IOException;
+
+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.LongWritable;
+import org.apache.hadoop.io.file.tfile.DTFile.Writer;
+import org.junit.Test;
+
+import static org.junit.Assert.*;
+
+public class TestTFileComparator2 {
+ private static final String ROOT = System.getProperty("test.build.data",
+ "/tmp/tfile-test");
+ private static final String name = "test-tfile-comparator2";
+ private final static int BLOCK_SIZE = 512;
+ private static final String VALUE = "value";
+ private static final String jClassLongWritableComparator = "jclass:"
+ + LongWritable.Comparator.class.getName();
+ private static final long NENTRY = 10000;
+
+ private static long cube(long n) {
+ return n*n*n;
+ }
+
+ private static String buildValue(long i) {
+ return String.format("%s-%d", VALUE, i);
+ }
+
+ @Test
+ public void testSortedLongWritable() throws IOException {
+ Configuration conf = new Configuration();
+ Path path = new Path(ROOT, name);
+ FileSystem fs = path.getFileSystem(conf);
+ FSDataOutputStream out = fs.create(path);
+ try {
+ DTFile.Writer writer = new Writer(out, BLOCK_SIZE, "gz",
+ jClassLongWritableComparator, conf);
+ try {
+ LongWritable key = new LongWritable(0);
+ for (long i=0; i<NENTRY; ++i) {
+ key.set(cube(i-NENTRY/2));
+ DataOutputStream dos = writer.prepareAppendKey(-1);
+ try {
+ key.write(dos);
+ } finally {
+ dos.close();
+ }
+ dos = writer.prepareAppendValue(-1);
+ try {
+ dos.write(buildValue(i).getBytes());
+ } finally {
+ dos.close();
+ }
+ }
+ } finally {
+ writer.close();
+ }
+ } finally {
+ out.close();
+ }
+
+ FSDataInputStream in = fs.open(path);
+ try {
+ DTFile.Reader reader = new DTFile.Reader(in, fs.getFileStatus(path)
+ .getLen(), conf);
+ try {
+ DTFile.Reader.Scanner scanner = reader.createScanner();
+ long i=0;
+ BytesWritable value = new BytesWritable();
+ for (; !scanner.atEnd(); scanner.advance()) {
+ scanner.entry().getValue(value);
+ assertEquals(buildValue(i), new String(value.getBytes(), 0, value
+ .getLength()));
+ ++i;
+ }
+ } finally {
+ reader.close();
+ }
+ } finally {
+ in.close();
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-apex-malhar/blob/02f48e1b/library/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileComparators.java
----------------------------------------------------------------------
diff --git a/library/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileComparators.java b/library/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileComparators.java
new file mode 100644
index 0000000..0a10468
--- /dev/null
+++ b/library/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileComparators.java
@@ -0,0 +1,123 @@
+/**
+ * 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.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.DTFile.Writer;
+import org.junit.Assert;
+
+/**
+ *
+ * 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;
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-apex-malhar/blob/02f48e1b/library/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileJClassComparatorByteArrays.java
----------------------------------------------------------------------
diff --git a/library/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileJClassComparatorByteArrays.java b/library/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileJClassComparatorByteArrays.java
new file mode 100644
index 0000000..301cffc
--- /dev/null
+++ b/library/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileJClassComparatorByteArrays.java
@@ -0,0 +1,59 @@
+/**
+ * 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.Serializable;
+
+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 TestDTFileByteArrays {
+ /**
+ * 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");
+ super.setUp();
+ }
+}
+
+class MyComparator implements RawComparator<byte[]>, Serializable {
+
+ @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);
+ }
+
+}
+
http://git-wip-us.apache.org/repos/asf/incubator-apex-malhar/blob/02f48e1b/library/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileLzoCodecsByteArrays.java
----------------------------------------------------------------------
diff --git a/library/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileLzoCodecsByteArrays.java b/library/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileLzoCodecsByteArrays.java
new file mode 100644
index 0000000..20cff9e
--- /dev/null
+++ b/library/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileLzoCodecsByteArrays.java
@@ -0,0 +1,41 @@
+/**
+ * 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 TestDTFileByteArrays {
+ /**
+ * 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", 2605, 2558);
+ if (!skip)
+ super.setUp();
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-apex-malhar/blob/02f48e1b/library/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileLzoCodecsStreams.java
----------------------------------------------------------------------
diff --git a/library/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileLzoCodecsStreams.java b/library/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileLzoCodecsStreams.java
new file mode 100644
index 0000000..7c6581d
--- /dev/null
+++ b/library/src/test/java/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");
+ if (!skip)
+ super.setUp();
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-apex-malhar/blob/02f48e1b/library/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileNoneCodecsByteArrays.java
----------------------------------------------------------------------
diff --git a/library/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileNoneCodecsByteArrays.java b/library/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileNoneCodecsByteArrays.java
new file mode 100644
index 0000000..c304743
--- /dev/null
+++ b/library/src/test/java/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 TestDTFileByteArrays {
+ /**
+ * 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", 24, 24);
+ super.setUp();
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-apex-malhar/blob/02f48e1b/library/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileNoneCodecsJClassComparatorByteArrays.java
----------------------------------------------------------------------
diff --git a/library/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileNoneCodecsJClassComparatorByteArrays.java b/library/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileNoneCodecsJClassComparatorByteArrays.java
new file mode 100644
index 0000000..31e3cad
--- /dev/null
+++ b/library/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileNoneCodecsJClassComparatorByteArrays.java
@@ -0,0 +1,40 @@
+/**
+ * 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;
+
+/**
+ *
+ * Byte arrays test case class using GZ compression codec, base class of none
+ * and LZO compression classes.
+ *
+ */
+
+public class TestTFileNoneCodecsJClassComparatorByteArrays extends TestDTFileByteArrays {
+ /**
+ * 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", 24, 24);
+ super.setUp();
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-apex-malhar/blob/02f48e1b/library/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileNoneCodecsStreams.java
----------------------------------------------------------------------
diff --git a/library/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileNoneCodecsStreams.java b/library/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileNoneCodecsStreams.java
new file mode 100644
index 0000000..06d086b
--- /dev/null
+++ b/library/src/test/java/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");
+ super.setUp();
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-apex-malhar/blob/02f48e1b/library/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileSeek.java
----------------------------------------------------------------------
diff --git a/library/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileSeek.java b/library/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileSeek.java
new file mode 100644
index 0000000..9f6b3ce
--- /dev/null
+++ b/library/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileSeek.java
@@ -0,0 +1,505 @@
+/**
+ * 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.DTFile.Reader;
+import org.apache.hadoop.io.file.tfile.DTFile.Writer;
+import org.apache.hadoop.io.file.tfile.DTFile.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();
+ }
+}
[6/8] incubator-apex-malhar git commit: MLHR-1877 #resolve #comment
moved DTFile implementation to from contrib to lib
Posted by hs...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-apex-malhar/blob/02f48e1b/contrib/src/test/java/org/apache/hadoop/io/file/tfile/DTFileTest.java
----------------------------------------------------------------------
diff --git a/contrib/src/test/java/org/apache/hadoop/io/file/tfile/DTFileTest.java b/contrib/src/test/java/org/apache/hadoop/io/file/tfile/DTFileTest.java
deleted file mode 100644
index 37bff4b..0000000
--- a/contrib/src/test/java/org/apache/hadoop/io/file/tfile/DTFileTest.java
+++ /dev/null
@@ -1,217 +0,0 @@
-/**
- * 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.junit.Assert;
-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.junit.Before;
-import org.junit.Test;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Random;
-
-public class DTFileTest
-{
- private Configuration conf;
- private Path path;
- private FileSystem fs;
- private NanoTimer timer;
- private Random rng;
- private RandomDistribution.DiscreteRNG keyLenGen;
- private KVGenerator kvGen;
-
-
- static class TestConf {
- public int minWordLen = 5;
- public int maxWordLen = 20;
- public int dictSize = 1000;
- int minKeyLen = 10;
- int maxKeyLen = 50;
- int minValLength = 100;
- int maxValLength = 200;
- int minBlockSize = 64 * 1024;
- int fsOutputBufferSize = 1;
- int fsInputBufferSize = 256 * 1024;
- long fileSize = 3 * 1024 * 1024;
- long seekCount = 1000;
- String compress = "gz";
-
- }
-
- TestConf tconf = new TestConf();
-
- public void setUp() throws IOException
- {
- conf = new Configuration();
-
- conf.setInt("tfile.fs.input.buffer.size", tconf.fsInputBufferSize);
- conf.setInt("tfile.fs.output.buffer.size", tconf.fsOutputBufferSize);
- path = new Path("tmp/dtfile");
- fs = path.getFileSystem(conf);
- timer = new NanoTimer(false);
- rng = new Random();
- keyLenGen =
- new RandomDistribution.Zipf(new Random(rng.nextLong()),
- tconf.minKeyLen, tconf.maxKeyLen, 1.2);
- RandomDistribution.DiscreteRNG valLenGen =
- new RandomDistribution.Flat(new Random(rng.nextLong()),
- tconf.minValLength, tconf.maxValLength);
- RandomDistribution.DiscreteRNG wordLenGen =
- new RandomDistribution.Flat(new Random(rng.nextLong()),
- tconf.minWordLen, tconf.maxWordLen);
- kvGen =
- new KVGenerator(rng, true, keyLenGen, valLenGen, wordLenGen,
- tconf.dictSize);
- }
-
-
- 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;
- }
-
- int tuples = 0;
-
- private void writeTFile() throws IOException
- {
-
- FSDataOutputStream fout = createFSOutput(path, fs);
- byte[] key = new byte[16];
- ByteBuffer bb = ByteBuffer.wrap(key);
- try {
- DTFile.Writer writer =
- new DTFile.Writer(fout, tconf.minBlockSize, tconf.compress, "memcmp",
- conf);
- try {
- BytesWritable tmpKey = new BytesWritable();
- BytesWritable val = new BytesWritable();
- for (long i = 0; true; ++i) {
- if (i % 1000 == 0) { // test the size for every 1000 rows.
- if (fs.getFileStatus(path).getLen() >= tconf.fileSize) {
- break;
- }
- }
- bb.clear();
- bb.putLong(i);
- kvGen.next(tmpKey, val, false);
- writer.append(key, 0, key.length, val.get(), 0, val
- .getSize());
- tuples++;
- }
- }
- finally {
- writer.close();
- }
- }
- finally {
- fout.close();
- }
-
- long fsize = fs.getFileStatus(path).getLen();
-
- System.out.println("Total tuple wrote " + tuples + " File size " + fsize / (1024.0 * 1024));
- }
-
-
-
- @Test
- public void seekDTFile() throws IOException
- {
- Random random = new Random();
- int ikey = random.nextInt(tuples);
- byte[] key = new byte[16];
- ByteBuffer bb = ByteBuffer.wrap(key);
- bb.putLong(ikey);
-
- FSDataInputStream fsdis = fs.open(path);
-
- CacheManager.setEnableStats(true);
- Assert.assertEquals("Cache Contains no block", CacheManager.getCacheSize(), 0);
-
- DTFile.Reader reader = new DTFile.Reader(fsdis, fs.getFileStatus(path).getLen(), conf);
- DTFile.Reader.Scanner scanner = reader.createScanner();
-
- /* Read first key in the file */
- long numBlocks = CacheManager.getCacheSize();
- scanner.lowerBound(key);
- Assert.assertEquals("Cache contains some blocks ", numBlocks + 1, CacheManager.getCacheSize());
-
- /* Next key does not add a new block in cache, it reads directly from cache */
- // close scanner, so that it does not use its own cache.
- scanner.close();
- ikey++;
- bb.clear();
- bb.putLong(ikey);
-
- numBlocks = CacheManager.getCacheSize();
- long hit = CacheManager.getCache().stats().hitCount();
- scanner.lowerBound(key);
- Assert.assertEquals("Cache contains some blocks ", CacheManager.getCacheSize(), numBlocks);
- Assert.assertEquals("Cache hit ", CacheManager.getCache().stats().hitCount(), hit+1);
-
- /* test cache miss */
- scanner.close();
- hit = CacheManager.getCache().stats().hitCount();
- long oldmiss = CacheManager.getCache().stats().missCount();
- ikey = tuples-1;
- bb.clear();
- bb.putLong(ikey);
- numBlocks = CacheManager.getCacheSize();
- scanner.lowerBound(key);
- Assert.assertEquals("Cache contains one more blocks ", CacheManager.getCacheSize(), numBlocks + 1);
- Assert.assertEquals("No cache hit ", CacheManager.getCache().stats().hitCount(), hit);
- Assert.assertEquals("Cache miss", CacheManager.getCache().stats().missCount(), oldmiss + 1);
-
- Assert.assertEquals("Reverse lookup cache and block cache has same number of entries",
- reader.readerBCF.getCacheKeys().size(), CacheManager.getCacheSize());
- reader.close();
- Assert.assertEquals("Cache blocks are deleted on reader close ", CacheManager.getCacheSize(), 0);
- Assert.assertEquals("Size of reverse lookup cache is zero ", 0, reader.readerBCF.getCacheKeys().size());
- }
-
- @Test
- public void checkInvalidKeys()
- {
- /* invalidating non existing key do not throw exception */
- List<String> lst = new LinkedList<String>();
- lst.add("One");
- lst.add("Two");
- CacheManager.getCache().invalidateAll(lst);
- }
-
- @Before
- public void createDTfile() throws IOException
- {
- setUp();
- writeTFile();
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-apex-malhar/blob/02f48e1b/contrib/src/test/java/org/apache/hadoop/io/file/tfile/TestDTFile.java
----------------------------------------------------------------------
diff --git a/contrib/src/test/java/org/apache/hadoop/io/file/tfile/TestDTFile.java b/contrib/src/test/java/org/apache/hadoop/io/file/tfile/TestDTFile.java
deleted file mode 100644
index 49fedeb..0000000
--- a/contrib/src/test/java/org/apache/hadoop/io/file/tfile/TestDTFile.java
+++ /dev/null
@@ -1,432 +0,0 @@
-/**
- * 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.DTFile.Reader;
-import org.apache.hadoop.io.file.tfile.DTFile.Writer;
-import org.apache.hadoop.io.file.tfile.DTFile.Reader.Scanner;
-
-/**
- * test tfile features.
- *
- */
-public class TestDTFile extends TestCase {
- private static String ROOT =
- System.getProperty("test.build.data", "/tmp/tfile-test");
- private FileSystem fs;
- private Configuration conf;
- private static final int minBlockSize = 512;
- private static final int largeVal = 3 * 1024 * 1024;
- private static final 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.createScannerByKey(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();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-apex-malhar/blob/02f48e1b/contrib/src/test/java/org/apache/hadoop/io/file/tfile/TestDTFileByteArrays.java
----------------------------------------------------------------------
diff --git a/contrib/src/test/java/org/apache/hadoop/io/file/tfile/TestDTFileByteArrays.java b/contrib/src/test/java/org/apache/hadoop/io/file/tfile/TestDTFileByteArrays.java
deleted file mode 100644
index a1fa5c8..0000000
--- a/contrib/src/test/java/org/apache/hadoop/io/file/tfile/TestDTFileByteArrays.java
+++ /dev/null
@@ -1,773 +0,0 @@
-/**
- * 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 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.compress.zlib.ZlibFactory;
-import org.apache.hadoop.io.file.tfile.DTFile.Reader;
-import org.apache.hadoop.io.file.tfile.DTFile.Writer;
-import org.apache.hadoop.io.file.tfile.DTFile.Reader.Location;
-import org.apache.hadoop.io.file.tfile.DTFile.Reader.Scanner;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-/**
- *
- * Byte arrays test case class using GZ compression codec, base class of none
- * and LZO compression classes.
- *
- */
-public class TestDTFileByteArrays {
- 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 = new Configuration();
- private Path path;
- private FSDataOutputStream out;
- private Writer writer;
-
- private String compression = Compression.Algorithm.GZ.getName();
- private String comparator = "memcmp";
- private final String outputFile = getClass().getSimpleName();
-
- /*
- * pre-sampled numbers of records in one block, based on the given the
- * generated key and value strings. This is slightly different based on
- * whether or not the native libs are present.
- */
- private boolean usingNative = ZlibFactory.isNativeZlibLoaded(conf);
- private int records1stBlock = usingNative ? 5674 : 4480;
- private int records2ndBlock = usingNative ? 5574 : 4263;
-
- public void init(String compression, String comparator,
- int numRecords1stBlock, int numRecords2ndBlock) {
- init(compression, comparator);
- this.records1stBlock = numRecords1stBlock;
- this.records2ndBlock = numRecords2ndBlock;
- }
-
- public void init(String compression, String comparator) {
- this.compression = compression;
- this.comparator = comparator;
- }
-
- @Before
- public void setUp() throws IOException {
- path = new Path(ROOT, outputFile);
- fs = path.getFileSystem(conf);
- out = fs.create(path);
- writer = new Writer(out, BLOCK_SIZE, compression, comparator, conf);
- }
-
- @After
- public void tearDown() throws IOException {
- if (!skip)
- fs.delete(path, true);
- }
-
- @Test
- 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();
- }
-
- @Test
- public void testOneDataEntry() throws IOException {
- if (skip)
- return;
- writeRecords(1);
- readRecords(1);
-
- checkBlockIndex(0, 0);
- readValueBeforeKey(0);
- readKeyWithoutValue(0);
- readValueWithoutKey(0);
- readKeyManyTimes(0);
- }
-
- @Test
- public void testTwoDataEntries() throws IOException {
- if (skip)
- return;
- writeRecords(2);
- readRecords(2);
- }
-
- /**
- * Fill up exactly one block.
- *
- * @throws IOException
- */
- @Test
- 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 - 1, 0);
- }
-
- /**
- * One block plus one record.
- *
- * @throws IOException
- */
- @Test
- public void testOneBlockPlusOneEntry() throws IOException {
- if (skip)
- return;
- writeRecords(records1stBlock + 1);
- readRecords(records1stBlock + 1);
- checkBlockIndex(records1stBlock - 1, 0);
- checkBlockIndex(records1stBlock, 1);
- }
-
- @Test
- public void testTwoBlocks() throws IOException {
- if (skip)
- return;
- writeRecords(records1stBlock + 5);
- readRecords(records1stBlock + 5);
- checkBlockIndex(records1stBlock + 4, 1);
- }
-
- @Test
- public void testThreeBlocks() throws IOException {
- if (skip)
- return;
- writeRecords(2 * records1stBlock + 5);
- readRecords(2 * records1stBlock + 5);
-
- checkBlockIndex(2 * records1stBlock + 4, 2);
- // 1st key in file
- readValueBeforeKey(0);
- readKeyWithoutValue(0);
- readValueWithoutKey(0);
- readKeyManyTimes(0);
- // last key in file
- readValueBeforeKey(2 * records1stBlock + 4);
- readKeyWithoutValue(2 * records1stBlock + 4);
- readValueWithoutKey(2 * records1stBlock + 4);
- readKeyManyTimes(2 * records1stBlock + 4);
-
- // 1st key in mid block, verify block indexes then read
- checkBlockIndex(records1stBlock - 1, 0);
- checkBlockIndex(records1stBlock, 1);
- readValueBeforeKey(records1stBlock);
- readKeyWithoutValue(records1stBlock);
- readValueWithoutKey(records1stBlock);
- readKeyManyTimes(records1stBlock);
-
- // last key in mid block, verify block indexes then read
- checkBlockIndex(records1stBlock + records2ndBlock
- - 1, 1);
- checkBlockIndex(records1stBlock + records2ndBlock, 2);
- readValueBeforeKey(records1stBlock
- + records2ndBlock - 1);
- readKeyWithoutValue(records1stBlock
- + records2ndBlock - 1);
- readValueWithoutKey(records1stBlock
- + records2ndBlock - 1);
- readKeyManyTimes(records1stBlock + records2ndBlock
- - 1);
-
- // mid in mid block
- readValueBeforeKey(records1stBlock + 10);
- readKeyWithoutValue(records1stBlock + 10);
- readValueWithoutKey(records1stBlock + 10);
- readKeyManyTimes(records1stBlock + 10);
- }
-
- Location locate(Scanner scanner, byte[] key) throws IOException {
- if (scanner.seekTo(key) == true) {
- return scanner.currentLocation;
- }
- return scanner.endLocation;
- }
-
- @Test
- 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();
- locate(scanner, composeSortedKey(KEY, 2).getBytes());
- locate(scanner, composeSortedKey(KEY, records1stBlock - 1).getBytes());
- locate(scanner, composeSortedKey(KEY, records1stBlock).getBytes());
- Location locX = locate(scanner, "keyX".getBytes());
- Assert.assertEquals(scanner.endLocation, locX);
- scanner.close();
- reader.close();
- }
-
- @Test
- 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();
- }
- }
- }
-
- @Test
- 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 {
- writer.prepareMetaBlock("testX", Compression.Algorithm.GZ.getName());
- Assert.fail("Cannot create metablocks with the same name.");
- } catch (Exception e) {
- // noop, expecting exceptions
- }
- closeOutput();
- }
-
- @Test
- 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.fail("Error on handling non-existent metablocks.");
- } catch (Exception e) {
- // noop, expecting exceptions
- }
- reader.close();
- }
-
- @Test
- 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();
- }
-
- @Test
- 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();
- }
-
- @Test
- 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();
- }
- }
-
- @Test
- 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 {
- new Reader(fs.open(path), fs.getFileStatus(path).getLen(), conf);
- Assert.fail("Error on handling empty files.");
- } catch (EOFException e) {
- // noop, expecting exceptions
- }
- }
-
- @Test
- 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 {
- new Reader(fs.open(path), fs.getFileStatus(path).getLen(), conf);
- Assert.fail("Error on handling random files.");
- } catch (IOException e) {
- // noop, expecting exceptions
- }
- }
-
- @Test
- 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();
- }
-
- @Test
- 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();
- }
-
- @Test
- 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();
- }
-
- @Test
- 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();
- }
-
- @Test
- 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();
- }
-
- @Test
- 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();
- }
-
- @Test
- 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();
- }
- }
-
- @Test
- 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();
- }
-
- @Test
- 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, 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 prefix
- * @param value value
- * @return sorted key
- */
- static String composeSortedKey(String prefix, int value) {
- return String.format("%s%010d", prefix, value);
- }
-
- 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,
- 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 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, recordIndex).getBytes());
- Assert.assertEquals(blockIndexExpected, scanner.currentLocation
- .getBlockIndex());
- scanner.close();
- reader.close();
- }
-
- private void readValueBeforeKey(int recordIndex)
- throws IOException {
- Reader reader =
- new Reader(fs.open(path), fs.getFileStatus(path).getLen(), conf);
- Scanner scanner =
- reader.createScannerByKey(composeSortedKey(KEY, 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,
- recordIndex));
- } finally {
- scanner.close();
- reader.close();
- }
- }
-
- private void readKeyWithoutValue(int recordIndex)
- throws IOException {
- Reader reader = new Reader(fs.open(path), fs.getFileStatus(path).getLen(), conf);
- Scanner scanner =
- reader.createScannerByKey(composeSortedKey(KEY, 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,
- 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,
- recordIndex + 1));
- }
- } finally {
- scanner.close();
- reader.close();
- }
- }
-
- private void readValueWithoutKey(int recordIndex)
- throws IOException {
- Reader reader = new Reader(fs.open(path), fs.getFileStatus(path).getLen(), conf);
-
- Scanner scanner =
- reader.createScannerByKey(composeSortedKey(KEY, 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 recordIndex) throws IOException {
- Reader reader = new Reader(fs.open(path), fs.getFileStatus(path).getLen(), conf);
-
- Scanner scanner =
- reader.createScannerByKey(composeSortedKey(KEY, 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,
- recordIndex));
-
- klen1 = scanner.entry().getKeyLength();
- scanner.entry().getKey(kbuf1);
- Assert.assertEquals(new String(kbuf1, 0, klen1), composeSortedKey(KEY,
- recordIndex));
-
- klen1 = scanner.entry().getKeyLength();
- scanner.entry().getKey(kbuf1);
- Assert.assertEquals(new String(kbuf1, 0, klen1), composeSortedKey(KEY,
- recordIndex));
-
- scanner.close();
- reader.close();
- }
-
- private void closeOutput() throws IOException {
- if (writer != null) {
- writer.close();
- writer = null;
- }
- if (out != null) {
- out.close();
- out = null;
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-apex-malhar/blob/02f48e1b/contrib/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileComparator2.java
----------------------------------------------------------------------
diff --git a/contrib/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileComparator2.java b/contrib/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileComparator2.java
deleted file mode 100644
index c313813..0000000
--- a/contrib/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileComparator2.java
+++ /dev/null
@@ -1,108 +0,0 @@
-/**
- * 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.IOException;
-
-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.LongWritable;
-import org.apache.hadoop.io.file.tfile.DTFile.Writer;
-import org.junit.Test;
-
-import static org.junit.Assert.*;
-
-public class TestTFileComparator2 {
- private static final String ROOT = System.getProperty("test.build.data",
- "/tmp/tfile-test");
- private static final String name = "test-tfile-comparator2";
- private final static int BLOCK_SIZE = 512;
- private static final String VALUE = "value";
- private static final String jClassLongWritableComparator = "jclass:"
- + LongWritable.Comparator.class.getName();
- private static final long NENTRY = 10000;
-
- private static long cube(long n) {
- return n*n*n;
- }
-
- private static String buildValue(long i) {
- return String.format("%s-%d", VALUE, i);
- }
-
- @Test
- public void testSortedLongWritable() throws IOException {
- Configuration conf = new Configuration();
- Path path = new Path(ROOT, name);
- FileSystem fs = path.getFileSystem(conf);
- FSDataOutputStream out = fs.create(path);
- try {
- DTFile.Writer writer = new Writer(out, BLOCK_SIZE, "gz",
- jClassLongWritableComparator, conf);
- try {
- LongWritable key = new LongWritable(0);
- for (long i=0; i<NENTRY; ++i) {
- key.set(cube(i-NENTRY/2));
- DataOutputStream dos = writer.prepareAppendKey(-1);
- try {
- key.write(dos);
- } finally {
- dos.close();
- }
- dos = writer.prepareAppendValue(-1);
- try {
- dos.write(buildValue(i).getBytes());
- } finally {
- dos.close();
- }
- }
- } finally {
- writer.close();
- }
- } finally {
- out.close();
- }
-
- FSDataInputStream in = fs.open(path);
- try {
- DTFile.Reader reader = new DTFile.Reader(in, fs.getFileStatus(path)
- .getLen(), conf);
- try {
- DTFile.Reader.Scanner scanner = reader.createScanner();
- long i=0;
- BytesWritable value = new BytesWritable();
- for (; !scanner.atEnd(); scanner.advance()) {
- scanner.entry().getValue(value);
- assertEquals(buildValue(i), new String(value.getBytes(), 0, value
- .getLength()));
- ++i;
- }
- } finally {
- reader.close();
- }
- } finally {
- in.close();
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-apex-malhar/blob/02f48e1b/contrib/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileComparators.java
----------------------------------------------------------------------
diff --git a/contrib/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileComparators.java b/contrib/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileComparators.java
deleted file mode 100644
index 0a10468..0000000
--- a/contrib/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileComparators.java
+++ /dev/null
@@ -1,123 +0,0 @@
-/**
- * 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.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.DTFile.Writer;
-import org.junit.Assert;
-
-/**
- *
- * 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;
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-apex-malhar/blob/02f48e1b/contrib/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileJClassComparatorByteArrays.java
----------------------------------------------------------------------
diff --git a/contrib/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileJClassComparatorByteArrays.java b/contrib/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileJClassComparatorByteArrays.java
deleted file mode 100644
index 301cffc..0000000
--- a/contrib/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileJClassComparatorByteArrays.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/**
- * 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.Serializable;
-
-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 TestDTFileByteArrays {
- /**
- * 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");
- super.setUp();
- }
-}
-
-class MyComparator implements RawComparator<byte[]>, Serializable {
-
- @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);
- }
-
-}
-
http://git-wip-us.apache.org/repos/asf/incubator-apex-malhar/blob/02f48e1b/contrib/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileLzoCodecsByteArrays.java
----------------------------------------------------------------------
diff --git a/contrib/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileLzoCodecsByteArrays.java b/contrib/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileLzoCodecsByteArrays.java
deleted file mode 100644
index 20cff9e..0000000
--- a/contrib/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileLzoCodecsByteArrays.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/**
- * 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 TestDTFileByteArrays {
- /**
- * 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", 2605, 2558);
- if (!skip)
- super.setUp();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-apex-malhar/blob/02f48e1b/contrib/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileLzoCodecsStreams.java
----------------------------------------------------------------------
diff --git a/contrib/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileLzoCodecsStreams.java b/contrib/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileLzoCodecsStreams.java
deleted file mode 100644
index 7c6581d..0000000
--- a/contrib/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileLzoCodecsStreams.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/**
- * 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");
- if (!skip)
- super.setUp();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-apex-malhar/blob/02f48e1b/contrib/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileNoneCodecsByteArrays.java
----------------------------------------------------------------------
diff --git a/contrib/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileNoneCodecsByteArrays.java b/contrib/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileNoneCodecsByteArrays.java
deleted file mode 100644
index c304743..0000000
--- a/contrib/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileNoneCodecsByteArrays.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/**
- * 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 TestDTFileByteArrays {
- /**
- * 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", 24, 24);
- super.setUp();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-apex-malhar/blob/02f48e1b/contrib/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileNoneCodecsJClassComparatorByteArrays.java
----------------------------------------------------------------------
diff --git a/contrib/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileNoneCodecsJClassComparatorByteArrays.java b/contrib/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileNoneCodecsJClassComparatorByteArrays.java
deleted file mode 100644
index 31e3cad..0000000
--- a/contrib/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileNoneCodecsJClassComparatorByteArrays.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/**
- * 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;
-
-/**
- *
- * Byte arrays test case class using GZ compression codec, base class of none
- * and LZO compression classes.
- *
- */
-
-public class TestTFileNoneCodecsJClassComparatorByteArrays extends TestDTFileByteArrays {
- /**
- * 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", 24, 24);
- super.setUp();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-apex-malhar/blob/02f48e1b/contrib/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileNoneCodecsStreams.java
----------------------------------------------------------------------
diff --git a/contrib/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileNoneCodecsStreams.java b/contrib/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileNoneCodecsStreams.java
deleted file mode 100644
index 06d086b..0000000
--- a/contrib/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileNoneCodecsStreams.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/**
- * 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");
- super.setUp();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-apex-malhar/blob/02f48e1b/contrib/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileSeek.java
----------------------------------------------------------------------
diff --git a/contrib/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileSeek.java b/contrib/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileSeek.java
deleted file mode 100644
index 9f6b3ce..0000000
--- a/contrib/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileSeek.java
+++ /dev/null
@@ -1,505 +0,0 @@
-/**
- * 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.DTFile.Reader;
-import org.apache.hadoop.io.file.tfile.DTFile.Writer;
-import org.apache.hadoop.io.file.tfile.DTFile.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();
- }
-}
[3/8] incubator-apex-malhar git commit: MLHR-1877 #resolve #comment
moved DTFile implementation to from contrib to lib
Posted by hs...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-apex-malhar/blob/02f48e1b/library/src/main/java/org/apache/hadoop/io/file/tfile/DTFile.java
----------------------------------------------------------------------
diff --git a/library/src/main/java/org/apache/hadoop/io/file/tfile/DTFile.java b/library/src/main/java/org/apache/hadoop/io/file/tfile/DTFile.java
new file mode 100644
index 0000000..f1c87ba
--- /dev/null
+++ b/library/src/main/java/org/apache/hadoop/io/file/tfile/DTFile.java
@@ -0,0 +1,2399 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hadoop.io.file.tfile;
+
+import java.io.ByteArrayInputStream;
+import java.io.Closeable;
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.DataOutputStream;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.Comparator;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.io.BoundedByteArrayOutputStream;
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.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.ByteArray;
+import org.apache.hadoop.io.file.tfile.MetaBlockAlreadyExists;
+import org.apache.hadoop.io.file.tfile.MetaBlockDoesNotExist;
+import org.apache.hadoop.io.file.tfile.RawComparable;
+import org.apache.hadoop.io.file.tfile.Utils;
+import org.apache.hadoop.io.file.tfile.DTBCFile.Reader.BlockReader;
+import org.apache.hadoop.io.file.tfile.DTBCFile.Writer.BlockAppender;
+import org.apache.hadoop.io.file.tfile.Utils.Version;
+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.serializer.JavaSerializationComparator;
+
+/**
+ * <ul>
+ * <li>The file format of DTFile is same as {@link TFile} with different reader implementation.
+ * It reads data block by block and cache the binary block data into memory to speed up the random read.
+ *
+ * <li>The public api of {@link Reader} is as same as it is in {@link TFile} {@link org.apache.hadoop.io.file.tfile.TFile.Reader} implementation.
+ * Besides, it provides getBlockBuffer(), getKeyOffset(), getKeyLength(), getValueOffset(), getValueLength() method
+ * to expose raw block, key, value data to user to avoid unnecessary internal/external data copy
+ *
+ * <li>In the performance test, It shows no difference in sequential reads and 20x faster in random reads(If most of them hit memory)
+ * </ul>
+ *
+ * A TFile is a container of key-value pairs. Both keys and values are type-less
+ * bytes. Keys are restricted to 64KB, value length is not restricted
+ * (practically limited to the available disk storage). TFile further provides
+ * the following features:
+ * <ul>
+ * <li>Block Compression.
+ * <li>Named meta data blocks.
+ * <li>Sorted or unsorted keys.
+ * <li>Seek by key or by file offset.
+ * </ul>
+ * The memory footprint of a TFile includes the following:
+ * <ul>
+ * <li>Some constant overhead of reading or writing a compressed block.
+ * <ul>
+ * <li>Each compressed block requires one compression/decompression codec for
+ * I/O.
+ * <li>Temporary space to buffer the key.
+ * <li>Temporary space to buffer the value (for TFile.Writer only). Values are
+ * chunk encoded, so that we buffer at most one chunk of user data. By default,
+ * the chunk buffer is 1MB. Reading chunked value does not require additional
+ * memory.
+ * </ul>
+ * <li>TFile index, which is proportional to the total number of Data Blocks.
+ * The total amount of memory needed to hold the index can be estimated as
+ * (56+AvgKeySize)*NumBlocks.
+ * <li>MetaBlock index, which is proportional to the total number of Meta
+ * Blocks.The total amount of memory needed to hold the index for Meta Blocks
+ * can be estimated as (40+AvgMetaBlockName)*NumMetaBlock.
+ * </ul>
+ * <p>
+ * The behavior of TFile can be customized by the following variables through
+ * Configuration:
+ * <ul>
+ * <li><b>tfile.io.chunk.size</b>: Value chunk size. Integer (in bytes). Default
+ * to 1MB. Values of the length less than the chunk size is guaranteed to have
+ * known value length in read time (See
+ * {@link DTFile.Reader.Scanner.Entry#isValueLengthKnown()}).
+ * <li><b>tfile.fs.output.buffer.size</b>: Buffer size used for
+ * FSDataOutputStream. Integer (in bytes). Default to 256KB.
+ * <li><b>tfile.fs.input.buffer.size</b>: Buffer size used for
+ * FSDataInputStream. Integer (in bytes). Default to 256KB.
+ * </ul>
+ * <p>
+ * Suggestions on performance optimization.
+ * <ul>
+ * <li>Minimum block size. We recommend a setting of minimum block size between
+ * 256KB to 1MB for general usage. Larger block size is preferred if files are
+ * primarily for sequential access. However, it would lead to inefficient random
+ * access (because there are more data to decompress). Smaller blocks are good
+ * for random access, but require more memory to hold the block index, and may
+ * be slower to create (because we must flush the compressor stream at the
+ * conclusion of each data block, which leads to an FS I/O flush). Further, due
+ * to the internal caching in Compression codec, the smallest possible block
+ * size would be around 20KB-30KB.
+ * <li>The current implementation does not offer true multi-threading for
+ * reading. The implementation uses FSDataInputStream seek()+read(), which is
+ * shown to be much faster than positioned-read call in single thread mode.
+ * However, it also means that if multiple threads attempt to access the same
+ * TFile (using multiple scanners) simultaneously, the actual I/O is carried out
+ * sequentially even if they access different DFS blocks.
+ * <li>Compression codec. Use "none" if the data is not very compressable (by
+ * compressable, I mean a compression ratio at least 2:1). Generally, use "lzo"
+ * as the starting point for experimenting. "gz" overs slightly better
+ * compression ratio over "lzo" but requires 4x CPU to compress and 2x CPU to
+ * decompress, comparing to "lzo".
+ * <li>File system buffering, if the underlying FSDataInputStream and
+ * FSDataOutputStream is already adequately buffered; or if applications
+ * reads/writes keys and values in large buffers, we can reduce the sizes of
+ * input/output buffering in TFile layer by setting the configuration parameters
+ * "tfile.fs.input.buffer.size" and "tfile.fs.output.buffer.size".
+ * </ul>
+ *
+ * Some design rationale behind TFile can be found at <a
+ * href=https://issues.apache.org/jira/browse/HADOOP-3315>Hadoop-3315</a>.
+ *
+ * @since 2.0.0
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class DTFile {
+ static final Log LOG = LogFactory.getLog(DTFile.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";
+
+ public static final int DEFAULT_INPUT_FS_BUF_SIZE = 256 * 1024;
+
+ 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) {
+ int buffserSize = conf.getInt(FS_INPUT_BUF_SIZE_ATTR, DEFAULT_INPUT_FS_BUF_SIZE);
+ if (buffserSize <= 0)
+ buffserSize = DEFAULT_INPUT_FS_BUF_SIZE;
+ return buffserSize;
+ }
+
+ static int getFSOutputBufferSize(Configuration conf) {
+ return conf.getInt(FS_OUTPUT_BUF_SIZE_ATTR, 256 * 1024);
+ }
+
+ private static final int MAX_KEY_SIZE = 64 * 1024; // 64KB
+ static final Version API_VERSION = new Version((short) 1, (short) 0);
+
+ /** compression: gzip */
+ public static final String COMPRESSION_GZ = "gz";
+ /** compression: lzo */
+ public static final String COMPRESSION_LZO = "lzo";
+ /** compression: none */
+ public static final String COMPRESSION_NONE = "none";
+ /** comparator: memcmp */
+ public static final String COMPARATOR_MEMCMP = "memcmp";
+ /** comparator prefix: java class */
+ public static final String COMPARATOR_JCLASS = "jclass:";
+
+ /**
+ * Make a raw comparator from a string name.
+ *
+ * @param name
+ * Comparator name
+ * @return A RawComparable comparator.
+ */
+ static public Comparator<RawComparable> makeComparator(String name) {
+ return TFileMeta.makeComparator(name);
+ }
+
+ // Prevent the instantiation of TFiles
+ private DTFile() {
+ // nothing
+ }
+
+ /**
+ * Get names of supported compression algorithms. The names are acceptable by
+ * TFile.Writer.
+ *
+ * @return Array of strings, each represents a supported compression
+ * algorithm. Currently, the following compression algorithms are
+ * supported.
+ * <ul>
+ * <li>"none" - No compression.
+ * <li>"lzo" - LZO compression.
+ * <li>"gz" - GZIP compression.
+ * </ul>
+ */
+ public static String[] getSupportedCompressionAlgorithms() {
+ return Compression.getSupportedAlgorithms();
+ }
+
+ /**
+ * TFile Writer.
+ */
+ @InterfaceStability.Evolving
+ public static class Writer implements Closeable {
+ // minimum compressed size for a block.
+ private final int sizeMinBlock;
+
+ // Meta blocks.
+ final TFileIndex tfileIndex;
+ final TFileMeta tfileMeta;
+
+ // reference to the underlying BCFile.
+ private DTBCFile.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 DTFile#getSupportedCompressionAlgorithms()}.
+ * @param comparator
+ * Leave comparator as null or empty string if TFile is not sorted.
+ * Otherwise, provide the string name for the comparison algorithm
+ * for keys. Two kinds of comparators are supported.
+ * <ul>
+ * <li>Algorithmic comparator: binary comparators that is language
+ * independent. Currently, only "memcmp" is supported.
+ * <li>Language-specific comparator: binary comparators that can
+ * only be constructed in specific language. For Java, the syntax
+ * is "jclass:", followed by the class name of the RawComparator.
+ * Currently, we only support RawComparators that can be
+ * constructed through the default constructor (with no
+ * parameters). Parameterized RawComparators such as
+ * {@link WritableComparator} or
+ * {@link JavaSerializationComparator} may not be directly used.
+ * One should write a wrapper class that inherits from such classes
+ * and use its default constructor to perform proper
+ * initialization.
+ * </ul>
+ * @param conf
+ * The configuration object.
+ * @throws IOException
+ */
+ public Writer(FSDataOutputStream fsdos, int minBlockSize,
+ String compressName, String comparator, Configuration conf)
+ throws IOException {
+ sizeMinBlock = minBlockSize;
+ tfileMeta = new TFileMeta(comparator);
+ tfileIndex = new TFileIndex(tfileMeta.getComparator());
+
+ writerBCF = new DTBCFile.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.
+ */
+ @Override
+ public void close() throws IOException {
+ if ((state == State.CLOSED)) {
+ return;
+ }
+ try {
+ // First try the normal finish.
+ // Terminate upon the first Exception.
+ if (errorCount == 0) {
+ if (state != State.READY) {
+ throw new IllegalStateException(
+ "Cannot close TFile in the middle of key-value insertion.");
+ }
+
+ finishDataBlock(true);
+
+ // first, write out data:TFile.meta
+ BlockAppender outMeta =
+ writerBCF
+ .prepareMetaBlock(TFileMeta.BLOCK_NAME, COMPRESSION_NONE);
+ try {
+ tfileMeta.write(outMeta);
+ } finally {
+ outMeta.close();
+ }
+
+ // second, write out data:TFile.index
+ BlockAppender outIndex =
+ writerBCF.prepareMetaBlock(TFileIndex.BLOCK_NAME);
+ try {
+ tfileIndex.write(outIndex);
+ } finally {
+ outIndex.close();
+ }
+
+ writerBCF.close();
+ }
+ } finally {
+ IOUtils.cleanup(LOG, blkAppender, writerBCF);
+ blkAppender = null;
+ writerBCF = null;
+ state = State.CLOSED;
+ }
+ }
+
+ /**
+ * Adding a new key-value pair to the TFile. This is synonymous to
+ * append(key, 0, key.length, value, 0, value.length)
+ *
+ * @param key
+ * Buffer for key.
+ * @param value
+ * Buffer for value.
+ * @throws IOException
+ */
+ public void append(byte[] key, byte[] value) throws IOException {
+ append(key, 0, key.length, value, 0, value.length);
+ }
+
+ /**
+ * Adding a new key-value pair to TFile.
+ *
+ * @param key
+ * buffer for key.
+ * @param koff
+ * offset in key buffer.
+ * @param klen
+ * length of key.
+ * @param value
+ * buffer for value.
+ * @param voff
+ * offset in value buffer.
+ * @param vlen
+ * length of value.
+ * @throws IOException
+ * Upon IO errors.
+ * <p>
+ * If an exception is thrown, the TFile will be in an inconsistent
+ * state. The only legitimate call after that would be close
+ */
+ public void append(byte[] key, int koff, int klen, byte[] value, int voff,
+ int vlen) throws IOException {
+ if ((koff | klen | (koff + klen) | (key.length - (koff + klen))) < 0) {
+ throw new IndexOutOfBoundsException(
+ "Bad key buffer offset-length combination.");
+ }
+
+ if ((voff | vlen | (voff + vlen) | (value.length - (voff + vlen))) < 0) {
+ throw new IndexOutOfBoundsException(
+ "Bad value buffer offset-length combination.");
+ }
+
+ try {
+ DataOutputStream dosKey = prepareAppendKey(klen);
+ try {
+ ++errorCount;
+ dosKey.write(key, koff, klen);
+ --errorCount;
+ } finally {
+ dosKey.close();
+ }
+
+ DataOutputStream dosValue = prepareAppendValue(vlen);
+ try {
+ ++errorCount;
+ dosValue.write(value, voff, vlen);
+ --errorCount;
+ } finally {
+ dosValue.close();
+ }
+ } finally {
+ state = State.READY;
+ }
+ }
+
+ /**
+ * Helper class to register key after close call on key append stream.
+ */
+ private class KeyRegister extends DataOutputStream {
+ private final int expectedLength;
+ private boolean closed = false;
+
+ public KeyRegister(int len) {
+ super(currentKeyBufferOS);
+ if (len >= 0) {
+ currentKeyBufferOS.reset(len);
+ } else {
+ currentKeyBufferOS.reset();
+ }
+ expectedLength = len;
+ }
+
+ @Override
+ public void close() throws IOException {
+ if (closed == true) {
+ return;
+ }
+
+ try {
+ ++errorCount;
+ byte[] key = currentKeyBufferOS.getBuffer();
+ int len = currentKeyBufferOS.size();
+ /**
+ * verify length.
+ */
+ if (expectedLength >= 0 && expectedLength != len) {
+ throw new IOException("Incorrect key length: expected="
+ + expectedLength + " actual=" + len);
+ }
+
+ Utils.writeVInt(blkAppender, len);
+ blkAppender.write(key, 0, len);
+ if (tfileIndex.getFirstKey() == null) {
+ tfileIndex.setFirstKey(key, 0, len);
+ }
+
+ if (tfileMeta.isSorted() && tfileMeta.getRecordCount()>0) {
+ byte[] lastKey = lastKeyBufferOS.getBuffer();
+ int lastLen = lastKeyBufferOS.size();
+ if (tfileMeta.getComparator().compare(key, 0, len, lastKey, 0,
+ lastLen) < 0) {
+ throw new IOException("Keys are not added in sorted order");
+ }
+ }
+
+ BoundedByteArrayOutputStream tmp = currentKeyBufferOS;
+ currentKeyBufferOS = lastKeyBufferOS;
+ lastKeyBufferOS = tmp;
+ --errorCount;
+ } finally {
+ closed = true;
+ state = State.END_KEY;
+ }
+ }
+ }
+
+ /**
+ * Helper class to register value after close call on value append stream.
+ */
+ private class ValueRegister extends DataOutputStream {
+ private boolean closed = false;
+
+ public ValueRegister(OutputStream os) {
+ super(os);
+ }
+
+ // Avoiding flushing call to down stream.
+ @Override
+ public void flush() {
+ // do nothing
+ }
+
+ @Override
+ public void close() throws IOException {
+ if (closed == true) {
+ return;
+ }
+
+ try {
+ ++errorCount;
+ super.close();
+ blkRecordCount++;
+ // bump up the total record count in the whole file
+ tfileMeta.incRecordCount();
+ finishDataBlock(false);
+ --errorCount;
+ } finally {
+ closed = true;
+ state = State.READY;
+ }
+ }
+ }
+
+ /**
+ * Obtain an output stream for writing a key into TFile. This may only be
+ * called when there is no active Key appending stream or value appending
+ * stream.
+ *
+ * @param length
+ * The expected length of the key. If length of the key is not
+ * known, set length = -1. Otherwise, the application must write
+ * exactly as many bytes as specified here before calling close on
+ * the returned output stream.
+ * @return The key appending output stream.
+ * @throws IOException
+ *
+ */
+ public DataOutputStream prepareAppendKey(int length) throws IOException {
+ if (state != State.READY) {
+ throw new IllegalStateException("Incorrect state to start a new key: "
+ + state.name());
+ }
+
+ initDataBlock();
+ DataOutputStream ret = new KeyRegister(length);
+ state = State.IN_KEY;
+ return ret;
+ }
+
+ /**
+ * Obtain an output stream for writing a value into TFile. This may only be
+ * called right after a key appending operation (the key append stream must
+ * be closed).
+ *
+ * @param length
+ * The expected length of the value. If length of the value is not
+ * known, set length = -1. Otherwise, the application must write
+ * exactly as many bytes as specified here before calling close on
+ * the returned output stream. Advertising the value size up-front
+ * guarantees that the value is encoded in one chunk, and avoids
+ * intermediate chunk buffering.
+ * @throws IOException
+ *
+ */
+ public DataOutputStream prepareAppendValue(int length) throws IOException {
+ if (state != State.END_KEY) {
+ throw new IllegalStateException(
+ "Incorrect state to start a new value: " + state.name());
+ }
+
+ DataOutputStream ret;
+
+ // unknown length
+ if (length < 0) {
+ if (valueBuffer == null) {
+ valueBuffer = new byte[getChunkBufferSize(conf)];
+ }
+ ret = new ValueRegister(new ChunkEncoder(blkAppender, valueBuffer));
+ } else {
+ ret =
+ new ValueRegister(new Chunk.SingleChunkEncoder(blkAppender, length));
+ }
+
+ state = State.IN_VALUE;
+ return ret;
+ }
+
+ /**
+ * Obtain an output stream for creating a meta block. This function may not
+ * be called when there is a key append stream or value append stream
+ * active. No more key-value insertion is allowed after a meta data block
+ * has been added to TFile.
+ *
+ * @param name
+ * Name of the meta block.
+ * @param compressName
+ * Name of the compression algorithm to be used. Must be one of the
+ * strings returned by
+ * {@link DTFile#getSupportedCompressionAlgorithms()}.
+ * @return A DataOutputStream that can be used to write Meta Block data.
+ * Closing the stream would signal the ending of the block.
+ * @throws IOException
+ * @throws MetaBlockAlreadyExists
+ * the Meta Block with the same name already exists.
+ */
+ public DataOutputStream prepareMetaBlock(String name, String compressName)
+ throws IOException, MetaBlockAlreadyExists {
+ if (state != State.READY) {
+ throw new IllegalStateException(
+ "Incorrect state to start a Meta Block: " + state.name());
+ }
+
+ finishDataBlock(true);
+ DataOutputStream outputStream =
+ writerBCF.prepareMetaBlock(name, compressName);
+ return outputStream;
+ }
+
+ /**
+ * Obtain an output stream for creating a meta block. This function may not
+ * be called when there is a key append stream or value append stream
+ * active. No more key-value insertion is allowed after a meta data block
+ * has been added to TFile. Data will be compressed using the default
+ * compressor as defined in Writer's constructor.
+ *
+ * @param name
+ * Name of the meta block.
+ * @return A DataOutputStream that can be used to write Meta Block data.
+ * Closing the stream would signal the ending of the block.
+ * @throws IOException
+ * @throws MetaBlockAlreadyExists
+ * the Meta Block with the same name already exists.
+ */
+ public DataOutputStream prepareMetaBlock(String name) throws IOException,
+ MetaBlockAlreadyExists {
+ if (state != State.READY) {
+ throw new IllegalStateException(
+ "Incorrect state to start a Meta Block: " + state.name());
+ }
+
+ finishDataBlock(true);
+ return writerBCF.prepareMetaBlock(name);
+ }
+
+ /**
+ * Check if we need to start a new data block.
+ *
+ * @throws IOException
+ */
+ private void initDataBlock() throws IOException {
+ // for each new block, get a new appender
+ if (blkAppender == null) {
+ blkAppender = writerBCF.prepareDataBlock();
+ }
+ }
+
+ /**
+ * Close the current data block if necessary.
+ *
+ * @param bForceFinish
+ * Force the closure regardless of the block size.
+ * @throws IOException
+ */
+ void finishDataBlock(boolean bForceFinish) throws IOException {
+ if (blkAppender == null) {
+ return;
+ }
+
+ // exceeded the size limit, do the compression and finish the block
+ if (bForceFinish || blkAppender.getCompressedSize() >= sizeMinBlock) {
+ // keep tracks of the last key of each data block, no padding
+ // for now
+ TFileIndexEntry keyLast =
+ new TFileIndexEntry(lastKeyBufferOS.getBuffer(), 0, lastKeyBufferOS
+ .size(), blkRecordCount);
+ tfileIndex.addEntry(keyLast);
+ // close the appender
+ blkAppender.close();
+ blkAppender = null;
+ blkRecordCount = 0;
+ }
+ }
+ }
+
+ /**
+ * TFile Reader. Users may only read TFiles by creating TFile.Reader.Scanner.
+ * objects. A scanner may scan the whole TFile ({@link Reader#createScanner()}
+ * ) , a portion of TFile based on byte offsets (
+ * {@link Reader#createScannerByByteRange(long, long)}), or a portion of TFile with keys
+ * fall in a certain key range (for sorted TFile only,
+ * {@link Reader#createScannerByKey(byte[], byte[])} or
+ * {@link Reader#createScannerByKey(RawComparable, RawComparable)}).
+ */
+ @InterfaceStability.Evolving
+ public static class Reader implements Closeable {
+ // The underlying BCFile reader.
+ final DTBCFile.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<Location>, 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() throws CloneNotSupportedException {
+ return (Location)super.clone();
+ }
+
+ /**
+ * @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 DTBCFile.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.
+ */
+ @Override
+ 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<Scanner.Entry> getEntryComparator() {
+ if (!isSorted()) {
+ throw new RuntimeException(
+ "Entries are not comparable for unsorted TFiles");
+ }
+
+ return new Comparator<Scanner.Entry>() {
+ /**
+ * 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.getBlockBuffer(), o1.getKeyOffset(), o1.getKeyLength(), o2
+ .getBlockBuffer(), o2.getKeyOffset(), 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<RawComparable> 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);
+ }
+
+ Location getLocationByRecordNum(long recNum) throws IOException {
+ checkTFileDataIndex();
+ return tfileIndex.getLocationByRecordNum(recNum);
+ }
+
+ long getRecordNumByLocation(Location location) throws IOException {
+ checkTFileDataIndex();
+ return tfileIndex.getRecordNumByLocation(location);
+ }
+
+ 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 the RecordNum for 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 RecordNum to the corresponding entry. If no such entry
+ * exists, it returns the total entry count.
+ * @throws IOException
+ */
+ public long getRecordNumNear(long offset) throws IOException {
+ return getRecordNumByLocation(getLocationNear(offset));
+ }
+
+ /**
+ * 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 createScannerByByteRange(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
+ *
+ * @deprecated Use {@link #createScannerByKey(byte[], byte[])} instead.
+ */
+ @Deprecated
+ public Scanner createScanner(byte[] beginKey, byte[] endKey)
+ throws IOException {
+ return createScannerByKey(beginKey, endKey);
+ }
+
+ /**
+ * 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 createScannerByKey(byte[] beginKey, byte[] endKey)
+ throws IOException {
+ return createScannerByKey((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
+ *
+ * @deprecated Use {@link #createScannerByKey(RawComparable, RawComparable)}
+ * instead.
+ */
+ @Deprecated
+ public Scanner createScanner(RawComparable beginKey, RawComparable endKey)
+ throws IOException {
+ return createScannerByKey(beginKey, endKey);
+ }
+
+ /**
+ * 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 createScannerByKey(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);
+ }
+
+ /**
+ * Create a scanner that covers a range of records.
+ *
+ * @param beginRecNum
+ * The RecordNum for the first record (inclusive).
+ * @param endRecNum
+ * The RecordNum for the last record (exclusive). To scan the whole
+ * file, either specify endRecNum==-1 or endRecNum==getEntryCount().
+ * @return The TFile scanner that covers the specified range of records.
+ * @throws IOException
+ */
+ public Scanner createScannerByRecordNum(long beginRecNum, long endRecNum)
+ throws IOException {
+ if (beginRecNum < 0) beginRecNum = 0;
+ if (endRecNum < 0 || endRecNum > getEntryCount()) {
+ endRecNum = getEntryCount();
+ }
+ return new Scanner(this, getLocationByRecordNum(beginRecNum),
+ getLocationByRecordNum(endRecNum));
+ }
+
+ /**
+ * 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.
+ * <p>
+ * Use {@link Scanner#atEnd()} to test whether the cursor is at the end
+ * location of the scanner.
+ * <p>
+ * 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.
+ * <p>
+ * 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;
+ private byte[] blockBuffer;
+
+ 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;
+ int keyOffset = 0;
+
+ static final int MAX_VAL_TRANSFER_BUF_SIZE = 128 * 1024;
+ BytesWritable valTransferBuffer;
+
+// DataInputBuffer keyDataInputStream;
+ ChunkDecoder valueBufferInputStream;
+ DataInputStream valueDataInputStream;
+ // vlen == -1 if unknown.
+ int vlen;
+ int valueOffset = 0;
+
+ /**
+ * 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 <K,V>
+ * entry of the TFile.
+ * @param endKey
+ * End key of the scan. If null, scan up to the last <K, V> 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);
+ blockBuffer = blkReader.getBlockDataInputStream().getBuf();
+ 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.
+ */
+ @Override
+ 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);
+ keyOffset = blkReader.getBlockDataInputStream().getPos();
+ blkReader.getBlockDataInputStream().skip(klen);
+ valueBufferInputStream.reset(blkReader);
+ if (valueBufferInputStream.isLastChunk()) {
+ vlen = valueBufferInputStream.getRemain();
+ valueOffset = blkReader.getBlockDataInputStream().getPos();
+ }
+ }
+
+ /**
+ * 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();
+ }
+
+ /**
+ * Get the RecordNum corresponding to the entry pointed by the cursor.
+ * @return The RecordNum corresponding to the entry pointed by the cursor.
+ * @throws IOException
+ */
+ public long getRecordNum() throws IOException {
+ return reader.getRecordNumByLocation(currentLocation);
+ }
+
+ /**
+ * 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(blockBuffer, keyOffset, klen, other.buffer(), other
+ .offset(), other.size());
+ }
+
+ /**
+ * Entry to a <Key, Value> pair.
+ */
+ public class Entry implements Comparable<RawComparable> {
+ /**
+ * Get the length of the key.
+ *
+ * @return the length of the key.
+ */
+ public int getKeyLength() {
+ return klen;
+ }
+
+ public int getKeyOffset() {
+ return keyOffset;
+ }
+
+ public int getValueOffset() {
+ return valueOffset;
+ }
+
+ public byte[] getBlockBuffer() {
+ return blockBuffer;
+ }
+
+ /**
+ * 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.getBytes());
+ return key.getLength();
+ }
+
+ /**
+ * 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.getBytes(), size, remain);
+ size += remain;
+ }
+ return value.getLength();
+ } 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(blockBuffer, keyOffset, 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.getBytes(), 0, chunkSize);
+ out.write(valTransferBuffer.getBytes(), 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(blockBuffer, keyOffset, 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(blockBuffer, getKeyOffset(), 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(blockBuffer, getKeyOffset(), getKeyLength()) == 0;
+ }
+
+ @Override
+ public int hashCode() {
+ return WritableComparator.hashBytes(blockBuffer, getKeyOffset(), 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 = DTFile.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(DTFile.API_VERSION)) {
+ throw new RuntimeException("Incompatible TFile fileVersion.");
+ }
+ recordCount = Utils.readVLong(in);
+ strComparator = Utils.readString(in);
+ comparator = makeComparator(strComparator);
+ }
+
+ @SuppressWarnings("unchecked")
+ 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<Object>) 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 {
+ DTFile.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.isEmpty();
+ }
+
+ 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<TFileIndexEntry> index;
+ private final ArrayList<Long> recordNumIndex;
+ private final BytesComparator comparator;
+ private long sum = 0;
+
+ /**
+ * For reading from file.
+ *
+ * @throws IOException
+ */
+ public TFileIndex(int entryCount, DataInput in, BytesComparator comparator)
+ throws IOException {
+ index = new ArrayList<TFileIndexEntry>(entryCount);
+ recordNumIndex = new ArrayList<Long>(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);
+ sum += idx.entries();
+ recordNumIndex.add(sum);
+ }
+ } 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;
+ }
+
+ /**
+ * @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 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<TFileIndexEntry>();
+ recordNumIndex = new ArrayList<Long>();
+ this.comparator = comparator;
+ }
+
+ public RawComparable getFirstKey() {
+ return firstKey;
+ }
+
+ public Reader.Location getLocationByRecordNum(long recNum) {
+ int idx = Utils.upperBound(recordNumIndex, recNum);
+ long lastRecNum = (idx == 0)? 0: recordNumIndex.get(idx-1);
+ return new Reader.Location(idx, recNum-lastRecNum);
+ }
+
+ public long getRecordNumByLocation(Reader.Location location) {
+ int blkIndex = location.getBlockIndex();
+ long lastRecNum = (blkIndex == 0) ? 0: recordNumIndex.get(blkIndex-1);
+ return lastRecNum + location.getRecordIndex();
+ }
+
+ 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);
+ sum += keyEntry.entries();
+ recordNumIndex.add(sum);
+ }
+
+ 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 <key, value> 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", DTFile.API_VERSION
+ .toString(), DTBCFile.API_VERSION.toString());
+ if (args.length == 0) {
+ System.out
+ .println("Usage: java ... com.datatorrent.contrib.hdht.tfile.withcache.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);
+ }
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-apex-malhar/blob/02f48e1b/library/src/main/java/org/apache/hadoop/io/file/tfile/ReusableByteArrayInputStream.java
----------------------------------------------------------------------
diff --git a/library/src/main/java/org/apache/hadoop/io/file/tfile/ReusableByteArrayInputStream.java b/library/src/main/java/org/apache/hadoop/io/file/tfile/ReusableByteArrayInputStream.java
new file mode 100644
index 0000000..25e4f27
--- /dev/null
+++ b/library/src/main/java/org/apache/hadoop/io/file/tfile/ReusableByteArrayInputStream.java
@@ -0,0 +1,66 @@
+/**
+ * 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;
+
+/**
+ * A reusable ByteArrayInputStream extends {@link ByteArrayInputStream} to avoid creating stream object on same byte array.
+ * <br><br>Call renew() method to reuse this stream from beginning
+ *
+ * @since 2.0.0
+ */
+public class ReusableByteArrayInputStream extends ByteArrayInputStream
+{
+
+ private final int initialOffset;
+
+ private final int initialLength;
+
+ public ReusableByteArrayInputStream(byte[] buf, int offset, int length)
+ {
+ super(buf, offset, length);
+ this.initialLength = Math.min(offset + length, buf.length);
+ this.initialOffset = offset;
+ }
+
+ public ReusableByteArrayInputStream(byte[] buf)
+ {
+ super(buf);
+ this.initialLength = buf.length;
+ this.initialOffset = 0;
+ }
+
+ public void renew()
+ {
+ pos = initialOffset;
+ count = initialLength;
+ mark = 0;
+ }
+
+
+ public int getPos(){
+ return pos;
+ }
+
+ public byte[] getBuf(){
+ return buf;
+ }
+
+}
[7/8] incubator-apex-malhar git commit: MLHR-1877 #resolve #comment
moved DTFile implementation to from contrib to lib
Posted by hs...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-apex-malhar/blob/02f48e1b/contrib/src/main/java/org/apache/hadoop/io/file/tfile/DTFile.java
----------------------------------------------------------------------
diff --git a/contrib/src/main/java/org/apache/hadoop/io/file/tfile/DTFile.java b/contrib/src/main/java/org/apache/hadoop/io/file/tfile/DTFile.java
deleted file mode 100644
index f1c87ba..0000000
--- a/contrib/src/main/java/org/apache/hadoop/io/file/tfile/DTFile.java
+++ /dev/null
@@ -1,2399 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.hadoop.io.file.tfile;
-
-import java.io.ByteArrayInputStream;
-import java.io.Closeable;
-import java.io.DataInput;
-import java.io.DataInputStream;
-import java.io.DataOutput;
-import java.io.DataOutputStream;
-import java.io.EOFException;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.util.ArrayList;
-import java.util.Comparator;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.io.BoundedByteArrayOutputStream;
-import org.apache.hadoop.io.BytesWritable;
-import org.apache.hadoop.io.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.ByteArray;
-import org.apache.hadoop.io.file.tfile.MetaBlockAlreadyExists;
-import org.apache.hadoop.io.file.tfile.MetaBlockDoesNotExist;
-import org.apache.hadoop.io.file.tfile.RawComparable;
-import org.apache.hadoop.io.file.tfile.Utils;
-import org.apache.hadoop.io.file.tfile.DTBCFile.Reader.BlockReader;
-import org.apache.hadoop.io.file.tfile.DTBCFile.Writer.BlockAppender;
-import org.apache.hadoop.io.file.tfile.Utils.Version;
-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.serializer.JavaSerializationComparator;
-
-/**
- * <ul>
- * <li>The file format of DTFile is same as {@link TFile} with different reader implementation.
- * It reads data block by block and cache the binary block data into memory to speed up the random read.
- *
- * <li>The public api of {@link Reader} is as same as it is in {@link TFile} {@link org.apache.hadoop.io.file.tfile.TFile.Reader} implementation.
- * Besides, it provides getBlockBuffer(), getKeyOffset(), getKeyLength(), getValueOffset(), getValueLength() method
- * to expose raw block, key, value data to user to avoid unnecessary internal/external data copy
- *
- * <li>In the performance test, It shows no difference in sequential reads and 20x faster in random reads(If most of them hit memory)
- * </ul>
- *
- * A TFile is a container of key-value pairs. Both keys and values are type-less
- * bytes. Keys are restricted to 64KB, value length is not restricted
- * (practically limited to the available disk storage). TFile further provides
- * the following features:
- * <ul>
- * <li>Block Compression.
- * <li>Named meta data blocks.
- * <li>Sorted or unsorted keys.
- * <li>Seek by key or by file offset.
- * </ul>
- * The memory footprint of a TFile includes the following:
- * <ul>
- * <li>Some constant overhead of reading or writing a compressed block.
- * <ul>
- * <li>Each compressed block requires one compression/decompression codec for
- * I/O.
- * <li>Temporary space to buffer the key.
- * <li>Temporary space to buffer the value (for TFile.Writer only). Values are
- * chunk encoded, so that we buffer at most one chunk of user data. By default,
- * the chunk buffer is 1MB. Reading chunked value does not require additional
- * memory.
- * </ul>
- * <li>TFile index, which is proportional to the total number of Data Blocks.
- * The total amount of memory needed to hold the index can be estimated as
- * (56+AvgKeySize)*NumBlocks.
- * <li>MetaBlock index, which is proportional to the total number of Meta
- * Blocks.The total amount of memory needed to hold the index for Meta Blocks
- * can be estimated as (40+AvgMetaBlockName)*NumMetaBlock.
- * </ul>
- * <p>
- * The behavior of TFile can be customized by the following variables through
- * Configuration:
- * <ul>
- * <li><b>tfile.io.chunk.size</b>: Value chunk size. Integer (in bytes). Default
- * to 1MB. Values of the length less than the chunk size is guaranteed to have
- * known value length in read time (See
- * {@link DTFile.Reader.Scanner.Entry#isValueLengthKnown()}).
- * <li><b>tfile.fs.output.buffer.size</b>: Buffer size used for
- * FSDataOutputStream. Integer (in bytes). Default to 256KB.
- * <li><b>tfile.fs.input.buffer.size</b>: Buffer size used for
- * FSDataInputStream. Integer (in bytes). Default to 256KB.
- * </ul>
- * <p>
- * Suggestions on performance optimization.
- * <ul>
- * <li>Minimum block size. We recommend a setting of minimum block size between
- * 256KB to 1MB for general usage. Larger block size is preferred if files are
- * primarily for sequential access. However, it would lead to inefficient random
- * access (because there are more data to decompress). Smaller blocks are good
- * for random access, but require more memory to hold the block index, and may
- * be slower to create (because we must flush the compressor stream at the
- * conclusion of each data block, which leads to an FS I/O flush). Further, due
- * to the internal caching in Compression codec, the smallest possible block
- * size would be around 20KB-30KB.
- * <li>The current implementation does not offer true multi-threading for
- * reading. The implementation uses FSDataInputStream seek()+read(), which is
- * shown to be much faster than positioned-read call in single thread mode.
- * However, it also means that if multiple threads attempt to access the same
- * TFile (using multiple scanners) simultaneously, the actual I/O is carried out
- * sequentially even if they access different DFS blocks.
- * <li>Compression codec. Use "none" if the data is not very compressable (by
- * compressable, I mean a compression ratio at least 2:1). Generally, use "lzo"
- * as the starting point for experimenting. "gz" overs slightly better
- * compression ratio over "lzo" but requires 4x CPU to compress and 2x CPU to
- * decompress, comparing to "lzo".
- * <li>File system buffering, if the underlying FSDataInputStream and
- * FSDataOutputStream is already adequately buffered; or if applications
- * reads/writes keys and values in large buffers, we can reduce the sizes of
- * input/output buffering in TFile layer by setting the configuration parameters
- * "tfile.fs.input.buffer.size" and "tfile.fs.output.buffer.size".
- * </ul>
- *
- * Some design rationale behind TFile can be found at <a
- * href=https://issues.apache.org/jira/browse/HADOOP-3315>Hadoop-3315</a>.
- *
- * @since 2.0.0
- */
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
-public class DTFile {
- static final Log LOG = LogFactory.getLog(DTFile.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";
-
- public static final int DEFAULT_INPUT_FS_BUF_SIZE = 256 * 1024;
-
- 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) {
- int buffserSize = conf.getInt(FS_INPUT_BUF_SIZE_ATTR, DEFAULT_INPUT_FS_BUF_SIZE);
- if (buffserSize <= 0)
- buffserSize = DEFAULT_INPUT_FS_BUF_SIZE;
- return buffserSize;
- }
-
- static int getFSOutputBufferSize(Configuration conf) {
- return conf.getInt(FS_OUTPUT_BUF_SIZE_ATTR, 256 * 1024);
- }
-
- private static final int MAX_KEY_SIZE = 64 * 1024; // 64KB
- static final Version API_VERSION = new Version((short) 1, (short) 0);
-
- /** compression: gzip */
- public static final String COMPRESSION_GZ = "gz";
- /** compression: lzo */
- public static final String COMPRESSION_LZO = "lzo";
- /** compression: none */
- public static final String COMPRESSION_NONE = "none";
- /** comparator: memcmp */
- public static final String COMPARATOR_MEMCMP = "memcmp";
- /** comparator prefix: java class */
- public static final String COMPARATOR_JCLASS = "jclass:";
-
- /**
- * Make a raw comparator from a string name.
- *
- * @param name
- * Comparator name
- * @return A RawComparable comparator.
- */
- static public Comparator<RawComparable> makeComparator(String name) {
- return TFileMeta.makeComparator(name);
- }
-
- // Prevent the instantiation of TFiles
- private DTFile() {
- // nothing
- }
-
- /**
- * Get names of supported compression algorithms. The names are acceptable by
- * TFile.Writer.
- *
- * @return Array of strings, each represents a supported compression
- * algorithm. Currently, the following compression algorithms are
- * supported.
- * <ul>
- * <li>"none" - No compression.
- * <li>"lzo" - LZO compression.
- * <li>"gz" - GZIP compression.
- * </ul>
- */
- public static String[] getSupportedCompressionAlgorithms() {
- return Compression.getSupportedAlgorithms();
- }
-
- /**
- * TFile Writer.
- */
- @InterfaceStability.Evolving
- public static class Writer implements Closeable {
- // minimum compressed size for a block.
- private final int sizeMinBlock;
-
- // Meta blocks.
- final TFileIndex tfileIndex;
- final TFileMeta tfileMeta;
-
- // reference to the underlying BCFile.
- private DTBCFile.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 DTFile#getSupportedCompressionAlgorithms()}.
- * @param comparator
- * Leave comparator as null or empty string if TFile is not sorted.
- * Otherwise, provide the string name for the comparison algorithm
- * for keys. Two kinds of comparators are supported.
- * <ul>
- * <li>Algorithmic comparator: binary comparators that is language
- * independent. Currently, only "memcmp" is supported.
- * <li>Language-specific comparator: binary comparators that can
- * only be constructed in specific language. For Java, the syntax
- * is "jclass:", followed by the class name of the RawComparator.
- * Currently, we only support RawComparators that can be
- * constructed through the default constructor (with no
- * parameters). Parameterized RawComparators such as
- * {@link WritableComparator} or
- * {@link JavaSerializationComparator} may not be directly used.
- * One should write a wrapper class that inherits from such classes
- * and use its default constructor to perform proper
- * initialization.
- * </ul>
- * @param conf
- * The configuration object.
- * @throws IOException
- */
- public Writer(FSDataOutputStream fsdos, int minBlockSize,
- String compressName, String comparator, Configuration conf)
- throws IOException {
- sizeMinBlock = minBlockSize;
- tfileMeta = new TFileMeta(comparator);
- tfileIndex = new TFileIndex(tfileMeta.getComparator());
-
- writerBCF = new DTBCFile.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.
- */
- @Override
- public void close() throws IOException {
- if ((state == State.CLOSED)) {
- return;
- }
- try {
- // First try the normal finish.
- // Terminate upon the first Exception.
- if (errorCount == 0) {
- if (state != State.READY) {
- throw new IllegalStateException(
- "Cannot close TFile in the middle of key-value insertion.");
- }
-
- finishDataBlock(true);
-
- // first, write out data:TFile.meta
- BlockAppender outMeta =
- writerBCF
- .prepareMetaBlock(TFileMeta.BLOCK_NAME, COMPRESSION_NONE);
- try {
- tfileMeta.write(outMeta);
- } finally {
- outMeta.close();
- }
-
- // second, write out data:TFile.index
- BlockAppender outIndex =
- writerBCF.prepareMetaBlock(TFileIndex.BLOCK_NAME);
- try {
- tfileIndex.write(outIndex);
- } finally {
- outIndex.close();
- }
-
- writerBCF.close();
- }
- } finally {
- IOUtils.cleanup(LOG, blkAppender, writerBCF);
- blkAppender = null;
- writerBCF = null;
- state = State.CLOSED;
- }
- }
-
- /**
- * Adding a new key-value pair to the TFile. This is synonymous to
- * append(key, 0, key.length, value, 0, value.length)
- *
- * @param key
- * Buffer for key.
- * @param value
- * Buffer for value.
- * @throws IOException
- */
- public void append(byte[] key, byte[] value) throws IOException {
- append(key, 0, key.length, value, 0, value.length);
- }
-
- /**
- * Adding a new key-value pair to TFile.
- *
- * @param key
- * buffer for key.
- * @param koff
- * offset in key buffer.
- * @param klen
- * length of key.
- * @param value
- * buffer for value.
- * @param voff
- * offset in value buffer.
- * @param vlen
- * length of value.
- * @throws IOException
- * Upon IO errors.
- * <p>
- * If an exception is thrown, the TFile will be in an inconsistent
- * state. The only legitimate call after that would be close
- */
- public void append(byte[] key, int koff, int klen, byte[] value, int voff,
- int vlen) throws IOException {
- if ((koff | klen | (koff + klen) | (key.length - (koff + klen))) < 0) {
- throw new IndexOutOfBoundsException(
- "Bad key buffer offset-length combination.");
- }
-
- if ((voff | vlen | (voff + vlen) | (value.length - (voff + vlen))) < 0) {
- throw new IndexOutOfBoundsException(
- "Bad value buffer offset-length combination.");
- }
-
- try {
- DataOutputStream dosKey = prepareAppendKey(klen);
- try {
- ++errorCount;
- dosKey.write(key, koff, klen);
- --errorCount;
- } finally {
- dosKey.close();
- }
-
- DataOutputStream dosValue = prepareAppendValue(vlen);
- try {
- ++errorCount;
- dosValue.write(value, voff, vlen);
- --errorCount;
- } finally {
- dosValue.close();
- }
- } finally {
- state = State.READY;
- }
- }
-
- /**
- * Helper class to register key after close call on key append stream.
- */
- private class KeyRegister extends DataOutputStream {
- private final int expectedLength;
- private boolean closed = false;
-
- public KeyRegister(int len) {
- super(currentKeyBufferOS);
- if (len >= 0) {
- currentKeyBufferOS.reset(len);
- } else {
- currentKeyBufferOS.reset();
- }
- expectedLength = len;
- }
-
- @Override
- public void close() throws IOException {
- if (closed == true) {
- return;
- }
-
- try {
- ++errorCount;
- byte[] key = currentKeyBufferOS.getBuffer();
- int len = currentKeyBufferOS.size();
- /**
- * verify length.
- */
- if (expectedLength >= 0 && expectedLength != len) {
- throw new IOException("Incorrect key length: expected="
- + expectedLength + " actual=" + len);
- }
-
- Utils.writeVInt(blkAppender, len);
- blkAppender.write(key, 0, len);
- if (tfileIndex.getFirstKey() == null) {
- tfileIndex.setFirstKey(key, 0, len);
- }
-
- if (tfileMeta.isSorted() && tfileMeta.getRecordCount()>0) {
- byte[] lastKey = lastKeyBufferOS.getBuffer();
- int lastLen = lastKeyBufferOS.size();
- if (tfileMeta.getComparator().compare(key, 0, len, lastKey, 0,
- lastLen) < 0) {
- throw new IOException("Keys are not added in sorted order");
- }
- }
-
- BoundedByteArrayOutputStream tmp = currentKeyBufferOS;
- currentKeyBufferOS = lastKeyBufferOS;
- lastKeyBufferOS = tmp;
- --errorCount;
- } finally {
- closed = true;
- state = State.END_KEY;
- }
- }
- }
-
- /**
- * Helper class to register value after close call on value append stream.
- */
- private class ValueRegister extends DataOutputStream {
- private boolean closed = false;
-
- public ValueRegister(OutputStream os) {
- super(os);
- }
-
- // Avoiding flushing call to down stream.
- @Override
- public void flush() {
- // do nothing
- }
-
- @Override
- public void close() throws IOException {
- if (closed == true) {
- return;
- }
-
- try {
- ++errorCount;
- super.close();
- blkRecordCount++;
- // bump up the total record count in the whole file
- tfileMeta.incRecordCount();
- finishDataBlock(false);
- --errorCount;
- } finally {
- closed = true;
- state = State.READY;
- }
- }
- }
-
- /**
- * Obtain an output stream for writing a key into TFile. This may only be
- * called when there is no active Key appending stream or value appending
- * stream.
- *
- * @param length
- * The expected length of the key. If length of the key is not
- * known, set length = -1. Otherwise, the application must write
- * exactly as many bytes as specified here before calling close on
- * the returned output stream.
- * @return The key appending output stream.
- * @throws IOException
- *
- */
- public DataOutputStream prepareAppendKey(int length) throws IOException {
- if (state != State.READY) {
- throw new IllegalStateException("Incorrect state to start a new key: "
- + state.name());
- }
-
- initDataBlock();
- DataOutputStream ret = new KeyRegister(length);
- state = State.IN_KEY;
- return ret;
- }
-
- /**
- * Obtain an output stream for writing a value into TFile. This may only be
- * called right after a key appending operation (the key append stream must
- * be closed).
- *
- * @param length
- * The expected length of the value. If length of the value is not
- * known, set length = -1. Otherwise, the application must write
- * exactly as many bytes as specified here before calling close on
- * the returned output stream. Advertising the value size up-front
- * guarantees that the value is encoded in one chunk, and avoids
- * intermediate chunk buffering.
- * @throws IOException
- *
- */
- public DataOutputStream prepareAppendValue(int length) throws IOException {
- if (state != State.END_KEY) {
- throw new IllegalStateException(
- "Incorrect state to start a new value: " + state.name());
- }
-
- DataOutputStream ret;
-
- // unknown length
- if (length < 0) {
- if (valueBuffer == null) {
- valueBuffer = new byte[getChunkBufferSize(conf)];
- }
- ret = new ValueRegister(new ChunkEncoder(blkAppender, valueBuffer));
- } else {
- ret =
- new ValueRegister(new Chunk.SingleChunkEncoder(blkAppender, length));
- }
-
- state = State.IN_VALUE;
- return ret;
- }
-
- /**
- * Obtain an output stream for creating a meta block. This function may not
- * be called when there is a key append stream or value append stream
- * active. No more key-value insertion is allowed after a meta data block
- * has been added to TFile.
- *
- * @param name
- * Name of the meta block.
- * @param compressName
- * Name of the compression algorithm to be used. Must be one of the
- * strings returned by
- * {@link DTFile#getSupportedCompressionAlgorithms()}.
- * @return A DataOutputStream that can be used to write Meta Block data.
- * Closing the stream would signal the ending of the block.
- * @throws IOException
- * @throws MetaBlockAlreadyExists
- * the Meta Block with the same name already exists.
- */
- public DataOutputStream prepareMetaBlock(String name, String compressName)
- throws IOException, MetaBlockAlreadyExists {
- if (state != State.READY) {
- throw new IllegalStateException(
- "Incorrect state to start a Meta Block: " + state.name());
- }
-
- finishDataBlock(true);
- DataOutputStream outputStream =
- writerBCF.prepareMetaBlock(name, compressName);
- return outputStream;
- }
-
- /**
- * Obtain an output stream for creating a meta block. This function may not
- * be called when there is a key append stream or value append stream
- * active. No more key-value insertion is allowed after a meta data block
- * has been added to TFile. Data will be compressed using the default
- * compressor as defined in Writer's constructor.
- *
- * @param name
- * Name of the meta block.
- * @return A DataOutputStream that can be used to write Meta Block data.
- * Closing the stream would signal the ending of the block.
- * @throws IOException
- * @throws MetaBlockAlreadyExists
- * the Meta Block with the same name already exists.
- */
- public DataOutputStream prepareMetaBlock(String name) throws IOException,
- MetaBlockAlreadyExists {
- if (state != State.READY) {
- throw new IllegalStateException(
- "Incorrect state to start a Meta Block: " + state.name());
- }
-
- finishDataBlock(true);
- return writerBCF.prepareMetaBlock(name);
- }
-
- /**
- * Check if we need to start a new data block.
- *
- * @throws IOException
- */
- private void initDataBlock() throws IOException {
- // for each new block, get a new appender
- if (blkAppender == null) {
- blkAppender = writerBCF.prepareDataBlock();
- }
- }
-
- /**
- * Close the current data block if necessary.
- *
- * @param bForceFinish
- * Force the closure regardless of the block size.
- * @throws IOException
- */
- void finishDataBlock(boolean bForceFinish) throws IOException {
- if (blkAppender == null) {
- return;
- }
-
- // exceeded the size limit, do the compression and finish the block
- if (bForceFinish || blkAppender.getCompressedSize() >= sizeMinBlock) {
- // keep tracks of the last key of each data block, no padding
- // for now
- TFileIndexEntry keyLast =
- new TFileIndexEntry(lastKeyBufferOS.getBuffer(), 0, lastKeyBufferOS
- .size(), blkRecordCount);
- tfileIndex.addEntry(keyLast);
- // close the appender
- blkAppender.close();
- blkAppender = null;
- blkRecordCount = 0;
- }
- }
- }
-
- /**
- * TFile Reader. Users may only read TFiles by creating TFile.Reader.Scanner.
- * objects. A scanner may scan the whole TFile ({@link Reader#createScanner()}
- * ) , a portion of TFile based on byte offsets (
- * {@link Reader#createScannerByByteRange(long, long)}), or a portion of TFile with keys
- * fall in a certain key range (for sorted TFile only,
- * {@link Reader#createScannerByKey(byte[], byte[])} or
- * {@link Reader#createScannerByKey(RawComparable, RawComparable)}).
- */
- @InterfaceStability.Evolving
- public static class Reader implements Closeable {
- // The underlying BCFile reader.
- final DTBCFile.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<Location>, 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() throws CloneNotSupportedException {
- return (Location)super.clone();
- }
-
- /**
- * @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 DTBCFile.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.
- */
- @Override
- 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<Scanner.Entry> getEntryComparator() {
- if (!isSorted()) {
- throw new RuntimeException(
- "Entries are not comparable for unsorted TFiles");
- }
-
- return new Comparator<Scanner.Entry>() {
- /**
- * 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.getBlockBuffer(), o1.getKeyOffset(), o1.getKeyLength(), o2
- .getBlockBuffer(), o2.getKeyOffset(), 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<RawComparable> 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);
- }
-
- Location getLocationByRecordNum(long recNum) throws IOException {
- checkTFileDataIndex();
- return tfileIndex.getLocationByRecordNum(recNum);
- }
-
- long getRecordNumByLocation(Location location) throws IOException {
- checkTFileDataIndex();
- return tfileIndex.getRecordNumByLocation(location);
- }
-
- 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 the RecordNum for 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 RecordNum to the corresponding entry. If no such entry
- * exists, it returns the total entry count.
- * @throws IOException
- */
- public long getRecordNumNear(long offset) throws IOException {
- return getRecordNumByLocation(getLocationNear(offset));
- }
-
- /**
- * 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 createScannerByByteRange(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
- *
- * @deprecated Use {@link #createScannerByKey(byte[], byte[])} instead.
- */
- @Deprecated
- public Scanner createScanner(byte[] beginKey, byte[] endKey)
- throws IOException {
- return createScannerByKey(beginKey, endKey);
- }
-
- /**
- * 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 createScannerByKey(byte[] beginKey, byte[] endKey)
- throws IOException {
- return createScannerByKey((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
- *
- * @deprecated Use {@link #createScannerByKey(RawComparable, RawComparable)}
- * instead.
- */
- @Deprecated
- public Scanner createScanner(RawComparable beginKey, RawComparable endKey)
- throws IOException {
- return createScannerByKey(beginKey, endKey);
- }
-
- /**
- * 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 createScannerByKey(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);
- }
-
- /**
- * Create a scanner that covers a range of records.
- *
- * @param beginRecNum
- * The RecordNum for the first record (inclusive).
- * @param endRecNum
- * The RecordNum for the last record (exclusive). To scan the whole
- * file, either specify endRecNum==-1 or endRecNum==getEntryCount().
- * @return The TFile scanner that covers the specified range of records.
- * @throws IOException
- */
- public Scanner createScannerByRecordNum(long beginRecNum, long endRecNum)
- throws IOException {
- if (beginRecNum < 0) beginRecNum = 0;
- if (endRecNum < 0 || endRecNum > getEntryCount()) {
- endRecNum = getEntryCount();
- }
- return new Scanner(this, getLocationByRecordNum(beginRecNum),
- getLocationByRecordNum(endRecNum));
- }
-
- /**
- * 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.
- * <p>
- * Use {@link Scanner#atEnd()} to test whether the cursor is at the end
- * location of the scanner.
- * <p>
- * 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.
- * <p>
- * 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;
- private byte[] blockBuffer;
-
- 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;
- int keyOffset = 0;
-
- static final int MAX_VAL_TRANSFER_BUF_SIZE = 128 * 1024;
- BytesWritable valTransferBuffer;
-
-// DataInputBuffer keyDataInputStream;
- ChunkDecoder valueBufferInputStream;
- DataInputStream valueDataInputStream;
- // vlen == -1 if unknown.
- int vlen;
- int valueOffset = 0;
-
- /**
- * 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 <K,V>
- * entry of the TFile.
- * @param endKey
- * End key of the scan. If null, scan up to the last <K, V> 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);
- blockBuffer = blkReader.getBlockDataInputStream().getBuf();
- 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.
- */
- @Override
- 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);
- keyOffset = blkReader.getBlockDataInputStream().getPos();
- blkReader.getBlockDataInputStream().skip(klen);
- valueBufferInputStream.reset(blkReader);
- if (valueBufferInputStream.isLastChunk()) {
- vlen = valueBufferInputStream.getRemain();
- valueOffset = blkReader.getBlockDataInputStream().getPos();
- }
- }
-
- /**
- * 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();
- }
-
- /**
- * Get the RecordNum corresponding to the entry pointed by the cursor.
- * @return The RecordNum corresponding to the entry pointed by the cursor.
- * @throws IOException
- */
- public long getRecordNum() throws IOException {
- return reader.getRecordNumByLocation(currentLocation);
- }
-
- /**
- * 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(blockBuffer, keyOffset, klen, other.buffer(), other
- .offset(), other.size());
- }
-
- /**
- * Entry to a <Key, Value> pair.
- */
- public class Entry implements Comparable<RawComparable> {
- /**
- * Get the length of the key.
- *
- * @return the length of the key.
- */
- public int getKeyLength() {
- return klen;
- }
-
- public int getKeyOffset() {
- return keyOffset;
- }
-
- public int getValueOffset() {
- return valueOffset;
- }
-
- public byte[] getBlockBuffer() {
- return blockBuffer;
- }
-
- /**
- * 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.getBytes());
- return key.getLength();
- }
-
- /**
- * 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.getBytes(), size, remain);
- size += remain;
- }
- return value.getLength();
- } 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(blockBuffer, keyOffset, 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.getBytes(), 0, chunkSize);
- out.write(valTransferBuffer.getBytes(), 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(blockBuffer, keyOffset, 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(blockBuffer, getKeyOffset(), 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(blockBuffer, getKeyOffset(), getKeyLength()) == 0;
- }
-
- @Override
- public int hashCode() {
- return WritableComparator.hashBytes(blockBuffer, getKeyOffset(), 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 = DTFile.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(DTFile.API_VERSION)) {
- throw new RuntimeException("Incompatible TFile fileVersion.");
- }
- recordCount = Utils.readVLong(in);
- strComparator = Utils.readString(in);
- comparator = makeComparator(strComparator);
- }
-
- @SuppressWarnings("unchecked")
- 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<Object>) 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 {
- DTFile.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.isEmpty();
- }
-
- 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<TFileIndexEntry> index;
- private final ArrayList<Long> recordNumIndex;
- private final BytesComparator comparator;
- private long sum = 0;
-
- /**
- * For reading from file.
- *
- * @throws IOException
- */
- public TFileIndex(int entryCount, DataInput in, BytesComparator comparator)
- throws IOException {
- index = new ArrayList<TFileIndexEntry>(entryCount);
- recordNumIndex = new ArrayList<Long>(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);
- sum += idx.entries();
- recordNumIndex.add(sum);
- }
- } 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;
- }
-
- /**
- * @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 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<TFileIndexEntry>();
- recordNumIndex = new ArrayList<Long>();
- this.comparator = comparator;
- }
-
- public RawComparable getFirstKey() {
- return firstKey;
- }
-
- public Reader.Location getLocationByRecordNum(long recNum) {
- int idx = Utils.upperBound(recordNumIndex, recNum);
- long lastRecNum = (idx == 0)? 0: recordNumIndex.get(idx-1);
- return new Reader.Location(idx, recNum-lastRecNum);
- }
-
- public long getRecordNumByLocation(Reader.Location location) {
- int blkIndex = location.getBlockIndex();
- long lastRecNum = (blkIndex == 0) ? 0: recordNumIndex.get(blkIndex-1);
- return lastRecNum + location.getRecordIndex();
- }
-
- 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);
- sum += keyEntry.entries();
- recordNumIndex.add(sum);
- }
-
- 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 <key, value> 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", DTFile.API_VERSION
- .toString(), DTBCFile.API_VERSION.toString());
- if (args.length == 0) {
- System.out
- .println("Usage: java ... com.datatorrent.contrib.hdht.tfile.withcache.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);
- }
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-apex-malhar/blob/02f48e1b/contrib/src/main/java/org/apache/hadoop/io/file/tfile/ReusableByteArrayInputStream.java
----------------------------------------------------------------------
diff --git a/contrib/src/main/java/org/apache/hadoop/io/file/tfile/ReusableByteArrayInputStream.java b/contrib/src/main/java/org/apache/hadoop/io/file/tfile/ReusableByteArrayInputStream.java
deleted file mode 100644
index 25e4f27..0000000
--- a/contrib/src/main/java/org/apache/hadoop/io/file/tfile/ReusableByteArrayInputStream.java
+++ /dev/null
@@ -1,66 +0,0 @@
-/**
- * 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;
-
-/**
- * A reusable ByteArrayInputStream extends {@link ByteArrayInputStream} to avoid creating stream object on same byte array.
- * <br><br>Call renew() method to reuse this stream from beginning
- *
- * @since 2.0.0
- */
-public class ReusableByteArrayInputStream extends ByteArrayInputStream
-{
-
- private final int initialOffset;
-
- private final int initialLength;
-
- public ReusableByteArrayInputStream(byte[] buf, int offset, int length)
- {
- super(buf, offset, length);
- this.initialLength = Math.min(offset + length, buf.length);
- this.initialOffset = offset;
- }
-
- public ReusableByteArrayInputStream(byte[] buf)
- {
- super(buf);
- this.initialLength = buf.length;
- this.initialOffset = 0;
- }
-
- public void renew()
- {
- pos = initialOffset;
- count = initialLength;
- mark = 0;
- }
-
-
- public int getPos(){
- return pos;
- }
-
- public byte[] getBuf(){
- return buf;
- }
-
-}
[4/8] incubator-apex-malhar git commit: MLHR-1877 #resolve #comment
moved DTFile implementation to from contrib to lib
Posted by hs...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-apex-malhar/blob/02f48e1b/library/src/main/java/org/apache/hadoop/io/file/tfile/DTBCFile.java
----------------------------------------------------------------------
diff --git a/library/src/main/java/org/apache/hadoop/io/file/tfile/DTBCFile.java b/library/src/main/java/org/apache/hadoop/io/file/tfile/DTBCFile.java
new file mode 100644
index 0000000..779b0f0
--- /dev/null
+++ b/library/src/main/java/org/apache/hadoop/io/file/tfile/DTBCFile.java
@@ -0,0 +1,1044 @@
+/**
+ * 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.commons.io.output.ByteArrayOutputStream;
+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.Utils.Version;
+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;
+
+
+/**
+ *
+ * <ul>
+ * <li>The file format of DTFile is same as {@link TFile} with different reader implementation.
+ * It reads data block by block and cache the binary block data into memory to speed up the random read.
+ *
+ * <li>The public api of {@link Reader} is as same as it is in {@link TFile} {@link org.apache.hadoop.io.file.tfile.TFile.Reader} implementation.
+ * Besides, it provides getBlockBuffer(), getKeyOffset(), getKeyLength(), getValueOffset(), getValueLength() method
+ * to expose raw block, key, value data to user to avoid unnecessary internal/external data copy
+ *
+ * <li>In the performance test, It shows no difference in sequential reads and 20x faster in random reads(If most of them hit memory)
+ * </ul>
+ *
+ * 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.
+ *
+ * @since 2.0.0
+ */
+final class DTBCFile {
+ // 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(DTBCFile.class);
+
+ /**
+ * Prevent the instantiation of BCFile objects.
+ */
+ private DTBCFile() {
+ // 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(DTFile.getFSOutputBufferSize(conf));
+
+ this.fsBufferedOutput =
+ new SimpleBufferedOutputStream(this.fsOut, fsOutputBuffer.getBytes());
+ this.compressor = compressAlgo.getCompressor();
+
+ try {
+ this.out =
+ compressionAlgo.createCompressionStream(fsBufferedOutput,
+ compressor, 0);
+ } catch (IOException e) {
+ compressAlgo.returnCompressor(compressor);
+ throw e;
+ }
+ }
+
+ /**
+ * Get the output stream for BlockAppender's consumption.
+ *
+ * @return the output stream suitable for writing block data.
+ */
+ OutputStream getOutputStream() {
+ return out;
+ }
+
+ /**
+ * Get the current position in file.
+ *
+ * @return The current byte offset in underlying file.
+ * @throws IOException
+ */
+ long getCurrentPos() throws IOException {
+ return fsOut.getPos() + fsBufferedOutput.size();
+ }
+
+ long getStartPos() {
+ return posStart;
+ }
+
+ /**
+ * Current size of compressed data.
+ *
+ * @return
+ * @throws IOException
+ */
+ long getCompressedSize() throws IOException {
+ long ret = getCurrentPos() - posStart;
+ return ret;
+ }
+
+ /**
+ * Finishing up the current block.
+ */
+ public void finish() throws IOException {
+ try {
+ if (out != null) {
+ out.flush();
+ out = null;
+ }
+ } finally {
+ compressAlgo.returnCompressor(compressor);
+ compressor = null;
+ }
+ }
+ }
+
+ /**
+ * Access point to stuff data into a block.
+ *
+ * TODO: Change DataOutputStream to something else that tracks the size as
+ * long instead of int. Currently, we will wrap around if the row block size
+ * is greater than 4GB.
+ */
+ public class BlockAppender extends DataOutputStream {
+ private final BlockRegister blockRegister;
+ private final WBlockState wBlkState;
+ @SuppressWarnings("hiding")
+ private boolean closed = false;
+
+ /**
+ * Constructor
+ *
+ * @param register
+ * the block register, which is called when the block is closed.
+ * @param wbs
+ * The writable compression block state.
+ */
+ BlockAppender(BlockRegister register, WBlockState wbs) {
+ super(wbs.getOutputStream());
+ this.blockRegister = register;
+ this.wBlkState = wbs;
+ }
+
+ /**
+ * Get the raw size of the block.
+ *
+ * @return the number of uncompressed bytes written through the
+ * BlockAppender so far.
+ * @throws IOException
+ */
+ public long getRawSize() throws IOException {
+ /**
+ * Expecting the size() of a block not exceeding 4GB. Assuming the
+ * size() will wrap to negative integer if it exceeds 2GB.
+ */
+ return size() & 0x00000000ffffffffL;
+ }
+
+ /**
+ * Get the compressed size of the block in progress.
+ *
+ * @return the number of compressed bytes written to the underlying FS
+ * file. The size may be smaller than actual need to compress the
+ * all data written due to internal buffering inside the
+ * compressor.
+ * @throws IOException
+ */
+ public long getCompressedSize() throws IOException {
+ return wBlkState.getCompressedSize();
+ }
+
+ @Override
+ public void flush() {
+ // The down stream is a special kind of stream that finishes a
+ // compression block upon flush. So we disable flush() here.
+ }
+
+ /**
+ * Signaling the end of write to the block. The block register will be
+ * called for registering the finished block.
+ */
+ @Override
+ public void close() throws IOException {
+ if (closed == true) {
+ return;
+ }
+ try {
+ ++errorCount;
+ wBlkState.finish();
+ blockRegister.register(getRawSize(), wBlkState.getStartPos(),
+ wBlkState.getCurrentPos());
+ --errorCount;
+ } finally {
+ closed = true;
+ blkInProgress = false;
+ }
+ }
+ }
+
+ /**
+ * Constructor
+ *
+ * @param fout
+ * FS output stream.
+ * @param compressionName
+ * Name of the compression algorithm, which will be used for all
+ * data blocks.
+ * @throws IOException
+ * @see Compression#getSupportedAlgorithms
+ */
+ public Writer(FSDataOutputStream fout, String compressionName,
+ Configuration conf) throws IOException {
+ if (fout.getPos() != 0) {
+ throw new IOException("Output file not at zero offset.");
+ }
+
+ this.out = fout;
+ this.conf = conf;
+ dataIndex = new DataIndex(compressionName);
+ metaIndex = new MetaIndex();
+ fsOutputBuffer = new BytesWritable();
+ Magic.write(fout);
+ }
+
+ /**
+ * Close the BCFile Writer. Attempting to use the Writer after calling
+ * <code>close</code> is not allowed and may lead to undetermined results.
+ */
+ @Override
+ 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;
+ }
+
+ @Override
+ 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
+ }
+
+ @Override
+ 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;
+ //
+ private ByteArrayOutputStream baos;
+ private ArrayList<String> cacheKeys;
+
+ public ArrayList<String> getCacheKeys()
+ {
+ return cacheKeys;
+ }
+
+ /**
+ * Intermediate class that maintain the state of a Readable Compression
+ * Block.
+ */
+ static private final class RBlockState {
+ private final Algorithm compressAlgo;
+ private final ReusableByteArrayInputStream rbain;
+ private final BlockRegion region;
+
+ public RBlockState(Algorithm compressionAlgo, FSDataInputStream fsin, BlockRegion region, Configuration conf, Reader r) throws IOException
+ {
+ this.compressAlgo = compressionAlgo;
+ Decompressor decompressor = compressionAlgo.getDecompressor();
+ this.region = region;
+ try {
+
+ InputStream in = compressAlgo.createDecompressionStream(new BoundedRangeFileInputStream(fsin, region.getOffset(), region.getCompressedSize()), decompressor, DTFile.getFSInputBufferSize(conf));
+ int l = 1;
+ r.baos.reset();
+ byte[] buf = new byte[DTFile.getFSInputBufferSize(conf)];
+ while (l >= 0) {
+ l = in.read(buf);
+ if (l > 0) {
+ r.baos.write(buf, 0, l);
+ }
+ }
+ // keep decompressed data into cache
+ byte[] blockData = r.baos.toByteArray();
+ rbain = new ReusableByteArrayInputStream(blockData);
+ } 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 ReusableByteArrayInputStream getInputStream() {
+ return rbain;
+ }
+
+ public String getCompressionName() {
+ return compressAlgo.getName();
+ }
+
+ public BlockRegion getBlockRegion() {
+ return region;
+ }
+
+ public void finish() throws IOException {
+ try {
+ rbain.close();
+ } finally {
+ }
+ }
+
+ public void renew()
+ {
+ rbain.renew();
+ }
+ }
+
+ /**
+ * Access point to read a block.
+ */
+ public static class BlockReader extends DataInputStream {
+ private final RBlockState rBlkState;
+ private boolean closed = false;
+
+ private ReusableByteArrayInputStream wrappedInputStream = null;
+
+ BlockReader(RBlockState rbs) {
+ super(rbs.getInputStream());
+ rBlkState = rbs;
+ wrappedInputStream = rbs.getInputStream();
+ }
+
+ /**
+ * 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();
+ }
+
+ public void renew()
+ {
+ closed = false;
+ rBlkState.renew();
+ }
+
+ public ReusableByteArrayInputStream getBlockDataInputStream()
+ {
+ return wrappedInputStream;
+ }
+ }
+
+ /**
+ * 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;
+ // A reader buffer to read the block
+ baos = new ByteArrayOutputStream(DTFile.getFSInputBufferSize(conf) * 2);
+ this.cacheKeys = new ArrayList<String>();
+ // 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(DTBCFile.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.
+ */
+ @Override
+ public void close() {
+ // Delete buffers in cache for this reader.
+ CacheManager.invalidateKeys(cacheKeys);
+ cacheKeys.clear();
+ }
+
+ /**
+ * 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 {
+ BlockReader br = (BlockReader) CacheManager.get(region.getOffset() + this.toString());
+ if(br==null){
+ RBlockState rbs = new RBlockState(compressAlgo, in, region, conf, this);
+ br = new BlockReader(rbs);
+ String cacheKey = region.getOffset() + this.toString();
+ CacheManager.put(cacheKey, br);
+ cacheKeys.add(cacheKey);
+ } else {
+ br.renew();
+ }
+ return br;
+ }
+
+ /**
+ * Find the smallest Block index whose starting offset is greater than or
+ * equal to the specified offset.
+ *
+ * @param offset
+ * User-specific offset.
+ * @return the index to the data Block if such block exists; or -1
+ * otherwise.
+ */
+ public int getBlockIndexNear(long offset) {
+ ArrayList<BlockRegion> list = dataIndex.getBlockRegionList();
+ int idx =
+ Utils
+ .lowerBound(list, new ScalarLong(offset), new ScalarComparator());
+
+ if (idx == list.size()) {
+ return -1;
+ }
+
+ return idx;
+ }
+ }
+
+ /**
+ * Index for all Meta blocks.
+ */
+ static class MetaIndex {
+ // use a tree map, for getting a meta block entry by name
+ final Map<String, MetaIndexEntry> index;
+
+ // for write
+ public MetaIndex() {
+ index = new TreeMap<String, MetaIndexEntry>();
+ }
+
+ // for read, construct the map from the file
+ public MetaIndex(DataInput in) throws IOException {
+ int count = Utils.readVInt(in);
+ index = new TreeMap<String, MetaIndexEntry>();
+
+ for (int nx = 0; nx < count; nx++) {
+ MetaIndexEntry indexEntry = new MetaIndexEntry(in);
+ index.put(indexEntry.getMetaName(), indexEntry);
+ }
+ }
+
+ public void addEntry(MetaIndexEntry indexEntry) {
+ index.put(indexEntry.getMetaName(), indexEntry);
+ }
+
+ public MetaIndexEntry getMetaByName(String name) {
+ return index.get(name);
+ }
+
+ public void write(DataOutput out) throws IOException {
+ Utils.writeVInt(out, index.size());
+
+ for (MetaIndexEntry indexEntry : index.values()) {
+ indexEntry.write(out);
+ }
+ }
+ }
+
+ /**
+ * An entry describes a meta block in the MetaIndex.
+ */
+ static final class MetaIndexEntry {
+ private final String metaName;
+ private final Algorithm compressionAlgorithm;
+ private final static String defaultPrefix = "data:";
+
+ private final BlockRegion region;
+
+ public MetaIndexEntry(DataInput in) throws IOException {
+ String fullMetaName = Utils.readString(in);
+ if (fullMetaName.startsWith(defaultPrefix)) {
+ metaName =
+ fullMetaName.substring(defaultPrefix.length(), fullMetaName
+ .length());
+ } else {
+ throw new IOException("Corrupted Meta region Index");
+ }
+
+ compressionAlgorithm =
+ Compression.getCompressionAlgorithmByName(Utils.readString(in));
+ region = new BlockRegion(in);
+ }
+
+ public MetaIndexEntry(String metaName, Algorithm compressionAlgorithm,
+ BlockRegion region) {
+ this.metaName = metaName;
+ this.compressionAlgorithm = compressionAlgorithm;
+ this.region = region;
+ }
+
+ public String getMetaName() {
+ return metaName;
+ }
+
+ public Algorithm getCompressionAlgorithm() {
+ return compressionAlgorithm;
+ }
+
+ public BlockRegion getRegion() {
+ return region;
+ }
+
+ public void write(DataOutput out) throws IOException {
+ Utils.writeString(out, defaultPrefix + metaName);
+ Utils.writeString(out, compressionAlgorithm.getName());
+
+ region.write(out);
+ }
+ }
+
+ /**
+ * Index of all compressed data blocks.
+ */
+ static class DataIndex {
+ final static String BLOCK_NAME = "BCFile.index";
+
+ private final Algorithm defaultCompressionAlgorithm;
+
+ // for data blocks, each entry specifies a block's offset, compressed size
+ // and raw size
+ private final ArrayList<BlockRegion> listRegions;
+
+ // for read, deserialized from a file
+ public DataIndex(DataInput in) throws IOException {
+ defaultCompressionAlgorithm =
+ Compression.getCompressionAlgorithmByName(Utils.readString(in));
+
+ int n = Utils.readVInt(in);
+ listRegions = new ArrayList<BlockRegion>(n);
+
+ for (int i = 0; i < n; i++) {
+ BlockRegion region = new BlockRegion(in);
+ listRegions.add(region);
+ }
+ }
+
+ // for write
+ public DataIndex(String defaultCompressionAlgorithmName) {
+ this.defaultCompressionAlgorithm =
+ Compression
+ .getCompressionAlgorithmByName(defaultCompressionAlgorithmName);
+ listRegions = new ArrayList<BlockRegion>();
+ }
+
+ public Algorithm getDefaultCompressionAlgorithm() {
+ return defaultCompressionAlgorithm;
+ }
+
+ public ArrayList<BlockRegion> getBlockRegionList() {
+ return listRegions;
+ }
+
+ public void addBlockRegion(BlockRegion region) {
+ listRegions.add(region);
+ }
+
+ public void write(DataOutput out) throws IOException {
+ Utils.writeString(out, defaultCompressionAlgorithm.getName());
+
+ Utils.writeVInt(out, listRegions.size());
+
+ for (BlockRegion region : listRegions) {
+ region.write(out);
+ }
+ }
+ }
+
+ /**
+ * Magic number uniquely identifying a BCFile in the header/footer.
+ */
+ static final class Magic {
+ private final static byte[] AB_MAGIC_BCFILE =
+ {
+ // ... total of 16 bytes
+ (byte) 0xd1, (byte) 0x11, (byte) 0xd3, (byte) 0x68, (byte) 0x91,
+ (byte) 0xb5, (byte) 0xd7, (byte) 0xb6, (byte) 0x39, (byte) 0xdf,
+ (byte) 0x41, (byte) 0x40, (byte) 0x92, (byte) 0xba, (byte) 0xe1,
+ (byte) 0x50 };
+
+ public static void readAndVerify(DataInput in) throws IOException {
+ byte[] abMagic = new byte[size()];
+ in.readFully(abMagic);
+
+ // check against AB_MAGIC_BCFILE, if not matching, throw an
+ // Exception
+ if (!Arrays.equals(abMagic, AB_MAGIC_BCFILE)) {
+ throw new IOException("Not a valid BCFile.");
+ }
+ }
+
+ public static void write(DataOutput out) throws IOException {
+ out.write(AB_MAGIC_BCFILE);
+ }
+
+ public static int size() {
+ return AB_MAGIC_BCFILE.length;
+ }
+ }
+
+ /**
+ * Block region.
+ */
+ static final class BlockRegion implements Scalar {
+ private final long offset;
+ private final long compressedSize;
+ private final long rawSize;
+
+ public BlockRegion(DataInput in) throws IOException {
+ offset = Utils.readVLong(in);
+ compressedSize = Utils.readVLong(in);
+ rawSize = Utils.readVLong(in);
+ }
+
+ public BlockRegion(long offset, long compressedSize, long rawSize) {
+ this.offset = offset;
+ this.compressedSize = compressedSize;
+ this.rawSize = rawSize;
+ }
+
+ public void write(DataOutput out) throws IOException {
+ Utils.writeVLong(out, offset);
+ Utils.writeVLong(out, compressedSize);
+ Utils.writeVLong(out, rawSize);
+ }
+
+ public long getOffset() {
+ return offset;
+ }
+
+ public long getCompressedSize() {
+ return compressedSize;
+ }
+
+ public long getRawSize() {
+ return rawSize;
+ }
+
+ @Override
+ public long magnitude() {
+ return offset;
+ }
+ }
+}
[5/8] incubator-apex-malhar git commit: MLHR-1877 #resolve #comment
moved DTFile implementation to from contrib to lib
Posted by hs...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-apex-malhar/blob/02f48e1b/contrib/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileSeqFileComparison.java
----------------------------------------------------------------------
diff --git a/contrib/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileSeqFileComparison.java b/contrib/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileSeqFileComparison.java
deleted file mode 100644
index e513ccd..0000000
--- a/contrib/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileSeqFileComparison.java
+++ /dev/null
@@ -1,802 +0,0 @@
-/**
- * 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.fs.CommonConfigurationKeys;
-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.DTFile.Reader.Scanner.Entry;
-import org.apache.hadoop.util.Time;
-
-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 = Time.now();
- System.out.println(formatTime() + " Started timing.");
- }
-
- public void stopTime() throws IOException {
- finishTimeEpoch = Time.now();
- 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(Time.now());
- }
-
- 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);
- }
-
- @Override
- public void append(BytesWritable key, BytesWritable value)
- throws IOException {
- writer.append(key.get(), 0, key.getSize(), value.get(), 0, value
- .getSize());
- }
-
- @Override
- public void close() throws IOException {
- writer.close();
- fsdos.close();
- }
- }
-
- static class TFileReadable implements KVReadable {
- private FSDataInputStream fsdis;
- private DTFile.Reader reader;
- private DTFile.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 DTFile.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)];
- }
-
- @Override
- public byte[] getKey() {
- return keyBuffer;
- }
-
- @Override
- public int getKeyLength() {
- return keyLength;
- }
-
- @Override
- public byte[] getValue() {
- return valueBuffer;
- }
-
- @Override
- public int getValueLength() {
- return valueLength;
- }
-
- @Override
- 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;
- }
-
- @Override
- 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(CommonConfigurationKeys.IO_NATIVE_LIB_AVAILABLE_KEY,
- 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);
- }
- }
-
- @Override
- public void append(BytesWritable key, BytesWritable value)
- throws IOException {
- writer.append(key, value);
- }
-
- @Override
- 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();
- }
-
- @Override
- public byte[] getKey() {
- return key.get();
- }
-
- @Override
- public int getKeyLength() {
- return key.getSize();
- }
-
- @Override
- public byte[] getValue() {
- return value.get();
- }
-
- @Override
- public int getValueLength() {
- return value.getSize();
- }
-
- @Override
- public boolean next() throws IOException {
- return reader.next(key, value);
- }
-
- @Override
- 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();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-apex-malhar/blob/02f48e1b/contrib/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileSplit.java
----------------------------------------------------------------------
diff --git a/contrib/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileSplit.java b/contrib/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileSplit.java
deleted file mode 100644
index aad563d..0000000
--- a/contrib/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileSplit.java
+++ /dev/null
@@ -1,194 +0,0 @@
-/**
- * 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.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.DTFile.Reader;
-import org.apache.hadoop.io.file.tfile.DTFile.Writer;
-import org.apache.hadoop.io.file.tfile.DTFile.Reader.Scanner;
-import org.junit.Assert;
-
-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 Random random = new Random();
-
- 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.createScannerByByteRange(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();
- }
-
- /* Similar to readFile(), tests the scanner created
- * by record numbers rather than the offsets.
- */
- void readRowSplits(int numSplits) throws IOException {
-
- Reader reader =
- new Reader(fs.open(path), fs.getFileStatus(path).getLen(), conf);
-
- long totalRecords = reader.getEntryCount();
- for (int i=0; i<numSplits; i++) {
- long startRec = i*totalRecords/numSplits;
- long endRec = (i+1)*totalRecords/numSplits;
- if (i == numSplits-1) {
- endRec = totalRecords;
- }
- Scanner scanner = reader.createScannerByRecordNum(startRec, endRec);
- int count = 0;
- BytesWritable key = new BytesWritable();
- BytesWritable value = new BytesWritable();
- long x=startRec;
- while (!scanner.atEnd()) {
- assertEquals("Incorrect RecNum returned by scanner", scanner.getRecordNum(), x);
- scanner.entry().get(key, value);
- ++count;
- assertEquals("Incorrect RecNum returned by scanner", scanner.getRecordNum(), x);
- scanner.advance();
- ++x;
- }
- scanner.close();
- Assert.assertTrue(count == (endRec - startRec));
- }
- // make sure specifying range at the end gives zero records.
- Scanner scanner = reader.createScannerByRecordNum(totalRecords, -1);
- Assert.assertTrue(scanner.atEnd());
- }
-
- static String composeSortedKey(String prefix, int total, int value) {
- return String.format("%s%010d", prefix, value);
- }
-
- void checkRecNums() throws IOException {
- long fileLen = fs.getFileStatus(path).getLen();
- Reader reader = new Reader(fs.open(path), fileLen, conf);
- long totalRecs = reader.getEntryCount();
- long begin = random.nextLong() % (totalRecs / 2);
- if (begin < 0)
- begin += (totalRecs / 2);
- long end = random.nextLong() % (totalRecs / 2);
- if (end < 0)
- end += (totalRecs / 2);
- end += (totalRecs / 2) + 1;
-
- assertEquals("RecNum for offset=0 should be 0", 0, reader
- .getRecordNumNear(0));
- for (long x : new long[] { fileLen, fileLen + 1, 2 * fileLen }) {
- assertEquals("RecNum for offset>=fileLen should be total entries",
- totalRecs, reader.getRecordNumNear(x));
- }
-
- for (long i = 0; i < 100; ++i) {
- assertEquals("Locaton to RecNum conversion not symmetric", i, reader
- .getRecordNumByLocation(reader.getLocationByRecordNum(i)));
- }
-
- for (long i = 1; i < 100; ++i) {
- long x = totalRecs - i;
- assertEquals("Locaton to RecNum conversion not symmetric", x, reader
- .getRecordNumByLocation(reader.getLocationByRecordNum(x)));
- }
-
- for (long i = begin; i < end; ++i) {
- assertEquals("Locaton to RecNum conversion not symmetric", i, reader
- .getRecordNumByLocation(reader.getLocationByRecordNum(i)));
- }
-
- for (int i = 0; i < 1000; ++i) {
- long x = random.nextLong() % totalRecs;
- if (x < 0) x += totalRecs;
- assertEquals("Locaton to RecNum conversion not symmetric", x, reader
- .getRecordNumByLocation(reader.getLocationByRecordNum(x)));
- }
- }
-
- public void testSplit() throws IOException {
- System.out.println("testSplit");
- createFile(100000, Compression.Algorithm.NONE.getName());
- checkRecNums();
- readFile();
- readRowSplits(10);
- fs.delete(path, true);
- createFile(500000, Compression.Algorithm.GZ.getName());
- checkRecNums();
- readFile();
- readRowSplits(83);
- fs.delete(path, true);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-apex-malhar/blob/02f48e1b/contrib/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileStreams.java
----------------------------------------------------------------------
diff --git a/contrib/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileStreams.java b/contrib/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileStreams.java
deleted file mode 100644
index 2e0506c..0000000
--- a/contrib/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileStreams.java
+++ /dev/null
@@ -1,423 +0,0 @@
-/**
- * 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.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.DTFile.Reader;
-import org.apache.hadoop.io.file.tfile.DTFile.Writer;
-import org.apache.hadoop.io.file.tfile.DTFile.Reader.Scanner;
-import org.junit.Assert;
-
-/**
- *
- * 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 final String outputFile = getClass().getSimpleName();
-
- public void init(String compression, String comparator) {
- this.compression = compression;
- this.comparator = comparator;
- }
-
- @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();
- TestDTFileByteArrays.readRecords(fs, path, 0, conf);
- }
-
- public void testOneEntryKnownLength() throws IOException {
- if (skip)
- return;
- writeRecords(1, true, true);
-
- TestDTFileByteArrays.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
- TestDTFileByteArrays.readRecords(fs, path, 1, conf);
- }
-
- // known key length, unknown value length
- public void testOneEntryMixedLengths1() throws IOException {
- if (skip)
- return;
- writeRecords(1, true, false);
-
- TestDTFileByteArrays.readRecords(fs, path, 1, conf);
- }
-
- // unknown key length, known value length
- public void testOneEntryMixedLengths2() throws IOException {
- if (skip)
- return;
- writeRecords(1, false, true);
-
- TestDTFileByteArrays.readRecords(fs, path, 1, conf);
- }
-
- public void testTwoEntriesKnownLength() throws IOException {
- if (skip)
- return;
- writeRecords(2, true, true);
-
- TestDTFileByteArrays.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 = TestDTFileByteArrays.composeSortedKey("key", 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;
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-apex-malhar/blob/02f48e1b/contrib/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileUnsortedByteArrays.java
----------------------------------------------------------------------
diff --git a/contrib/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileUnsortedByteArrays.java b/contrib/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileUnsortedByteArrays.java
deleted file mode 100644
index a58f649..0000000
--- a/contrib/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileUnsortedByteArrays.java
+++ /dev/null
@@ -1,239 +0,0 @@
-/**
- * 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.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.DTFile.Reader;
-import org.apache.hadoop.io.file.tfile.DTFile.Writer;
-import org.apache.hadoop.io.file.tfile.DTFile.Reader.Scanner;
-import org.junit.Assert;
-
-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.createScannerByKey("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;
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-apex-malhar/blob/02f48e1b/library/src/main/java/org/apache/hadoop/io/file/tfile/CacheManager.java
----------------------------------------------------------------------
diff --git a/library/src/main/java/org/apache/hadoop/io/file/tfile/CacheManager.java b/library/src/main/java/org/apache/hadoop/io/file/tfile/CacheManager.java
new file mode 100644
index 0000000..2f47a76
--- /dev/null
+++ b/library/src/main/java/org/apache/hadoop/io/file/tfile/CacheManager.java
@@ -0,0 +1,185 @@
+/**
+ * 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.lang.management.ManagementFactory;
+import java.util.Collection;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.io.file.tfile.DTBCFile.Reader.BlockReader;
+import com.google.common.cache.Cache;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.Weigher;
+
+/**
+ * A single global managed cache
+ * User can limit the cache size by num of entries, memory size (bytes) or percentage of total heap size
+ * <br>
+ * <br>
+ * Please refer to <a href="https://code.google.com/p/guava-libraries/wiki/CachesExplained">Guava Cache</a> fir details
+ * <br>
+ * <br>
+ * It keeps {@link String} as key and {@link BlockReader} as value
+ *
+ * @since 2.0.0
+ */
+public class CacheManager
+{
+ public static final int STRING_OVERHEAD = 64;
+
+ public static final int BLOCK_READER_OVERHEAD = 368;
+
+ public static final float DEFAULT_HEAP_MEMORY_PERCENTAGE = 0.25f;
+
+ private static Cache<String, BlockReader> singleCache;
+
+ private static boolean enableStats = false;
+
+ public static final Cache<String, BlockReader> buildCache(CacheBuilder builder) {
+ if (singleCache != null) {
+ singleCache.cleanUp();
+ }
+ if (enableStats) {
+ //todo: when we upgrade to a newer guava version we can use this
+ // builder.recordStats();
+ }
+ singleCache = builder.build();
+ return singleCache;
+ }
+
+ /**
+ * (Re)Create the cache by limiting the maximum entries
+ * @param concurrencyLevel
+ * @param initialCapacity
+ * @param maximunSize
+ * @return The cache.
+ */
+ public static final Cache<String, BlockReader> createCache(int concurrencyLevel,int initialCapacity, int maximunSize){
+ CacheBuilder builder = CacheBuilder.newBuilder().
+ concurrencyLevel(concurrencyLevel).
+ initialCapacity(initialCapacity).
+ maximumSize(maximunSize);
+
+ return buildCache(builder);
+ }
+
+
+ /**
+ * (Re)Create the cache by limiting the memory(in bytes)
+ * @param concurrencyLevel
+ * @param initialCapacity
+ * @param maximumMemory
+ * @return The cache.
+ */
+ public static final Cache<String, BlockReader> createCache(int concurrencyLevel,int initialCapacity, long maximumMemory){
+
+ CacheBuilder builder = CacheBuilder.newBuilder().
+ concurrencyLevel(concurrencyLevel).
+ initialCapacity(initialCapacity).
+ maximumWeight(maximumMemory).weigher(new KVWeigher());
+
+ return buildCache(builder);
+ }
+
+ /**
+ * (Re)Create the cache by limiting percentage of the total heap memory
+ * @param concurrencyLevel
+ * @param initialCapacity
+ * @param heapMemPercentage
+ * @return The cache.
+ */
+ public static final Cache<String, BlockReader> createCache(int concurrencyLevel,int initialCapacity, float heapMemPercentage){
+ CacheBuilder builder = CacheBuilder.newBuilder().
+ concurrencyLevel(concurrencyLevel).
+ initialCapacity(initialCapacity).
+ maximumWeight((long) (ManagementFactory.getMemoryMXBean().getHeapMemoryUsage().getMax() * heapMemPercentage)).weigher(new KVWeigher());
+ return buildCache(builder);
+ }
+
+ public static final void createDefaultCache(){
+
+ long availableMemory = (long) (ManagementFactory.getMemoryMXBean().getHeapMemoryUsage().getMax() * DEFAULT_HEAP_MEMORY_PERCENTAGE);
+ CacheBuilder<String, BlockReader> builder = CacheBuilder.newBuilder().maximumWeight(availableMemory).weigher(new KVWeigher());
+
+ singleCache = buildCache(builder);
+ }
+
+ public static final void put(String key, BlockReader blk){
+ if (singleCache == null) {
+ createDefaultCache();
+ }
+ singleCache.put(key, blk);
+ }
+
+ public static final BlockReader get(String key){
+ if (singleCache == null) {
+ return null;
+ }
+ return singleCache.getIfPresent(key);
+ }
+
+ public static final void invalidateKeys(Collection<String> keys)
+ {
+ if (singleCache != null)
+ singleCache.invalidateAll(keys);
+ }
+
+ public static final long getCacheSize() {
+ if (singleCache != null)
+ return singleCache.size();
+ return 0;
+ }
+
+ public static final class KVWeigher implements Weigher<String, BlockReader> {
+
+ @Override
+ public int weigh(String key, BlockReader value)
+ {
+ return (STRING_OVERHEAD + BLOCK_READER_OVERHEAD) +
+ key.getBytes().length +
+ value.getBlockDataInputStream().getBuf().length;
+ }
+
+ }
+
+ @VisibleForTesting
+ protected static Cache<String, BlockReader> getCache() {
+ return singleCache;
+ }
+
+ public static final void setEnableStats(boolean enable) {
+ enableStats = enable;
+ }
+
+ public static void main(String[] args)
+ {
+
+ //code to eitsmate the overhead of the instance of the key value objects
+ // it depends on hbase file
+// System.out.println(ClassSize.estimateBase(BlockReader.class, true) +
+// ClassSize.estimateBase(Algorithm.class, true) +
+// ClassSize.estimateBase(RBlockState.class, true) +
+// ClassSize.estimateBase(ReusableByteArrayInputStream.class, true) +
+// ClassSize.estimateBase(BlockRegion.class, true));
+//
+// System.out.println(
+// ClassSize.estimateBase(String.class, true));
+ }
+
+}
[8/8] incubator-apex-malhar git commit: MLHR-1877 #resolve #comment
moved DTFile implementation to from contrib to lib
Posted by hs...@apache.org.
MLHR-1877 #resolve #comment moved DTFile implementation to from contrib to lib
Project: http://git-wip-us.apache.org/repos/asf/incubator-apex-malhar/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-apex-malhar/commit/02f48e1b
Tree: http://git-wip-us.apache.org/repos/asf/incubator-apex-malhar/tree/02f48e1b
Diff: http://git-wip-us.apache.org/repos/asf/incubator-apex-malhar/diff/02f48e1b
Branch: refs/heads/devel-3
Commit: 02f48e1b295920cb3c26b84eb802cfe18e3f9ea4
Parents: c1ebde9
Author: Chandni Singh <cs...@apache.org>
Authored: Fri Nov 6 16:40:12 2015 -0800
Committer: Chandni Singh <cs...@apache.org>
Committed: Fri Nov 6 16:41:11 2015 -0800
----------------------------------------------------------------------
.../hadoop/io/file/tfile/CacheManager.java | 183 --
.../apache/hadoop/io/file/tfile/DTBCFile.java | 1044 --------
.../org/apache/hadoop/io/file/tfile/DTFile.java | 2399 ------------------
.../tfile/ReusableByteArrayInputStream.java | 66 -
.../apache/hadoop/io/file/tfile/DTFileTest.java | 217 --
.../apache/hadoop/io/file/tfile/TestDTFile.java | 432 ----
.../io/file/tfile/TestDTFileByteArrays.java | 773 ------
.../io/file/tfile/TestTFileComparator2.java | 108 -
.../io/file/tfile/TestTFileComparators.java | 123 -
.../TestTFileJClassComparatorByteArrays.java | 59 -
.../tfile/TestTFileLzoCodecsByteArrays.java | 41 -
.../file/tfile/TestTFileLzoCodecsStreams.java | 39 -
.../tfile/TestTFileNoneCodecsByteArrays.java | 32 -
...ileNoneCodecsJClassComparatorByteArrays.java | 40 -
.../file/tfile/TestTFileNoneCodecsStreams.java | 32 -
.../hadoop/io/file/tfile/TestTFileSeek.java | 505 ----
.../file/tfile/TestTFileSeqFileComparison.java | 802 ------
.../hadoop/io/file/tfile/TestTFileSplit.java | 194 --
.../hadoop/io/file/tfile/TestTFileStreams.java | 423 ---
.../file/tfile/TestTFileUnsortedByteArrays.java | 239 --
.../hadoop/io/file/tfile/CacheManager.java | 185 ++
.../apache/hadoop/io/file/tfile/DTBCFile.java | 1044 ++++++++
.../org/apache/hadoop/io/file/tfile/DTFile.java | 2399 ++++++++++++++++++
.../tfile/ReusableByteArrayInputStream.java | 66 +
.../apache/hadoop/io/file/tfile/DTFileTest.java | 220 ++
.../apache/hadoop/io/file/tfile/TestDTFile.java | 432 ++++
.../io/file/tfile/TestDTFileByteArrays.java | 773 ++++++
.../io/file/tfile/TestTFileComparator2.java | 108 +
.../io/file/tfile/TestTFileComparators.java | 123 +
.../TestTFileJClassComparatorByteArrays.java | 59 +
.../tfile/TestTFileLzoCodecsByteArrays.java | 41 +
.../file/tfile/TestTFileLzoCodecsStreams.java | 39 +
.../tfile/TestTFileNoneCodecsByteArrays.java | 32 +
...ileNoneCodecsJClassComparatorByteArrays.java | 40 +
.../file/tfile/TestTFileNoneCodecsStreams.java | 32 +
.../hadoop/io/file/tfile/TestTFileSeek.java | 505 ++++
.../file/tfile/TestTFileSeqFileComparison.java | 802 ++++++
.../hadoop/io/file/tfile/TestTFileSplit.java | 194 ++
.../hadoop/io/file/tfile/TestTFileStreams.java | 423 +++
.../file/tfile/TestTFileUnsortedByteArrays.java | 239 ++
pom.xml | 1 +
41 files changed, 7757 insertions(+), 7751 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-apex-malhar/blob/02f48e1b/contrib/src/main/java/org/apache/hadoop/io/file/tfile/CacheManager.java
----------------------------------------------------------------------
diff --git a/contrib/src/main/java/org/apache/hadoop/io/file/tfile/CacheManager.java b/contrib/src/main/java/org/apache/hadoop/io/file/tfile/CacheManager.java
deleted file mode 100644
index 2c82d09..0000000
--- a/contrib/src/main/java/org/apache/hadoop/io/file/tfile/CacheManager.java
+++ /dev/null
@@ -1,183 +0,0 @@
-/**
- * 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.lang.management.ManagementFactory;
-import java.util.Collection;
-
-import com.google.common.annotations.VisibleForTesting;
-import org.apache.hadoop.io.file.tfile.DTBCFile.Reader.BlockReader;
-import com.google.common.cache.Cache;
-import com.google.common.cache.CacheBuilder;
-import com.google.common.cache.Weigher;
-
-/**
- * A single global managed cache
- * User can limit the cache size by num of entries, memory size (bytes) or percentage of total heap size
- * <br>
- * <br>
- * Please refer to <a href="https://code.google.com/p/guava-libraries/wiki/CachesExplained">Guava Cache</a> fir details
- * <br>
- * <br>
- * It keeps {@link String} as key and {@link BlockReader} as value
- *
- * @since 2.0.0
- */
-public class CacheManager
-{
- public static final int STRING_OVERHEAD = 64;
-
- public static final int BLOCK_READER_OVERHEAD = 368;
-
- public static final float DEFAULT_HEAP_MEMORY_PERCENTAGE = 0.25f;
-
- private static Cache<String, BlockReader> singleCache;
-
- private static boolean enableStats = false;
-
- public static final Cache<String, BlockReader> buildCache(CacheBuilder builder) {
- if (singleCache != null) {
- singleCache.cleanUp();
- }
- if (enableStats)
- builder.recordStats();
- singleCache = builder.build();
- return singleCache;
- }
-
- /**
- * (Re)Create the cache by limiting the maximum entries
- * @param concurrencyLevel
- * @param initialCapacity
- * @param maximunSize
- * @return The cache.
- */
- public static final Cache<String, BlockReader> createCache(int concurrencyLevel,int initialCapacity, int maximunSize){
- CacheBuilder builder = CacheBuilder.newBuilder().
- concurrencyLevel(concurrencyLevel).
- initialCapacity(initialCapacity).
- maximumSize(maximunSize);
-
- return buildCache(builder);
- }
-
-
- /**
- * (Re)Create the cache by limiting the memory(in bytes)
- * @param concurrencyLevel
- * @param initialCapacity
- * @param maximumMemory
- * @return The cache.
- */
- public static final Cache<String, BlockReader> createCache(int concurrencyLevel,int initialCapacity, long maximumMemory){
-
- CacheBuilder builder = CacheBuilder.newBuilder().
- concurrencyLevel(concurrencyLevel).
- initialCapacity(initialCapacity).
- maximumWeight(maximumMemory).weigher(new KVWeigher());
-
- return buildCache(builder);
- }
-
- /**
- * (Re)Create the cache by limiting percentage of the total heap memory
- * @param concurrencyLevel
- * @param initialCapacity
- * @param heapMemPercentage
- * @return The cache.
- */
- public static final Cache<String, BlockReader> createCache(int concurrencyLevel,int initialCapacity, float heapMemPercentage){
- CacheBuilder builder = CacheBuilder.newBuilder().
- concurrencyLevel(concurrencyLevel).
- initialCapacity(initialCapacity).
- maximumWeight((long) (ManagementFactory.getMemoryMXBean().getHeapMemoryUsage().getMax() * heapMemPercentage)).weigher(new KVWeigher());
- return buildCache(builder);
- }
-
- public static final void createDefaultCache(){
-
- long availableMemory = (long) (ManagementFactory.getMemoryMXBean().getHeapMemoryUsage().getMax() * DEFAULT_HEAP_MEMORY_PERCENTAGE);
- CacheBuilder<String, BlockReader> builder = CacheBuilder.newBuilder().maximumWeight(availableMemory).weigher(new KVWeigher());
-
- singleCache = buildCache(builder);
- }
-
- public static final void put(String key, BlockReader blk){
- if (singleCache == null) {
- createDefaultCache();
- }
- singleCache.put(key, blk);
- }
-
- public static final BlockReader get(String key){
- if (singleCache == null) {
- return null;
- }
- return singleCache.getIfPresent(key);
- }
-
- public static final void invalidateKeys(Collection<String> keys)
- {
- if (singleCache != null)
- singleCache.invalidateAll(keys);
- }
-
- public static final long getCacheSize() {
- if (singleCache != null)
- return singleCache.size();
- return 0;
- }
-
- public static final class KVWeigher implements Weigher<String, BlockReader> {
-
- @Override
- public int weigh(String key, BlockReader value)
- {
- return (STRING_OVERHEAD + BLOCK_READER_OVERHEAD) +
- key.getBytes().length +
- value.getBlockDataInputStream().getBuf().length;
- }
-
- }
-
- @VisibleForTesting
- protected static Cache<String, BlockReader> getCache() {
- return singleCache;
- }
-
- public static final void setEnableStats(boolean enable) {
- enableStats = enable;
- }
-
- public static void main(String[] args)
- {
-
- //code to eitsmate the overhead of the instance of the key value objects
- // it depends on hbase file
-// System.out.println(ClassSize.estimateBase(BlockReader.class, true) +
-// ClassSize.estimateBase(Algorithm.class, true) +
-// ClassSize.estimateBase(RBlockState.class, true) +
-// ClassSize.estimateBase(ReusableByteArrayInputStream.class, true) +
-// ClassSize.estimateBase(BlockRegion.class, true));
-//
-// System.out.println(
-// ClassSize.estimateBase(String.class, true));
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-apex-malhar/blob/02f48e1b/contrib/src/main/java/org/apache/hadoop/io/file/tfile/DTBCFile.java
----------------------------------------------------------------------
diff --git a/contrib/src/main/java/org/apache/hadoop/io/file/tfile/DTBCFile.java b/contrib/src/main/java/org/apache/hadoop/io/file/tfile/DTBCFile.java
deleted file mode 100644
index 779b0f0..0000000
--- a/contrib/src/main/java/org/apache/hadoop/io/file/tfile/DTBCFile.java
+++ /dev/null
@@ -1,1044 +0,0 @@
-/**
- * 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.commons.io.output.ByteArrayOutputStream;
-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.Utils.Version;
-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;
-
-
-/**
- *
- * <ul>
- * <li>The file format of DTFile is same as {@link TFile} with different reader implementation.
- * It reads data block by block and cache the binary block data into memory to speed up the random read.
- *
- * <li>The public api of {@link Reader} is as same as it is in {@link TFile} {@link org.apache.hadoop.io.file.tfile.TFile.Reader} implementation.
- * Besides, it provides getBlockBuffer(), getKeyOffset(), getKeyLength(), getValueOffset(), getValueLength() method
- * to expose raw block, key, value data to user to avoid unnecessary internal/external data copy
- *
- * <li>In the performance test, It shows no difference in sequential reads and 20x faster in random reads(If most of them hit memory)
- * </ul>
- *
- * 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.
- *
- * @since 2.0.0
- */
-final class DTBCFile {
- // 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(DTBCFile.class);
-
- /**
- * Prevent the instantiation of BCFile objects.
- */
- private DTBCFile() {
- // 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(DTFile.getFSOutputBufferSize(conf));
-
- this.fsBufferedOutput =
- new SimpleBufferedOutputStream(this.fsOut, fsOutputBuffer.getBytes());
- this.compressor = compressAlgo.getCompressor();
-
- try {
- this.out =
- compressionAlgo.createCompressionStream(fsBufferedOutput,
- compressor, 0);
- } catch (IOException e) {
- compressAlgo.returnCompressor(compressor);
- throw e;
- }
- }
-
- /**
- * Get the output stream for BlockAppender's consumption.
- *
- * @return the output stream suitable for writing block data.
- */
- OutputStream getOutputStream() {
- return out;
- }
-
- /**
- * Get the current position in file.
- *
- * @return The current byte offset in underlying file.
- * @throws IOException
- */
- long getCurrentPos() throws IOException {
- return fsOut.getPos() + fsBufferedOutput.size();
- }
-
- long getStartPos() {
- return posStart;
- }
-
- /**
- * Current size of compressed data.
- *
- * @return
- * @throws IOException
- */
- long getCompressedSize() throws IOException {
- long ret = getCurrentPos() - posStart;
- return ret;
- }
-
- /**
- * Finishing up the current block.
- */
- public void finish() throws IOException {
- try {
- if (out != null) {
- out.flush();
- out = null;
- }
- } finally {
- compressAlgo.returnCompressor(compressor);
- compressor = null;
- }
- }
- }
-
- /**
- * Access point to stuff data into a block.
- *
- * TODO: Change DataOutputStream to something else that tracks the size as
- * long instead of int. Currently, we will wrap around if the row block size
- * is greater than 4GB.
- */
- public class BlockAppender extends DataOutputStream {
- private final BlockRegister blockRegister;
- private final WBlockState wBlkState;
- @SuppressWarnings("hiding")
- private boolean closed = false;
-
- /**
- * Constructor
- *
- * @param register
- * the block register, which is called when the block is closed.
- * @param wbs
- * The writable compression block state.
- */
- BlockAppender(BlockRegister register, WBlockState wbs) {
- super(wbs.getOutputStream());
- this.blockRegister = register;
- this.wBlkState = wbs;
- }
-
- /**
- * Get the raw size of the block.
- *
- * @return the number of uncompressed bytes written through the
- * BlockAppender so far.
- * @throws IOException
- */
- public long getRawSize() throws IOException {
- /**
- * Expecting the size() of a block not exceeding 4GB. Assuming the
- * size() will wrap to negative integer if it exceeds 2GB.
- */
- return size() & 0x00000000ffffffffL;
- }
-
- /**
- * Get the compressed size of the block in progress.
- *
- * @return the number of compressed bytes written to the underlying FS
- * file. The size may be smaller than actual need to compress the
- * all data written due to internal buffering inside the
- * compressor.
- * @throws IOException
- */
- public long getCompressedSize() throws IOException {
- return wBlkState.getCompressedSize();
- }
-
- @Override
- public void flush() {
- // The down stream is a special kind of stream that finishes a
- // compression block upon flush. So we disable flush() here.
- }
-
- /**
- * Signaling the end of write to the block. The block register will be
- * called for registering the finished block.
- */
- @Override
- public void close() throws IOException {
- if (closed == true) {
- return;
- }
- try {
- ++errorCount;
- wBlkState.finish();
- blockRegister.register(getRawSize(), wBlkState.getStartPos(),
- wBlkState.getCurrentPos());
- --errorCount;
- } finally {
- closed = true;
- blkInProgress = false;
- }
- }
- }
-
- /**
- * Constructor
- *
- * @param fout
- * FS output stream.
- * @param compressionName
- * Name of the compression algorithm, which will be used for all
- * data blocks.
- * @throws IOException
- * @see Compression#getSupportedAlgorithms
- */
- public Writer(FSDataOutputStream fout, String compressionName,
- Configuration conf) throws IOException {
- if (fout.getPos() != 0) {
- throw new IOException("Output file not at zero offset.");
- }
-
- this.out = fout;
- this.conf = conf;
- dataIndex = new DataIndex(compressionName);
- metaIndex = new MetaIndex();
- fsOutputBuffer = new BytesWritable();
- Magic.write(fout);
- }
-
- /**
- * Close the BCFile Writer. Attempting to use the Writer after calling
- * <code>close</code> is not allowed and may lead to undetermined results.
- */
- @Override
- 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;
- }
-
- @Override
- 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
- }
-
- @Override
- 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;
- //
- private ByteArrayOutputStream baos;
- private ArrayList<String> cacheKeys;
-
- public ArrayList<String> getCacheKeys()
- {
- return cacheKeys;
- }
-
- /**
- * Intermediate class that maintain the state of a Readable Compression
- * Block.
- */
- static private final class RBlockState {
- private final Algorithm compressAlgo;
- private final ReusableByteArrayInputStream rbain;
- private final BlockRegion region;
-
- public RBlockState(Algorithm compressionAlgo, FSDataInputStream fsin, BlockRegion region, Configuration conf, Reader r) throws IOException
- {
- this.compressAlgo = compressionAlgo;
- Decompressor decompressor = compressionAlgo.getDecompressor();
- this.region = region;
- try {
-
- InputStream in = compressAlgo.createDecompressionStream(new BoundedRangeFileInputStream(fsin, region.getOffset(), region.getCompressedSize()), decompressor, DTFile.getFSInputBufferSize(conf));
- int l = 1;
- r.baos.reset();
- byte[] buf = new byte[DTFile.getFSInputBufferSize(conf)];
- while (l >= 0) {
- l = in.read(buf);
- if (l > 0) {
- r.baos.write(buf, 0, l);
- }
- }
- // keep decompressed data into cache
- byte[] blockData = r.baos.toByteArray();
- rbain = new ReusableByteArrayInputStream(blockData);
- } 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 ReusableByteArrayInputStream getInputStream() {
- return rbain;
- }
-
- public String getCompressionName() {
- return compressAlgo.getName();
- }
-
- public BlockRegion getBlockRegion() {
- return region;
- }
-
- public void finish() throws IOException {
- try {
- rbain.close();
- } finally {
- }
- }
-
- public void renew()
- {
- rbain.renew();
- }
- }
-
- /**
- * Access point to read a block.
- */
- public static class BlockReader extends DataInputStream {
- private final RBlockState rBlkState;
- private boolean closed = false;
-
- private ReusableByteArrayInputStream wrappedInputStream = null;
-
- BlockReader(RBlockState rbs) {
- super(rbs.getInputStream());
- rBlkState = rbs;
- wrappedInputStream = rbs.getInputStream();
- }
-
- /**
- * 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();
- }
-
- public void renew()
- {
- closed = false;
- rBlkState.renew();
- }
-
- public ReusableByteArrayInputStream getBlockDataInputStream()
- {
- return wrappedInputStream;
- }
- }
-
- /**
- * 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;
- // A reader buffer to read the block
- baos = new ByteArrayOutputStream(DTFile.getFSInputBufferSize(conf) * 2);
- this.cacheKeys = new ArrayList<String>();
- // 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(DTBCFile.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.
- */
- @Override
- public void close() {
- // Delete buffers in cache for this reader.
- CacheManager.invalidateKeys(cacheKeys);
- cacheKeys.clear();
- }
-
- /**
- * 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 {
- BlockReader br = (BlockReader) CacheManager.get(region.getOffset() + this.toString());
- if(br==null){
- RBlockState rbs = new RBlockState(compressAlgo, in, region, conf, this);
- br = new BlockReader(rbs);
- String cacheKey = region.getOffset() + this.toString();
- CacheManager.put(cacheKey, br);
- cacheKeys.add(cacheKey);
- } else {
- br.renew();
- }
- return br;
- }
-
- /**
- * Find the smallest Block index whose starting offset is greater than or
- * equal to the specified offset.
- *
- * @param offset
- * User-specific offset.
- * @return the index to the data Block if such block exists; or -1
- * otherwise.
- */
- public int getBlockIndexNear(long offset) {
- ArrayList<BlockRegion> list = dataIndex.getBlockRegionList();
- int idx =
- Utils
- .lowerBound(list, new ScalarLong(offset), new ScalarComparator());
-
- if (idx == list.size()) {
- return -1;
- }
-
- return idx;
- }
- }
-
- /**
- * Index for all Meta blocks.
- */
- static class MetaIndex {
- // use a tree map, for getting a meta block entry by name
- final Map<String, MetaIndexEntry> index;
-
- // for write
- public MetaIndex() {
- index = new TreeMap<String, MetaIndexEntry>();
- }
-
- // for read, construct the map from the file
- public MetaIndex(DataInput in) throws IOException {
- int count = Utils.readVInt(in);
- index = new TreeMap<String, MetaIndexEntry>();
-
- for (int nx = 0; nx < count; nx++) {
- MetaIndexEntry indexEntry = new MetaIndexEntry(in);
- index.put(indexEntry.getMetaName(), indexEntry);
- }
- }
-
- public void addEntry(MetaIndexEntry indexEntry) {
- index.put(indexEntry.getMetaName(), indexEntry);
- }
-
- public MetaIndexEntry getMetaByName(String name) {
- return index.get(name);
- }
-
- public void write(DataOutput out) throws IOException {
- Utils.writeVInt(out, index.size());
-
- for (MetaIndexEntry indexEntry : index.values()) {
- indexEntry.write(out);
- }
- }
- }
-
- /**
- * An entry describes a meta block in the MetaIndex.
- */
- static final class MetaIndexEntry {
- private final String metaName;
- private final Algorithm compressionAlgorithm;
- private final static String defaultPrefix = "data:";
-
- private final BlockRegion region;
-
- public MetaIndexEntry(DataInput in) throws IOException {
- String fullMetaName = Utils.readString(in);
- if (fullMetaName.startsWith(defaultPrefix)) {
- metaName =
- fullMetaName.substring(defaultPrefix.length(), fullMetaName
- .length());
- } else {
- throw new IOException("Corrupted Meta region Index");
- }
-
- compressionAlgorithm =
- Compression.getCompressionAlgorithmByName(Utils.readString(in));
- region = new BlockRegion(in);
- }
-
- public MetaIndexEntry(String metaName, Algorithm compressionAlgorithm,
- BlockRegion region) {
- this.metaName = metaName;
- this.compressionAlgorithm = compressionAlgorithm;
- this.region = region;
- }
-
- public String getMetaName() {
- return metaName;
- }
-
- public Algorithm getCompressionAlgorithm() {
- return compressionAlgorithm;
- }
-
- public BlockRegion getRegion() {
- return region;
- }
-
- public void write(DataOutput out) throws IOException {
- Utils.writeString(out, defaultPrefix + metaName);
- Utils.writeString(out, compressionAlgorithm.getName());
-
- region.write(out);
- }
- }
-
- /**
- * Index of all compressed data blocks.
- */
- static class DataIndex {
- final static String BLOCK_NAME = "BCFile.index";
-
- private final Algorithm defaultCompressionAlgorithm;
-
- // for data blocks, each entry specifies a block's offset, compressed size
- // and raw size
- private final ArrayList<BlockRegion> listRegions;
-
- // for read, deserialized from a file
- public DataIndex(DataInput in) throws IOException {
- defaultCompressionAlgorithm =
- Compression.getCompressionAlgorithmByName(Utils.readString(in));
-
- int n = Utils.readVInt(in);
- listRegions = new ArrayList<BlockRegion>(n);
-
- for (int i = 0; i < n; i++) {
- BlockRegion region = new BlockRegion(in);
- listRegions.add(region);
- }
- }
-
- // for write
- public DataIndex(String defaultCompressionAlgorithmName) {
- this.defaultCompressionAlgorithm =
- Compression
- .getCompressionAlgorithmByName(defaultCompressionAlgorithmName);
- listRegions = new ArrayList<BlockRegion>();
- }
-
- public Algorithm getDefaultCompressionAlgorithm() {
- return defaultCompressionAlgorithm;
- }
-
- public ArrayList<BlockRegion> getBlockRegionList() {
- return listRegions;
- }
-
- public void addBlockRegion(BlockRegion region) {
- listRegions.add(region);
- }
-
- public void write(DataOutput out) throws IOException {
- Utils.writeString(out, defaultCompressionAlgorithm.getName());
-
- Utils.writeVInt(out, listRegions.size());
-
- for (BlockRegion region : listRegions) {
- region.write(out);
- }
- }
- }
-
- /**
- * Magic number uniquely identifying a BCFile in the header/footer.
- */
- static final class Magic {
- private final static byte[] AB_MAGIC_BCFILE =
- {
- // ... total of 16 bytes
- (byte) 0xd1, (byte) 0x11, (byte) 0xd3, (byte) 0x68, (byte) 0x91,
- (byte) 0xb5, (byte) 0xd7, (byte) 0xb6, (byte) 0x39, (byte) 0xdf,
- (byte) 0x41, (byte) 0x40, (byte) 0x92, (byte) 0xba, (byte) 0xe1,
- (byte) 0x50 };
-
- public static void readAndVerify(DataInput in) throws IOException {
- byte[] abMagic = new byte[size()];
- in.readFully(abMagic);
-
- // check against AB_MAGIC_BCFILE, if not matching, throw an
- // Exception
- if (!Arrays.equals(abMagic, AB_MAGIC_BCFILE)) {
- throw new IOException("Not a valid BCFile.");
- }
- }
-
- public static void write(DataOutput out) throws IOException {
- out.write(AB_MAGIC_BCFILE);
- }
-
- public static int size() {
- return AB_MAGIC_BCFILE.length;
- }
- }
-
- /**
- * Block region.
- */
- static final class BlockRegion implements Scalar {
- private final long offset;
- private final long compressedSize;
- private final long rawSize;
-
- public BlockRegion(DataInput in) throws IOException {
- offset = Utils.readVLong(in);
- compressedSize = Utils.readVLong(in);
- rawSize = Utils.readVLong(in);
- }
-
- public BlockRegion(long offset, long compressedSize, long rawSize) {
- this.offset = offset;
- this.compressedSize = compressedSize;
- this.rawSize = rawSize;
- }
-
- public void write(DataOutput out) throws IOException {
- Utils.writeVLong(out, offset);
- Utils.writeVLong(out, compressedSize);
- Utils.writeVLong(out, rawSize);
- }
-
- public long getOffset() {
- return offset;
- }
-
- public long getCompressedSize() {
- return compressedSize;
- }
-
- public long getRawSize() {
- return rawSize;
- }
-
- @Override
- public long magnitude() {
- return offset;
- }
- }
-}