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 &lt;Key, Value&gt; 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 &lt;Key, Value&gt; 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;
-    }
-  }
-}