You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by om...@apache.org on 2016/05/20 21:22:39 UTC

[01/27] hive git commit: HIVE-11417. Move the ReaderImpl and RowReaderImpl to the ORC module, by making shims for the row by row reader. (omalley reviewed by prasanth_j)

Repository: hive
Updated Branches:
  refs/heads/master fd06601eb -> ffb79509b


http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorUtils.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorUtils.java b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorUtils.java
index 932ae0b..6415bf8 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorUtils.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorUtils.java
@@ -27,6 +27,7 @@ import java.sql.Timestamp;
 import java.util.HashMap;
 import java.util.Map;
 
+import org.apache.hadoop.hive.ql.util.TimestampUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hive.common.type.HiveChar;
@@ -1088,13 +1089,13 @@ public final class PrimitiveObjectInspectorUtils {
       result = TimestampWritable.longToTimestamp(longValue, intToTimestampInSeconds);
       break;
     case FLOAT:
-      result = TimestampWritable.doubleToTimestamp(((FloatObjectInspector) inputOI).get(o));
+      result = TimestampUtils.doubleToTimestamp(((FloatObjectInspector) inputOI).get(o));
       break;
     case DOUBLE:
-      result = TimestampWritable.doubleToTimestamp(((DoubleObjectInspector) inputOI).get(o));
+      result = TimestampUtils.doubleToTimestamp(((DoubleObjectInspector) inputOI).get(o));
       break;
     case DECIMAL:
-      result = TimestampWritable.decimalToTimestamp(((HiveDecimalObjectInspector) inputOI)
+      result = TimestampUtils.decimalToTimestamp(((HiveDecimalObjectInspector) inputOI)
                                                     .getPrimitiveJavaObject(o));
       break;
     case STRING:

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/serde/src/test/org/apache/hadoop/hive/serde2/io/TestTimestampWritable.java
----------------------------------------------------------------------
diff --git a/serde/src/test/org/apache/hadoop/hive/serde2/io/TestTimestampWritable.java b/serde/src/test/org/apache/hadoop/hive/serde2/io/TestTimestampWritable.java
index 6c763bc..7619efa 100644
--- a/serde/src/test/org/apache/hadoop/hive/serde2/io/TestTimestampWritable.java
+++ b/serde/src/test/org/apache/hadoop/hive/serde2/io/TestTimestampWritable.java
@@ -35,6 +35,7 @@ import java.util.List;
 import java.util.Random;
 import java.util.TimeZone;
 
+import org.apache.hadoop.hive.ql.util.TimestampUtils;
 import org.junit.*;
 import static org.junit.Assert.*;
 
@@ -70,7 +71,7 @@ public class TestTimestampWritable {
     long seconds = (ts.getTime() - ts.getNanos() / 1000000) / 1000;
 
     // It should also be possible to calculate this based on ts.getTime() only.
-    assertEquals(seconds, TimestampWritable.millisToSeconds(ts.getTime()));
+    assertEquals(seconds, TimestampUtils.millisToSeconds(ts.getTime()));
 
     return seconds;
   }
@@ -335,10 +336,10 @@ public class TestTimestampWritable {
             Math.pow(10, 9 - nanosPrecision));
         assertEquals(String.format("Invalid nanosecond part recovered from %f", asDouble),
           nanos, recoveredNanos);
-        assertEquals(ts, TimestampWritable.doubleToTimestamp(asDouble));
+        assertEquals(ts, TimestampUtils.doubleToTimestamp(asDouble));
         // decimalToTimestamp should be consistent with doubleToTimestamp for this level of
         // precision.
-        assertEquals(ts, TimestampWritable.decimalToTimestamp(
+        assertEquals(ts, TimestampUtils.decimalToTimestamp(
             HiveDecimal.create(BigDecimal.valueOf(asDouble))));
       }
     }
@@ -358,7 +359,7 @@ public class TestTimestampWritable {
       Timestamp ts = new Timestamp(
           randomMillis(MIN_FOUR_DIGIT_YEAR_MILLIS, MAX_FOUR_DIGIT_YEAR_MILLIS, rand));
       ts.setNanos(randomNanos(rand, 9));  // full precision
-      assertEquals(ts, TimestampWritable.decimalToTimestamp(timestampToDecimal(ts)));
+      assertEquals(ts, TimestampUtils.decimalToTimestamp(timestampToDecimal(ts)));
     }
   }
 
@@ -371,8 +372,8 @@ public class TestTimestampWritable {
     for (int nanos : new int[] { 100000, 900000, 999100000, 999900000 }) {
       ts.setNanos(nanos);
       HiveDecimal d = timestampToDecimal(ts);
-      assertEquals(ts, TimestampWritable.decimalToTimestamp(d));
-      assertEquals(ts, TimestampWritable.doubleToTimestamp(d.bigDecimalValue().doubleValue()));
+      assertEquals(ts, TimestampUtils.decimalToTimestamp(d));
+      assertEquals(ts, TimestampUtils.doubleToTimestamp(d.bigDecimalValue().doubleValue()));
     }
   }
 
@@ -435,20 +436,20 @@ public class TestTimestampWritable {
   @Concurrent(count=4)
   @Repeating(repetition=100)
   public void testMillisToSeconds() {
-    assertEquals(0, TimestampWritable.millisToSeconds(0));
-    assertEquals(-1, TimestampWritable.millisToSeconds(-1));
-    assertEquals(-1, TimestampWritable.millisToSeconds(-999));
-    assertEquals(-1, TimestampWritable.millisToSeconds(-1000));
-    assertEquals(-2, TimestampWritable.millisToSeconds(-1001));
-    assertEquals(-2, TimestampWritable.millisToSeconds(-1999));
-    assertEquals(-2, TimestampWritable.millisToSeconds(-2000));
-    assertEquals(-3, TimestampWritable.millisToSeconds(-2001));
-    assertEquals(-99, TimestampWritable.millisToSeconds(-99000));
-    assertEquals(-100, TimestampWritable.millisToSeconds(-99001));
-    assertEquals(-100, TimestampWritable.millisToSeconds(-100000));
-    assertEquals(1, TimestampWritable.millisToSeconds(1500));
-    assertEquals(19, TimestampWritable.millisToSeconds(19999));
-    assertEquals(20, TimestampWritable.millisToSeconds(20000));
+    assertEquals(0, TimestampUtils.millisToSeconds(0));
+    assertEquals(-1, TimestampUtils.millisToSeconds(-1));
+    assertEquals(-1, TimestampUtils.millisToSeconds(-999));
+    assertEquals(-1, TimestampUtils.millisToSeconds(-1000));
+    assertEquals(-2, TimestampUtils.millisToSeconds(-1001));
+    assertEquals(-2, TimestampUtils .millisToSeconds(-1999));
+    assertEquals(-2, TimestampUtils .millisToSeconds(-2000));
+    assertEquals(-3, TimestampUtils .millisToSeconds(-2001));
+    assertEquals(-99, TimestampUtils .millisToSeconds(-99000));
+    assertEquals(-100, TimestampUtils .millisToSeconds(-99001));
+    assertEquals(-100, TimestampUtils .millisToSeconds(-100000));
+    assertEquals(1, TimestampUtils .millisToSeconds(1500));
+    assertEquals(19, TimestampUtils .millisToSeconds(19999));
+    assertEquals(20, TimestampUtils .millisToSeconds(20000));
   }
 
   private static int compareEqualLengthByteArrays(byte[] a, byte[] b) {

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java
----------------------------------------------------------------------
diff --git a/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java b/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java
index 2b7c747..ef2b7f7 100644
--- a/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java
+++ b/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java
@@ -102,42 +102,18 @@ import org.apache.tez.test.MiniTezCluster;
 public class Hadoop23Shims extends HadoopShimsSecure {
 
   HadoopShims.MiniDFSShim cluster = null;
-  final boolean zeroCopy;
   final boolean storagePolicy;
-  final boolean fastread;
 
   public Hadoop23Shims() {
-    boolean zcr = false;
+    // in-memory HDFS
     boolean storage = false;
-    boolean fastread = false;
     try {
-      Class.forName("org.apache.hadoop.fs.CacheFlag", false,
-          ShimLoader.class.getClassLoader());
-      zcr = true;
-    } catch (ClassNotFoundException ce) {
-    }
-
-    if (zcr) {
-      // in-memory HDFS is only available after zcr
-      try {
-        Class.forName("org.apache.hadoop.hdfs.protocol.BlockStoragePolicy",
+      Class.forName("org.apache.hadoop.hdfs.protocol.BlockStoragePolicy",
             false, ShimLoader.class.getClassLoader());
-        storage = true;
-      } catch (ClassNotFoundException ce) {
-      }
-    }
-
-    if (storage) {
-      for (Method m : Text.class.getMethods()) {
-        if ("readWithKnownLength".equals(m.getName())) {
-          fastread = true;
-        }
-      }
+      storage = true;
+    } catch (ClassNotFoundException ce) {
     }
-
     this.storagePolicy = storage;
-    this.zeroCopy = zcr;
-    this.fastread = fastread;
   }
 
   @Override
@@ -854,15 +830,6 @@ public class Hadoop23Shims extends HadoopShimsSecure {
   }
 
   @Override
-  public ZeroCopyReaderShim getZeroCopyReader(FSDataInputStream in, ByteBufferPoolShim pool) throws IOException {
-    if(zeroCopy) {
-      return ZeroCopyShims.getZeroCopyReader(in, pool);
-    }
-    /* not supported */
-    return null;
-  }
-
-  @Override
   public Configuration getConfiguration(org.apache.hadoop.mapreduce.JobContext context) {
     return context.getConfiguration();
   }
@@ -1302,26 +1269,4 @@ public class Hadoop23Shims extends HadoopShimsSecure {
   public long getFileId(FileSystem fs, String path) throws IOException {
     return ensureDfs(fs).getClient().getFileInfo(path).getFileId();
   }
-
-  private final class FastTextReaderShim implements TextReaderShim {
-    private final DataInputStream din;
-
-    public FastTextReaderShim(InputStream in) {
-      this.din = new DataInputStream(in);
-    }
-
-    @Override
-    public void read(Text txt, int len) throws IOException {
-      txt.readWithKnownLength(din, len);
-    }
-  }
-
-  @Override
-  public TextReaderShim getTextReaderShim(InputStream in) throws IOException {
-    if (!fastread) {
-      return super.getTextReaderShim(in);
-    }
-    return new FastTextReaderShim(in);
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/ZeroCopyShims.java
----------------------------------------------------------------------
diff --git a/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/ZeroCopyShims.java b/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/ZeroCopyShims.java
deleted file mode 100644
index 6ef0467..0000000
--- a/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/ZeroCopyShims.java
+++ /dev/null
@@ -1,86 +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.hive.shims;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.EnumSet;
-
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.ReadOption;
-import org.apache.hadoop.io.ByteBufferPool;
-
-import org.apache.hadoop.hive.shims.HadoopShims.ByteBufferPoolShim;
-import org.apache.hadoop.hive.shims.HadoopShims.ZeroCopyReaderShim;
-
-class ZeroCopyShims {
-  private static final class ByteBufferPoolAdapter implements ByteBufferPool {
-    private ByteBufferPoolShim pool;
-
-    public ByteBufferPoolAdapter(ByteBufferPoolShim pool) {
-      this.pool = pool;
-    }
-
-    @Override
-    public final ByteBuffer getBuffer(boolean direct, int length) {
-      return this.pool.getBuffer(direct, length);
-    }
-
-    @Override
-    public final void putBuffer(ByteBuffer buffer) {
-      this.pool.putBuffer(buffer);
-    }
-  }
-
-  private static final class ZeroCopyAdapter implements ZeroCopyReaderShim {
-    private final FSDataInputStream in;
-    private final ByteBufferPoolAdapter pool;
-    private final static EnumSet<ReadOption> CHECK_SUM = EnumSet
-        .noneOf(ReadOption.class);
-    private final static EnumSet<ReadOption> NO_CHECK_SUM = EnumSet
-        .of(ReadOption.SKIP_CHECKSUMS);
-
-    public ZeroCopyAdapter(FSDataInputStream in, ByteBufferPoolShim poolshim) {
-      this.in = in;
-      if (poolshim != null) {
-        pool = new ByteBufferPoolAdapter(poolshim);
-      } else {
-        pool = null;
-      }
-    }
-
-    public final ByteBuffer readBuffer(int maxLength, boolean verifyChecksums)
-        throws IOException {
-      EnumSet<ReadOption> options = NO_CHECK_SUM;
-      if (verifyChecksums) {
-        options = CHECK_SUM;
-      }
-      return this.in.read(this.pool, maxLength, options);
-    }
-
-    public final void releaseBuffer(ByteBuffer buffer) {
-      this.in.releaseBuffer(buffer);
-    }
-  }
-
-  public static ZeroCopyReaderShim getZeroCopyReader(FSDataInputStream in,
-      ByteBufferPoolShim pool) throws IOException {
-    return new ZeroCopyAdapter(in, pool);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/shims/common/src/main/java/org/apache/hadoop/hive/shims/HadoopShims.java
----------------------------------------------------------------------
diff --git a/shims/common/src/main/java/org/apache/hadoop/hive/shims/HadoopShims.java b/shims/common/src/main/java/org/apache/hadoop/hive/shims/HadoopShims.java
index 37eb8f6..4a96355 100644
--- a/shims/common/src/main/java/org/apache/hadoop/hive/shims/HadoopShims.java
+++ b/shims/common/src/main/java/org/apache/hadoop/hive/shims/HadoopShims.java
@@ -403,57 +403,6 @@ public interface HadoopShims {
   public StoragePolicyShim getStoragePolicyShim(FileSystem fs);
 
   /**
-   * a hadoop.io ByteBufferPool shim.
-   */
-  public interface ByteBufferPoolShim {
-    /**
-     * Get a new ByteBuffer from the pool.  The pool can provide this from
-     * removing a buffer from its internal cache, or by allocating a
-     * new buffer.
-     *
-     * @param direct     Whether the buffer should be direct.
-     * @param length     The minimum length the buffer will have.
-     * @return           A new ByteBuffer. Its capacity can be less
-     *                   than what was requested, but must be at
-     *                   least 1 byte.
-     */
-    ByteBuffer getBuffer(boolean direct, int length);
-
-    /**
-     * Release a buffer back to the pool.
-     * The pool may choose to put this buffer into its cache/free it.
-     *
-     * @param buffer    a direct bytebuffer
-     */
-    void putBuffer(ByteBuffer buffer);
-  }
-
-  /**
-   * Provides an HDFS ZeroCopyReader shim.
-   * @param in FSDataInputStream to read from (where the cached/mmap buffers are tied to)
-   * @param in ByteBufferPoolShim to allocate fallback buffers with
-   *
-   * @return returns null if not supported
-   */
-  public ZeroCopyReaderShim getZeroCopyReader(FSDataInputStream in, ByteBufferPoolShim pool) throws IOException;
-
-  public interface ZeroCopyReaderShim {
-    /**
-     * Get a ByteBuffer from the FSDataInputStream - this can be either a HeapByteBuffer or an MappedByteBuffer.
-     * Also move the in stream by that amount. The data read can be small than maxLength.
-     *
-     * @return ByteBuffer read from the stream,
-     */
-    public ByteBuffer readBuffer(int maxLength, boolean verifyChecksums) throws IOException;
-    /**
-     * Release a ByteBuffer obtained from a read on the
-     * Also move the in stream by that amount. The data read can be small than maxLength.
-     *
-     */
-    public void releaseBuffer(ByteBuffer buffer);
-  }
-
-  /**
    * Get configuration from JobContext
    */
   public Configuration getConfiguration(JobContext context);
@@ -692,23 +641,4 @@ public interface HadoopShims {
    */
   long getFileId(FileSystem fs, String path) throws IOException;
 
-  /**
-   * Read data into a Text object in the fastest way possible
-   */
-  public interface TextReaderShim {
-    /**
-     * @param txt
-     * @param len
-     * @return bytes read
-     * @throws IOException
-     */
-    void read(Text txt, int size) throws IOException;
-  }
-
-  /**
-   * Wrap a TextReaderShim around an input stream. The reader shim will not
-   * buffer any reads from the underlying stream and will only consume bytes
-   * which are required for TextReaderShim.read() input.
-   */
-  public TextReaderShim getTextReaderShim(InputStream input) throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/shims/common/src/main/java/org/apache/hadoop/hive/shims/HadoopShimsSecure.java
----------------------------------------------------------------------
diff --git a/shims/common/src/main/java/org/apache/hadoop/hive/shims/HadoopShimsSecure.java b/shims/common/src/main/java/org/apache/hadoop/hive/shims/HadoopShimsSecure.java
index 87682e6..224ce3b 100644
--- a/shims/common/src/main/java/org/apache/hadoop/hive/shims/HadoopShimsSecure.java
+++ b/shims/common/src/main/java/org/apache/hadoop/hive/shims/HadoopShimsSecure.java
@@ -392,33 +392,4 @@ public abstract class HadoopShimsSecure implements HadoopShims {
 
   @Override
   abstract public void addDelegationTokens(FileSystem fs, Credentials cred, String uname) throws IOException;
-
-  private final class BasicTextReaderShim implements TextReaderShim {
-    private final InputStream in;
-
-    public BasicTextReaderShim(InputStream in) {
-      this.in = in;
-    }
-
-    @Override
-    public void read(Text txt, int len) throws IOException {
-      int offset = 0;
-      byte[] bytes = new byte[len];
-      while (len > 0) {
-        int written = in.read(bytes, offset, len);
-        if (written < 0) {
-          throw new EOFException("Can't finish read from " + in + " read "
-              + (offset) + " bytes out of " + bytes.length);
-        }
-        len -= written;
-        offset += written;
-      }
-      txt.set(bytes);
-    }
-  }
-
-  @Override
-  public TextReaderShim getTextReaderShim(InputStream in) throws IOException {
-    return new BasicTextReaderShim(in);
-  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/TimestampColumnVector.java
----------------------------------------------------------------------
diff --git a/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/TimestampColumnVector.java b/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/TimestampColumnVector.java
index d971339..228461a 100644
--- a/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/TimestampColumnVector.java
+++ b/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/TimestampColumnVector.java
@@ -281,8 +281,13 @@ public class TimestampColumnVector extends ColumnVector {
    * @param timestamp
    */
   public void set(int elementNum, Timestamp timestamp) {
-    this.time[elementNum] = timestamp.getTime();
-    this.nanos[elementNum] = timestamp.getNanos();
+    if (timestamp == null) {
+      this.noNulls = false;
+      this.isNull[elementNum] = true;
+    } else {
+      this.time[elementNum] = timestamp.getTime();
+      this.nanos[elementNum] = timestamp.getNanos();
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringExpr.java
----------------------------------------------------------------------
diff --git a/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringExpr.java b/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringExpr.java
new file mode 100644
index 0000000..90817a5
--- /dev/null
+++ b/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringExpr.java
@@ -0,0 +1,354 @@
+/**
+ * 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.hive.ql.exec.vector.expressions;
+
+import java.util.Arrays;
+
+import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
+
+/**
+ * String expression evaluation helper functions.
+ */
+public class StringExpr {
+
+  /* Compare two strings from two byte arrays each
+   * with their own start position and length.
+   * Use lexicographic unsigned byte value order.
+   * This is what's used for UTF-8 sort order.
+   * Return negative value if arg1 < arg2, 0 if arg1 = arg2,
+   * positive if arg1 > arg2.
+   */
+  public static int compare(byte[] arg1, int start1, int len1, byte[] arg2, int start2, int len2) {
+    for (int i = 0; i < len1 && i < len2; i++) {
+      // Note the "& 0xff" is just a way to convert unsigned bytes to signed integer.
+      int b1 = arg1[i + start1] & 0xff;
+      int b2 = arg2[i + start2] & 0xff;
+      if (b1 != b2) {
+        return b1 - b2;
+      }
+    }
+    return len1 - len2;
+  }
+
+  /* Determine if two strings are equal from two byte arrays each
+   * with their own start position and length.
+   * Use lexicographic unsigned byte value order.
+   * This is what's used for UTF-8 sort order.
+   */
+  public static boolean equal(byte[] arg1, final int start1, final int len1,
+      byte[] arg2, final int start2, final int len2) {
+    if (len1 != len2) {
+      return false;
+    }
+    if (len1 == 0) {
+      return true;
+    }
+
+    // do bounds check for OOB exception
+    if (arg1[start1] != arg2[start2]
+        || arg1[start1 + len1 - 1] != arg2[start2 + len2 - 1]) {
+      return false;
+    }
+
+    if (len1 == len2) {
+      // prove invariant to the compiler: len1 = len2
+      // all array access between (start1, start1+len1) 
+      // and (start2, start2+len2) are valid
+      // no more OOB exceptions are possible
+      final int step = 8;
+      final int remainder = len1 % step;
+      final int wlen = len1 - remainder;
+      // suffix first
+      for (int i = wlen; i < len1; i++) {
+        if (arg1[start1 + i] != arg2[start2 + i]) {
+          return false;
+        }
+      }
+      // SIMD loop
+      for (int i = 0; i < wlen; i += step) {
+        final int s1 = start1 + i;
+        final int s2 = start2 + i;
+        boolean neq = false;
+        for (int j = 0; j < step; j++) {
+          neq = (arg1[s1 + j] != arg2[s2 + j]) || neq;
+        }
+        if (neq) {
+          return false;
+        }
+      }
+    }
+
+    return true;
+  }
+
+  public static int characterCount(byte[] bytes) {
+    int end = bytes.length;
+
+    // count characters
+    int j = 0;
+    int charCount = 0;
+    while(j < end) {
+      // UTF-8 continuation bytes have 2 high bits equal to 0x80.
+      if ((bytes[j] & 0xc0) != 0x80) {
+        ++charCount;
+      }
+      j++;
+    }
+    return charCount;
+  }
+
+  public static int characterCount(byte[] bytes, int start, int length) {
+    int end = start + length;
+
+    // count characters
+    int j = start;
+    int charCount = 0;
+    while(j < end) {
+      // UTF-8 continuation bytes have 2 high bits equal to 0x80.
+      if ((bytes[j] & 0xc0) != 0x80) {
+        ++charCount;
+      }
+      j++;
+    }
+    return charCount;
+  }
+
+  // A setVal with the same function signature as rightTrim, leftTrim, truncate, etc, below.
+  // Useful for class generation via templates.
+  public static void assign(BytesColumnVector outV, int i, byte[] bytes, int start, int length) {
+    // set output vector
+    outV.setVal(i, bytes, start, length);
+  }
+
+  /*
+   * Right trim a slice of a byte array and return the new byte length.
+   */
+  public static int rightTrim(byte[] bytes, int start, int length) {
+    // skip trailing blank characters
+    int j = start + length - 1;
+    while(j >= start && bytes[j] == 0x20) {
+      j--;
+    }
+
+    return (j - start) + 1;
+  }
+
+  /*
+   * Right trim a slice of a byte array and place the result into element i of a vector.
+   */
+  public static void rightTrim(BytesColumnVector outV, int i, byte[] bytes, int start, int length) {
+    // skip trailing blank characters
+    int j = start + length - 1;
+    while(j >= start && bytes[j] == 0x20) {
+      j--;
+    }
+
+    // set output vector
+    outV.setVal(i, bytes, start, (j - start) + 1);
+  }
+
+  /*
+   * Truncate a slice of a byte array to a maximum number of characters and
+   * return the new byte length.
+   */
+  public static int truncate(byte[] bytes, int start, int length, int maxLength) {
+    int end = start + length;
+
+    // count characters forward
+    int j = start;
+    int charCount = 0;
+    while(j < end) {
+      // UTF-8 continuation bytes have 2 high bits equal to 0x80.
+      if ((bytes[j] & 0xc0) != 0x80) {
+        if (charCount == maxLength) {
+          break;
+        }
+        ++charCount;
+      }
+      j++;
+    }
+    return (j - start);
+  }
+
+  /*
+   * Truncate a slice of a byte array to a maximum number of characters and
+   * place the result into element i of a vector.
+   */
+  public static void truncate(BytesColumnVector outV, int i, byte[] bytes, int start, int length, int maxLength) {
+    int end = start + length;
+
+    // count characters forward
+    int j = start;
+    int charCount = 0;
+    while(j < end) {
+      // UTF-8 continuation bytes have 2 high bits equal to 0x80.
+      if ((bytes[j] & 0xc0) != 0x80) {
+        if (charCount == maxLength) {
+          break;
+        }
+        ++charCount;
+      }
+      j++;
+    }
+
+    // set output vector
+    outV.setVal(i, bytes, start, (j - start));
+  }
+
+  /*
+   * Truncate a byte array to a maximum number of characters and
+   * return a byte array with only truncated bytes.
+   */
+  public static byte[] truncateScalar(byte[] bytes, int maxLength) {
+    int end = bytes.length;
+
+    // count characters forward
+    int j = 0;
+    int charCount = 0;
+    while(j < end) {
+      // UTF-8 continuation bytes have 2 high bits equal to 0x80.
+      if ((bytes[j] & 0xc0) != 0x80) {
+        if (charCount == maxLength) {
+          break;
+        }
+        ++charCount;
+      }
+      j++;
+    }
+    if (j == end) {
+      return bytes;
+    } else {
+      return Arrays.copyOf(bytes, j);
+    }
+  }
+
+  /*
+   * Right trim and truncate a slice of a byte array to a maximum number of characters and
+   * return the new byte length.
+   */
+  public static int rightTrimAndTruncate(byte[] bytes, int start, int length, int maxLength) {
+    int end = start + length;
+
+    // count characters forward and watch for final run of pads
+    int j = start;
+    int charCount = 0;
+    int padRunStart = -1;
+    while(j < end) {
+      // UTF-8 continuation bytes have 2 high bits equal to 0x80.
+      if ((bytes[j] & 0xc0) != 0x80) {
+        if (charCount == maxLength) {
+          break;
+        }
+        if (bytes[j] == 0x20) {
+          if (padRunStart == -1) {
+            padRunStart = j;
+          }
+        } else {
+          padRunStart = -1;
+        }
+        ++charCount;
+      } else {
+        padRunStart = -1;
+      }
+      j++;
+    }
+    if (padRunStart != -1) {
+      return (padRunStart - start);
+    } else {
+      return (j - start);
+    }
+  }
+
+  /*
+   * Right trim and truncate a slice of a byte array to a maximum number of characters and
+   * place the result into element i of a vector.
+   */
+  public static void rightTrimAndTruncate(BytesColumnVector outV, int i, byte[] bytes, int start, int length, int maxLength) {
+    int end = start + length;
+
+    // count characters forward and watch for final run of pads
+    int j = start;
+    int charCount = 0;
+    int padRunStart = -1;
+    while(j < end) {
+      // UTF-8 continuation bytes have 2 high bits equal to 0x80.
+      if ((bytes[j] & 0xc0) != 0x80) {
+        if (charCount == maxLength) {
+          break;
+        }
+        if (bytes[j] == 0x20) {
+          if (padRunStart == -1) {
+            padRunStart = j;
+          }
+        } else {
+          padRunStart = -1;
+        }
+        ++charCount;
+      } else {
+        padRunStart = -1;
+      }
+      j++;
+    }
+    // set output vector
+    if (padRunStart != -1) {
+      outV.setVal(i, bytes, start, (padRunStart - start));
+    } else {
+      outV.setVal(i, bytes, start, (j - start) );
+    }
+  }
+
+  /*
+   * Right trim and truncate a byte array to a maximum number of characters and
+   * return a byte array with only the trimmed and truncated bytes.
+   */
+  public static byte[] rightTrimAndTruncateScalar(byte[] bytes, int maxLength) {
+    int end = bytes.length;
+
+    // count characters forward and watch for final run of pads
+    int j = 0;
+    int charCount = 0;
+    int padRunStart = -1;
+    while(j < end) {
+      // UTF-8 continuation bytes have 2 high bits equal to 0x80.
+      if ((bytes[j] & 0xc0) != 0x80) {
+        if (charCount == maxLength) {
+          break;
+        }
+        if (bytes[j] == 0x20) {
+          if (padRunStart == -1) {
+            padRunStart = j;
+          }
+        } else {
+          padRunStart = -1;
+        }
+        ++charCount;
+      } else {
+        padRunStart = -1;
+      }
+      j++;
+    }
+    if (padRunStart != -1) {
+      return Arrays.copyOf(bytes, padRunStart);
+    } else if (j == end) {
+      return bytes;
+    } else {
+      return Arrays.copyOf(bytes, j);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/storage-api/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgumentImpl.java
----------------------------------------------------------------------
diff --git a/storage-api/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgumentImpl.java b/storage-api/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgumentImpl.java
index 8c5bab2..10d8c51 100644
--- a/storage-api/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgumentImpl.java
+++ b/storage-api/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgumentImpl.java
@@ -32,11 +32,11 @@ import java.util.Queue;
 import java.util.Set;
 
 /**
- * The implementation of SearchArguments.
+ * The implementation of SearchArguments. Visible for testing only.
  */
-final class SearchArgumentImpl implements SearchArgument {
+public final class SearchArgumentImpl implements SearchArgument {
 
-  static final class PredicateLeafImpl implements PredicateLeaf {
+  public static final class PredicateLeafImpl implements PredicateLeaf {
     private final Operator operator;
     private final Type type;
     private String columnName;
@@ -53,11 +53,11 @@ final class SearchArgumentImpl implements SearchArgument {
       literalList = null;
     }
 
-    PredicateLeafImpl(Operator operator,
-                      Type type,
-                      String columnName,
-                      Object literal,
-                      List<Object> literalList) {
+    public PredicateLeafImpl(Operator operator,
+                             Type type,
+                             String columnName,
+                             Object literal,
+                             List<Object> literalList) {
       this.operator = operator;
       this.type = type;
       this.columnName = columnName;

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/storage-api/src/java/org/apache/hadoop/hive/ql/util/TimestampUtils.java
----------------------------------------------------------------------
diff --git a/storage-api/src/java/org/apache/hadoop/hive/ql/util/TimestampUtils.java b/storage-api/src/java/org/apache/hadoop/hive/ql/util/TimestampUtils.java
new file mode 100644
index 0000000..189ead5
--- /dev/null
+++ b/storage-api/src/java/org/apache/hadoop/hive/ql/util/TimestampUtils.java
@@ -0,0 +1,94 @@
+/**
+ * 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.hive.ql.util;
+
+import org.apache.hadoop.hive.common.type.HiveDecimal;
+
+import java.math.BigDecimal;
+import java.sql.Timestamp;
+
+/**
+ * Utitilities for Timestamps and the relevant conversions.
+ */
+public class TimestampUtils {
+  public static final BigDecimal BILLION_BIG_DECIMAL = BigDecimal.valueOf(1000000000);
+
+  /**
+   * Convert the timestamp to a double measured in seconds.
+   * @return double representation of the timestamp, accurate to nanoseconds
+   */
+  public static double getDouble(Timestamp ts) {
+    long seconds = millisToSeconds(ts.getTime());
+    return seconds + ((double) ts.getNanos()) / 1000000000;
+  }
+
+  public static Timestamp doubleToTimestamp(double f) {
+    long seconds = (long) f;
+
+    // We must ensure the exactness of the double's fractional portion.
+    // 0.6 as the fraction part will be converted to 0.59999... and
+    // significantly reduce the savings from binary serialization
+    BigDecimal bd;
+    try {
+      bd = new BigDecimal(String.valueOf(f));
+    } catch (NumberFormatException nfe) {
+      return null;
+    }
+    bd = bd.subtract(new BigDecimal(seconds)).multiply(new BigDecimal(1000000000));
+    int nanos = bd.intValue();
+
+    // Convert to millis
+    long millis = seconds * 1000;
+    if (nanos < 0) {
+      millis -= 1000;
+      nanos += 1000000000;
+    }
+    Timestamp t = new Timestamp(millis);
+
+    // Set remaining fractional portion to nanos
+    t.setNanos(nanos);
+    return t;
+  }
+
+  public static Timestamp decimalToTimestamp(HiveDecimal d) {
+    BigDecimal nanoInstant = d.bigDecimalValue().multiply(BILLION_BIG_DECIMAL);
+    int nanos = nanoInstant.remainder(BILLION_BIG_DECIMAL).intValue();
+    if (nanos < 0) {
+      nanos += 1000000000;
+    }
+    long seconds =
+        nanoInstant.subtract(new BigDecimal(nanos)).divide(BILLION_BIG_DECIMAL).longValue();
+    Timestamp t = new Timestamp(seconds * 1000);
+    t.setNanos(nanos);
+
+    return t;
+  }
+
+  /**
+   * Rounds the number of milliseconds relative to the epoch down to the nearest whole number of
+   * seconds. 500 would round to 0, -500 would round to -1.
+   */
+  public static long millisToSeconds(long millis) {
+    if (millis >= 0) {
+      return millis / 1000;
+    } else {
+      return (millis - 999) / 1000;
+    }
+  }
+}


[16/27] hive git commit: HIVE-11417. Move the ReaderImpl and RowReaderImpl to the ORC module, by making shims for the row by row reader. (omalley reviewed by prasanth_j)

Posted by om...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/orc/src/test/org/apache/orc/tools/TestFileDump.java
----------------------------------------------------------------------
diff --git a/orc/src/test/org/apache/orc/tools/TestFileDump.java b/orc/src/test/org/apache/orc/tools/TestFileDump.java
new file mode 100644
index 0000000..ce3381e
--- /dev/null
+++ b/orc/src/test/org/apache/orc/tools/TestFileDump.java
@@ -0,0 +1,486 @@
+/**
+ * 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.orc.tools;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
+import java.io.BufferedReader;
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.FileReader;
+import java.io.PrintStream;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.common.type.HiveDecimal;
+import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.ListColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.MapColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.StructColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.serde2.io.DateWritable;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.orc.CompressionKind;
+import org.apache.orc.OrcConf;
+import org.apache.orc.OrcFile;
+import org.apache.orc.TypeDescription;
+import org.apache.orc.Writer;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestFileDump {
+
+  Path workDir = new Path(System.getProperty("test.tmp.dir"));
+  Configuration conf;
+  FileSystem fs;
+  Path testFilePath;
+
+  @Before
+  public void openFileSystem () throws Exception {
+    conf = new Configuration();
+    fs = FileSystem.getLocal(conf);
+    fs.setWorkingDirectory(workDir);
+    testFilePath = new Path("TestFileDump.testDump.orc");
+    fs.delete(testFilePath, false);
+  }
+
+  static TypeDescription getMyRecordType() {
+    return TypeDescription.createStruct()
+        .addField("i", TypeDescription.createInt())
+        .addField("l", TypeDescription.createLong())
+        .addField("s", TypeDescription.createString());
+  }
+
+  static void appendMyRecord(VectorizedRowBatch batch,
+                             int i,
+                             long l,
+                             String str) {
+    ((LongColumnVector) batch.cols[0]).vector[batch.size] = i;
+    ((LongColumnVector) batch.cols[1]).vector[batch.size] = l;
+    if (str == null) {
+      batch.cols[2].noNulls = false;
+      batch.cols[2].isNull[batch.size] = true;
+    } else {
+      ((BytesColumnVector) batch.cols[2]).setVal(batch.size,
+          str.getBytes());
+    }
+    batch.size += 1;
+  }
+
+  static TypeDescription getAllTypesType() {
+    return TypeDescription.createStruct()
+        .addField("b", TypeDescription.createBoolean())
+        .addField("bt", TypeDescription.createByte())
+        .addField("s", TypeDescription.createShort())
+        .addField("i", TypeDescription.createInt())
+        .addField("l", TypeDescription.createLong())
+        .addField("f", TypeDescription.createFloat())
+        .addField("d", TypeDescription.createDouble())
+        .addField("de", TypeDescription.createDecimal())
+        .addField("t", TypeDescription.createTimestamp())
+        .addField("dt", TypeDescription.createDate())
+        .addField("str", TypeDescription.createString())
+        .addField("c", TypeDescription.createChar().withMaxLength(5))
+        .addField("vc", TypeDescription.createVarchar().withMaxLength(10))
+        .addField("m", TypeDescription.createMap(
+            TypeDescription.createString(),
+            TypeDescription.createString()))
+        .addField("a", TypeDescription.createList(TypeDescription.createInt()))
+        .addField("st", TypeDescription.createStruct()
+                .addField("i", TypeDescription.createInt())
+                .addField("s", TypeDescription.createString()));
+  }
+
+  static void appendAllTypes(VectorizedRowBatch batch,
+                             boolean b,
+                             byte bt,
+                             short s,
+                             int i,
+                             long l,
+                             float f,
+                             double d,
+                             HiveDecimalWritable de,
+                             Timestamp t,
+                             DateWritable dt,
+                             String str,
+                             String c,
+                             String vc,
+                             Map<String, String> m,
+                             List<Integer> a,
+                             int sti,
+                             String sts) {
+    int row = batch.size++;
+    ((LongColumnVector) batch.cols[0]).vector[row] = b ? 1 : 0;
+    ((LongColumnVector) batch.cols[1]).vector[row] = bt;
+    ((LongColumnVector) batch.cols[2]).vector[row] = s;
+    ((LongColumnVector) batch.cols[3]).vector[row] = i;
+    ((LongColumnVector) batch.cols[4]).vector[row] = l;
+    ((DoubleColumnVector) batch.cols[5]).vector[row] = f;
+    ((DoubleColumnVector) batch.cols[6]).vector[row] = d;
+    ((DecimalColumnVector) batch.cols[7]).vector[row].set(de);
+    ((TimestampColumnVector) batch.cols[8]).set(row, t);
+    ((LongColumnVector) batch.cols[9]).vector[row] = dt.getDays();
+    ((BytesColumnVector) batch.cols[10]).setVal(row, str.getBytes());
+    ((BytesColumnVector) batch.cols[11]).setVal(row, c.getBytes());
+    ((BytesColumnVector) batch.cols[12]).setVal(row, vc.getBytes());
+    MapColumnVector map = (MapColumnVector) batch.cols[13];
+    int offset = map.childCount;
+    map.offsets[row] = offset;
+    map.lengths[row] = m.size();
+    map.childCount += map.lengths[row];
+    for(Map.Entry<String, String> entry: m.entrySet()) {
+      ((BytesColumnVector) map.keys).setVal(offset, entry.getKey().getBytes());
+      ((BytesColumnVector) map.values).setVal(offset++,
+          entry.getValue().getBytes());
+    }
+    ListColumnVector list = (ListColumnVector) batch.cols[14];
+    offset = list.childCount;
+    list.offsets[row] = offset;
+    list.lengths[row] = a.size();
+    list.childCount += list.lengths[row];
+    for(int e=0; e < a.size(); ++e) {
+      ((LongColumnVector) list.child).vector[offset + e] = a.get(e);
+    }
+    StructColumnVector struct = (StructColumnVector) batch.cols[15];
+    ((LongColumnVector) struct.fields[0]).vector[row] = sti;
+    ((BytesColumnVector) struct.fields[1]).setVal(row, sts.getBytes());
+  }
+
+  public static void checkOutput(String expected,
+                                 String actual) throws Exception {
+    BufferedReader eStream =
+        new BufferedReader(new FileReader
+            (TestJsonFileDump.getFileFromClasspath(expected)));
+    BufferedReader aStream =
+        new BufferedReader(new FileReader(actual));
+    String expectedLine = eStream.readLine().trim();
+    while (expectedLine != null) {
+      String actualLine = aStream.readLine().trim();
+      System.out.println("actual:   " + actualLine);
+      System.out.println("expected: " + expectedLine);
+      Assert.assertEquals(expectedLine, actualLine);
+      expectedLine = eStream.readLine();
+      expectedLine = expectedLine == null ? null : expectedLine.trim();
+    }
+    Assert.assertNull(eStream.readLine());
+    Assert.assertNull(aStream.readLine());
+    eStream.close();
+    aStream.close();
+  }
+
+  @Test
+  public void testDump() throws Exception {
+    TypeDescription schema = getMyRecordType();
+    conf.set(OrcConf.ENCODING_STRATEGY.getAttribute(), "COMPRESSION");
+    Writer writer = OrcFile.createWriter(testFilePath,
+        OrcFile.writerOptions(conf)
+            .fileSystem(fs)
+            .setSchema(schema)
+            .compress(CompressionKind.ZLIB)
+            .stripeSize(100000)
+            .rowIndexStride(1000));
+    Random r1 = new Random(1);
+    String[] words = new String[]{"It", "was", "the", "best", "of", "times,",
+        "it", "was", "the", "worst", "of", "times,", "it", "was", "the", "age",
+        "of", "wisdom,", "it", "was", "the", "age", "of", "foolishness,", "it",
+        "was", "the", "epoch", "of", "belief,", "it", "was", "the", "epoch",
+        "of", "incredulity,", "it", "was", "the", "season", "of", "Light,",
+        "it", "was", "the", "season", "of", "Darkness,", "it", "was", "the",
+        "spring", "of", "hope,", "it", "was", "the", "winter", "of", "despair,",
+        "we", "had", "everything", "before", "us,", "we", "had", "nothing",
+        "before", "us,", "we", "were", "all", "going", "direct", "to",
+        "Heaven,", "we", "were", "all", "going", "direct", "the", "other",
+        "way"};
+    VectorizedRowBatch batch = schema.createRowBatch(1000);
+    for(int i=0; i < 21000; ++i) {
+      appendMyRecord(batch, r1.nextInt(), r1.nextLong(),
+          words[r1.nextInt(words.length)]);
+      if (batch.size == batch.getMaxSize()) {
+        writer.addRowBatch(batch);
+        batch.reset();
+      }
+    }
+    if (batch.size > 0) {
+      writer.addRowBatch(batch);
+    }
+    writer.close();
+    PrintStream origOut = System.out;
+    String outputFilename = "orc-file-dump.out";
+    FileOutputStream myOut = new FileOutputStream(workDir + File.separator + outputFilename);
+
+    // replace stdout and run command
+    System.setOut(new PrintStream(myOut));
+    FileDump.main(new String[]{testFilePath.toString(), "--rowindex=1,2,3"});
+    System.out.flush();
+    System.setOut(origOut);
+
+
+    checkOutput(outputFilename, workDir + File.separator + outputFilename);
+  }
+
+  @Test
+  public void testDataDump() throws Exception {
+    TypeDescription schema = getAllTypesType();
+    Writer writer = OrcFile.createWriter(testFilePath,
+        OrcFile.writerOptions(conf)
+            .fileSystem(fs)
+            .setSchema(schema)
+            .stripeSize(100000)
+            .compress(CompressionKind.NONE)
+            .bufferSize(10000)
+            .rowIndexStride(1000));
+    VectorizedRowBatch batch = schema.createRowBatch(1000);
+    Map<String, String> m = new HashMap<String, String>(2);
+    m.put("k1", "v1");
+    appendAllTypes(batch,
+        true,
+        (byte) 10,
+        (short) 100,
+        1000,
+        10000L,
+        4.0f,
+        20.0,
+        new HiveDecimalWritable("4.2222"),
+        new Timestamp(1416967764000L),
+        new DateWritable(new Date(1416967764000L)),
+        "string",
+        "hello",
+       "hello",
+        m,
+        Arrays.asList(100, 200),
+        10, "foo");
+    m.clear();
+    m.put("k3", "v3");
+    appendAllTypes(
+        batch,
+        false,
+        (byte)20,
+        (short)200,
+        2000,
+        20000L,
+        8.0f,
+        40.0,
+        new HiveDecimalWritable("2.2222"),
+        new Timestamp(1416967364000L),
+        new DateWritable(new Date(1411967764000L)),
+        "abcd",
+        "world",
+        "world",
+        m,
+        Arrays.asList(200, 300),
+        20, "bar");
+    writer.addRowBatch(batch);
+
+    writer.close();
+    PrintStream origOut = System.out;
+    ByteArrayOutputStream myOut = new ByteArrayOutputStream();
+
+    // replace stdout and run command
+    System.setOut(new PrintStream(myOut));
+    FileDump.main(new String[]{testFilePath.toString(), "-d"});
+    System.out.flush();
+    System.setOut(origOut);
+    String[] lines = myOut.toString().split("\n");
+    Assert.assertEquals("{\"b\":true,\"bt\":10,\"s\":100,\"i\":1000,\"l\":10000,\"f\":4,\"d\":20,\"de\":\"4.2222\",\"t\":\"2014-11-25 18:09:24.0\",\"dt\":\"2014-11-25\",\"str\":\"string\",\"c\":\"hello\",\"vc\":\"hello\",\"m\":[{\"_key\":\"k1\",\"_value\":\"v1\"}],\"a\":[100,200],\"st\":{\"i\":10,\"s\":\"foo\"}}", lines[0]);
+    Assert.assertEquals("{\"b\":false,\"bt\":20,\"s\":200,\"i\":2000,\"l\":20000,\"f\":8,\"d\":40,\"de\":\"2.2222\",\"t\":\"2014-11-25 18:02:44.0\",\"dt\":\"2014-09-28\",\"str\":\"abcd\",\"c\":\"world\",\"vc\":\"world\",\"m\":[{\"_key\":\"k3\",\"_value\":\"v3\"}],\"a\":[200,300],\"st\":{\"i\":20,\"s\":\"bar\"}}", lines[1]);
+  }
+  
+  // Test that if the fraction of rows that have distinct strings is greater than the configured
+  // threshold dictionary encoding is turned off.  If dictionary encoding is turned off the length
+  // of the dictionary stream for the column will be 0 in the ORC file dump.
+  @Test
+  public void testDictionaryThreshold() throws Exception {
+    TypeDescription schema = getMyRecordType();
+    Configuration conf = new Configuration();
+    conf.set(OrcConf.ENCODING_STRATEGY.getAttribute(), "COMPRESSION");
+    conf.setFloat(OrcConf.DICTIONARY_KEY_SIZE_THRESHOLD.getAttribute(), 0.49f);
+    Writer writer = OrcFile.createWriter(testFilePath,
+        OrcFile.writerOptions(conf)
+            .fileSystem(fs)
+            .setSchema(schema)
+            .stripeSize(100000)
+            .compress(CompressionKind.ZLIB)
+            .rowIndexStride(1000)
+            .bufferSize(10000));
+    VectorizedRowBatch batch = schema.createRowBatch(1000);
+    Random r1 = new Random(1);
+    String[] words = new String[]{"It", "was", "the", "best", "of", "times,",
+        "it", "was", "the", "worst", "of", "times,", "it", "was", "the", "age",
+        "of", "wisdom,", "it", "was", "the", "age", "of", "foolishness,", "it",
+        "was", "the", "epoch", "of", "belief,", "it", "was", "the", "epoch",
+        "of", "incredulity,", "it", "was", "the", "season", "of", "Light,",
+        "it", "was", "the", "season", "of", "Darkness,", "it", "was", "the",
+        "spring", "of", "hope,", "it", "was", "the", "winter", "of", "despair,",
+        "we", "had", "everything", "before", "us,", "we", "had", "nothing",
+        "before", "us,", "we", "were", "all", "going", "direct", "to",
+        "Heaven,", "we", "were", "all", "going", "direct", "the", "other",
+        "way"};
+    int nextInt = 0;
+    for(int i=0; i < 21000; ++i) {
+      // Write out the same string twice, this guarantees the fraction of rows with
+      // distinct strings is 0.5
+      if (i % 2 == 0) {
+        nextInt = r1.nextInt(words.length);
+        // Append the value of i to the word, this guarantees when an index or word is repeated
+        // the actual string is unique.
+        words[nextInt] += "-" + i;
+      }
+      appendMyRecord(batch, r1.nextInt(), r1.nextLong(), words[nextInt]);
+      if (batch.size == batch.getMaxSize()) {
+        writer.addRowBatch(batch);
+        batch.reset();
+      }
+    }
+    if (batch.size != 0) {
+      writer.addRowBatch(batch);
+    }
+    writer.close();
+    PrintStream origOut = System.out;
+    String outputFilename = "orc-file-dump-dictionary-threshold.out";
+    FileOutputStream myOut = new FileOutputStream(workDir + File.separator + outputFilename);
+
+    // replace stdout and run command
+    System.setOut(new PrintStream(myOut));
+    FileDump.main(new String[]{testFilePath.toString(), "--rowindex=1,2,3"});
+    System.out.flush();
+    System.setOut(origOut);
+
+    checkOutput(outputFilename, workDir + File.separator + outputFilename);
+  }
+
+  @Test
+  public void testBloomFilter() throws Exception {
+    TypeDescription schema = getMyRecordType();
+    conf.set(OrcConf.ENCODING_STRATEGY.getAttribute(), "COMPRESSION");
+    OrcFile.WriterOptions options = OrcFile.writerOptions(conf)
+        .fileSystem(fs)
+        .setSchema(schema)
+        .stripeSize(100000)
+        .compress(CompressionKind.ZLIB)
+        .bufferSize(10000)
+        .rowIndexStride(1000)
+        .bloomFilterColumns("S");
+    Writer writer = OrcFile.createWriter(testFilePath, options);
+    Random r1 = new Random(1);
+    String[] words = new String[]{"It", "was", "the", "best", "of", "times,",
+        "it", "was", "the", "worst", "of", "times,", "it", "was", "the", "age",
+        "of", "wisdom,", "it", "was", "the", "age", "of", "foolishness,", "it",
+        "was", "the", "epoch", "of", "belief,", "it", "was", "the", "epoch",
+        "of", "incredulity,", "it", "was", "the", "season", "of", "Light,",
+        "it", "was", "the", "season", "of", "Darkness,", "it", "was", "the",
+        "spring", "of", "hope,", "it", "was", "the", "winter", "of", "despair,",
+        "we", "had", "everything", "before", "us,", "we", "had", "nothing",
+        "before", "us,", "we", "were", "all", "going", "direct", "to",
+        "Heaven,", "we", "were", "all", "going", "direct", "the", "other",
+        "way"};
+    VectorizedRowBatch batch = schema.createRowBatch(1000);
+    for(int i=0; i < 21000; ++i) {
+      appendMyRecord(batch, r1.nextInt(), r1.nextLong(),
+          words[r1.nextInt(words.length)]);
+      if (batch.size == batch.getMaxSize()) {
+        writer.addRowBatch(batch);
+        batch.reset();
+      }
+    }
+    if (batch.size > 0) {
+      writer.addRowBatch(batch);
+    }
+    writer.close();
+    PrintStream origOut = System.out;
+    String outputFilename = "orc-file-dump-bloomfilter.out";
+    FileOutputStream myOut = new FileOutputStream(workDir + File.separator + outputFilename);
+
+    // replace stdout and run command
+    System.setOut(new PrintStream(myOut));
+    FileDump.main(new String[]{testFilePath.toString(), "--rowindex=3"});
+    System.out.flush();
+    System.setOut(origOut);
+
+
+    checkOutput(outputFilename, workDir + File.separator + outputFilename);
+  }
+
+  @Test
+  public void testBloomFilter2() throws Exception {
+    TypeDescription schema = getMyRecordType();
+    conf.set(OrcConf.ENCODING_STRATEGY.getAttribute(), "COMPRESSION");
+    OrcFile.WriterOptions options = OrcFile.writerOptions(conf)
+        .fileSystem(fs)
+        .setSchema(schema)
+        .stripeSize(100000)
+        .compress(CompressionKind.ZLIB)
+        .bufferSize(10000)
+        .rowIndexStride(1000)
+        .bloomFilterColumns("l")
+        .bloomFilterFpp(0.01);
+    VectorizedRowBatch batch = schema.createRowBatch(1000);
+    Writer writer = OrcFile.createWriter(testFilePath, options);
+    Random r1 = new Random(1);
+    String[] words = new String[]{"It", "was", "the", "best", "of", "times,",
+        "it", "was", "the", "worst", "of", "times,", "it", "was", "the", "age",
+        "of", "wisdom,", "it", "was", "the", "age", "of", "foolishness,", "it",
+        "was", "the", "epoch", "of", "belief,", "it", "was", "the", "epoch",
+        "of", "incredulity,", "it", "was", "the", "season", "of", "Light,",
+        "it", "was", "the", "season", "of", "Darkness,", "it", "was", "the",
+        "spring", "of", "hope,", "it", "was", "the", "winter", "of", "despair,",
+        "we", "had", "everything", "before", "us,", "we", "had", "nothing",
+        "before", "us,", "we", "were", "all", "going", "direct", "to",
+        "Heaven,", "we", "were", "all", "going", "direct", "the", "other",
+        "way"};
+    for(int i=0; i < 21000; ++i) {
+      appendMyRecord(batch, r1.nextInt(), r1.nextLong(),
+          words[r1.nextInt(words.length)]);
+      if (batch.size == batch.getMaxSize()) {
+        writer.addRowBatch(batch);
+        batch.reset();
+      }
+    }
+    if (batch.size > 0) {
+      writer.addRowBatch(batch);
+    }
+    writer.close();
+    PrintStream origOut = System.out;
+    String outputFilename = "orc-file-dump-bloomfilter2.out";
+    FileOutputStream myOut = new FileOutputStream(workDir + File.separator + outputFilename);
+
+    // replace stdout and run command
+    System.setOut(new PrintStream(myOut));
+    FileDump.main(new String[]{testFilePath.toString(), "--rowindex=2"});
+    System.out.flush();
+    System.setOut(origOut);
+
+
+    checkOutput(outputFilename, workDir + File.separator + outputFilename);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/orc/src/test/org/apache/orc/tools/TestJsonFileDump.java
----------------------------------------------------------------------
diff --git a/orc/src/test/org/apache/orc/tools/TestJsonFileDump.java b/orc/src/test/org/apache/orc/tools/TestJsonFileDump.java
new file mode 100644
index 0000000..a514824
--- /dev/null
+++ b/orc/src/test/org/apache/orc/tools/TestJsonFileDump.java
@@ -0,0 +1,150 @@
+/**
+ * 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.orc.tools;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.FileReader;
+import java.io.PrintStream;
+import java.net.URL;
+import java.util.Random;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.orc.CompressionKind;
+import org.apache.orc.OrcConf;
+import org.apache.orc.OrcFile;
+import org.apache.orc.TypeDescription;
+import org.apache.orc.Writer;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestJsonFileDump {
+  public static String getFileFromClasspath(String name) {
+    URL url = ClassLoader.getSystemResource(name);
+    if (url == null) {
+      throw new IllegalArgumentException("Could not find " + name);
+    }
+    return url.getPath();
+  }
+
+  Path workDir = new Path(System.getProperty("test.tmp.dir"));
+  Configuration conf;
+  FileSystem fs;
+  Path testFilePath;
+
+  @Before
+  public void openFileSystem () throws Exception {
+    conf = new Configuration();
+    fs = FileSystem.getLocal(conf);
+    fs.setWorkingDirectory(workDir);
+    testFilePath = new Path("TestFileDump.testDump.orc");
+    fs.delete(testFilePath, false);
+  }
+
+  static void checkOutput(String expected,
+                                  String actual) throws Exception {
+    BufferedReader eStream =
+        new BufferedReader(new FileReader(getFileFromClasspath(expected)));
+    BufferedReader aStream =
+        new BufferedReader(new FileReader(actual));
+    String expectedLine = eStream.readLine();
+    while (expectedLine != null) {
+      String actualLine = aStream.readLine();
+      System.out.println("actual:   " + actualLine);
+      System.out.println("expected: " + expectedLine);
+      assertEquals(expectedLine, actualLine);
+      expectedLine = eStream.readLine();
+    }
+    assertNull(eStream.readLine());
+    assertNull(aStream.readLine());
+  }
+
+  @Test
+  public void testJsonDump() throws Exception {
+    TypeDescription schema = TypeDescription.createStruct()
+        .addField("i", TypeDescription.createInt())
+        .addField("l", TypeDescription.createLong())
+        .addField("s", TypeDescription.createString());
+    conf.set(OrcConf.ENCODING_STRATEGY.getAttribute(), "COMPRESSION");
+    OrcFile.WriterOptions options = OrcFile.writerOptions(conf)
+        .fileSystem(fs)
+        .setSchema(schema)
+        .stripeSize(100000)
+        .compress(CompressionKind.ZLIB)
+        .bufferSize(10000)
+        .rowIndexStride(1000)
+        .bloomFilterColumns("s");
+    Writer writer = OrcFile.createWriter(testFilePath, options);
+    Random r1 = new Random(1);
+    String[] words = new String[]{"It", "was", "the", "best", "of", "times,",
+        "it", "was", "the", "worst", "of", "times,", "it", "was", "the", "age",
+        "of", "wisdom,", "it", "was", "the", "age", "of", "foolishness,", "it",
+        "was", "the", "epoch", "of", "belief,", "it", "was", "the", "epoch",
+        "of", "incredulity,", "it", "was", "the", "season", "of", "Light,",
+        "it", "was", "the", "season", "of", "Darkness,", "it", "was", "the",
+        "spring", "of", "hope,", "it", "was", "the", "winter", "of", "despair,",
+        "we", "had", "everything", "before", "us,", "we", "had", "nothing",
+        "before", "us,", "we", "were", "all", "going", "direct", "to",
+        "Heaven,", "we", "were", "all", "going", "direct", "the", "other",
+        "way"};
+    VectorizedRowBatch batch = schema.createRowBatch(1000);
+    for(int i=0; i < 21000; ++i) {
+      ((LongColumnVector) batch.cols[0]).vector[batch.size] = r1.nextInt();
+      ((LongColumnVector) batch.cols[1]).vector[batch.size] = r1.nextLong();
+      if (i % 100 == 0) {
+        batch.cols[2].noNulls = false;
+        batch.cols[2].isNull[batch.size] = true;
+      } else {
+        ((BytesColumnVector) batch.cols[2]).setVal(batch.size,
+            words[r1.nextInt(words.length)].getBytes());
+      }
+      batch.size += 1;
+      if (batch.size == batch.getMaxSize()) {
+        writer.addRowBatch(batch);
+        batch.reset();
+      }
+    }
+    if (batch.size > 0) {
+      writer.addRowBatch(batch);
+    }
+
+    writer.close();
+    PrintStream origOut = System.out;
+    String outputFilename = "orc-file-dump.json";
+    FileOutputStream myOut = new FileOutputStream(workDir + File.separator + outputFilename);
+
+    // replace stdout and run command
+    System.setOut(new PrintStream(myOut));
+    FileDump.main(new String[]{testFilePath.toString(), "-j", "-p", "--rowindex=3"});
+    System.out.flush();
+    System.setOut(origOut);
+
+
+    checkOutput(outputFilename, workDir + File.separator + outputFilename);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/orc/src/test/resources/orc-file-11-format.orc
----------------------------------------------------------------------
diff --git a/orc/src/test/resources/orc-file-11-format.orc b/orc/src/test/resources/orc-file-11-format.orc
new file mode 100644
index 0000000..41653c8
Binary files /dev/null and b/orc/src/test/resources/orc-file-11-format.orc differ

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/orc/src/test/resources/orc-file-dump-bloomfilter.out
----------------------------------------------------------------------
diff --git a/orc/src/test/resources/orc-file-dump-bloomfilter.out b/orc/src/test/resources/orc-file-dump-bloomfilter.out
new file mode 100644
index 0000000..18fd2fb
--- /dev/null
+++ b/orc/src/test/resources/orc-file-dump-bloomfilter.out
@@ -0,0 +1,179 @@
+Structure for TestFileDump.testDump.orc
+File Version: 0.12 with HIVE_13083
+Rows: 21000
+Compression: ZLIB
+Compression size: 4096
+Type: struct<i:int,l:bigint,s:string>
+
+Stripe Statistics:
+  Stripe 1:
+    Column 0: count: 5000 hasNull: false
+    Column 1: count: 5000 hasNull: false min: -2146021688 max: 2147223299 sum: 515792826
+    Column 2: count: 5000 hasNull: false min: -9218592812243954469 max: 9221614132680747961
+    Column 3: count: 5000 hasNull: false min: Darkness, max: worst sum: 19280
+  Stripe 2:
+    Column 0: count: 5000 hasNull: false
+    Column 1: count: 5000 hasNull: false min: -2146733128 max: 2147001622 sum: 7673427
+    Column 2: count: 5000 hasNull: false min: -9220818777591257749 max: 9222259462014003839
+    Column 3: count: 5000 hasNull: false min: Darkness, max: worst sum: 19504
+  Stripe 3:
+    Column 0: count: 5000 hasNull: false
+    Column 1: count: 5000 hasNull: false min: -2146993718 max: 2147378179 sum: 132660742551
+    Column 2: count: 5000 hasNull: false min: -9218342074710552826 max: 9222303228623055266
+    Column 3: count: 5000 hasNull: false min: Darkness, max: worst sum: 19641
+  Stripe 4:
+    Column 0: count: 5000 hasNull: false
+    Column 1: count: 5000 hasNull: false min: -2146658006 max: 2145520931 sum: 8533549236
+    Column 2: count: 5000 hasNull: false min: -9222758097219661129 max: 9221043130193737406
+    Column 3: count: 5000 hasNull: false min: Darkness, max: worst sum: 19470
+  Stripe 5:
+    Column 0: count: 1000 hasNull: false
+    Column 1: count: 1000 hasNull: false min: -2146245500 max: 2146378640 sum: 51299706363
+    Column 2: count: 1000 hasNull: false min: -9208193203370316142 max: 9218567213558056476
+    Column 3: count: 1000 hasNull: false min: Darkness, max: worst sum: 3866
+
+File Statistics:
+  Column 0: count: 21000 hasNull: false
+  Column 1: count: 21000 hasNull: false min: -2146993718 max: 2147378179 sum: 193017464403
+  Column 2: count: 21000 hasNull: false min: -9222758097219661129 max: 9222303228623055266
+  Column 3: count: 21000 hasNull: false min: Darkness, max: worst sum: 81761
+
+Stripes:
+  Stripe: offset: 3 data: 63786 rows: 5000 tail: 86 index: 951
+    Stream: column 0 section ROW_INDEX start: 3 length 17
+    Stream: column 1 section ROW_INDEX start: 20 length 166
+    Stream: column 2 section ROW_INDEX start: 186 length 169
+    Stream: column 3 section ROW_INDEX start: 355 length 87
+    Stream: column 3 section BLOOM_FILTER start: 442 length 512
+    Stream: column 1 section DATA start: 954 length 20035
+    Stream: column 2 section DATA start: 20989 length 40050
+    Stream: column 3 section DATA start: 61039 length 3543
+    Stream: column 3 section LENGTH start: 64582 length 25
+    Stream: column 3 section DICTIONARY_DATA start: 64607 length 133
+    Encoding column 0: DIRECT
+    Encoding column 1: DIRECT_V2
+    Encoding column 2: DIRECT_V2
+    Encoding column 3: DICTIONARY_V2[35]
+    Row group indices for column 3:
+      Entry 0: count: 1000 hasNull: false min: Darkness, max: worst sum: 3862 positions: 0,0,0
+      Entry 1: count: 1000 hasNull: false min: Darkness, max: worst sum: 3884 positions: 0,659,149
+      Entry 2: count: 1000 hasNull: false min: Darkness, max: worst sum: 3893 positions: 0,1531,3
+      Entry 3: count: 1000 hasNull: false min: Darkness, max: worst sum: 3798 positions: 0,2281,32
+      Entry 4: count: 1000 hasNull: false min: Darkness, max: worst sum: 3843 positions: 0,3033,45
+    Bloom filters for column 3:
+      Entry 0: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
+      Entry 1: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
+      Entry 2: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
+      Entry 3: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
+      Entry 4: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
+      Stripe level merge: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
+  Stripe: offset: 64826 data: 63775 rows: 5000 tail: 86 index: 944
+    Stream: column 0 section ROW_INDEX start: 64826 length 17
+    Stream: column 1 section ROW_INDEX start: 64843 length 164
+    Stream: column 2 section ROW_INDEX start: 65007 length 168
+    Stream: column 3 section ROW_INDEX start: 65175 length 83
+    Stream: column 3 section BLOOM_FILTER start: 65258 length 512
+    Stream: column 1 section DATA start: 65770 length 20035
+    Stream: column 2 section DATA start: 85805 length 40050
+    Stream: column 3 section DATA start: 125855 length 3532
+    Stream: column 3 section LENGTH start: 129387 length 25
+    Stream: column 3 section DICTIONARY_DATA start: 129412 length 133
+    Encoding column 0: DIRECT
+    Encoding column 1: DIRECT_V2
+    Encoding column 2: DIRECT_V2
+    Encoding column 3: DICTIONARY_V2[35]
+    Row group indices for column 3:
+      Entry 0: count: 1000 hasNull: false min: Darkness, max: worst sum: 3923 positions: 0,0,0
+      Entry 1: count: 1000 hasNull: false min: Darkness, max: worst sum: 3869 positions: 0,761,12
+      Entry 2: count: 1000 hasNull: false min: Darkness, max: worst sum: 3817 positions: 0,1472,70
+      Entry 3: count: 1000 hasNull: false min: Darkness, max: worst sum: 3931 positions: 0,2250,43
+      Entry 4: count: 1000 hasNull: false min: Darkness, max: worst sum: 3964 positions: 0,2978,88
+    Bloom filters for column 3:
+      Entry 0: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
+      Entry 1: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
+      Entry 2: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
+      Entry 3: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
+      Entry 4: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
+      Stripe level merge: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
+  Stripe: offset: 129631 data: 63787 rows: 5000 tail: 86 index: 950
+    Stream: column 0 section ROW_INDEX start: 129631 length 17
+    Stream: column 1 section ROW_INDEX start: 129648 length 163
+    Stream: column 2 section ROW_INDEX start: 129811 length 168
+    Stream: column 3 section ROW_INDEX start: 129979 length 90
+    Stream: column 3 section BLOOM_FILTER start: 130069 length 512
+    Stream: column 1 section DATA start: 130581 length 20035
+    Stream: column 2 section DATA start: 150616 length 40050
+    Stream: column 3 section DATA start: 190666 length 3544
+    Stream: column 3 section LENGTH start: 194210 length 25
+    Stream: column 3 section DICTIONARY_DATA start: 194235 length 133
+    Encoding column 0: DIRECT
+    Encoding column 1: DIRECT_V2
+    Encoding column 2: DIRECT_V2
+    Encoding column 3: DICTIONARY_V2[35]
+    Row group indices for column 3:
+      Entry 0: count: 1000 hasNull: false min: Darkness, max: worst sum: 3817 positions: 0,0,0
+      Entry 1: count: 1000 hasNull: false min: Darkness, max: worst sum: 4008 positions: 0,634,174
+      Entry 2: count: 1000 hasNull: false min: Darkness, max: worst sum: 3999 positions: 0,1469,69
+      Entry 3: count: 1000 hasNull: false min: Darkness, max: worst sum: 3817 positions: 0,2133,194
+      Entry 4: count: 1000 hasNull: false min: Darkness, max: worst sum: 4000 positions: 0,3005,43
+    Bloom filters for column 3:
+      Entry 0: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
+      Entry 1: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
+      Entry 2: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
+      Entry 3: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
+      Entry 4: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
+      Stripe level merge: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
+  Stripe: offset: 194454 data: 63817 rows: 5000 tail: 86 index: 952
+    Stream: column 0 section ROW_INDEX start: 194454 length 17
+    Stream: column 1 section ROW_INDEX start: 194471 length 165
+    Stream: column 2 section ROW_INDEX start: 194636 length 167
+    Stream: column 3 section ROW_INDEX start: 194803 length 91
+    Stream: column 3 section BLOOM_FILTER start: 194894 length 512
+    Stream: column 1 section DATA start: 195406 length 20035
+    Stream: column 2 section DATA start: 215441 length 40050
+    Stream: column 3 section DATA start: 255491 length 3574
+    Stream: column 3 section LENGTH start: 259065 length 25
+    Stream: column 3 section DICTIONARY_DATA start: 259090 length 133
+    Encoding column 0: DIRECT
+    Encoding column 1: DIRECT_V2
+    Encoding column 2: DIRECT_V2
+    Encoding column 3: DICTIONARY_V2[35]
+    Row group indices for column 3:
+      Entry 0: count: 1000 hasNull: false min: Darkness, max: worst sum: 3901 positions: 0,0,0
+      Entry 1: count: 1000 hasNull: false min: Darkness, max: worst sum: 3900 positions: 0,431,431
+      Entry 2: count: 1000 hasNull: false min: Darkness, max: worst sum: 3909 positions: 0,1485,52
+      Entry 3: count: 1000 hasNull: false min: Darkness, max: worst sum: 3947 positions: 0,2196,104
+      Entry 4: count: 1000 hasNull: false min: Darkness, max: worst sum: 3813 positions: 0,2934,131
+    Bloom filters for column 3:
+      Entry 0: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
+      Entry 1: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
+      Entry 2: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
+      Entry 3: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
+      Entry 4: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
+      Stripe level merge: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
+  Stripe: offset: 259309 data: 12943 rows: 1000 tail: 78 index: 432
+    Stream: column 0 section ROW_INDEX start: 259309 length 12
+    Stream: column 1 section ROW_INDEX start: 259321 length 38
+    Stream: column 2 section ROW_INDEX start: 259359 length 41
+    Stream: column 3 section ROW_INDEX start: 259400 length 40
+    Stream: column 3 section BLOOM_FILTER start: 259440 length 301
+    Stream: column 1 section DATA start: 259741 length 4007
+    Stream: column 2 section DATA start: 263748 length 8010
+    Stream: column 3 section DATA start: 271758 length 768
+    Stream: column 3 section LENGTH start: 272526 length 25
+    Stream: column 3 section DICTIONARY_DATA start: 272551 length 133
+    Encoding column 0: DIRECT
+    Encoding column 1: DIRECT_V2
+    Encoding column 2: DIRECT_V2
+    Encoding column 3: DICTIONARY_V2[35]
+    Row group indices for column 3:
+      Entry 0: count: 1000 hasNull: false min: Darkness, max: worst sum: 3866 positions: 0,0,0
+    Bloom filters for column 3:
+      Entry 0: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
+      Stripe level merge: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
+
+File length: 273307 bytes
+Padding length: 0 bytes
+Padding ratio: 0%
+________________________________________________________________________________________________________________________
+

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/orc/src/test/resources/orc-file-dump-bloomfilter2.out
----------------------------------------------------------------------
diff --git a/orc/src/test/resources/orc-file-dump-bloomfilter2.out b/orc/src/test/resources/orc-file-dump-bloomfilter2.out
new file mode 100644
index 0000000..fa5cc2d
--- /dev/null
+++ b/orc/src/test/resources/orc-file-dump-bloomfilter2.out
@@ -0,0 +1,179 @@
+Structure for TestFileDump.testDump.orc
+File Version: 0.12 with HIVE_13083
+Rows: 21000
+Compression: ZLIB
+Compression size: 4096
+Type: struct<i:int,l:bigint,s:string>
+
+Stripe Statistics:
+  Stripe 1:
+    Column 0: count: 5000 hasNull: false
+    Column 1: count: 5000 hasNull: false min: -2146021688 max: 2147223299 sum: 515792826
+    Column 2: count: 5000 hasNull: false min: -9218592812243954469 max: 9221614132680747961
+    Column 3: count: 5000 hasNull: false min: Darkness, max: worst sum: 19280
+  Stripe 2:
+    Column 0: count: 5000 hasNull: false
+    Column 1: count: 5000 hasNull: false min: -2146733128 max: 2147001622 sum: 7673427
+    Column 2: count: 5000 hasNull: false min: -9220818777591257749 max: 9222259462014003839
+    Column 3: count: 5000 hasNull: false min: Darkness, max: worst sum: 19504
+  Stripe 3:
+    Column 0: count: 5000 hasNull: false
+    Column 1: count: 5000 hasNull: false min: -2146993718 max: 2147378179 sum: 132660742551
+    Column 2: count: 5000 hasNull: false min: -9218342074710552826 max: 9222303228623055266
+    Column 3: count: 5000 hasNull: false min: Darkness, max: worst sum: 19641
+  Stripe 4:
+    Column 0: count: 5000 hasNull: false
+    Column 1: count: 5000 hasNull: false min: -2146658006 max: 2145520931 sum: 8533549236
+    Column 2: count: 5000 hasNull: false min: -9222758097219661129 max: 9221043130193737406
+    Column 3: count: 5000 hasNull: false min: Darkness, max: worst sum: 19470
+  Stripe 5:
+    Column 0: count: 1000 hasNull: false
+    Column 1: count: 1000 hasNull: false min: -2146245500 max: 2146378640 sum: 51299706363
+    Column 2: count: 1000 hasNull: false min: -9208193203370316142 max: 9218567213558056476
+    Column 3: count: 1000 hasNull: false min: Darkness, max: worst sum: 3866
+
+File Statistics:
+  Column 0: count: 21000 hasNull: false
+  Column 1: count: 21000 hasNull: false min: -2146993718 max: 2147378179 sum: 193017464403
+  Column 2: count: 21000 hasNull: false min: -9222758097219661129 max: 9222303228623055266
+  Column 3: count: 21000 hasNull: false min: Darkness, max: worst sum: 81761
+
+Stripes:
+  Stripe: offset: 3 data: 63786 rows: 5000 tail: 85 index: 6974
+    Stream: column 0 section ROW_INDEX start: 3 length 17
+    Stream: column 1 section ROW_INDEX start: 20 length 166
+    Stream: column 2 section ROW_INDEX start: 186 length 169
+    Stream: column 2 section BLOOM_FILTER start: 355 length 6535
+    Stream: column 3 section ROW_INDEX start: 6890 length 87
+    Stream: column 1 section DATA start: 6977 length 20035
+    Stream: column 2 section DATA start: 27012 length 40050
+    Stream: column 3 section DATA start: 67062 length 3543
+    Stream: column 3 section LENGTH start: 70605 length 25
+    Stream: column 3 section DICTIONARY_DATA start: 70630 length 133
+    Encoding column 0: DIRECT
+    Encoding column 1: DIRECT_V2
+    Encoding column 2: DIRECT_V2
+    Encoding column 3: DICTIONARY_V2[35]
+    Row group indices for column 2:
+      Entry 0: count: 1000 hasNull: false min: -9200577545527640566 max: 9175500305011173751 positions: 0,0,0
+      Entry 1: count: 1000 hasNull: false min: -9203618157670445774 max: 9208123824411178101 positions: 4099,2,488
+      Entry 2: count: 1000 hasNull: false min: -9218592812243954469 max: 9221351515892923972 positions: 12297,6,464
+      Entry 3: count: 1000 hasNull: false min: -9206585617947511272 max: 9167703224425685487 positions: 20495,10,440
+      Entry 4: count: 1000 hasNull: false min: -9206645795733282496 max: 9221614132680747961 positions: 28693,14,416
+    Bloom filters for column 2:
+      Entry 0: numHashFunctions: 7 bitCount: 9600 popCount: 4931 loadFactor: 0.5136 expectedFpp: 0.009432924
+      Entry 1: numHashFunctions: 7 bitCount: 9600 popCount: 4956 loadFactor: 0.5163 expectedFpp: 0.009772834
+      Entry 2: numHashFunctions: 7 bitCount: 9600 popCount: 4971 loadFactor: 0.5178 expectedFpp: 0.009981772
+      Entry 3: numHashFunctions: 7 bitCount: 9600 popCount: 4971 loadFactor: 0.5178 expectedFpp: 0.009981772
+      Entry 4: numHashFunctions: 7 bitCount: 9600 popCount: 4949 loadFactor: 0.5155 expectedFpp: 0.009676614
+      Stripe level merge: numHashFunctions: 7 bitCount: 9600 popCount: 9347 loadFactor: 0.9736 expectedFpp: 0.829482
+  Stripe: offset: 70848 data: 63775 rows: 5000 tail: 85 index: 6965
+    Stream: column 0 section ROW_INDEX start: 70848 length 17
+    Stream: column 1 section ROW_INDEX start: 70865 length 164
+    Stream: column 2 section ROW_INDEX start: 71029 length 168
+    Stream: column 2 section BLOOM_FILTER start: 71197 length 6533
+    Stream: column 3 section ROW_INDEX start: 77730 length 83
+    Stream: column 1 section DATA start: 77813 length 20035
+    Stream: column 2 section DATA start: 97848 length 40050
+    Stream: column 3 section DATA start: 137898 length 3532
+    Stream: column 3 section LENGTH start: 141430 length 25
+    Stream: column 3 section DICTIONARY_DATA start: 141455 length 133
+    Encoding column 0: DIRECT
+    Encoding column 1: DIRECT_V2
+    Encoding column 2: DIRECT_V2
+    Encoding column 3: DICTIONARY_V2[35]
+    Row group indices for column 2:
+      Entry 0: count: 1000 hasNull: false min: -9218450653857701562 max: 9189819526332228512 positions: 0,0,0
+      Entry 1: count: 1000 hasNull: false min: -9220818777591257749 max: 9178821722829648113 positions: 4099,2,488
+      Entry 2: count: 1000 hasNull: false min: -9220031433030423388 max: 9210838931786956852 positions: 12297,6,464
+      Entry 3: count: 1000 hasNull: false min: -9208195729739635607 max: 9222259462014003839 positions: 20495,10,440
+      Entry 4: count: 1000 hasNull: false min: -9174271499932339698 max: 9212277876771676916 positions: 28693,14,416
+    Bloom filters for column 2:
+      Entry 0: numHashFunctions: 7 bitCount: 9600 popCount: 4971 loadFactor: 0.5178 expectedFpp: 0.009981772
+      Entry 1: numHashFunctions: 7 bitCount: 9600 popCount: 4988 loadFactor: 0.5196 expectedFpp: 0.010223193
+      Entry 2: numHashFunctions: 7 bitCount: 9600 popCount: 5002 loadFactor: 0.521 expectedFpp: 0.01042575
+      Entry 3: numHashFunctions: 7 bitCount: 9600 popCount: 4962 loadFactor: 0.5169 expectedFpp: 0.009855959
+      Entry 4: numHashFunctions: 7 bitCount: 9600 popCount: 4966 loadFactor: 0.5173 expectedFpp: 0.009911705
+      Stripe level merge: numHashFunctions: 7 bitCount: 9600 popCount: 9344 loadFactor: 0.9733 expectedFpp: 0.8276205
+  Stripe: offset: 141673 data: 63787 rows: 5000 tail: 85 index: 6971
+    Stream: column 0 section ROW_INDEX start: 141673 length 17
+    Stream: column 1 section ROW_INDEX start: 141690 length 163
+    Stream: column 2 section ROW_INDEX start: 141853 length 168
+    Stream: column 2 section BLOOM_FILTER start: 142021 length 6533
+    Stream: column 3 section ROW_INDEX start: 148554 length 90
+    Stream: column 1 section DATA start: 148644 length 20035
+    Stream: column 2 section DATA start: 168679 length 40050
+    Stream: column 3 section DATA start: 208729 length 3544
+    Stream: column 3 section LENGTH start: 212273 length 25
+    Stream: column 3 section DICTIONARY_DATA start: 212298 length 133
+    Encoding column 0: DIRECT
+    Encoding column 1: DIRECT_V2
+    Encoding column 2: DIRECT_V2
+    Encoding column 3: DICTIONARY_V2[35]
+    Row group indices for column 2:
+      Entry 0: count: 1000 hasNull: false min: -9211978436552246208 max: 9179058898902097152 positions: 0,0,0
+      Entry 1: count: 1000 hasNull: false min: -9195645160817780503 max: 9189147759444307708 positions: 4099,2,488
+      Entry 2: count: 1000 hasNull: false min: -9202888157616520823 max: 9193561362676960747 positions: 12297,6,464
+      Entry 3: count: 1000 hasNull: false min: -9216318198067839390 max: 9221286760675829363 positions: 20495,10,440
+      Entry 4: count: 1000 hasNull: false min: -9218342074710552826 max: 9222303228623055266 positions: 28693,14,416
+    Bloom filters for column 2:
+      Entry 0: numHashFunctions: 7 bitCount: 9600 popCount: 4967 loadFactor: 0.5174 expectedFpp: 0.009925688
+      Entry 1: numHashFunctions: 7 bitCount: 9600 popCount: 5002 loadFactor: 0.521 expectedFpp: 0.01042575
+      Entry 2: numHashFunctions: 7 bitCount: 9600 popCount: 4964 loadFactor: 0.5171 expectedFpp: 0.009883798
+      Entry 3: numHashFunctions: 7 bitCount: 9600 popCount: 4943 loadFactor: 0.5149 expectedFpp: 0.009594797
+      Entry 4: numHashFunctions: 7 bitCount: 9600 popCount: 4930 loadFactor: 0.5135 expectedFpp: 0.009419539
+      Stripe level merge: numHashFunctions: 7 bitCount: 9600 popCount: 9333 loadFactor: 0.9722 expectedFpp: 0.82082444
+  Stripe: offset: 212516 data: 63817 rows: 5000 tail: 85 index: 6964
+    Stream: column 0 section ROW_INDEX start: 212516 length 17
+    Stream: column 1 section ROW_INDEX start: 212533 length 165
+    Stream: column 2 section ROW_INDEX start: 212698 length 167
+    Stream: column 2 section BLOOM_FILTER start: 212865 length 6524
+    Stream: column 3 section ROW_INDEX start: 219389 length 91
+    Stream: column 1 section DATA start: 219480 length 20035
+    Stream: column 2 section DATA start: 239515 length 40050
+    Stream: column 3 section DATA start: 279565 length 3574
+    Stream: column 3 section LENGTH start: 283139 length 25
+    Stream: column 3 section DICTIONARY_DATA start: 283164 length 133
+    Encoding column 0: DIRECT
+    Encoding column 1: DIRECT_V2
+    Encoding column 2: DIRECT_V2
+    Encoding column 3: DICTIONARY_V2[35]
+    Row group indices for column 2:
+      Entry 0: count: 1000 hasNull: false min: -9222731174895935707 max: 9214167447015056056 positions: 0,0,0
+      Entry 1: count: 1000 hasNull: false min: -9222758097219661129 max: 9221043130193737406 positions: 4099,2,488
+      Entry 2: count: 1000 hasNull: false min: -9174483776261243438 max: 9208134757538374043 positions: 12297,6,464
+      Entry 3: count: 1000 hasNull: false min: -9174329712613510612 max: 9197412874152820822 positions: 20495,10,440
+      Entry 4: count: 1000 hasNull: false min: -9221162005892422758 max: 9220625004936875965 positions: 28693,14,416
+    Bloom filters for column 2:
+      Entry 0: numHashFunctions: 7 bitCount: 9600 popCount: 4951 loadFactor: 0.5157 expectedFpp: 0.009704026
+      Entry 1: numHashFunctions: 7 bitCount: 9600 popCount: 4969 loadFactor: 0.5176 expectedFpp: 0.009953696
+      Entry 2: numHashFunctions: 7 bitCount: 9600 popCount: 4994 loadFactor: 0.5202 expectedFpp: 0.010309587
+      Entry 3: numHashFunctions: 7 bitCount: 9600 popCount: 4941 loadFactor: 0.5147 expectedFpp: 0.009567649
+      Entry 4: numHashFunctions: 7 bitCount: 9600 popCount: 4993 loadFactor: 0.5201 expectedFpp: 0.010295142
+      Stripe level merge: numHashFunctions: 7 bitCount: 9600 popCount: 9353 loadFactor: 0.9743 expectedFpp: 0.8332165
+  Stripe: offset: 283382 data: 12943 rows: 1000 tail: 78 index: 1468
+    Stream: column 0 section ROW_INDEX start: 283382 length 12
+    Stream: column 1 section ROW_INDEX start: 283394 length 38
+    Stream: column 2 section ROW_INDEX start: 283432 length 41
+    Stream: column 2 section BLOOM_FILTER start: 283473 length 1337
+    Stream: column 3 section ROW_INDEX start: 284810 length 40
+    Stream: column 1 section DATA start: 284850 length 4007
+    Stream: column 2 section DATA start: 288857 length 8010
+    Stream: column 3 section DATA start: 296867 length 768
+    Stream: column 3 section LENGTH start: 297635 length 25
+    Stream: column 3 section DICTIONARY_DATA start: 297660 length 133
+    Encoding column 0: DIRECT
+    Encoding column 1: DIRECT_V2
+    Encoding column 2: DIRECT_V2
+    Encoding column 3: DICTIONARY_V2[35]
+    Row group indices for column 2:
+      Entry 0: count: 1000 hasNull: false min: -9208193203370316142 max: 9218567213558056476 positions: 0,0,0
+    Bloom filters for column 2:
+      Entry 0: numHashFunctions: 7 bitCount: 9600 popCount: 4948 loadFactor: 0.5154 expectedFpp: 0.00966294
+      Stripe level merge: numHashFunctions: 7 bitCount: 9600 popCount: 4948 loadFactor: 0.5154 expectedFpp: 0.00966294
+
+File length: 298416 bytes
+Padding length: 0 bytes
+Padding ratio: 0%
+________________________________________________________________________________________________________________________
+

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/orc/src/test/resources/orc-file-dump-dictionary-threshold.out
----------------------------------------------------------------------
diff --git a/orc/src/test/resources/orc-file-dump-dictionary-threshold.out b/orc/src/test/resources/orc-file-dump-dictionary-threshold.out
new file mode 100644
index 0000000..17a964b
--- /dev/null
+++ b/orc/src/test/resources/orc-file-dump-dictionary-threshold.out
@@ -0,0 +1,190 @@
+Structure for TestFileDump.testDump.orc
+File Version: 0.12 with HIVE_13083
+Rows: 21000
+Compression: ZLIB
+Compression size: 4096
+Type: struct<i:int,l:bigint,s:string>
+
+Stripe Statistics:
+  Stripe 1:
+    Column 0: count: 5000 hasNull: false
+    Column 1: count: 5000 hasNull: false min: -2147115959 max: 2145911404 sum: 159677169195
+    Column 2: count: 5000 hasNull: false min: -9216505819108477308 max: 9217851628057711416
+    Column 3: count: 5000 hasNull: false min: Darkness,-230 max: worst-54-290-346-648-908-996-1038-1080-1560-1584-1620-1744-1770-1798-1852-1966-2162-2244-2286-2296-2534-2660-3114-3676-3788-4068-4150-4706-4744 sum: 381254
+  Stripe 2:
+    Column 0: count: 5000 hasNull: false
+    Column 1: count: 5000 hasNull: false min: -2147390285 max: 2147224606 sum: -14961457759
+    Column 2: count: 5000 hasNull: false min: -9222178666167296739 max: 9221301751385928177
+    Column 3: count: 5000 hasNull: false min: Darkness,-230-368-488-586-862-930-1686-2044-2636-2652-2872-3108-3162-3192-3404-3442-3508-3542-3550-3712-3980-4146-4204-4336-4390-4418-4424-4490-4512-4650-4768-4924-4950-5210 max: worst-54-290-346-648-908-996-1038-1080-1560-1584-1620-1744-1770-1798-1852-1966-2162-2244-2286-2296-2534-2660-3114-3676-3788-4068-4150-4706-4744-5350-5420-5582-5696-5726-6006-6020-6024-6098-6184-6568-6636-6802-6994-7004-7318-7498-7758-7780-7798-7920-7952-7960-7988-8232-8256-8390-8416-8478-8620-8840-8984-9038-9128-9236-9248-9344-9594-9650-9714-9928-9938 sum: 1117994
+  Stripe 3:
+    Column 0: count: 5000 hasNull: false
+    Column 1: count: 5000 hasNull: false min: -2145842720 max: 2146718321 sum: 141092475520
+    Column 2: count: 5000 hasNull: false min: -9221963099397084326 max: 9222722740629726770
+    Column 3: count: 5000 hasNull: false min: Darkness,-230-368-488-586-862-930-1686-2044-2636-2652-2872-3108-3162-3192-3404-3442-3508-3542-3550-3712-3980-4146-4204-4336-4390-4418-4424-4490-4512-4650-4768-4924-4950-5210-5524-5630-5678-5710-5758-5952-6238-6252-6300-6366-6668-6712-6926-6942-7100-7194-7802-8030-8452-8608-8640-8862-8868-9134-9234-9412-9602-9608-9642-9678-9740-9780-10426 max: worst-54-290-346-648-908-996-1038-1080-1560-1584-1620-1744-1770-1798-1852-1966-2162-2244-2286-2296-2534-2660-3114-3676-3788-4068-4150-4706-4744-5350-5420-5582-5696-5726-6006-6020-6024-6098-6184-6568-6636-6802-6994-7004-7318-7498-7758-7780-7798-7920-7952-7960-7988-8232-8256-8390-8416-8478-8620-8840-8984-9038-9128-9236-9248-9344-9594-9650-9714-9928-9938-10178-10368-10414-10502-10732-10876-11008-11158-11410-11722-11836-11964-12054-12096-12126-12136-12202-12246-12298-12616-12774-12782-12790-12802-12976-13216-13246-13502-13766-14454-14974 sum: 1925226
+  Stripe 4:
+    Column 0: count: 5000 hasNull: false
+    Column 1: count: 5000 hasNull: false min: -2145378214 max: 2147453086 sum: -153680004530
+    Column 2: count: 5000 hasNull: false min: -9222731174895935707 max: 9222919052987871506
+    Column 3: count: 5000 hasNull: false min: Darkness,-230-368-488-586-862-930-1686-2044-2636-2652-2872-3108-3162-3192-3404-3442-3508-3542-3550-3712-3980-4146-4204-4336-4390-4418-4424-4490-4512-4650-4768-4924-4950-5210-5524-5630-5678-5710-5758-5952-6238-6252-6300-6366-6668-6712-6926-6942-7100-7194-7802-8030-8452-8608-8640-8862-8868-9134-9234-9412-9602-9608-9642-9678-9740-9780-10426-10510-10514-10706-10814-10870-10942-11028-11244-11326-11462-11496-11656-11830-12022-12178-12418-12832-13304-13448-13590-13618-13908-14188-14246-14340-14364-14394-14762-14850-14964-15048 max: worst-54-290-346-648-908-996-1038-1080-1560-1584-1620-1744-1770-1798-1852-1966-2162-2244-2286-2296-2534-2660-3114-3676-3788-4068-4150-4706-4744-5350-5420-5582-5696-5726-6006-6020-6024-6098-6184-6568-6636-6802-6994-7004-7318-7498-7758-7780-7798-7920-7952-7960-7988-8232-8256-8390-8416-8478-8620-8840-8984-9038-9128-9236-9248-9344-9594-9650-9714-9928-9938-10178-10368-10414-10502-10732-10876-11008-11158-11410-11722-11836-
 11964-12054-12096-12126-12136-12202-12246-12298-12616-12774-12782-12790-12802-12976-13216-13246-13502-13766-14454-14974-15004-15124-15252-15294-15356-15530-15610-16316-16936-17024-17122-17214-17310-17528-17682-17742-17870-17878-18010-18410-18524-18788-19204-19254-19518-19596-19786-19874-19904 sum: 2815002
+  Stripe 5:
+    Column 0: count: 1000 hasNull: false
+    Column 1: count: 1000 hasNull: false min: -2143595397 max: 2136858458 sum: -22999664100
+    Column 2: count: 1000 hasNull: false min: -9212379634781416464 max: 9197412874152820822
+    Column 3: count: 1000 hasNull: false min: Darkness,-230-368-488-586-862-930-1686-2044-2636-2652-2872-3108-3162-3192-3404-3442-3508-3542-3550-3712-3980-4146-4204-4336-4390-4418-4424-4490-4512-4650-4768-4924-4950-5210-5524-5630-5678-5710-5758-5952-6238-6252-6300-6366-6668-6712-6926-6942-7100-7194-7802-8030-8452-8608-8640-8862-8868-9134-9234-9412-9602-9608-9642-9678-9740-9780-10426-10510-10514-10706-10814-10870-10942-11028-11244-11326-11462-11496-11656-11830-12022-12178-12418-12832-13304-13448-13590-13618-13908-14188-14246-14340-14364-14394-14762-14850-14964-15048-15494-15674-15726-16006-16056-16180-16304-16332-16452-16598-16730-16810-16994-17210-17268-17786-17962-18214-18444-18446-18724-18912-18952-19164-19348-19400-19546-19776-19896-20084 max: worst-54-290-346-648-908-996-1038-1080-1560-1584-1620-1744-1770-1798-1852-1966-2162-2244-2286-2296-2534-2660-3114-3676-3788-4068-4150-4706-4744-5350-5420-5582-5696-5726-6006-6020-6024-6098-6184-6568-6636-6802-6994-7004-7318-7498-7758-7780-7
 798-7920-7952-7960-7988-8232-8256-8390-8416-8478-8620-8840-8984-9038-9128-9236-9248-9344-9594-9650-9714-9928-9938-10178-10368-10414-10502-10732-10876-11008-11158-11410-11722-11836-11964-12054-12096-12126-12136-12202-12246-12298-12616-12774-12782-12790-12802-12976-13216-13246-13502-13766-14454-14974-15004-15124-15252-15294-15356-15530-15610-16316-16936-17024-17122-17214-17310-17528-17682-17742-17870-17878-18010-18410-18524-18788-19204-19254-19518-19596-19786-19874-19904-20390-20752-20936 sum: 670762
+
+File Statistics:
+  Column 0: count: 21000 hasNull: false
+  Column 1: count: 21000 hasNull: false min: -2147390285 max: 2147453086 sum: 109128518326
+  Column 2: count: 21000 hasNull: false min: -9222731174895935707 max: 9222919052987871506
+  Column 3: count: 21000 hasNull: false min: Darkness,-230 max: worst-54-290-346-648-908-996-1038-1080-1560-1584-1620-1744-1770-1798-1852-1966-2162-2244-2286-2296-2534-2660-3114-3676-3788-4068-4150-4706-4744-5350-5420-5582-5696-5726-6006-6020-6024-6098-6184-6568-6636-6802-6994-7004-7318-7498-7758-7780-7798-7920-7952-7960-7988-8232-8256-8390-8416-8478-8620-8840-8984-9038-9128-9236-9248-9344-9594-9650-9714-9928-9938-10178-10368-10414-10502-10732-10876-11008-11158-11410-11722-11836-11964-12054-12096-12126-12136-12202-12246-12298-12616-12774-12782-12790-12802-12976-13216-13246-13502-13766-14454-14974-15004-15124-15252-15294-15356-15530-15610-16316-16936-17024-17122-17214-17310-17528-17682-17742-17870-17878-18010-18410-18524-18788-19204-19254-19518-19596-19786-19874-19904-20390-20752-20936 sum: 6910238
+
+Stripes:
+  Stripe: offset: 3 data: 163602 rows: 5000 tail: 68 index: 720
+    Stream: column 0 section ROW_INDEX start: 3 length 17
+    Stream: column 1 section ROW_INDEX start: 20 length 166
+    Stream: column 2 section ROW_INDEX start: 186 length 171
+    Stream: column 3 section ROW_INDEX start: 357 length 366
+    Stream: column 1 section DATA start: 723 length 20035
+    Stream: column 2 section DATA start: 20758 length 40050
+    Stream: column 3 section DATA start: 60808 length 99226
+    Stream: column 3 section LENGTH start: 160034 length 4291
+    Encoding column 0: DIRECT
+    Encoding column 1: DIRECT_V2
+    Encoding column 2: DIRECT_V2
+    Encoding column 3: DIRECT_V2
+    Row group indices for column 1:
+      Entry 0: count: 1000 hasNull: false min: -2132329551 max: 2145911404 sum: 61941331718 positions: 0,0,0
+      Entry 1: count: 1000 hasNull: false min: -2138433136 max: 2145210552 sum: 14574030042 positions: 0,2050,488
+      Entry 2: count: 1000 hasNull: false min: -2147115959 max: 2137805337 sum: -2032493169 positions: 4099,2054,464
+      Entry 3: count: 1000 hasNull: false min: -2137828953 max: 2145877119 sum: -3167202608 positions: 8198,2058,440
+      Entry 4: count: 1000 hasNull: false min: -2146452517 max: 2142394906 sum: 88361503212 positions: 12297,2062,416
+    Row group indices for column 2:
+      Entry 0: count: 1000 hasNull: false min: -9206837518492372266 max: 9169230975203934579 positions: 0,0,0
+      Entry 1: count: 1000 hasNull: false min: -9188878639954124284 max: 9213664245516510068 positions: 4099,2,488
+      Entry 2: count: 1000 hasNull: false min: -9211329013123260308 max: 9217851628057711416 positions: 12297,6,464
+      Entry 3: count: 1000 hasNull: false min: -9185745718227889962 max: 9181722705210917931 positions: 20495,10,440
+      Entry 4: count: 1000 hasNull: false min: -9216505819108477308 max: 9196474183833079923 positions: 28693,14,416
+    Row group indices for column 3:
+      Entry 0: count: 1000 hasNull: false min: Darkness,-230 max: worst-54-290-346-648-908-996 sum: 18442 positions: 0,0,0,0,0
+      Entry 1: count: 1000 hasNull: false min: Darkness,-230-368-488-586-862-930-1686 max: worst-54-290-346-648-908-996-1038-1080-1560-1584-1620-1744-1770-1798-1852-1966 sum: 46338 positions: 4767,2058,0,695,18
+      Entry 2: count: 1000 hasNull: false min: Darkness,-230-368-488-586-862-930-1686-2044 max: worst-54-290-346-648-908-996-1038-1080-1560-1584-1620-1744-1770-1798-1852-1966-2162-2244-2286-2296-2534-2660 sum: 75448 positions: 16464,3340,0,1554,14
+      Entry 3: count: 1000 hasNull: false min: Darkness,-230-368-488-586-862-930-1686-2044-2636-2652-2872-3108 max: worst-54-290-346-648-908-996-1038-1080-1560-1584-1620-1744-1770-1798-1852-1966-2162-2244-2286-2296-2534-2660-3114-3676-3788 sum: 104868 positions: 36532,964,0,2372,90
+      Entry 4: count: 1000 hasNull: false min: Darkness,-230-368-488-586-862-930-1686-2044-2636-2652-2872-3108-3162-3192-3404-3442-3508-3542-3550-3712-3980-4146 max: worst-54-290-346-648-908-996-1038-1080-1560-1584-1620-1744-1770-1798-1852-1966-2162-2244-2286-2296-2534-2660-3114-3676-3788-4068-4150-4706-4744 sum: 136158 positions: 63067,3432,0,3354,108
+  Stripe: offset: 164393 data: 368335 rows: 5000 tail: 69 index: 956
+    Stream: column 0 section ROW_INDEX start: 164393 length 17
+    Stream: column 1 section ROW_INDEX start: 164410 length 157
+    Stream: column 2 section ROW_INDEX start: 164567 length 166
+    Stream: column 3 section ROW_INDEX start: 164733 length 616
+    Stream: column 1 section DATA start: 165349 length 20035
+    Stream: column 2 section DATA start: 185384 length 40050
+    Stream: column 3 section DATA start: 225434 length 302715
+    Stream: column 3 section LENGTH start: 528149 length 5535
+    Encoding column 0: DIRECT
+    Encoding column 1: DIRECT_V2
+    Encoding column 2: DIRECT_V2
+    Encoding column 3: DIRECT_V2
+    Row group indices for column 1:
+      Entry 0: count: 1000 hasNull: false min: -2146021688 max: 2146838901 sum: -50979197646 positions: 0,0,0
+      Entry 1: count: 1000 hasNull: false min: -2143569489 max: 2141223179 sum: 22810066834 positions: 0,2050,488
+      Entry 2: count: 1000 hasNull: false min: -2140649392 max: 2146301701 sum: -31694882346 positions: 4099,2054,464
+      Entry 3: count: 1000 hasNull: false min: -2147390285 max: 2146299933 sum: 79371934221 positions: 8198,2058,440
+      Entry 4: count: 1000 hasNull: false min: -2145928262 max: 2147224606 sum: -34469378822 positions: 12297,2062,416
+    Row group indices for column 2:
+      Entry 0: count: 1000 hasNull: false min: -9222178666167296739 max: 9191250610515369723 positions: 0,0,0
+      Entry 1: count: 1000 hasNull: false min: -9220148577547102875 max: 9213945522531717278 positions: 4099,2,488
+      Entry 2: count: 1000 hasNull: false min: -9220818777591257749 max: 9221301751385928177 positions: 12297,6,464
+      Entry 3: count: 1000 hasNull: false min: -9220031433030423388 max: 9207856144487414148 positions: 20495,10,440
+      Entry 4: count: 1000 hasNull: false min: -9201438531577205959 max: 9212462124593119846 positions: 28693,14,416
+    Row group indices for column 3:
+      Entry 0: count: 1000 hasNull: false min: Darkness,-230-368-488-586-862-930-1686-2044-2636-2652-2872-3108-3162-3192-3404-3442-3508-3542-3550-3712-3980-4146-4204-4336-4390-4418-4424-4490-4512-4650-4768-4924-4950-5210 max: worst-54-290-346-648-908-996-1038-1080-1560-1584-1620-1744-1770-1798-1852-1966-2162-2244-2286-2296-2534-2660-3114-3676-3788-4068-4150-4706-4744-5350-5420-5582-5696-5726 sum: 166320 positions: 0,0,0,0,0
+      Entry 1: count: 1000 hasNull: false min: Darkness,-230-368-488-586-862-930-1686-2044-2636-2652-2872-3108-3162-3192-3404-3442-3508-3542-3550-3712-3980-4146-4204-4336-4390-4418-4424-4490-4512-4650-4768-4924-4950-5210-5524-5630-5678-5710-5758-5952-6238 max: worst-54-290-346-648-908-996-1038-1080-1560-1584-1620-1744-1770-1798-1852-1966-2162-2244-2286-2296-2534-2660-3114-3676-3788-4068-4150-4706-4744-5350-5420-5582-5696-5726-6006-6020-6024-6098-6184-6568-6636-6802-6994 sum: 193436 positions: 43833,2480,0,967,90
+      Entry 2: count: 1000 hasNull: false min: Darkness,-230-368-488-586-862-930-1686-2044-2636-2652-2872-3108-3162-3192-3404-3442-3508-3542-3550-3712-3980-4146-4204-4336-4390-4418-4424-4490-4512-4650-4768-4924-4950-5210-5524-5630-5678-5710-5758-5952-6238-6252-6300-6366-6668-6712-6926-6942-7100 max: worst-54-290-346-648-908-996-1038-1080-1560-1584-1620-1744-1770-1798-1852-1966-2162-2244-2286-2296-2534-2660-3114-3676-3788-4068-4150-4706-4744-5350-5420-5582-5696-5726-6006-6020-6024-6098-6184-6568-6636-6802-6994-7004-7318-7498-7758-7780-7798-7920-7952-7960-7988 sum: 224740 positions: 94117,3404,0,1945,222
+      Entry 3: count: 1000 hasNull: false min: Darkness,-230-368-488-586-862-930-1686-2044-2636-2652-2872-3108-3162-3192-3404-3442-3508-3542-3550-3712-3980-4146-4204-4336-4390-4418-4424-4490-4512-4650-4768-4924-4950-5210-5524-5630-5678-5710-5758-5952-6238-6252-6300-6366-6668-6712-6926-6942-7100-7194-7802-8030 max: worst-54-290-346-648-908-996-1038-1080-1560-1584-1620-1744-1770-1798-1852-1966-2162-2244-2286-2296-2534-2660-3114-3676-3788-4068-4150-4706-4744-5350-5420-5582-5696-5726-6006-6020-6024-6098-6184-6568-6636-6802-6994-7004-7318-7498-7758-7780-7798-7920-7952-7960-7988-8232-8256-8390-8416-8478-8620-8840-8984 sum: 252094 positions: 155111,2864,0,3268,48
+      Entry 4: count: 1000 hasNull: false min: Darkness,-230-368-488-586-862-930-1686-2044-2636-2652-2872-3108-3162-3192-3404-3442-3508-3542-3550-3712-3980-4146-4204-4336-4390-4418-4424-4490-4512-4650-4768-4924-4950-5210-5524-5630-5678-5710-5758-5952-6238-6252-6300-6366-6668-6712-6926-6942-7100-7194-7802-8030-8452-8608-8640-8862-8868-9134 max: worst-54-290-346-648-908-996-1038-1080-1560-1584-1620-1744-1770-1798-1852-1966-2162-2244-2286-2296-2534-2660-3114-3676-3788-4068-4150-4706-4744-5350-5420-5582-5696-5726-6006-6020-6024-6098-6184-6568-6636-6802-6994-7004-7318-7498-7758-7780-7798-7920-7952-7960-7988-8232-8256-8390-8416-8478-8620-8840-8984-9038-9128-9236-9248-9344-9594-9650-9714-9928-9938 sum: 281404 positions: 224570,1006,0,4064,342
+  Stripe: offset: 533753 data: 606074 rows: 5000 tail: 69 index: 1427
+    Stream: column 0 section ROW_INDEX start: 533753 length 17
+    Stream: column 1 section ROW_INDEX start: 533770 length 167
+    Stream: column 2 section ROW_INDEX start: 533937 length 168
+    Stream: column 3 section ROW_INDEX start: 534105 length 1075
+    Stream: column 1 section DATA start: 535180 length 20035
+    Stream: column 2 section DATA start: 555215 length 40050
+    Stream: column 3 section DATA start: 595265 length 540210
+    Stream: column 3 section LENGTH start: 1135475 length 5779
+    Encoding column 0: DIRECT
+    Encoding column 1: DIRECT_V2
+    Encoding column 2: DIRECT_V2
+    Encoding column 3: DIRECT_V2
+    Row group indices for column 1:
+      Entry 0: count: 1000 hasNull: false min: -2138229212 max: 2144818981 sum: -22823642812 positions: 0,0,0
+      Entry 1: count: 1000 hasNull: false min: -2145842720 max: 2144179881 sum: -12562754334 positions: 0,2050,488
+      Entry 2: count: 1000 hasNull: false min: -2143045885 max: 2146718321 sum: 82993638644 positions: 4099,2054,464
+      Entry 3: count: 1000 hasNull: false min: -2144745617 max: 2146570474 sum: 25138722367 positions: 8198,2058,440
+      Entry 4: count: 1000 hasNull: false min: -2140127150 max: 2135081620 sum: 68346511655 positions: 12297,2062,416
+    Row group indices for column 2:
+      Entry 0: count: 1000 hasNull: false min: -9204340807292138409 max: 9208698732685326961 positions: 0,0,0
+      Entry 1: count: 1000 hasNull: false min: -9221963099397084326 max: 9222722740629726770 positions: 4099,2,488
+      Entry 2: count: 1000 hasNull: false min: -9210480084701091299 max: 9207767402467343058 positions: 12297,6,464
+      Entry 3: count: 1000 hasNull: false min: -9195038026813631215 max: 9199201928563274421 positions: 20495,10,440
+      Entry 4: count: 1000 hasNull: false min: -9215483580266514322 max: 9220102792864959501 positions: 28693,14,416
+    Row group indices for column 3:
+      Entry 0: count: 1000 hasNull: false min: Darkness,-230-368-488-586-862-930-1686-2044-2636-2652-2872-3108-3162-3192-3404-3442-3508-3542-3550-3712-3980-4146-4204-4336-4390-4418-4424-4490-4512-4650-4768-4924-4950-5210-5524-5630-5678-5710-5758-5952-6238-6252-6300-6366-6668-6712-6926-6942-7100-7194-7802-8030-8452-8608-8640-8862-8868-9134-9234-9412-9602-9608-9642-9678-9740-9780-10426 max: worst-54-290-346-648-908-996-1038-1080-1560-1584-1620-1744-1770-1798-1852-1966-2162-2244-2286-2296-2534-2660-3114-3676-3788-4068-4150-4706-4744-5350-5420-5582-5696-5726-6006-6020-6024-6098-6184-6568-6636-6802-6994-7004-7318-7498-7758-7780-7798-7920-7952-7960-7988-8232-8256-8390-8416-8478-8620-8840-8984-9038-9128-9236-9248-9344-9594-9650-9714-9928-9938-10178-10368-10414-10502-10732-10876 sum: 313880 positions: 0,0,0,0,0
+      Entry 1: count: 1000 hasNull: false min: Darkness,-230-368-488-586-862-930-1686-2044-2636-2652-2872-3108-3162-3192-3404-3442-3508-3542-3550-3712-3980-4146-4204-4336-4390-4418-4424-4490-4512-4650-4768-4924-4950-5210-5524-5630-5678-5710-5758-5952-6238-6252-6300-6366-6668-6712-6926-6942-7100-7194-7802-8030-8452-8608-8640-8862-8868-9134-9234-9412-9602-9608-9642-9678-9740-9780-10426-10510-10514-10706-10814-10870-10942-11028 max: worst-54-290-346-648-908-996-1038-1080-1560-1584-1620-1744-1770-1798-1852-1966-2162-2244-2286-2296-2534-2660-3114-3676-3788-4068-4150-4706-4744-5350-5420-5582-5696-5726-6006-6020-6024-6098-6184-6568-6636-6802-6994-7004-7318-7498-7758-7780-7798-7920-7952-7960-7988-8232-8256-8390-8416-8478-8620-8840-8984-9038-9128-9236-9248-9344-9594-9650-9714-9928-9938-10178-10368-10414-10502-10732-10876-11008-11158-11410-11722-11836-11964 sum: 349542 positions: 87800,2584,0,1097,28
+      Entry 2: count: 1000 hasNull: false min: Darkness,-230-368-488-586-862-930-1686-2044-2636-2652-2872-3108-3162-3192-3404-3442-3508-3542-3550-3712-3980-4146-4204-4336-4390-4418-4424-4490-4512-4650-4768-4924-4950-5210-5524-5630-5678-5710-5758-5952-6238-6252-6300-6366-6668-6712-6926-6942-7100-7194-7802-8030-8452-8608-8640-8862-8868-9134-9234-9412-9602-9608-9642-9678-9740-9780-10426-10510-10514-10706-10814-10870-10942-11028-11244-11326-11462-11496-11656-11830-12022 max: worst-54-290-346-648-908-996-1038-1080-1560-1584-1620-1744-1770-1798-1852-1966-2162-2244-2286-2296-2534-2660-3114-3676-3788-4068-4150-4706-4744-5350-5420-5582-5696-5726-6006-6020-6024-6098-6184-6568-6636-6802-6994-7004-7318-7498-7758-7780-7798-7920-7952-7960-7988-8232-8256-8390-8416-8478-8620-8840-8984-9038-9128-9236-9248-9344-9594-9650-9714-9928-9938-10178-10368-10414-10502-10732-10876-11008-11158-11410-11722-11836-11964-12054-12096-12126-12136-12202-12246-12298-12616-12774-12782-12790-12802-12976 sum: 386538 posit
 ions: 185635,3966,0,2077,162
+      Entry 3: count: 1000 hasNull: false min: Darkness,-230-368-488-586-862-930-1686-2044-2636-2652-2872-3108-3162-3192-3404-3442-3508-3542-3550-3712-3980-4146-4204-4336-4390-4418-4424-4490-4512-4650-4768-4924-4950-5210-5524-5630-5678-5710-5758-5952-6238-6252-6300-6366-6668-6712-6926-6942-7100-7194-7802-8030-8452-8608-8640-8862-8868-9134-9234-9412-9602-9608-9642-9678-9740-9780-10426-10510-10514-10706-10814-10870-10942-11028-11244-11326-11462-11496-11656-11830-12022-12178-12418-12832-13304 max: worst-54-290-346-648-908-996-1038-1080-1560-1584-1620-1744-1770-1798-1852-1966-2162-2244-2286-2296-2534-2660-3114-3676-3788-4068-4150-4706-4744-5350-5420-5582-5696-5726-6006-6020-6024-6098-6184-6568-6636-6802-6994-7004-7318-7498-7758-7780-7798-7920-7952-7960-7988-8232-8256-8390-8416-8478-8620-8840-8984-9038-9128-9236-9248-9344-9594-9650-9714-9928-9938-10178-10368-10414-10502-10732-10876-11008-11158-11410-11722-11836-11964-12054-12096-12126-12136-12202-12246-12298-12616-12774-12782-12790-12802
 -12976-13216-13246-13502-13766 sum: 421660 positions: 295550,1384,0,3369,16
+      Entry 4: count: 1000 hasNull: false min: Darkness,-230-368-488-586-862-930-1686-2044-2636-2652-2872-3108-3162-3192-3404-3442-3508-3542-3550-3712-3980-4146-4204-4336-4390-4418-4424-4490-4512-4650-4768-4924-4950-5210-5524-5630-5678-5710-5758-5952-6238-6252-6300-6366-6668-6712-6926-6942-7100-7194-7802-8030-8452-8608-8640-8862-8868-9134-9234-9412-9602-9608-9642-9678-9740-9780-10426-10510-10514-10706-10814-10870-10942-11028-11244-11326-11462-11496-11656-11830-12022-12178-12418-12832-13304-13448-13590-13618-13908-14188 max: worst-54-290-346-648-908-996-1038-1080-1560-1584-1620-1744-1770-1798-1852-1966-2162-2244-2286-2296-2534-2660-3114-3676-3788-4068-4150-4706-4744-5350-5420-5582-5696-5726-6006-6020-6024-6098-6184-6568-6636-6802-6994-7004-7318-7498-7758-7780-7798-7920-7952-7960-7988-8232-8256-8390-8416-8478-8620-8840-8984-9038-9128-9236-9248-9344-9594-9650-9714-9928-9938-10178-10368-10414-10502-10732-10876-11008-11158-11410-11722-11836-11964-12054-12096-12126-12136-12202-12246-12298
 -12616-12774-12782-12790-12802-12976-13216-13246-13502-13766-14454-14974 sum: 453606 positions: 412768,1156,0,4041,470
+  Stripe: offset: 1141323 data: 864001 rows: 5000 tail: 69 index: 1975
+    Stream: column 0 section ROW_INDEX start: 1141323 length 17
+    Stream: column 1 section ROW_INDEX start: 1141340 length 156
+    Stream: column 2 section ROW_INDEX start: 1141496 length 168
+    Stream: column 3 section ROW_INDEX start: 1141664 length 1634
+    Stream: column 1 section DATA start: 1143298 length 20035
+    Stream: column 2 section DATA start: 1163333 length 40050
+    Stream: column 3 section DATA start: 1203383 length 798014
+    Stream: column 3 section LENGTH start: 2001397 length 5902
+    Encoding column 0: DIRECT
+    Encoding column 1: DIRECT_V2
+    Encoding column 2: DIRECT_V2
+    Encoding column 3: DIRECT_V2
+    Row group indices for column 1:
+      Entry 0: count: 1000 hasNull: false min: -2145319330 max: 2146998132 sum: -50856753363 positions: 0,0,0
+      Entry 1: count: 1000 hasNull: false min: -2134288866 max: 2147453086 sum: -17911019023 positions: 0,2050,488
+      Entry 2: count: 1000 hasNull: false min: -2139010804 max: 2144727593 sum: -24993151857 positions: 4099,2054,464
+      Entry 3: count: 1000 hasNull: false min: -2145378214 max: 2144098933 sum: -18055164052 positions: 8198,2058,440
+      Entry 4: count: 1000 hasNull: false min: -2140494429 max: 2144595861 sum: -41863916235 positions: 12297,2062,416
+    Row group indices for column 2:
+      Entry 0: count: 1000 hasNull: false min: -9172774601303513941 max: 9212917101275642143 positions: 0,0,0
+      Entry 1: count: 1000 hasNull: false min: -9218164880949195469 max: 9222919052987871506 positions: 4099,2,488
+      Entry 2: count: 1000 hasNull: false min: -9222731174895935707 max: 9214167447015056056 positions: 12297,6,464
+      Entry 3: count: 1000 hasNull: false min: -9196276654247395117 max: 9210639275226058005 positions: 20495,10,440
+      Entry 4: count: 1000 hasNull: false min: -9197393848859294562 max: 9208134757538374043 positions: 28693,14,416
+    Row group indices for column 3:
+      Entry 0: count: 1000 hasNull: false min: Darkness,-230-368-488-586-862-930-1686-2044-2636-2652-2872-3108-3162-3192-3404-3442-3508-3542-3550-3712-3980-4146-4204-4336-4390-4418-4424-4490-4512-4650-4768-4924-4950-5210-5524-5630-5678-5710-5758-5952-6238-6252-6300-6366-6668-6712-6926-6942-7100-7194-7802-8030-8452-8608-8640-8862-8868-9134-9234-9412-9602-9608-9642-9678-9740-9780-10426-10510-10514-10706-10814-10870-10942-11028-11244-11326-11462-11496-11656-11830-12022-12178-12418-12832-13304-13448-13590-13618-13908-14188-14246-14340-14364-14394-14762-14850-14964-15048 max: worst-54-290-346-648-908-996-1038-1080-1560-1584-1620-1744-1770-1798-1852-1966-2162-2244-2286-2296-2534-2660-3114-3676-3788-4068-4150-4706-4744-5350-5420-5582-5696-5726-6006-6020-6024-6098-6184-6568-6636-6802-6994-7004-7318-7498-7758-7780-7798-7920-7952-7960-7988-8232-8256-8390-8416-8478-8620-8840-8984-9038-9128-9236-9248-9344-9594-9650-9714-9928-9938-10178-10368-10414-10502-10732-10876-11008-11158-11410-11722-11836
 -11964-12054-12096-12126-12136-12202-12246-12298-12616-12774-12782-12790-12802-12976-13216-13246-13502-13766-14454-14974-15004-15124-15252-15294-15356-15530-15610 sum: 492916 positions: 0,0,0,0,0
+      Entry 1: count: 1000 hasNull: false min: Darkness,-230-368-488-586-862-930-1686-2044-2636-2652-2872-3108-3162-3192-3404-3442-3508-3542-3550-3712-3980-4146-4204-4336-4390-4418-4424-4490-4512-4650-4768-4924-4950-5210-5524-5630-5678-5710-5758-5952-6238-6252-6300-6366-6668-6712-6926-6942-7100-7194-7802-8030-8452-8608-8640-8862-8868-9134-9234-9412-9602-9608-9642-9678-9740-9780-10426-10510-10514-10706-10814-10870-10942-11028-11244-11326-11462-11496-11656-11830-12022-12178-12418-12832-13304-13448-13590-13618-13908-14188-14246-14340-14364-14394-14762-14850-14964-15048-15494-15674-15726-16006 max: worst-54-290-346-648-908-996-1038-1080-1560-1584-1620-1744-1770-1798-1852-1966-2162-2244-2286-2296-2534-2660-3114-3676-3788-4068-4150-4706-4744-5350-5420-5582-5696-5726-6006-6020-6024-6098-6184-6568-6636-6802-6994-7004-7318-7498-7758-7780-7798-7920-7952-7960-7988-8232-8256-8390-8416-8478-8620-8840-8984-9038-9128-9236-9248-9344-9594-9650-9714-9928-9938-10178-10368-10414-10502-10732-10876-11008
 -11158-11410-11722-11836-11964-12054-12096-12126-12136-12202-12246-12298-12616-12774-12782-12790-12802-12976-13216-13246-13502-13766-14454-14974-15004-15124-15252-15294-15356-15530-15610-16316-16936 sum: 527290 positions: 139298,1396,0,1077,140
+      Entry 2: count: 1000 hasNull: false min: Darkness,-230-368-488-586-862-930-1686-2044-2636-2652-2872-3108-3162-3192-3404-3442-3508-3542-3550-3712-3980-4146-4204-4336-4390-4418-4424-4490-4512-4650-4768-4924-4950-5210-5524-5630-5678-5710-5758-5952-6238-6252-6300-6366-6668-6712-6926-6942-7100-7194-7802-8030-8452-8608-8640-8862-8868-9134-9234-9412-9602-9608-9642-9678-9740-9780-10426-10510-10514-10706-10814-10870-10942-11028-11244-11326-11462-11496-11656-11830-12022-12178-12418-12832-13304-13448-13590-13618-13908-14188-14246-14340-14364-14394-14762-14850-14964-15048-15494-15674-15726-16006-16056-16180-16304-16332-16452-16598-16730-16810-16994-17210 max: worst-54-290-346-648-908-996-1038-1080-1560-1584-1620-1744-1770-1798-1852-1966-2162-2244-2286-2296-2534-2660-3114-3676-3788-4068-4150-4706-4744-5350-5420-5582-5696-5726-6006-6020-6024-6098-6184-6568-6636-6802-6994-7004-7318-7498-7758-7780-7798-7920-7952-7960-7988-8232-8256-8390-8416-8478-8620-8840-8984-9038-9128-9236-9248-9344-9594-9
 650-9714-9928-9938-10178-10368-10414-10502-10732-10876-11008-11158-11410-11722-11836-11964-12054-12096-12126-12136-12202-12246-12298-12616-12774-12782-12790-12802-12976-13216-13246-13502-13766-14454-14974-15004-15124-15252-15294-15356-15530-15610-16316-16936-17024-17122-17214-17310-17528-17682-17742-17870-17878 sum: 568274 positions: 286457,302,0,1926,462
+      Entry 3: count: 1000 hasNull: false min: Darkness,-230-368-488-586-862-930-1686-2044-2636-2652-2872-3108-3162-3192-3404-3442-3508-3542-3550-3712-3980-4146-4204-4336-4390-4418-4424-4490-4512-4650-4768-4924-4950-5210-5524-5630-5678-5710-5758-5952-6238-6252-6300-6366-6668-6712-6926-6942-7100-7194-7802-8030-8452-8608-8640-8862-8868-9134-9234-9412-9602-9608-9642-9678-9740-9780-10426-10510-10514-10706-10814-10870-10942-11028-11244-11326-11462-11496-11656-11830-12022-12178-12418-12832-13304-13448-13590-13618-13908-14188-14246-14340-14364-14394-14762-14850-14964-15048-15494-15674-15726-16006-16056-16180-16304-16332-16452-16598-16730-16810-16994-17210-17268-17786-17962-18214 max: worst-54-290-346-648-908-996-1038-1080-1560-1584-1620-1744-1770-1798-1852-1966-2162-2244-2286-2296-2534-2660-3114-3676-3788-4068-4150-4706-4744-5350-5420-5582-5696-5726-6006-6020-6024-6098-6184-6568-6636-6802-6994-7004-7318-7498-7758-7780-7798-7920-7952-7960-7988-8232-8256-8390-8416-8478-8620-8840-8984-9038-91
 28-9236-9248-9344-9594-9650-9714-9928-9938-10178-10368-10414-10502-10732-10876-11008-11158-11410-11722-11836-11964-12054-12096-12126-12136-12202-12246-12298-12616-12774-12782-12790-12802-12976-13216-13246-13502-13766-14454-14974-15004-15124-15252-15294-15356-15530-15610-16316-16936-17024-17122-17214-17310-17528-17682-17742-17870-17878-18010-18410-18524-18788 sum: 594578 positions: 447943,3328,0,3444,250
+      Entry 4: count: 1000 hasNull: false min: Darkness,-230-368-488-586-862-930-1686-2044-2636-2652-2872-3108-3162-3192-3404-3442-3508-3542-3550-3712-3980-4146-4204-4336-4390-4418-4424-4490-4512-4650-4768-4924-4950-5210-5524-5630-5678-5710-5758-5952-6238-6252-6300-6366-6668-6712-6926-6942-7100-7194-7802-8030-8452-8608-8640-8862-8868-9134-9234-9412-9602-9608-9642-9678-9740-9780-10426-10510-10514-10706-10814-10870-10942-11028-11244-11326-11462-11496-11656-11830-12022-12178-12418-12832-13304-13448-13590-13618-13908-14188-14246-14340-14364-14394-14762-14850-14964-15048-15494-15674-15726-16006-16056-16180-16304-16332-16452-16598-16730-16810-16994-17210-17268-17786-17962-18214-18444-18446-18724-18912-18952-19164 max: worst-54-290-346-648-908-996-1038-1080-1560-1584-1620-1744-1770-1798-1852-1966-2162-2244-2286-2296-2534-2660-3114-3676-3788-4068-4150-4706-4744-5350-5420-5582-5696-5726-6006-6020-6024-6098-6184-6568-6636-6802-6994-7004-7318-7498-7758-7780-7798-7920-7952-7960-7988-8232-8256-8
 390-8416-8478-8620-8840-8984-9038-9128-9236-9248-9344-9594-9650-9714-9928-9938-10178-10368-10414-10502-10732-10876-11008-11158-11410-11722-11836-11964-12054-12096-12126-12136-12202-12246-12298-12616-12774-12782-12790-12802-12976-13216-13246-13502-13766-14454-14974-15004-15124-15252-15294-15356-15530-15610-16316-16936-17024-17122-17214-17310-17528-17682-17742-17870-17878-18010-18410-18524-18788-19204-19254-19518-19596-19786-19874-19904 sum: 631944 positions: 616471,3986,3778,547,292
+  Stripe: offset: 2007368 data: 207295 rows: 1000 tail: 67 index: 841
+    Stream: column 0 section ROW_INDEX start: 2007368 length 12
+    Stream: column 1 section ROW_INDEX start: 2007380 length 38
+    Stream: column 2 section ROW_INDEX start: 2007418 length 41
+    Stream: column 3 section ROW_INDEX start: 2007459 length 750
+    Stream: column 1 section DATA start: 2008209 length 4007
+    Stream: column 2 section DATA start: 2012216 length 8010
+    Stream: column 3 section DATA start: 2020226 length 194018
+    Stream: column 3 section LENGTH start: 2214244 length 1260
+    Encoding column 0: DIRECT
+    Encoding column 1: DIRECT_V2
+    Encoding column 2: DIRECT_V2
+    Encoding column 3: DIRECT_V2
+    Row group indices for column 1:
+      Entry 0: count: 1000 hasNull: false min: -2143595397 max: 2136858458 sum: -22999664100 positions: 0,0,0
+    Row group indices for column 2:
+      Entry 0: count: 1000 hasNull: false min: -9212379634781416464 max: 9197412874152820822 positions: 0,0,0
+    Row group indices for column 3:
+      Entry 0: count: 1000 hasNull: false min: Darkness,-230-368-488-586-862-930-1686-2044-2636-2652-2872-3108-3162-3192-3404-3442-3508-3542-3550-3712-3980-4146-4204-4336-4390-4418-4424-4490-4512-4650-4768-4924-4950-5210-5524-5630-5678-5710-5758-5952-6238-6252-6300-6366-6668-6712-6926-6942-7100-7194-7802-8030-8452-8608-8640-8862-8868-9134-9234-9412-9602-9608-9642-9678-9740-9780-10426-10510-10514-10706-10814-10870-10942-11028-11244-11326-11462-11496-11656-11830-12022-12178-12418-12832-13304-13448-13590-13618-13908-14188-14246-14340-14364-14394-14762-14850-14964-15048-15494-15674-15726-16006-16056-16180-16304-16332-16452-16598-16730-16810-16994-17210-17268-17786-17962-18214-18444-18446-18724-18912-18952-19164-19348-19400-19546-19776-19896-20084 max: worst-54-290-346-648-908-996-1038-1080-1560-1584-1620-1744-1770-1798-1852-1966-2162-2244-2286-2296-2534-2660-3114-3676-3788-4068-4150-4706-4744-5350-5420-5582-5696-5726-6006-6020-6024-6098-6184-6568-6636-6802-6994-7004-7318-7498-7758-7780-
 7798-7920-7952-7960-7988-8232-8256-8390-8416-8478-8620-8840-8984-9038-9128-9236-9248-9344-9594-9650-9714-9928-9938-10178-10368-10414-10502-10732-10876-11008-11158-11410-11722-11836-11964-12054-12096-12126-12136-12202-12246-12298-12616-12774-12782-12790-12802-12976-13216-13246-13502-13766-14454-14974-15004-15124-15252-15294-15356-15530-15610-16316-16936-17024-17122-17214-17310-17528-17682-17742-17870-17878-18010-18410-18524-18788-19204-19254-19518-19596-19786-19874-19904-20390-20752-20936 sum: 670762 positions: 0,0,0,0,0
+
+File length: 2217685 bytes
+Padding length: 0 bytes
+Padding ratio: 0%
+________________________________________________________________________________________________________________________
+


[10/27] hive git commit: HIVE-11417. Move the ReaderImpl and RowReaderImpl to the ORC module, by making shims for the row by row reader. (omalley reviewed by prasanth_j)

Posted by om...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/ql/src/java/org/apache/hadoop/hive/ql/io/orc/TreeReaderFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/TreeReaderFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/TreeReaderFactory.java
deleted file mode 100644
index 6d1c256..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/TreeReaderFactory.java
+++ /dev/null
@@ -1,2525 +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.hive.ql.io.orc;
-
-import java.io.EOFException;
-import java.io.IOException;
-import java.math.BigInteger;
-import java.sql.Timestamp;
-import java.text.ParseException;
-import java.text.SimpleDateFormat;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.TimeZone;
-
-import org.apache.hadoop.hive.common.type.HiveDecimal;
-import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.ListColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.MapColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.StructColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.UnionColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.StringExpr;
-import org.apache.hadoop.hive.serde2.io.ByteWritable;
-import org.apache.hadoop.hive.serde2.io.DateWritable;
-import org.apache.hadoop.hive.serde2.io.DoubleWritable;
-import org.apache.hadoop.hive.serde2.io.HiveCharWritable;
-import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
-import org.apache.hadoop.hive.serde2.io.HiveVarcharWritable;
-import org.apache.hadoop.hive.serde2.io.ShortWritable;
-import org.apache.hadoop.hive.serde2.io.TimestampWritable;
-import org.apache.hadoop.hive.shims.HadoopShims.TextReaderShim;
-import org.apache.hadoop.hive.shims.ShimLoader;
-import org.apache.hadoop.io.BooleanWritable;
-import org.apache.hadoop.io.BytesWritable;
-import org.apache.hadoop.io.FloatWritable;
-import org.apache.hadoop.io.IntWritable;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.orc.TypeDescription;
-import org.apache.orc.impl.BitFieldReader;
-import org.apache.orc.impl.DynamicByteArray;
-import org.apache.orc.impl.InStream;
-import org.apache.orc.impl.IntegerReader;
-import org.apache.orc.OrcProto;
-import org.apache.orc.impl.PositionProvider;
-import org.apache.orc.impl.RunLengthByteReader;
-import org.apache.orc.impl.RunLengthIntegerReader;
-import org.apache.orc.impl.RunLengthIntegerReaderV2;
-import org.apache.orc.impl.SerializationUtils;
-import org.apache.orc.impl.StreamName;
-
-/**
- * Factory for creating ORC tree readers.
- */
-public class TreeReaderFactory {
-
-  public abstract static class TreeReader {
-    protected final int columnId;
-    protected BitFieldReader present = null;
-    protected boolean valuePresent = false;
-    protected int vectorColumnCount;
-
-    TreeReader(int columnId) throws IOException {
-      this(columnId, null);
-    }
-
-    protected TreeReader(int columnId, InStream in) throws IOException {
-      this.columnId = columnId;
-      if (in == null) {
-        present = null;
-        valuePresent = true;
-      } else {
-        present = new BitFieldReader(in, 1);
-      }
-      vectorColumnCount = -1;
-    }
-
-    void setVectorColumnCount(int vectorColumnCount) {
-      this.vectorColumnCount = vectorColumnCount;
-    }
-
-    void checkEncoding(OrcProto.ColumnEncoding encoding) throws IOException {
-      if (encoding.getKind() != OrcProto.ColumnEncoding.Kind.DIRECT) {
-        throw new IOException("Unknown encoding " + encoding + " in column " +
-            columnId);
-      }
-    }
-
-    static IntegerReader createIntegerReader(OrcProto.ColumnEncoding.Kind kind,
-        InStream in,
-        boolean signed, boolean skipCorrupt) throws IOException {
-      switch (kind) {
-        case DIRECT_V2:
-        case DICTIONARY_V2:
-          return new RunLengthIntegerReaderV2(in, signed, skipCorrupt);
-        case DIRECT:
-        case DICTIONARY:
-          return new RunLengthIntegerReader(in, signed);
-        default:
-          throw new IllegalArgumentException("Unknown encoding " + kind);
-      }
-    }
-
-    void startStripe(Map<StreamName, InStream> streams,
-        OrcProto.StripeFooter stripeFooter
-    ) throws IOException {
-      checkEncoding(stripeFooter.getColumnsList().get(columnId));
-      InStream in = streams.get(new StreamName(columnId,
-          OrcProto.Stream.Kind.PRESENT));
-      if (in == null) {
-        present = null;
-        valuePresent = true;
-      } else {
-        present = new BitFieldReader(in, 1);
-      }
-    }
-
-    /**
-     * Seek to the given position.
-     *
-     * @param index the indexes loaded from the file
-     * @throws IOException
-     */
-    void seek(PositionProvider[] index) throws IOException {
-      seek(index[columnId]);
-    }
-
-    public void seek(PositionProvider index) throws IOException {
-      if (present != null) {
-        present.seek(index);
-      }
-    }
-
-    protected long countNonNulls(long rows) throws IOException {
-      if (present != null) {
-        long result = 0;
-        for (long c = 0; c < rows; ++c) {
-          if (present.next() == 1) {
-            result += 1;
-          }
-        }
-        return result;
-      } else {
-        return rows;
-      }
-    }
-
-    abstract void skipRows(long rows) throws IOException;
-
-    void readValuePresent() throws IOException {
-      if (present != null) {
-        valuePresent = present.next() == 1;
-      }
-    }
-
-    Object next(Object previous) throws IOException {
-      if (present != null) {
-        valuePresent = present.next() == 1;
-      }
-      return previous;
-    }
-
-    /**
-     * Called at the top level to read into the given batch.
-     * @param batch the batch to read into
-     * @param batchSize the number of rows to read
-     * @throws IOException
-     */
-    public void nextBatch(VectorizedRowBatch batch,
-                          int batchSize) throws IOException {
-      batch.cols[0].reset();
-      batch.cols[0].ensureSize(batchSize, false);
-      nextVector(batch.cols[0], null, batchSize);
-    }
-
-    /**
-     * Populates the isNull vector array in the previousVector object based on
-     * the present stream values. This function is called from all the child
-     * readers, and they all set the values based on isNull field value.
-     *
-     * @param previous The columnVector object whose isNull value is populated
-     * @param isNull Whether the each value was null at a higher level. If
-     *               isNull is null, all values are non-null.
-     * @param batchSize      Size of the column vector
-     * @throws IOException
-     */
-    public void nextVector(ColumnVector previous,
-                           boolean[] isNull,
-                           final int batchSize) throws IOException {
-      if (present != null || isNull != null) {
-        // Set noNulls and isNull vector of the ColumnVector based on
-        // present stream
-        previous.noNulls = true;
-        boolean allNull = true;
-        for (int i = 0; i < batchSize; i++) {
-          if (isNull == null || !isNull[i]) {
-            if (present != null && present.next() != 1) {
-              previous.noNulls = false;
-              previous.isNull[i] = true;
-            } else {
-              previous.isNull[i] = false;
-              allNull = false;
-            }
-          } else {
-            previous.noNulls = false;
-            previous.isNull[i] = true;
-          }
-        }
-        previous.isRepeating = !previous.noNulls && allNull;
-      } else {
-        // There is no present stream, this means that all the values are
-        // present.
-        previous.noNulls = true;
-        for (int i = 0; i < batchSize; i++) {
-          previous.isNull[i] = false;
-        }
-      }
-    }
-
-    public BitFieldReader getPresent() {
-      return present;
-    }
-  }
-
-  public static class NullTreeReader extends TreeReader {
-
-    public NullTreeReader(int columnId) throws IOException {
-      super(columnId);
-    }
-
-    @Override
-    public void startStripe(Map<StreamName, InStream> streams,
-                            OrcProto.StripeFooter footer) {
-      // PASS
-    }
-
-    @Override
-    void skipRows(long rows) {
-      // PASS
-    }
-
-    @Override
-    public void seek(PositionProvider position) {
-      // PASS
-    }
-
-    @Override
-    public void seek(PositionProvider[] position) {
-      // PASS
-    }
-
-    @Override
-    Object next(Object previous) {
-      return null;
-    }
-
-    @Override
-    public void nextVector(ColumnVector vector, boolean[] isNull, final int batchSize) {
-      vector.noNulls = false;
-      vector.isNull[0] = true;
-      vector.isRepeating = true;
-    }
-  }
-
-  public static class BooleanTreeReader extends TreeReader {
-    protected BitFieldReader reader = null;
-
-    BooleanTreeReader(int columnId) throws IOException {
-      this(columnId, null, null);
-    }
-
-    protected BooleanTreeReader(int columnId, InStream present, InStream data) throws IOException {
-      super(columnId, present);
-      if (data != null) {
-        reader = new BitFieldReader(data, 1);
-      }
-    }
-
-    @Override
-    void startStripe(Map<StreamName, InStream> streams,
-        OrcProto.StripeFooter stripeFooter
-    ) throws IOException {
-      super.startStripe(streams, stripeFooter);
-      reader = new BitFieldReader(streams.get(new StreamName(columnId,
-          OrcProto.Stream.Kind.DATA)), 1);
-    }
-
-    @Override
-    void seek(PositionProvider[] index) throws IOException {
-      seek(index[columnId]);
-    }
-
-    @Override
-    public void seek(PositionProvider index) throws IOException {
-      super.seek(index);
-      reader.seek(index);
-    }
-
-    @Override
-    void skipRows(long items) throws IOException {
-      reader.skip(countNonNulls(items));
-    }
-
-    @Override
-    Object next(Object previous) throws IOException {
-      super.next(previous);
-      BooleanWritable result = null;
-      if (valuePresent) {
-        if (previous == null) {
-          result = new BooleanWritable();
-        } else {
-          result = (BooleanWritable) previous;
-        }
-        result.set(reader.next() == 1);
-      }
-      return result;
-    }
-
-    @Override
-    public void nextVector(ColumnVector previousVector,
-                           boolean[] isNull,
-                           final int batchSize) throws IOException {
-      LongColumnVector result = (LongColumnVector) previousVector;
-
-      // Read present/isNull stream
-      super.nextVector(result, isNull, batchSize);
-
-      // Read value entries based on isNull entries
-      reader.nextVector(result, batchSize);
-    }
-  }
-
-  public static class ByteTreeReader extends TreeReader {
-    protected RunLengthByteReader reader = null;
-
-    ByteTreeReader(int columnId) throws IOException {
-      this(columnId, null, null);
-    }
-
-    protected ByteTreeReader(int columnId, InStream present, InStream data) throws IOException {
-      super(columnId, present);
-      this.reader = new RunLengthByteReader(data);
-    }
-
-    @Override
-    void startStripe(Map<StreamName, InStream> streams,
-        OrcProto.StripeFooter stripeFooter
-    ) throws IOException {
-      super.startStripe(streams, stripeFooter);
-      reader = new RunLengthByteReader(streams.get(new StreamName(columnId,
-          OrcProto.Stream.Kind.DATA)));
-    }
-
-    @Override
-    void seek(PositionProvider[] index) throws IOException {
-      seek(index[columnId]);
-    }
-
-    @Override
-    public void seek(PositionProvider index) throws IOException {
-      super.seek(index);
-      reader.seek(index);
-    }
-
-    @Override
-    Object next(Object previous) throws IOException {
-      super.next(previous);
-      ByteWritable result = null;
-      if (valuePresent) {
-        if (previous == null) {
-          result = new ByteWritable();
-        } else {
-          result = (ByteWritable) previous;
-        }
-        result.set(reader.next());
-      }
-      return result;
-    }
-
-    @Override
-    public void nextVector(ColumnVector previousVector,
-                           boolean[] isNull,
-                           final int batchSize) throws IOException {
-      final LongColumnVector result = (LongColumnVector) previousVector;
-
-      // Read present/isNull stream
-      super.nextVector(result, isNull, batchSize);
-
-      // Read value entries based on isNull entries
-      reader.nextVector(result, result.vector, batchSize);
-    }
-
-    @Override
-    void skipRows(long items) throws IOException {
-      reader.skip(countNonNulls(items));
-    }
-  }
-
-  public static class ShortTreeReader extends TreeReader {
-    protected IntegerReader reader = null;
-
-    ShortTreeReader(int columnId) throws IOException {
-      this(columnId, null, null, null);
-    }
-
-    protected ShortTreeReader(int columnId, InStream present, InStream data,
-        OrcProto.ColumnEncoding encoding)
-        throws IOException {
-      super(columnId, present);
-      if (data != null && encoding != null) {
-        checkEncoding(encoding);
-        this.reader = createIntegerReader(encoding.getKind(), data, true, false);
-      }
-    }
-
-    @Override
-    void checkEncoding(OrcProto.ColumnEncoding encoding) throws IOException {
-      if ((encoding.getKind() != OrcProto.ColumnEncoding.Kind.DIRECT) &&
-          (encoding.getKind() != OrcProto.ColumnEncoding.Kind.DIRECT_V2)) {
-        throw new IOException("Unknown encoding " + encoding + " in column " +
-            columnId);
-      }
-    }
-
-    @Override
-    void startStripe(Map<StreamName, InStream> streams,
-        OrcProto.StripeFooter stripeFooter
-    ) throws IOException {
-      super.startStripe(streams, stripeFooter);
-      StreamName name = new StreamName(columnId,
-          OrcProto.Stream.Kind.DATA);
-      reader = createIntegerReader(stripeFooter.getColumnsList().get(columnId).getKind(),
-          streams.get(name), true, false);
-    }
-
-    @Override
-    void seek(PositionProvider[] index) throws IOException {
-      seek(index[columnId]);
-    }
-
-    @Override
-    public void seek(PositionProvider index) throws IOException {
-      super.seek(index);
-      reader.seek(index);
-    }
-
-    @Override
-    Object next(Object previous) throws IOException {
-      super.next(previous);
-      ShortWritable result = null;
-      if (valuePresent) {
-        if (previous == null) {
-          result = new ShortWritable();
-        } else {
-          result = (ShortWritable) previous;
-        }
-        result.set((short) reader.next());
-      }
-      return result;
-    }
-
-    @Override
-    public void nextVector(ColumnVector previousVector,
-                           boolean[] isNull,
-                           final int batchSize) throws IOException {
-      final LongColumnVector result = (LongColumnVector) previousVector;
-
-      // Read present/isNull stream
-      super.nextVector(result, isNull, batchSize);
-
-      // Read value entries based on isNull entries
-      reader.nextVector(result, result.vector, batchSize);
-    }
-
-    @Override
-    void skipRows(long items) throws IOException {
-      reader.skip(countNonNulls(items));
-    }
-  }
-
-  public static class IntTreeReader extends TreeReader {
-    protected IntegerReader reader = null;
-
-    IntTreeReader(int columnId) throws IOException {
-      this(columnId, null, null, null);
-    }
-
-    protected IntTreeReader(int columnId, InStream present, InStream data,
-        OrcProto.ColumnEncoding encoding)
-        throws IOException {
-      super(columnId, present);
-      if (data != null && encoding != null) {
-        checkEncoding(encoding);
-        this.reader = createIntegerReader(encoding.getKind(), data, true, false);
-      }
-    }
-
-    @Override
-    void checkEncoding(OrcProto.ColumnEncoding encoding) throws IOException {
-      if ((encoding.getKind() != OrcProto.ColumnEncoding.Kind.DIRECT) &&
-          (encoding.getKind() != OrcProto.ColumnEncoding.Kind.DIRECT_V2)) {
-        throw new IOException("Unknown encoding " + encoding + " in column " +
-            columnId);
-      }
-    }
-
-    @Override
-    void startStripe(Map<StreamName, InStream> streams,
-        OrcProto.StripeFooter stripeFooter
-    ) throws IOException {
-      super.startStripe(streams, stripeFooter);
-      StreamName name = new StreamName(columnId,
-          OrcProto.Stream.Kind.DATA);
-      reader = createIntegerReader(stripeFooter.getColumnsList().get(columnId).getKind(),
-          streams.get(name), true, false);
-    }
-
-    @Override
-    void seek(PositionProvider[] index) throws IOException {
-      seek(index[columnId]);
-    }
-
-    @Override
-    public void seek(PositionProvider index) throws IOException {
-      super.seek(index);
-      reader.seek(index);
-    }
-
-    @Override
-    Object next(Object previous) throws IOException {
-      super.next(previous);
-      IntWritable result = null;
-      if (valuePresent) {
-        if (previous == null) {
-          result = new IntWritable();
-        } else {
-          result = (IntWritable) previous;
-        }
-        result.set((int) reader.next());
-      }
-      return result;
-    }
-
-    @Override
-    public void nextVector(ColumnVector previousVector,
-                           boolean[] isNull,
-                           final int batchSize) throws IOException {
-      final LongColumnVector result = (LongColumnVector) previousVector;
-
-      // Read present/isNull stream
-      super.nextVector(result, isNull, batchSize);
-
-      // Read value entries based on isNull entries
-      reader.nextVector(result, result.vector, batchSize);
-    }
-
-    @Override
-    void skipRows(long items) throws IOException {
-      reader.skip(countNonNulls(items));
-    }
-  }
-
-  public static class LongTreeReader extends TreeReader {
-    protected IntegerReader reader = null;
-
-    LongTreeReader(int columnId, boolean skipCorrupt) throws IOException {
-      this(columnId, null, null, null, skipCorrupt);
-    }
-
-    protected LongTreeReader(int columnId, InStream present, InStream data,
-        OrcProto.ColumnEncoding encoding,
-        boolean skipCorrupt)
-        throws IOException {
-      super(columnId, present);
-      if (data != null && encoding != null) {
-        checkEncoding(encoding);
-        this.reader = createIntegerReader(encoding.getKind(), data, true, skipCorrupt);
-      }
-    }
-
-    @Override
-    void checkEncoding(OrcProto.ColumnEncoding encoding) throws IOException {
-      if ((encoding.getKind() != OrcProto.ColumnEncoding.Kind.DIRECT) &&
-          (encoding.getKind() != OrcProto.ColumnEncoding.Kind.DIRECT_V2)) {
-        throw new IOException("Unknown encoding " + encoding + " in column " +
-            columnId);
-      }
-    }
-
-    @Override
-    void startStripe(Map<StreamName, InStream> streams,
-        OrcProto.StripeFooter stripeFooter
-    ) throws IOException {
-      super.startStripe(streams, stripeFooter);
-      StreamName name = new StreamName(columnId,
-          OrcProto.Stream.Kind.DATA);
-      reader = createIntegerReader(stripeFooter.getColumnsList().get(columnId).getKind(),
-          streams.get(name), true, false);
-    }
-
-    @Override
-    void seek(PositionProvider[] index) throws IOException {
-      seek(index[columnId]);
-    }
-
-    @Override
-    public void seek(PositionProvider index) throws IOException {
-      super.seek(index);
-      reader.seek(index);
-    }
-
-    @Override
-    Object next(Object previous) throws IOException {
-      super.next(previous);
-      LongWritable result = null;
-      if (valuePresent) {
-        if (previous == null) {
-          result = new LongWritable();
-        } else {
-          result = (LongWritable) previous;
-        }
-        result.set(reader.next());
-      }
-      return result;
-    }
-
-    @Override
-    public void nextVector(ColumnVector previousVector,
-                           boolean[] isNull,
-                           final int batchSize) throws IOException {
-      final LongColumnVector result = (LongColumnVector) previousVector;
-
-      // Read present/isNull stream
-      super.nextVector(result, isNull, batchSize);
-
-      // Read value entries based on isNull entries
-      reader.nextVector(result, result.vector, batchSize);
-    }
-
-    @Override
-    void skipRows(long items) throws IOException {
-      reader.skip(countNonNulls(items));
-    }
-  }
-
-  public static class FloatTreeReader extends TreeReader {
-    protected InStream stream;
-    private final SerializationUtils utils;
-
-    FloatTreeReader(int columnId) throws IOException {
-      this(columnId, null, null);
-    }
-
-    protected FloatTreeReader(int columnId, InStream present, InStream data) throws IOException {
-      super(columnId, present);
-      this.utils = new SerializationUtils();
-      this.stream = data;
-    }
-
-    @Override
-    void startStripe(Map<StreamName, InStream> streams,
-        OrcProto.StripeFooter stripeFooter
-    ) throws IOException {
-      super.startStripe(streams, stripeFooter);
-      StreamName name = new StreamName(columnId,
-          OrcProto.Stream.Kind.DATA);
-      stream = streams.get(name);
-    }
-
-    @Override
-    void seek(PositionProvider[] index) throws IOException {
-      seek(index[columnId]);
-    }
-
-    @Override
-    public void seek(PositionProvider index) throws IOException {
-      super.seek(index);
-      stream.seek(index);
-    }
-
-    @Override
-    Object next(Object previous) throws IOException {
-      super.next(previous);
-      FloatWritable result = null;
-      if (valuePresent) {
-        if (previous == null) {
-          result = new FloatWritable();
-        } else {
-          result = (FloatWritable) previous;
-        }
-        result.set(utils.readFloat(stream));
-      }
-      return result;
-    }
-
-    @Override
-    public void nextVector(ColumnVector previousVector,
-                           boolean[] isNull,
-                           final int batchSize) throws IOException {
-      final DoubleColumnVector result = (DoubleColumnVector) previousVector;
-
-      // Read present/isNull stream
-      super.nextVector(result, isNull, batchSize);
-
-      final boolean hasNulls = !result.noNulls;
-      boolean allNulls = hasNulls;
-
-      if (hasNulls) {
-        // conditions to ensure bounds checks skips
-        for (int i = 0; batchSize <= result.isNull.length && i < batchSize; i++) {
-          allNulls = allNulls & result.isNull[i];
-        }
-        if (allNulls) {
-          result.vector[0] = Double.NaN;
-          result.isRepeating = true;
-        } else {
-          // some nulls
-          result.isRepeating = false;
-          // conditions to ensure bounds checks skips
-          for (int i = 0; batchSize <= result.isNull.length
-              && batchSize <= result.vector.length && i < batchSize; i++) {
-            if (!result.isNull[i]) {
-              result.vector[i] = utils.readFloat(stream);
-            } else {
-              // If the value is not present then set NaN
-              result.vector[i] = Double.NaN;
-            }
-          }
-        }
-      } else {
-        // no nulls & > 1 row (check repeating)
-        boolean repeating = (batchSize > 1);
-        final float f1 = utils.readFloat(stream);
-        result.vector[0] = f1;
-        // conditions to ensure bounds checks skips
-        for (int i = 1; i < batchSize && batchSize <= result.vector.length; i++) {
-          final float f2 = utils.readFloat(stream);
-          repeating = repeating && (f1 == f2);
-          result.vector[i] = f2;
-        }
-        result.isRepeating = repeating;
-      }
-    }
-
-    @Override
-    protected void skipRows(long items) throws IOException {
-      items = countNonNulls(items);
-      for (int i = 0; i < items; ++i) {
-        utils.readFloat(stream);
-      }
-    }
-  }
-
-  public static class DoubleTreeReader extends TreeReader {
-    protected InStream stream;
-    private final SerializationUtils utils;
-
-    DoubleTreeReader(int columnId) throws IOException {
-      this(columnId, null, null);
-    }
-
-    protected DoubleTreeReader(int columnId, InStream present, InStream data) throws IOException {
-      super(columnId, present);
-      this.utils = new SerializationUtils();
-      this.stream = data;
-    }
-
-    @Override
-    void startStripe(Map<StreamName, InStream> streams,
-        OrcProto.StripeFooter stripeFooter
-    ) throws IOException {
-      super.startStripe(streams, stripeFooter);
-      StreamName name =
-          new StreamName(columnId,
-              OrcProto.Stream.Kind.DATA);
-      stream = streams.get(name);
-    }
-
-    @Override
-    void seek(PositionProvider[] index) throws IOException {
-      seek(index[columnId]);
-    }
-
-    @Override
-    public void seek(PositionProvider index) throws IOException {
-      super.seek(index);
-      stream.seek(index);
-    }
-
-    @Override
-    Object next(Object previous) throws IOException {
-      super.next(previous);
-      DoubleWritable result = null;
-      if (valuePresent) {
-        if (previous == null) {
-          result = new DoubleWritable();
-        } else {
-          result = (DoubleWritable) previous;
-        }
-        result.set(utils.readDouble(stream));
-      }
-      return result;
-    }
-
-    @Override
-    public void nextVector(ColumnVector previousVector,
-                           boolean[] isNull,
-                           final int batchSize) throws IOException {
-      final DoubleColumnVector result = (DoubleColumnVector) previousVector;
-
-      // Read present/isNull stream
-      super.nextVector(result, isNull, batchSize);
-
-      final boolean hasNulls = !result.noNulls;
-      boolean allNulls = hasNulls;
-
-      if (hasNulls) {
-        // conditions to ensure bounds checks skips
-        for (int i = 0; i < batchSize && batchSize <= result.isNull.length; i++) {
-          allNulls = allNulls & result.isNull[i];
-        }
-        if (allNulls) {
-          result.vector[0] = Double.NaN;
-          result.isRepeating = true;
-        } else {
-          // some nulls
-          result.isRepeating = false;
-          // conditions to ensure bounds checks skips
-          for (int i = 0; batchSize <= result.isNull.length
-              && batchSize <= result.vector.length && i < batchSize; i++) {
-            if (!result.isNull[i]) {
-              result.vector[i] = utils.readDouble(stream);
-            } else {
-              // If the value is not present then set NaN
-              result.vector[i] = Double.NaN;
-            }
-          }
-        }
-      } else {
-        // no nulls
-        boolean repeating = (batchSize > 1);
-        final double d1 = utils.readDouble(stream);
-        result.vector[0] = d1;
-        // conditions to ensure bounds checks skips
-        for (int i = 1; i < batchSize && batchSize <= result.vector.length; i++) {
-          final double d2 = utils.readDouble(stream);
-          repeating = repeating && (d1 == d2);
-          result.vector[i] = d2;
-        }
-        result.isRepeating = repeating;
-      }
-    }
-
-    @Override
-    void skipRows(long items) throws IOException {
-      items = countNonNulls(items);
-      long len = items * 8;
-      while (len > 0) {
-        len -= stream.skip(len);
-      }
-    }
-  }
-
-  public static class BinaryTreeReader extends TreeReader {
-    protected InStream stream;
-    protected IntegerReader lengths = null;
-    protected final LongColumnVector scratchlcv;
-
-    BinaryTreeReader(int columnId) throws IOException {
-      this(columnId, null, null, null, null);
-    }
-
-    protected BinaryTreeReader(int columnId, InStream present, InStream data, InStream length,
-        OrcProto.ColumnEncoding encoding) throws IOException {
-      super(columnId, present);
-      scratchlcv = new LongColumnVector();
-      this.stream = data;
-      if (length != null && encoding != null) {
-        checkEncoding(encoding);
-        this.lengths = createIntegerReader(encoding.getKind(), length, false, false);
-      }
-    }
-
-    @Override
-    void checkEncoding(OrcProto.ColumnEncoding encoding) throws IOException {
-      if ((encoding.getKind() != OrcProto.ColumnEncoding.Kind.DIRECT) &&
-          (encoding.getKind() != OrcProto.ColumnEncoding.Kind.DIRECT_V2)) {
-        throw new IOException("Unknown encoding " + encoding + " in column " +
-            columnId);
-      }
-    }
-
-    @Override
-    void startStripe(Map<StreamName, InStream> streams,
-        OrcProto.StripeFooter stripeFooter
-    ) throws IOException {
-      super.startStripe(streams, stripeFooter);
-      StreamName name = new StreamName(columnId,
-          OrcProto.Stream.Kind.DATA);
-      stream = streams.get(name);
-      lengths = createIntegerReader(stripeFooter.getColumnsList().get(columnId).getKind(),
-          streams.get(new StreamName(columnId, OrcProto.Stream.Kind.LENGTH)), false, false);
-    }
-
-    @Override
-    void seek(PositionProvider[] index) throws IOException {
-      seek(index[columnId]);
-    }
-
-    @Override
-    public void seek(PositionProvider index) throws IOException {
-      super.seek(index);
-      stream.seek(index);
-      lengths.seek(index);
-    }
-
-    @Override
-    Object next(Object previous) throws IOException {
-      super.next(previous);
-      BytesWritable result = null;
-      if (valuePresent) {
-        if (previous == null) {
-          result = new BytesWritable();
-        } else {
-          result = (BytesWritable) previous;
-        }
-        int len = (int) lengths.next();
-        result.setSize(len);
-        int offset = 0;
-        while (len > 0) {
-          int written = stream.read(result.getBytes(), offset, len);
-          if (written < 0) {
-            throw new EOFException("Can't finish byte read from " + stream);
-          }
-          len -= written;
-          offset += written;
-        }
-      }
-      return result;
-    }
-
-    @Override
-    public void nextVector(ColumnVector previousVector,
-                           boolean[] isNull,
-                           final int batchSize) throws IOException {
-      final BytesColumnVector result = (BytesColumnVector) previousVector;
-
-      // Read present/isNull stream
-      super.nextVector(result, isNull, batchSize);
-
-      BytesColumnVectorUtil.readOrcByteArrays(stream, lengths, scratchlcv, result, batchSize);
-    }
-
-    @Override
-    void skipRows(long items) throws IOException {
-      items = countNonNulls(items);
-      long lengthToSkip = 0;
-      for (int i = 0; i < items; ++i) {
-        lengthToSkip += lengths.next();
-      }
-      while (lengthToSkip > 0) {
-        lengthToSkip -= stream.skip(lengthToSkip);
-      }
-    }
-  }
-
-  public static class TimestampTreeReader extends TreeReader {
-    protected IntegerReader data = null;
-    protected IntegerReader nanos = null;
-    private final boolean skipCorrupt;
-    private Map<String, Long> baseTimestampMap;
-    private long base_timestamp;
-    private final TimeZone readerTimeZone;
-    private TimeZone writerTimeZone;
-    private boolean hasSameTZRules;
-
-    TimestampTreeReader(int columnId, boolean skipCorrupt) throws IOException {
-      this(columnId, null, null, null, null, skipCorrupt);
-    }
-
-    protected TimestampTreeReader(int columnId, InStream presentStream, InStream dataStream,
-        InStream nanosStream, OrcProto.ColumnEncoding encoding, boolean skipCorrupt)
-        throws IOException {
-      super(columnId, presentStream);
-      this.skipCorrupt = skipCorrupt;
-      this.baseTimestampMap = new HashMap<>();
-      this.readerTimeZone = TimeZone.getDefault();
-      this.writerTimeZone = readerTimeZone;
-      this.hasSameTZRules = writerTimeZone.hasSameRules(readerTimeZone);
-      this.base_timestamp = getBaseTimestamp(readerTimeZone.getID());
-      if (encoding != null) {
-        checkEncoding(encoding);
-
-        if (dataStream != null) {
-          this.data = createIntegerReader(encoding.getKind(), dataStream, true, skipCorrupt);
-        }
-
-        if (nanosStream != null) {
-          this.nanos = createIntegerReader(encoding.getKind(), nanosStream, false, skipCorrupt);
-        }
-      }
-    }
-
-    @Override
-    void checkEncoding(OrcProto.ColumnEncoding encoding) throws IOException {
-      if ((encoding.getKind() != OrcProto.ColumnEncoding.Kind.DIRECT) &&
-          (encoding.getKind() != OrcProto.ColumnEncoding.Kind.DIRECT_V2)) {
-        throw new IOException("Unknown encoding " + encoding + " in column " +
-            columnId);
-      }
-    }
-
-    @Override
-    void startStripe(Map<StreamName, InStream> streams,
-        OrcProto.StripeFooter stripeFooter
-    ) throws IOException {
-      super.startStripe(streams, stripeFooter);
-      data = createIntegerReader(stripeFooter.getColumnsList().get(columnId).getKind(),
-          streams.get(new StreamName(columnId,
-              OrcProto.Stream.Kind.DATA)), true, skipCorrupt);
-      nanos = createIntegerReader(stripeFooter.getColumnsList().get(columnId).getKind(),
-          streams.get(new StreamName(columnId,
-              OrcProto.Stream.Kind.SECONDARY)), false, skipCorrupt);
-      base_timestamp = getBaseTimestamp(stripeFooter.getWriterTimezone());
-    }
-
-    private long getBaseTimestamp(String timeZoneId) throws IOException {
-      // to make sure new readers read old files in the same way
-      if (timeZoneId == null || timeZoneId.isEmpty()) {
-        timeZoneId = readerTimeZone.getID();
-      }
-
-      if (!baseTimestampMap.containsKey(timeZoneId)) {
-        writerTimeZone = TimeZone.getTimeZone(timeZoneId);
-        hasSameTZRules = writerTimeZone.hasSameRules(readerTimeZone);
-        SimpleDateFormat sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
-        sdf.setTimeZone(writerTimeZone);
-        try {
-          long epoch =
-              sdf.parse(WriterImpl.BASE_TIMESTAMP_STRING).getTime() / WriterImpl.MILLIS_PER_SECOND;
-          baseTimestampMap.put(timeZoneId, epoch);
-          return epoch;
-        } catch (ParseException e) {
-          throw new IOException("Unable to create base timestamp", e);
-        } finally {
-          sdf.setTimeZone(readerTimeZone);
-        }
-      }
-
-      return baseTimestampMap.get(timeZoneId);
-    }
-
-    @Override
-    void seek(PositionProvider[] index) throws IOException {
-      seek(index[columnId]);
-    }
-
-    @Override
-    public void seek(PositionProvider index) throws IOException {
-      super.seek(index);
-      data.seek(index);
-      nanos.seek(index);
-    }
-
-    @Override
-    Object next(Object previous) throws IOException {
-      super.next(previous);
-      TimestampWritable result = null;
-      if (valuePresent) {
-        if (previous == null) {
-          result = new TimestampWritable();
-        } else {
-          result = (TimestampWritable) previous;
-        }
-        long millis = (data.next() + base_timestamp) * WriterImpl.MILLIS_PER_SECOND;
-        int newNanos = parseNanos(nanos.next());
-        // fix the rounding when we divided by 1000.
-        if (millis >= 0) {
-          millis += newNanos / WriterImpl.NANOS_PER_MILLI;
-        } else {
-          millis -= newNanos / WriterImpl.NANOS_PER_MILLI;
-        }
-        long offset = 0;
-        // If reader and writer time zones have different rules, adjust the timezone difference
-        // between reader and writer taking day light savings into account.
-        if (!hasSameTZRules) {
-          offset = writerTimeZone.getOffset(millis) - readerTimeZone.getOffset(millis);
-        }
-        long adjustedMillis = millis + offset;
-        Timestamp ts = new Timestamp(adjustedMillis);
-        // Sometimes the reader timezone might have changed after adding the adjustedMillis.
-        // To account for that change, check for any difference in reader timezone after
-        // adding adjustedMillis. If so use the new offset (offset at adjustedMillis point of time).
-        if (!hasSameTZRules &&
-            (readerTimeZone.getOffset(millis) != readerTimeZone.getOffset(adjustedMillis))) {
-          long newOffset =
-              writerTimeZone.getOffset(millis) - readerTimeZone.getOffset(adjustedMillis);
-          adjustedMillis = millis + newOffset;
-          ts.setTime(adjustedMillis);
-        }
-        ts.setNanos(newNanos);
-        result.set(ts);
-      }
-      return result;
-    }
-
-    @Override
-    public void nextVector(ColumnVector previousVector,
-                           boolean[] isNull,
-                           final int batchSize) throws IOException {
-      TimestampColumnVector result = (TimestampColumnVector) previousVector;
-      super.nextVector(previousVector, isNull, batchSize);
-
-      for (int i = 0; i < batchSize; i++) {
-        if (result.noNulls || !result.isNull[i]) {
-          long millis = data.next() + base_timestamp;
-          int newNanos = parseNanos(nanos.next());
-          if (millis < 0 && newNanos != 0) {
-            millis -= 1;
-          }
-          millis *= WriterImpl.MILLIS_PER_SECOND;
-          long offset = 0;
-          // If reader and writer time zones have different rules, adjust the timezone difference
-          // between reader and writer taking day light savings into account.
-          if (!hasSameTZRules) {
-            offset = writerTimeZone.getOffset(millis) - readerTimeZone.getOffset(millis);
-          }
-          long adjustedMillis = millis + offset;
-          // Sometimes the reader timezone might have changed after adding the adjustedMillis.
-          // To account for that change, check for any difference in reader timezone after
-          // adding adjustedMillis. If so use the new offset (offset at adjustedMillis point of time).
-          if (!hasSameTZRules &&
-              (readerTimeZone.getOffset(millis) != readerTimeZone.getOffset(adjustedMillis))) {
-            long newOffset =
-                writerTimeZone.getOffset(millis) - readerTimeZone.getOffset(adjustedMillis);
-            adjustedMillis = millis + newOffset;
-          }
-          result.time[i] = adjustedMillis;
-          result.nanos[i] = newNanos;
-          if (result.isRepeating && i != 0 &&
-              (result.time[0] != result.time[i] ||
-                  result.nanos[0] != result.nanos[i])) {
-            result.isRepeating = false;
-          }
-        }
-      }
-    }
-
-    private static int parseNanos(long serialized) {
-      int zeros = 7 & (int) serialized;
-      int result = (int) (serialized >>> 3);
-      if (zeros != 0) {
-        for (int i = 0; i <= zeros; ++i) {
-          result *= 10;
-        }
-      }
-      return result;
-    }
-
-    @Override
-    void skipRows(long items) throws IOException {
-      items = countNonNulls(items);
-      data.skip(items);
-      nanos.skip(items);
-    }
-  }
-
-  public static class DateTreeReader extends TreeReader {
-    protected IntegerReader reader = null;
-
-    DateTreeReader(int columnId) throws IOException {
-      this(columnId, null, null, null);
-    }
-
-    protected DateTreeReader(int columnId, InStream present, InStream data,
-        OrcProto.ColumnEncoding encoding) throws IOException {
-      super(columnId, present);
-      if (data != null && encoding != null) {
-        checkEncoding(encoding);
-        reader = createIntegerReader(encoding.getKind(), data, true, false);
-      }
-    }
-
-    @Override
-    void checkEncoding(OrcProto.ColumnEncoding encoding) throws IOException {
-      if ((encoding.getKind() != OrcProto.ColumnEncoding.Kind.DIRECT) &&
-          (encoding.getKind() != OrcProto.ColumnEncoding.Kind.DIRECT_V2)) {
-        throw new IOException("Unknown encoding " + encoding + " in column " +
-            columnId);
-      }
-    }
-
-    @Override
-    void startStripe(Map<StreamName, InStream> streams,
-        OrcProto.StripeFooter stripeFooter
-    ) throws IOException {
-      super.startStripe(streams, stripeFooter);
-      StreamName name = new StreamName(columnId,
-          OrcProto.Stream.Kind.DATA);
-      reader = createIntegerReader(stripeFooter.getColumnsList().get(columnId).getKind(),
-          streams.get(name), true, false);
-    }
-
-    @Override
-    void seek(PositionProvider[] index) throws IOException {
-      seek(index[columnId]);
-    }
-
-    @Override
-    public void seek(PositionProvider index) throws IOException {
-      super.seek(index);
-      reader.seek(index);
-    }
-
-    @Override
-    Object next(Object previous) throws IOException {
-      super.next(previous);
-      DateWritable result = null;
-      if (valuePresent) {
-        if (previous == null) {
-          result = new DateWritable();
-        } else {
-          result = (DateWritable) previous;
-        }
-        result.set((int) reader.next());
-      }
-      return result;
-    }
-
-    @Override
-    public void nextVector(ColumnVector previousVector,
-                           boolean[] isNull,
-                           final int batchSize) throws IOException {
-      final LongColumnVector result = (LongColumnVector) previousVector;
-
-      // Read present/isNull stream
-      super.nextVector(result, isNull, batchSize);
-
-      // Read value entries based on isNull entries
-      reader.nextVector(result, result.vector, batchSize);
-    }
-
-    @Override
-    void skipRows(long items) throws IOException {
-      reader.skip(countNonNulls(items));
-    }
-  }
-
-  public static class DecimalTreeReader extends TreeReader {
-    protected InStream valueStream;
-    protected IntegerReader scaleReader = null;
-    private int[] scratchScaleVector;
-
-    private final int precision;
-    private final int scale;
-
-    DecimalTreeReader(int columnId, int precision, int scale) throws IOException {
-      this(columnId, precision, scale, null, null, null, null);
-    }
-
-    protected DecimalTreeReader(int columnId, int precision, int scale, InStream present,
-        InStream valueStream, InStream scaleStream, OrcProto.ColumnEncoding encoding)
-        throws IOException {
-      super(columnId, present);
-      this.precision = precision;
-      this.scale = scale;
-      this.scratchScaleVector = new int[VectorizedRowBatch.DEFAULT_SIZE];
-      this.valueStream = valueStream;
-      if (scaleStream != null && encoding != null) {
-        checkEncoding(encoding);
-        this.scaleReader = createIntegerReader(encoding.getKind(), scaleStream, true, false);
-      }
-    }
-
-    @Override
-    void checkEncoding(OrcProto.ColumnEncoding encoding) throws IOException {
-      if ((encoding.getKind() != OrcProto.ColumnEncoding.Kind.DIRECT) &&
-          (encoding.getKind() != OrcProto.ColumnEncoding.Kind.DIRECT_V2)) {
-        throw new IOException("Unknown encoding " + encoding + " in column " +
-            columnId);
-      }
-    }
-
-    @Override
-    void startStripe(Map<StreamName, InStream> streams,
-        OrcProto.StripeFooter stripeFooter
-    ) throws IOException {
-      super.startStripe(streams, stripeFooter);
-      valueStream = streams.get(new StreamName(columnId,
-          OrcProto.Stream.Kind.DATA));
-      scaleReader = createIntegerReader(stripeFooter.getColumnsList().get(columnId).getKind(),
-          streams.get(new StreamName(columnId, OrcProto.Stream.Kind.SECONDARY)), true, false);
-    }
-
-    @Override
-    void seek(PositionProvider[] index) throws IOException {
-      seek(index[columnId]);
-    }
-
-    @Override
-    public void seek(PositionProvider index) throws IOException {
-      super.seek(index);
-      valueStream.seek(index);
-      scaleReader.seek(index);
-    }
-
-    @Override
-    Object next(Object previous) throws IOException {
-      super.next(previous);
-      final HiveDecimalWritable result;
-      if (valuePresent) {
-        if (previous == null) {
-          result = new HiveDecimalWritable();
-        } else {
-          result = (HiveDecimalWritable) previous;
-        }
-        result.set(HiveDecimal.create(SerializationUtils.readBigInteger
-                (valueStream), (int) scaleReader.next()));
-        return HiveDecimalWritable.enforcePrecisionScale(result, precision,
-            scale);
-      }
-      return null;
-    }
-
-    @Override
-    public void nextVector(ColumnVector previousVector,
-                           boolean[] isNull,
-                           final int batchSize) throws IOException {
-      final DecimalColumnVector result = (DecimalColumnVector) previousVector;
-
-      // Read present/isNull stream
-      super.nextVector(result, isNull, batchSize);
-
-      if (batchSize > scratchScaleVector.length) {
-        scratchScaleVector = new int[(int) batchSize];
-      }
-      scaleReader.nextVector(result, scratchScaleVector, batchSize);
-      // Read value entries based on isNull entries
-      if (result.noNulls) {
-        for (int r=0; r < batchSize; ++r) {
-          BigInteger bInt = SerializationUtils.readBigInteger(valueStream);
-          HiveDecimal dec = HiveDecimal.create(bInt, scratchScaleVector[r]);
-          result.set(r, dec);
-        }
-      } else if (!result.isRepeating || !result.isNull[0]) {
-        for (int r=0; r < batchSize; ++r) {
-          if (!result.isNull[r]) {
-            BigInteger bInt = SerializationUtils.readBigInteger(valueStream);
-            HiveDecimal dec = HiveDecimal.create(bInt, scratchScaleVector[r]);
-            result.set(r, dec);
-          }
-        }
-      }
-    }
-
-    @Override
-    void skipRows(long items) throws IOException {
-      items = countNonNulls(items);
-      for (int i = 0; i < items; i++) {
-        SerializationUtils.readBigInteger(valueStream);
-      }
-      scaleReader.skip(items);
-    }
-  }
-
-  /**
-   * A tree reader that will read string columns. At the start of the
-   * stripe, it creates an internal reader based on whether a direct or
-   * dictionary encoding was used.
-   */
-  public static class StringTreeReader extends TreeReader {
-    protected TreeReader reader;
-
-    StringTreeReader(int columnId) throws IOException {
-      super(columnId);
-    }
-
-    protected StringTreeReader(int columnId, InStream present, InStream data, InStream length,
-        InStream dictionary, OrcProto.ColumnEncoding encoding) throws IOException {
-      super(columnId, present);
-      if (encoding != null) {
-        switch (encoding.getKind()) {
-          case DIRECT:
-          case DIRECT_V2:
-            reader = new StringDirectTreeReader(columnId, present, data, length,
-                encoding.getKind());
-            break;
-          case DICTIONARY:
-          case DICTIONARY_V2:
-            reader = new StringDictionaryTreeReader(columnId, present, data, length, dictionary,
-                encoding);
-            break;
-          default:
-            throw new IllegalArgumentException("Unsupported encoding " +
-                encoding.getKind());
-        }
-      }
-    }
-
-    @Override
-    void checkEncoding(OrcProto.ColumnEncoding encoding) throws IOException {
-      reader.checkEncoding(encoding);
-    }
-
-    @Override
-    void startStripe(Map<StreamName, InStream> streams,
-        OrcProto.StripeFooter stripeFooter
-    ) throws IOException {
-      // For each stripe, checks the encoding and initializes the appropriate
-      // reader
-      switch (stripeFooter.getColumnsList().get(columnId).getKind()) {
-        case DIRECT:
-        case DIRECT_V2:
-          reader = new StringDirectTreeReader(columnId);
-          break;
-        case DICTIONARY:
-        case DICTIONARY_V2:
-          reader = new StringDictionaryTreeReader(columnId);
-          break;
-        default:
-          throw new IllegalArgumentException("Unsupported encoding " +
-              stripeFooter.getColumnsList().get(columnId).getKind());
-      }
-      reader.startStripe(streams, stripeFooter);
-    }
-
-    @Override
-    void seek(PositionProvider[] index) throws IOException {
-      reader.seek(index);
-    }
-
-    @Override
-    public void seek(PositionProvider index) throws IOException {
-      reader.seek(index);
-    }
-
-    @Override
-    Object next(Object previous) throws IOException {
-      return reader.next(previous);
-    }
-
-    @Override
-    public void nextVector(ColumnVector previousVector,
-                           boolean[] isNull,
-                           final int batchSize) throws IOException {
-      reader.nextVector(previousVector, isNull, batchSize);
-    }
-
-    @Override
-    void skipRows(long items) throws IOException {
-      reader.skipRows(items);
-    }
-  }
-
-  // This class collects together very similar methods for reading an ORC vector of byte arrays and
-  // creating the BytesColumnVector.
-  //
-  public static class BytesColumnVectorUtil {
-
-    private static byte[] commonReadByteArrays(InStream stream, IntegerReader lengths,
-        LongColumnVector scratchlcv,
-        BytesColumnVector result, final int batchSize) throws IOException {
-      // Read lengths
-      scratchlcv.isNull = result.isNull;  // Notice we are replacing the isNull vector here...
-      lengths.nextVector(scratchlcv, scratchlcv.vector, batchSize);
-      int totalLength = 0;
-      if (!scratchlcv.isRepeating) {
-        for (int i = 0; i < batchSize; i++) {
-          if (!scratchlcv.isNull[i]) {
-            totalLength += (int) scratchlcv.vector[i];
-          }
-        }
-      } else {
-        if (!scratchlcv.isNull[0]) {
-          totalLength = (int) (batchSize * scratchlcv.vector[0]);
-        }
-      }
-
-      // Read all the strings for this batch
-      byte[] allBytes = new byte[totalLength];
-      int offset = 0;
-      int len = totalLength;
-      while (len > 0) {
-        int bytesRead = stream.read(allBytes, offset, len);
-        if (bytesRead < 0) {
-          throw new EOFException("Can't finish byte read from " + stream);
-        }
-        len -= bytesRead;
-        offset += bytesRead;
-      }
-
-      return allBytes;
-    }
-
-    // This method has the common code for reading in bytes into a BytesColumnVector.
-    public static void readOrcByteArrays(InStream stream,
-                                         IntegerReader lengths,
-                                         LongColumnVector scratchlcv,
-                                         BytesColumnVector result,
-                                         final int batchSize) throws IOException {
-      if (result.noNulls || !(result.isRepeating && result.isNull[0])) {
-        byte[] allBytes = commonReadByteArrays(stream, lengths, scratchlcv,
-            result, (int) batchSize);
-
-        // Too expensive to figure out 'repeating' by comparisons.
-        result.isRepeating = false;
-        int offset = 0;
-        if (!scratchlcv.isRepeating) {
-          for (int i = 0; i < batchSize; i++) {
-            if (!scratchlcv.isNull[i]) {
-              result.setRef(i, allBytes, offset, (int) scratchlcv.vector[i]);
-              offset += scratchlcv.vector[i];
-            } else {
-              result.setRef(i, allBytes, 0, 0);
-            }
-          }
-        } else {
-          for (int i = 0; i < batchSize; i++) {
-            if (!scratchlcv.isNull[i]) {
-              result.setRef(i, allBytes, offset, (int) scratchlcv.vector[0]);
-              offset += scratchlcv.vector[0];
-            } else {
-              result.setRef(i, allBytes, 0, 0);
-            }
-          }
-        }
-      }
-    }
-  }
-
-  /**
-   * A reader for string columns that are direct encoded in the current
-   * stripe.
-   */
-  public static class StringDirectTreeReader extends TreeReader {
-    protected InStream stream;
-    protected TextReaderShim data;
-    protected IntegerReader lengths;
-    private final LongColumnVector scratchlcv;
-
-    StringDirectTreeReader(int columnId) throws IOException {
-      this(columnId, null, null, null, null);
-    }
-
-    protected StringDirectTreeReader(int columnId, InStream present, InStream data,
-        InStream length, OrcProto.ColumnEncoding.Kind encoding) throws IOException {
-      super(columnId, present);
-      this.scratchlcv = new LongColumnVector();
-      this.stream = data;
-      if (length != null && encoding != null) {
-        this.lengths = createIntegerReader(encoding, length, false, false);
-        this.data = ShimLoader.getHadoopShims().getTextReaderShim(this.stream);
-      }
-    }
-
-    @Override
-    void checkEncoding(OrcProto.ColumnEncoding encoding) throws IOException {
-      if (encoding.getKind() != OrcProto.ColumnEncoding.Kind.DIRECT &&
-          encoding.getKind() != OrcProto.ColumnEncoding.Kind.DIRECT_V2) {
-        throw new IOException("Unknown encoding " + encoding + " in column " +
-            columnId);
-      }
-    }
-
-    @Override
-    void startStripe(Map<StreamName, InStream> streams,
-        OrcProto.StripeFooter stripeFooter
-    ) throws IOException {
-      super.startStripe(streams, stripeFooter);
-      StreamName name = new StreamName(columnId,
-          OrcProto.Stream.Kind.DATA);
-      stream = streams.get(name);
-      data = ShimLoader.getHadoopShims().getTextReaderShim(this.stream);
-      lengths = createIntegerReader(stripeFooter.getColumnsList().get(columnId).getKind(),
-          streams.get(new StreamName(columnId, OrcProto.Stream.Kind.LENGTH)),
-          false, false);
-    }
-
-    @Override
-    void seek(PositionProvider[] index) throws IOException {
-      seek(index[columnId]);
-    }
-
-    @Override
-    public void seek(PositionProvider index) throws IOException {
-      super.seek(index);
-      stream.seek(index);
-      // don't seek data stream
-      lengths.seek(index);
-    }
-
-    @Override
-    Object next(Object previous) throws IOException {
-      super.next(previous);
-      Text result = null;
-      if (valuePresent) {
-        if (previous == null) {
-          result = new Text();
-        } else {
-          result = (Text) previous;
-        }
-        int len = (int) lengths.next();
-        data.read(result, len);
-      }
-      return result;
-    }
-
-    @Override
-    public void nextVector(ColumnVector previousVector,
-                           boolean[] isNull,
-                           final int batchSize) throws IOException {
-      final BytesColumnVector result = (BytesColumnVector) previousVector;
-
-      // Read present/isNull stream
-      super.nextVector(result, isNull, batchSize);
-
-      BytesColumnVectorUtil.readOrcByteArrays(stream, lengths, scratchlcv,
-          result, batchSize);
-    }
-
-    @Override
-    void skipRows(long items) throws IOException {
-      items = countNonNulls(items);
-      long lengthToSkip = 0;
-      for (int i = 0; i < items; ++i) {
-        lengthToSkip += lengths.next();
-      }
-
-      while (lengthToSkip > 0) {
-        lengthToSkip -= stream.skip(lengthToSkip);
-      }
-    }
-
-    public IntegerReader getLengths() {
-      return lengths;
-    }
-
-    public InStream getStream() {
-      return stream;
-    }
-  }
-
-  /**
-   * A reader for string columns that are dictionary encoded in the current
-   * stripe.
-   */
-  public static class StringDictionaryTreeReader extends TreeReader {
-    private static final byte[] EMPTY_BYTE_ARRAY = new byte[0];
-    private DynamicByteArray dictionaryBuffer;
-    private int[] dictionaryOffsets;
-    protected IntegerReader reader;
-
-    private byte[] dictionaryBufferInBytesCache = null;
-    private final LongColumnVector scratchlcv;
-
-    StringDictionaryTreeReader(int columnId) throws IOException {
-      this(columnId, null, null, null, null, null);
-    }
-
-    protected StringDictionaryTreeReader(int columnId, InStream present, InStream data,
-        InStream length, InStream dictionary, OrcProto.ColumnEncoding encoding)
-        throws IOException {
-      super(columnId, present);
-      scratchlcv = new LongColumnVector();
-      if (data != null && encoding != null) {
-        this.reader = createIntegerReader(encoding.getKind(), data, false, false);
-      }
-
-      if (dictionary != null && encoding != null) {
-        readDictionaryStream(dictionary);
-      }
-
-      if (length != null && encoding != null) {
-        readDictionaryLengthStream(length, encoding);
-      }
-    }
-
-    @Override
-    void checkEncoding(OrcProto.ColumnEncoding encoding) throws IOException {
-      if (encoding.getKind() != OrcProto.ColumnEncoding.Kind.DICTIONARY &&
-          encoding.getKind() != OrcProto.ColumnEncoding.Kind.DICTIONARY_V2) {
-        throw new IOException("Unknown encoding " + encoding + " in column " +
-            columnId);
-      }
-    }
-
-    @Override
-    void startStripe(Map<StreamName, InStream> streams,
-        OrcProto.StripeFooter stripeFooter
-    ) throws IOException {
-      super.startStripe(streams, stripeFooter);
-
-      // read the dictionary blob
-      StreamName name = new StreamName(columnId,
-          OrcProto.Stream.Kind.DICTIONARY_DATA);
-      InStream in = streams.get(name);
-      readDictionaryStream(in);
-
-      // read the lengths
-      name = new StreamName(columnId, OrcProto.Stream.Kind.LENGTH);
-      in = streams.get(name);
-      readDictionaryLengthStream(in, stripeFooter.getColumnsList().get(columnId));
-
-      // set up the row reader
-      name = new StreamName(columnId, OrcProto.Stream.Kind.DATA);
-      reader = createIntegerReader(stripeFooter.getColumnsList().get(columnId).getKind(),
-          streams.get(name), false, false);
-    }
-
-    private void readDictionaryLengthStream(InStream in, OrcProto.ColumnEncoding encoding)
-        throws IOException {
-      int dictionarySize = encoding.getDictionarySize();
-      if (in != null) { // Guard against empty LENGTH stream.
-        IntegerReader lenReader = createIntegerReader(encoding.getKind(), in, false, false);
-        int offset = 0;
-        if (dictionaryOffsets == null ||
-            dictionaryOffsets.length < dictionarySize + 1) {
-          dictionaryOffsets = new int[dictionarySize + 1];
-        }
-        for (int i = 0; i < dictionarySize; ++i) {
-          dictionaryOffsets[i] = offset;
-          offset += (int) lenReader.next();
-        }
-        dictionaryOffsets[dictionarySize] = offset;
-        in.close();
-      }
-
-    }
-
-    private void readDictionaryStream(InStream in) throws IOException {
-      if (in != null) { // Guard against empty dictionary stream.
-        if (in.available() > 0) {
-          dictionaryBuffer = new DynamicByteArray(64, in.available());
-          dictionaryBuffer.readAll(in);
-          // Since its start of strip invalidate the cache.
-          dictionaryBufferInBytesCache = null;
-        }
-        in.close();
-      } else {
-        dictionaryBuffer = null;
-      }
-    }
-
-    @Override
-    void seek(PositionProvider[] index) throws IOException {
-      seek(index[columnId]);
-    }
-
-    @Override
-    public void seek(PositionProvider index) throws IOException {
-      super.seek(index);
-      reader.seek(index);
-    }
-
-    @Override
-    Object next(Object previous) throws IOException {
-      super.next(previous);
-      Text result = null;
-      if (valuePresent) {
-        int entry = (int) reader.next();
-        if (previous == null) {
-          result = new Text();
-        } else {
-          result = (Text) previous;
-        }
-        int offset = dictionaryOffsets[entry];
-        int length = getDictionaryEntryLength(entry, offset);
-        // If the column is just empty strings, the size will be zero,
-        // so the buffer will be null, in that case just return result
-        // as it will default to empty
-        if (dictionaryBuffer != null) {
-          dictionaryBuffer.setText(result, offset, length);
-        } else {
-          result.clear();
-        }
-      }
-      return result;
-    }
-
-    @Override
-    public void nextVector(ColumnVector previousVector,
-                           boolean[] isNull,
-                           final int batchSize) throws IOException {
-      final BytesColumnVector result = (BytesColumnVector) previousVector;
-      int offset;
-      int length;
-
-      // Read present/isNull stream
-      super.nextVector(result, isNull, batchSize);
-
-      if (dictionaryBuffer != null) {
-
-        // Load dictionaryBuffer into cache.
-        if (dictionaryBufferInBytesCache == null) {
-          dictionaryBufferInBytesCache = dictionaryBuffer.get();
-        }
-
-        // Read string offsets
-        scratchlcv.isNull = result.isNull;
-        scratchlcv.ensureSize((int) batchSize, false);
-        reader.nextVector(scratchlcv, scratchlcv.vector, batchSize);
-        if (!scratchlcv.isRepeating) {
-
-          // The vector has non-repeating strings. Iterate thru the batch
-          // and set strings one by one
-          for (int i = 0; i < batchSize; i++) {
-            if (!scratchlcv.isNull[i]) {
-              offset = dictionaryOffsets[(int) scratchlcv.vector[i]];
-              length = getDictionaryEntryLength((int) scratchlcv.vector[i], offset);
-              result.setRef(i, dictionaryBufferInBytesCache, offset, length);
-            } else {
-              // If the value is null then set offset and length to zero (null string)
-              result.setRef(i, dictionaryBufferInBytesCache, 0, 0);
-            }
-          }
-        } else {
-          // If the value is repeating then just set the first value in the
-          // vector and set the isRepeating flag to true. No need to iterate thru and
-          // set all the elements to the same value
-          offset = dictionaryOffsets[(int) scratchlcv.vector[0]];
-          length = getDictionaryEntryLength((int) scratchlcv.vector[0], offset);
-          result.setRef(0, dictionaryBufferInBytesCache, offset, length);
-        }
-        result.isRepeating = scratchlcv.isRepeating;
-      } else {
-        if (dictionaryOffsets == null) {
-          // Entire stripe contains null strings.
-          result.isRepeating = true;
-          result.noNulls = false;
-          result.isNull[0] = true;
-          result.setRef(0, EMPTY_BYTE_ARRAY, 0, 0);
-        } else {
-          // stripe contains nulls and empty strings
-          for (int i = 0; i < batchSize; i++) {
-            if (!result.isNull[i]) {
-              result.setRef(i, EMPTY_BYTE_ARRAY, 0, 0);
-            }
-          }
-        }
-      }
-    }
-
-    int getDictionaryEntryLength(int entry, int offset) {
-      final int length;
-      // if it isn't the last entry, subtract the offsets otherwise use
-      // the buffer length.
-      if (entry < dictionaryOffsets.length - 1) {
-        length = dictionaryOffsets[entry + 1] - offset;
-      } else {
-        length = dictionaryBuffer.size() - offset;
-      }
-      return length;
-    }
-
-    @Override
-    void skipRows(long items) throws IOException {
-      reader.skip(countNonNulls(items));
-    }
-
-    public IntegerReader getReader() {
-      return reader;
-    }
-  }
-
-  public static class CharTreeReader extends StringTreeReader {
-    int maxLength;
-
-    CharTreeReader(int columnId, int maxLength) throws IOException {
-      this(columnId, maxLength, null, null, null, null, null);
-    }
-
-    protected CharTreeReader(int columnId, int maxLength, InStream present, InStream data,
-        InStream length, InStream dictionary, OrcProto.ColumnEncoding encoding) throws IOException {
-      super(columnId, present, data, length, dictionary, encoding);
-      this.maxLength = maxLength;
-    }
-
-    @Override
-    Object next(Object previous) throws IOException {
-      final HiveCharWritable result;
-      if (previous == null) {
-        result = new HiveCharWritable();
-      } else {
-        result = (HiveCharWritable) previous;
-      }
-      // Use the string reader implementation to populate the internal Text value
-      Object textVal = super.next(result.getTextValue());
-      if (textVal == null) {
-        return null;
-      }
-      // result should now hold the value that was read in.
-      // enforce char length
-      result.enforceMaxLength(maxLength);
-      return result;
-    }
-
-    @Override
-    public void nextVector(ColumnVector previousVector,
-                           boolean[] isNull,
-                           final int batchSize) throws IOException {
-      // Get the vector of strings from StringTreeReader, then make a 2nd pass to
-      // adjust down the length (right trim and truncate) if necessary.
-      super.nextVector(previousVector, isNull, batchSize);
-      BytesColumnVector result = (BytesColumnVector) previousVector;
-      int adjustedDownLen;
-      if (result.isRepeating) {
-        if (result.noNulls || !result.isNull[0]) {
-          adjustedDownLen = StringExpr
-              .rightTrimAndTruncate(result.vector[0], result.start[0], result.length[0], maxLength);
-          if (adjustedDownLen < result.length[0]) {
-            result.setRef(0, result.vector[0], result.start[0], adjustedDownLen);
-          }
-        }
-      } else {
-        if (result.noNulls) {
-          for (int i = 0; i < batchSize; i++) {
-            adjustedDownLen = StringExpr
-                .rightTrimAndTruncate(result.vector[i], result.start[i], result.length[i],
-                    maxLength);
-            if (adjustedDownLen < result.length[i]) {
-              result.setRef(i, result.vector[i], result.start[i], adjustedDownLen);
-            }
-          }
-        } else {
-          for (int i = 0; i < batchSize; i++) {
-            if (!result.isNull[i]) {
-              adjustedDownLen = StringExpr
-                  .rightTrimAndTruncate(result.vector[i], result.start[i], result.length[i],
-                      maxLength);
-              if (adjustedDownLen < result.length[i]) {
-                result.setRef(i, result.vector[i], result.start[i], adjustedDownLen);
-              }
-            }
-          }
-        }
-      }
-    }
-  }
-
-  public static class VarcharTreeReader extends StringTreeReader {
-    int maxLength;
-
-    VarcharTreeReader(int columnId, int maxLength) throws IOException {
-      this(columnId, maxLength, null, null, null, null, null);
-    }
-
-    protected VarcharTreeReader(int columnId, int maxLength, InStream present, InStream data,
-        InStream length, InStream dictionary, OrcProto.ColumnEncoding encoding) throws IOException {
-      super(columnId, present, data, length, dictionary, encoding);
-      this.maxLength = maxLength;
-    }
-
-    @Override
-    Object next(Object previous) throws IOException {
-      final HiveVarcharWritable result;
-      if (previous == null) {
-        result = new HiveVarcharWritable();
-      } else {
-        result = (HiveVarcharWritable) previous;
-      }
-      // Use the string reader implementation to populate the internal Text value
-      Object textVal = super.next(result.getTextValue());
-      if (textVal == null) {
-        return null;
-      }
-      // result should now hold the value that was read in.
-      // enforce varchar length
-      result.enforceMaxLength(maxLength);
-      return result;
-    }
-
-    @Override
-    public void nextVector(ColumnVector previousVector,
-                           boolean[] isNull,
-                           final int batchSize) throws IOException {
-      // Get the vector of strings from StringTreeReader, then make a 2nd pass to
-      // adjust down the length (truncate) if necessary.
-      super.nextVector(previousVector, isNull, batchSize);
-      BytesColumnVector result = (BytesColumnVector) previousVector;
-
-      int adjustedDownLen;
-      if (result.isRepeating) {
-        if (result.noNulls || !result.isNull[0]) {
-          adjustedDownLen = StringExpr
-              .truncate(result.vector[0], result.start[0], result.length[0], maxLength);
-          if (adjustedDownLen < result.length[0]) {
-            result.setRef(0, result.vector[0], result.start[0], adjustedDownLen);
-          }
-        }
-      } else {
-        if (result.noNulls) {
-          for (int i = 0; i < batchSize; i++) {
-            adjustedDownLen = StringExpr
-                .truncate(result.vector[i], result.start[i], result.length[i], maxLength);
-            if (adjustedDownLen < result.length[i]) {
-              result.setRef(i, result.vector[i], result.start[i], adjustedDownLen);
-            }
-          }
-        } else {
-          for (int i = 0; i < batchSize; i++) {
-            if (!result.isNull[i]) {
-              adjustedDownLen = StringExpr
-                  .truncate(result.vector[i], result.start[i], result.length[i], maxLength);
-              if (adjustedDownLen < result.length[i]) {
-                result.setRef(i, result.vector[i], result.start[i], adjustedDownLen);
-              }
-            }
-          }
-        }
-      }
-    }
-  }
-
-  protected static class StructTreeReader extends TreeReader {
-    protected final TreeReader[] fields;
-
-    protected StructTreeReader(int columnId,
-                               TypeDescription readerSchema,
-                               SchemaEvolution evolution,
-                               boolean[] included,
-                               boolean skipCorrupt) throws IOException {
-      super(columnId);
-
-      TypeDescription fileSchema = evolution.getFileType(readerSchema);
-
-      List<TypeDescription> childrenTypes = readerSchema.getChildren();
-      this.fields = new TreeReader[childrenTypes.size()];
-      for (int i = 0; i < fields.length; ++i) {
-        TypeDescription subtype = childrenTypes.get(i);
-        this.fields[i] = createTreeReader(subtype, evolution, included, skipCorrupt);
-      }
-    }
-
-    @Override
-    void seek(PositionProvider[] index) throws IOException {
-      super.seek(index);
-      for (TreeReader kid : fields) {
-        if (kid != null) {
-          kid.seek(index);
-        }
-      }
-    }
-
-    @Override
-    Object next(Object previous) throws IOException {
-      super.next(previous);
-      OrcStruct result = null;
-      if (valuePresent) {
-        if (previous == null) {
-          result = new OrcStruct(fields.length);
-        } else {
-          result = (OrcStruct) previous;
-
-          // If the input format was initialized with a file with a
-          // different number of fields, the number of fields needs to
-          // be updated to the correct number
-          result.setNumFields(fields.length);
-        }
-        for (int i = 0; i < fields.length; ++i) {
-          if (fields[i] != null) {
-            result.setFieldValue(i, fields[i].next(result.getFieldValue(i)));
-          }
-        }
-      }
-      return result;
-    }
-
-    @Override
-    public void nextBatch(VectorizedRowBatch batch,
-                          int batchSize) throws IOException {
-      for(int i=0; i < fields.length &&
-          (vectorColumnCount == -1 || i < vectorColumnCount); ++i) {
-        batch.cols[i].reset();
-        batch.cols[i].ensureSize((int) batchSize, false);
-        fields[i].nextVector(batch.cols[i], null, batchSize);
-      }
-    }
-
-    @Override
-    public void nextVector(ColumnVector previousVector,
-                           boolean[] isNull,
-                           final int batchSize) throws IOException {
-      super.nextVector(previousVector, isNull, batchSize);
-      StructColumnVector result = (StructColumnVector) previousVector;
-      if (result.noNulls || !(result.isRepeating && result.isNull[0])) {
-        result.isRepeating = false;
-
-        // Read all the members of struct as column vectors
-        boolean[] mask = result.noNulls ? null : result.isNull;
-        for (int f = 0; f < fields.length; f++) {
-          if (fields[f] != null) {
-            fields[f].nextVector(result.fields[f], mask, batchSize);
-          }
-        }
-      }
-    }
-
-    @Override
-    void startStripe(Map<StreamName, InStream> streams,
-        OrcProto.StripeFooter stripeFooter
-    ) throws IOException {
-      super.startStripe(streams, stripeFooter);
-      for (TreeReader field : fields) {
-        if (field != null) {
-          field.startStripe(streams, stripeFooter);
-        }
-      }
-    }
-
-    @Override
-    void skipRows(long items) throws IOException {
-      items = countNonNulls(items);
-      for (TreeReader field : fields) {
-        if (field != null) {
-          field.skipRows(items);
-        }
-      }
-    }
-  }
-
-  public static class UnionTreeReader extends TreeReader {
-    protected final TreeReader[] fields;
-    protected RunLengthByteReader tags;
-
-    protected UnionTreeReader(int fileColumn,
-                              TypeDescription readerSchema,
-                              SchemaEvolution evolution,
-                              boolean[] included,
-                              boolean skipCorrupt) throws IOException {
-      super(fileColumn);
-      List<TypeDescription> childrenTypes = readerSchema.getChildren();
-      int fieldCount = childrenTypes.size();
-      this.fields = new TreeReader[fieldCount];
-      for (int i = 0; i < fieldCount; ++i) {
-        TypeDescription subtype = childrenTypes.get(i);
-        this.fields[i] = createTreeReader(subtype, evolution, included, skipCorrupt);
-      }
-    }
-
-    @Override
-    void seek(PositionProvider[] index) throws IOException {
-      super.seek(index);
-      tags.seek(index[columnId]);
-      for (TreeReader kid : fields) {
-        kid.seek(index);
-      }
-    }
-
-    @Override
-    Object next(Object previous) throws IOException {
-      super.next(previous);
-      OrcUnion result = null;
-      if (valuePresent) {
-        if (previous == null) {
-          result = new OrcUnion();
-        } else {
-          result = (OrcUnion) previous;
-        }
-        byte tag = tags.next();
-        Object previousVal = result.getObject();
-        result.set(tag, fields[tag].next(tag == result.getTag() ?
-            previousVal : null));
-      }
-      return result;
-    }
-
-    @Override
-    public void nextVector(ColumnVector previousVector,
-                           boolean[] isNull,
-                           final int batchSize) throws IOException {
-      UnionColumnVector result = (UnionColumnVector) previousVector;
-      super.nextVector(result, isNull, batchSize);
-      if (result.noNulls || !(result.isRepeating && result.isNull[0])) {
-        result.isRepeating = false;
-        tags.nextVector(result.noNulls ? null : result.isNull, result.tags,
-            batchSize);
-        boolean[] ignore = new boolean[(int) batchSize];
-        for (int f = 0; f < result.fields.length; ++f) {
-          // build the ignore list for this tag
-          for (int r = 0; r < batchSize; ++r) {
-            ignore[r] = (!result.noNulls && result.isNull[r]) ||
-                result.tags[r] != f;
-          }
-          fields[f].nextVector(result.fields[f], ignore, batchSize);
-        }
-      }
-    }
-
-    @Override
-    void startStripe(Map<StreamName, InStream> streams,
-        OrcProto.StripeFooter stripeFooter
-    ) throws IOException {
-      super.startStripe(streams, stripeFooter);
-      tags = new RunLengthByteReader(streams.get(new StreamName(columnId,
-          OrcProto.Stream.Kind.DATA)));
-      for (TreeReader field : fields) {
-        if (field != null) {
-          field.startStripe(streams, stripeFooter);
-        }
-      }
-    }
-
-    @Override
-    void skipRows(long items) throws IOException {
-      items = countNonNulls(items);
-      long[] counts = new long[fields.length];
-      for (int i = 0; i < items; ++i) {
-        counts[tags.next()] += 1;
-      }
-      for (int i = 0; i < counts.length; ++i) {
-        fields[i].skipRows(counts[i]);
-      }
-    }
-  }
-
-  public static class ListTreeReader extends TreeReader {
-    protected final TreeReader elementReader;
-    protected IntegerReader lengths = null;
-
-    protected ListTreeReader(int fileColumn,
-                             TypeDescription readerSchema,
-                             SchemaEvolution evolution,
-                             boolean[] included,
-                             boolean skipCorrupt) throws IOException {
-      super(fileColumn);
-      TypeDescription elementType = readerSchema.getChildren().get(0);
-      elementReader = createTreeReader(elementType, evolution, included,
-          skipCorrupt);
-    }
-
-    @Override
-    void seek(PositionProvider[] index) throws IOException {
-      super.seek(index);
-      lengths.seek(index[columnId]);
-      elementReader.seek(index);
-    }
-
-    @Override
-    @SuppressWarnings("unchecked")
-    Object next(Object previous) throws IOException {
-      super.next(previous);
-      List<Object> result = null;
-      if (valuePresent) {
-        if (previous == null) {
-          result = new ArrayList<>();
-        } else {
-          result = (ArrayList<Object>) previous;
-        }
-        int prevLength = result.size();
-        int length = (int) lengths.next();
-        // extend the list to the new length
-        for (int i = prevLength; i < length; ++i) {
-          result.add(null);
-        }
-        // read the new elements into the array
-        for (int i = 0; i < length; i++) {
-          result.set(i, elementReader.next(i < prevLength ?
-              result.get(i) : null));
-        }
-        // remove any extra elements
-        for (int i = prevLength - 1; i >= length; --i) {
-          result.remove(i);
-        }
-      }
-      return result;
-    }
-
-    @Override
-    public void nextVector(ColumnVector previous,
-                           boolean[] isNull,
-                           final int batchSize) throws IOException {
-      ListColumnVector result = (ListColumnVector) previous;
-      super.nextVector(result, isNull, batchSize);
-      // if we have some none-null values, then read them
-      if (result.noNulls || !(result.isRepeating && result.isNull[0])) {
-        lengths.nextVector(result, result.lengths, batchSize);
-        // even with repeating lengths, the list doesn't repeat
-        result.isRepeating = false;
-        // build the offsets vector and figure out how many children to read
-        result.childCount = 0;
-        for (int r = 0; r < batchSize; ++r) {
-          if (result.noNulls || !result.isNull[r]) {
-            result.offsets[r] = result.childCount;
-            result.childCount += result.lengths[r];
-          }
-        }
-        result.child.ensureSize(result.childCount, false);
-        elementReader.nextVector(result.child, null, result.childCount);
-      }
-    }
-
-    @Override
-    void checkEncoding(OrcProto.ColumnEncoding encoding) throws IOException {
-      if ((encoding.getKind() != OrcProto.ColumnEncoding.Kind.DIRECT) &&
-          (encoding.getKind() != OrcProto.ColumnEncoding.Kind.DIRECT_V2)) {
-        throw new IOException("Unknown encoding " + encoding + " in column " +
-            columnId);
-      }
-    }
-
-    @Override
-    void startStripe(Map<StreamName, InStream> streams,
-        OrcProto.StripeFooter stripeFooter
-    ) throws IOException {
-      super.startStripe(streams, stripeFooter);
-      lengths = createIntegerReader(stripeFooter.getColumnsList().get(columnId).getKind(),
-          streams.get(new StreamName(columnId,
-              OrcProto.Stream.Kind.LENGTH)), false, false);
-      if (elementReader != null) {
-        elementReader.startStripe(streams, stripeFooter);
-      }
-    }
-
-    @Override
-    void skipRows(long items) throws IOException {
-      items = countNonNulls(items);
-      long childSkip = 0;
-      for (long i = 0; i < items; ++i) {
-        childSkip += lengths.next();
-      }
-      elementReader.skipRows(childSkip);
-    }
-  }
-
-  public static class MapTreeReader extends TreeReader {
-    protected final TreeReader keyReader;
-    protected final TreeReader valueReader;
-    protected IntegerReader lengths = null;
-
-    protected MapTreeReader(int fileColumn,
-                            TypeDescription readerSchema,
-                            SchemaEvolution evolution,
-                            boolean[] included,
-                            boolean skipCorrupt) throws IOException {
-      super(fileColumn);
-      TypeDescription keyType = readerSchema.getChildren().get(0);
-      TypeDescription valueType = readerSchema.getChildren().get(1);
-      keyReader = createTreeReader(keyType, evolution, included, skipCorrupt);
-      valueReader = createTreeReader(valueType, evolution, included, skipCorrupt);
-    }
-
-    @Override
-    void seek(PositionProvider[] index) throws IOException {
-      super.seek(index);
-      lengths.seek(index[columnId]);
-      keyReader.seek(index);
-      valueReader.seek(index);
-    }
-
-    @Override
-    @SuppressWarnings("unchecked")
-    Object next(Object previous) throws IOException {
-      super.next(previous);
-      Map<Object, Object> result = null;
-      if (valuePresent) {
-        if (previous == null) {
-          result = new LinkedHashMap<>();
-        } else {
-          result = (LinkedHashMap<Object, Object>) previous;
-        }
-        // for now just clear and create new objects
-        result.clear();
-        int length = (int) lengths.next();
-        // read the new elements into the array
-        for (int i = 0; i < length; i++) {
-          result.put(keyReader.next(null), valueReader.next(null));
-        }
-      }
-      return result;
-    }
-
-    @Override
-    public void nextVector(ColumnVector previous,
-                           boolean[] isNull,
-                           final int batchSize) throws IOException {
-      MapColumnVector result = (MapColumnVector) previous;
-      super.nextVector(result, isNull, batchSize);
-      if (result.noNulls || !(result.isRepeating && result.isNull[0])) {
-        lengths.nextVector(result, result.lengths, batchSize);
-        // even with repeating lengths, the map doesn't repeat
-        result.isRepeating = false;
-        // build the offsets vector and figure out how many children to read
-        result.childCount = 0;
-        for (int r = 0; r < batchSize; ++r) {
-          if (result.noNulls || !result.isNull[r]) {
-            result.offsets[r] = result.childCount;
-            result.childCount += result.lengths[r];
-          }
-        }
-        result.keys.ensureSize(result.childCount, false);
-        result.values.ensureSize(result.childCount, false);
-        keyReader.nextVector(result.keys, null, result.childCount);
-        valueReader.nextVector(result.values, null, result.childCount);
-      }
-    }
-
-    @Override
-    void checkEncoding(OrcProto.ColumnEncoding encoding) throws IOException {
-      if ((encoding.getKind() != OrcProto.ColumnEncoding.Kind.DIRECT) &&
-          (encoding.getKind() != OrcProto.ColumnEncoding.Kind.DIRECT_V2)) {
-        throw new IOException("Unknown encoding " + encoding + " in column " +
-            columnId);
-      }
-    }
-
-    @Override
-    void startStripe(Map<StreamName, InStream> streams,
-        OrcProto.StripeFooter stripeFooter
-    ) throws IOException {
-      super.startStripe(streams, stripeFooter);
-      lengths = createIntegerReader(stripeFooter.getColumnsList().get(columnId).getKind(),
-          streams.get(new StreamName(columnId,
-              OrcProto.Stream.Kind.LENGTH)), false, false);
-      if (keyReader != null) {
-        keyReader.startStripe(streams, stripeFooter);
-      }
-      if (valueReader != null) {
-        valueReader.startStripe(streams, stripeFooter);
-      }
-    }
-
-    @Override
-    void skipRows(long items) throws IOException {
-      items = countNonNulls(items);
-      long childSkip = 0;
-      for (long i = 0; i < items; ++i) {
-        childSkip += lengths.next();
-      }
-      keyReader.skipRows(childSkip);
-      valueReader.skipRows(childSkip);
-    }
-  }
-
-  public static TreeReader createTreeReader(TypeDescription readerType,
-                                            SchemaEvolution evolution,
-                                            boolean[] included,
-                                            boolean skipCorrupt
-                                            ) throws IOException {
-    TypeDescription fileType = evolution.getFileType(readerType);
-    if (fileType == null ||
-        (included != null && !included[readerType.getId()])) {
-      return new NullTreeReader(0);
-    }
-    TypeDescription.Category readerTypeCategory = readerType.getCategory();
-    if (!fileType.getCategory().equals(readerTypeCategory) &&
-        (readerTypeCategory != TypeDescription.Category.STRUCT &&
-         readerTypeCategory != TypeDescription.Category.MAP &&
-         readerTypeCategory != TypeDescription.Category.LIST &&
-         readerTypeCategory != TypeDescription.Category.UNION)) {
-      // We only convert complex children.
-      return ConvertTreeReaderFactory.createConvertTreeReader(readerType, evolution,
-          included, skipCorrupt);
-    }
-    switch (readerTypeCategory) {
-      case BOOLEAN:
-        return new BooleanTreeReader(fileType.getId());
-      case BYTE:
-        return new ByteTreeReader(fileType.getId());
-      case DOUBLE:
-        return new DoubleTreeReader(fileType.getId());
-      case FLOAT:
-        return new FloatTreeReader(fileType.getId());
-      case SHORT:
-        return new ShortTreeReader(fileType.getId());
-      case INT:
-        return new IntTreeReader(fileType.getId());
-      case LONG:
-        return new LongTreeReader(fileType.getId(), skipCorrupt);
-      case STRING:
-        return new StringTreeReader(fileType.getId());
-      case CHAR:
-        return new CharTreeReader(fileType.getId(), readerType.getMaxLength());
-      case VARCHAR:
-        return new VarcharTreeReader(fileType.getId(), readerType.getMaxLength());
-      case BINARY:
-        return new BinaryTreeReader(fileType.getId());
-      case TIMESTAMP:
-        return new TimestampTreeReader(fileType.getId(), skipCorrupt);
-      case DATE:
-        return new DateTreeReader(fileType.getId());
-      case DECIMAL:
-        return new DecimalTreeReader(fileType.getId(), readerType.getPrecision(),
-            readerType.getScale());
-      case STRUCT:
-        return new StructTreeReader(fileType.getId(), readerType,
-            evolution, included, skipCorrupt);
-      case LIST:
-        return new ListTreeReader(fileType.getId(), readerType,
-            evolution, included, skipCorrupt);
-      case MAP:
-        return new MapTreeReader(fileType.getId(), readerType, evolution,
-            included, skipCorrupt);
-      case UNION:
-        return new UnionTreeReader(fileType.getId(), readerType,
-            evolution, included, skipCorrupt);
-      default:
-        throw new IllegalArgumentException("Unsupported type " +
-            readerTypeCategory);
-    }
-  }
-}


[14/27] hive git commit: HIVE-11417. Move the ReaderImpl and RowReaderImpl to the ORC module, by making shims for the row by row reader. (omalley reviewed by prasanth_j)

Posted by om...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/ql/src/java/org/apache/hadoop/hive/ql/io/orc/ConvertTreeReaderFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/ConvertTreeReaderFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/ConvertTreeReaderFactory.java
deleted file mode 100644
index 74a097e..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/ConvertTreeReaderFactory.java
+++ /dev/null
@@ -1,3750 +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.hive.ql.io.orc;
-
-import java.io.IOException;
-import java.io.UnsupportedEncodingException;
-import java.nio.charset.StandardCharsets;
-import java.sql.Date;
-import java.sql.Timestamp;
-import java.util.ArrayList;
-import java.util.EnumMap;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.hadoop.hive.common.type.HiveDecimal;
-import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.StringExpr;
-import org.apache.hadoop.hive.serde2.io.ByteWritable;
-import org.apache.hadoop.hive.serde2.io.DateWritable;
-import org.apache.hadoop.hive.serde2.io.DoubleWritable;
-import org.apache.hadoop.hive.serde2.io.HiveCharWritable;
-import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
-import org.apache.hadoop.hive.serde2.io.HiveVarcharWritable;
-import org.apache.hadoop.hive.serde2.io.ShortWritable;
-import org.apache.hadoop.hive.serde2.io.TimestampWritable;
-import org.apache.hadoop.io.BooleanWritable;
-import org.apache.hadoop.io.BytesWritable;
-import org.apache.hadoop.io.FloatWritable;
-import org.apache.hadoop.io.IntWritable;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
-import org.apache.orc.OrcProto;
-import org.apache.orc.TypeDescription;
-import org.apache.orc.TypeDescription.Category;
-import org.apache.orc.impl.InStream;
-import org.apache.orc.impl.PositionProvider;
-import org.apache.orc.impl.StreamName;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Convert ORC tree readers.
- */
-public class ConvertTreeReaderFactory extends TreeReaderFactory {
-
-  private static final Logger LOG =
-    LoggerFactory.getLogger(TreeReaderFactory.class);
-
-  /**
-   * Override methods like checkEncoding to pass-thru to the convert TreeReader.
-   */
-  public static class ConvertTreeReader extends TreeReader {
-
-    private TreeReader convertTreeReader;
-
-    ConvertTreeReader(int columnId) throws IOException {
-      super(columnId);
-    }
-
-    private static List<TypeDescription.Category> numericTypeList = new ArrayList<TypeDescription.Category>();
-
-    // The ordering of types here is used to determine which numeric types
-    // are common/convertible to one another. Probably better to rely on the
-    // ordering explicitly defined here than to assume that the enum values
-    // that were arbitrarily assigned in PrimitiveCategory work for our purposes.
-    private static EnumMap<TypeDescription.Category, Integer> numericTypes =
-        new EnumMap<TypeDescription.Category, Integer>(TypeDescription.Category.class);
-
-    static {
-      registerNumericType(TypeDescription.Category.BOOLEAN, 1);
-      registerNumericType(TypeDescription.Category.BYTE, 2);
-      registerNumericType(TypeDescription.Category.SHORT, 3);
-      registerNumericType(TypeDescription.Category.INT, 4);
-      registerNumericType(TypeDescription.Category.LONG, 5);
-      registerNumericType(TypeDescription.Category.FLOAT, 6);
-      registerNumericType(TypeDescription.Category.DOUBLE, 7);
-      registerNumericType(TypeDescription.Category.DECIMAL, 8);
-    }
-
-    private static void registerNumericType(TypeDescription.Category kind, int level) {
-      numericTypeList.add(kind);
-      numericTypes.put(kind, level);
-    }
-
-    protected void setConvertTreeReader(TreeReader convertTreeReader) {
-      this.convertTreeReader = convertTreeReader;
-    }
-
-    protected TreeReader getStringGroupTreeReader(int columnId,
-        TypeDescription fileType) throws IOException {
-      switch (fileType.getCategory()) {
-      case STRING:
-        return new StringTreeReader(columnId);
-      case CHAR:
-        return new CharTreeReader(columnId, fileType.getMaxLength());
-      case VARCHAR:
-        return new VarcharTreeReader(columnId, fileType.getMaxLength());
-      default:
-        throw new RuntimeException("Unexpected type kind " + fileType.getCategory().name());
-      }
-    }
-
-    protected Writable getStringGroupWritable(TypeDescription fileType)
-        throws IOException {
-      switch (fileType.getCategory()) {
-      case STRING:
-        return new Text();
-      case CHAR:
-        return new HiveCharWritable();
-      case VARCHAR:
-        return new HiveVarcharWritable();
-      default:
-        throw new RuntimeException("Unexpected type kind " + fileType.getCategory().name());
-      }
-    }
-
-    protected Writable getStringGroupResultFromString(Object previous,
-        TypeDescription readerType, String string) {
-      switch (readerType.getCategory()) {
-      case STRING:
-      {
-          Text textResult;
-          if (previous == null) {
-            textResult = new Text();
-          } else {
-            textResult = (Text) previous;
-          }
-          textResult.set(string);
-          return textResult;
-        }
-      case CHAR:
-        {
-          HiveCharWritable hiveCharResult;
-          if (previous == null) {
-            hiveCharResult = new HiveCharWritable();
-          } else {
-            hiveCharResult = (HiveCharWritable) previous;
-          }
-          hiveCharResult.set(string, readerType.getMaxLength());
-          return hiveCharResult;
-        }
-      case VARCHAR:
-      {
-        HiveVarcharWritable hiveVarcharResult;
-        if (previous == null) {
-          hiveVarcharResult = new HiveVarcharWritable();
-        } else {
-          hiveVarcharResult = (HiveVarcharWritable) previous;
-        }
-        hiveVarcharResult.set(string, readerType.getMaxLength());
-        return hiveVarcharResult;
-      }
-      default:
-        throw new RuntimeException("Unexpected type kind " + readerType.getCategory().name());
-      }
-    }
-
-    protected void assignStringGroupVectorEntry(BytesColumnVector bytesColVector,
-        int elementNum, TypeDescription readerType, byte[] bytes) {
-      assignStringGroupVectorEntry(bytesColVector,
-          elementNum, readerType, bytes, 0, bytes.length);
-    }
-
-    /*
-     * Assign a BytesColumnVector entry when we have a byte array, start, and
-     * length for the string group which can be (STRING, CHAR, VARCHAR).
-     */
-    protected void assignStringGroupVectorEntry(BytesColumnVector bytesColVector,
-        int elementNum, TypeDescription readerType, byte[] bytes, int start, int length) {
-      switch (readerType.getCategory()) {
-      case STRING:
-        bytesColVector.setVal(elementNum, bytes, start, length);
-        break;
-      case CHAR:
-        {
-          int adjustedDownLen =
-              StringExpr.rightTrimAndTruncate(bytes, start, length, readerType.getMaxLength());
-          bytesColVector.setVal(elementNum, bytes, start, adjustedDownLen);
-        }
-        break;
-      case VARCHAR:
-        {
-          int adjustedDownLen =
-              StringExpr.truncate(bytes, start, length, readerType.getMaxLength());
-          bytesColVector.setVal(elementNum, bytes, start, adjustedDownLen);
-        }
-        break;
-      default:
-        throw new RuntimeException("Unexpected type kind " + readerType.getCategory().name());
-      }
-    }
-
-    protected void convertStringGroupVectorElement(BytesColumnVector bytesColVector,
-        int elementNum, TypeDescription readerType) {
-      switch (readerType.getCategory()) {
-      case STRING:
-        // No conversion needed.
-        break;
-      case CHAR:
-        {
-          int length = bytesColVector.length[elementNum];
-          int adjustedDownLen = StringExpr
-            .rightTrimAndTruncate(bytesColVector.vector[elementNum],
-                bytesColVector.start[elementNum], length,
-                readerType.getMaxLength());
-          if (adjustedDownLen < length) {
-            bytesColVector.length[elementNum] = adjustedDownLen;
-          }
-        }
-        break;
-      case VARCHAR:
-        {
-          int length = bytesColVector.length[elementNum];
-          int adjustedDownLen = StringExpr
-            .truncate(bytesColVector.vector[elementNum],
-                bytesColVector.start[elementNum], length,
-                readerType.getMaxLength());
-          if (adjustedDownLen < length) {
-            bytesColVector.length[elementNum] = adjustedDownLen;
-          }
-        }
-        break;
-      default:
-        throw new RuntimeException("Unexpected type kind " + readerType.getCategory().name());
-      }
-    }
-
-    private boolean isParseError;
-
-    /*
-     * We do this because we want the various parse methods return a primitive.
-     *
-     * @return true if there was a parse error in the last call to
-     * parseLongFromString, etc.
-     */
-    protected boolean getIsParseError() {
-      return isParseError;
-    }
-
-    protected long parseLongFromString(String string) {
-      try {
-        long longValue = Long.parseLong(string);
-        isParseError = false;
-        return longValue;
-      } catch (NumberFormatException e) {
-        isParseError = true;
-        return 0;
-      }
-    }
-
-    protected float parseFloatFromString(String string) {
-      try {
-        float floatValue = Float.parseFloat(string);
-        isParseError = false;
-        return floatValue;
-      } catch (NumberFormatException e) {
-        isParseError = true;
-        return Float.NaN;
-      }
-    }
-
-    protected double parseDoubleFromString(String string) {
-      try {
-        double value = Double.parseDouble(string);
-        isParseError = false;
-        return value;
-      } catch (NumberFormatException e) {
-        isParseError = true;
-        return Double.NaN;
-      }
-    }
-
-    /**
-     * @param string
-     * @return the HiveDecimal parsed, or null if there was a parse error.
-     */
-    protected HiveDecimal parseDecimalFromString(String string) {
-      try {
-        HiveDecimal value = HiveDecimal.create(string);
-        return value;
-      } catch (NumberFormatException e) {
-        return null;
-      }
-    }
-
-    /**
-     * @param string
-     * @return the Timestamp parsed, or null if there was a parse error.
-     */
-    protected Timestamp parseTimestampFromString(String string) {
-      try {
-        Timestamp value = Timestamp.valueOf(string);
-        return value;
-      } catch (IllegalArgumentException e) {
-        return null;
-      }
-    }
-
-    /**
-     * @param string
-     * @return the Date parsed, or null if there was a parse error.
-     */
-    protected Date parseDateFromString(String string) {
-      try {
-        Date value = Date.valueOf(string);
-        return value;
-      } catch (IllegalArgumentException e) {
-        return null;
-      }
-    }
-
-    protected String stringFromStringGroupTreeReader(
-        TreeReader stringGroupTreeReader, Writable writable, 
-        TypeDescription fileType) throws IOException {
-      switch (fileType.getCategory()) {
-      case STRING:
-        {
-          Text readTextResult =
-            (Text) ((StringTreeReader) stringGroupTreeReader).next(writable);
-          if (readTextResult == null) {
-            return null;
-          }
-          return readTextResult.toString();
-        }
-      case CHAR:
-        {
-          HiveCharWritable readHiveCharResult =
-            (HiveCharWritable) ((CharTreeReader) stringGroupTreeReader).next(writable);
-          if (readHiveCharResult == null) {
-            return null;
-          }
-          return readHiveCharResult.getStrippedValue().toString();
-        }
-      case VARCHAR:
-        {
-          HiveVarcharWritable readHiveVarcharResult =
-            (HiveVarcharWritable) ((VarcharTreeReader) stringGroupTreeReader).next(writable);
-          if (readHiveVarcharResult == null) {
-            return null;
-          }
-          return readHiveVarcharResult.toString();
-        }
-      default:
-        throw new RuntimeException("Unexpected type kind " + fileType.getCategory().name());
-      }
-    }
-
-    protected String stringFromBytesColumnVectorEntry(
-        BytesColumnVector bytesColVector, int elementNum) {
-      String string;
-
-      string = new String(
-          bytesColVector.vector[elementNum],
-          bytesColVector.start[elementNum], bytesColVector.length[elementNum],
-          StandardCharsets.UTF_8);
- 
-      return string;
-    }
-
-    @Override
-    void checkEncoding(OrcProto.ColumnEncoding encoding) throws IOException {
-      // Pass-thru.
-      convertTreeReader.checkEncoding(encoding);
-    }
-
-    @Override
-    void startStripe(Map<StreamName, InStream> streams,
-        OrcProto.StripeFooter stripeFooter
-    ) throws IOException {
-      // Pass-thru.
-      convertTreeReader.startStripe(streams, stripeFooter);
-    }
-
-    @Override
-    void seek(PositionProvider[] index) throws IOException {
-     // Pass-thru.
-      convertTreeReader.seek(index);
-    }
-
-    @Override
-    public void seek(PositionProvider index) throws IOException {
-      // Pass-thru.
-      convertTreeReader.seek(index);
-    }
-
-    @Override
-    void skipRows(long items) throws IOException {
-      // Pass-thru.
-      convertTreeReader.skipRows(items);
-    }
-
-    /**
-     * Override this to use convertVector.
-     * Source and result are member variables in the subclass with the right
-     * type.
-     * @param elementNum
-     * @throws IOException
-     */
-    // Override this to use convertVector.
-    public void setConvertVectorElement(int elementNum) throws IOException {
-      throw new RuntimeException("Expected this method to be overriden");
-    }
-
-    // Common code used by the conversion.
-    public void convertVector(ColumnVector fromColVector,
-        ColumnVector resultColVector, final int batchSize) throws IOException {
-
-      resultColVector.reset();
-      if (fromColVector.isRepeating) {
-        resultColVector.isRepeating = true;
-        if (fromColVector.noNulls || !fromColVector.isNull[0]) {
-          setConvertVectorElement(0);
-        } else {
-          resultColVector.noNulls = false;
-          resultColVector.isNull[0] = true;
-        }
-      } else if (fromColVector.noNulls){
-        for (int i = 0; i < batchSize; i++) {
-          setConvertVectorElement(i);
-        }
-      } else {
-        for (int i = 0; i < batchSize; i++) {
-          if (!fromColVector.isNull[i]) {
-            setConvertVectorElement(i);
-          } else {
-            resultColVector.noNulls = false;
-            resultColVector.isNull[i] = true;
-          }
-        }
-      }
-    }
-
-    public long downCastAnyInteger(long input, TypeDescription readerType) {
-      switch (readerType.getCategory()) {
-      case BOOLEAN:
-        return input == 0 ? 0 : 1;
-      case BYTE:
-        return (byte) input;
-      case SHORT:
-        return (short) input;
-      case INT:
-        return (int) input;
-      case LONG:
-        return input;
-      default:
-        throw new RuntimeException("Unexpected type kind " + readerType.getCategory().name());
-      }
-    }
-
-    protected Writable anyIntegerWritable(long longValue, Object previous,
-        TypeDescription readerType) {
-      switch (readerType.getCategory()) {
-        case BOOLEAN:
-        {
-          BooleanWritable booleanResult;
-          if (previous == null) {
-            booleanResult = new BooleanWritable();
-          } else {
-            booleanResult = (BooleanWritable) previous;
-          }
-          booleanResult.set(longValue != 0);
-          return booleanResult;
-        }
-      case BYTE:
-        {
-          ByteWritable byteResult;
-          if (previous == null) {
-            byteResult = new ByteWritable();
-          } else {
-            byteResult = (ByteWritable) previous;
-          }
-          byteResult.set((byte) longValue);
-          return byteResult;
-        }
-      case SHORT:
-        {
-          ShortWritable shortResult;
-          if (previous == null) {
-            shortResult = new ShortWritable();
-          } else {
-            shortResult = (ShortWritable) previous;
-          }
-          shortResult.set((short) longValue);
-          return shortResult;
-        }
-      case INT:
-        {
-          IntWritable intResult;
-          if (previous == null) {
-            intResult = new IntWritable();
-          } else {
-            intResult = (IntWritable) previous;
-          }
-          intResult.set((int) longValue);
-          return intResult;
-        }
-      case LONG:
-        {
-          LongWritable longResult;
-          if (previous == null) {
-            longResult = new LongWritable();
-          } else {
-            longResult = (LongWritable) previous;
-          }
-          longResult.set(longValue);
-          return longResult;
-        }
-      default:
-        throw new RuntimeException("Unexpected type kind " + readerType.getCategory().name());
-      }
-    }
-
-    protected boolean integerDownCastNeeded(TypeDescription fileType, TypeDescription readerType) {
-      Integer fileLevel = numericTypes.get(fileType.getCategory());
-      Integer schemaLevel = numericTypes.get(readerType.getCategory());
-      return (schemaLevel.intValue() < fileLevel.intValue());
-    }
-  }
-
-  public static class AnyIntegerTreeReader extends ConvertTreeReader {
-
-    private TypeDescription.Category fileTypeCategory;
-    private TreeReader anyIntegerTreeReader;
-
-    private long longValue;
-
-    AnyIntegerTreeReader(int columnId, TypeDescription fileType,
-        boolean skipCorrupt) throws IOException {
-      super(columnId);
-      this.fileTypeCategory = fileType.getCategory();
-      switch (fileTypeCategory) {
-      case BOOLEAN:
-        anyIntegerTreeReader = new BooleanTreeReader(columnId);
-        break;
-      case BYTE:
-        anyIntegerTreeReader = new ByteTreeReader(columnId);
-        break;
-      case SHORT:
-        anyIntegerTreeReader = new ShortTreeReader(columnId);
-        break;
-      case INT:
-        anyIntegerTreeReader = new IntTreeReader(columnId);
-        break;
-      case LONG:
-        anyIntegerTreeReader = new LongTreeReader(columnId, skipCorrupt);
-        break;
-      default:
-        throw new RuntimeException("Unexpected type kind " + fileType.getCategory().name());
-      }
-      setConvertTreeReader(anyIntegerTreeReader);
-    }
-
-    @Override
-    Object next(Object previous) throws IOException {
-      throw new RuntimeException("Call read() and getLong instead");
-    }
-
-    protected boolean read() throws IOException {
-      anyIntegerTreeReader.readValuePresent();
-      if (!anyIntegerTreeReader.valuePresent) {
-        return false;
-      }
-      switch (fileTypeCategory) {
-      case BOOLEAN:
-        longValue = ((BooleanTreeReader) anyIntegerTreeReader).reader.next();
-        break;
-      case BYTE:
-        longValue = ((ByteTreeReader) anyIntegerTreeReader).reader.next();
-        break;
-      case SHORT:
-        longValue = ((ShortTreeReader) anyIntegerTreeReader).reader.next();
-        break;
-      case INT:
-        longValue = ((IntTreeReader) anyIntegerTreeReader).reader.next();
-        break;
-      case LONG:
-        longValue = ((LongTreeReader) anyIntegerTreeReader).reader.next();
-        break;
-      default:
-        throw new RuntimeException("Unexpected type kind " + fileTypeCategory.name());
-      }
-      return true;
-    }
-
-    protected long getLong() throws IOException {
-      return longValue;
-    }
-
-    protected String getString(long longValue) {
-      if (fileTypeCategory == TypeDescription.Category.BOOLEAN) {
-        return longValue == 0 ? "FALSE" : "TRUE";
-      } else {
-        return Long.toString(longValue);
-      }
-    }
-
-    protected String getString() {
-      return getString(longValue);
-    }
-
-    @Override
-    public void nextVector(ColumnVector previousVector,
-                           boolean[] isNull,
-                           final int batchSize) throws IOException {
-      anyIntegerTreeReader.nextVector(previousVector, isNull, batchSize);
-    }
-  }
-
-  public static class AnyIntegerFromAnyIntegerTreeReader extends ConvertTreeReader {
-
-    private AnyIntegerTreeReader anyIntegerAsLongTreeReader;
-
-    private final TypeDescription readerType;
-    private final boolean downCastNeeded;
-
-    AnyIntegerFromAnyIntegerTreeReader(int columnId, TypeDescription fileType, TypeDescription readerType, boolean skipCorrupt) throws IOException {
-      super(columnId);
-      this.readerType = readerType;
-      anyIntegerAsLongTreeReader = new AnyIntegerTreeReader(columnId, fileType, skipCorrupt);
-      setConvertTreeReader(anyIntegerAsLongTreeReader);
-      downCastNeeded = integerDownCastNeeded(fileType, readerType);
-    }
-
-    @Override
-    Object next(Object previous) throws IOException {
-      Writable result = null;
-      if (anyIntegerAsLongTreeReader.read()) {
-        long longValue = anyIntegerAsLongTreeReader.getLong();
-        result = anyIntegerWritable(longValue, previous, readerType);
-      }
-      return result;
-    }
-
-    @Override
-    public void nextVector(ColumnVector previousVector,
-                           boolean[] isNull,
-                           final int batchSize) throws IOException {
-      anyIntegerAsLongTreeReader.nextVector(previousVector, isNull, batchSize);
-      LongColumnVector resultColVector = (LongColumnVector) previousVector;
-      if (downCastNeeded) {
-        long[] resultVector = resultColVector.vector;
-        if (resultColVector.isRepeating) {
-          if (resultColVector.noNulls || !resultColVector.isNull[0]) {
-            resultVector[0] = downCastAnyInteger(resultVector[0], readerType);
-          } else {
-            resultColVector.noNulls = false;
-            resultColVector.isNull[0] = true;
-          }
-        } else if (resultColVector.noNulls){
-          for (int i = 0; i < batchSize; i++) {
-            resultVector[i] = downCastAnyInteger(resultVector[i], readerType);
-          }
-        } else {
-          for (int i = 0; i < batchSize; i++) {
-            if (!resultColVector.isNull[i]) {
-              resultVector[i] = downCastAnyInteger(resultVector[i], readerType);
-            } else {
-              resultColVector.noNulls = false;
-              resultColVector.isNull[i] = true;
-            }
-          }
-        }
-      }
-    }
-  }
-
-  public static class AnyIntegerFromFloatTreeReader extends ConvertTreeReader {
-
-    private FloatTreeReader floatTreeReader;
-
-    private final TypeDescription readerType;
-    private FloatWritable floatResult;
-    private DoubleColumnVector doubleColVector;
-    private LongColumnVector longColVector;
-
-    AnyIntegerFromFloatTreeReader(int columnId, TypeDescription readerType)
-        throws IOException {
-      super(columnId);
-      this.readerType = readerType;
-      floatTreeReader = new FloatTreeReader(columnId);
-      setConvertTreeReader(floatTreeReader);
-      floatResult = new FloatWritable();
-    }
-
-    @Override
-    Object next(Object previous) throws IOException {
-
-      FloatWritable readfloatResult =
-          (FloatWritable) floatTreeReader.next(floatResult);
-
-      Writable result = null;
-      if (readfloatResult != null) {
-        long longValue = (long) readfloatResult.get();
-        result = anyIntegerWritable(longValue, previous, readerType);
-      }
-      return result;
-    }
-
-    @Override
-    public void setConvertVectorElement(int elementNum) throws IOException {
-      float floatValue = (float) doubleColVector.vector[elementNum];
-      longColVector.vector[elementNum] =
-          downCastAnyInteger(
-              (long) floatValue, readerType);
-    }
-
-    @Override
-    public void nextVector(ColumnVector previousVector,
-                           boolean[] isNull,
-                           final int batchSize) throws IOException {
-      if (doubleColVector == null) {
-        // Allocate column vector for file; cast column vector for reader.
-        doubleColVector = new DoubleColumnVector();
-        longColVector = (LongColumnVector) previousVector;
-      }
-      // Read present/isNull stream
-      floatTreeReader.nextVector(doubleColVector, isNull, batchSize);
-
-      convertVector(doubleColVector, longColVector, batchSize);
-    }
-  }
-
-  public static class AnyIntegerFromDoubleTreeReader extends ConvertTreeReader {
-
-    private DoubleTreeReader doubleTreeReader;
-
-    private final TypeDescription readerType;
-    private DoubleWritable doubleResult;
-    private DoubleColumnVector doubleColVector;
-    private LongColumnVector longColVector;
-
-    AnyIntegerFromDoubleTreeReader(int columnId, TypeDescription readerType)
-        throws IOException {
-      super(columnId);
-      this.readerType = readerType;
-      doubleTreeReader = new DoubleTreeReader(columnId);
-      setConvertTreeReader(doubleTreeReader);
-      doubleResult = new DoubleWritable();
-    }
-
-    @Override
-    Object next(Object previous) throws IOException {
-
-      DoubleWritable readDoubleResult =
-          (DoubleWritable) doubleTreeReader.next(doubleResult);
-
-      Writable result = null;
-      if (readDoubleResult != null) {
-        long longValue = (long) readDoubleResult.get();
-        result = anyIntegerWritable(longValue, previous, readerType);
-      }
-      return result;
-    }
-
-    @Override
-    public void setConvertVectorElement(int elementNum) throws IOException {
-      longColVector.vector[elementNum] =
-          downCastAnyInteger(
-              (long) doubleColVector.vector[elementNum], readerType);
-    }
-
-    @Override
-    public void nextVector(ColumnVector previousVector,
-                           boolean[] isNull,
-                           final int batchSize) throws IOException {
-      if (doubleColVector == null) {
-        // Allocate column vector for file; cast column vector for reader.
-        doubleColVector = new DoubleColumnVector();
-        longColVector = (LongColumnVector) previousVector;
-      }
-      // Read present/isNull stream
-      doubleTreeReader.nextVector(doubleColVector, isNull, batchSize);
-
-      convertVector(doubleColVector, longColVector, batchSize);
-    }
-  }
-
-  public static class AnyIntegerFromDecimalTreeReader extends ConvertTreeReader {
-
-    private DecimalTreeReader decimalTreeReader;
-
-    private final int precision;
-    private final int scale;
-    private final TypeDescription readerType;
-    private HiveDecimalWritable hiveDecimalResult;
-    private DecimalColumnVector decimalColVector;
-    private LongColumnVector longColVector;
-
-    AnyIntegerFromDecimalTreeReader(int columnId, TypeDescription fileType,
-        TypeDescription readerType) throws IOException {
-      super(columnId);
-      this.precision = fileType.getPrecision();
-      this.scale = fileType.getScale();
-      this.readerType = readerType;
-      decimalTreeReader = new DecimalTreeReader(columnId, precision, scale);
-      setConvertTreeReader(decimalTreeReader);
-      hiveDecimalResult = new HiveDecimalWritable();
-    }
-
-    @Override
-    Object next(Object previous) throws IOException {
-
-      HiveDecimalWritable readHiveDecimalResult =
-          (HiveDecimalWritable) decimalTreeReader.next(hiveDecimalResult);
-
-      Writable result = null;
-      if (readHiveDecimalResult != null) {
-        long longValue = readHiveDecimalResult.getHiveDecimal().longValue();
-        result = anyIntegerWritable(longValue, previous, readerType);
-      }
-      return result;
-    }
-
-    @Override
-    public void setConvertVectorElement(int elementNum) throws IOException {
-      longColVector.vector[elementNum] =
-          downCastAnyInteger(
-              decimalColVector.vector[elementNum].getHiveDecimal().longValue(),
-              readerType);
-    }
-
-    @Override
-    public void nextVector(ColumnVector previousVector,
-                           boolean[] isNull,
-                           final int batchSize) throws IOException {
-      if (decimalColVector == null) {
-        // Allocate column vector for file; cast column vector for reader.
-        decimalColVector = new DecimalColumnVector(precision, scale);
-        longColVector = (LongColumnVector) previousVector;
-      }
-      // Read present/isNull stream
-      decimalTreeReader.nextVector(decimalColVector, isNull, batchSize);
-
-      convertVector(decimalColVector, longColVector, batchSize);
-    }
-  }
-
-  public static class AnyIntegerFromStringGroupTreeReader extends ConvertTreeReader {
-
-    private TreeReader stringGroupTreeReader;
-
-    private final TypeDescription fileType;
-    private final TypeDescription readerType;
-    private Writable writable;
-    private BytesColumnVector bytesColVector;
-    private LongColumnVector longColVector;
-
-    AnyIntegerFromStringGroupTreeReader(int columnId, TypeDescription fileType,
-        TypeDescription readerType) throws IOException {
-      super(columnId);
-      this.fileType = fileType;
-      this.readerType = readerType;
-      stringGroupTreeReader = getStringGroupTreeReader(columnId, fileType);
-      setConvertTreeReader(stringGroupTreeReader);
-      writable = getStringGroupWritable(fileType);
-    }
-
-    @Override
-    Object next(Object previous) throws IOException {
-
-      String stringValue = stringFromStringGroupTreeReader(
-          stringGroupTreeReader, writable, fileType);
-
-      Writable result = null;
-      if (stringValue != null) {
-        long longValue = parseLongFromString(stringValue);
-        if (!getIsParseError()) {
-          result = anyIntegerWritable(longValue, previous, readerType);
-        }
-      }
-      return result;
-    }
-
-    @Override
-    public void setConvertVectorElement(int elementNum) throws IOException {
-      String string = stringFromBytesColumnVectorEntry(bytesColVector, elementNum);
-      long longValue = parseLongFromString(string);
-      if (!getIsParseError()) {
-        longColVector.vector[elementNum] =
-            downCastAnyInteger(longValue, readerType);
-      } else {
-        longColVector.noNulls = false;
-        longColVector.isNull[elementNum] = true;
-      }
-    }
-
-    @Override
-    public void nextVector(ColumnVector previousVector,
-                           boolean[] isNull,
-                           final int batchSize) throws IOException {
-      if (bytesColVector == null) {
-        // Allocate column vector for file; cast column vector for reader.
-        bytesColVector = new BytesColumnVector();
-        longColVector = (LongColumnVector) previousVector;
-      }
-      // Read present/isNull stream
-      stringGroupTreeReader.nextVector(bytesColVector, isNull, batchSize);
-
-      convertVector(bytesColVector, longColVector, batchSize);
-    }
-  }
-
-  public static class AnyIntegerFromTimestampTreeReader extends ConvertTreeReader {
-
-    private TimestampTreeReader timestampTreeReader;
-
-    private final TypeDescription readerType;
-    private TimestampWritable timestampResult;
-    private TimestampColumnVector timestampColVector;
-    private LongColumnVector longColVector;
-
-    AnyIntegerFromTimestampTreeReader(int columnId, TypeDescription readerType,
-        boolean skipCorrupt) throws IOException {
-      super(columnId);
-      this.readerType = readerType;
-      timestampTreeReader = new TimestampTreeReader(columnId, skipCorrupt);
-      setConvertTreeReader(timestampTreeReader);
-      timestampResult = new TimestampWritable();
-    }
-
-    @Override
-    Object next(Object previous) throws IOException {
-
-      TimestampWritable readHiveTimestampResult =
-          (TimestampWritable) timestampTreeReader.next(timestampResult);
-
-      Writable result = null;
-      if (readHiveTimestampResult != null) {
-        // Use TimestampWritable's getSeconds.
-        long longValue = readHiveTimestampResult.getSeconds();
-        result = anyIntegerWritable(longValue, previous, readerType);
-      }
-      return result;
-    }
-
-    @Override
-    public void setConvertVectorElement(int elementNum) throws IOException {
-      timestampResult.set(timestampColVector.asScratchTimestamp(elementNum));
-      // Use TimestampWritable's getSeconds.
-      long longValue = timestampResult.getSeconds();
-      longColVector.vector[elementNum] =
-          downCastAnyInteger(longValue, readerType);
-    }
-
-    @Override
-    public void nextVector(ColumnVector previousVector,
-                           boolean[] isNull,
-                           final int batchSize) throws IOException {
-      if (timestampColVector == null) {
-        // Allocate column vector for file; cast column vector for reader.
-        timestampColVector = new TimestampColumnVector();
-        longColVector = (LongColumnVector) previousVector;
-      }
-      // Read present/isNull stream
-      timestampTreeReader.nextVector(timestampColVector, isNull, batchSize);
-
-      convertVector(timestampColVector, longColVector, batchSize);
-    }
-  }
-
-  public static class FloatFromAnyIntegerTreeReader extends ConvertTreeReader {
-
-    private AnyIntegerTreeReader anyIntegerAsLongTreeReader;
-
-    private LongColumnVector longColVector;
-    private DoubleColumnVector doubleColVector;
-
-    FloatFromAnyIntegerTreeReader(int columnId, TypeDescription fileType,
-        boolean skipCorrupt) throws IOException {
-      super(columnId);
-      anyIntegerAsLongTreeReader =
-          new AnyIntegerTreeReader(columnId, fileType, skipCorrupt);
-      setConvertTreeReader(anyIntegerAsLongTreeReader);
-    }
-
-    @Override
-    Object next(Object previous) throws IOException {
-      FloatWritable result = null;
-      if (anyIntegerAsLongTreeReader.read()) {
-        long longValue = anyIntegerAsLongTreeReader.getLong();
-        float floatValue = (float) longValue;
-        if (!Float.isNaN(floatValue)){
-          if (previous == null) {
-            result = new FloatWritable();
-          } else {
-            result = (FloatWritable) previous;
-          }
-          result.set(floatValue);
-        }
-      }
-      return result;
-    }
-
-    @Override
-    public void setConvertVectorElement(int elementNum) throws IOException {
-      float floatValue = (float) longColVector.vector[elementNum];
-      if (!Float.isNaN(floatValue)) {
-        doubleColVector.vector[elementNum] = floatValue;
-      } else {
-        doubleColVector.vector[elementNum] = Double.NaN;
-        doubleColVector.noNulls = false;
-        doubleColVector.isNull[elementNum] = true;
-      }
-    }
-
-    @Override
-    public void nextVector(ColumnVector previousVector,
-                           boolean[] isNull,
-                           final int batchSize) throws IOException {
-      if (longColVector == null) {
-        // Allocate column vector for file; cast column vector for reader.
-        longColVector = new LongColumnVector();
-        doubleColVector = (DoubleColumnVector) previousVector;
-      }
-      // Read present/isNull stream
-      anyIntegerAsLongTreeReader.nextVector(longColVector, isNull, batchSize);
-
-      convertVector(longColVector, doubleColVector, batchSize);
-    }
-  }
-
-  public static class FloatFromDoubleTreeReader extends ConvertTreeReader {
-
-    private DoubleTreeReader doubleTreeReader;
-
-    private DoubleWritable doubleResult;
-
-    FloatFromDoubleTreeReader(int columnId) throws IOException {
-      super(columnId);
-      doubleTreeReader = new DoubleTreeReader(columnId);
-      setConvertTreeReader(doubleTreeReader);
-      doubleResult = new DoubleWritable();
-    }
-
-    @Override
-    Object next(Object previous) throws IOException {
-
-      DoubleWritable readDoubleResult =
-          (DoubleWritable) doubleTreeReader.next(doubleResult);
-
-      FloatWritable result = null;
-      if (readDoubleResult != null) {
-        if (previous == null) {
-          result = new FloatWritable();
-        } else {
-          result = (FloatWritable) previous;
-        }
-        result.set((float) readDoubleResult.get());
-      }
-      return result;
-    }
-
-    @Override
-    public void nextVector(ColumnVector previousVector,
-                           boolean[] isNull,
-                           final int batchSize) throws IOException {
-      doubleTreeReader.nextVector(previousVector, isNull, batchSize);
-
-      DoubleColumnVector resultColVector = (DoubleColumnVector) previousVector;
-      double[] resultVector = resultColVector.vector;
-      if (resultColVector.isRepeating) {
-        if (resultColVector.noNulls || !resultColVector.isNull[0]) {
-          resultVector[0] = (float) resultVector[0];
-        } else {
-          resultColVector.noNulls = false;
-          resultColVector.isNull[0] = true;
-        }
-      } else if (resultColVector.noNulls){
-        for (int i = 0; i < batchSize; i++) {
-          resultVector[i] = (float) resultVector[i];
-        }
-      } else {
-        for (int i = 0; i < batchSize; i++) {
-          if (!resultColVector.isNull[i]) {
-            resultVector[i] = (float) resultVector[i];
-          } else {
-            resultColVector.noNulls = false;
-            resultColVector.isNull[i] = true;
-          }
-        }
-      }
-    }
-  }
-
-  public static class FloatFromDecimalTreeReader extends ConvertTreeReader {
-
-    private DecimalTreeReader decimalTreeReader;
-
-    private final int precision;
-    private final int scale;
-    private final TypeDescription readerType;
-    private HiveDecimalWritable hiveDecimalResult;
-    private DecimalColumnVector decimalColVector;
-    private DoubleColumnVector doubleColVector;
-
-    FloatFromDecimalTreeReader(int columnId, TypeDescription fileType,
-        TypeDescription readerType) throws IOException {
-      super(columnId);
-      this.precision = fileType.getPrecision();
-      this.scale = fileType.getScale();
-      this.readerType = readerType;
-      decimalTreeReader = new DecimalTreeReader(columnId, precision, scale);
-      setConvertTreeReader(decimalTreeReader);
-      hiveDecimalResult = new HiveDecimalWritable();
-    }
-
-    @Override
-    Object next(Object previous) throws IOException {
-
-      HiveDecimalWritable readHiveDecimalResult =
-          (HiveDecimalWritable) decimalTreeReader.next(hiveDecimalResult);
-
-      FloatWritable result = null;
-      if (readHiveDecimalResult != null) {
-        double doubleValue = readHiveDecimalResult.getHiveDecimal().doubleValue();
-        if (previous == null) {
-          result = new FloatWritable();
-        } else {
-          result = (FloatWritable) previous;
-        }
-        result.set((float) doubleValue);
-      }
-      return result;
-    }
-
-    @Override
-    public void setConvertVectorElement(int elementNum) throws IOException {
-      doubleColVector.vector[elementNum] =
-          (float) decimalColVector.vector[elementNum].getHiveDecimal().doubleValue();
-    }
-
-    @Override
-    public void nextVector(ColumnVector previousVector,
-                           boolean[] isNull,
-                           final int batchSize) throws IOException {
-      if (decimalColVector == null) {
-        // Allocate column vector for file; cast column vector for reader.
-        decimalColVector = new DecimalColumnVector(precision, scale);
-        doubleColVector = (DoubleColumnVector) previousVector;
-      }
-      // Read present/isNull stream
-      decimalTreeReader.nextVector(decimalColVector, isNull, batchSize);
-
-      convertVector(decimalColVector, doubleColVector, batchSize);
-    }
-  }
-
-  public static class FloatFromStringGroupTreeReader extends ConvertTreeReader {
-
-    private TreeReader stringGroupTreeReader;
-
-    private final TypeDescription fileType;
-    private Writable writable;
-    private BytesColumnVector bytesColVector;
-    private DoubleColumnVector doubleColVector;
-
-    FloatFromStringGroupTreeReader(int columnId, TypeDescription fileType)
-        throws IOException {
-      super(columnId);
-      this.fileType = fileType;
-      stringGroupTreeReader = getStringGroupTreeReader(columnId, fileType);
-      setConvertTreeReader(stringGroupTreeReader);
-      writable = getStringGroupWritable(fileType);
-    }
-
-    @Override
-    Object next(Object previous) throws IOException {
-
-      String stringValue = stringFromStringGroupTreeReader(
-          stringGroupTreeReader, writable, fileType);
-
-      FloatWritable result = null;
-      if (stringValue != null) {
-        float floatValue = parseFloatFromString(stringValue);
-        if (!getIsParseError()) {
-          if (previous == null) {
-            result = new FloatWritable();
-          } else {
-            result = (FloatWritable) previous;
-          }
-          result.set(floatValue);
-        }
-      }
-      return result;
-    }
-
-    @Override
-    public void setConvertVectorElement(int elementNum) throws IOException {
-      String string = stringFromBytesColumnVectorEntry(bytesColVector, elementNum);
-      float floatValue = parseFloatFromString(string);
-      if (!getIsParseError()) {
-        doubleColVector.vector[elementNum] = floatValue;
-      } else {
-        doubleColVector.vector[elementNum] = Double.NaN;
-        doubleColVector.noNulls = false;
-        doubleColVector.isNull[elementNum] = true;
-      }
-    }
-
-    @Override
-    public void nextVector(ColumnVector previousVector,
-                           boolean[] isNull,
-                           final int batchSize) throws IOException {
-      if (bytesColVector == null) {
-        // Allocate column vector for file; cast column vector for reader.
-        bytesColVector = new BytesColumnVector();
-        doubleColVector = (DoubleColumnVector) previousVector;
-      }
-      // Read present/isNull stream
-      stringGroupTreeReader.nextVector(bytesColVector, isNull, batchSize);
-
-      convertVector(bytesColVector, doubleColVector, batchSize);
-    }
-  }
-
-  public static class FloatFromTimestampTreeReader extends ConvertTreeReader {
-
-    private TimestampTreeReader timestampTreeReader;
-
-    private final TypeDescription readerType;
-    private TimestampWritable timestampResult;
-    private TimestampColumnVector timestampColVector;
-    private DoubleColumnVector doubleColVector;
-
-    FloatFromTimestampTreeReader(int columnId, TypeDescription readerType,
-        boolean skipCorrupt) throws IOException {
-      super(columnId);
-      this.readerType = readerType;
-      timestampTreeReader = new TimestampTreeReader(columnId, skipCorrupt);
-      setConvertTreeReader(timestampTreeReader);
-      timestampResult = new TimestampWritable();
-    }
-
-    @Override
-    Object next(Object previous) throws IOException {
-
-      TimestampWritable readTimestampResult =
-          (TimestampWritable) timestampTreeReader.next(timestampResult);
-
-      FloatWritable result = null;
-      if (readTimestampResult != null) {
-        double doubleValue = readTimestampResult.getDouble();
-        if (previous == null) {
-          result = new FloatWritable();
-        } else {
-          result = (FloatWritable) previous;
-        }
-        result.set((float) doubleValue);
-      }
-      return result;
-    }
-
-    @Override
-    public void setConvertVectorElement(int elementNum) throws IOException {
-      timestampResult.set(timestampColVector.asScratchTimestamp(elementNum));
-      doubleColVector.vector[elementNum] = (float) timestampResult.getDouble();
-    }
-
-    @Override
-    public void nextVector(ColumnVector previousVector,
-                           boolean[] isNull,
-                           final int batchSize) throws IOException {
-      if (timestampColVector == null) {
-        // Allocate column vector for file; cast column vector for reader.
-        timestampColVector = new TimestampColumnVector();
-        doubleColVector = (DoubleColumnVector) previousVector;
-      }
-      // Read present/isNull stream
-      timestampTreeReader.nextVector(timestampColVector, isNull, batchSize);
-
-      convertVector(timestampColVector, doubleColVector, batchSize);
-    }
-  }
-
-  public static class DoubleFromAnyIntegerTreeReader extends ConvertTreeReader {
-
-    private AnyIntegerTreeReader anyIntegerAsLongTreeReader;
-
-    private LongColumnVector longColVector;
-    private DoubleColumnVector doubleColVector;
-
-    DoubleFromAnyIntegerTreeReader(int columnId, TypeDescription fileType,
-        boolean skipCorrupt) throws IOException {
-      super(columnId);
-      anyIntegerAsLongTreeReader =
-          new AnyIntegerTreeReader(columnId, fileType, skipCorrupt);
-      setConvertTreeReader(anyIntegerAsLongTreeReader);
-    }
-
-    @Override
-    Object next(Object previous) throws IOException {
-      DoubleWritable result = null;
-      if (anyIntegerAsLongTreeReader.read()) {
-        long longValue = anyIntegerAsLongTreeReader.getLong();
-        double doubleValue = (double) longValue;
-        if (!Double.isNaN(doubleValue)) {
-          if (previous == null) {
-            result = new DoubleWritable();
-          } else {
-            result = (DoubleWritable) previous;
-          }
-          result.set(doubleValue);
-        }
-      }
-      return result;
-    }
-
-    @Override
-    public void setConvertVectorElement(int elementNum) {
-
-      double doubleValue = (double) longColVector.vector[elementNum];
-      if (!Double.isNaN(doubleValue)) {
-        doubleColVector.vector[elementNum] = doubleValue;
-      } else {
-        doubleColVector.vector[elementNum] = Double.NaN;
-        doubleColVector.noNulls = false;
-        doubleColVector.isNull[elementNum] = true;
-      }
-    }
-
-    @Override
-    public void nextVector(ColumnVector previousVector,
-                           boolean[] isNull,
-                           final int batchSize) throws IOException {
-      if (longColVector == null) {
-        // Allocate column vector for file; cast column vector for reader.
-        longColVector = new LongColumnVector();
-        doubleColVector = (DoubleColumnVector) previousVector;
-      }
-      // Read present/isNull stream
-      anyIntegerAsLongTreeReader.nextVector(longColVector, isNull, batchSize);
-
-      convertVector(longColVector, doubleColVector, batchSize);
-    }
-  }
-
-  public static class DoubleFromFloatTreeReader extends ConvertTreeReader {
-
-    private FloatTreeReader floatTreeReader;
-
-    private FloatWritable floatResult;
-
-    DoubleFromFloatTreeReader(int columnId) throws IOException {
-      super(columnId);
-      floatTreeReader = new FloatTreeReader(columnId);
-      setConvertTreeReader(floatTreeReader);
-      floatResult = new FloatWritable();
-    }
-
-    @Override
-    Object next(Object previous) throws IOException {
-
-      FloatWritable readFloatResult =
-          (FloatWritable) floatTreeReader.next(floatResult);
-
-      DoubleWritable result = null;
-      if (readFloatResult != null) {
-        if (previous == null) {
-          result = new DoubleWritable();
-        } else {
-          result = (DoubleWritable) previous;
-        }
-        result.set(readFloatResult.get());
-      }
-      return result;
-    }
-
-    @Override
-    public void nextVector(ColumnVector previousVector,
-                           boolean[] isNull,
-                           final int batchSize) throws IOException {
-      // The DoubleColumnVector produced by FloatTreeReader is what we want.
-      floatTreeReader.nextVector(previousVector, isNull, batchSize);
-    }
-  }
-
-  public static class DoubleFromDecimalTreeReader extends ConvertTreeReader {
-
-    private DecimalTreeReader decimalTreeReader;
-
-    private final int precision;
-    private final int scale;
-    private final TypeDescription readerType;
-    private HiveDecimalWritable hiveDecimalResult;
-    private DecimalColumnVector decimalColVector;
-    private DoubleColumnVector doubleColVector;
-
-    DoubleFromDecimalTreeReader(int columnId, TypeDescription fileType,
-        TypeDescription readerType) throws IOException {
-      super(columnId);
-      this.precision = fileType.getPrecision();
-      this.scale = fileType.getScale();
-      this.readerType = readerType;
-      decimalTreeReader = new DecimalTreeReader(columnId, precision, scale);
-      setConvertTreeReader(decimalTreeReader);
-      hiveDecimalResult = new HiveDecimalWritable();
-    }
-
-    @Override
-    Object next(Object previous) throws IOException {
-
-      HiveDecimalWritable readHiveDecimalResult =
-          (HiveDecimalWritable) decimalTreeReader.next(hiveDecimalResult);
-
-      DoubleWritable result = null;
-      if (readHiveDecimalResult != null) {
-        double doubleValue = readHiveDecimalResult.getHiveDecimal().doubleValue();
-        if (previous == null) {
-          result = new DoubleWritable();
-        } else {
-          result = (DoubleWritable) previous;
-        }
-        result.set(doubleValue);
-      }
-      return result;
-    }
-
-    @Override
-    public void setConvertVectorElement(int elementNum) throws IOException {
-      doubleColVector.vector[elementNum] =
-          decimalColVector.vector[elementNum].getHiveDecimal().doubleValue();
-    }
-
-    @Override
-    public void nextVector(ColumnVector previousVector,
-                           boolean[] isNull,
-                           final int batchSize) throws IOException {
-      if (decimalColVector == null) {
-        // Allocate column vector for file; cast column vector for reader.
-        decimalColVector = new DecimalColumnVector(precision, scale);
-        doubleColVector = (DoubleColumnVector) previousVector;
-      }
-      // Read present/isNull stream
-      decimalTreeReader.nextVector(decimalColVector, isNull, batchSize);
-
-      convertVector(decimalColVector, doubleColVector, batchSize);
-    }
-  }
-
-  public static class DoubleFromStringGroupTreeReader extends ConvertTreeReader {
-
-    private TreeReader stringGroupTreeReader;
-
-    private final TypeDescription fileType;
-    private Writable writable;
-    private BytesColumnVector bytesColVector;
-    private DoubleColumnVector doubleColVector;
-
-    DoubleFromStringGroupTreeReader(int columnId, TypeDescription fileType)
-        throws IOException {
-      super(columnId);
-      this.fileType = fileType;
-      stringGroupTreeReader = getStringGroupTreeReader(columnId, fileType);
-      setConvertTreeReader(stringGroupTreeReader);
-      writable = getStringGroupWritable(fileType);
-    }
-
-    @Override
-    Object next(Object previous) throws IOException {
-
-      String stringValue = stringFromStringGroupTreeReader(
-          stringGroupTreeReader, writable, fileType);
-
-      DoubleWritable result = null;
-      if (stringValue != null) {
-        double doubleValue = parseDoubleFromString(stringValue);
-        if (!getIsParseError()) {
-          if (previous == null) {
-            result = new DoubleWritable();
-          } else {
-            result = (DoubleWritable) previous;
-          }
-          result.set(doubleValue);
-        }
-      }
-      return result;
-    }
-
-    @Override
-    public void setConvertVectorElement(int elementNum) throws IOException {
-      String string = stringFromBytesColumnVectorEntry(bytesColVector, elementNum);
-      double doubleValue = parseDoubleFromString(string);
-      if (!getIsParseError()) {
-        doubleColVector.vector[elementNum] = doubleValue;
-      } else {
-        doubleColVector.noNulls = false;
-        doubleColVector.isNull[elementNum] = true;
-      }
-    }
-
-    @Override
-    public void nextVector(ColumnVector previousVector,
-                           boolean[] isNull,
-                           final int batchSize) throws IOException {
-      if (bytesColVector == null) {
-        // Allocate column vector for file; cast column vector for reader.
-        bytesColVector = new BytesColumnVector();
-        doubleColVector = (DoubleColumnVector) previousVector;
-      }
-      // Read present/isNull stream
-      stringGroupTreeReader.nextVector(bytesColVector, isNull, batchSize);
-
-      convertVector(bytesColVector, doubleColVector, batchSize);
-    }
-  }
-
-  public static class DoubleFromTimestampTreeReader extends ConvertTreeReader {
-
-    private TimestampTreeReader timestampTreeReader;
-
-    private final TypeDescription readerType;
-    private TimestampWritable timestampResult;
-    private TimestampColumnVector timestampColVector;
-    private DoubleColumnVector doubleColVector;
-
-    DoubleFromTimestampTreeReader(int columnId, TypeDescription readerType,
-        boolean skipCorrupt) throws IOException {
-      super(columnId);
-      this.readerType = readerType;
-      timestampTreeReader = new TimestampTreeReader(columnId, skipCorrupt);
-      setConvertTreeReader(timestampTreeReader);
-      timestampResult = new TimestampWritable();
-    }
-
-    @Override
-    Object next(Object previous) throws IOException {
-
-      TimestampWritable readTimestampResult =
-          (TimestampWritable) timestampTreeReader.next(timestampResult);
-
-      DoubleWritable result = null;
-      if (readTimestampResult != null) {
-        double doubleValue = readTimestampResult.getDouble();
-        if (previous == null) {
-          result = new DoubleWritable();
-        } else {
-          result = (DoubleWritable) previous;
-        }
-        result.set(doubleValue);
-      }
-      return result;
-    }
-
-    @Override
-    public void setConvertVectorElement(int elementNum) throws IOException {
-      timestampResult.set(timestampColVector.asScratchTimestamp(elementNum));
-      doubleColVector.vector[elementNum] = timestampResult.getDouble();
-    }
-
-    @Override
-    public void nextVector(ColumnVector previousVector,
-                           boolean[] isNull,
-                           final int batchSize) throws IOException {
-      if (timestampColVector == null) {
-        // Allocate column vector for file; cast column vector for reader.
-        timestampColVector = new TimestampColumnVector();
-        doubleColVector = (DoubleColumnVector) previousVector;
-      }
-      // Read present/isNull stream
-      timestampTreeReader.nextVector(timestampColVector, isNull, batchSize);
-
-      convertVector(timestampColVector, doubleColVector, batchSize);
-    }
-  }
-
-  public static class DecimalFromAnyIntegerTreeReader extends ConvertTreeReader {
-
-    private AnyIntegerTreeReader anyIntegerAsLongTreeReader;
-
-    private int precision;
-    private int scale;
-    private LongColumnVector longColVector;
-    private DecimalColumnVector decimalColVector;
-
-    DecimalFromAnyIntegerTreeReader(int columnId, TypeDescription fileType,
-        TypeDescription readerType, boolean skipCorrupt) throws IOException {
-      super(columnId);
-      this.precision = readerType.getPrecision();
-      this.scale = readerType.getScale();
-      anyIntegerAsLongTreeReader =
-          new AnyIntegerTreeReader(columnId, fileType, skipCorrupt);
-      setConvertTreeReader(anyIntegerAsLongTreeReader);
-    }
-
-    @Override
-    Object next(Object previous) throws IOException {
-      HiveDecimalWritable result = null;
-      if (anyIntegerAsLongTreeReader.read()) {
-        long longValue = anyIntegerAsLongTreeReader.getLong();
-        result = new HiveDecimalWritable(longValue);
-      }
-      return result;
-    }
-
-    @Override
-    public void setConvertVectorElement(int elementNum) {
-      long longValue = longColVector.vector[elementNum];
-      HiveDecimalWritable hiveDecimalWritable =
-          new HiveDecimalWritable(longValue);
-      decimalColVector.set(elementNum, hiveDecimalWritable);
-    }
-
-    @Override
-    public void nextVector(ColumnVector previousVector,
-        boolean[] isNull,
-        final int batchSize) throws IOException {
-      if (longColVector == null) {
-        // Allocate column vector for file; cast column vector for reader.
-        longColVector = new LongColumnVector();
-        decimalColVector = (DecimalColumnVector) previousVector;
-      }
-      // Read present/isNull stream
-      anyIntegerAsLongTreeReader.nextVector(longColVector, isNull, batchSize);
-
-      convertVector(longColVector, decimalColVector, batchSize);
-    }
-  }
-
-  public static class DecimalFromFloatTreeReader extends ConvertTreeReader {
-
-    private FloatTreeReader floatTreeReader;
-
-    private int precision;
-    private int scale;
-    private FloatWritable floatResult;
-    private DoubleColumnVector doubleColVector;
-    private DecimalColumnVector decimalColVector;
-
-    DecimalFromFloatTreeReader(int columnId, TypeDescription readerType)
-        throws IOException {
-      super(columnId);
-      this.precision = readerType.getPrecision();
-      this.scale = readerType.getScale();
-      floatTreeReader = new FloatTreeReader(columnId);
-      setConvertTreeReader(floatTreeReader);
-      floatResult = new FloatWritable();
-    }
-
-    @Override
-    Object next(Object previous) throws IOException {
-
-      FloatWritable readFloatResult =
-          (FloatWritable) floatTreeReader.next(floatResult);
-
-      HiveDecimalWritable result = null;
-      if (readFloatResult != null) {
-        HiveDecimal value =
-            HiveDecimal.create(Float.toString(readFloatResult.get()));
-        if (value != null) {
-          if (previous == null) {
-            result = new HiveDecimalWritable();
-          } else {
-            result = (HiveDecimalWritable) previous;
-          }
-          result.set(value);
-        }
-      }
-      return result;
-    }
-
-    @Override
-    public void setConvertVectorElement(int elementNum) throws IOException {
-      float floatValue = (float) doubleColVector.vector[elementNum];
-      if (!Float.isNaN(floatValue)) {
-        HiveDecimal value =
-            HiveDecimal.create(Float.toString(floatValue));
-        if (value != null) {
-          decimalColVector.set(elementNum, value);
-        } else {
-          decimalColVector.noNulls = false;
-          decimalColVector.isNull[elementNum] = true;
-        }
-      } else {
-        decimalColVector.noNulls = false;
-        decimalColVector.isNull[elementNum] = true;
-      }
-    }
-
-    @Override
-    public void nextVector(ColumnVector previousVector,
-                           boolean[] isNull,
-                           final int batchSize) throws IOException {
-      if (doubleColVector == null) {
-        // Allocate column vector for file; cast column vector for reader.
-        doubleColVector = new DoubleColumnVector();
-        decimalColVector = (DecimalColumnVector) previousVector;
-      }
-      // Read present/isNull stream
-      floatTreeReader.nextVector(doubleColVector, isNull, batchSize);
-
-      convertVector(doubleColVector, decimalColVector, batchSize);
-    }
-  }
-
-  public static class DecimalFromDoubleTreeReader extends ConvertTreeReader {
-
-    private DoubleTreeReader doubleTreeReader;
-
-    private int precision;
-    private int scale;
-    private DoubleWritable doubleResult;
-    private DoubleColumnVector doubleColVector;
-    private DecimalColumnVector decimalColVector;
-
-    DecimalFromDoubleTreeReader(int columnId, TypeDescription readerType)
-        throws IOException {
-      super(columnId);
-      this.precision = readerType.getPrecision();
-      this.scale = readerType.getScale();
-      doubleTreeReader = new DoubleTreeReader(columnId);
-      setConvertTreeReader(doubleTreeReader);
-      doubleResult = new DoubleWritable();
-    }
-
-    @Override
-    Object next(Object previous) throws IOException {
-
-      DoubleWritable readDoubleResult =
-          (DoubleWritable) doubleTreeReader.next(doubleResult);
-
-      HiveDecimalWritable result = null;
-      if (readDoubleResult != null) {
-        HiveDecimal value =
-            HiveDecimal.create(Double.toString(readDoubleResult.get()));
-        if (value != null) {
-          if (previous == null) {
-            result = new HiveDecimalWritable();
-          } else {
-            result = (HiveDecimalWritable) previous;
-          }
-          result.set(value);
-        }
-      }
-      return result;
-    }
-
-    @Override
-    public void setConvertVectorElement(int elementNum) throws IOException {
-      HiveDecimal value =
-          HiveDecimal.create(Double.toString(doubleColVector.vector[elementNum]));
-      if (value != null) {
-        decimalColVector.set(elementNum, value);
-      } else {
-        decimalColVector.noNulls = false;
-        decimalColVector.isNull[elementNum] = true;
-      }
-    }
-
-    @Override
-    public void nextVector(ColumnVector previousVector,
-                           boolean[] isNull,
-                           final int batchSize) throws IOException {
-      if (doubleColVector == null) {
-        // Allocate column vector for file; cast column vector for reader.
-        doubleColVector = new DoubleColumnVector();
-        decimalColVector = (DecimalColumnVector) previousVector;
-      }
-      // Read present/isNull stream
-      doubleTreeReader.nextVector(doubleColVector, isNull, batchSize);
-
-      convertVector(doubleColVector, decimalColVector, batchSize);
-    }
-  }
-
-  public static class DecimalFromStringGroupTreeReader extends ConvertTreeReader {
-
-    private TreeReader stringGroupTreeReader;
-
-    private final TypeDescription fileType;
-    private Writable writable;
-    private BytesColumnVector bytesColVector;
-    private int precision;
-    private int scale;
-    private DecimalColumnVector decimalColVector;
-
-    DecimalFromStringGroupTreeReader(int columnId, TypeDescription fileType,
-        TypeDescription readerType) throws IOException {
-      super(columnId);
-      this.fileType = fileType;
-      this.precision = readerType.getPrecision();
-      this.scale = readerType.getScale();
-      stringGroupTreeReader = getStringGroupTreeReader(columnId, fileType);
-      setConvertTreeReader(stringGroupTreeReader);
-      writable = getStringGroupWritable(fileType);
-    }
-
-    @Override
-    Object next(Object previous) throws IOException {
-
-      String stringValue = stringFromStringGroupTreeReader(
-          stringGroupTreeReader, writable, fileType);
-
-      HiveDecimalWritable result = null;
-      if (stringValue != null) {
-        HiveDecimal value = parseDecimalFromString(stringValue);
-        if (value != null) {
-          if (previous == null) {
-            result = new HiveDecimalWritable();
-          } else {
-            result = (HiveDecimalWritable) previous;
-          }
-          result.set(value, precision, scale);
-        }
-      }
-      return result;
-    }
-
-    @Override
-    public void setConvertVectorElement(int elementNum) throws IOException {
-      String string = stringFromBytesColumnVectorEntry(bytesColVector, elementNum);
-      HiveDecimal value = parseDecimalFromString(string);
-      if (value != null) {
-        decimalColVector.set(elementNum, value);
-      } else {
-        decimalColVector.noNulls = false;
-        decimalColVector.isNull[elementNum] = true;
-      }
-    }
-
-    @Override
-    public void nextVector(ColumnVector previousVector,
-                           boolean[] isNull,
-                           final int batchSize) throws IOException {
-      if (bytesColVector == null) {
-        // Allocate column vector for file; cast column vector for reader.
-        bytesColVector = new BytesColumnVector();
-        decimalColVector = (DecimalColumnVector) previousVector;
-      }
-      // Read present/isNull stream
-      stringGroupTreeReader.nextVector(bytesColVector, isNull, batchSize);
-
-      convertVector(bytesColVector, decimalColVector, batchSize);
-    }
-  }
-
-  public static class DecimalFromTimestampTreeReader extends ConvertTreeReader {
-
-    private TimestampTreeReader timestampTreeReader;
-
-    private final TypeDescription readerType;
-    private TimestampWritable timestampResult;
-    private TimestampColumnVector timestampColVector;
-    private int precision;
-    private int scale;
-    private DecimalColumnVector decimalColVector;
-
-    DecimalFromTimestampTreeReader(int columnId, TypeDescription readerType,
-        boolean skipCorrupt) throws IOException {
-      super(columnId);
-      this.readerType = readerType;
-      this.precision = readerType.getPrecision();
-      this.scale = readerType.getScale();
-      timestampTreeReader = new TimestampTreeReader(columnId, skipCorrupt);
-      setConvertTreeReader(timestampTreeReader);
-      timestampResult = new TimestampWritable();
-    }
-
-    @Override
-    Object next(Object previous) throws IOException {
-
-      TimestampWritable readTimestampResult =
-          (TimestampWritable) timestampTreeReader.next(timestampResult);
-
-      HiveDecimalWritable result = null;
-      if (readTimestampResult != null) {
-        double doubleValue = readTimestampResult.getDouble();
-        HiveDecimal value = HiveDecimal.create(Double.toString(doubleValue));
-        if (value != null) {
-          if (previous == null) {
-            result = new HiveDecimalWritable();
-          } else {
-            result = (HiveDecimalWritable) previous;
-          }
-          result.set(value, precision, scale);
-        }
-      }
-      return result;
-    }
-
-    @Override
-    public void setConvertVectorElement(int elementNum) throws IOException {
-      timestampResult.set(timestampColVector.asScratchTimestamp(elementNum));
-      double doubleValue = timestampResult.getDouble();
-      HiveDecimal value = HiveDecimal.create(Double.toString(doubleValue));
-      if (value != null) {
-        decimalColVector.set(elementNum, value);
-      } else {
-        decimalColVector.noNulls = false;
-        decimalColVector.isNull[elementNum] = true;
-      }
-    }
-
-    @Override
-    public void nextVector(ColumnVector previousVector,
-                           boolean[] isNull,
-                           final int batchSize) throws IOException {
-      if (timestampColVector == null) {
-        // Allocate column vector for file; cast column vector for reader.
-        timestampColVector = new TimestampColumnVector();
-        decimalColVector = (DecimalColumnVector) previousVector;
-      }
-      // Read present/isNull stream
-      timestampTreeReader.nextVector(timestampColVector, isNull, batchSize);
-
-      convertVector(timestampColVector, decimalColVector, batchSize);
-    }
-  }
-
-  public static class StringGroupFromAnyIntegerTreeReader extends ConvertTreeReader {
-
-    private AnyIntegerTreeReader anyIntegerAsLongTreeReader;
-
-    private final TypeDescription fileType;
-    private final TypeDescription readerType;
-    private LongColumnVector longColVector;
-    private BytesColumnVector bytesColVector;
-
-    StringGroupFromAnyIntegerTreeReader(int columnId, TypeDescription fileType,
-        TypeDescription readerType, boolean skipCorrupt) throws IOException {
-      super(columnId);
-      this.fileType = fileType;
-      this.readerType = readerType;
-      anyIntegerAsLongTreeReader =
-          new AnyIntegerTreeReader(columnId, fileType, skipCorrupt);
-      setConvertTreeReader(anyIntegerAsLongTreeReader);
-    }
-
-    @Override
-    Object next(Object previous) throws IOException {
-      Writable result = null;
-      if (anyIntegerAsLongTreeReader.read()) {
-        result = getStringGroupResultFromString(
-            previous, readerType, anyIntegerAsLongTreeReader.getString());
-      }
-      return result;
-    }
-
-    @Override
-    public void setConvertVectorElement(int elementNum) {
-      long longValue = longColVector.vector[elementNum];
-      String string = anyIntegerAsLongTreeReader.getString(longValue);
-      byte[] bytes = string.getBytes();
-      assignStringGroupVectorEntry(bytesColVector, elementNum, readerType, bytes);
-    }
-
-    @Override
-    public void nextVector(ColumnVector previousVector,
-                           boolean[] isNull,
-                           final int batchSize) throws IOException {
-      if (longColVector == null) {
-        // Allocate column vector for file; cast column vector for reader.
-        longColVector = new LongColumnVector();
-        bytesColVector = (BytesColumnVector) previousVector;
-      }
-      // Read present/isNull stream
-      anyIntegerAsLongTreeReader.nextVector(longColVector, isNull, batchSize);
-
-      convertVector(longColVector, bytesColVector, batchSize);
-    }
-  }
-
-  public static class StringGroupFromFloatTreeReader extends ConvertTreeReader {
-
-    private FloatTreeReader floatTreeReader;
-
-    private final TypeDescription readerType;
-    private FloatWritable floatResult;
-    private DoubleColumnVector doubleColVector;
-    private BytesColumnVector bytesColVector;
-
-
-    StringGroupFromFloatTreeReader(int columnId, TypeDescription readerType,
-        boolean skipCorrupt) throws IOException {
-      super(columnId);
-      this.readerType = readerType;
-      floatTreeReader = new FloatTreeReader(columnId);
-      setConvertTreeReader(floatTreeReader);
-      floatResult = new FloatWritable();
-    }
-
-    @Override
-    Object next(Object previous) throws IOException {
-
-      FloatWritable readFloatResult =
-          (FloatWritable) floatTreeReader.next(floatResult);
-
-      Writable result = null;
-      if (readFloatResult != null) {
-        float floatValue = readFloatResult.get();
-        if (!Float.isNaN(floatValue)) {
-          result = getStringGroupResultFromString(
-              previous, readerType, String.valueOf(floatValue));
-        }
-      }
-      return result;
-    }
-
-    @Override
-    public void setConvertVectorElement(int elementNum) {
-      float floatValue = (float) doubleColVector.vector[elementNum];
-      if (!Float.isNaN(floatValue)) {
-        String string = String.valueOf(floatValue);
-        byte[] bytes = string.getBytes();
-        assignStringGroupVectorEntry(bytesColVector, elementNum, readerType, bytes);
-      } else {
-        bytesColVector.noNulls = false;
-        bytesColVector.isNull[elementNum] = true;
-      }
-    }
-
-    @Override
-    public void nextVector(ColumnVector previousVector,
-                           boolean[] isNull,
-                           final int batchSize) throws IOException {
-      if (doubleColVector == null) {
-        // Allocate column vector for file; cast column vector for reader.
-        doubleColVector = new DoubleColumnVector();
-        bytesColVector = (BytesColumnVector) previousVector;
-      }
-      // Read present/isNull stream
-      floatTreeReader.nextVector(doubleColVector, isNull, batchSize);
-
-      convertVector(doubleColVector, bytesColVector, batchSize);
-    }
-  }
-
-  public static class StringGroupFromDoubleTreeReader extends ConvertTreeReader {
-
-    private DoubleTreeReader doubleTreeReader;
-
-    private final TypeDescription readerType;
-    private DoubleWritable doubleResult;
-    private DoubleColumnVector doubleColVector;
-    private BytesColumnVector bytesColVector;
-
-    StringGroupFromDoubleTreeReader(int columnId, TypeDescription readerType,
-        boolean skipCorrupt) throws IOException {
-      super(columnId);
-      this.readerType = readerType;
-      doubleTreeReader = new DoubleTreeReader(columnId);
-      setConvertTreeReader(doubleTreeReader);
-      doubleResult = new DoubleWritable();
-    }
-
-    @Override
-    Object next(Object previous) throws IOException {
-
-      DoubleWritable readDoubleResult =
-          (DoubleWritable) doubleTreeReader.next(doubleResult);
-
-      Writable result = null;
-      if (readDoubleResult != null) {
-        double doubleValue = readDoubleResult.get();
-        if (!Double.isNaN(doubleValue)) {
-          result = getStringGroupResultFromString(
-              previous, readerType, String.valueOf(doubleValue));
-        }
-      }
-      return result;
-    }
-
-    @Override
-    public void setConvertVectorElement(int elementNum) {
-      double doubleValue = doubleColVector.vector[elementNum];
-      if (!Double.isNaN(doubleValue)) {
-        String string = String.valueOf(doubleValue);
-        byte[] bytes = string.getBytes();
-        assignStringGroupVectorEntry(bytesColVector, elementNum, readerType, bytes);
-      } else {
-        bytesColVector.noNulls = false;
-        bytesColVector.isNull[elementNum] = true;
-      }
-    }
-
-    @Override
-    public void nextVector(ColumnVector previousVector,
-                           boolean[] isNull,
-                           final int batchSize) throws IOException {
-      if (doubleColVector == null) {
-        // Allocate column vector for file; cast column vector for reader.
-        doubleColVector = new DoubleColumnVector();
-        bytesColVector = (BytesColumnVector) previousVector;
-      }
-      // Read present/isNull stream
-      doubleTreeReader.nextVector(doubleColVector, isNull, batchSize);
-
-      convertVector(doubleColVector, bytesColVector, batchSize);
-    }
-  }
-
-
-
-  public static class StringGroupFromDecimalTreeReader extends ConvertTreeReader {
-
-    private DecimalTreeReader decimalTreeReader;
-
-    private int precision;
-    private int scale;
-    private final TypeDescription readerType;
-    private HiveDecimalWritable hiveDecimalResult;
-    private DecimalColumnVector decimalColVector;
-    private BytesColumnVector bytesColVector;
-
-    StringGroupFromDecimalTreeReader(int columnId, TypeDescription fileType,
-        TypeDescription readerType, boolean skipCorrupt) throws IOException {
-      super(columnId);
-      this.precision = fileType.getPrecision();
-      this.scale = fileType.getScale();
-      this.readerType = readerType;
-      decimalTreeReader = new DecimalTreeReader(columnId, precision, scale);
-      setConvertTreeReader(decimalTreeReader);
-      hiveDecimalResult = new HiveDecimalWritable();
-    }
-
-    @Override
-    Object next(Object previous) throws IOException {
-
-      HiveDecimalWritable readHiveDecimalResult =
-          (HiveDecimalWritable) decimalTreeReader.next(hiveDecimalResult);
-
-      Writable result = null;
-      if (readHiveDecimalResult != null) {
-        result = getStringGroupResultFromString(
-            previous, readerType, readHiveDecimalResult.getHiveDecimal().toString());
-      }
-      return result;
-    }
-
-    @Override
-    public void setConvertVectorElement(int elementNum) {
-      String string = decimalColVector.vector[elementNum].getHiveDecimal().toString();
-      byte[] bytes = string.getBytes();
-      assignStringGroupVectorEntry(bytesColVector, elementNum, readerType, bytes);
-    }
-
-    @Override
-    public void nextVector(ColumnVector previousVector,
-                           boolean[] isNull,
-                           final int batchSize) throws IOException {
-      if (decimalColVector == null) {
-        // Allocate column vector for file; cast column vector for reader.
-        decimalColVector = new DecimalColumnVector(precision, scale);
-        bytesColVector = (BytesColumnVector) previousVector;
-      }
-      // Read present/isNull stream
-      decimalTreeReader.nextVector(decimalColVector, isNull, batchSize);
-
-      convertVector(decimalColVector, bytesColVector, batchSize);
-    }
-  }
-
-  public static class StringGroupFromTimestampTreeReader extends ConvertTreeReader {
-
-    private TimestampTreeReader timestampTreeReader;
-
-    private final TypeDescription readerType;
-    private TimestampWritable timestampWritableResult;
-    private TimestampColumnVector timestampColVector;
-    private BytesColumnVector bytesColVector;
-
-    StringGroupFromTimestampTreeReader(int columnId, TypeDescription readerType,
-        boolean skipCorrupt) throws IOException {
-      super(columnId);
-      this.readerType = readerType;
-      timestampTreeReader = new TimestampTreeReader(columnId, skipCorrupt);
-      setConvertTreeReader(timestampTreeReader);
-      timestampWritableResult = new TimestampWritable();
-    }
-
-    @Override
-    Object next(Object previous) throws IOException {
-
-      TimestampWritable readTimestampWritableResult =
-          (TimestampWritable) timestampTreeReader.next(timestampWritableResult);
-
-      Writable result = null;
-      if (readTimestampWritableResult != null) {
-        result = getStringGroupResultFromString(
-            previous, readerType, readTimestampWritableResult.toString());
-      }
-
-      return result;
-    }
-
-    @Override
-    public void setConvertVectorElement(int elementNum) throws IOException {
-      String string =
-          timestampColVector.asScratchTimestamp(elementNum).toString();
-      byte[] bytes = string.getBytes();
-      assignStringGroupVectorEntry(bytesColVector, elementNum, readerType, bytes);
-    }
-
-    @Override
-    public void nextVector(ColumnVector previousVector,
-                           boolean[] isNull,
-                           final int batchSize) throws IOException {
-      if (timestampColVector == null) {
-        // Allocate column vector for file; cast column vector for reader.
-        timestampColVector = new TimestampColumnVector();
-        bytesColVector = (BytesColumnVector) previousVector;
-      }
-      // Read present/isNull stream
-      timestampTreeReader.nextVector(timestampColVector, isNull, batchSize);
-
-      convertVector(timestampColVector, bytesColVector, batchSize);
-    }
-  }
-
-  public static class StringGroupFromDateTreeReader extends ConvertTreeReader {
-
-    private DateTreeReader dateTreeReader;
-
-    private final TypeDescription readerType;
-    private LongColumnVector longColVector;
-    private BytesColumnVector bytesColVector;
-    private DateWritable dateWritableResult;
-    private Date date;
-
-    StringGroupFromDateTreeReader(int columnId, TypeDescription readerType,
-        boolean skipCorrupt) throws IOException {
-      super(columnId);
-      this.readerType = readerType;
-      dateTreeReader = new DateTreeReader(columnId);
-      setConvertTreeReader(dateTreeReader);
-      dateWritableResult = new DateWritable();
-      date = new Date(0);
-    }
-
-    @Override
-    Object next(Object previous) throws IOException {
-
-      DateWritable readDateWritableResult =
-          (DateWritable) dateTreeReader.next(dateWritableResult);
-
-      Writable result = null;
-      if (readDateWritableResult != null) {
-        result = getStringGroupResultFromString(
-            previous, readerType, readDateWritableResult.toString());
-      }
-
-      return result;
-    }
-
-    @Override
-    public void setConvertVectorElement(int elementNum) throws IOException {
-      date.setTime(DateWritable.daysToMillis((int) longColVector.vector[elementNum]));
-      String string = date.toString();
-      byte[] bytes = string.getBytes();
-      assignStringGroupVectorEntry(bytesColVector, elementNum, readerType, bytes);
-    }
-
-    @Override
-    public void nextVector(ColumnVector previousVector,
-                           boolean[] isNull,
-                           final int batchSize) throws IOException {
-      if (longColVector == null) {
-        // Allocate column vector for file; cast column vector for reader.
-        longColVector = new LongColumnVector();
-        bytesColVector = (BytesColumnVector) previousVector;
-      }
-      // Read present/isNull stream
-      dateTreeReader.nextVector(longColVector, isNull, batchSize);
-
-      convertVector(longColVector, bytesColVector, batchSize);
-    }
-  }
-
-  public static class StringGroupFromStringGroupTreeReader extends ConvertTreeReader {
-
-    private TreeReader stringGroupTreeReader;
-
-    private final TypeDescription fileType;
-    private final TypeDescription readerType;
-    private Writable writable;
-
-    StringGroupFromStringGroupTreeReader(int columnId, TypeDescription fileType,
-        TypeDescription readerType) throws IOException {
-      super(columnId);
-      this.fileType = fileType;
-      this.readerType = readerType;
-      stringGroupTreeReader = getStringGroupTreeReader(columnId, fileType);
-      setConvertTreeReader(stringGroupTreeReader);
-      writable = getStringGroupWritable(fileType);
-    }
-
-    @Override
-    Object next(Object previous) throws IOException {
-
-      String stringValue = stringFromStringGroupTreeReader(
-          stringGroupTreeReader, writable, fileType);
-
-      Writable result = null;
-      if (stringValue != null) {
-        result = getStringGroupResultFromString(
-            previous, readerType, stringValue);
-      }
-      return result;
-    }
-
-    @Override
-    public void nextVector(ColumnVector previousVector,
-                           boolean[] isNull,
-                           final int batchSize) throws IOException {
-      stringGroupTreeReader.nextVector(previousVector, isNull, batchSize);
-
-      BytesColumnVector resultColVector = (BytesColumnVector) previousVector;
-
-      if (resultColVector.isRepeating) {
-        if (resultColVector.noNulls || !resultColVector.isNull[0]) {
-          convertStringGroupVectorElement(resultColVector, 0, readerType);
-        } else {
-          resultColVector.noNulls = false;
-          resultColVector.isNull[0] = true;
-        }
-      } else if (resultColVector.noNulls){
-        for (int i = 0; i < batchSize; i++) {
-          convertStringGroupVectorElement(resultColVector, i, readerType);
-        }
-      } else {
-        for (int i = 0; i < batchSize; i++) {
-          if (!resultColVector.isNull[i]) {
-            convertStringGroupVectorElement(resultColVector, i, readerType);
-          } else {
-            resultColVector.noNulls = false;
-            resultColVector.isNull[i] = true;
-          }
-        }
-      }
-    }
-  }
-
-  public static class StringGroupFromBinaryTreeReader extends ConvertTreeReader {
-
-    private BinaryTreeReader binaryTreeReader;
-
-    private final TypeDescription readerType;
-    private BytesWritable binaryWritableResult;
-    private BytesColumnVector inBytesColVector;
-    private BytesColumnVector outBytesColVector;
-
-    StringGroupFromBinaryTreeReader(int columnId, TypeDescription readerType,
-        boolean skipCorrupt) throws IOException {
-      super(columnId);
-      this.readerType = readerType;
-      binaryTreeReader = new BinaryTreeReader(columnId);
-      setConvertTreeReader(binaryTreeReader);
-      binaryWritableResult = new BytesWritable();
-    }
-
-    @Override
-    Object next(Object previous) throws IOException {
-
-      BytesWritable readBytesWritableResult =
-          (BytesWritable) binaryTreeReader.next(binaryWritableResult);
-
-      Writable result = null;
-      if (readBytesWritableResult != null) {
-        result = getStringGroupResultFromString(
-            previous, readerType, readBytesWritableResult.toString());
-      }
-
-      return result;
-    }
-
-    @Override
-    public void setConvertVectorElement(int elementNum) throws IOException {
-      // UNDONE: Binary to StringGroup conversion?
-      byte[] bytes = inBytesColVector.vector[elementNum];
-      int start = inBytesColVector.start[elementNum];
-      int length = inBytesColVector.length[elementNum];
-      assignStringGroupVectorEntry(outBytesColVector, elementNum, readerType, bytes, start, length);
-    }
-
-    @Override
-    public void nextVector(ColumnVector previousVector,
-                           boolean[] isNull,
-                           final int batchSize) throws IOException {
-      if (inBytesColVector == null) {
-        // Allocate column vector for file; cast column vector for reader.
-        inBytesColVector = new BytesColumnVector();
-        outBytesColVector = (BytesColumnVector) previousVector;
-      }
-      // Read present/isNull stream
-      binaryTreeReader.nextVector(inBytesColVector, isNull, batchSize);
-
-      convertVector(inBytesColVector, outBytesColVector, batchSize);
-    }
-  }
-
-  public static class TimestampFromAnyIntegerTreeReader extends ConvertTreeReader {
-
-    private AnyIntegerTreeReader anyIntegerAsLongTreeReader;
-
-    private LongColumnVector longColVector;
-    private TimestampColumnVector timestampColVector;
-
-    TimestampFromAnyIntegerTreeReader(int columnId, TypeDescription fileType,
-        boolean skipCorrupt) throws IOException {
-      super(columnId);
-      anyIntegerAsLongTreeReader =
-          new AnyIntegerTreeReader(columnId, fileType, skipCorrupt);
-      setConvertTreeReader(anyIntegerAsLongTreeReader);
-    }
-
-    @Override
-    Object next(Object previous) throws IOException {
-      TimestampWritable result = null;
-      if (anyIntegerAsLongTreeReader.read()) {
-        long longValue = anyIntegerAsLongTreeReader.getLong();
-          if (previous == null) {
-            result = new TimestampWritable();
-          } else {
-            result = (TimestampWritable) previous;
-          }
-          // UNDONE: What does the boolean setting need to be?
-          result.set(TimestampWritable.longToTimestamp(longValue, false));
-      }
-      return result;
-    }
-
-    @Override
-    public void setConvertVectorElement(int elementNum) {
-      long longValue = longColVector.vector[elementNum];
-      // UNDONE: What does the boolean setting need to be?
-      timestampColVector.set(elementNum, TimestampWritable.longToTimestamp(longValue, false));
-    }
-
-    @Override
-    public void nextVector(ColumnVector previousVector,
-                           boolean[] isNull,
-                           final int batchSize) throws IOException {
-      if (longColVector == null) {
-        // Allocate column vector for file; cast column vector for reader.
-        longColVector = new LongColumnVector();
-        timestampColVector = (TimestampColumnVector) previousVector;
-      }
-      // Read present/isNull stream
-      anyIntegerAsLongTreeReader.nextVector(longColVector, isNull, batchSize);
-
-      convertVector(longColVector, timestampColVector, batchSize);
-    }
-  }
-
-  public static class TimestampFromFloatTreeReader extends ConvertTreeReader {
-
-    private FloatTreeReader floatTreeReader;
-
-    private FloatWritable floatResult;
-    private DoubleColumnVector doubleColVector;
-    private TimestampColumnVector timestampColVector;
-
-    TimestampFromFloatTreeReader(int columnId, TypeDescription fileType,
-        boolean skipCorrupt) throws IOException {
-      super(columnId);
-      floatTreeReader = new FloatTreeReader(columnId);
-      setConvertTreeReader(floatTreeReader);
-      floatResult = new FloatWritable();
-    }
-
-    @Override
-    Object next(Object previous) throws IOException {
-
-      FloatWritable readFloatResult =
-          (FloatWritable) floatTreeReader.next(floatResult);
-
-      TimestampWritable result = null;
-      if (readFloatResult != null) {
-        float floatValue = readFloatResult.get();
-        if (previous == null) {
-          result = new TimestampWritable();
-        } else {
-          result = (TimestampWritable) previous;
-        }
-        result.set(TimestampWritable.doubleToTimestamp(floatValue));
-      }
-      return result;
-    }
-
-    @Override
-    public void setConvertVectorElement(int elementNum) {
-      float floatValue = (float) doubleColVe

<TRUNCATED>

[11/27] hive git commit: HIVE-11417. Move the ReaderImpl and RowReaderImpl to the ORC module, by making shims for the row by row reader. (omalley reviewed by prasanth_j)

Posted by om...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderUtils.java
deleted file mode 100644
index 4192588..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderUtils.java
+++ /dev/null
@@ -1,586 +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.hive.ql.io.orc;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-import java.util.TreeMap;
-
-import com.google.common.collect.Lists;
-import org.apache.commons.lang.builder.HashCodeBuilder;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hive.common.io.DiskRange;
-import org.apache.hadoop.hive.common.io.DiskRangeList;
-import org.apache.hadoop.hive.common.io.DiskRangeList.CreateHelper;
-import org.apache.hadoop.hive.common.io.DiskRangeList.MutateHelper;
-import org.apache.hadoop.hive.shims.HadoopShims;
-import org.apache.hadoop.hive.shims.ShimLoader;
-import org.apache.hadoop.hive.shims.HadoopShims.ByteBufferPoolShim;
-import org.apache.hadoop.hive.shims.HadoopShims.ZeroCopyReaderShim;
-import org.apache.orc.StripeInformation;
-import org.apache.orc.impl.BufferChunk;
-import org.apache.orc.CompressionCodec;
-import org.apache.orc.DataReader;
-import org.apache.orc.impl.DataReaderProperties;
-import org.apache.orc.impl.DirectDecompressionCodec;
-import org.apache.orc.OrcProto;
-
-import com.google.common.collect.ComparisonChain;
-import org.apache.orc.impl.InStream;
-import org.apache.orc.impl.OrcIndex;
-import org.apache.orc.impl.OutStream;
-
-/**
- * Stateless methods shared between RecordReaderImpl and EncodedReaderImpl.
- */
-public class RecordReaderUtils {
-  private static final HadoopShims SHIMS = ShimLoader.getHadoopShims();
-
-  private static class DefaultDataReader implements DataReader {
-    private FSDataInputStream file = null;
-    private final ByteBufferAllocatorPool pool;
-    private ZeroCopyReaderShim zcr = null;
-    private final FileSystem fs;
-    private final Path path;
-    private final boolean useZeroCopy;
-    private final CompressionCodec codec;
-    private final int bufferSize;
-    private final int typeCount;
-
-    private DefaultDataReader(DefaultDataReader other) {
-      this.pool = other.pool;
-      this.zcr = other.zcr;
-      this.bufferSize = other.bufferSize;
-      this.typeCount = other.typeCount;
-      this.fs = other.fs;
-      this.path = other.path;
-      this.useZeroCopy = other.useZeroCopy;
-      this.codec = other.codec;
-    }
-
-    private DefaultDataReader(DataReaderProperties properties) {
-      this.fs = properties.getFileSystem();
-      this.path = properties.getPath();
-      this.useZeroCopy = properties.getZeroCopy();
-      this.codec = WriterImpl.createCodec(properties.getCompression());
-      this.bufferSize = properties.getBufferSize();
-      this.typeCount = properties.getTypeCount();
-      if (useZeroCopy) {
-        this.pool = new ByteBufferAllocatorPool();
-      } else {
-        this.pool = null;
-      }
-    }
-
-    @Override
-    public void open() throws IOException {
-      this.file = fs.open(path);
-      if (useZeroCopy) {
-        zcr = RecordReaderUtils.createZeroCopyShim(file, codec, pool);
-      } else {
-        zcr = null;
-      }
-    }
-
-    @Override
-    public OrcIndex readRowIndex(StripeInformation stripe,
-                                 OrcProto.StripeFooter footer,
-                                 boolean[] included,
-                                 OrcProto.RowIndex[] indexes,
-                                 boolean[] sargColumns,
-                                 OrcProto.BloomFilterIndex[] bloomFilterIndices
-                                 ) throws IOException {
-      if (file == null) {
-        open();
-      }
-      if (footer == null) {
-        footer = readStripeFooter(stripe);
-      }
-      if (indexes == null) {
-        indexes = new OrcProto.RowIndex[typeCount];
-      }
-      if (bloomFilterIndices == null) {
-        bloomFilterIndices = new OrcProto.BloomFilterIndex[typeCount];
-      }
-      long offset = stripe.getOffset();
-      List<OrcProto.Stream> streams = footer.getStreamsList();
-      for (int i = 0; i < streams.size(); i++) {
-        OrcProto.Stream stream = streams.get(i);
-        OrcProto.Stream nextStream = null;
-        if (i < streams.size() - 1) {
-          nextStream = streams.get(i+1);
-        }
-        int col = stream.getColumn();
-        int len = (int) stream.getLength();
-        // row index stream and bloom filter are interlaced, check if the sarg column contains bloom
-        // filter and combine the io to read row index and bloom filters for that column together
-        if (stream.hasKind() && (stream.getKind() == OrcProto.Stream.Kind.ROW_INDEX)) {
-          boolean readBloomFilter = false;
-          if (sargColumns != null && sargColumns[col] &&
-              nextStream.getKind() == OrcProto.Stream.Kind.BLOOM_FILTER) {
-            len += nextStream.getLength();
-            i += 1;
-            readBloomFilter = true;
-          }
-          if ((included == null || included[col]) && indexes[col] == null) {
-            byte[] buffer = new byte[len];
-            file.readFully(offset, buffer, 0, buffer.length);
-            ByteBuffer bb = ByteBuffer.wrap(buffer);
-            indexes[col] = OrcProto.RowIndex.parseFrom(InStream.create("index",
-                Lists.<DiskRange>newArrayList(new BufferChunk(bb, 0)), stream.getLength(),
-                codec, bufferSize));
-            if (readBloomFilter) {
-              bb.position((int) stream.getLength());
-              bloomFilterIndices[col] = OrcProto.BloomFilterIndex.parseFrom(InStream.create(
-                  "bloom_filter", Lists.<DiskRange>newArrayList(new BufferChunk(bb, 0)),
-                  nextStream.getLength(), codec, bufferSize));
-            }
-          }
-        }
-        offset += len;
-      }
-
-      OrcIndex index = new OrcIndex(indexes, bloomFilterIndices);
-      return index;
-    }
-
-    @Override
-    public OrcProto.StripeFooter readStripeFooter(StripeInformation stripe) throws IOException {
-      if (file == null) {
-        open();
-      }
-      long offset = stripe.getOffset() + stripe.getIndexLength() + stripe.getDataLength();
-      int tailLength = (int) stripe.getFooterLength();
-
-      // read the footer
-      ByteBuffer tailBuf = ByteBuffer.allocate(tailLength);
-      file.readFully(offset, tailBuf.array(), tailBuf.arrayOffset(), tailLength);
-      return OrcProto.StripeFooter.parseFrom(InStream.createCodedInputStream("footer",
-          Lists.<DiskRange>newArrayList(new BufferChunk(tailBuf, 0)),
-          tailLength, codec, bufferSize));
-    }
-
-    @Override
-    public DiskRangeList readFileData(
-        DiskRangeList range, long baseOffset, boolean doForceDirect) throws IOException {
-      return RecordReaderUtils.readDiskRanges(file, zcr, baseOffset, range, doForceDirect);
-    }
-
-    @Override
-    public void close() throws IOException {
-      if (file != null) {
-        file.close();
-      }
-      if (pool != null) {
-        pool.clear();
-      }
-    }
-
-    @Override
-    public boolean isTrackingDiskRanges() {
-      return zcr != null;
-    }
-
-    @Override
-    public void releaseBuffer(ByteBuffer buffer) {
-      zcr.releaseBuffer(buffer);
-    }
-
-    @Override
-    public DataReader clone() {
-      return new DefaultDataReader(this);
-    }
-
-  }
-
-  public static DataReader createDefaultDataReader(DataReaderProperties properties) {
-    return new DefaultDataReader(properties);
-  }
-
-  public static boolean[] findPresentStreamsByColumn(
-      List<OrcProto.Stream> streamList, List<OrcProto.Type> types) {
-    boolean[] hasNull = new boolean[types.size()];
-    for(OrcProto.Stream stream: streamList) {
-      if (stream.hasKind() && (stream.getKind() == OrcProto.Stream.Kind.PRESENT)) {
-        hasNull[stream.getColumn()] = true;
-      }
-    }
-    return hasNull;
-  }
-
-  /**
-   * Does region A overlap region B? The end points are inclusive on both sides.
-   * @param leftA A's left point
-   * @param rightA A's right point
-   * @param leftB B's left point
-   * @param rightB B's right point
-   * @return Does region A overlap region B?
-   */
-  static boolean overlap(long leftA, long rightA, long leftB, long rightB) {
-    if (leftA <= leftB) {
-      return rightA >= leftB;
-    }
-    return rightB >= leftA;
-  }
-
-  public static void addEntireStreamToRanges(
-      long offset, long length, CreateHelper list, boolean doMergeBuffers) {
-    list.addOrMerge(offset, offset + length, doMergeBuffers, false);
-  }
-
-  public static void addRgFilteredStreamToRanges(OrcProto.Stream stream,
-      boolean[] includedRowGroups, boolean isCompressed, OrcProto.RowIndex index,
-      OrcProto.ColumnEncoding encoding, OrcProto.Type type, int compressionSize, boolean hasNull,
-      long offset, long length, CreateHelper list, boolean doMergeBuffers) {
-    for (int group = 0; group < includedRowGroups.length; ++group) {
-      if (!includedRowGroups[group]) continue;
-      int posn = getIndexPosition(
-          encoding.getKind(), type.getKind(), stream.getKind(), isCompressed, hasNull);
-      long start = index.getEntry(group).getPositions(posn);
-      final long nextGroupOffset;
-      boolean isLast = group == (includedRowGroups.length - 1);
-      nextGroupOffset = isLast ? length : index.getEntry(group + 1).getPositions(posn);
-
-      start += offset;
-      long end = offset + estimateRgEndOffset(
-          isCompressed, isLast, nextGroupOffset, length, compressionSize);
-      list.addOrMerge(start, end, doMergeBuffers, true);
-    }
-  }
-
-  public static long estimateRgEndOffset(boolean isCompressed, boolean isLast,
-      long nextGroupOffset, long streamLength, int bufferSize) {
-    // figure out the worst case last location
-    // if adjacent groups have the same compressed block offset then stretch the slop
-    // by factor of 2 to safely accommodate the next compression block.
-    // One for the current compression block and another for the next compression block.
-    long slop = isCompressed ? 2 * (OutStream.HEADER_SIZE + bufferSize) : WORST_UNCOMPRESSED_SLOP;
-    return isLast ? streamLength : Math.min(streamLength, nextGroupOffset + slop);
-  }
-
-  private static final int BYTE_STREAM_POSITIONS = 1;
-  private static final int RUN_LENGTH_BYTE_POSITIONS = BYTE_STREAM_POSITIONS + 1;
-  private static final int BITFIELD_POSITIONS = RUN_LENGTH_BYTE_POSITIONS + 1;
-  private static final int RUN_LENGTH_INT_POSITIONS = BYTE_STREAM_POSITIONS + 1;
-
-  /**
-   * Get the offset in the index positions for the column that the given
-   * stream starts.
-   * @param columnEncoding the encoding of the column
-   * @param columnType the type of the column
-   * @param streamType the kind of the stream
-   * @param isCompressed is the file compressed
-   * @param hasNulls does the column have a PRESENT stream?
-   * @return the number of positions that will be used for that stream
-   */
-  public static int getIndexPosition(OrcProto.ColumnEncoding.Kind columnEncoding,
-                              OrcProto.Type.Kind columnType,
-                              OrcProto.Stream.Kind streamType,
-                              boolean isCompressed,
-                              boolean hasNulls) {
-    if (streamType == OrcProto.Stream.Kind.PRESENT) {
-      return 0;
-    }
-    int compressionValue = isCompressed ? 1 : 0;
-    int base = hasNulls ? (BITFIELD_POSITIONS + compressionValue) : 0;
-    switch (columnType) {
-      case BOOLEAN:
-      case BYTE:
-      case SHORT:
-      case INT:
-      case LONG:
-      case FLOAT:
-      case DOUBLE:
-      case DATE:
-      case STRUCT:
-      case MAP:
-      case LIST:
-      case UNION:
-        return base;
-      case CHAR:
-      case VARCHAR:
-      case STRING:
-        if (columnEncoding == OrcProto.ColumnEncoding.Kind.DICTIONARY ||
-            columnEncoding == OrcProto.ColumnEncoding.Kind.DICTIONARY_V2) {
-          return base;
-        } else {
-          if (streamType == OrcProto.Stream.Kind.DATA) {
-            return base;
-          } else {
-            return base + BYTE_STREAM_POSITIONS + compressionValue;
-          }
-        }
-      case BINARY:
-        if (streamType == OrcProto.Stream.Kind.DATA) {
-          return base;
-        }
-        return base + BYTE_STREAM_POSITIONS + compressionValue;
-      case DECIMAL:
-        if (streamType == OrcProto.Stream.Kind.DATA) {
-          return base;
-        }
-        return base + BYTE_STREAM_POSITIONS + compressionValue;
-      case TIMESTAMP:
-        if (streamType == OrcProto.Stream.Kind.DATA) {
-          return base;
-        }
-        return base + RUN_LENGTH_INT_POSITIONS + compressionValue;
-      default:
-        throw new IllegalArgumentException("Unknown type " + columnType);
-    }
-  }
-
-  // for uncompressed streams, what is the most overlap with the following set
-  // of rows (long vint literal group).
-  static final int WORST_UNCOMPRESSED_SLOP = 2 + 8 * 512;
-
-  /**
-   * Is this stream part of a dictionary?
-   * @return is this part of a dictionary?
-   */
-  public static boolean isDictionary(OrcProto.Stream.Kind kind,
-                              OrcProto.ColumnEncoding encoding) {
-    assert kind != OrcProto.Stream.Kind.DICTIONARY_COUNT;
-    OrcProto.ColumnEncoding.Kind encodingKind = encoding.getKind();
-    return kind == OrcProto.Stream.Kind.DICTIONARY_DATA ||
-      (kind == OrcProto.Stream.Kind.LENGTH &&
-       (encodingKind == OrcProto.ColumnEncoding.Kind.DICTIONARY ||
-        encodingKind == OrcProto.ColumnEncoding.Kind.DICTIONARY_V2));
-  }
-
-  /**
-   * Build a string representation of a list of disk ranges.
-   * @param range ranges to stringify
-   * @return the resulting string
-   */
-  public static String stringifyDiskRanges(DiskRangeList range) {
-    StringBuilder buffer = new StringBuilder();
-    buffer.append("[");
-    boolean isFirst = true;
-    while (range != null) {
-      if (!isFirst) {
-        buffer.append(", {");
-      } else {
-        buffer.append("{");
-      }
-      isFirst = false;
-      buffer.append(range.toString());
-      buffer.append("}");
-      range = range.next;
-    }
-    buffer.append("]");
-    return buffer.toString();
-  }
-
-  /**
-   * Read the list of ranges from the file.
-   * @param file the file to read
-   * @param base the base of the stripe
-   * @param range the disk ranges within the stripe to read
-   * @return the bytes read for each disk range, which is the same length as
-   *    ranges
-   * @throws IOException
-   */
-  static DiskRangeList readDiskRanges(FSDataInputStream file,
-                                 ZeroCopyReaderShim zcr,
-                                 long base,
-                                 DiskRangeList range,
-                                 boolean doForceDirect) throws IOException {
-    if (range == null) return null;
-    DiskRangeList prev = range.prev;
-    if (prev == null) {
-      prev = new MutateHelper(range);
-    }
-    while (range != null) {
-      if (range.hasData()) {
-        range = range.next;
-        continue;
-      }
-      int len = (int) (range.getEnd() - range.getOffset());
-      long off = range.getOffset();
-      if (zcr != null) {
-        file.seek(base + off);
-        boolean hasReplaced = false;
-        while (len > 0) {
-          ByteBuffer partial = zcr.readBuffer(len, false);
-          BufferChunk bc = new BufferChunk(partial, off);
-          if (!hasReplaced) {
-            range.replaceSelfWith(bc);
-            hasReplaced = true;
-          } else {
-            range.insertAfter(bc);
-          }
-          range = bc;
-          int read = partial.remaining();
-          len -= read;
-          off += read;
-        }
-      } else {
-        // Don't use HDFS ByteBuffer API because it has no readFully, and is buggy and pointless.
-        byte[] buffer = new byte[len];
-        file.readFully((base + off), buffer, 0, buffer.length);
-        ByteBuffer bb = null;
-        if (doForceDirect) {
-          bb = ByteBuffer.allocateDirect(len);
-          bb.put(buffer);
-          bb.position(0);
-          bb.limit(len);
-        } else {
-          bb = ByteBuffer.wrap(buffer);
-        }
-        range = range.replaceSelfWith(new BufferChunk(bb, range.getOffset()));
-      }
-      range = range.next;
-    }
-    return prev.next;
-  }
-
-
-  static List<DiskRange> getStreamBuffers(DiskRangeList range, long offset, long length) {
-    // This assumes sorted ranges (as do many other parts of ORC code.
-    ArrayList<DiskRange> buffers = new ArrayList<DiskRange>();
-    if (length == 0) return buffers;
-    long streamEnd = offset + length;
-    boolean inRange = false;
-    while (range != null) {
-      if (!inRange) {
-        if (range.getEnd() <= offset) {
-          range = range.next;
-          continue; // Skip until we are in range.
-        }
-        inRange = true;
-        if (range.getOffset() < offset) {
-          // Partial first buffer, add a slice of it.
-          buffers.add(range.sliceAndShift(offset, Math.min(streamEnd, range.getEnd()), -offset));
-          if (range.getEnd() >= streamEnd) break; // Partial first buffer is also partial last buffer.
-          range = range.next;
-          continue;
-        }
-      } else if (range.getOffset() >= streamEnd) {
-        break;
-      }
-      if (range.getEnd() > streamEnd) {
-        // Partial last buffer (may also be the first buffer), add a slice of it.
-        buffers.add(range.sliceAndShift(range.getOffset(), streamEnd, -offset));
-        break;
-      }
-      // Buffer that belongs entirely to one stream.
-      // TODO: ideally we would want to reuse the object and remove it from the list, but we cannot
-      //       because bufferChunks is also used by clearStreams for zcr. Create a useless dup.
-      buffers.add(range.sliceAndShift(range.getOffset(), range.getEnd(), -offset));
-      if (range.getEnd() == streamEnd) break;
-      range = range.next;
-    }
-    return buffers;
-  }
-
-  static ZeroCopyReaderShim createZeroCopyShim(FSDataInputStream file,
-      CompressionCodec codec, ByteBufferAllocatorPool pool) throws IOException {
-    if ((codec == null || ((codec instanceof DirectDecompressionCodec)
-            && ((DirectDecompressionCodec) codec).isAvailable()))) {
-      /* codec is null or is available */
-      return ShimLoader.getHadoopShims().getZeroCopyReader(file, pool);
-    }
-    return null;
-  }
-
-  // this is an implementation copied from ElasticByteBufferPool in hadoop-2,
-  // which lacks a clear()/clean() operation
-  public final static class ByteBufferAllocatorPool implements ByteBufferPoolShim {
-    private static final class Key implements Comparable<Key> {
-      private final int capacity;
-      private final long insertionGeneration;
-
-      Key(int capacity, long insertionGeneration) {
-        this.capacity = capacity;
-        this.insertionGeneration = insertionGeneration;
-      }
-
-      @Override
-      public int compareTo(Key other) {
-        return ComparisonChain.start().compare(capacity, other.capacity)
-            .compare(insertionGeneration, other.insertionGeneration).result();
-      }
-
-      @Override
-      public boolean equals(Object rhs) {
-        if (rhs == null) {
-          return false;
-        }
-        try {
-          Key o = (Key) rhs;
-          return (compareTo(o) == 0);
-        } catch (ClassCastException e) {
-          return false;
-        }
-      }
-
-      @Override
-      public int hashCode() {
-        return new HashCodeBuilder().append(capacity).append(insertionGeneration)
-            .toHashCode();
-      }
-    }
-
-    private final TreeMap<Key, ByteBuffer> buffers = new TreeMap<Key, ByteBuffer>();
-
-    private final TreeMap<Key, ByteBuffer> directBuffers = new TreeMap<Key, ByteBuffer>();
-
-    private long currentGeneration = 0;
-
-    private final TreeMap<Key, ByteBuffer> getBufferTree(boolean direct) {
-      return direct ? directBuffers : buffers;
-    }
-
-    public void clear() {
-      buffers.clear();
-      directBuffers.clear();
-    }
-
-    @Override
-    public ByteBuffer getBuffer(boolean direct, int length) {
-      TreeMap<Key, ByteBuffer> tree = getBufferTree(direct);
-      Map.Entry<Key, ByteBuffer> entry = tree.ceilingEntry(new Key(length, 0));
-      if (entry == null) {
-        return direct ? ByteBuffer.allocateDirect(length) : ByteBuffer
-            .allocate(length);
-      }
-      tree.remove(entry.getKey());
-      return entry.getValue();
-    }
-
-    @Override
-    public void putBuffer(ByteBuffer buffer) {
-      TreeMap<Key, ByteBuffer> tree = getBufferTree(buffer.isDirect());
-      while (true) {
-        Key key = new Key(buffer.capacity(), currentGeneration++);
-        if (!tree.containsKey(key)) {
-          tree.put(key, buffer);
-          return;
-        }
-        // Buffers are indexed by (capacity, generation).
-        // If our key is not unique on the first try, we try again
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/ql/src/java/org/apache/hadoop/hive/ql/io/orc/SchemaEvolution.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/SchemaEvolution.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/SchemaEvolution.java
deleted file mode 100644
index 046665b..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/SchemaEvolution.java
+++ /dev/null
@@ -1,190 +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.hive.ql.io.orc;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.orc.TypeDescription;
-
-/**
- * Take the file types and the (optional) configuration column names/types and see if there
- * has been schema evolution.
- */
-public class SchemaEvolution {
-  private final Map<TypeDescription, TypeDescription> readerToFile;
-  private final boolean[] included;
-  private final TypeDescription readerSchema;
-  private static final Log LOG = LogFactory.getLog(SchemaEvolution.class);
-
-  public SchemaEvolution(TypeDescription readerSchema, boolean[] included) {
-    this.included = included;
-    readerToFile = null;
-    this.readerSchema = readerSchema;
-  }
-
-  public SchemaEvolution(TypeDescription fileSchema,
-                         TypeDescription readerSchema,
-                         boolean[] included) throws IOException {
-    readerToFile = new HashMap<>(readerSchema.getMaximumId() + 1);
-    this.included = included;
-    if (checkAcidSchema(fileSchema)) {
-      this.readerSchema = createEventSchema(readerSchema);
-    } else {
-      this.readerSchema = readerSchema;
-    }
-    buildMapping(fileSchema, this.readerSchema);
-  }
-
-  public TypeDescription getReaderSchema() {
-    return readerSchema;
-  }
-
-  public TypeDescription getFileType(TypeDescription readerType) {
-    TypeDescription result;
-    if (readerToFile == null) {
-      if (included == null || included[readerType.getId()]) {
-        result = readerType;
-      } else {
-        result = null;
-      }
-    } else {
-      result = readerToFile.get(readerType);
-    }
-    return result;
-  }
-
-  void buildMapping(TypeDescription fileType,
-                    TypeDescription readerType) throws IOException {
-    // if the column isn't included, don't map it
-    if (included != null && !included[readerType.getId()]) {
-      return;
-    }
-    boolean isOk = true;
-    // check the easy case first
-    if (fileType.getCategory() == readerType.getCategory()) {
-      switch (readerType.getCategory()) {
-        case BOOLEAN:
-        case BYTE:
-        case SHORT:
-        case INT:
-        case LONG:
-        case DOUBLE:
-        case FLOAT:
-        case STRING:
-        case TIMESTAMP:
-        case BINARY:
-        case DATE:
-          // these are always a match
-          break;
-        case CHAR:
-        case VARCHAR:
-          // HIVE-13648: Look at ORC data type conversion edge cases (CHAR, VARCHAR, DECIMAL)
-          isOk = fileType.getMaxLength() == readerType.getMaxLength();
-          break;
-        case DECIMAL:
-          // HIVE-13648: Look at ORC data type conversion edge cases (CHAR, VARCHAR, DECIMAL)
-          // TODO we don't enforce scale and precision checks, but probably should
-          break;
-        case UNION:
-        case MAP:
-        case LIST: {
-          // these must be an exact match
-          List<TypeDescription> fileChildren = fileType.getChildren();
-          List<TypeDescription> readerChildren = readerType.getChildren();
-          if (fileChildren.size() == readerChildren.size()) {
-            for(int i=0; i < fileChildren.size(); ++i) {
-              buildMapping(fileChildren.get(i), readerChildren.get(i));
-            }
-          } else {
-            isOk = false;
-          }
-          break;
-        }
-        case STRUCT: {
-          // allow either side to have fewer fields than the other
-          List<TypeDescription> fileChildren = fileType.getChildren();
-          List<TypeDescription> readerChildren = readerType.getChildren();
-          int jointSize = Math.min(fileChildren.size(), readerChildren.size());
-          for(int i=0; i < jointSize; ++i) {
-            buildMapping(fileChildren.get(i), readerChildren.get(i));
-          }
-          break;
-        }
-        default:
-          throw new IllegalArgumentException("Unknown type " + readerType);
-      }
-    } else {
-      /*
-       * Check for the few cases where will not convert....
-       */
-
-      isOk = ConvertTreeReaderFactory.canConvert(fileType, readerType);
-    }
-    if (isOk) {
-      readerToFile.put(readerType, fileType);
-    } else {
-      throw new IOException(
-          String.format(
-              "ORC does not support type conversion from file type %s (%d) to reader type %s (%d)",
-              fileType.toString(), fileType.getId(),
-              readerType.toString(), readerType.getId()));
-    }
-  }
-
-  private static boolean checkAcidSchema(TypeDescription type) {
-    if (type.getCategory().equals(TypeDescription.Category.STRUCT)) {
-      List<String> rootFields = type.getFieldNames();
-      if (acidEventFieldNames.equals(rootFields)) {
-        return true;
-      }
-    }
-    return false;
-  }
-
-  /**
-   * @param typeDescr
-   * @return ORC types for the ACID event based on the row's type description
-   */
-  public static TypeDescription createEventSchema(TypeDescription typeDescr) {
-    TypeDescription result = TypeDescription.createStruct()
-        .addField("operation", TypeDescription.createInt())
-        .addField("originalTransaction", TypeDescription.createLong())
-        .addField("bucket", TypeDescription.createInt())
-        .addField("rowId", TypeDescription.createLong())
-        .addField("currentTransaction", TypeDescription.createLong())
-        .addField("row", typeDescr.clone());
-    return result;
-  }
-
-  public static final List<String> acidEventFieldNames= new ArrayList<String>();
-  static {
-    acidEventFieldNames.add("operation");
-    acidEventFieldNames.add("originalTransaction");
-    acidEventFieldNames.add("bucket");
-    acidEventFieldNames.add("rowId");
-    acidEventFieldNames.add("currentTransaction");
-    acidEventFieldNames.add("row");
-  }
-}


[24/27] hive git commit: HIVE-11417. Move the ReaderImpl and RowReaderImpl to the ORC module, by making shims for the row by row reader. (omalley reviewed by prasanth_j)

Posted by om...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/orc/src/java/org/apache/orc/impl/RecordReaderUtils.java
----------------------------------------------------------------------
diff --git a/orc/src/java/org/apache/orc/impl/RecordReaderUtils.java b/orc/src/java/org/apache/orc/impl/RecordReaderUtils.java
new file mode 100644
index 0000000..1067957
--- /dev/null
+++ b/orc/src/java/org/apache/orc/impl/RecordReaderUtils.java
@@ -0,0 +1,578 @@
+/**
+ * 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.orc.impl;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+
+import com.google.common.collect.Lists;
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.common.io.DiskRange;
+import org.apache.hadoop.hive.common.io.DiskRangeList;
+import org.apache.hadoop.hive.common.io.DiskRangeList.CreateHelper;
+import org.apache.hadoop.hive.common.io.DiskRangeList.MutateHelper;
+import org.apache.orc.CompressionCodec;
+import org.apache.orc.DataReader;
+import org.apache.orc.OrcProto;
+
+import com.google.common.collect.ComparisonChain;
+import org.apache.orc.StripeInformation;
+
+/**
+ * Stateless methods shared between RecordReaderImpl and EncodedReaderImpl.
+ */
+public class RecordReaderUtils {
+  private static final HadoopShims SHIMS = HadoopShims.Factory.get();
+
+  private static class DefaultDataReader implements DataReader {
+    private FSDataInputStream file = null;
+    private final ByteBufferAllocatorPool pool;
+    private HadoopShims.ZeroCopyReaderShim zcr = null;
+    private final FileSystem fs;
+    private final Path path;
+    private final boolean useZeroCopy;
+    private final CompressionCodec codec;
+    private final int bufferSize;
+    private final int typeCount;
+
+    private DefaultDataReader(DefaultDataReader other) {
+      this.pool = other.pool;
+      this.bufferSize = other.bufferSize;
+      this.typeCount = other.typeCount;
+      this.fs = other.fs;
+      this.path = other.path;
+      this.useZeroCopy = other.useZeroCopy;
+      this.codec = other.codec;
+    }
+
+    private DefaultDataReader(DataReaderProperties properties) {
+      this.fs = properties.getFileSystem();
+      this.path = properties.getPath();
+      this.useZeroCopy = properties.getZeroCopy();
+      this.codec = WriterImpl.createCodec(properties.getCompression());
+      this.bufferSize = properties.getBufferSize();
+      this.typeCount = properties.getTypeCount();
+      if (useZeroCopy) {
+        this.pool = new ByteBufferAllocatorPool();
+      } else {
+        this.pool = null;
+      }
+    }
+
+    @Override
+    public void open() throws IOException {
+      this.file = fs.open(path);
+      if (useZeroCopy) {
+        zcr = RecordReaderUtils.createZeroCopyShim(file, codec, pool);
+      } else {
+        zcr = null;
+      }
+    }
+
+    @Override
+    public OrcIndex readRowIndex(StripeInformation stripe,
+                                 OrcProto.StripeFooter footer,
+                                 boolean[] included,
+                                 OrcProto.RowIndex[] indexes,
+                                 boolean[] sargColumns,
+                                 OrcProto.BloomFilterIndex[] bloomFilterIndices
+                                 ) throws IOException {
+      if (file == null) {
+        open();
+      }
+      if (footer == null) {
+        footer = readStripeFooter(stripe);
+      }
+      if (indexes == null) {
+        indexes = new OrcProto.RowIndex[typeCount];
+      }
+      if (bloomFilterIndices == null) {
+        bloomFilterIndices = new OrcProto.BloomFilterIndex[typeCount];
+      }
+      long offset = stripe.getOffset();
+      List<OrcProto.Stream> streams = footer.getStreamsList();
+      for (int i = 0; i < streams.size(); i++) {
+        OrcProto.Stream stream = streams.get(i);
+        OrcProto.Stream nextStream = null;
+        if (i < streams.size() - 1) {
+          nextStream = streams.get(i+1);
+        }
+        int col = stream.getColumn();
+        int len = (int) stream.getLength();
+        // row index stream and bloom filter are interlaced, check if the sarg column contains bloom
+        // filter and combine the io to read row index and bloom filters for that column together
+        if (stream.hasKind() && (stream.getKind() == OrcProto.Stream.Kind.ROW_INDEX)) {
+          boolean readBloomFilter = false;
+          if (sargColumns != null && sargColumns[col] &&
+              nextStream.getKind() == OrcProto.Stream.Kind.BLOOM_FILTER) {
+            len += nextStream.getLength();
+            i += 1;
+            readBloomFilter = true;
+          }
+          if ((included == null || included[col]) && indexes[col] == null) {
+            byte[] buffer = new byte[len];
+            file.readFully(offset, buffer, 0, buffer.length);
+            ByteBuffer bb = ByteBuffer.wrap(buffer);
+            indexes[col] = OrcProto.RowIndex.parseFrom(InStream.create("index",
+                Lists.<DiskRange>newArrayList(new BufferChunk(bb, 0)), stream.getLength(),
+                codec, bufferSize));
+            if (readBloomFilter) {
+              bb.position((int) stream.getLength());
+              bloomFilterIndices[col] = OrcProto.BloomFilterIndex.parseFrom(InStream.create(
+                  "bloom_filter", Lists.<DiskRange>newArrayList(new BufferChunk(bb, 0)),
+                  nextStream.getLength(), codec, bufferSize));
+            }
+          }
+        }
+        offset += len;
+      }
+
+      OrcIndex index = new OrcIndex(indexes, bloomFilterIndices);
+      return index;
+    }
+
+    @Override
+    public OrcProto.StripeFooter readStripeFooter(StripeInformation stripe) throws IOException {
+      if (file == null) {
+        open();
+      }
+      long offset = stripe.getOffset() + stripe.getIndexLength() + stripe.getDataLength();
+      int tailLength = (int) stripe.getFooterLength();
+
+      // read the footer
+      ByteBuffer tailBuf = ByteBuffer.allocate(tailLength);
+      file.readFully(offset, tailBuf.array(), tailBuf.arrayOffset(), tailLength);
+      return OrcProto.StripeFooter.parseFrom(InStream.createCodedInputStream("footer",
+          Lists.<DiskRange>newArrayList(new BufferChunk(tailBuf, 0)),
+          tailLength, codec, bufferSize));
+    }
+
+    @Override
+    public DiskRangeList readFileData(
+        DiskRangeList range, long baseOffset, boolean doForceDirect) throws IOException {
+      return RecordReaderUtils.readDiskRanges(file, zcr, baseOffset, range, doForceDirect);
+    }
+
+    @Override
+    public void close() throws IOException {
+      if (pool != null) {
+        pool.clear();
+      }
+      // close both zcr and file
+      try (HadoopShims.ZeroCopyReaderShim myZcr = zcr) {
+        if (file != null) {
+          file.close();
+        }
+      }
+    }
+
+    @Override
+    public boolean isTrackingDiskRanges() {
+      return zcr != null;
+    }
+
+    @Override
+    public void releaseBuffer(ByteBuffer buffer) {
+      zcr.releaseBuffer(buffer);
+    }
+
+    @Override
+    public DataReader clone() {
+      return new DefaultDataReader(this);
+    }
+
+  }
+
+  public static DataReader createDefaultDataReader(DataReaderProperties properties) {
+    return new DefaultDataReader(properties);
+  }
+
+  public static boolean[] findPresentStreamsByColumn(
+      List<OrcProto.Stream> streamList, List<OrcProto.Type> types) {
+    boolean[] hasNull = new boolean[types.size()];
+    for(OrcProto.Stream stream: streamList) {
+      if (stream.hasKind() && (stream.getKind() == OrcProto.Stream.Kind.PRESENT)) {
+        hasNull[stream.getColumn()] = true;
+      }
+    }
+    return hasNull;
+  }
+
+  /**
+   * Does region A overlap region B? The end points are inclusive on both sides.
+   * @param leftA A's left point
+   * @param rightA A's right point
+   * @param leftB B's left point
+   * @param rightB B's right point
+   * @return Does region A overlap region B?
+   */
+  static boolean overlap(long leftA, long rightA, long leftB, long rightB) {
+    if (leftA <= leftB) {
+      return rightA >= leftB;
+    }
+    return rightB >= leftA;
+  }
+
+  public static void addEntireStreamToRanges(
+      long offset, long length, CreateHelper list, boolean doMergeBuffers) {
+    list.addOrMerge(offset, offset + length, doMergeBuffers, false);
+  }
+
+  public static void addRgFilteredStreamToRanges(OrcProto.Stream stream,
+      boolean[] includedRowGroups, boolean isCompressed, OrcProto.RowIndex index,
+      OrcProto.ColumnEncoding encoding, OrcProto.Type type, int compressionSize, boolean hasNull,
+      long offset, long length, CreateHelper list, boolean doMergeBuffers) {
+    for (int group = 0; group < includedRowGroups.length; ++group) {
+      if (!includedRowGroups[group]) continue;
+      int posn = getIndexPosition(
+          encoding.getKind(), type.getKind(), stream.getKind(), isCompressed, hasNull);
+      long start = index.getEntry(group).getPositions(posn);
+      final long nextGroupOffset;
+      boolean isLast = group == (includedRowGroups.length - 1);
+      nextGroupOffset = isLast ? length : index.getEntry(group + 1).getPositions(posn);
+
+      start += offset;
+      long end = offset + estimateRgEndOffset(
+          isCompressed, isLast, nextGroupOffset, length, compressionSize);
+      list.addOrMerge(start, end, doMergeBuffers, true);
+    }
+  }
+
+  public static long estimateRgEndOffset(boolean isCompressed, boolean isLast,
+      long nextGroupOffset, long streamLength, int bufferSize) {
+    // figure out the worst case last location
+    // if adjacent groups have the same compressed block offset then stretch the slop
+    // by factor of 2 to safely accommodate the next compression block.
+    // One for the current compression block and another for the next compression block.
+    long slop = isCompressed ? 2 * (OutStream.HEADER_SIZE + bufferSize) : WORST_UNCOMPRESSED_SLOP;
+    return isLast ? streamLength : Math.min(streamLength, nextGroupOffset + slop);
+  }
+
+  private static final int BYTE_STREAM_POSITIONS = 1;
+  private static final int RUN_LENGTH_BYTE_POSITIONS = BYTE_STREAM_POSITIONS + 1;
+  private static final int BITFIELD_POSITIONS = RUN_LENGTH_BYTE_POSITIONS + 1;
+  private static final int RUN_LENGTH_INT_POSITIONS = BYTE_STREAM_POSITIONS + 1;
+
+  /**
+   * Get the offset in the index positions for the column that the given
+   * stream starts.
+   * @param columnEncoding the encoding of the column
+   * @param columnType the type of the column
+   * @param streamType the kind of the stream
+   * @param isCompressed is the file compressed
+   * @param hasNulls does the column have a PRESENT stream?
+   * @return the number of positions that will be used for that stream
+   */
+  public static int getIndexPosition(OrcProto.ColumnEncoding.Kind columnEncoding,
+                              OrcProto.Type.Kind columnType,
+                              OrcProto.Stream.Kind streamType,
+                              boolean isCompressed,
+                              boolean hasNulls) {
+    if (streamType == OrcProto.Stream.Kind.PRESENT) {
+      return 0;
+    }
+    int compressionValue = isCompressed ? 1 : 0;
+    int base = hasNulls ? (BITFIELD_POSITIONS + compressionValue) : 0;
+    switch (columnType) {
+      case BOOLEAN:
+      case BYTE:
+      case SHORT:
+      case INT:
+      case LONG:
+      case FLOAT:
+      case DOUBLE:
+      case DATE:
+      case STRUCT:
+      case MAP:
+      case LIST:
+      case UNION:
+        return base;
+      case CHAR:
+      case VARCHAR:
+      case STRING:
+        if (columnEncoding == OrcProto.ColumnEncoding.Kind.DICTIONARY ||
+            columnEncoding == OrcProto.ColumnEncoding.Kind.DICTIONARY_V2) {
+          return base;
+        } else {
+          if (streamType == OrcProto.Stream.Kind.DATA) {
+            return base;
+          } else {
+            return base + BYTE_STREAM_POSITIONS + compressionValue;
+          }
+        }
+      case BINARY:
+        if (streamType == OrcProto.Stream.Kind.DATA) {
+          return base;
+        }
+        return base + BYTE_STREAM_POSITIONS + compressionValue;
+      case DECIMAL:
+        if (streamType == OrcProto.Stream.Kind.DATA) {
+          return base;
+        }
+        return base + BYTE_STREAM_POSITIONS + compressionValue;
+      case TIMESTAMP:
+        if (streamType == OrcProto.Stream.Kind.DATA) {
+          return base;
+        }
+        return base + RUN_LENGTH_INT_POSITIONS + compressionValue;
+      default:
+        throw new IllegalArgumentException("Unknown type " + columnType);
+    }
+  }
+
+  // for uncompressed streams, what is the most overlap with the following set
+  // of rows (long vint literal group).
+  static final int WORST_UNCOMPRESSED_SLOP = 2 + 8 * 512;
+
+  /**
+   * Is this stream part of a dictionary?
+   * @return is this part of a dictionary?
+   */
+  public static boolean isDictionary(OrcProto.Stream.Kind kind,
+                              OrcProto.ColumnEncoding encoding) {
+    assert kind != OrcProto.Stream.Kind.DICTIONARY_COUNT;
+    OrcProto.ColumnEncoding.Kind encodingKind = encoding.getKind();
+    return kind == OrcProto.Stream.Kind.DICTIONARY_DATA ||
+      (kind == OrcProto.Stream.Kind.LENGTH &&
+       (encodingKind == OrcProto.ColumnEncoding.Kind.DICTIONARY ||
+        encodingKind == OrcProto.ColumnEncoding.Kind.DICTIONARY_V2));
+  }
+
+  /**
+   * Build a string representation of a list of disk ranges.
+   * @param range ranges to stringify
+   * @return the resulting string
+   */
+  public static String stringifyDiskRanges(DiskRangeList range) {
+    StringBuilder buffer = new StringBuilder();
+    buffer.append("[");
+    boolean isFirst = true;
+    while (range != null) {
+      if (!isFirst) {
+        buffer.append(", {");
+      } else {
+        buffer.append("{");
+      }
+      isFirst = false;
+      buffer.append(range.toString());
+      buffer.append("}");
+      range = range.next;
+    }
+    buffer.append("]");
+    return buffer.toString();
+  }
+
+  /**
+   * Read the list of ranges from the file.
+   * @param file the file to read
+   * @param base the base of the stripe
+   * @param range the disk ranges within the stripe to read
+   * @return the bytes read for each disk range, which is the same length as
+   *    ranges
+   * @throws IOException
+   */
+  static DiskRangeList readDiskRanges(FSDataInputStream file,
+                                      HadoopShims.ZeroCopyReaderShim zcr,
+                                 long base,
+                                 DiskRangeList range,
+                                 boolean doForceDirect) throws IOException {
+    if (range == null) return null;
+    DiskRangeList prev = range.prev;
+    if (prev == null) {
+      prev = new MutateHelper(range);
+    }
+    while (range != null) {
+      if (range.hasData()) {
+        range = range.next;
+        continue;
+      }
+      int len = (int) (range.getEnd() - range.getOffset());
+      long off = range.getOffset();
+      if (zcr != null) {
+        file.seek(base + off);
+        boolean hasReplaced = false;
+        while (len > 0) {
+          ByteBuffer partial = zcr.readBuffer(len, false);
+          BufferChunk bc = new BufferChunk(partial, off);
+          if (!hasReplaced) {
+            range.replaceSelfWith(bc);
+            hasReplaced = true;
+          } else {
+            range.insertAfter(bc);
+          }
+          range = bc;
+          int read = partial.remaining();
+          len -= read;
+          off += read;
+        }
+      } else {
+        // Don't use HDFS ByteBuffer API because it has no readFully, and is buggy and pointless.
+        byte[] buffer = new byte[len];
+        file.readFully((base + off), buffer, 0, buffer.length);
+        ByteBuffer bb = null;
+        if (doForceDirect) {
+          bb = ByteBuffer.allocateDirect(len);
+          bb.put(buffer);
+          bb.position(0);
+          bb.limit(len);
+        } else {
+          bb = ByteBuffer.wrap(buffer);
+        }
+        range = range.replaceSelfWith(new BufferChunk(bb, range.getOffset()));
+      }
+      range = range.next;
+    }
+    return prev.next;
+  }
+
+
+  static List<DiskRange> getStreamBuffers(DiskRangeList range, long offset, long length) {
+    // This assumes sorted ranges (as do many other parts of ORC code.
+    ArrayList<DiskRange> buffers = new ArrayList<DiskRange>();
+    if (length == 0) return buffers;
+    long streamEnd = offset + length;
+    boolean inRange = false;
+    while (range != null) {
+      if (!inRange) {
+        if (range.getEnd() <= offset) {
+          range = range.next;
+          continue; // Skip until we are in range.
+        }
+        inRange = true;
+        if (range.getOffset() < offset) {
+          // Partial first buffer, add a slice of it.
+          buffers.add(range.sliceAndShift(offset, Math.min(streamEnd, range.getEnd()), -offset));
+          if (range.getEnd() >= streamEnd) break; // Partial first buffer is also partial last buffer.
+          range = range.next;
+          continue;
+        }
+      } else if (range.getOffset() >= streamEnd) {
+        break;
+      }
+      if (range.getEnd() > streamEnd) {
+        // Partial last buffer (may also be the first buffer), add a slice of it.
+        buffers.add(range.sliceAndShift(range.getOffset(), streamEnd, -offset));
+        break;
+      }
+      // Buffer that belongs entirely to one stream.
+      // TODO: ideally we would want to reuse the object and remove it from the list, but we cannot
+      //       because bufferChunks is also used by clearStreams for zcr. Create a useless dup.
+      buffers.add(range.sliceAndShift(range.getOffset(), range.getEnd(), -offset));
+      if (range.getEnd() == streamEnd) break;
+      range = range.next;
+    }
+    return buffers;
+  }
+
+  static HadoopShims.ZeroCopyReaderShim createZeroCopyShim(FSDataInputStream file,
+      CompressionCodec codec, ByteBufferAllocatorPool pool) throws IOException {
+    if ((codec == null || ((codec instanceof DirectDecompressionCodec)
+            && ((DirectDecompressionCodec) codec).isAvailable()))) {
+      /* codec is null or is available */
+      return SHIMS.getZeroCopyReader(file, pool);
+    }
+    return null;
+  }
+
+  // this is an implementation copied from ElasticByteBufferPool in hadoop-2,
+  // which lacks a clear()/clean() operation
+  public final static class ByteBufferAllocatorPool implements HadoopShims.ByteBufferPoolShim {
+    private static final class Key implements Comparable<Key> {
+      private final int capacity;
+      private final long insertionGeneration;
+
+      Key(int capacity, long insertionGeneration) {
+        this.capacity = capacity;
+        this.insertionGeneration = insertionGeneration;
+      }
+
+      @Override
+      public int compareTo(Key other) {
+        return ComparisonChain.start().compare(capacity, other.capacity)
+            .compare(insertionGeneration, other.insertionGeneration).result();
+      }
+
+      @Override
+      public boolean equals(Object rhs) {
+        if (rhs == null) {
+          return false;
+        }
+        try {
+          Key o = (Key) rhs;
+          return (compareTo(o) == 0);
+        } catch (ClassCastException e) {
+          return false;
+        }
+      }
+
+      @Override
+      public int hashCode() {
+        return new HashCodeBuilder().append(capacity).append(insertionGeneration)
+            .toHashCode();
+      }
+    }
+
+    private final TreeMap<Key, ByteBuffer> buffers = new TreeMap<Key, ByteBuffer>();
+
+    private final TreeMap<Key, ByteBuffer> directBuffers = new TreeMap<Key, ByteBuffer>();
+
+    private long currentGeneration = 0;
+
+    private final TreeMap<Key, ByteBuffer> getBufferTree(boolean direct) {
+      return direct ? directBuffers : buffers;
+    }
+
+    public void clear() {
+      buffers.clear();
+      directBuffers.clear();
+    }
+
+    @Override
+    public ByteBuffer getBuffer(boolean direct, int length) {
+      TreeMap<Key, ByteBuffer> tree = getBufferTree(direct);
+      Map.Entry<Key, ByteBuffer> entry = tree.ceilingEntry(new Key(length, 0));
+      if (entry == null) {
+        return direct ? ByteBuffer.allocateDirect(length) : ByteBuffer
+            .allocate(length);
+      }
+      tree.remove(entry.getKey());
+      return entry.getValue();
+    }
+
+    @Override
+    public void putBuffer(ByteBuffer buffer) {
+      TreeMap<Key, ByteBuffer> tree = getBufferTree(buffer.isDirect());
+      while (true) {
+        Key key = new Key(buffer.capacity(), currentGeneration++);
+        if (!tree.containsKey(key)) {
+          tree.put(key, buffer);
+          return;
+        }
+        // Buffers are indexed by (capacity, generation).
+        // If our key is not unique on the first try, we try again
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/orc/src/java/org/apache/orc/impl/SchemaEvolution.java
----------------------------------------------------------------------
diff --git a/orc/src/java/org/apache/orc/impl/SchemaEvolution.java b/orc/src/java/org/apache/orc/impl/SchemaEvolution.java
new file mode 100644
index 0000000..2c80aaa
--- /dev/null
+++ b/orc/src/java/org/apache/orc/impl/SchemaEvolution.java
@@ -0,0 +1,190 @@
+/**
+ * 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.orc.impl;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.orc.TypeDescription;
+
+/**
+ * Take the file types and the (optional) configuration column names/types and see if there
+ * has been schema evolution.
+ */
+public class SchemaEvolution {
+  private final Map<TypeDescription, TypeDescription> readerToFile;
+  private final boolean[] included;
+  private final TypeDescription readerSchema;
+  private static final Log LOG = LogFactory.getLog(SchemaEvolution.class);
+
+  public SchemaEvolution(TypeDescription readerSchema, boolean[] included) {
+    this.included = included;
+    readerToFile = null;
+    this.readerSchema = readerSchema;
+  }
+
+  public SchemaEvolution(TypeDescription fileSchema,
+                         TypeDescription readerSchema,
+                         boolean[] included) throws IOException {
+    readerToFile = new HashMap<>(readerSchema.getMaximumId() + 1);
+    this.included = included;
+    if (checkAcidSchema(fileSchema)) {
+      this.readerSchema = createEventSchema(readerSchema);
+    } else {
+      this.readerSchema = readerSchema;
+    }
+    buildMapping(fileSchema, this.readerSchema);
+  }
+
+  public TypeDescription getReaderSchema() {
+    return readerSchema;
+  }
+
+  public TypeDescription getFileType(TypeDescription readerType) {
+    TypeDescription result;
+    if (readerToFile == null) {
+      if (included == null || included[readerType.getId()]) {
+        result = readerType;
+      } else {
+        result = null;
+      }
+    } else {
+      result = readerToFile.get(readerType);
+    }
+    return result;
+  }
+
+  void buildMapping(TypeDescription fileType,
+                    TypeDescription readerType) throws IOException {
+    // if the column isn't included, don't map it
+    if (included != null && !included[readerType.getId()]) {
+      return;
+    }
+    boolean isOk = true;
+    // check the easy case first
+    if (fileType.getCategory() == readerType.getCategory()) {
+      switch (readerType.getCategory()) {
+        case BOOLEAN:
+        case BYTE:
+        case SHORT:
+        case INT:
+        case LONG:
+        case DOUBLE:
+        case FLOAT:
+        case STRING:
+        case TIMESTAMP:
+        case BINARY:
+        case DATE:
+          // these are always a match
+          break;
+        case CHAR:
+        case VARCHAR:
+          // HIVE-13648: Look at ORC data type conversion edge cases (CHAR, VARCHAR, DECIMAL)
+          isOk = fileType.getMaxLength() == readerType.getMaxLength();
+          break;
+        case DECIMAL:
+          // HIVE-13648: Look at ORC data type conversion edge cases (CHAR, VARCHAR, DECIMAL)
+          // TODO we don't enforce scale and precision checks, but probably should
+          break;
+        case UNION:
+        case MAP:
+        case LIST: {
+          // these must be an exact match
+          List<TypeDescription> fileChildren = fileType.getChildren();
+          List<TypeDescription> readerChildren = readerType.getChildren();
+          if (fileChildren.size() == readerChildren.size()) {
+            for(int i=0; i < fileChildren.size(); ++i) {
+              buildMapping(fileChildren.get(i), readerChildren.get(i));
+            }
+          } else {
+            isOk = false;
+          }
+          break;
+        }
+        case STRUCT: {
+          // allow either side to have fewer fields than the other
+          List<TypeDescription> fileChildren = fileType.getChildren();
+          List<TypeDescription> readerChildren = readerType.getChildren();
+          int jointSize = Math.min(fileChildren.size(), readerChildren.size());
+          for(int i=0; i < jointSize; ++i) {
+            buildMapping(fileChildren.get(i), readerChildren.get(i));
+          }
+          break;
+        }
+        default:
+          throw new IllegalArgumentException("Unknown type " + readerType);
+      }
+    } else {
+      /*
+       * Check for the few cases where will not convert....
+       */
+
+      isOk = ConvertTreeReaderFactory.canConvert(fileType, readerType);
+    }
+    if (isOk) {
+      readerToFile.put(readerType, fileType);
+    } else {
+      throw new IOException(
+          String.format(
+              "ORC does not support type conversion from file type %s (%d) to reader type %s (%d)",
+              fileType.toString(), fileType.getId(),
+              readerType.toString(), readerType.getId()));
+    }
+  }
+
+  private static boolean checkAcidSchema(TypeDescription type) {
+    if (type.getCategory().equals(TypeDescription.Category.STRUCT)) {
+      List<String> rootFields = type.getFieldNames();
+      if (acidEventFieldNames.equals(rootFields)) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * @param typeDescr
+   * @return ORC types for the ACID event based on the row's type description
+   */
+  public static TypeDescription createEventSchema(TypeDescription typeDescr) {
+    TypeDescription result = TypeDescription.createStruct()
+        .addField("operation", TypeDescription.createInt())
+        .addField("originalTransaction", TypeDescription.createLong())
+        .addField("bucket", TypeDescription.createInt())
+        .addField("rowId", TypeDescription.createLong())
+        .addField("currentTransaction", TypeDescription.createLong())
+        .addField("row", typeDescr.clone());
+    return result;
+  }
+
+  public static final List<String> acidEventFieldNames= new ArrayList<String>();
+  static {
+    acidEventFieldNames.add("operation");
+    acidEventFieldNames.add("originalTransaction");
+    acidEventFieldNames.add("bucket");
+    acidEventFieldNames.add("rowId");
+    acidEventFieldNames.add("currentTransaction");
+    acidEventFieldNames.add("row");
+  }
+}


[22/27] hive git commit: HIVE-11417. Move the ReaderImpl and RowReaderImpl to the ORC module, by making shims for the row by row reader. (omalley reviewed by prasanth_j)

Posted by om...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/orc/src/java/org/apache/orc/tools/FileDump.java
----------------------------------------------------------------------
diff --git a/orc/src/java/org/apache/orc/tools/FileDump.java b/orc/src/java/org/apache/orc/tools/FileDump.java
new file mode 100644
index 0000000..e32027f
--- /dev/null
+++ b/orc/src/java/org/apache/orc/tools/FileDump.java
@@ -0,0 +1,934 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.orc.tools;
+
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.io.PrintStream;
+import java.text.DecimalFormat;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.GnuParser;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.OptionBuilder;
+import org.apache.commons.cli.Options;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.ListColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.MapColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.StructColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.UnionColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.serde2.io.DateWritable;
+import org.apache.orc.BloomFilterIO;
+import org.apache.orc.ColumnStatistics;
+import org.apache.orc.CompressionKind;
+import org.apache.orc.OrcFile;
+import org.apache.orc.Reader;
+import org.apache.orc.RecordReader;
+import org.apache.orc.TypeDescription;
+import org.apache.orc.Writer;
+import org.apache.orc.impl.AcidStats;
+import org.apache.orc.impl.ColumnStatisticsImpl;
+import org.apache.orc.impl.OrcAcidUtils;
+import org.apache.orc.impl.OrcIndex;
+import org.apache.orc.OrcProto;
+import org.apache.orc.StripeInformation;
+import org.apache.orc.StripeStatistics;
+import org.apache.orc.impl.RecordReaderImpl;
+import org.codehaus.jettison.json.JSONException;
+import org.codehaus.jettison.json.JSONWriter;
+
+import com.google.common.base.Joiner;
+import com.google.common.base.Strings;
+import com.google.common.collect.Lists;
+
+/**
+ * A tool for printing out the file structure of ORC files.
+ */
+public final class FileDump {
+  public static final String UNKNOWN = "UNKNOWN";
+  public static final String SEPARATOR = Strings.repeat("_", 120) + "\n";
+  public static final int DEFAULT_BLOCK_SIZE = 256 * 1024 * 1024;
+  public static final String DEFAULT_BACKUP_PATH = System.getProperty("java.io.tmpdir");
+  public static final PathFilter HIDDEN_AND_SIDE_FILE_FILTER = new PathFilter() {
+    public boolean accept(Path p) {
+      String name = p.getName();
+      return !name.startsWith("_") && !name.startsWith(".") && !name.endsWith(
+          OrcAcidUtils.DELTA_SIDE_FILE_SUFFIX);
+    }
+  };
+
+  // not used
+  private FileDump() {
+  }
+
+  public static void main(String[] args) throws Exception {
+    Configuration conf = new Configuration();
+
+    List<Integer> rowIndexCols = null;
+    Options opts = createOptions();
+    CommandLine cli = new GnuParser().parse(opts, args);
+
+    if (cli.hasOption('h')) {
+      HelpFormatter formatter = new HelpFormatter();
+      formatter.printHelp("orcfiledump", opts);
+      return;
+    }
+
+    boolean dumpData = cli.hasOption('d');
+    boolean recover = cli.hasOption("recover");
+    boolean skipDump = cli.hasOption("skip-dump");
+    String backupPath = DEFAULT_BACKUP_PATH;
+    if (cli.hasOption("backup-path")) {
+      backupPath = cli.getOptionValue("backup-path");
+    }
+
+    if (cli.hasOption("r")) {
+      String[] colStrs = cli.getOptionValue("r").split(",");
+      rowIndexCols = new ArrayList<Integer>(colStrs.length);
+      for (String colStr : colStrs) {
+        rowIndexCols.add(Integer.parseInt(colStr));
+      }
+    }
+
+    boolean printTimeZone = cli.hasOption('t');
+    boolean jsonFormat = cli.hasOption('j');
+    String[] files = cli.getArgs();
+    if (files.length == 0) {
+      System.err.println("Error : ORC files are not specified");
+      return;
+    }
+
+    // if the specified path is directory, iterate through all files and print the file dump
+    List<String> filesInPath = Lists.newArrayList();
+    for (String filename : files) {
+      Path path = new Path(filename);
+      filesInPath.addAll(getAllFilesInPath(path, conf));
+    }
+
+    if (dumpData) {
+      printData(filesInPath, conf);
+    } else if (recover && skipDump) {
+      recoverFiles(filesInPath, conf, backupPath);
+    } else {
+      if (jsonFormat) {
+        boolean prettyPrint = cli.hasOption('p');
+        JsonFileDump.printJsonMetaData(filesInPath, conf, rowIndexCols, prettyPrint, printTimeZone);
+      } else {
+        printMetaData(filesInPath, conf, rowIndexCols, printTimeZone, recover, backupPath);
+      }
+    }
+  }
+
+  /**
+   * This method returns an ORC reader object if the specified file is readable. If the specified
+   * file has side file (_flush_length) file, then max footer offset will be read from the side
+   * file and orc reader will be created from that offset. Since both data file and side file
+   * use hflush() for flushing the data, there could be some inconsistencies and both files could be
+   * out-of-sync. Following are the cases under which null will be returned
+   *
+   * 1) If the file specified by path or its side file is still open for writes
+   * 2) If *_flush_length file does not return any footer offset
+   * 3) If *_flush_length returns a valid footer offset but the data file is not readable at that
+   *    position (incomplete data file)
+   * 4) If *_flush_length file length is not a multiple of 8, then reader will be created from
+   *    previous valid footer. If there is no such footer (file length > 0 and < 8), then null will
+   *    be returned
+   *
+   * Also, if this method detects any file corruption (mismatch between data file and side file)
+   * then it will add the corresponding file to the specified input list for corrupted files.
+   *
+   * In all other cases, where the file is readable this method will return a reader object.
+   *
+   * @param path - file to get reader for
+   * @param conf - configuration object
+   * @param corruptFiles - fills this list with all possible corrupted files
+   * @return - reader for the specified file or null
+   * @throws IOException
+   */
+  static Reader getReader(final Path path, final Configuration conf,
+      final List<String> corruptFiles) throws IOException {
+    FileSystem fs = path.getFileSystem(conf);
+    long dataFileLen = fs.getFileStatus(path).getLen();
+    System.err.println("Processing data file " + path + " [length: " + dataFileLen + "]");
+    Path sideFile = OrcAcidUtils.getSideFile(path);
+    final boolean sideFileExists = fs.exists(sideFile);
+    boolean openDataFile = false;
+    boolean openSideFile = false;
+    if (fs instanceof DistributedFileSystem) {
+      DistributedFileSystem dfs = (DistributedFileSystem) fs;
+      openDataFile = !dfs.isFileClosed(path);
+      openSideFile = sideFileExists && !dfs.isFileClosed(sideFile);
+    }
+
+    if (openDataFile || openSideFile) {
+      if (openDataFile && openSideFile) {
+        System.err.println("Unable to perform file dump as " + path + " and " + sideFile +
+            " are still open for writes.");
+      } else if (openSideFile) {
+        System.err.println("Unable to perform file dump as " + sideFile +
+            " is still open for writes.");
+      } else {
+        System.err.println("Unable to perform file dump as " + path +
+            " is still open for writes.");
+      }
+
+      return null;
+    }
+
+    Reader reader = null;
+    if (sideFileExists) {
+      final long maxLen = OrcAcidUtils.getLastFlushLength(fs, path);
+      final long sideFileLen = fs.getFileStatus(sideFile).getLen();
+      System.err.println("Found flush length file " + sideFile
+          + " [length: " + sideFileLen + ", maxFooterOffset: " + maxLen + "]");
+      // no offsets read from side file
+      if (maxLen == -1) {
+
+        // if data file is larger than last flush length, then additional data could be recovered
+        if (dataFileLen > maxLen) {
+          System.err.println("Data file has more data than max footer offset:" + maxLen +
+              ". Adding data file to recovery list.");
+          if (corruptFiles != null) {
+            corruptFiles.add(path.toUri().toString());
+          }
+        }
+        return null;
+      }
+
+      try {
+        reader = OrcFile.createReader(path, OrcFile.readerOptions(conf).maxLength(maxLen));
+
+        // if data file is larger than last flush length, then additional data could be recovered
+        if (dataFileLen > maxLen) {
+          System.err.println("Data file has more data than max footer offset:" + maxLen +
+              ". Adding data file to recovery list.");
+          if (corruptFiles != null) {
+            corruptFiles.add(path.toUri().toString());
+          }
+        }
+      } catch (Exception e) {
+        if (corruptFiles != null) {
+          corruptFiles.add(path.toUri().toString());
+        }
+        System.err.println("Unable to read data from max footer offset." +
+            " Adding data file to recovery list.");
+        return null;
+      }
+    } else {
+      reader = OrcFile.createReader(path, OrcFile.readerOptions(conf));
+    }
+
+    return reader;
+  }
+
+  public static Collection<String> getAllFilesInPath(final Path path,
+      final Configuration conf) throws IOException {
+    List<String> filesInPath = Lists.newArrayList();
+    FileSystem fs = path.getFileSystem(conf);
+    FileStatus fileStatus = fs.getFileStatus(path);
+    if (fileStatus.isDir()) {
+      FileStatus[] fileStatuses = fs.listStatus(path, HIDDEN_AND_SIDE_FILE_FILTER);
+      for (FileStatus fileInPath : fileStatuses) {
+        if (fileInPath.isDir()) {
+          filesInPath.addAll(getAllFilesInPath(fileInPath.getPath(), conf));
+        } else {
+          filesInPath.add(fileInPath.getPath().toString());
+        }
+      }
+    } else {
+      filesInPath.add(path.toString());
+    }
+
+    return filesInPath;
+  }
+
+  private static void printData(List<String> files,
+      Configuration conf) throws IOException,
+      JSONException {
+    for (String file : files) {
+      try {
+        Path path = new Path(file);
+        Reader reader = getReader(path, conf, Lists.<String>newArrayList());
+        if (reader == null) {
+          continue;
+        }
+        printJsonData(reader);
+        System.out.println(SEPARATOR);
+      } catch (Exception e) {
+        System.err.println("Unable to dump data for file: " + file);
+        continue;
+      }
+    }
+  }
+
+  private static void printMetaData(List<String> files, Configuration conf,
+      List<Integer> rowIndexCols, boolean printTimeZone, final boolean recover,
+      final String backupPath)
+      throws IOException {
+    List<String> corruptFiles = Lists.newArrayList();
+    for (String filename : files) {
+      printMetaDataImpl(filename, conf, rowIndexCols, printTimeZone, corruptFiles);
+      System.out.println(SEPARATOR);
+    }
+
+    if (!corruptFiles.isEmpty()) {
+      if (recover) {
+        recoverFiles(corruptFiles, conf, backupPath);
+      } else {
+        System.err.println(corruptFiles.size() + " file(s) are corrupted." +
+            " Run the following command to recover corrupted files.\n");
+        String fileNames = Joiner.on(" ").skipNulls().join(corruptFiles);
+        System.err.println("hive --orcfiledump --recover --skip-dump " + fileNames);
+        System.out.println(SEPARATOR);
+      }
+    }
+  }
+
+  private static void printMetaDataImpl(final String filename,
+      final Configuration conf, final List<Integer> rowIndexCols, final boolean printTimeZone,
+      final List<String> corruptFiles) throws IOException {
+    Path file = new Path(filename);
+    Reader reader = getReader(file, conf, corruptFiles);
+    // if we can create reader then footer is not corrupt and file will readable
+    if (reader == null) {
+      return;
+    }
+
+    System.out.println("Structure for " + filename);
+    System.out.println("File Version: " + reader.getFileVersion().getName() +
+        " with " + reader.getWriterVersion());
+    RecordReaderImpl rows = (RecordReaderImpl) reader.rows();
+    System.out.println("Rows: " + reader.getNumberOfRows());
+    System.out.println("Compression: " + reader.getCompressionKind());
+    if (reader.getCompressionKind() != CompressionKind.NONE) {
+      System.out.println("Compression size: " + reader.getCompressionSize());
+    }
+    System.out.println("Type: " + reader.getSchema().toString());
+    System.out.println("\nStripe Statistics:");
+    List<StripeStatistics> stripeStats = reader.getStripeStatistics();
+    for (int n = 0; n < stripeStats.size(); n++) {
+      System.out.println("  Stripe " + (n + 1) + ":");
+      StripeStatistics ss = stripeStats.get(n);
+      for (int i = 0; i < ss.getColumnStatistics().length; ++i) {
+        System.out.println("    Column " + i + ": " +
+            ss.getColumnStatistics()[i].toString());
+      }
+    }
+    ColumnStatistics[] stats = reader.getStatistics();
+    int colCount = stats.length;
+    System.out.println("\nFile Statistics:");
+    for (int i = 0; i < stats.length; ++i) {
+      System.out.println("  Column " + i + ": " + stats[i].toString());
+    }
+    System.out.println("\nStripes:");
+    int stripeIx = -1;
+    for (StripeInformation stripe : reader.getStripes()) {
+      ++stripeIx;
+      long stripeStart = stripe.getOffset();
+      OrcProto.StripeFooter footer = rows.readStripeFooter(stripe);
+      if (printTimeZone) {
+        String tz = footer.getWriterTimezone();
+        if (tz == null || tz.isEmpty()) {
+          tz = UNKNOWN;
+        }
+        System.out.println("  Stripe: " + stripe.toString() + " timezone: " + tz);
+      } else {
+        System.out.println("  Stripe: " + stripe.toString());
+      }
+      long sectionStart = stripeStart;
+      for (OrcProto.Stream section : footer.getStreamsList()) {
+        String kind = section.hasKind() ? section.getKind().name() : UNKNOWN;
+        System.out.println("    Stream: column " + section.getColumn() +
+            " section " + kind + " start: " + sectionStart +
+            " length " + section.getLength());
+        sectionStart += section.getLength();
+      }
+      for (int i = 0; i < footer.getColumnsCount(); ++i) {
+        OrcProto.ColumnEncoding encoding = footer.getColumns(i);
+        StringBuilder buf = new StringBuilder();
+        buf.append("    Encoding column ");
+        buf.append(i);
+        buf.append(": ");
+        buf.append(encoding.getKind());
+        if (encoding.getKind() == OrcProto.ColumnEncoding.Kind.DICTIONARY ||
+            encoding.getKind() == OrcProto.ColumnEncoding.Kind.DICTIONARY_V2) {
+          buf.append("[");
+          buf.append(encoding.getDictionarySize());
+          buf.append("]");
+        }
+        System.out.println(buf);
+      }
+      if (rowIndexCols != null && !rowIndexCols.isEmpty()) {
+        // include the columns that are specified, only if the columns are included, bloom filter
+        // will be read
+        boolean[] sargColumns = new boolean[colCount];
+        for (int colIdx : rowIndexCols) {
+          sargColumns[colIdx] = true;
+        }
+        OrcIndex indices = rows
+            .readRowIndex(stripeIx, null, null, null, sargColumns);
+        for (int col : rowIndexCols) {
+          StringBuilder buf = new StringBuilder();
+          String rowIdxString = getFormattedRowIndices(col, indices.getRowGroupIndex());
+          buf.append(rowIdxString);
+          String bloomFilString = getFormattedBloomFilters(col, indices.getBloomFilterIndex());
+          buf.append(bloomFilString);
+          System.out.println(buf);
+        }
+      }
+    }
+
+    FileSystem fs = file.getFileSystem(conf);
+    long fileLen = fs.getFileStatus(file).getLen();
+    long paddedBytes = getTotalPaddingSize(reader);
+    // empty ORC file is ~45 bytes. Assumption here is file length always >0
+    double percentPadding = ((double) paddedBytes / (double) fileLen) * 100;
+    DecimalFormat format = new DecimalFormat("##.##");
+    System.out.println("\nFile length: " + fileLen + " bytes");
+    System.out.println("Padding length: " + paddedBytes + " bytes");
+    System.out.println("Padding ratio: " + format.format(percentPadding) + "%");
+    AcidStats acidStats = OrcAcidUtils.parseAcidStats(reader);
+    if (acidStats != null) {
+      System.out.println("ACID stats:" + acidStats);
+    }
+    rows.close();
+  }
+
+  private static void recoverFiles(final List<String> corruptFiles, final Configuration conf,
+      final String backup)
+      throws IOException {
+    for (String corruptFile : corruptFiles) {
+      System.err.println("Recovering file " + corruptFile);
+      Path corruptPath = new Path(corruptFile);
+      FileSystem fs = corruptPath.getFileSystem(conf);
+      FSDataInputStream fdis = fs.open(corruptPath);
+      try {
+        long corruptFileLen = fs.getFileStatus(corruptPath).getLen();
+        long remaining = corruptFileLen;
+        List<Long> footerOffsets = Lists.newArrayList();
+
+        // start reading the data file form top to bottom and record the valid footers
+        while (remaining > 0) {
+          int toRead = (int) Math.min(DEFAULT_BLOCK_SIZE, remaining);
+          byte[] data = new byte[toRead];
+          long startPos = corruptFileLen - remaining;
+          fdis.readFully(startPos, data, 0, toRead);
+
+          // find all MAGIC string and see if the file is readable from there
+          int index = 0;
+          long nextFooterOffset;
+
+          while (index != -1) {
+            index = indexOf(data, OrcFile.MAGIC.getBytes(), index + 1);
+            if (index != -1) {
+              nextFooterOffset = startPos + index + OrcFile.MAGIC.length() + 1;
+              if (isReadable(corruptPath, conf, nextFooterOffset)) {
+                footerOffsets.add(nextFooterOffset);
+              }
+            }
+          }
+
+          System.err.println("Scanning for valid footers - startPos: " + startPos +
+              " toRead: " + toRead + " remaining: " + remaining);
+          remaining = remaining - toRead;
+        }
+
+        System.err.println("Readable footerOffsets: " + footerOffsets);
+        recoverFile(corruptPath, fs, conf, footerOffsets, backup);
+      } catch (Exception e) {
+        Path recoveryFile = getRecoveryFile(corruptPath);
+        if (fs.exists(recoveryFile)) {
+          fs.delete(recoveryFile, false);
+        }
+        System.err.println("Unable to recover file " + corruptFile);
+        e.printStackTrace();
+        System.err.println(SEPARATOR);
+        continue;
+      } finally {
+        fdis.close();
+      }
+      System.err.println(corruptFile + " recovered successfully!");
+      System.err.println(SEPARATOR);
+    }
+  }
+
+  private static void recoverFile(final Path corruptPath, final FileSystem fs,
+      final Configuration conf, final List<Long> footerOffsets, final String backup)
+      throws IOException {
+
+    // first recover the file to .recovered file and then once successful rename it to actual file
+    Path recoveredPath = getRecoveryFile(corruptPath);
+
+    // make sure that file does not exist
+    if (fs.exists(recoveredPath)) {
+      fs.delete(recoveredPath, false);
+    }
+
+    // if there are no valid footers, the file should still be readable so create an empty orc file
+    if (footerOffsets == null || footerOffsets.isEmpty()) {
+      System.err.println("No readable footers found. Creating empty orc file.");
+      TypeDescription schema = TypeDescription.createStruct();
+      Writer writer = OrcFile.createWriter(recoveredPath,
+          OrcFile.writerOptions(conf).setSchema(schema));
+      writer.close();
+    } else {
+      FSDataInputStream fdis = fs.open(corruptPath);
+      FileStatus fileStatus = fs.getFileStatus(corruptPath);
+      // read corrupt file and copy it to recovered file until last valid footer
+      FSDataOutputStream fdos = fs.create(recoveredPath, true,
+          conf.getInt("io.file.buffer.size", 4096),
+          fileStatus.getReplication(),
+          fileStatus.getBlockSize());
+      try {
+        long fileLen = footerOffsets.get(footerOffsets.size() - 1);
+        long remaining = fileLen;
+
+        while (remaining > 0) {
+          int toRead = (int) Math.min(DEFAULT_BLOCK_SIZE, remaining);
+          byte[] data = new byte[toRead];
+          long startPos = fileLen - remaining;
+          fdis.readFully(startPos, data, 0, toRead);
+          fdos.write(data);
+          System.err.println("Copying data to recovery file - startPos: " + startPos +
+              " toRead: " + toRead + " remaining: " + remaining);
+          remaining = remaining - toRead;
+        }
+      } catch (Exception e) {
+        fs.delete(recoveredPath, false);
+        throw new IOException(e);
+      } finally {
+        fdis.close();
+        fdos.close();
+      }
+    }
+
+    // validate the recovered file once again and start moving corrupt files to backup folder
+    if (isReadable(recoveredPath, conf, Long.MAX_VALUE)) {
+      Path backupDataPath;
+      String scheme = corruptPath.toUri().getScheme();
+      String authority = corruptPath.toUri().getAuthority();
+      String filePath = corruptPath.toUri().getPath();
+
+      // use the same filesystem as corrupt file if backup-path is not explicitly specified
+      if (backup.equals(DEFAULT_BACKUP_PATH)) {
+        backupDataPath = new Path(scheme, authority, DEFAULT_BACKUP_PATH + filePath);
+      } else {
+        backupDataPath = Path.mergePaths(new Path(backup), corruptPath);
+      }
+
+      // Move data file to backup path
+      moveFiles(fs, corruptPath, backupDataPath);
+
+      // Move side file to backup path
+      Path sideFilePath = OrcAcidUtils.getSideFile(corruptPath);
+      Path backupSideFilePath = new Path(backupDataPath.getParent(), sideFilePath.getName());
+      moveFiles(fs, sideFilePath, backupSideFilePath);
+
+      // finally move recovered file to actual file
+      moveFiles(fs, recoveredPath, corruptPath);
+
+      // we are done recovering, backing up and validating
+      System.err.println("Validation of recovered file successful!");
+    }
+  }
+
+  private static void moveFiles(final FileSystem fs, final Path src, final Path dest)
+      throws IOException {
+    try {
+      // create the dest directory if not exist
+      if (!fs.exists(dest.getParent())) {
+        fs.mkdirs(dest.getParent());
+      }
+
+      // if the destination file exists for some reason delete it
+      fs.delete(dest, false);
+
+      if (fs.rename(src, dest)) {
+        System.err.println("Moved " + src + " to " + dest);
+      } else {
+        throw new IOException("Unable to move " + src + " to " + dest);
+      }
+
+    } catch (Exception e) {
+      throw new IOException("Unable to move " + src + " to " + dest, e);
+    }
+  }
+
+  private static Path getRecoveryFile(final Path corruptPath) {
+    return new Path(corruptPath.getParent(), corruptPath.getName() + ".recovered");
+  }
+
+  private static boolean isReadable(final Path corruptPath, final Configuration conf,
+      final long maxLen) {
+    try {
+      OrcFile.createReader(corruptPath, OrcFile.readerOptions(conf).maxLength(maxLen));
+      return true;
+    } catch (Exception e) {
+      // ignore this exception as maxLen is unreadable
+      return false;
+    }
+  }
+
+  // search for byte pattern in another byte array
+  private static int indexOf(final byte[] data, final byte[] pattern, final int index) {
+    if (data == null || data.length == 0 || pattern == null || pattern.length == 0 ||
+        index > data.length || index < 0) {
+      return -1;
+    }
+
+    int j = 0;
+    for (int i = index; i < data.length; i++) {
+      if (pattern[j] == data[i]) {
+        j++;
+      } else {
+        j = 0;
+      }
+
+      if (j == pattern.length) {
+        return i - pattern.length + 1;
+      }
+    }
+
+    return -1;
+  }
+
+  private static String getFormattedBloomFilters(int col,
+      OrcProto.BloomFilterIndex[] bloomFilterIndex) {
+    StringBuilder buf = new StringBuilder();
+    BloomFilterIO stripeLevelBF = null;
+    if (bloomFilterIndex != null && bloomFilterIndex[col] != null) {
+      int idx = 0;
+      buf.append("\n    Bloom filters for column ").append(col).append(":");
+      for (OrcProto.BloomFilter bf : bloomFilterIndex[col].getBloomFilterList()) {
+        BloomFilterIO toMerge = new BloomFilterIO(bf);
+        buf.append("\n      Entry ").append(idx++).append(":").append(getBloomFilterStats(toMerge));
+        if (stripeLevelBF == null) {
+          stripeLevelBF = toMerge;
+        } else {
+          stripeLevelBF.merge(toMerge);
+        }
+      }
+      String bloomFilterStats = getBloomFilterStats(stripeLevelBF);
+      buf.append("\n      Stripe level merge:").append(bloomFilterStats);
+    }
+    return buf.toString();
+  }
+
+  private static String getBloomFilterStats(BloomFilterIO bf) {
+    StringBuilder sb = new StringBuilder();
+    int bitCount = bf.getBitSize();
+    int popCount = 0;
+    for (long l : bf.getBitSet()) {
+      popCount += Long.bitCount(l);
+    }
+    int k = bf.getNumHashFunctions();
+    float loadFactor = (float) popCount / (float) bitCount;
+    float expectedFpp = (float) Math.pow(loadFactor, k);
+    DecimalFormat df = new DecimalFormat("###.####");
+    sb.append(" numHashFunctions: ").append(k);
+    sb.append(" bitCount: ").append(bitCount);
+    sb.append(" popCount: ").append(popCount);
+    sb.append(" loadFactor: ").append(df.format(loadFactor));
+    sb.append(" expectedFpp: ").append(expectedFpp);
+    return sb.toString();
+  }
+
+  private static String getFormattedRowIndices(int col,
+                                               OrcProto.RowIndex[] rowGroupIndex) {
+    StringBuilder buf = new StringBuilder();
+    OrcProto.RowIndex index;
+    buf.append("    Row group indices for column ").append(col).append(":");
+    if (rowGroupIndex == null || (col >= rowGroupIndex.length) ||
+        ((index = rowGroupIndex[col]) == null)) {
+      buf.append(" not found\n");
+      return buf.toString();
+    }
+
+    for (int entryIx = 0; entryIx < index.getEntryCount(); ++entryIx) {
+      buf.append("\n      Entry ").append(entryIx).append(": ");
+      OrcProto.RowIndexEntry entry = index.getEntry(entryIx);
+      if (entry == null) {
+        buf.append("unknown\n");
+        continue;
+      }
+      OrcProto.ColumnStatistics colStats = entry.getStatistics();
+      if (colStats == null) {
+        buf.append("no stats at ");
+      } else {
+        ColumnStatistics cs = ColumnStatisticsImpl.deserialize(colStats);
+        buf.append(cs.toString());
+      }
+      buf.append(" positions: ");
+      for (int posIx = 0; posIx < entry.getPositionsCount(); ++posIx) {
+        if (posIx != 0) {
+          buf.append(",");
+        }
+        buf.append(entry.getPositions(posIx));
+      }
+    }
+    return buf.toString();
+  }
+
+  public static long getTotalPaddingSize(Reader reader) throws IOException {
+    long paddedBytes = 0;
+    List<StripeInformation> stripes = reader.getStripes();
+    for (int i = 1; i < stripes.size(); i++) {
+      long prevStripeOffset = stripes.get(i - 1).getOffset();
+      long prevStripeLen = stripes.get(i - 1).getLength();
+      paddedBytes += stripes.get(i).getOffset() - (prevStripeOffset + prevStripeLen);
+    }
+    return paddedBytes;
+  }
+
+  static Options createOptions() {
+    Options result = new Options();
+
+    // add -d and --data to print the rows
+    result.addOption(OptionBuilder
+        .withLongOpt("data")
+        .withDescription("Should the data be printed")
+        .create('d'));
+
+    // to avoid breaking unit tests (when run in different time zones) for file dump, printing
+    // of timezone is made optional
+    result.addOption(OptionBuilder
+        .withLongOpt("timezone")
+        .withDescription("Print writer's time zone")
+        .create('t'));
+
+    result.addOption(OptionBuilder
+        .withLongOpt("help")
+        .withDescription("print help message")
+        .create('h'));
+
+    result.addOption(OptionBuilder
+        .withLongOpt("rowindex")
+        .withArgName("comma separated list of column ids for which row index should be printed")
+        .withDescription("Dump stats for column number(s)")
+        .hasArg()
+        .create('r'));
+
+    result.addOption(OptionBuilder
+        .withLongOpt("json")
+        .withDescription("Print metadata in JSON format")
+        .create('j'));
+
+    result.addOption(OptionBuilder
+        .withLongOpt("pretty")
+        .withDescription("Pretty print json metadata output")
+        .create('p'));
+
+    result.addOption(OptionBuilder
+        .withLongOpt("recover")
+        .withDescription("recover corrupted orc files generated by streaming")
+        .create());
+
+    result.addOption(OptionBuilder
+        .withLongOpt("skip-dump")
+        .withDescription("used along with --recover to directly recover files without dumping")
+        .create());
+
+    result.addOption(OptionBuilder
+        .withLongOpt("backup-path")
+        .withDescription("specify a backup path to store the corrupted files (default: /tmp)")
+        .hasArg()
+        .create());
+    return result;
+  }
+
+  private static void printMap(JSONWriter writer,
+                               MapColumnVector vector,
+                               TypeDescription schema,
+                               int row) throws JSONException {
+    writer.array();
+    TypeDescription keyType = schema.getChildren().get(0);
+    TypeDescription valueType = schema.getChildren().get(1);
+    int offset = (int) vector.offsets[row];
+    for (int i = 0; i < vector.lengths[row]; ++i) {
+      writer.object();
+      writer.key("_key");
+      printValue(writer, vector.keys, keyType, offset + i);
+      writer.key("_value");
+      printValue(writer, vector.values, valueType, offset + i);
+      writer.endObject();
+    }
+    writer.endArray();
+  }
+
+  private static void printList(JSONWriter writer,
+                                ListColumnVector vector,
+                                TypeDescription schema,
+                                int row) throws JSONException {
+    writer.array();
+    int offset = (int) vector.offsets[row];
+    TypeDescription childType = schema.getChildren().get(0);
+    for (int i = 0; i < vector.lengths[row]; ++i) {
+      printValue(writer, vector.child, childType, offset + i);
+    }
+    writer.endArray();
+  }
+
+  private static void printUnion(JSONWriter writer,
+                                 UnionColumnVector vector,
+                                 TypeDescription schema,
+                                 int row) throws JSONException {
+    int tag = vector.tags[row];
+    printValue(writer, vector.fields[tag], schema.getChildren().get(tag), row);
+  }
+
+  static void printStruct(JSONWriter writer,
+                          StructColumnVector batch,
+                          TypeDescription schema,
+                          int row) throws JSONException {
+    writer.object();
+    List<String> fieldNames = schema.getFieldNames();
+    List<TypeDescription> fieldTypes = schema.getChildren();
+    for (int i = 0; i < fieldTypes.size(); ++i) {
+      writer.key(fieldNames.get(i));
+      printValue(writer, batch.fields[i], fieldTypes.get(i), row);
+    }
+    writer.endObject();
+  }
+
+  static void printBinary(JSONWriter writer, BytesColumnVector vector,
+                          int row) throws JSONException {
+    writer.array();
+    int offset = vector.start[row];
+    for(int i=0; i < vector.length[row]; ++i) {
+      writer.value(0xff & (int) vector.vector[row][offset + i]);
+    }
+    writer.endArray();
+  }
+  static void printValue(JSONWriter writer, ColumnVector vector,
+                         TypeDescription schema, int row) throws JSONException {
+    if (vector.isRepeating) {
+      row = 0;
+    }
+    if (vector.noNulls || !vector.isNull[row]) {
+      switch (schema.getCategory()) {
+        case BOOLEAN:
+          writer.value(((LongColumnVector) vector).vector[row] != 0);
+          break;
+        case BYTE:
+        case SHORT:
+        case INT:
+        case LONG:
+          writer.value(((LongColumnVector) vector).vector[row]);
+          break;
+        case FLOAT:
+        case DOUBLE:
+          writer.value(((DoubleColumnVector) vector).vector[row]);
+          break;
+        case STRING:
+        case CHAR:
+        case VARCHAR:
+          writer.value(((BytesColumnVector) vector).toString(row));
+          break;
+        case BINARY:
+          printBinary(writer, (BytesColumnVector) vector, row);
+          break;
+        case DECIMAL:
+          writer.value(((DecimalColumnVector) vector).vector[row].toString());
+          break;
+        case DATE:
+          writer.value(new DateWritable(
+              (int) ((LongColumnVector) vector).vector[row]).toString());
+          break;
+        case TIMESTAMP:
+          writer.value(((TimestampColumnVector) vector)
+              .asScratchTimestamp(row).toString());
+          break;
+        case LIST:
+          printList(writer, (ListColumnVector) vector, schema, row);
+          break;
+        case MAP:
+          printMap(writer, (MapColumnVector) vector, schema, row);
+          break;
+        case STRUCT:
+          printStruct(writer, (StructColumnVector) vector, schema, row);
+          break;
+        case UNION:
+          printUnion(writer, (UnionColumnVector) vector, schema, row);
+          break;
+        default:
+          throw new IllegalArgumentException("Unknown type " +
+              schema.toString());
+      }
+    } else {
+      writer.value(null);
+    }
+  }
+
+  static void printRow(JSONWriter writer,
+                       VectorizedRowBatch batch,
+                       TypeDescription schema,
+                       int row) throws JSONException {
+    if (schema.getCategory() == TypeDescription.Category.STRUCT) {
+      List<TypeDescription> fieldTypes = schema.getChildren();
+      List<String> fieldNames = schema.getFieldNames();
+      writer.object();
+      for (int c = 0; c < batch.cols.length; ++c) {
+        writer.key(fieldNames.get(c));
+        printValue(writer, batch.cols[c], fieldTypes.get(c), row);
+      }
+      writer.endObject();
+    } else {
+      printValue(writer, batch.cols[0], schema, row);
+    }
+  }
+
+  static void printJsonData(final Reader reader) throws IOException, JSONException {
+    PrintStream printStream = System.out;
+    OutputStreamWriter out = new OutputStreamWriter(printStream, "UTF-8");
+    RecordReader rows = reader.rows();
+    try {
+      TypeDescription schema = reader.getSchema();
+      VectorizedRowBatch batch = schema.createRowBatch();
+      while (rows.nextBatch(batch)) {
+        for(int r=0; r < batch.size; ++r) {
+          JSONWriter writer = new JSONWriter(out);
+          printRow(writer, batch, schema, r);
+          out.write("\n");
+          out.flush();
+          if (printStream.checkError()) {
+            throw new IOException("Error encountered when writing to stdout.");
+          }
+        }
+      }
+    } finally {
+      rows.close();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/orc/src/java/org/apache/orc/tools/JsonFileDump.java
----------------------------------------------------------------------
diff --git a/orc/src/java/org/apache/orc/tools/JsonFileDump.java b/orc/src/java/org/apache/orc/tools/JsonFileDump.java
new file mode 100644
index 0000000..75153a2
--- /dev/null
+++ b/orc/src/java/org/apache/orc/tools/JsonFileDump.java
@@ -0,0 +1,406 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.orc.tools;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.orc.CompressionKind;
+import org.apache.orc.Reader;
+import org.apache.orc.impl.AcidStats;
+import org.apache.orc.impl.OrcAcidUtils;
+import org.apache.orc.impl.RecordReaderImpl;
+import org.codehaus.jettison.json.JSONArray;
+import org.apache.orc.BloomFilterIO;
+import org.apache.orc.BinaryColumnStatistics;
+import org.apache.orc.BooleanColumnStatistics;
+import org.apache.orc.ColumnStatistics;
+import org.apache.orc.impl.ColumnStatisticsImpl;
+import org.apache.orc.DateColumnStatistics;
+import org.apache.orc.DecimalColumnStatistics;
+import org.apache.orc.DoubleColumnStatistics;
+import org.apache.orc.IntegerColumnStatistics;
+import org.apache.orc.impl.OrcIndex;
+import org.apache.orc.OrcProto;
+import org.apache.orc.StringColumnStatistics;
+import org.apache.orc.StripeInformation;
+import org.apache.orc.StripeStatistics;
+import org.apache.orc.TimestampColumnStatistics;
+import org.codehaus.jettison.json.JSONException;
+import org.codehaus.jettison.json.JSONObject;
+import org.codehaus.jettison.json.JSONStringer;
+import org.codehaus.jettison.json.JSONWriter;
+
+/**
+ * File dump tool with json formatted output.
+ */
+public class JsonFileDump {
+
+  public static void printJsonMetaData(List<String> files,
+      Configuration conf,
+      List<Integer> rowIndexCols, boolean prettyPrint, boolean printTimeZone)
+      throws JSONException, IOException {
+    if (files.isEmpty()) {
+      return;
+    }
+    JSONStringer writer = new JSONStringer();
+    boolean multiFile = files.size() > 1;
+    if (multiFile) {
+      writer.array();
+    } else {
+      writer.object();
+    }
+    for (String filename : files) {
+      try {
+        if (multiFile) {
+          writer.object();
+        }
+        writer.key("fileName").value(filename);
+        Path path = new Path(filename);
+        Reader reader = FileDump.getReader(path, conf, null);
+        if (reader == null) {
+          writer.key("status").value("FAILED");
+          continue;
+        }
+        writer.key("fileVersion").value(reader.getFileVersion().getName());
+        writer.key("writerVersion").value(reader.getWriterVersion());
+        RecordReaderImpl rows = (RecordReaderImpl) reader.rows();
+        writer.key("numberOfRows").value(reader.getNumberOfRows());
+        writer.key("compression").value(reader.getCompressionKind());
+        if (reader.getCompressionKind() != CompressionKind.NONE) {
+          writer.key("compressionBufferSize").value(reader.getCompressionSize());
+        }
+        writer.key("schemaString").value(reader.getSchema().toString());
+        writer.key("schema").array();
+        writeSchema(writer, reader.getTypes());
+        writer.endArray();
+
+        writer.key("stripeStatistics").array();
+        List<StripeStatistics> stripeStatistics = reader.getStripeStatistics();
+        for (int n = 0; n < stripeStatistics.size(); n++) {
+          writer.object();
+          writer.key("stripeNumber").value(n + 1);
+          StripeStatistics ss = stripeStatistics.get(n);
+          writer.key("columnStatistics").array();
+          for (int i = 0; i < ss.getColumnStatistics().length; i++) {
+            writer.object();
+            writer.key("columnId").value(i);
+            writeColumnStatistics(writer, ss.getColumnStatistics()[i]);
+            writer.endObject();
+          }
+          writer.endArray();
+          writer.endObject();
+        }
+        writer.endArray();
+
+        ColumnStatistics[] stats = reader.getStatistics();
+        int colCount = stats.length;
+        writer.key("fileStatistics").array();
+        for (int i = 0; i < stats.length; ++i) {
+          writer.object();
+          writer.key("columnId").value(i);
+          writeColumnStatistics(writer, stats[i]);
+          writer.endObject();
+        }
+        writer.endArray();
+
+        writer.key("stripes").array();
+        int stripeIx = -1;
+        for (StripeInformation stripe : reader.getStripes()) {
+          ++stripeIx;
+          long stripeStart = stripe.getOffset();
+          OrcProto.StripeFooter footer = rows.readStripeFooter(stripe);
+          writer.object(); // start of stripe information
+          writer.key("stripeNumber").value(stripeIx + 1);
+          writer.key("stripeInformation");
+          writeStripeInformation(writer, stripe);
+          if (printTimeZone) {
+            writer.key("writerTimezone").value(
+                footer.hasWriterTimezone() ? footer.getWriterTimezone() : FileDump.UNKNOWN);
+          }
+          long sectionStart = stripeStart;
+
+          writer.key("streams").array();
+          for (OrcProto.Stream section : footer.getStreamsList()) {
+            writer.object();
+            String kind = section.hasKind() ? section.getKind().name() : FileDump.UNKNOWN;
+            writer.key("columnId").value(section.getColumn());
+            writer.key("section").value(kind);
+            writer.key("startOffset").value(sectionStart);
+            writer.key("length").value(section.getLength());
+            sectionStart += section.getLength();
+            writer.endObject();
+          }
+          writer.endArray();
+
+          writer.key("encodings").array();
+          for (int i = 0; i < footer.getColumnsCount(); ++i) {
+            writer.object();
+            OrcProto.ColumnEncoding encoding = footer.getColumns(i);
+            writer.key("columnId").value(i);
+            writer.key("kind").value(encoding.getKind());
+            if (encoding.getKind() == OrcProto.ColumnEncoding.Kind.DICTIONARY ||
+                encoding.getKind() == OrcProto.ColumnEncoding.Kind.DICTIONARY_V2) {
+              writer.key("dictionarySize").value(encoding.getDictionarySize());
+            }
+            writer.endObject();
+          }
+          writer.endArray();
+
+          if (rowIndexCols != null && !rowIndexCols.isEmpty()) {
+            // include the columns that are specified, only if the columns are included, bloom filter
+            // will be read
+            boolean[] sargColumns = new boolean[colCount];
+            for (int colIdx : rowIndexCols) {
+              sargColumns[colIdx] = true;
+            }
+            OrcIndex indices = rows.readRowIndex(stripeIx, null, sargColumns);
+            writer.key("indexes").array();
+            for (int col : rowIndexCols) {
+              writer.object();
+              writer.key("columnId").value(col);
+              writeRowGroupIndexes(writer, col, indices.getRowGroupIndex());
+              writeBloomFilterIndexes(writer, col, indices.getBloomFilterIndex());
+              writer.endObject();
+            }
+            writer.endArray();
+          }
+          writer.endObject(); // end of stripe information
+        }
+        writer.endArray();
+
+        FileSystem fs = path.getFileSystem(conf);
+        long fileLen = fs.getContentSummary(path).getLength();
+        long paddedBytes = FileDump.getTotalPaddingSize(reader);
+        // empty ORC file is ~45 bytes. Assumption here is file length always >0
+        double percentPadding = ((double) paddedBytes / (double) fileLen) * 100;
+        writer.key("fileLength").value(fileLen);
+        writer.key("paddingLength").value(paddedBytes);
+        writer.key("paddingRatio").value(percentPadding);
+        AcidStats acidStats = OrcAcidUtils.parseAcidStats(reader);
+        if (acidStats != null) {
+          writer.key("numInserts").value(acidStats.inserts);
+          writer.key("numDeletes").value(acidStats.deletes);
+          writer.key("numUpdates").value(acidStats.updates);
+        }
+        writer.key("status").value("OK");
+        rows.close();
+
+        writer.endObject();
+      } catch (Exception e) {
+        writer.key("status").value("FAILED");
+        throw e;
+      }
+    }
+    if (multiFile) {
+      writer.endArray();
+    }
+
+    if (prettyPrint) {
+      final String prettyJson;
+      if (multiFile) {
+        JSONArray jsonArray = new JSONArray(writer.toString());
+        prettyJson = jsonArray.toString(2);
+      } else {
+        JSONObject jsonObject = new JSONObject(writer.toString());
+        prettyJson = jsonObject.toString(2);
+      }
+      System.out.println(prettyJson);
+    } else {
+      System.out.println(writer.toString());
+    }
+  }
+
+  private static void writeSchema(JSONStringer writer, List<OrcProto.Type> types)
+      throws JSONException {
+    int i = 0;
+    for(OrcProto.Type type : types) {
+      writer.object();
+      writer.key("columnId").value(i++);
+      writer.key("columnType").value(type.getKind());
+      if (type.getFieldNamesCount() > 0) {
+        writer.key("childColumnNames").array();
+        for (String field : type.getFieldNamesList()) {
+          writer.value(field);
+        }
+        writer.endArray();
+        writer.key("childColumnIds").array();
+        for (Integer colId : type.getSubtypesList()) {
+          writer.value(colId);
+        }
+        writer.endArray();
+      }
+      if (type.hasPrecision()) {
+        writer.key("precision").value(type.getPrecision());
+      }
+
+      if (type.hasScale()) {
+        writer.key("scale").value(type.getScale());
+      }
+
+      if (type.hasMaximumLength()) {
+        writer.key("maxLength").value(type.getMaximumLength());
+      }
+      writer.endObject();
+    }
+  }
+
+  private static void writeStripeInformation(JSONWriter writer, StripeInformation stripe)
+      throws JSONException {
+    writer.object();
+    writer.key("offset").value(stripe.getOffset());
+    writer.key("indexLength").value(stripe.getIndexLength());
+    writer.key("dataLength").value(stripe.getDataLength());
+    writer.key("footerLength").value(stripe.getFooterLength());
+    writer.key("rowCount").value(stripe.getNumberOfRows());
+    writer.endObject();
+  }
+
+  private static void writeColumnStatistics(JSONWriter writer, ColumnStatistics cs)
+      throws JSONException {
+    if (cs != null) {
+      writer.key("count").value(cs.getNumberOfValues());
+      writer.key("hasNull").value(cs.hasNull());
+      if (cs instanceof BinaryColumnStatistics) {
+        writer.key("totalLength").value(((BinaryColumnStatistics) cs).getSum());
+        writer.key("type").value(OrcProto.Type.Kind.BINARY);
+      } else if (cs instanceof BooleanColumnStatistics) {
+        writer.key("trueCount").value(((BooleanColumnStatistics) cs).getTrueCount());
+        writer.key("falseCount").value(((BooleanColumnStatistics) cs).getFalseCount());
+        writer.key("type").value(OrcProto.Type.Kind.BOOLEAN);
+      } else if (cs instanceof IntegerColumnStatistics) {
+        writer.key("min").value(((IntegerColumnStatistics) cs).getMinimum());
+        writer.key("max").value(((IntegerColumnStatistics) cs).getMaximum());
+        if (((IntegerColumnStatistics) cs).isSumDefined()) {
+          writer.key("sum").value(((IntegerColumnStatistics) cs).getSum());
+        }
+        writer.key("type").value(OrcProto.Type.Kind.LONG);
+      } else if (cs instanceof DoubleColumnStatistics) {
+        writer.key("min").value(((DoubleColumnStatistics) cs).getMinimum());
+        writer.key("max").value(((DoubleColumnStatistics) cs).getMaximum());
+        writer.key("sum").value(((DoubleColumnStatistics) cs).getSum());
+        writer.key("type").value(OrcProto.Type.Kind.DOUBLE);
+      } else if (cs instanceof StringColumnStatistics) {
+        writer.key("min").value(((StringColumnStatistics) cs).getMinimum());
+        writer.key("max").value(((StringColumnStatistics) cs).getMaximum());
+        writer.key("totalLength").value(((StringColumnStatistics) cs).getSum());
+        writer.key("type").value(OrcProto.Type.Kind.STRING);
+      } else if (cs instanceof DateColumnStatistics) {
+        if (((DateColumnStatistics) cs).getMaximum() != null) {
+          writer.key("min").value(((DateColumnStatistics) cs).getMinimum());
+          writer.key("max").value(((DateColumnStatistics) cs).getMaximum());
+        }
+        writer.key("type").value(OrcProto.Type.Kind.DATE);
+      } else if (cs instanceof TimestampColumnStatistics) {
+        if (((TimestampColumnStatistics) cs).getMaximum() != null) {
+          writer.key("min").value(((TimestampColumnStatistics) cs).getMinimum());
+          writer.key("max").value(((TimestampColumnStatistics) cs).getMaximum());
+        }
+        writer.key("type").value(OrcProto.Type.Kind.TIMESTAMP);
+      } else if (cs instanceof DecimalColumnStatistics) {
+        if (((DecimalColumnStatistics) cs).getMaximum() != null) {
+          writer.key("min").value(((DecimalColumnStatistics) cs).getMinimum());
+          writer.key("max").value(((DecimalColumnStatistics) cs).getMaximum());
+          writer.key("sum").value(((DecimalColumnStatistics) cs).getSum());
+        }
+        writer.key("type").value(OrcProto.Type.Kind.DECIMAL);
+      }
+    }
+  }
+
+  private static void writeBloomFilterIndexes(JSONWriter writer, int col,
+      OrcProto.BloomFilterIndex[] bloomFilterIndex) throws JSONException {
+
+    BloomFilterIO stripeLevelBF = null;
+    if (bloomFilterIndex != null && bloomFilterIndex[col] != null) {
+      int entryIx = 0;
+      writer.key("bloomFilterIndexes").array();
+      for (OrcProto.BloomFilter bf : bloomFilterIndex[col].getBloomFilterList()) {
+        writer.object();
+        writer.key("entryId").value(entryIx++);
+        BloomFilterIO toMerge = new BloomFilterIO(bf);
+        writeBloomFilterStats(writer, toMerge);
+        if (stripeLevelBF == null) {
+          stripeLevelBF = toMerge;
+        } else {
+          stripeLevelBF.merge(toMerge);
+        }
+        writer.endObject();
+      }
+      writer.endArray();
+    }
+    if (stripeLevelBF != null) {
+      writer.key("stripeLevelBloomFilter");
+      writer.object();
+      writeBloomFilterStats(writer, stripeLevelBF);
+      writer.endObject();
+    }
+  }
+
+  private static void writeBloomFilterStats(JSONWriter writer, BloomFilterIO bf)
+      throws JSONException {
+    int bitCount = bf.getBitSize();
+    int popCount = 0;
+    for (long l : bf.getBitSet()) {
+      popCount += Long.bitCount(l);
+    }
+    int k = bf.getNumHashFunctions();
+    float loadFactor = (float) popCount / (float) bitCount;
+    float expectedFpp = (float) Math.pow(loadFactor, k);
+    writer.key("numHashFunctions").value(k);
+    writer.key("bitCount").value(bitCount);
+    writer.key("popCount").value(popCount);
+    writer.key("loadFactor").value(loadFactor);
+    writer.key("expectedFpp").value(expectedFpp);
+  }
+
+  private static void writeRowGroupIndexes(JSONWriter writer, int col,
+      OrcProto.RowIndex[] rowGroupIndex)
+      throws JSONException {
+
+    OrcProto.RowIndex index;
+    if (rowGroupIndex == null || (col >= rowGroupIndex.length) ||
+        ((index = rowGroupIndex[col]) == null)) {
+      return;
+    }
+
+    writer.key("rowGroupIndexes").array();
+    for (int entryIx = 0; entryIx < index.getEntryCount(); ++entryIx) {
+      writer.object();
+      writer.key("entryId").value(entryIx);
+      OrcProto.RowIndexEntry entry = index.getEntry(entryIx);
+      if (entry == null) {
+        continue;
+      }
+      OrcProto.ColumnStatistics colStats = entry.getStatistics();
+      writeColumnStatistics(writer, ColumnStatisticsImpl.deserialize(colStats));
+      writer.key("positions").array();
+      for (int posIx = 0; posIx < entry.getPositionsCount(); ++posIx) {
+        writer.value(entry.getPositions(posIx));
+      }
+      writer.endArray();
+      writer.endObject();
+    }
+    writer.endArray();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/orc/src/test/org/apache/orc/TestColumnStatistics.java
----------------------------------------------------------------------
diff --git a/orc/src/test/org/apache/orc/TestColumnStatistics.java b/orc/src/test/org/apache/orc/TestColumnStatistics.java
new file mode 100644
index 0000000..1837dbb
--- /dev/null
+++ b/orc/src/test/org/apache/orc/TestColumnStatistics.java
@@ -0,0 +1,364 @@
+/**
+ * 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.orc;
+
+import static junit.framework.Assert.assertEquals;
+import static org.junit.Assume.assumeTrue;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.PrintStream;
+import java.sql.Timestamp;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.common.type.HiveDecimal;
+import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.serde2.io.DateWritable;
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.orc.impl.ColumnStatisticsImpl;
+import org.apache.orc.tools.FileDump;
+import org.apache.orc.tools.TestFileDump;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+
+/**
+ * Test ColumnStatisticsImpl for ORC.
+ */
+public class TestColumnStatistics {
+
+  @Test
+  public void testLongMerge() throws Exception {
+    TypeDescription schema = TypeDescription.createInt();
+
+    ColumnStatisticsImpl stats1 = ColumnStatisticsImpl.create(schema);
+    ColumnStatisticsImpl stats2 = ColumnStatisticsImpl.create(schema);
+    stats1.updateInteger(10, 2);
+    stats2.updateInteger(1, 1);
+    stats2.updateInteger(1000, 1);
+    stats1.merge(stats2);
+    IntegerColumnStatistics typed = (IntegerColumnStatistics) stats1;
+    assertEquals(1, typed.getMinimum());
+    assertEquals(1000, typed.getMaximum());
+    stats1.reset();
+    stats1.updateInteger(-10, 1);
+    stats1.updateInteger(10000, 1);
+    stats1.merge(stats2);
+    assertEquals(-10, typed.getMinimum());
+    assertEquals(10000, typed.getMaximum());
+  }
+
+  @Test
+  public void testDoubleMerge() throws Exception {
+    TypeDescription schema = TypeDescription.createDouble();
+
+    ColumnStatisticsImpl stats1 = ColumnStatisticsImpl.create(schema);
+    ColumnStatisticsImpl stats2 = ColumnStatisticsImpl.create(schema);
+    stats1.updateDouble(10.0);
+    stats1.updateDouble(100.0);
+    stats2.updateDouble(1.0);
+    stats2.updateDouble(1000.0);
+    stats1.merge(stats2);
+    DoubleColumnStatistics typed = (DoubleColumnStatistics) stats1;
+    assertEquals(1.0, typed.getMinimum(), 0.001);
+    assertEquals(1000.0, typed.getMaximum(), 0.001);
+    stats1.reset();
+    stats1.updateDouble(-10);
+    stats1.updateDouble(10000);
+    stats1.merge(stats2);
+    assertEquals(-10, typed.getMinimum(), 0.001);
+    assertEquals(10000, typed.getMaximum(), 0.001);
+  }
+
+
+  @Test
+  public void testStringMerge() throws Exception {
+    TypeDescription schema = TypeDescription.createString();
+
+    ColumnStatisticsImpl stats1 = ColumnStatisticsImpl.create(schema);
+    ColumnStatisticsImpl stats2 = ColumnStatisticsImpl.create(schema);
+    stats1.updateString(new Text("bob"));
+    stats1.updateString(new Text("david"));
+    stats1.updateString(new Text("charles"));
+    stats2.updateString(new Text("anne"));
+    byte[] erin = new byte[]{0, 1, 2, 3, 4, 5, 101, 114, 105, 110};
+    stats2.updateString(erin, 6, 4, 5);
+    assertEquals(24, ((StringColumnStatistics)stats2).getSum());
+    stats1.merge(stats2);
+    StringColumnStatistics typed = (StringColumnStatistics) stats1;
+    assertEquals("anne", typed.getMinimum());
+    assertEquals("erin", typed.getMaximum());
+    assertEquals(39, typed.getSum());
+    stats1.reset();
+    stats1.updateString(new Text("aaa"));
+    stats1.updateString(new Text("zzz"));
+    stats1.merge(stats2);
+    assertEquals("aaa", typed.getMinimum());
+    assertEquals("zzz", typed.getMaximum());
+  }
+
+  @Test
+  public void testDateMerge() throws Exception {
+    TypeDescription schema = TypeDescription.createDate();
+
+    ColumnStatisticsImpl stats1 = ColumnStatisticsImpl.create(schema);
+    ColumnStatisticsImpl stats2 = ColumnStatisticsImpl.create(schema);
+    stats1.updateDate(new DateWritable(1000));
+    stats1.updateDate(new DateWritable(100));
+    stats2.updateDate(new DateWritable(10));
+    stats2.updateDate(new DateWritable(2000));
+    stats1.merge(stats2);
+    DateColumnStatistics typed = (DateColumnStatistics) stats1;
+    assertEquals(new DateWritable(10).get(), typed.getMinimum());
+    assertEquals(new DateWritable(2000).get(), typed.getMaximum());
+    stats1.reset();
+    stats1.updateDate(new DateWritable(-10));
+    stats1.updateDate(new DateWritable(10000));
+    stats1.merge(stats2);
+    assertEquals(new DateWritable(-10).get(), typed.getMinimum());
+    assertEquals(new DateWritable(10000).get(), typed.getMaximum());
+  }
+
+  @Test
+  public void testTimestampMerge() throws Exception {
+    TypeDescription schema = TypeDescription.createTimestamp();
+
+    ColumnStatisticsImpl stats1 = ColumnStatisticsImpl.create(schema);
+    ColumnStatisticsImpl stats2 = ColumnStatisticsImpl.create(schema);
+    stats1.updateTimestamp(new Timestamp(10));
+    stats1.updateTimestamp(new Timestamp(100));
+    stats2.updateTimestamp(new Timestamp(1));
+    stats2.updateTimestamp(new Timestamp(1000));
+    stats1.merge(stats2);
+    TimestampColumnStatistics typed = (TimestampColumnStatistics) stats1;
+    assertEquals(1, typed.getMinimum().getTime());
+    assertEquals(1000, typed.getMaximum().getTime());
+    stats1.reset();
+    stats1.updateTimestamp(new Timestamp(-10));
+    stats1.updateTimestamp(new Timestamp(10000));
+    stats1.merge(stats2);
+    assertEquals(-10, typed.getMinimum().getTime());
+    assertEquals(10000, typed.getMaximum().getTime());
+  }
+
+  @Test
+  public void testDecimalMerge() throws Exception {
+    TypeDescription schema = TypeDescription.createDecimal()
+        .withPrecision(38).withScale(16);
+
+    ColumnStatisticsImpl stats1 = ColumnStatisticsImpl.create(schema);
+    ColumnStatisticsImpl stats2 = ColumnStatisticsImpl.create(schema);
+    stats1.updateDecimal(HiveDecimal.create(10));
+    stats1.updateDecimal(HiveDecimal.create(100));
+    stats2.updateDecimal(HiveDecimal.create(1));
+    stats2.updateDecimal(HiveDecimal.create(1000));
+    stats1.merge(stats2);
+    DecimalColumnStatistics typed = (DecimalColumnStatistics) stats1;
+    assertEquals(1, typed.getMinimum().longValue());
+    assertEquals(1000, typed.getMaximum().longValue());
+    stats1.reset();
+    stats1.updateDecimal(HiveDecimal.create(-10));
+    stats1.updateDecimal(HiveDecimal.create(10000));
+    stats1.merge(stats2);
+    assertEquals(-10, typed.getMinimum().longValue());
+    assertEquals(10000, typed.getMaximum().longValue());
+  }
+
+
+  Path workDir = new Path(System.getProperty("test.tmp.dir",
+      "target" + File.separator + "test" + File.separator + "tmp"));
+
+  Configuration conf;
+  FileSystem fs;
+  Path testFilePath;
+
+  @Rule
+  public TestName testCaseName = new TestName();
+
+  @Before
+  public void openFileSystem() throws Exception {
+    conf = new Configuration();
+    fs = FileSystem.getLocal(conf);
+    fs.setWorkingDirectory(workDir);
+    testFilePath = new Path("TestOrcFile." + testCaseName.getMethodName() + ".orc");
+    fs.delete(testFilePath, false);
+  }
+
+  private static BytesWritable bytes(int... items) {
+    BytesWritable result = new BytesWritable();
+    result.setSize(items.length);
+    for (int i = 0; i < items.length; ++i) {
+      result.getBytes()[i] = (byte) items[i];
+    }
+    return result;
+  }
+
+  void appendRow(VectorizedRowBatch batch, BytesWritable bytes,
+                 String str) {
+    int row = batch.size++;
+    if (bytes == null) {
+      batch.cols[0].noNulls = false;
+      batch.cols[0].isNull[row] = true;
+    } else {
+      ((BytesColumnVector) batch.cols[0]).setVal(row, bytes.getBytes(),
+          0, bytes.getLength());
+    }
+    if (str == null) {
+      batch.cols[1].noNulls = false;
+      batch.cols[1].isNull[row] = true;
+    } else {
+      ((BytesColumnVector) batch.cols[1]).setVal(row, str.getBytes());
+    }
+  }
+
+  @Test
+  public void testHasNull() throws Exception {
+    TypeDescription schema =
+        TypeDescription.createStruct()
+        .addField("bytes1", TypeDescription.createBinary())
+        .addField("string1", TypeDescription.createString());
+    Writer writer = OrcFile.createWriter(testFilePath,
+        OrcFile.writerOptions(conf)
+            .setSchema(schema)
+            .rowIndexStride(1000)
+            .stripeSize(10000)
+            .bufferSize(10000));
+    VectorizedRowBatch batch = schema.createRowBatch(5000);
+    // STRIPE 1
+    // RG1
+    for(int i=0; i<1000; i++) {
+      appendRow(batch, bytes(1, 2, 3), "RG1");
+    }
+    writer.addRowBatch(batch);
+    batch.reset();
+    // RG2
+    for(int i=0; i<1000; i++) {
+      appendRow(batch, bytes(1, 2, 3), null);
+    }
+    writer.addRowBatch(batch);
+    batch.reset();
+    // RG3
+    for(int i=0; i<1000; i++) {
+      appendRow(batch, bytes(1, 2, 3), "RG3");
+    }
+    writer.addRowBatch(batch);
+    batch.reset();
+    // RG4
+    for (int i = 0; i < 1000; i++) {
+      appendRow(batch, bytes(1,2,3), null);
+    }
+    writer.addRowBatch(batch);
+    batch.reset();
+    // RG5
+    for(int i=0; i<1000; i++) {
+      appendRow(batch, bytes(1, 2, 3), null);
+    }
+    writer.addRowBatch(batch);
+    batch.reset();
+    // STRIPE 2
+    for (int i = 0; i < 5000; i++) {
+      appendRow(batch, bytes(1,2,3), null);
+    }
+    writer.addRowBatch(batch);
+    batch.reset();
+    // STRIPE 3
+    for (int i = 0; i < 5000; i++) {
+      appendRow(batch, bytes(1,2,3), "STRIPE-3");
+    }
+    writer.addRowBatch(batch);
+    batch.reset();
+    // STRIPE 4
+    for (int i = 0; i < 5000; i++) {
+      appendRow(batch, bytes(1,2,3), null);
+    }
+    writer.addRowBatch(batch);
+    batch.reset();
+    writer.close();
+    Reader reader = OrcFile.createReader(testFilePath,
+        OrcFile.readerOptions(conf).filesystem(fs));
+
+    // check the file level stats
+    ColumnStatistics[] stats = reader.getStatistics();
+    assertEquals(20000, stats[0].getNumberOfValues());
+    assertEquals(20000, stats[1].getNumberOfValues());
+    assertEquals(7000, stats[2].getNumberOfValues());
+    assertEquals(false, stats[0].hasNull());
+    assertEquals(false, stats[1].hasNull());
+    assertEquals(true, stats[2].hasNull());
+
+    // check the stripe level stats
+    List<StripeStatistics> stripeStats = reader.getStripeStatistics();
+    // stripe 1 stats
+    StripeStatistics ss1 = stripeStats.get(0);
+    ColumnStatistics ss1_cs1 = ss1.getColumnStatistics()[0];
+    ColumnStatistics ss1_cs2 = ss1.getColumnStatistics()[1];
+    ColumnStatistics ss1_cs3 = ss1.getColumnStatistics()[2];
+    assertEquals(false, ss1_cs1.hasNull());
+    assertEquals(false, ss1_cs2.hasNull());
+    assertEquals(true, ss1_cs3.hasNull());
+
+    // stripe 2 stats
+    StripeStatistics ss2 = stripeStats.get(1);
+    ColumnStatistics ss2_cs1 = ss2.getColumnStatistics()[0];
+    ColumnStatistics ss2_cs2 = ss2.getColumnStatistics()[1];
+    ColumnStatistics ss2_cs3 = ss2.getColumnStatistics()[2];
+    assertEquals(false, ss2_cs1.hasNull());
+    assertEquals(false, ss2_cs2.hasNull());
+    assertEquals(true, ss2_cs3.hasNull());
+
+    // stripe 3 stats
+    StripeStatistics ss3 = stripeStats.get(2);
+    ColumnStatistics ss3_cs1 = ss3.getColumnStatistics()[0];
+    ColumnStatistics ss3_cs2 = ss3.getColumnStatistics()[1];
+    ColumnStatistics ss3_cs3 = ss3.getColumnStatistics()[2];
+    assertEquals(false, ss3_cs1.hasNull());
+    assertEquals(false, ss3_cs2.hasNull());
+    assertEquals(false, ss3_cs3.hasNull());
+
+    // stripe 4 stats
+    StripeStatistics ss4 = stripeStats.get(3);
+    ColumnStatistics ss4_cs1 = ss4.getColumnStatistics()[0];
+    ColumnStatistics ss4_cs2 = ss4.getColumnStatistics()[1];
+    ColumnStatistics ss4_cs3 = ss4.getColumnStatistics()[2];
+    assertEquals(false, ss4_cs1.hasNull());
+    assertEquals(false, ss4_cs2.hasNull());
+    assertEquals(true, ss4_cs3.hasNull());
+
+    // Test file dump
+    PrintStream origOut = System.out;
+    String outputFilename = "orc-file-has-null.out";
+    FileOutputStream myOut = new FileOutputStream(workDir + File.separator + outputFilename);
+
+    // replace stdout and run command
+    System.setOut(new PrintStream(myOut));
+    FileDump.main(new String[]{testFilePath.toString(), "--rowindex=2"});
+    System.out.flush();
+    System.setOut(origOut);
+    // If called with an expression evaluating to false, the test will halt
+    // and be ignored.
+    assumeTrue(!System.getProperty("os.name").startsWith("Windows"));
+    TestFileDump.checkOutput(outputFilename, workDir + File.separator + outputFilename);
+  }
+}


[06/27] hive git commit: HIVE-11417. Move the ReaderImpl and RowReaderImpl to the ORC module, by making shims for the row by row reader. (omalley reviewed by prasanth_j)

Posted by om...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestRecordReaderImpl.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestRecordReaderImpl.java b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestRecordReaderImpl.java
deleted file mode 100644
index 8731be0..0000000
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestRecordReaderImpl.java
+++ /dev/null
@@ -1,1678 +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.hive.ql.io.orc;
-
-import static junit.framework.Assert.assertEquals;
-import static org.hamcrest.core.Is.is;
-import static org.junit.Assert.*;
-import static org.mockito.Mockito.any;
-import static org.mockito.Mockito.atLeastOnce;
-import static org.mockito.Mockito.doThrow;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-
-import java.io.File;
-import java.io.IOException;
-import java.io.InputStream;
-import java.sql.Timestamp;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.PositionedReadable;
-import org.apache.hadoop.fs.Seekable;
-import org.apache.hadoop.hive.common.io.DiskRangeList;
-import org.apache.hadoop.hive.common.type.HiveDecimal;
-import org.apache.hive.common.util.HiveTestUtils;
-import org.apache.orc.BloomFilterIO;
-import org.apache.hadoop.hive.ql.io.orc.RecordReaderImpl.Location;
-import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
-import org.apache.hadoop.hive.ql.io.sarg.SearchArgument.TruthValue;
-import org.apache.hadoop.hive.ql.io.sarg.TestSearchArgumentImpl;
-import org.apache.hadoop.hive.serde2.io.DateWritable;
-import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
-import org.apache.hadoop.io.DataOutputBuffer;
-import org.apache.orc.ColumnStatistics;
-import org.apache.orc.DataReader;
-import org.apache.orc.StripeInformation;
-import org.apache.orc.TypeDescription;
-import org.apache.orc.impl.ColumnStatisticsImpl;
-import org.apache.orc.OrcProto;
-
-import org.junit.Test;
-import org.mockito.MockSettings;
-import org.mockito.Mockito;
-
-public class TestRecordReaderImpl {
-
-  // can add .verboseLogging() to cause Mockito to log invocations
-  private final MockSettings settings = Mockito.withSettings().verboseLogging();
-
-  static class BufferInStream
-      extends InputStream implements PositionedReadable, Seekable {
-    private final byte[] buffer;
-    private final int length;
-    private int position = 0;
-
-    BufferInStream(byte[] bytes, int length) {
-      this.buffer = bytes;
-      this.length = length;
-    }
-
-    @Override
-    public int read() {
-      if (position < length) {
-        return buffer[position++];
-      }
-      return -1;
-    }
-
-    @Override
-    public int read(byte[] bytes, int offset, int length) {
-      int lengthToRead = Math.min(length, this.length - this.position);
-      if (lengthToRead >= 0) {
-        for(int i=0; i < lengthToRead; ++i) {
-          bytes[offset + i] = buffer[position++];
-        }
-        return lengthToRead;
-      } else {
-        return -1;
-      }
-    }
-
-    @Override
-    public int read(long position, byte[] bytes, int offset, int length) {
-      this.position = (int) position;
-      return read(bytes, offset, length);
-    }
-
-    @Override
-    public void readFully(long position, byte[] bytes, int offset,
-                          int length) throws IOException {
-      this.position = (int) position;
-      while (length > 0) {
-        int result = read(bytes, offset, length);
-        offset += result;
-        length -= result;
-        if (result < 0) {
-          throw new IOException("Read past end of buffer at " + offset);
-        }
-      }
-    }
-
-    @Override
-    public void readFully(long position, byte[] bytes) throws IOException {
-      readFully(position, bytes, 0, bytes.length);
-    }
-
-    @Override
-    public void seek(long position) {
-      this.position = (int) position;
-    }
-
-    @Override
-    public long getPos() {
-      return position;
-    }
-
-    @Override
-    public boolean seekToNewSource(long position) throws IOException {
-      this.position = (int) position;
-      return false;
-    }
-  }
-
-  @Test
-  public void testMaxLengthToReader() throws Exception {
-    Configuration conf = new Configuration();
-    OrcProto.Type rowType = OrcProto.Type.newBuilder()
-        .setKind(OrcProto.Type.Kind.STRUCT).build();
-    OrcProto.Footer footer = OrcProto.Footer.newBuilder()
-        .setHeaderLength(0).setContentLength(0).setNumberOfRows(0)
-        .setRowIndexStride(0).addTypes(rowType).build();
-    OrcProto.PostScript ps = OrcProto.PostScript.newBuilder()
-        .setCompression(OrcProto.CompressionKind.NONE)
-        .setFooterLength(footer.getSerializedSize())
-        .setMagic("ORC").addVersion(0).addVersion(11).build();
-    DataOutputBuffer buffer = new DataOutputBuffer();
-    footer.writeTo(buffer);
-    ps.writeTo(buffer);
-    buffer.write(ps.getSerializedSize());
-    FileSystem fs = mock(FileSystem.class, settings);
-    FSDataInputStream file =
-        new FSDataInputStream(new BufferInStream(buffer.getData(),
-            buffer.getLength()));
-    Path p = new Path("/dir/file.orc");
-    when(fs.open(p)).thenReturn(file);
-    OrcFile.ReaderOptions options = OrcFile.readerOptions(conf);
-    options.filesystem(fs);
-    options.maxLength(buffer.getLength());
-    when(fs.getFileStatus(p))
-        .thenReturn(new FileStatus(10, false, 3, 3000, 0, p));
-    Reader reader = OrcFile.createReader(p, options);
-  }
-
-  @Test
-  public void testCompareToRangeInt() throws Exception {
-    assertEquals(Location.BEFORE,
-      RecordReaderImpl.compareToRange(19L, 20L, 40L));
-    assertEquals(Location.AFTER,
-      RecordReaderImpl.compareToRange(41L, 20L, 40L));
-    assertEquals(Location.MIN,
-        RecordReaderImpl.compareToRange(20L, 20L, 40L));
-    assertEquals(Location.MIDDLE,
-        RecordReaderImpl.compareToRange(21L, 20L, 40L));
-    assertEquals(Location.MAX,
-      RecordReaderImpl.compareToRange(40L, 20L, 40L));
-    assertEquals(Location.BEFORE,
-      RecordReaderImpl.compareToRange(0L, 1L, 1L));
-    assertEquals(Location.MIN,
-      RecordReaderImpl.compareToRange(1L, 1L, 1L));
-    assertEquals(Location.AFTER,
-      RecordReaderImpl.compareToRange(2L, 1L, 1L));
-  }
-
-  @Test
-  public void testCompareToRangeString() throws Exception {
-    assertEquals(Location.BEFORE,
-        RecordReaderImpl.compareToRange("a", "b", "c"));
-    assertEquals(Location.AFTER,
-        RecordReaderImpl.compareToRange("d", "b", "c"));
-    assertEquals(Location.MIN,
-        RecordReaderImpl.compareToRange("b", "b", "c"));
-    assertEquals(Location.MIDDLE,
-        RecordReaderImpl.compareToRange("bb", "b", "c"));
-    assertEquals(Location.MAX,
-        RecordReaderImpl.compareToRange("c", "b", "c"));
-    assertEquals(Location.BEFORE,
-        RecordReaderImpl.compareToRange("a", "b", "b"));
-    assertEquals(Location.MIN,
-        RecordReaderImpl.compareToRange("b", "b", "b"));
-    assertEquals(Location.AFTER,
-        RecordReaderImpl.compareToRange("c", "b", "b"));
-  }
-
-  @Test
-  public void testCompareToCharNeedConvert() throws Exception {
-    assertEquals(Location.BEFORE,
-      RecordReaderImpl.compareToRange("apple", "hello", "world"));
-    assertEquals(Location.AFTER,
-      RecordReaderImpl.compareToRange("zombie", "hello", "world"));
-    assertEquals(Location.MIN,
-        RecordReaderImpl.compareToRange("hello", "hello", "world"));
-    assertEquals(Location.MIDDLE,
-        RecordReaderImpl.compareToRange("pilot", "hello", "world"));
-    assertEquals(Location.MAX,
-      RecordReaderImpl.compareToRange("world", "hello", "world"));
-    assertEquals(Location.BEFORE,
-      RecordReaderImpl.compareToRange("apple", "hello", "hello"));
-    assertEquals(Location.MIN,
-      RecordReaderImpl.compareToRange("hello", "hello", "hello"));
-    assertEquals(Location.AFTER,
-      RecordReaderImpl.compareToRange("zombie", "hello", "hello"));
-  }
-
-  @Test
-  public void testGetMin() throws Exception {
-    assertEquals(10L, RecordReaderImpl.getMin(
-      ColumnStatisticsImpl.deserialize(createIntStats(10L, 100L))));
-    assertEquals(10.0d, RecordReaderImpl.getMin(ColumnStatisticsImpl.deserialize(
-      OrcProto.ColumnStatistics.newBuilder()
-        .setDoubleStatistics(OrcProto.DoubleStatistics.newBuilder()
-          .setMinimum(10.0d).setMaximum(100.0d).build()).build())));
-    assertEquals(null, RecordReaderImpl.getMin(ColumnStatisticsImpl.deserialize(
-      OrcProto.ColumnStatistics.newBuilder()
-        .setStringStatistics(OrcProto.StringStatistics.newBuilder().build())
-        .build())));
-    assertEquals("a", RecordReaderImpl.getMin(ColumnStatisticsImpl.deserialize(
-      OrcProto.ColumnStatistics.newBuilder()
-        .setStringStatistics(OrcProto.StringStatistics.newBuilder()
-          .setMinimum("a").setMaximum("b").build()).build())));
-    assertEquals("hello", RecordReaderImpl.getMin(ColumnStatisticsImpl
-      .deserialize(createStringStats("hello", "world"))));
-    assertEquals(HiveDecimal.create("111.1"), RecordReaderImpl.getMin(ColumnStatisticsImpl
-      .deserialize(createDecimalStats("111.1", "112.1"))));
-  }
-
-  private static OrcProto.ColumnStatistics createIntStats(Long min,
-                                                          Long max) {
-    OrcProto.IntegerStatistics.Builder intStats =
-        OrcProto.IntegerStatistics.newBuilder();
-    if (min != null) {
-      intStats.setMinimum(min);
-    }
-    if (max != null) {
-      intStats.setMaximum(max);
-    }
-    return OrcProto.ColumnStatistics.newBuilder()
-        .setIntStatistics(intStats.build()).build();
-  }
-
-  private static OrcProto.ColumnStatistics createBooleanStats(int n, int trueCount) {
-    OrcProto.BucketStatistics.Builder boolStats = OrcProto.BucketStatistics.newBuilder();
-    boolStats.addCount(trueCount);
-    return OrcProto.ColumnStatistics.newBuilder().setNumberOfValues(n).setBucketStatistics(
-      boolStats.build()).build();
-  }
-
-  private static OrcProto.ColumnStatistics createIntStats(int min, int max) {
-    OrcProto.IntegerStatistics.Builder intStats = OrcProto.IntegerStatistics.newBuilder();
-    intStats.setMinimum(min);
-    intStats.setMaximum(max);
-    return OrcProto.ColumnStatistics.newBuilder().setIntStatistics(intStats.build()).build();
-  }
-
-  private static OrcProto.ColumnStatistics createDoubleStats(double min, double max) {
-    OrcProto.DoubleStatistics.Builder dblStats = OrcProto.DoubleStatistics.newBuilder();
-    dblStats.setMinimum(min);
-    dblStats.setMaximum(max);
-    return OrcProto.ColumnStatistics.newBuilder().setDoubleStatistics(dblStats.build()).build();
-  }
-
-  private static OrcProto.ColumnStatistics createStringStats(String min, String max,
-      boolean hasNull) {
-    OrcProto.StringStatistics.Builder strStats = OrcProto.StringStatistics.newBuilder();
-    strStats.setMinimum(min);
-    strStats.setMaximum(max);
-    return OrcProto.ColumnStatistics.newBuilder().setStringStatistics(strStats.build())
-        .setHasNull(hasNull).build();
-  }
-
-  private static OrcProto.ColumnStatistics createStringStats(String min, String max) {
-    OrcProto.StringStatistics.Builder strStats = OrcProto.StringStatistics.newBuilder();
-    strStats.setMinimum(min);
-    strStats.setMaximum(max);
-    return OrcProto.ColumnStatistics.newBuilder().setStringStatistics(strStats.build()).build();
-  }
-
-  private static OrcProto.ColumnStatistics createDateStats(int min, int max) {
-    OrcProto.DateStatistics.Builder dateStats = OrcProto.DateStatistics.newBuilder();
-    dateStats.setMinimum(min);
-    dateStats.setMaximum(max);
-    return OrcProto.ColumnStatistics.newBuilder().setDateStatistics(dateStats.build()).build();
-  }
-
-  private static OrcProto.ColumnStatistics createTimestampStats(long min, long max) {
-    OrcProto.TimestampStatistics.Builder tsStats = OrcProto.TimestampStatistics.newBuilder();
-    tsStats.setMinimum(min);
-    tsStats.setMaximum(max);
-    return OrcProto.ColumnStatistics.newBuilder().setTimestampStatistics(tsStats.build()).build();
-  }
-
-  private static OrcProto.ColumnStatistics createDecimalStats(String min, String max) {
-    OrcProto.DecimalStatistics.Builder decStats = OrcProto.DecimalStatistics.newBuilder();
-    decStats.setMinimum(min);
-    decStats.setMaximum(max);
-    return OrcProto.ColumnStatistics.newBuilder().setDecimalStatistics(decStats.build()).build();
-  }
-
-  private static OrcProto.ColumnStatistics createDecimalStats(String min, String max,
-      boolean hasNull) {
-    OrcProto.DecimalStatistics.Builder decStats = OrcProto.DecimalStatistics.newBuilder();
-    decStats.setMinimum(min);
-    decStats.setMaximum(max);
-    return OrcProto.ColumnStatistics.newBuilder().setDecimalStatistics(decStats.build())
-        .setHasNull(hasNull).build();
-  }
-
-  @Test
-  public void testGetMax() throws Exception {
-    assertEquals(100L, RecordReaderImpl.getMax(ColumnStatisticsImpl.deserialize(createIntStats(10L, 100L))));
-    assertEquals(100.0d, RecordReaderImpl.getMax(ColumnStatisticsImpl.deserialize(
-        OrcProto.ColumnStatistics.newBuilder()
-            .setDoubleStatistics(OrcProto.DoubleStatistics.newBuilder()
-                .setMinimum(10.0d).setMaximum(100.0d).build()).build())));
-    assertEquals(null, RecordReaderImpl.getMax(ColumnStatisticsImpl.deserialize(
-        OrcProto.ColumnStatistics.newBuilder()
-            .setStringStatistics(OrcProto.StringStatistics.newBuilder().build())
-            .build())));
-    assertEquals("b", RecordReaderImpl.getMax(ColumnStatisticsImpl.deserialize(
-        OrcProto.ColumnStatistics.newBuilder()
-            .setStringStatistics(OrcProto.StringStatistics.newBuilder()
-                .setMinimum("a").setMaximum("b").build()).build())));
-    assertEquals("world", RecordReaderImpl.getMax(ColumnStatisticsImpl
-      .deserialize(createStringStats("hello", "world"))));
-    assertEquals(HiveDecimal.create("112.1"), RecordReaderImpl.getMax(ColumnStatisticsImpl
-      .deserialize(createDecimalStats("111.1", "112.1"))));
-  }
-
-  @Test
-  public void testPredEvalWithBooleanStats() throws Exception {
-    PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf(
-        PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.BOOLEAN, "x", true, null);
-    assertEquals(TruthValue.YES_NO,
-        RecordReaderImpl.evaluatePredicateProto(createBooleanStats(10, 10), pred, null));
-    assertEquals(TruthValue.NO,
-        RecordReaderImpl.evaluatePredicateProto(createBooleanStats(10, 0), pred, null));
-
-    pred = TestSearchArgumentImpl.createPredicateLeaf(
-        PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.BOOLEAN, "x", true, null);
-    assertEquals(TruthValue.YES_NO,
-        RecordReaderImpl.evaluatePredicateProto(createBooleanStats(10, 10), pred, null));
-    assertEquals(TruthValue.NO,
-        RecordReaderImpl.evaluatePredicateProto(createBooleanStats(10, 0), pred, null));
-
-    pred = TestSearchArgumentImpl.createPredicateLeaf(
-        PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.BOOLEAN, "x", false, null);
-    assertEquals(TruthValue.NO,
-      RecordReaderImpl.evaluatePredicateProto(createBooleanStats(10, 10), pred, null));
-    assertEquals(TruthValue.YES_NO,
-      RecordReaderImpl.evaluatePredicateProto(createBooleanStats(10, 0), pred, null));
-  }
-
-  @Test
-  public void testPredEvalWithIntStats() throws Exception {
-    PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf(
-      PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.LONG, "x", 15L, null);
-    assertEquals(TruthValue.YES_NO,
-        RecordReaderImpl.evaluatePredicateProto(createIntStats(10, 100), pred, null));
-
-    pred = TestSearchArgumentImpl.createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
-        PredicateLeaf.Type.FLOAT, "x", 15.0, null);
-    assertEquals(TruthValue.YES_NO,
-        RecordReaderImpl.evaluatePredicateProto(createIntStats(10, 100), pred, null));
-
-    // Stats gets converted to column type. "15" is outside of "10" and "100"
-    pred = TestSearchArgumentImpl.createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
-        PredicateLeaf.Type.STRING, "x", "15", null);
-    assertEquals(TruthValue.NO,
-        RecordReaderImpl.evaluatePredicateProto(createIntStats(10, 100), pred, null));
-
-    // Integer stats will not be converted date because of days/seconds/millis ambiguity
-    pred = TestSearchArgumentImpl.createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
-        PredicateLeaf.Type.DATE, "x", new DateWritable(15).get(), null);
-    assertEquals(TruthValue.YES_NO,
-        RecordReaderImpl.evaluatePredicateProto(createIntStats(10, 100), pred, null));
-
-    pred = TestSearchArgumentImpl.createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
-        PredicateLeaf.Type.DECIMAL, "x", new HiveDecimalWritable("15"), null);
-    assertEquals(TruthValue.YES_NO,
-        RecordReaderImpl.evaluatePredicateProto(createIntStats(10, 100), pred, null));
-
-    pred = TestSearchArgumentImpl.createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
-        PredicateLeaf.Type.TIMESTAMP, "x", new Timestamp(15), null);
-    assertEquals(TruthValue.YES_NO,
-      RecordReaderImpl.evaluatePredicateProto(createIntStats(10, 100), pred, null));
-  }
-
-  @Test
-  public void testPredEvalWithDoubleStats() throws Exception {
-    PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf(
-        PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.LONG, "x", 15L, null);
-    assertEquals(TruthValue.YES_NO,
-        RecordReaderImpl.evaluatePredicateProto(createDoubleStats(10.0, 100.0), pred, null));
-
-    pred = TestSearchArgumentImpl.createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
-        PredicateLeaf.Type.FLOAT, "x", 15.0, null);
-    assertEquals(TruthValue.YES_NO,
-        RecordReaderImpl.evaluatePredicateProto(createDoubleStats(10.0, 100.0), pred, null));
-
-    // Stats gets converted to column type. "15.0" is outside of "10.0" and "100.0"
-    pred = TestSearchArgumentImpl.createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
-        PredicateLeaf.Type.STRING, "x", "15", null);
-    assertEquals(TruthValue.NO,
-        RecordReaderImpl.evaluatePredicateProto(createDoubleStats(10.0, 100.0), pred, null));
-
-    // Double is not converted to date type because of days/seconds/millis ambiguity
-    pred = TestSearchArgumentImpl.createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
-        PredicateLeaf.Type.DATE, "x", new DateWritable(15).get(), null);
-    assertEquals(TruthValue.YES_NO,
-        RecordReaderImpl.evaluatePredicateProto(createDoubleStats(10.0, 100.0), pred, null));
-
-    pred = TestSearchArgumentImpl.createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
-        PredicateLeaf.Type.DECIMAL, "x", new HiveDecimalWritable("15"), null);
-    assertEquals(TruthValue.YES_NO,
-        RecordReaderImpl.evaluatePredicateProto(createDoubleStats(10.0, 100.0), pred, null));
-
-    pred = TestSearchArgumentImpl.createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
-        PredicateLeaf.Type.TIMESTAMP, "x", new Timestamp(15*1000L), null);
-    assertEquals(TruthValue.YES_NO,
-        RecordReaderImpl.evaluatePredicateProto(createDoubleStats(10.0, 100.0), pred, null));
-
-    pred = TestSearchArgumentImpl.createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
-        PredicateLeaf.Type.TIMESTAMP, "x", new Timestamp(150*1000L), null);
-    assertEquals(TruthValue.NO,
-        RecordReaderImpl.evaluatePredicateProto(createDoubleStats(10.0, 100.0), pred, null));
-  }
-
-  @Test
-  public void testPredEvalWithStringStats() throws Exception {
-    PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf(
-        PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.LONG, "x", 100L, null);
-    assertEquals(TruthValue.YES_NO,
-        RecordReaderImpl.evaluatePredicateProto(createStringStats("10", "1000"), pred, null));
-
-    pred = TestSearchArgumentImpl.createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
-        PredicateLeaf.Type.FLOAT, "x", 100.0, null);
-    assertEquals(TruthValue.YES_NO,
-        RecordReaderImpl.evaluatePredicateProto(createStringStats("10", "1000"), pred, null));
-
-    pred = TestSearchArgumentImpl.createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
-        PredicateLeaf.Type.STRING, "x", "100", null);
-    assertEquals(TruthValue.YES_NO,
-        RecordReaderImpl.evaluatePredicateProto(createStringStats("10", "1000"), pred, null));
-
-    // IllegalArgumentException is thrown when converting String to Date, hence YES_NO
-    pred = TestSearchArgumentImpl.createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
-        PredicateLeaf.Type.DATE, "x", new DateWritable(100).get(), null);
-    assertEquals(TruthValue.YES_NO,
-        RecordReaderImpl.evaluatePredicateProto(createDateStats(10, 1000), pred, null));
-
-    pred = TestSearchArgumentImpl.createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
-        PredicateLeaf.Type.DECIMAL, "x", new HiveDecimalWritable("100"), null);
-    assertEquals(TruthValue.YES_NO,
-        RecordReaderImpl.evaluatePredicateProto(createStringStats("10", "1000"), pred, null));
-
-    pred = TestSearchArgumentImpl.createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
-        PredicateLeaf.Type.TIMESTAMP, "x", new Timestamp(100), null);
-    assertEquals(TruthValue.YES_NO,
-        RecordReaderImpl.evaluatePredicateProto(createStringStats("10", "1000"), pred, null));
-  }
-
-  @Test
-  public void testPredEvalWithDateStats() throws Exception {
-    PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf(
-        PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.LONG, "x", 15L, null);
-    // Date to Integer conversion is not possible.
-    assertEquals(TruthValue.YES_NO,
-        RecordReaderImpl.evaluatePredicateProto(createDateStats(10, 100), pred, null));
-
-    // Date to Float conversion is also not possible.
-    pred = TestSearchArgumentImpl.createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
-        PredicateLeaf.Type.FLOAT, "x", 15.0, null);
-    assertEquals(TruthValue.YES_NO,
-        RecordReaderImpl.evaluatePredicateProto(createDateStats(10, 100), pred, null));
-
-    pred = TestSearchArgumentImpl.createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
-        PredicateLeaf.Type.STRING, "x", "15", null);
-    assertEquals(TruthValue.NO,
-        RecordReaderImpl.evaluatePredicateProto(createDateStats(10, 100), pred, null));
-
-    pred = TestSearchArgumentImpl.createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
-        PredicateLeaf.Type.STRING, "x", "1970-01-11", null);
-    assertEquals(TruthValue.YES_NO,
-        RecordReaderImpl.evaluatePredicateProto(createDateStats(10, 100), pred, null));
-
-    pred = TestSearchArgumentImpl.createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
-        PredicateLeaf.Type.STRING, "x", "15.1", null);
-    assertEquals(TruthValue.NO,
-        RecordReaderImpl.evaluatePredicateProto(createDateStats(10, 100), pred, null));
-
-    pred = TestSearchArgumentImpl.createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
-        PredicateLeaf.Type.STRING, "x", "__a15__1", null);
-    assertEquals(TruthValue.NO,
-        RecordReaderImpl.evaluatePredicateProto(createDateStats(10, 100), pred, null));
-
-    pred = TestSearchArgumentImpl.createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
-        PredicateLeaf.Type.STRING, "x", "2000-01-16", null);
-    assertEquals(TruthValue.NO,
-        RecordReaderImpl.evaluatePredicateProto(createDateStats(10, 100), pred, null));
-
-    pred = TestSearchArgumentImpl.createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
-        PredicateLeaf.Type.STRING, "x", "1970-01-16", null);
-    assertEquals(TruthValue.YES_NO,
-        RecordReaderImpl.evaluatePredicateProto(createDateStats(10, 100), pred, null));
-
-    pred = TestSearchArgumentImpl.createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
-        PredicateLeaf.Type.DATE, "x", new DateWritable(15).get(), null);
-    assertEquals(TruthValue.YES_NO,
-        RecordReaderImpl.evaluatePredicateProto(createDateStats(10, 100), pred, null));
-
-    pred = TestSearchArgumentImpl.createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
-        PredicateLeaf.Type.DATE, "x", new DateWritable(150).get(), null);
-    assertEquals(TruthValue.NO,
-        RecordReaderImpl.evaluatePredicateProto(createDateStats(10, 100), pred, null));
-
-    // Date to Decimal conversion is also not possible.
-    pred = TestSearchArgumentImpl.createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
-        PredicateLeaf.Type.DECIMAL, "x", new HiveDecimalWritable("15"), null);
-    assertEquals(TruthValue.YES_NO,
-        RecordReaderImpl.evaluatePredicateProto(createDateStats(10, 100), pred, null));
-
-    pred = TestSearchArgumentImpl.createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
-        PredicateLeaf.Type.TIMESTAMP, "x", new Timestamp(15), null);
-    assertEquals(TruthValue.NO,
-        RecordReaderImpl.evaluatePredicateProto(createDateStats(10, 100), pred, null));
-
-    pred = TestSearchArgumentImpl.createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
-        PredicateLeaf.Type.TIMESTAMP, "x", new Timestamp(15L * 24L * 60L * 60L * 1000L), null);
-    assertEquals(TruthValue.YES_NO,
-        RecordReaderImpl.evaluatePredicateProto(createDateStats(10, 100), pred, null));
-  }
-
-  @Test
-  public void testPredEvalWithDecimalStats() throws Exception {
-    PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf(
-        PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.LONG, "x", 15L, null);
-    assertEquals(TruthValue.YES_NO,
-        RecordReaderImpl.evaluatePredicateProto(createDecimalStats("10.0", "100.0"), pred, null));
-
-    pred = TestSearchArgumentImpl.createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
-        PredicateLeaf.Type.FLOAT, "x", 15.0, null);
-    assertEquals(TruthValue.YES_NO,
-        RecordReaderImpl.evaluatePredicateProto(createDecimalStats("10.0", "100.0"), pred, null));
-
-    // "15" out of range of "10.0" and "100.0"
-    pred = TestSearchArgumentImpl.createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
-        PredicateLeaf.Type.STRING, "x", "15", null);
-    assertEquals(TruthValue.NO,
-        RecordReaderImpl.evaluatePredicateProto(createDecimalStats("10.0", "100.0"), pred, null));
-
-    // Decimal to Date not possible.
-    pred = TestSearchArgumentImpl.createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
-        PredicateLeaf.Type.DATE, "x", new DateWritable(15).get(), null);
-    assertEquals(TruthValue.YES_NO,
-        RecordReaderImpl.evaluatePredicateProto(createDecimalStats("10.0", "100.0"), pred, null));
-
-    pred = TestSearchArgumentImpl.createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
-        PredicateLeaf.Type.DECIMAL, "x", new HiveDecimalWritable("15"), null);
-    assertEquals(TruthValue.YES_NO,
-        RecordReaderImpl.evaluatePredicateProto(createDecimalStats("10.0", "100.0"), pred, null));
-
-    pred = TestSearchArgumentImpl.createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
-        PredicateLeaf.Type.TIMESTAMP, "x", new Timestamp(15 * 1000L), null);
-    assertEquals(TruthValue.YES_NO,
-        RecordReaderImpl.evaluatePredicateProto(createDecimalStats("10.0", "100.0"), pred, null));
-
-    pred = TestSearchArgumentImpl.createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
-        PredicateLeaf.Type.TIMESTAMP, "x", new Timestamp(150 * 1000L), null);
-    assertEquals(TruthValue.NO,
-        RecordReaderImpl.evaluatePredicateProto(createDecimalStats("10.0", "100.0"), pred, null));
-  }
-
-  @Test
-  public void testPredEvalWithTimestampStats() throws Exception {
-    PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf(
-        PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.LONG, "x", 15L, null);
-    assertEquals(TruthValue.YES_NO,
-        RecordReaderImpl.evaluatePredicateProto(createTimestampStats(10, 100), pred, null));
-
-    pred = TestSearchArgumentImpl.createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
-        PredicateLeaf.Type.FLOAT, "x", 15.0, null);
-    assertEquals(TruthValue.NO,
-        RecordReaderImpl.evaluatePredicateProto(createTimestampStats(10, 100), pred, null));
-    assertEquals(TruthValue.YES_NO,
-        RecordReaderImpl.evaluatePredicateProto(createTimestampStats(10000, 100000), pred, null));
-
-    pred = TestSearchArgumentImpl.createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
-        PredicateLeaf.Type.STRING, "x", "15", null);
-    assertEquals(TruthValue.NO,
-        RecordReaderImpl.evaluatePredicateProto(createTimestampStats(10, 100), pred, null));
-
-    pred = TestSearchArgumentImpl.createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
-        PredicateLeaf.Type.STRING, "x", new Timestamp(15).toString(), null);
-    assertEquals(TruthValue.YES_NO,
-        RecordReaderImpl.evaluatePredicateProto(createTimestampStats(10, 100), pred, null));
-
-    pred = TestSearchArgumentImpl.createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
-        PredicateLeaf.Type.DATE, "x", new DateWritable(15).get(), null);
-    assertEquals(TruthValue.NO,
-        RecordReaderImpl.evaluatePredicateProto(createTimestampStats(10, 100), pred, null));
-    assertEquals(TruthValue.YES_NO,
-        RecordReaderImpl.evaluatePredicateProto(createTimestampStats(10 * 24L * 60L * 60L * 1000L,
-          100 * 24L * 60L * 60L * 1000L), pred, null));
-
-    pred = TestSearchArgumentImpl.createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
-        PredicateLeaf.Type.DECIMAL, "x", new HiveDecimalWritable("15"), null);
-    assertEquals(TruthValue.NO,
-        RecordReaderImpl.evaluatePredicateProto(createTimestampStats(10, 100), pred, null));
-    assertEquals(TruthValue.YES_NO,
-        RecordReaderImpl.evaluatePredicateProto(createTimestampStats(10000, 100000), pred, null));
-
-    pred = TestSearchArgumentImpl.createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
-        PredicateLeaf.Type.TIMESTAMP, "x", new Timestamp(15), null);
-    assertEquals(TruthValue.YES_NO,
-        RecordReaderImpl.evaluatePredicateProto(createTimestampStats(10, 100), pred, null));
-    assertEquals(TruthValue.NO,
-        RecordReaderImpl.evaluatePredicateProto(createTimestampStats(10000, 100000), pred, null));
-  }
-
-  @Test
-  public void testEquals() throws Exception {
-    PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf
-        (PredicateLeaf.Operator.EQUALS, PredicateLeaf.Type.LONG,
-            "x", 15L, null);
-    assertEquals(TruthValue.NO_NULL,
-        RecordReaderImpl.evaluatePredicateProto(createIntStats(20L, 30L), pred, null));
-    assertEquals(TruthValue.YES_NO_NULL,
-        RecordReaderImpl.evaluatePredicateProto(createIntStats(15L, 30L), pred, null));
-    assertEquals(TruthValue.YES_NO_NULL,
-        RecordReaderImpl.evaluatePredicateProto(createIntStats(10L, 30L), pred, null));
-    assertEquals(TruthValue.YES_NO_NULL,
-        RecordReaderImpl.evaluatePredicateProto(createIntStats(10L, 15L), pred, null));
-    assertEquals(TruthValue.NO_NULL,
-        RecordReaderImpl.evaluatePredicateProto(createIntStats(0L, 10L), pred, null));
-    assertEquals(TruthValue.YES_NULL,
-        RecordReaderImpl.evaluatePredicateProto(createIntStats(15L, 15L), pred, null));
-  }
-
-  @Test
-  public void testNullSafeEquals() throws Exception {
-    PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf
-        (PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.LONG,
-            "x", 15L, null);
-    assertEquals(TruthValue.NO,
-        RecordReaderImpl.evaluatePredicateProto(createIntStats(20L, 30L), pred, null));
-    assertEquals(TruthValue.YES_NO,
-        RecordReaderImpl.evaluatePredicateProto(createIntStats(15L, 30L), pred, null));
-    assertEquals(TruthValue.YES_NO,
-        RecordReaderImpl.evaluatePredicateProto(createIntStats(10L, 30L), pred, null));
-    assertEquals(TruthValue.YES_NO,
-        RecordReaderImpl.evaluatePredicateProto(createIntStats(10L, 15L), pred, null));
-    assertEquals(TruthValue.NO,
-        RecordReaderImpl.evaluatePredicateProto(createIntStats(0L, 10L), pred, null));
-    assertEquals(TruthValue.YES_NO,
-        RecordReaderImpl.evaluatePredicateProto(createIntStats(15L, 15L), pred, null));
-  }
-
-  @Test
-  public void testLessThan() throws Exception {
-    PredicateLeaf lessThan = TestSearchArgumentImpl.createPredicateLeaf
-        (PredicateLeaf.Operator.LESS_THAN, PredicateLeaf.Type.LONG,
-            "x", 15L, null);
-    assertEquals(TruthValue.NO_NULL,
-        RecordReaderImpl.evaluatePredicateProto(createIntStats(20L, 30L), lessThan, null));
-    assertEquals(TruthValue.NO_NULL,
-        RecordReaderImpl.evaluatePredicateProto(createIntStats(15L, 30L), lessThan, null));
-    assertEquals(TruthValue.YES_NO_NULL,
-        RecordReaderImpl.evaluatePredicateProto(createIntStats(10L, 30L), lessThan, null));
-    assertEquals(TruthValue.YES_NO_NULL,
-        RecordReaderImpl.evaluatePredicateProto(createIntStats(10L, 15L), lessThan, null));
-    assertEquals(TruthValue.YES_NULL,
-        RecordReaderImpl.evaluatePredicateProto(createIntStats(0L, 10L), lessThan, null));
-  }
-
-  @Test
-  public void testLessThanEquals() throws Exception {
-    PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf
-        (PredicateLeaf.Operator.LESS_THAN_EQUALS, PredicateLeaf.Type.LONG,
-            "x", 15L, null);
-    assertEquals(TruthValue.NO_NULL,
-        RecordReaderImpl.evaluatePredicateProto(createIntStats(20L, 30L), pred, null));
-    assertEquals(TruthValue.YES_NO_NULL,
-        RecordReaderImpl.evaluatePredicateProto(createIntStats(15L, 30L), pred, null));
-    assertEquals(TruthValue.YES_NO_NULL,
-        RecordReaderImpl.evaluatePredicateProto(createIntStats(10L, 30L), pred, null));
-    assertEquals(TruthValue.YES_NULL,
-        RecordReaderImpl.evaluatePredicateProto(createIntStats(10L, 15L), pred, null));
-    assertEquals(TruthValue.YES_NULL,
-        RecordReaderImpl.evaluatePredicateProto(createIntStats(0L, 10L), pred, null));
-  }
-
-  @Test
-  public void testIn() throws Exception {
-    List<Object> args = new ArrayList<Object>();
-    args.add(10L);
-    args.add(20L);
-    PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf
-        (PredicateLeaf.Operator.IN, PredicateLeaf.Type.LONG,
-            "x", null, args);
-    assertEquals(TruthValue.YES_NULL,
-        RecordReaderImpl.evaluatePredicateProto(createIntStats(20L, 20L), pred, null));
-    assertEquals(TruthValue.NO_NULL,
-        RecordReaderImpl.evaluatePredicateProto(createIntStats(30L, 30L), pred, null));
-    assertEquals(TruthValue.YES_NO_NULL,
-        RecordReaderImpl.evaluatePredicateProto(createIntStats(10L, 30L), pred, null));
-    assertEquals(TruthValue.NO_NULL,
-        RecordReaderImpl.evaluatePredicateProto(createIntStats(12L, 18L), pred, null));
-  }
-
-  @Test
-  public void testBetween() throws Exception {
-    List<Object> args = new ArrayList<Object>();
-    args.add(10L);
-    args.add(20L);
-    PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf
-        (PredicateLeaf.Operator.BETWEEN, PredicateLeaf.Type.LONG,
-            "x", null, args);
-    assertEquals(TruthValue.NO_NULL,
-        RecordReaderImpl.evaluatePredicateProto(createIntStats(0L, 5L), pred, null));
-    assertEquals(TruthValue.NO_NULL,
-      RecordReaderImpl.evaluatePredicateProto(createIntStats(30L, 40L), pred, null));
-    assertEquals(TruthValue.YES_NO_NULL,
-      RecordReaderImpl.evaluatePredicateProto(createIntStats(5L, 15L), pred, null));
-    assertEquals(TruthValue.YES_NO_NULL,
-        RecordReaderImpl.evaluatePredicateProto(createIntStats(15L, 25L), pred, null));
-    assertEquals(TruthValue.YES_NO_NULL,
-        RecordReaderImpl.evaluatePredicateProto(createIntStats(5L, 25L), pred, null));
-    assertEquals(TruthValue.YES_NULL,
-        RecordReaderImpl.evaluatePredicateProto(createIntStats(10L, 20L), pred, null));
-    assertEquals(TruthValue.YES_NULL,
-        RecordReaderImpl.evaluatePredicateProto(createIntStats(12L, 18L), pred, null));
-  }
-
-  @Test
-  public void testIsNull() throws Exception {
-    PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf
-        (PredicateLeaf.Operator.IS_NULL, PredicateLeaf.Type.LONG,
-            "x", null, null);
-    assertEquals(TruthValue.YES_NO,
-        RecordReaderImpl.evaluatePredicateProto(createIntStats(20L, 30L), pred, null));
-  }
-
-
-  @Test
-  public void testEqualsWithNullInStats() throws Exception {
-    PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf
-        (PredicateLeaf.Operator.EQUALS, PredicateLeaf.Type.STRING,
-            "x", "c", null);
-    assertEquals(TruthValue.NO_NULL,
-        RecordReaderImpl.evaluatePredicateProto(createStringStats("d", "e", true), pred, null)); // before
-    assertEquals(TruthValue.NO_NULL,
-        RecordReaderImpl.evaluatePredicateProto(createStringStats("a", "b", true), pred, null)); // after
-    assertEquals(TruthValue.YES_NO_NULL,
-        RecordReaderImpl.evaluatePredicateProto(createStringStats("b", "c", true), pred, null)); // max
-    assertEquals(TruthValue.YES_NO_NULL,
-        RecordReaderImpl.evaluatePredicateProto(createStringStats("c", "d", true), pred, null)); // min
-    assertEquals(TruthValue.YES_NO_NULL,
-        RecordReaderImpl.evaluatePredicateProto(createStringStats("b", "d", true), pred, null)); // middle
-    assertEquals(TruthValue.YES_NULL,
-        RecordReaderImpl.evaluatePredicateProto(createStringStats("c", "c", true), pred, null)); // same
-  }
-
-  @Test
-  public void testNullSafeEqualsWithNullInStats() throws Exception {
-    PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf
-        (PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.STRING,
-            "x", "c", null);
-    assertEquals(TruthValue.NO,
-        RecordReaderImpl.evaluatePredicateProto(createStringStats("d", "e", true), pred, null)); // before
-    assertEquals(TruthValue.NO,
-        RecordReaderImpl.evaluatePredicateProto(createStringStats("a", "b", true), pred, null)); // after
-    assertEquals(TruthValue.YES_NO,
-        RecordReaderImpl.evaluatePredicateProto(createStringStats("b", "c", true), pred, null)); // max
-    assertEquals(TruthValue.YES_NO,
-        RecordReaderImpl.evaluatePredicateProto(createStringStats("c", "d", true), pred, null)); // min
-    assertEquals(TruthValue.YES_NO,
-        RecordReaderImpl.evaluatePredicateProto(createStringStats("b", "d", true), pred, null)); // middle
-    assertEquals(TruthValue.YES_NO,
-        RecordReaderImpl.evaluatePredicateProto(createStringStats("c", "c", true), pred, null)); // same
-  }
-
-  @Test
-  public void testLessThanWithNullInStats() throws Exception {
-    PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf
-        (PredicateLeaf.Operator.LESS_THAN, PredicateLeaf.Type.STRING,
-            "x", "c", null);
-    assertEquals(TruthValue.NO_NULL,
-        RecordReaderImpl.evaluatePredicateProto(createStringStats("d", "e", true), pred, null)); // before
-    assertEquals(TruthValue.YES_NULL,
-        RecordReaderImpl.evaluatePredicateProto(createStringStats("a", "b", true), pred, null)); // after
-    assertEquals(TruthValue.YES_NO_NULL,
-        RecordReaderImpl.evaluatePredicateProto(createStringStats("b", "c", true), pred, null)); // max
-    assertEquals(TruthValue.NO_NULL,
-        RecordReaderImpl.evaluatePredicateProto(createStringStats("c", "d", true), pred, null)); // min
-    assertEquals(TruthValue.YES_NO_NULL,
-        RecordReaderImpl.evaluatePredicateProto(createStringStats("b", "d", true), pred, null)); // middle
-    assertEquals(TruthValue.NO_NULL, // min, same stats
-        RecordReaderImpl.evaluatePredicateProto(createStringStats("c", "c", true), pred, null));
-  }
-
-  @Test
-  public void testLessThanEqualsWithNullInStats() throws Exception {
-    PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf
-        (PredicateLeaf.Operator.LESS_THAN_EQUALS, PredicateLeaf.Type.STRING,
-            "x", "c", null);
-    assertEquals(TruthValue.NO_NULL,
-        RecordReaderImpl.evaluatePredicateProto(createStringStats("d", "e", true), pred, null)); // before
-    assertEquals(TruthValue.YES_NULL,
-        RecordReaderImpl.evaluatePredicateProto(createStringStats("a", "b", true), pred, null)); // after
-    assertEquals(TruthValue.YES_NULL,
-        RecordReaderImpl.evaluatePredicateProto(createStringStats("b", "c", true), pred, null)); // max
-    assertEquals(TruthValue.YES_NO_NULL,
-        RecordReaderImpl.evaluatePredicateProto(createStringStats("c", "d", true), pred, null)); // min
-    assertEquals(TruthValue.YES_NO_NULL,
-        RecordReaderImpl.evaluatePredicateProto(createStringStats("b", "d", true), pred, null)); // middle
-    assertEquals(TruthValue.YES_NO_NULL,
-        RecordReaderImpl.evaluatePredicateProto(createStringStats("c", "c", true), pred, null)); // same
-  }
-
-  @Test
-  public void testInWithNullInStats() throws Exception {
-    List<Object> args = new ArrayList<Object>();
-    args.add("c");
-    args.add("f");
-    PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf
-        (PredicateLeaf.Operator.IN, PredicateLeaf.Type.STRING,
-            "x", null, args);
-    assertEquals(TruthValue.NO_NULL, // before & after
-        RecordReaderImpl.evaluatePredicateProto(createStringStats("d", "e", true), pred, null));
-    assertEquals(TruthValue.NO_NULL,
-        RecordReaderImpl.evaluatePredicateProto(createStringStats("a", "b", true), pred, null)); // after
-    assertEquals(TruthValue.YES_NO_NULL,
-        RecordReaderImpl.evaluatePredicateProto(createStringStats("e", "f", true), pred, null)); // max
-    assertEquals(TruthValue.YES_NO_NULL,
-        RecordReaderImpl.evaluatePredicateProto(createStringStats("c", "d", true), pred, null)); // min
-    assertEquals(TruthValue.YES_NO_NULL,
-        RecordReaderImpl.evaluatePredicateProto(createStringStats("b", "d", true), pred, null)); // middle
-    assertEquals(TruthValue.YES_NULL,
-        RecordReaderImpl.evaluatePredicateProto(createStringStats("c", "c", true), pred, null)); // same
-  }
-
-  @Test
-  public void testBetweenWithNullInStats() throws Exception {
-    List<Object> args = new ArrayList<Object>();
-    args.add("c");
-    args.add("f");
-    PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf
-        (PredicateLeaf.Operator.BETWEEN, PredicateLeaf.Type.STRING,
-            "x", null, args);
-    assertEquals(TruthValue.YES_NULL, // before & after
-        RecordReaderImpl.evaluatePredicateProto(createStringStats("d", "e", true), pred, null));
-    assertEquals(TruthValue.YES_NULL, // before & max
-        RecordReaderImpl.evaluatePredicateProto(createStringStats("e", "f", true), pred, null));
-    assertEquals(TruthValue.NO_NULL, // before & before
-        RecordReaderImpl.evaluatePredicateProto(createStringStats("h", "g", true), pred, null));
-    assertEquals(TruthValue.YES_NO_NULL, // before & min
-        RecordReaderImpl.evaluatePredicateProto(createStringStats("f", "g", true), pred, null));
-    assertEquals(TruthValue.YES_NO_NULL, // before & middle
-      RecordReaderImpl.evaluatePredicateProto(createStringStats("e", "g", true), pred, null));
-
-    assertEquals(TruthValue.YES_NULL, // min & after
-      RecordReaderImpl.evaluatePredicateProto(createStringStats("c", "e", true), pred, null));
-    assertEquals(TruthValue.YES_NULL, // min & max
-        RecordReaderImpl.evaluatePredicateProto(createStringStats("c", "f", true), pred, null));
-    assertEquals(TruthValue.YES_NO_NULL, // min & middle
-        RecordReaderImpl.evaluatePredicateProto(createStringStats("c", "g", true), pred, null));
-
-    assertEquals(TruthValue.NO_NULL,
-        RecordReaderImpl.evaluatePredicateProto(createStringStats("a", "b", true), pred, null)); // after
-    assertEquals(TruthValue.YES_NO_NULL,
-        RecordReaderImpl.evaluatePredicateProto(createStringStats("a", "c", true), pred, null)); // max
-    assertEquals(TruthValue.YES_NO_NULL,
-        RecordReaderImpl.evaluatePredicateProto(createStringStats("b", "d", true), pred, null)); // middle
-    assertEquals(TruthValue.YES_NULL, // min & after, same stats
-        RecordReaderImpl.evaluatePredicateProto(createStringStats("c", "c", true), pred, null));
-  }
-
-  @Test
-  public void testIsNullWithNullInStats() throws Exception {
-    PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf
-        (PredicateLeaf.Operator.IS_NULL, PredicateLeaf.Type.STRING,
-            "x", null, null);
-    assertEquals(TruthValue.YES_NO,
-        RecordReaderImpl.evaluatePredicateProto(createStringStats("c", "d", true), pred, null));
-    assertEquals(TruthValue.NO,
-        RecordReaderImpl.evaluatePredicateProto(createStringStats("c", "d", false), pred, null));
-  }
-
-  @Test
-  public void testOverlap() throws Exception {
-    assertTrue(!RecordReaderUtils.overlap(0, 10, -10, -1));
-    assertTrue(RecordReaderUtils.overlap(0, 10, -1, 0));
-    assertTrue(RecordReaderUtils.overlap(0, 10, -1, 1));
-    assertTrue(RecordReaderUtils.overlap(0, 10, 2, 8));
-    assertTrue(RecordReaderUtils.overlap(0, 10, 5, 10));
-    assertTrue(RecordReaderUtils.overlap(0, 10, 10, 11));
-    assertTrue(RecordReaderUtils.overlap(0, 10, 0, 10));
-    assertTrue(RecordReaderUtils.overlap(0, 10, -1, 11));
-    assertTrue(!RecordReaderUtils.overlap(0, 10, 11, 12));
-  }
-
-  private static DiskRangeList diskRanges(Integer... points) {
-    DiskRangeList head = null, tail = null;
-    for(int i = 0; i < points.length; i += 2) {
-      DiskRangeList range = new DiskRangeList(points[i], points[i+1]);
-      if (tail == null) {
-        head = tail = range;
-      } else {
-        tail = tail.insertAfter(range);
-      }
-    }
-    return head;
-  }
-
-  @Test
-  public void testGetIndexPosition() throws Exception {
-    assertEquals(0, RecordReaderUtils.getIndexPosition
-        (OrcProto.ColumnEncoding.Kind.DIRECT, OrcProto.Type.Kind.INT,
-            OrcProto.Stream.Kind.PRESENT, true, true));
-    assertEquals(4, RecordReaderUtils.getIndexPosition
-        (OrcProto.ColumnEncoding.Kind.DIRECT, OrcProto.Type.Kind.INT,
-            OrcProto.Stream.Kind.DATA, true, true));
-    assertEquals(3, RecordReaderUtils.getIndexPosition
-        (OrcProto.ColumnEncoding.Kind.DIRECT, OrcProto.Type.Kind.INT,
-            OrcProto.Stream.Kind.DATA, false, true));
-    assertEquals(0, RecordReaderUtils.getIndexPosition
-        (OrcProto.ColumnEncoding.Kind.DIRECT, OrcProto.Type.Kind.INT,
-            OrcProto.Stream.Kind.DATA, true, false));
-    assertEquals(4, RecordReaderUtils.getIndexPosition
-        (OrcProto.ColumnEncoding.Kind.DICTIONARY, OrcProto.Type.Kind.STRING,
-            OrcProto.Stream.Kind.DATA, true, true));
-    assertEquals(4, RecordReaderUtils.getIndexPosition
-        (OrcProto.ColumnEncoding.Kind.DIRECT, OrcProto.Type.Kind.BINARY,
-            OrcProto.Stream.Kind.DATA, true, true));
-    assertEquals(3, RecordReaderUtils.getIndexPosition
-        (OrcProto.ColumnEncoding.Kind.DIRECT, OrcProto.Type.Kind.BINARY,
-            OrcProto.Stream.Kind.DATA, false, true));
-    assertEquals(6, RecordReaderUtils.getIndexPosition
-        (OrcProto.ColumnEncoding.Kind.DIRECT, OrcProto.Type.Kind.BINARY,
-            OrcProto.Stream.Kind.LENGTH, true, true));
-    assertEquals(4, RecordReaderUtils.getIndexPosition
-        (OrcProto.ColumnEncoding.Kind.DIRECT, OrcProto.Type.Kind.BINARY,
-            OrcProto.Stream.Kind.LENGTH, false, true));
-    assertEquals(4, RecordReaderUtils.getIndexPosition
-        (OrcProto.ColumnEncoding.Kind.DIRECT, OrcProto.Type.Kind.DECIMAL,
-            OrcProto.Stream.Kind.DATA, true, true));
-    assertEquals(3, RecordReaderUtils.getIndexPosition
-        (OrcProto.ColumnEncoding.Kind.DIRECT, OrcProto.Type.Kind.DECIMAL,
-            OrcProto.Stream.Kind.DATA, false, true));
-    assertEquals(6, RecordReaderUtils.getIndexPosition
-        (OrcProto.ColumnEncoding.Kind.DIRECT, OrcProto.Type.Kind.DECIMAL,
-            OrcProto.Stream.Kind.SECONDARY, true, true));
-    assertEquals(4, RecordReaderUtils.getIndexPosition
-        (OrcProto.ColumnEncoding.Kind.DIRECT, OrcProto.Type.Kind.DECIMAL,
-            OrcProto.Stream.Kind.SECONDARY, false, true));
-    assertEquals(4, RecordReaderUtils.getIndexPosition
-        (OrcProto.ColumnEncoding.Kind.DIRECT, OrcProto.Type.Kind.TIMESTAMP,
-            OrcProto.Stream.Kind.DATA, true, true));
-    assertEquals(3, RecordReaderUtils.getIndexPosition
-        (OrcProto.ColumnEncoding.Kind.DIRECT, OrcProto.Type.Kind.TIMESTAMP,
-            OrcProto.Stream.Kind.DATA, false, true));
-    assertEquals(7, RecordReaderUtils.getIndexPosition
-        (OrcProto.ColumnEncoding.Kind.DIRECT, OrcProto.Type.Kind.TIMESTAMP,
-            OrcProto.Stream.Kind.SECONDARY, true, true));
-    assertEquals(5, RecordReaderUtils.getIndexPosition
-        (OrcProto.ColumnEncoding.Kind.DIRECT, OrcProto.Type.Kind.TIMESTAMP,
-            OrcProto.Stream.Kind.SECONDARY, false, true));
-  }
-
-  @Test
-  public void testPartialPlan() throws Exception {
-    DiskRangeList result;
-
-    // set the streams
-    List<OrcProto.Stream> streams = new ArrayList<OrcProto.Stream>();
-    streams.add(OrcProto.Stream.newBuilder()
-        .setKind(OrcProto.Stream.Kind.PRESENT)
-        .setColumn(1).setLength(1000).build());
-    streams.add(OrcProto.Stream.newBuilder()
-        .setKind(OrcProto.Stream.Kind.DATA)
-        .setColumn(1).setLength(99000).build());
-    streams.add(OrcProto.Stream.newBuilder()
-        .setKind(OrcProto.Stream.Kind.PRESENT)
-        .setColumn(2).setLength(2000).build());
-    streams.add(OrcProto.Stream.newBuilder()
-        .setKind(OrcProto.Stream.Kind.DATA)
-        .setColumn(2).setLength(98000).build());
-
-    boolean[] columns = new boolean[]{true, true, false};
-    boolean[] rowGroups = new boolean[]{true, true, false, false, true, false};
-
-    // set the index
-    OrcProto.RowIndex[] indexes = new OrcProto.RowIndex[columns.length];
-    indexes[1] = OrcProto.RowIndex.newBuilder()
-        .addEntry(OrcProto.RowIndexEntry.newBuilder()
-            .addPositions(0).addPositions(-1).addPositions(-1)
-            .addPositions(0)
-            .build())
-        .addEntry(OrcProto.RowIndexEntry.newBuilder()
-            .addPositions(100).addPositions(-1).addPositions(-1)
-            .addPositions(10000)
-            .build())
-        .addEntry(OrcProto.RowIndexEntry.newBuilder()
-            .addPositions(200).addPositions(-1).addPositions(-1)
-            .addPositions(20000)
-            .build())
-        .addEntry(OrcProto.RowIndexEntry.newBuilder()
-            .addPositions(300).addPositions(-1).addPositions(-1)
-            .addPositions(30000)
-            .build())
-        .addEntry(OrcProto.RowIndexEntry.newBuilder()
-            .addPositions(400).addPositions(-1).addPositions(-1)
-            .addPositions(40000)
-            .build())
-        .addEntry(OrcProto.RowIndexEntry.newBuilder()
-            .addPositions(500).addPositions(-1).addPositions(-1)
-            .addPositions(50000)
-            .build())
-        .build();
-
-    // set encodings
-    List<OrcProto.ColumnEncoding> encodings =
-        new ArrayList<OrcProto.ColumnEncoding>();
-    encodings.add(OrcProto.ColumnEncoding.newBuilder()
-                    .setKind(OrcProto.ColumnEncoding.Kind.DIRECT).build());
-    encodings.add(OrcProto.ColumnEncoding.newBuilder()
-        .setKind(OrcProto.ColumnEncoding.Kind.DIRECT).build());
-    encodings.add(OrcProto.ColumnEncoding.newBuilder()
-        .setKind(OrcProto.ColumnEncoding.Kind.DIRECT).build());
-
-    // set types struct{x: int, y: int}
-    List<OrcProto.Type> types = new ArrayList<OrcProto.Type>();
-    types.add(OrcProto.Type.newBuilder().setKind(OrcProto.Type.Kind.STRUCT)
-                .addSubtypes(1).addSubtypes(2).addFieldNames("x")
-                .addFieldNames("y").build());
-    types.add(OrcProto.Type.newBuilder().setKind(OrcProto.Type.Kind.INT).build());
-    types.add(OrcProto.Type.newBuilder().setKind(OrcProto.Type.Kind.INT).build());
-
-    // filter by rows and groups
-    result = RecordReaderImpl.planReadPartialDataStreams(streams, indexes,
-        columns, rowGroups, false, encodings, types, 32768, false);
-    assertThat(result, is(diskRanges(0, 1000, 100, 1000, 400, 1000,
-        1000, 11000 + RecordReaderUtils.WORST_UNCOMPRESSED_SLOP,
-        11000, 21000 + RecordReaderUtils.WORST_UNCOMPRESSED_SLOP,
-        41000, 51000 + RecordReaderUtils.WORST_UNCOMPRESSED_SLOP)));
-    result = RecordReaderImpl.planReadPartialDataStreams(streams, indexes,
-        columns, rowGroups, false, encodings, types, 32768, true);
-    assertThat(result, is(diskRanges(0, 21000 + RecordReaderUtils.WORST_UNCOMPRESSED_SLOP,
-        41000, 51000 + RecordReaderUtils.WORST_UNCOMPRESSED_SLOP)));
-
-    // if we read no rows, don't read any bytes
-    rowGroups = new boolean[]{false, false, false, false, false, false};
-    result = RecordReaderImpl.planReadPartialDataStreams(streams, indexes,
-        columns, rowGroups, false, encodings, types, 32768, false);
-    assertNull(result);
-
-    // all rows, but only columns 0 and 2.
-    rowGroups = null;
-    columns = new boolean[]{true, false, true};
-    result = RecordReaderImpl.planReadPartialDataStreams(streams, indexes,
-        columns, null, false, encodings, types, 32768, false);
-    assertThat(result, is(diskRanges(100000, 102000, 102000, 200000)));
-    result = RecordReaderImpl.planReadPartialDataStreams(streams, indexes,
-        columns, null, false, encodings, types, 32768, true);
-    assertThat(result, is(diskRanges(100000, 200000)));
-
-    rowGroups = new boolean[]{false, true, false, false, false, false};
-    indexes[2] = indexes[1];
-    indexes[1] = null;
-    result = RecordReaderImpl.planReadPartialDataStreams(streams, indexes,
-        columns, rowGroups, false, encodings, types, 32768, false);
-    assertThat(result, is(diskRanges(100100, 102000,
-        112000, 122000 + RecordReaderUtils.WORST_UNCOMPRESSED_SLOP)));
-    result = RecordReaderImpl.planReadPartialDataStreams(streams, indexes,
-        columns, rowGroups, false, encodings, types, 32768, true);
-    assertThat(result, is(diskRanges(100100, 102000,
-        112000, 122000 + RecordReaderUtils.WORST_UNCOMPRESSED_SLOP)));
-
-    rowGroups = new boolean[]{false, false, false, false, false, true};
-    indexes[1] = indexes[2];
-    columns = new boolean[]{true, true, true};
-    result = RecordReaderImpl.planReadPartialDataStreams(streams, indexes,
-        columns, rowGroups, false, encodings, types, 32768, false);
-    assertThat(result, is(diskRanges(500, 1000, 51000, 100000, 100500, 102000,
-        152000, 200000)));
-    result = RecordReaderImpl.planReadPartialDataStreams(streams, indexes,
-        columns, rowGroups, false, encodings, types, 32768, true);
-    assertThat(result, is(diskRanges(500, 1000, 51000, 100000, 100500, 102000,
-        152000, 200000)));
-  }
-
-
-  @Test
-  public void testPartialPlanCompressed() throws Exception {
-    DiskRangeList result;
-
-    // set the streams
-    List<OrcProto.Stream> streams = new ArrayList<OrcProto.Stream>();
-    streams.add(OrcProto.Stream.newBuilder()
-        .setKind(OrcProto.Stream.Kind.PRESENT)
-        .setColumn(1).setLength(1000).build());
-    streams.add(OrcProto.Stream.newBuilder()
-        .setKind(OrcProto.Stream.Kind.DATA)
-        .setColumn(1).setLength(99000).build());
-    streams.add(OrcProto.Stream.newBuilder()
-        .setKind(OrcProto.Stream.Kind.PRESENT)
-        .setColumn(2).setLength(2000).build());
-    streams.add(OrcProto.Stream.newBuilder()
-        .setKind(OrcProto.Stream.Kind.DATA)
-        .setColumn(2).setLength(98000).build());
-
-    boolean[] columns = new boolean[]{true, true, false};
-    boolean[] rowGroups = new boolean[]{true, true, false, false, true, false};
-
-    // set the index
-    OrcProto.RowIndex[] indexes = new OrcProto.RowIndex[columns.length];
-    indexes[1] = OrcProto.RowIndex.newBuilder()
-        .addEntry(OrcProto.RowIndexEntry.newBuilder()
-            .addPositions(0).addPositions(-1).addPositions(-1).addPositions(-1)
-            .addPositions(0)
-            .build())
-        .addEntry(OrcProto.RowIndexEntry.newBuilder()
-            .addPositions(100).addPositions(-1).addPositions(-1).addPositions(-1)
-            .addPositions(10000)
-            .build())
-        .addEntry(OrcProto.RowIndexEntry.newBuilder()
-            .addPositions(200).addPositions(-1).addPositions(-1).addPositions(-1)
-            .addPositions(20000)
-            .build())
-        .addEntry(OrcProto.RowIndexEntry.newBuilder()
-            .addPositions(300).addPositions(-1).addPositions(-1).addPositions(-1)
-            .addPositions(30000)
-            .build())
-        .addEntry(OrcProto.RowIndexEntry.newBuilder()
-            .addPositions(400).addPositions(-1).addPositions(-1).addPositions(-1)
-            .addPositions(40000)
-            .build())
-        .addEntry(OrcProto.RowIndexEntry.newBuilder()
-            .addPositions(500).addPositions(-1).addPositions(-1).addPositions(-1)
-            .addPositions(50000)
-            .build())
-        .build();
-
-    // set encodings
-    List<OrcProto.ColumnEncoding> encodings =
-        new ArrayList<OrcProto.ColumnEncoding>();
-    encodings.add(OrcProto.ColumnEncoding.newBuilder()
-        .setKind(OrcProto.ColumnEncoding.Kind.DIRECT).build());
-    encodings.add(OrcProto.ColumnEncoding.newBuilder()
-        .setKind(OrcProto.ColumnEncoding.Kind.DIRECT).build());
-    encodings.add(OrcProto.ColumnEncoding.newBuilder()
-        .setKind(OrcProto.ColumnEncoding.Kind.DIRECT).build());
-
-    // set types struct{x: int, y: int}
-    List<OrcProto.Type> types = new ArrayList<OrcProto.Type>();
-    types.add(OrcProto.Type.newBuilder().setKind(OrcProto.Type.Kind.STRUCT)
-        .addSubtypes(1).addSubtypes(2).addFieldNames("x")
-        .addFieldNames("y").build());
-    types.add(OrcProto.Type.newBuilder().setKind(OrcProto.Type.Kind.INT).build());
-    types.add(OrcProto.Type.newBuilder().setKind(OrcProto.Type.Kind.INT).build());
-
-    // filter by rows and groups
-    result = RecordReaderImpl.planReadPartialDataStreams(streams, indexes,
-        columns, rowGroups, true, encodings, types, 32768, false);
-    assertThat(result, is(diskRanges(0, 1000, 100, 1000,
-        400, 1000, 1000, 11000+(2*32771),
-        11000, 21000+(2*32771), 41000, 100000)));
-
-    rowGroups = new boolean[]{false, false, false, false, false, true};
-    result = RecordReaderImpl.planReadPartialDataStreams(streams, indexes,
-        columns, rowGroups, true, encodings, types, 32768, false);
-    assertThat(result, is(diskRanges(500, 1000, 51000, 100000)));
-  }
-
-  @Test
-  public void testPartialPlanString() throws Exception {
-    DiskRangeList result;
-
-    // set the streams
-    List<OrcProto.Stream> streams = new ArrayList<OrcProto.Stream>();
-    streams.add(OrcProto.Stream.newBuilder()
-        .setKind(OrcProto.Stream.Kind.PRESENT)
-        .setColumn(1).setLength(1000).build());
-    streams.add(OrcProto.Stream.newBuilder()
-        .setKind(OrcProto.Stream.Kind.DATA)
-        .setColumn(1).setLength(94000).build());
-    streams.add(OrcProto.Stream.newBuilder()
-        .setKind(OrcProto.Stream.Kind.LENGTH)
-        .setColumn(1).setLength(2000).build());
-    streams.add(OrcProto.Stream.newBuilder()
-        .setKind(OrcProto.Stream.Kind.DICTIONARY_DATA)
-        .setColumn(1).setLength(3000).build());
-    streams.add(OrcProto.Stream.newBuilder()
-        .setKind(OrcProto.Stream.Kind.PRESENT)
-        .setColumn(2).setLength(2000).build());
-    streams.add(OrcProto.Stream.newBuilder()
-        .setKind(OrcProto.Stream.Kind.DATA)
-        .setColumn(2).setLength(98000).build());
-
-    boolean[] columns = new boolean[]{true, true, false};
-    boolean[] rowGroups = new boolean[]{false, true, false, false, true, true};
-
-    // set the index
-    OrcProto.RowIndex[] indexes = new OrcProto.RowIndex[columns.length];
-    indexes[1] = OrcProto.RowIndex.newBuilder()
-        .addEntry(OrcProto.RowIndexEntry.newBuilder()
-            .addPositions(0).addPositions(-1).addPositions(-1)
-            .addPositions(0)
-            .build())
-        .addEntry(OrcProto.RowIndexEntry.newBuilder()
-            .addPositions(100).addPositions(-1).addPositions(-1)
-            .addPositions(10000)
-            .build())
-        .addEntry(OrcProto.RowIndexEntry.newBuilder()
-            .addPositions(200).addPositions(-1).addPositions(-1)
-            .addPositions(20000)
-            .build())
-        .addEntry(OrcProto.RowIndexEntry.newBuilder()
-            .addPositions(300).addPositions(-1).addPositions(-1)
-            .addPositions(30000)
-            .build())
-        .addEntry(OrcProto.RowIndexEntry.newBuilder()
-            .addPositions(400).addPositions(-1).addPositions(-1)
-            .addPositions(40000)
-            .build())
-        .addEntry(OrcProto.RowIndexEntry.newBuilder()
-            .addPositions(500).addPositions(-1).addPositions(-1)
-            .addPositions(50000)
-            .build())
-        .build();
-
-    // set encodings
-    List<OrcProto.ColumnEncoding> encodings =
-        new ArrayList<OrcProto.ColumnEncoding>();
-    encodings.add(OrcProto.ColumnEncoding.newBuilder()
-        .setKind(OrcProto.ColumnEncoding.Kind.DIRECT).build());
-    encodings.add(OrcProto.ColumnEncoding.newBuilder()
-        .setKind(OrcProto.ColumnEncoding.Kind.DICTIONARY).build());
-    encodings.add(OrcProto.ColumnEncoding.newBuilder()
-        .setKind(OrcProto.ColumnEncoding.Kind.DIRECT).build());
-
-    // set types struct{x: string, y: int}
-    List<OrcProto.Type> types = new ArrayList<OrcProto.Type>();
-    types.add(OrcProto.Type.newBuilder().setKind(OrcProto.Type.Kind.STRUCT)
-        .addSubtypes(1).addSubtypes(2).addFieldNames("x")
-        .addFieldNames("y").build());
-    types.add(OrcProto.Type.newBuilder().setKind(OrcProto.Type.Kind.STRING).build());
-    types.add(OrcProto.Type.newBuilder().setKind(OrcProto.Type.Kind.INT).build());
-
-    // filter by rows and groups
-    result = RecordReaderImpl.planReadPartialDataStreams(streams, indexes,
-        columns, rowGroups, false, encodings, types, 32768, false);
-    assertThat(result, is(diskRanges(100, 1000, 400, 1000, 500, 1000,
-        11000, 21000 + RecordReaderUtils.WORST_UNCOMPRESSED_SLOP,
-        41000, 51000 + RecordReaderUtils.WORST_UNCOMPRESSED_SLOP,
-        51000, 95000, 95000, 97000, 97000, 100000)));
-  }
-
-  @Test
-  public void testIntNullSafeEqualsBloomFilter() throws Exception {
-    PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf(
-        PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.LONG, "x", 15L, null);
-    BloomFilterIO bf = new BloomFilterIO(10000);
-    for (int i = 20; i < 1000; i++) {
-      bf.addLong(i);
-    }
-    ColumnStatistics cs = ColumnStatisticsImpl.deserialize(createIntStats(10, 100));
-    assertEquals(TruthValue.NO, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
-
-    bf.addLong(15);
-    assertEquals(TruthValue.YES_NO, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
-  }
-
-  @Test
-  public void testIntEqualsBloomFilter() throws Exception {
-    PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf(
-        PredicateLeaf.Operator.EQUALS, PredicateLeaf.Type.LONG, "x", 15L, null);
-    BloomFilterIO bf = new BloomFilterIO(10000);
-    for (int i = 20; i < 1000; i++) {
-      bf.addLong(i);
-    }
-    ColumnStatistics cs = ColumnStatisticsImpl.deserialize(createIntStats(10, 100));
-    assertEquals(TruthValue.NO_NULL, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
-
-    bf.addLong(15);
-    assertEquals(TruthValue.YES_NO_NULL, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
-  }
-
-  @Test
-  public void testIntInBloomFilter() throws Exception {
-    List<Object> args = new ArrayList<Object>();
-    args.add(15L);
-    args.add(19L);
-    PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf
-        (PredicateLeaf.Operator.IN, PredicateLeaf.Type.LONG,
-            "x", null, args);
-    BloomFilterIO bf = new BloomFilterIO(10000);
-    for (int i = 20; i < 1000; i++) {
-      bf.addLong(i);
-    }
-    ColumnStatistics cs = ColumnStatisticsImpl.deserialize(createIntStats(10, 100));
-    assertEquals(TruthValue.NO_NULL, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
-
-    bf.addLong(19);
-    assertEquals(TruthValue.YES_NO_NULL, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
-
-    bf.addLong(15);
-    assertEquals(TruthValue.YES_NO_NULL, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
-  }
-
-  @Test
-  public void testDoubleNullSafeEqualsBloomFilter() throws Exception {
-    PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf(
-        PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.FLOAT, "x", 15.0, null);
-    BloomFilterIO bf = new BloomFilterIO(10000);
-    for (int i = 20; i < 1000; i++) {
-      bf.addDouble(i);
-    }
-    ColumnStatistics cs = ColumnStatisticsImpl.deserialize(createDoubleStats(10.0, 100.0));
-    assertEquals(TruthValue.NO, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
-
-    bf.addDouble(15.0);
-    assertEquals(TruthValue.YES_NO, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
-  }
-
-  @Test
-  public void testDoubleEqualsBloomFilter() throws Exception {
-    PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf(
-        PredicateLeaf.Operator.EQUALS, PredicateLeaf.Type.FLOAT, "x", 15.0, null);
-    BloomFilterIO bf = new BloomFilterIO(10000);
-    for (int i = 20; i < 1000; i++) {
-      bf.addDouble(i);
-    }
-    ColumnStatistics cs = ColumnStatisticsImpl.deserialize(createDoubleStats(10.0, 100.0));
-    assertEquals(TruthValue.NO_NULL, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
-
-    bf.addDouble(15.0);
-    assertEquals(TruthValue.YES_NO_NULL, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
-  }
-
-  @Test
-  public void testDoubleInBloomFilter() throws Exception {
-    List<Object> args = new ArrayList<Object>();
-    args.add(15.0);
-    args.add(19.0);
-    PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf
-        (PredicateLeaf.Operator.IN, PredicateLeaf.Type.FLOAT,
-            "x", null, args);
-    BloomFilterIO bf = new BloomFilterIO(10000);
-    for (int i = 20; i < 1000; i++) {
-      bf.addDouble(i);
-    }
-    ColumnStatistics cs = ColumnStatisticsImpl.deserialize(createDoubleStats(10.0, 100.0));
-    assertEquals(TruthValue.NO_NULL, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
-
-    bf.addDouble(19.0);
-    assertEquals(TruthValue.YES_NO_NULL, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
-
-    bf.addDouble(15.0);
-    assertEquals(TruthValue.YES_NO_NULL, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
-  }
-
-  @Test
-  public void testStringNullSafeEqualsBloomFilter() throws Exception {
-    PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf(
-        PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.STRING, "x", "str_15", null);
-    BloomFilterIO bf = new BloomFilterIO(10000);
-    for (int i = 20; i < 1000; i++) {
-      bf.addString("str_" + i);
-    }
-    ColumnStatistics cs = ColumnStatisticsImpl.deserialize(createStringStats("str_10", "str_200"));
-    assertEquals(TruthValue.NO, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
-
-    bf.addString("str_15");
-    assertEquals(TruthValue.YES_NO, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
-  }
-
-  @Test
-  public void testStringEqualsBloomFilter() throws Exception {
-    PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf(
-        PredicateLeaf.Operator.EQUALS, PredicateLeaf.Type.STRING, "x", "str_15", null);
-    BloomFilterIO bf = new BloomFilterIO(10000);
-    for (int i = 20; i < 1000; i++) {
-      bf.addString("str_" + i);
-    }
-    ColumnStatistics cs = ColumnStatisticsImpl.deserialize(createStringStats("str_10", "str_200"));
-    assertEquals(TruthValue.NO_NULL, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
-
-    bf.addString("str_15");
-    assertEquals(TruthValue.YES_NO_NULL, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
-  }
-
-  @Test
-  public void testStringInBloomFilter() throws Exception {
-    List<Object> args = new ArrayList<Object>();
-    args.add("str_15");
-    args.add("str_19");
-    PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf
-        (PredicateLeaf.Operator.IN, PredicateLeaf.Type.STRING,
-            "x", null, args);
-    BloomFilterIO bf = new BloomFilterIO(10000);
-    for (int i = 20; i < 1000; i++) {
-      bf.addString("str_" + i);
-    }
-    ColumnStatistics cs = ColumnStatisticsImpl.deserialize(createStringStats("str_10", "str_200"));
-    assertEquals(TruthValue.NO_NULL, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
-
-    bf.addString("str_19");
-    assertEquals(TruthValue.YES_NO_NULL, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
-
-    bf.addString("str_15");
-    assertEquals(TruthValue.YES_NO_NULL, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
-  }
-
-  @Test
-  public void testDateWritableNullSafeEqualsBloomFilter() throws Exception {
-    PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf(
-        PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.DATE, "x",
-        new DateWritable(15).get(), null);
-    BloomFilterIO bf = new BloomFilterIO(10000);
-    for (int i = 20; i < 1000; i++) {
-      bf.addLong((new DateWritable(i)).getDays());
-    }
-    ColumnStatistics cs = ColumnStatisticsImpl.deserialize(createDateStats(10, 100));
-    assertEquals(TruthValue.NO, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
-
-    bf.addLong((new DateWritable(15)).getDays());
-    assertEquals(TruthValue.YES_NO, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
-  }
-
-  @Test
-  public void testDateWritableEqualsBloomFilter() throws Exception {
-    PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf(
-        PredicateLeaf.Operator.EQUALS, PredicateLeaf.Type.DATE, "x",
-        new DateWritable(15).get(), null);
-    BloomFilterIO bf = new BloomFilterIO(10000);
-    for (int i = 20; i < 1000; i++) {
-      bf.addLong((new DateWritable(i)).getDays());
-    }
-    ColumnStatistics cs = ColumnStatisticsImpl.deserialize(createDateStats(10, 100));
-    assertEquals(TruthValue.NO_NULL, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
-
-    bf.addLong((new DateWritable(15)).getDays());
-    assertEquals(TruthValue.YES_NO_NULL, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
-  }
-
-  @Test
-  public void testDateWritableInBloomFilter() throws Exception {
-    List<Object> args = new ArrayList<Object>();
-    args.add(new DateWritable(15).get());
-    args.add(new DateWritable(19).get());
-    PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf
-        (PredicateLeaf.Operator.IN, PredicateLeaf.Type.DATE,
-            "x", null, args);
-    BloomFilterIO bf = new BloomFilterIO(10000);
-    for (int i = 20; i < 1000; i++) {
-      bf.addLong((new DateWritable(i)).getDays());
-    }
-    ColumnStatistics cs = ColumnStatisticsImpl.deserialize(createDateStats(10, 100));
-    assertEquals(TruthValue.NO_NULL, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
-
-    bf.addLong((new DateWritable(19)).getDays());
-    assertEquals(TruthValue.YES_NO_NULL, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
-
-    bf.addLong((new DateWritable(15)).getDays());
-    assertEquals(TruthValue.YES_NO_NULL, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
-  }
-
-  @Test
-  public void testTimestampNullSafeEqualsBloomFilter() throws Exception {
-    PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf(
-        PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.TIMESTAMP, "x",
-        new Timestamp(15),
-        null);
-    BloomFilterIO bf = new BloomFilterIO(10000);
-    for (int i = 20; i < 1000; i++) {
-      bf.addLong((new Timestamp(i)).getTime());
-    }
-    ColumnStatistics cs = ColumnStatisticsImpl.deserialize(createTimestampStats(10, 100));
-    assertEquals(TruthValue.NO, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
-
-    bf.addLong((new Timestamp(15)).getTime());
-    assertEquals(TruthValue.YES_NO, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
-  }
-
-  @Test
-  public void testTimestampEqualsBloomFilter() throws Exception {
-    PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf(
-        PredicateLeaf.Operator.EQUALS, PredicateLeaf.Type.TIMESTAMP, "x", new Timestamp(15), null);
-    BloomFilterIO bf = new BloomFilterIO(10000);
-    for (int i = 20; i < 1000; i++) {
-      bf.addLong((new Timestamp(i)).getTime());
-    }
-    ColumnStatistics cs = ColumnStatisticsImpl.deserialize(createTimestampStats(10, 100));
-    assertEquals(TruthValue.NO_NULL, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
-
-    bf.addLong((new Timestamp(15)).getTime());
-    assertEquals(TruthValue.YES_NO_NULL, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
-  }
-
-  @Test
-  public void testTimestampInBloomFilter() throws Exception {
-    List<Object> args = new ArrayList<Object>();
-    args.add(new Timestamp(15));
-    args.add(new Timestamp(19));
-    PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf
-        (PredicateLeaf.Operator.IN, PredicateLeaf.Type.TIMESTAMP,
-            "x", null, args);
-    BloomFilterIO bf = new BloomFilterIO(10000);
-    for (int i = 20; i < 1000; i++) {
-      bf.addLong((new Timestamp(i)).getTime());
-    }
-    ColumnStatistics cs = ColumnStatisticsImpl.deserialize(createTimestampStats(10, 100));
-    assertEquals(TruthValue.NO_NULL, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
-
-    bf.addLong((new Timestamp(19)).getTime());
-    assertEquals(TruthValue.YES_NO_NULL, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
-
-    bf.addLong((new Timestamp(15)).getTime());
-    assertEquals(TruthValue.YES_NO_NULL, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
-  }
-
-  @Test
-  public void testDecimalNullSafeEqualsBloomFilter() throws Exception {
-    PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf(
-        PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.DECIMAL, "x",
-        new HiveDecimalWritable("15"),
-        null);
-    BloomFilterIO bf = new BloomFilterIO(10000);
-    for (int i = 20; i < 1000; i++) {
-      bf.addString(HiveDecimal.create(i).toString());
-    }
-    ColumnStatistics cs = ColumnStatisticsImpl.deserialize(createDecimalStats("10", "200"));
-    assertEquals(TruthValue.NO, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
-
-    bf.addString(HiveDecimal.create(15).toString());
-    assertEquals(TruthValue.YES_NO, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
-  }
-
-  @Test
-  public void testDecimalEqualsBloomFilter() throws Exception {
-    PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf(
-        PredicateLeaf.Operator.EQUALS, PredicateLeaf.Type.DECIMAL, "x",
-        new HiveDecimalWritable("15"),
-        null);
-    BloomFilterIO bf = new BloomFilterIO(10000);
-    for (int i = 20; i < 1000; i++) {
-      bf.addString(HiveDecimal.create(i).toString());
-    }
-    ColumnStatistics cs = ColumnStatisticsImpl.deserialize(createDecimalStats("10", "200"));
-    assertEquals(TruthValue.NO_NULL, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
-
-    bf.addString(HiveDecimal.create(15).toString());
-    assertEquals(TruthValue.YES_NO_NULL, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
-  }
-
-  @Test
-  public void testDecimalInBloomFilter() throws Exception {
-    List<Object> args = new ArrayList<Object>();
-    args.add(new HiveDecimalWritable("15"));
-    args.add(new HiveDecimalWritable("19"));
-    PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf
-        (PredicateLeaf.Operator.IN, PredicateLeaf.Type.DECIMAL,
-            "x", null, args);
-    BloomFilterIO bf = new BloomFilterIO(10000);
-    for (int i = 20; i < 1000; i++) {
-      bf.addString(HiveDecimal.create(i).toString());
-    }
-    ColumnStatistics cs = ColumnStatisticsImpl.deserialize(createDecimalStats("10", "200"));
-    assertEquals(TruthValue.NO_NULL, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
-
-    bf.addString(HiveDecimal.create(19).toString());
-    assertEquals(TruthValue.YES_NO_NULL, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
-
-    bf.addString(HiveDecimal.create(15).toString());
-    assertEquals(TruthValue.YES_NO_NULL, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
-  }
-
-  @Test
-  public void testNullsInBloomFilter() throws Exception {
-    List<Object> args = new ArrayList<Object>();
-    args.add(new HiveDecimalWritable("15"));
-    args.add(null);
-    args.add(new HiveDecimalWritable("19"));
-    PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf
-        (PredicateLeaf.Operator.IN, PredicateLeaf.Type.DECIMAL,
-            "x", null, args);
-    BloomFilterIO bf = new BloomFilterIO(10000);
-    for (int i = 20; i < 1000; i++) {
-      bf.addString(HiveDecimal.create(i).toString());
-    }
-    ColumnStatistics cs = ColumnStatisticsImpl.deserialize(createDecimalStats("10", "200", false));
-    // hasNull is false, so bloom filter should return NO
-    assertEquals(TruthValue.NO, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
-
-    cs = ColumnStatisticsImpl.deserialize(createDecimalStats("10", "200", true));
-    // hasNull is true, so bloom filter should return YES_NO_NULL
-    assertEquals(TruthValue.YES_NO_NULL, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
-
-    bf.addString(HiveDecimal.create(19).toString());
-    assertEquals(TruthValue.YES_NO_NULL, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
-
-    bf.addString(HiveDecimal.create(15).toString());
-    assertEquals(TruthValue.YES_NO_NULL, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
-  }
-
-  @Test
-  public void testClose() throws Exception {
-    DataReader mockedDataReader = mock(DataReader.class);
-    closeMockedRecordReader(mockedDataReader);
-
-    verify(mockedDataReader, atLeastOnce()).close();
-  }
-
-  @Test
-  public void testCloseWithException() throws Exception {
-    DataReader mockedDataReader = mock(DataReader.class);
-    doThrow(IOException.class).when(mockedDataReader).close();
-
-    try {
-      closeMockedRecordReader(mockedDataReader);
-      fail("Exception should have been thrown when Record Reader was closed");
-    } catch (IOException expected) {
-
-    }
-
-    verify(mockedDataReader, atLeastOnce()).close();
-  }
-
-  Path workDir = new Path(System.getProperty("test.tmp.dir",
-      "target" + File.separator + "test" + File.separator + "tmp"));
-
-  private void closeMockedRecordReader(DataReader mockedDataReader) throws IOException {
-    Configuration conf = new Configuration();
-    Path path = new Path(workDir, "empty.orc");
-    FileSystem.get(conf).delete(path, true);
-    Writer writer = OrcFile.createWriter(path, OrcFile.writerOptions(conf)
-        .setSchema(TypeDescription.createLong()));
-    writer.close();
-    Reader reader = OrcFile.createReader(path, OrcFile.readerOptions(conf));
-
-    RecordReader recordReader = reader.rowsOptions(new Reader.Options()
-        .dataReader(mockedDataReader));
-
-    recordReader.close();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestStreamName.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestStreamName.java b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestStreamName.java
deleted file mode 100644
index dfccd9a..0000000
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestStreamName.java
+++ /dev/null
@@ -1,50 +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.hive.ql.io.orc;
-
-import org.apache.orc.OrcProto;
-import org.apache.orc.impl.StreamName;
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-
-public class TestStreamName {
-
-  @Test
-  public void test1() throws Exception {
-    StreamName s1 = new StreamName(3, OrcProto.Stream.Kind.DATA);
-    StreamName s2 = new StreamName(3,
-        OrcProto.Stream.Kind.DICTIONARY_DATA);
-    StreamName s3 = new StreamName(5, OrcProto.Stream.Kind.DATA);
-    StreamName s4 = new StreamName(5,
-        OrcProto.Stream.Kind.DICTIONARY_DATA);
-    StreamName s1p = new StreamName(3, OrcProto.Stream.Kind.DATA);
-    assertEquals(true, s1.equals(s1));
-    assertEquals(false, s1.equals(s2));
-    assertEquals(false, s1.equals(s3));
-    assertEquals(true, s1.equals(s1p));
-    assertEquals(true, s1.compareTo(null) < 0);
-    assertEquals(false, s1.equals(null));
-    assertEquals(true, s1.compareTo(s2) < 0);
-    assertEquals(true, s2.compareTo(s3) < 0);
-    assertEquals(true, s3.compareTo(s4) < 0);
-    assertEquals(true, s4.compareTo(s1p) > 0);
-    assertEquals(0, s1p.compareTo(s1));
-  }
-}


[21/27] hive git commit: HIVE-11417. Move the ReaderImpl and RowReaderImpl to the ORC module, by making shims for the row by row reader. (omalley reviewed by prasanth_j)

Posted by om...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/orc/src/test/org/apache/orc/TestNewIntegerEncoding.java
----------------------------------------------------------------------
diff --git a/orc/src/test/org/apache/orc/TestNewIntegerEncoding.java b/orc/src/test/org/apache/orc/TestNewIntegerEncoding.java
new file mode 100644
index 0000000..526dd81
--- /dev/null
+++ b/orc/src/test/org/apache/orc/TestNewIntegerEncoding.java
@@ -0,0 +1,1373 @@
+/**
+ * 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.orc;
+
+import static junit.framework.Assert.assertEquals;
+
+import java.io.File;
+import java.sql.Timestamp;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Random;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+import com.google.common.collect.Lists;
+import com.google.common.primitives.Longs;
+
+@RunWith(value = Parameterized.class)
+public class TestNewIntegerEncoding {
+
+  private OrcFile.EncodingStrategy encodingStrategy;
+
+  public TestNewIntegerEncoding( OrcFile.EncodingStrategy es) {
+    this.encodingStrategy = es;
+  }
+
+  @Parameters
+  public static Collection<Object[]> data() {
+    Object[][] data = new Object[][] { {  OrcFile.EncodingStrategy.COMPRESSION },
+        {  OrcFile.EncodingStrategy.SPEED } };
+    return Arrays.asList(data);
+  }
+
+  public static class TSRow {
+    Timestamp ts;
+
+    public TSRow(Timestamp ts) {
+      this.ts = ts;
+    }
+  }
+
+  public static TypeDescription getRowSchema() {
+    return TypeDescription.createStruct()
+        .addField("int1", TypeDescription.createInt())
+        .addField("long1", TypeDescription.createLong());
+  }
+
+  public static void appendRow(VectorizedRowBatch batch,
+                               int int1, long long1) {
+    int row = batch.size++;
+    ((LongColumnVector) batch.cols[0]).vector[row] = int1;
+    ((LongColumnVector) batch.cols[1]).vector[row] = long1;
+  }
+
+  public static void appendLong(VectorizedRowBatch batch,
+                                long long1) {
+    int row = batch.size++;
+    ((LongColumnVector) batch.cols[0]).vector[row] = long1;
+  }
+
+  Path workDir = new Path(System.getProperty("test.tmp.dir", "target"
+      + File.separator + "test" + File.separator + "tmp"));
+
+  Configuration conf;
+  FileSystem fs;
+  Path testFilePath;
+
+  @Rule
+  public TestName testCaseName = new TestName();
+
+  @Before
+  public void openFileSystem() throws Exception {
+    conf = new Configuration();
+    fs = FileSystem.getLocal(conf);
+    testFilePath = new Path(workDir, "TestOrcFile."
+        + testCaseName.getMethodName() + ".orc");
+    fs.delete(testFilePath, false);
+  }
+
+  @Test
+  public void testBasicRow() throws Exception {
+    TypeDescription schema= getRowSchema();
+    Writer writer = OrcFile.createWriter(testFilePath,
+                                         OrcFile.writerOptions(conf)
+                                         .setSchema(schema)
+                                         .stripeSize(100000)
+                                         .compress(CompressionKind.NONE)
+                                         .bufferSize(10000)
+                                         .encodingStrategy(encodingStrategy));
+    VectorizedRowBatch batch = schema.createRowBatch();
+    appendRow(batch, 111, 1111L);
+    appendRow(batch, 111, 1111L);
+    appendRow(batch, 111, 1111L);
+    writer.addRowBatch(batch);
+    writer.close();
+
+    Reader reader = OrcFile.createReader(testFilePath,
+        OrcFile.readerOptions(conf).filesystem(fs));
+    RecordReader rows = reader.rows();
+    batch = reader.getSchema().createRowBatch();
+    while (rows.nextBatch(batch)) {
+      for(int r=0; r < batch.size; ++r) {
+        assertEquals(111, ((LongColumnVector) batch.cols[0]).vector[r]);
+        assertEquals(1111, ((LongColumnVector) batch.cols[1]).vector[r]);
+      }
+    }
+  }
+
+  @Test
+  public void testBasicOld() throws Exception {
+    TypeDescription schema = TypeDescription.createLong();
+    long[] inp = new long[] { 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 2, 3, 4, 5, 6,
+        7, 8, 9, 10, 1, 1, 1, 1, 1, 1, 10, 9, 7, 6, 5, 4, 3, 2, 1, 1, 1, 1, 1,
+        2, 5, 1, 3, 7, 1, 9, 2, 6, 3, 7, 1, 9, 2, 6, 3, 7, 1, 9, 2, 6, 3, 7, 1,
+        9, 2, 6, 3, 7, 1, 9, 2, 6, 2000, 2, 1, 1, 1, 1, 1, 3, 7, 1, 9, 2, 6, 1,
+        1, 1, 1, 1 };
+    List<Long> input = Lists.newArrayList(Longs.asList(inp));
+    Writer writer = OrcFile.createWriter(testFilePath,
+                                         OrcFile.writerOptions(conf)
+                                         .setSchema(schema)
+                                         .compress(CompressionKind.NONE)
+                                         .version(OrcFile.Version.V_0_11)
+                                         .bufferSize(10000)
+                                         .encodingStrategy(encodingStrategy));
+    VectorizedRowBatch batch = schema.createRowBatch();
+    for(Long l : input) {
+      appendLong(batch, l);
+    }
+    writer.addRowBatch(batch);
+    writer.close();
+
+    Reader reader = OrcFile.createReader(testFilePath,
+        OrcFile.readerOptions(conf).filesystem(fs));
+    RecordReader rows = reader.rows();
+    int idx = 0;
+    batch = reader.getSchema().createRowBatch();
+    while (rows.nextBatch(batch)) {
+      for(int r=0; r < batch.size; ++r) {
+        assertEquals(input.get(idx++).longValue(),
+            ((LongColumnVector) batch.cols[0]).vector[r]);
+      }
+    }
+  }
+
+  @Test
+  public void testBasicNew() throws Exception {
+    TypeDescription schema = TypeDescription.createLong();
+
+    long[] inp = new long[] { 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 2, 3, 4, 5, 6,
+        7, 8, 9, 10, 1, 1, 1, 1, 1, 1, 10, 9, 7, 6, 5, 4, 3, 2, 1, 1, 1, 1, 1,
+        2, 5, 1, 3, 7, 1, 9, 2, 6, 3, 7, 1, 9, 2, 6, 3, 7, 1, 9, 2, 6, 3, 7, 1,
+        9, 2, 6, 3, 7, 1, 9, 2, 6, 2000, 2, 1, 1, 1, 1, 1, 3, 7, 1, 9, 2, 6, 1,
+        1, 1, 1, 1 };
+    List<Long> input = Lists.newArrayList(Longs.asList(inp));
+
+    Writer writer = OrcFile.createWriter(testFilePath,
+        OrcFile.writerOptions(conf)
+            .setSchema(schema)
+            .stripeSize(100000)
+            .compress(CompressionKind.NONE)
+            .bufferSize(10000)
+            .encodingStrategy(encodingStrategy));
+    VectorizedRowBatch batch = schema.createRowBatch();
+    for(Long l : input) {
+      appendLong(batch, l);
+    }
+    writer.addRowBatch(batch);
+    writer.close();
+
+    Reader reader = OrcFile.createReader(testFilePath,
+        OrcFile.readerOptions(conf).filesystem(fs));
+    RecordReader rows = reader.rows();
+    int idx = 0;
+    batch = reader.getSchema().createRowBatch();
+    while (rows.nextBatch(batch)) {
+      for(int r=0; r < batch.size; ++r) {
+        assertEquals(input.get(idx++).longValue(),
+            ((LongColumnVector) batch.cols[0]).vector[r]);
+      }
+    }
+  }
+  
+  @Test
+  public void testBasicDelta1() throws Exception {
+    TypeDescription schema = TypeDescription.createLong();
+
+    long[] inp = new long[] { -500, -400, -350, -325, -310 };
+    List<Long> input = Lists.newArrayList(Longs.asList(inp));
+
+    Writer writer = OrcFile.createWriter(testFilePath,
+        OrcFile.writerOptions(conf)
+            .setSchema(schema)
+            .stripeSize(100000)
+            .compress(CompressionKind.NONE)
+            .bufferSize(10000)
+            .encodingStrategy(encodingStrategy));
+    VectorizedRowBatch batch = schema.createRowBatch();
+    for(Long l : input) {
+      appendLong(batch, l);
+    }
+    writer.addRowBatch(batch);
+    writer.close();
+
+    Reader reader = OrcFile.createReader(testFilePath,
+        OrcFile.readerOptions(conf).filesystem(fs));
+    RecordReader rows = reader.rows();
+    batch = reader.getSchema().createRowBatch();
+    int idx = 0;
+    while (rows.nextBatch(batch)) {
+      for(int r=0; r < batch.size; ++r) {
+        assertEquals(input.get(idx++).longValue(),
+            ((LongColumnVector) batch.cols[0]).vector[r]);
+      }
+    }
+  }
+
+  @Test
+  public void testBasicDelta2() throws Exception {
+    TypeDescription schema = TypeDescription.createLong();
+
+    long[] inp = new long[] { -500, -600, -650, -675, -710 };
+    List<Long> input = Lists.newArrayList(Longs.asList(inp));
+
+    Writer writer = OrcFile.createWriter(testFilePath,
+        OrcFile.writerOptions(conf)
+            .setSchema(schema)
+            .stripeSize(100000)
+            .compress(CompressionKind.NONE)
+            .bufferSize(10000)
+            .encodingStrategy(encodingStrategy));
+    VectorizedRowBatch batch = schema.createRowBatch();
+    for(Long l : input) {
+      appendLong(batch, l);
+    }
+    writer.addRowBatch(batch);
+    writer.close();
+
+    Reader reader = OrcFile.createReader(testFilePath,
+        OrcFile.readerOptions(conf).filesystem(fs));
+    RecordReader rows = reader.rows();
+    batch = reader.getSchema().createRowBatch();
+    int idx = 0;
+    while (rows.nextBatch(batch)) {
+      for(int r=0; r < batch.size; ++r) {
+        assertEquals(input.get(idx++).longValue(),
+            ((LongColumnVector) batch.cols[0]).vector[r]);
+      }
+    }
+  }
+
+  @Test
+  public void testBasicDelta3() throws Exception {
+    TypeDescription schema = TypeDescription.createLong();
+
+    long[] inp = new long[] { 500, 400, 350, 325, 310 };
+    List<Long> input = Lists.newArrayList(Longs.asList(inp));
+
+    Writer writer = OrcFile.createWriter(testFilePath,
+        OrcFile.writerOptions(conf)
+            .setSchema(schema)
+            .stripeSize(100000)
+            .compress(CompressionKind.NONE)
+            .bufferSize(10000)
+            .encodingStrategy(encodingStrategy));
+    VectorizedRowBatch batch = schema.createRowBatch();
+    for(Long l : input) {
+      appendLong(batch, l);
+    }
+    writer.addRowBatch(batch);
+    writer.close();
+
+    Reader reader = OrcFile.createReader(testFilePath,
+        OrcFile.readerOptions(conf).filesystem(fs));
+    RecordReader rows = reader.rows();
+    batch = reader.getSchema().createRowBatch();
+    int idx = 0;
+    while (rows.nextBatch(batch)) {
+      for(int r=0; r < batch.size; ++r) {
+        assertEquals(input.get(idx++).longValue(),
+            ((LongColumnVector) batch.cols[0]).vector[r]);
+      }
+    }
+  }
+
+  @Test
+  public void testBasicDelta4() throws Exception {
+    TypeDescription schema = TypeDescription.createLong();
+
+    long[] inp = new long[] { 500, 600, 650, 675, 710 };
+    List<Long> input = Lists.newArrayList(Longs.asList(inp));
+
+    Writer writer = OrcFile.createWriter(testFilePath,
+        OrcFile.writerOptions(conf)
+            .setSchema(schema)
+            .stripeSize(100000)
+            .compress(CompressionKind.NONE)
+            .bufferSize(10000)
+            .encodingStrategy(encodingStrategy));
+    VectorizedRowBatch batch = schema.createRowBatch();
+    for(Long l : input) {
+      appendLong(batch, l);
+    }
+    writer.addRowBatch(batch);
+    writer.close();
+
+    Reader reader = OrcFile.createReader(testFilePath,
+        OrcFile.readerOptions(conf).filesystem(fs));
+    RecordReader rows = reader.rows();
+    batch = reader.getSchema().createRowBatch();
+    int idx = 0;
+    while (rows.nextBatch(batch)) {
+      for(int r=0; r < batch.size; ++r) {
+        assertEquals(input.get(idx++).longValue(),
+            ((LongColumnVector) batch.cols[0]).vector[r]);
+      }
+    }
+  }
+
+  @Test
+  public void testDeltaOverflow() throws Exception {
+    TypeDescription schema = TypeDescription.createLong();
+
+    long[] inp = new long[]{4513343538618202719l, 4513343538618202711l,
+        2911390882471569739l,
+        -9181829309989854913l};
+    List<Long> input = Lists.newArrayList(Longs.asList(inp));
+
+    Writer writer = OrcFile.createWriter(
+        testFilePath,
+        OrcFile.writerOptions(conf).setSchema(schema).stripeSize(100000)
+            .compress(CompressionKind.NONE).bufferSize(10000));
+    VectorizedRowBatch batch = schema.createRowBatch();
+    for (Long l : input) {
+      appendLong(batch, l);
+    }
+    writer.addRowBatch(batch);
+    writer.close();
+
+    Reader reader = OrcFile
+        .createReader(testFilePath, OrcFile.readerOptions(conf).filesystem(fs));
+    RecordReader rows = reader.rows();
+    batch = reader.getSchema().createRowBatch();
+    int idx = 0;
+    while (rows.nextBatch(batch)) {
+      for(int r=0; r < batch.size; ++r) {
+        assertEquals(input.get(idx++).longValue(),
+            ((LongColumnVector) batch.cols[0]).vector[r]);
+      }
+    }
+  }
+
+  @Test
+  public void testDeltaOverflow2() throws Exception {
+    TypeDescription schema = TypeDescription.createLong();
+
+    long[] inp = new long[]{Long.MAX_VALUE, 4513343538618202711l,
+        2911390882471569739l,
+        Long.MIN_VALUE};
+    List<Long> input = Lists.newArrayList(Longs.asList(inp));
+
+    Writer writer = OrcFile.createWriter(
+        testFilePath,
+        OrcFile.writerOptions(conf).setSchema(schema).stripeSize(100000)
+            .compress(CompressionKind.NONE).bufferSize(10000));
+    VectorizedRowBatch batch = schema.createRowBatch();
+    for (Long l : input) {
+      appendLong(batch, l);
+    }
+    writer.addRowBatch(batch);
+    writer.close();
+
+    Reader reader = OrcFile
+        .createReader(testFilePath, OrcFile.readerOptions(conf).filesystem(fs));
+    RecordReader rows = reader.rows();
+    batch = reader.getSchema().createRowBatch();
+    int idx = 0;
+    while (rows.nextBatch(batch)) {
+      for(int r=0; r < batch.size; ++r) {
+        assertEquals(input.get(idx++).longValue(),
+            ((LongColumnVector) batch.cols[0]).vector[r]);
+      }
+    }
+  }
+
+  @Test
+  public void testDeltaOverflow3() throws Exception {
+    TypeDescription schema = TypeDescription.createLong();
+
+    long[] inp = new long[]{-4513343538618202711l, -2911390882471569739l, -2,
+        Long.MAX_VALUE};
+    List<Long> input = Lists.newArrayList(Longs.asList(inp));
+
+    Writer writer = OrcFile.createWriter(
+        testFilePath,
+        OrcFile.writerOptions(conf).setSchema(schema).stripeSize(100000)
+            .compress(CompressionKind.NONE).bufferSize(10000));
+    VectorizedRowBatch batch = schema.createRowBatch();
+    for (Long l : input) {
+      appendLong(batch, l);
+    }
+    writer.addRowBatch(batch);
+    writer.close();
+
+    Reader reader = OrcFile
+        .createReader(testFilePath, OrcFile.readerOptions(conf).filesystem(fs));
+    RecordReader rows = reader.rows();
+    batch = reader.getSchema().createRowBatch();
+    int idx = 0;
+    while (rows.nextBatch(batch)) {
+      for(int r=0; r < batch.size; ++r) {
+        assertEquals(input.get(idx++).longValue(),
+            ((LongColumnVector) batch.cols[0]).vector[r]);
+      }
+    }
+  }
+
+  @Test
+  public void testIntegerMin() throws Exception {
+    TypeDescription schema = TypeDescription.createLong();
+
+    List<Long> input = Lists.newArrayList();
+    input.add((long) Integer.MIN_VALUE);
+
+    Writer writer = OrcFile.createWriter(testFilePath,
+        OrcFile.writerOptions(conf)
+            .setSchema(schema)
+            .stripeSize(100000)
+            .bufferSize(10000)
+            .encodingStrategy(encodingStrategy));
+    VectorizedRowBatch batch = schema.createRowBatch();
+    for(Long l : input) {
+      appendLong(batch, l);
+    }
+    writer.addRowBatch(batch);
+    writer.close();
+
+    Reader reader = OrcFile.createReader(testFilePath,
+        OrcFile.readerOptions(conf).filesystem(fs));
+    RecordReader rows = reader.rows();
+    batch = reader.getSchema().createRowBatch();
+    int idx = 0;
+    while (rows.nextBatch(batch)) {
+      for(int r=0; r < batch.size; ++r) {
+        assertEquals(input.get(idx++).longValue(),
+            ((LongColumnVector) batch.cols[0]).vector[r]);
+      }
+    }
+  }
+
+  @Test
+  public void testIntegerMax() throws Exception {
+    TypeDescription schema = TypeDescription.createLong();
+
+    List<Long> input = Lists.newArrayList();
+    input.add((long) Integer.MAX_VALUE);
+
+    Writer writer = OrcFile.createWriter(testFilePath,
+        OrcFile.writerOptions(conf)
+            .setSchema(schema)
+            .stripeSize(100000)
+            .compress(CompressionKind.NONE)
+            .bufferSize(10000)
+            .encodingStrategy(encodingStrategy));
+    VectorizedRowBatch batch = schema.createRowBatch();
+    for(Long l : input) {
+      appendLong(batch, l);
+    }
+    writer.addRowBatch(batch);
+    writer.close();
+
+    Reader reader = OrcFile.createReader(testFilePath,
+        OrcFile.readerOptions(conf).filesystem(fs));
+    RecordReader rows = reader.rows();
+    batch = reader.getSchema().createRowBatch();
+    int idx = 0;
+    while (rows.nextBatch(batch)) {
+      for(int r=0; r < batch.size; ++r) {
+        assertEquals(input.get(idx++).longValue(),
+            ((LongColumnVector) batch.cols[0]).vector[r]);
+      }
+    }
+  }
+
+  @Test
+  public void testLongMin() throws Exception {
+    TypeDescription schema = TypeDescription.createLong();
+
+    List<Long> input = Lists.newArrayList();
+    input.add(Long.MIN_VALUE);
+
+    Writer writer = OrcFile.createWriter(testFilePath,
+        OrcFile.writerOptions(conf)
+            .setSchema(schema)
+            .stripeSize(100000)
+            .compress(CompressionKind.NONE)
+            .bufferSize(10000)
+            .encodingStrategy(encodingStrategy));
+    VectorizedRowBatch batch = schema.createRowBatch();
+    for(Long l : input) {
+      appendLong(batch, l);
+    }
+    writer.addRowBatch(batch);
+    writer.close();
+
+    Reader reader = OrcFile.createReader(testFilePath,
+        OrcFile.readerOptions(conf).filesystem(fs));
+    RecordReader rows = reader.rows();
+    batch = reader.getSchema().createRowBatch();
+    int idx = 0;
+    while (rows.nextBatch(batch)) {
+      for(int r=0; r < batch.size; ++r) {
+        assertEquals(input.get(idx++).longValue(),
+            ((LongColumnVector) batch.cols[0]).vector[r]);
+      }
+    }
+  }
+
+  @Test
+  public void testLongMax() throws Exception {
+    TypeDescription schema = TypeDescription.createLong();
+
+    List<Long> input = Lists.newArrayList();
+    input.add(Long.MAX_VALUE);
+
+    Writer writer = OrcFile.createWriter(testFilePath,
+        OrcFile.writerOptions(conf)
+            .setSchema(schema)
+            .stripeSize(100000)
+            .compress(CompressionKind.NONE)
+            .bufferSize(10000)
+            .encodingStrategy(encodingStrategy));
+    VectorizedRowBatch batch = schema.createRowBatch();
+    for(Long l : input) {
+      appendLong(batch, l);
+    }
+    writer.addRowBatch(batch);
+    writer.close();
+
+    Reader reader = OrcFile.createReader(testFilePath,
+        OrcFile.readerOptions(conf).filesystem(fs));
+    RecordReader rows = reader.rows();
+    batch = reader.getSchema().createRowBatch();
+    int idx = 0;
+    while (rows.nextBatch(batch)) {
+      for(int r=0; r < batch.size; ++r) {
+        assertEquals(input.get(idx++).longValue(),
+            ((LongColumnVector) batch.cols[0]).vector[r]);
+      }
+    }
+  }
+
+  @Test
+  public void testRandomInt() throws Exception {
+    TypeDescription schema = TypeDescription.createLong();
+
+    List<Long> input = Lists.newArrayList();
+    Random rand = new Random();
+    for(int i = 0; i < 100000; i++) {
+      input.add((long) rand.nextInt());
+    }
+
+    Writer writer = OrcFile.createWriter(testFilePath,
+        OrcFile.writerOptions(conf)
+            .setSchema(schema)
+            .stripeSize(100000)
+            .compress(CompressionKind.NONE)
+            .bufferSize(10000)
+            .encodingStrategy(encodingStrategy));
+    VectorizedRowBatch batch = schema.createRowBatch(100000);
+    for(Long l : input) {
+      appendLong(batch, l);
+    }
+    writer.addRowBatch(batch);
+    writer.close();
+
+    Reader reader = OrcFile.createReader(testFilePath,
+        OrcFile.readerOptions(conf).filesystem(fs));
+    RecordReader rows = reader.rows();
+    batch = reader.getSchema().createRowBatch();
+    int idx = 0;
+    while (rows.nextBatch(batch)) {
+      for(int r=0; r < batch.size; ++r) {
+        assertEquals(input.get(idx++).longValue(),
+            ((LongColumnVector) batch.cols[0]).vector[r]);
+      }
+    }
+  }
+
+  @Test
+  public void testRandomLong() throws Exception {
+    TypeDescription schema = TypeDescription.createLong();
+
+    List<Long> input = Lists.newArrayList();
+    Random rand = new Random();
+    for(int i = 0; i < 100000; i++) {
+      input.add(rand.nextLong());
+    }
+
+    Writer writer = OrcFile.createWriter(testFilePath,
+        OrcFile.writerOptions(conf)
+            .setSchema(schema)
+            .stripeSize(100000)
+            .compress(CompressionKind.NONE)
+            .bufferSize(10000)
+            .encodingStrategy(encodingStrategy));
+    VectorizedRowBatch batch = schema.createRowBatch(100000);
+    for(Long l : input) {
+      appendLong(batch, l);
+    }
+    writer.addRowBatch(batch);
+    writer.close();
+
+    Reader reader = OrcFile.createReader(testFilePath,
+        OrcFile.readerOptions(conf).filesystem(fs));
+    RecordReader rows = reader.rows();
+    batch = reader.getSchema().createRowBatch();
+    int idx = 0;
+    while (rows.nextBatch(batch)) {
+      for(int r=0; r < batch.size; ++r) {
+        assertEquals(input.get(idx++).longValue(),
+            ((LongColumnVector) batch.cols[0]).vector[r]);
+      }
+    }
+  }
+
+  @Test
+  public void testPatchedBaseNegativeMin() throws Exception {
+    TypeDescription schema = TypeDescription.createLong();
+
+    long[] inp = new long[] { 20, 2, 3, 2, 1, 3, 17, 71, 35, 2, 1, 139, 2, 2,
+        3, 1783, 475, 2, 1, 1, 3, 1, 3, 2, 32, 1, 2, 3, 1, 8, 30, 1, 3, 414, 1,
+        1, 135, 3, 3, 1, 414, 2, 1, 2, 2, 594, 2, 5, 6, 4, 11, 1, 2, 2, 1, 1,
+        52, 4, 1, 2, 7, 1, 17, 334, 1, 2, 1, 2, 2, 6, 1, 266, 1, 2, 217, 2, 6,
+        2, 13, 2, 2, 1, 2, 3, 5, 1, 2, 1, 7244, 11813, 1, 33, 2, -13, 1, 2, 3,
+        13, 1, 92, 3, 13, 5, 14, 9, 141, 12, 6, 15, 25, 1, 1, 1, 46, 2, 1, 1,
+        141, 3, 1, 1, 1, 1, 2, 1, 4, 34, 5, 78, 8, 1, 2, 2, 1, 9, 10, 2, 1, 4,
+        13, 1, 5, 4, 4, 19, 5, 1, 1, 1, 68, 33, 399, 1, 1885, 25, 5, 2, 4, 1,
+        1, 2, 16, 1, 2966, 3, 1, 1, 25501, 1, 1, 1, 66, 1, 3, 8, 131, 14, 5, 1,
+        2, 2, 1, 1, 8, 1, 1, 2, 1, 5, 9, 2, 3, 112, 13, 2, 2, 1, 5, 10, 3, 1,
+        1, 13, 2, 3, 4, 1, 3, 1, 1, 2, 1, 1, 2, 4, 2, 207, 1, 1, 2, 4, 3, 3, 2,
+        2, 16 };
+    List<Long> input = Lists.newArrayList(Longs.asList(inp));
+
+    Writer writer = OrcFile.createWriter(testFilePath,
+        OrcFile.writerOptions(conf)
+            .setSchema(schema)
+            .stripeSize(100000)
+            .compress(CompressionKind.NONE)
+            .bufferSize(10000)
+            .encodingStrategy(encodingStrategy));
+    VectorizedRowBatch batch = schema.createRowBatch();
+    for(Long l : input) {
+      appendLong(batch, l);
+    }
+    writer.addRowBatch(batch);
+    writer.close();
+
+    Reader reader = OrcFile.createReader(testFilePath,
+        OrcFile.readerOptions(conf).filesystem(fs));
+    RecordReader rows = reader.rows();
+    batch = reader.getSchema().createRowBatch();
+    int idx = 0;
+    while (rows.nextBatch(batch)) {
+      for(int r=0; r < batch.size; ++r) {
+        assertEquals(input.get(idx++).longValue(),
+            ((LongColumnVector) batch.cols[0]).vector[r]);
+      }
+    }
+  }
+
+  @Test
+  public void testPatchedBaseNegativeMin2() throws Exception {
+    TypeDescription schema = TypeDescription.createLong();
+
+    long[] inp = new long[] { 20, 2, 3, 2, 1, 3, 17, 71, 35, 2, 1, 139, 2, 2,
+        3, 1783, 475, 2, 1, 1, 3, 1, 3, 2, 32, 1, 2, 3, 1, 8, 30, 1, 3, 414, 1,
+        1, 135, 3, 3, 1, 414, 2, 1, 2, 2, 594, 2, 5, 6, 4, 11, 1, 2, 2, 1, 1,
+        52, 4, 1, 2, 7, 1, 17, 334, 1, 2, 1, 2, 2, 6, 1, 266, 1, 2, 217, 2, 6,
+        2, 13, 2, 2, 1, 2, 3, 5, 1, 2, 1, 7244, 11813, 1, 33, 2, -1, 1, 2, 3,
+        13, 1, 92, 3, 13, 5, 14, 9, 141, 12, 6, 15, 25, 1, 1, 1, 46, 2, 1, 1,
+        141, 3, 1, 1, 1, 1, 2, 1, 4, 34, 5, 78, 8, 1, 2, 2, 1, 9, 10, 2, 1, 4,
+        13, 1, 5, 4, 4, 19, 5, 1, 1, 1, 68, 33, 399, 1, 1885, 25, 5, 2, 4, 1,
+        1, 2, 16, 1, 2966, 3, 1, 1, 25501, 1, 1, 1, 66, 1, 3, 8, 131, 14, 5, 1,
+        2, 2, 1, 1, 8, 1, 1, 2, 1, 5, 9, 2, 3, 112, 13, 2, 2, 1, 5, 10, 3, 1,
+        1, 13, 2, 3, 4, 1, 3, 1, 1, 2, 1, 1, 2, 4, 2, 207, 1, 1, 2, 4, 3, 3, 2,
+        2, 16 };
+    List<Long> input = Lists.newArrayList(Longs.asList(inp));
+
+    Writer writer = OrcFile.createWriter(testFilePath,
+        OrcFile.writerOptions(conf)
+            .setSchema(schema)
+            .stripeSize(100000)
+            .compress(CompressionKind.NONE)
+            .bufferSize(10000)
+            .encodingStrategy(encodingStrategy));
+    VectorizedRowBatch batch = schema.createRowBatch();
+    for(Long l : input) {
+      appendLong(batch, l);
+    }
+    writer.addRowBatch(batch);
+    writer.close();
+
+    Reader reader = OrcFile.createReader(testFilePath,
+        OrcFile.readerOptions(conf).filesystem(fs));
+    RecordReader rows = reader.rows();
+    batch = reader.getSchema().createRowBatch();
+    int idx = 0;
+    while (rows.nextBatch(batch)) {
+      for(int r=0; r < batch.size; ++r) {
+        assertEquals(input.get(idx++).longValue(),
+            ((LongColumnVector) batch.cols[0]).vector[r]);
+      }
+    }
+  }
+
+  @Test
+  public void testPatchedBaseNegativeMin3() throws Exception {
+    TypeDescription schema = TypeDescription.createLong();
+
+    long[] inp = new long[] { 20, 2, 3, 2, 1, 3, 17, 71, 35, 2, 1, 139, 2, 2,
+        3, 1783, 475, 2, 1, 1, 3, 1, 3, 2, 32, 1, 2, 3, 1, 8, 30, 1, 3, 414, 1,
+        1, 135, 3, 3, 1, 414, 2, 1, 2, 2, 594, 2, 5, 6, 4, 11, 1, 2, 2, 1, 1,
+        52, 4, 1, 2, 7, 1, 17, 334, 1, 2, 1, 2, 2, 6, 1, 266, 1, 2, 217, 2, 6,
+        2, 13, 2, 2, 1, 2, 3, 5, 1, 2, 1, 7244, 11813, 1, 33, 2, 0, 1, 2, 3,
+        13, 1, 92, 3, 13, 5, 14, 9, 141, 12, 6, 15, 25, 1, 1, 1, 46, 2, 1, 1,
+        141, 3, 1, 1, 1, 1, 2, 1, 4, 34, 5, 78, 8, 1, 2, 2, 1, 9, 10, 2, 1, 4,
+        13, 1, 5, 4, 4, 19, 5, 1, 1, 1, 68, 33, 399, 1, 1885, 25, 5, 2, 4, 1,
+        1, 2, 16, 1, 2966, 3, 1, 1, 25501, 1, 1, 1, 66, 1, 3, 8, 131, 14, 5, 1,
+        2, 2, 1, 1, 8, 1, 1, 2, 1, 5, 9, 2, 3, 112, 13, 2, 2, 1, 5, 10, 3, 1,
+        1, 13, 2, 3, 4, 1, 3, 1, 1, 2, 1, 1, 2, 4, 2, 207, 1, 1, 2, 4, 3, 3, 2,
+        2, 16 };
+    List<Long> input = Lists.newArrayList(Longs.asList(inp));
+
+    Writer writer = OrcFile.createWriter(testFilePath,
+        OrcFile.writerOptions(conf)
+            .setSchema(schema)
+            .stripeSize(100000)
+            .compress(CompressionKind.NONE)
+            .bufferSize(10000)
+            .encodingStrategy(encodingStrategy));
+    VectorizedRowBatch batch = schema.createRowBatch();
+    for(Long l : input) {
+      appendLong(batch, l);
+    }
+    writer.addRowBatch(batch);
+    writer.close();
+
+    Reader reader = OrcFile.createReader(testFilePath,
+        OrcFile.readerOptions(conf).filesystem(fs));
+    RecordReader rows = reader.rows();
+    batch = reader.getSchema().createRowBatch();
+    int idx = 0;
+    while (rows.nextBatch(batch)) {
+      for(int r=0; r < batch.size; ++r) {
+        assertEquals(input.get(idx++).longValue(),
+            ((LongColumnVector) batch.cols[0]).vector[r]);
+      }
+    }
+  }
+
+  @Test
+  public void testPatchedBaseNegativeMin4() throws Exception {
+    TypeDescription schema = TypeDescription.createLong();
+
+    long[] inp = new long[] { 13, 13, 11, 8, 13, 10, 10, 11, 11, 14, 11, 7, 13,
+        12, 12, 11, 15, 12, 12, 9, 8, 10, 13, 11, 8, 6, 5, 6, 11, 7, 15, 10, 7,
+        6, 8, 7, 9, 9, 11, 33, 11, 3, 7, 4, 6, 10, 14, 12, 5, 14, 7, 6 };
+    List<Long> input = Lists.newArrayList(Longs.asList(inp));
+
+    Writer writer = OrcFile.createWriter(testFilePath,
+        OrcFile.writerOptions(conf)
+            .setSchema(schema)
+            .stripeSize(100000)
+            .compress(CompressionKind.NONE)
+            .bufferSize(10000)
+            .encodingStrategy(encodingStrategy));
+    VectorizedRowBatch batch = schema.createRowBatch();
+    for(Long l : input) {
+      appendLong(batch, l);
+    }
+    writer.addRowBatch(batch);
+    writer.close();
+
+    Reader reader = OrcFile.createReader(testFilePath,
+        OrcFile.readerOptions(conf).filesystem(fs));
+    RecordReader rows = reader.rows();
+    batch = reader.getSchema().createRowBatch();
+    int idx = 0;
+    while (rows.nextBatch(batch)) {
+      for(int r=0; r < batch.size; ++r) {
+        assertEquals(input.get(idx++).longValue(),
+            ((LongColumnVector) batch.cols[0]).vector[r]);
+      }
+    }
+  }
+
+  @Test
+  public void testPatchedBaseAt0() throws Exception {
+    TypeDescription schema = TypeDescription.createLong();
+
+    List<Long> input = Lists.newArrayList();
+    Random rand = new Random();
+    for(int i = 0; i < 5120; i++) {
+      input.add((long) rand.nextInt(100));
+    }
+    input.set(0, 20000L);
+
+    Writer writer = OrcFile.createWriter(testFilePath,
+        OrcFile.writerOptions(conf)
+            .setSchema(schema)
+            .stripeSize(100000)
+            .compress(CompressionKind.NONE)
+            .bufferSize(10000)
+            .encodingStrategy(encodingStrategy));
+    VectorizedRowBatch batch = schema.createRowBatch(5120);
+    for(Long l : input) {
+      appendLong(batch, l);
+    }
+    writer.addRowBatch(batch);
+    writer.close();
+
+    Reader reader = OrcFile.createReader(testFilePath,
+        OrcFile.readerOptions(conf).filesystem(fs));
+    RecordReader rows = reader.rows();
+    batch = reader.getSchema().createRowBatch();
+    int idx = 0;
+    while (rows.nextBatch(batch)) {
+      for(int r=0; r < batch.size; ++r) {
+        assertEquals(input.get(idx++).longValue(),
+            ((LongColumnVector) batch.cols[0]).vector[r]);
+      }
+    }
+  }
+
+  @Test
+  public void testPatchedBaseAt1() throws Exception {
+    TypeDescription schema = TypeDescription.createLong();
+
+    List<Long> input = Lists.newArrayList();
+    Random rand = new Random();
+    for(int i = 0; i < 5120; i++) {
+      input.add((long) rand.nextInt(100));
+    }
+    input.set(1, 20000L);
+
+    Writer writer = OrcFile.createWriter(testFilePath,
+        OrcFile.writerOptions(conf)
+            .setSchema(schema)
+            .stripeSize(100000)
+            .compress(CompressionKind.NONE)
+            .bufferSize(10000)
+            .encodingStrategy(encodingStrategy));
+    VectorizedRowBatch batch = schema.createRowBatch(5120);
+    for(Long l : input) {
+      appendLong(batch, l);
+    }
+    writer.addRowBatch(batch);
+    writer.close();
+
+    Reader reader = OrcFile.createReader(testFilePath,
+        OrcFile.readerOptions(conf).filesystem(fs));
+    RecordReader rows = reader.rows();
+    int idx = 0;
+    while (rows.nextBatch(batch)) {
+      for(int r=0; r < batch.size; ++r) {
+        assertEquals(input.get(idx++).longValue(),
+            ((LongColumnVector) batch.cols[0]).vector[r]);
+      }
+    }
+  }
+
+  @Test
+  public void testPatchedBaseAt255() throws Exception {
+    TypeDescription schema = TypeDescription.createLong();
+
+    List<Long> input = Lists.newArrayList();
+    Random rand = new Random();
+    for(int i = 0; i < 5120; i++) {
+      input.add((long) rand.nextInt(100));
+    }
+    input.set(255, 20000L);
+
+    Writer writer = OrcFile.createWriter(testFilePath,
+        OrcFile.writerOptions(conf)
+            .setSchema(schema)
+            .stripeSize(100000)
+            .bufferSize(10000)
+            .encodingStrategy(encodingStrategy));
+    VectorizedRowBatch batch = schema.createRowBatch(5120);
+    for(Long l : input) {
+      appendLong(batch, l);
+    }
+    writer.addRowBatch(batch);
+    writer.close();
+
+    Reader reader = OrcFile.createReader(testFilePath,
+        OrcFile.readerOptions(conf).filesystem(fs));
+    RecordReader rows = reader.rows();
+    batch = reader.getSchema().createRowBatch();
+    int idx = 0;
+    while (rows.nextBatch(batch)) {
+      for(int r=0; r < batch.size; ++r) {
+        assertEquals(input.get(idx++).longValue(),
+            ((LongColumnVector) batch.cols[0]).vector[r]);
+      }
+    }
+  }
+
+  @Test
+  public void testPatchedBaseAt256() throws Exception {
+    TypeDescription schema = TypeDescription.createLong();
+
+    List<Long> input = Lists.newArrayList();
+    Random rand = new Random();
+    for(int i = 0; i < 5120; i++) {
+      input.add((long) rand.nextInt(100));
+    }
+    input.set(256, 20000L);
+
+    Writer writer = OrcFile.createWriter(testFilePath,
+        OrcFile.writerOptions(conf)
+            .setSchema(schema)
+            .stripeSize(100000)
+            .bufferSize(10000)
+            .encodingStrategy(encodingStrategy));
+    VectorizedRowBatch batch = schema.createRowBatch(5120);
+    for(Long l : input) {
+      appendLong(batch, l);
+    }
+    writer.addRowBatch(batch);
+    writer.close();
+
+    Reader reader = OrcFile.createReader(testFilePath,
+        OrcFile.readerOptions(conf).filesystem(fs));
+    RecordReader rows = reader.rows();
+    batch = reader.getSchema().createRowBatch();
+    int idx = 0;
+    while (rows.nextBatch(batch)) {
+      for(int r=0; r < batch.size; ++r) {
+        assertEquals(input.get(idx++).longValue(),
+            ((LongColumnVector) batch.cols[0]).vector[r]);
+      }
+    }
+  }
+
+  @Test
+  public void testPatchedBase510() throws Exception {
+    TypeDescription schema = TypeDescription.createLong();
+
+    List<Long> input = Lists.newArrayList();
+    Random rand = new Random();
+    for(int i = 0; i < 5120; i++) {
+      input.add((long) rand.nextInt(100));
+    }
+    input.set(510, 20000L);
+
+    Writer writer = OrcFile.createWriter(testFilePath,
+        OrcFile.writerOptions(conf)
+            .setSchema(schema)
+            .stripeSize(100000)
+            .bufferSize(10000)
+            .encodingStrategy(encodingStrategy));
+    VectorizedRowBatch batch = schema.createRowBatch(5120);
+    for(Long l : input) {
+      appendLong(batch, l);
+    }
+    writer.addRowBatch(batch);
+    writer.close();
+
+    Reader reader = OrcFile.createReader(testFilePath,
+        OrcFile.readerOptions(conf).filesystem(fs));
+    RecordReader rows = reader.rows();
+    batch = reader.getSchema().createRowBatch();
+    int idx = 0;
+    while (rows.nextBatch(batch)) {
+      for(int r=0; r < batch.size; ++r) {
+        assertEquals(input.get(idx++).longValue(),
+            ((LongColumnVector) batch.cols[0]).vector[r]);
+      }
+    }
+  }
+
+  @Test
+  public void testPatchedBase511() throws Exception {
+    TypeDescription schema = TypeDescription.createLong();
+
+    List<Long> input = Lists.newArrayList();
+    Random rand = new Random();
+    for(int i = 0; i < 5120; i++) {
+      input.add((long) rand.nextInt(100));
+    }
+    input.set(511, 20000L);
+
+    Writer writer = OrcFile.createWriter(testFilePath,
+        OrcFile.writerOptions(conf)
+            .setSchema(schema)
+            .stripeSize(100000)
+            .bufferSize(10000)
+            .encodingStrategy(encodingStrategy));
+    VectorizedRowBatch batch = schema.createRowBatch(5120);
+    for(Long l : input) {
+      appendLong(batch, l);
+    }
+    writer.addRowBatch(batch);
+    writer.close();
+
+    Reader reader = OrcFile.createReader(testFilePath,
+        OrcFile.readerOptions(conf).filesystem(fs));
+    RecordReader rows = reader.rows();
+    batch = reader.getSchema().createRowBatch();
+    int idx = 0;
+    while (rows.nextBatch(batch)) {
+      for(int r=0; r < batch.size; ++r) {
+        assertEquals(input.get(idx++).longValue(),
+            ((LongColumnVector) batch.cols[0]).vector[r]);
+      }
+    }
+  }
+
+  @Test
+  public void testPatchedBaseMax1() throws Exception {
+    TypeDescription schema = TypeDescription.createLong();
+
+    List<Long> input = Lists.newArrayList();
+    Random rand = new Random();
+    for (int i = 0; i < 5120; i++) {
+      input.add((long) rand.nextInt(60));
+    }
+    input.set(511, Long.MAX_VALUE);
+
+    Writer writer = OrcFile.createWriter(testFilePath,
+        OrcFile.writerOptions(conf)
+            .setSchema(schema)
+            .stripeSize(100000)
+            .bufferSize(10000)
+            .encodingStrategy(encodingStrategy));
+    VectorizedRowBatch batch = schema.createRowBatch(5120);
+    for (Long l : input) {
+      appendLong(batch, l);
+    }
+    writer.addRowBatch(batch);
+    writer.close();
+
+    Reader reader = OrcFile.createReader(testFilePath,
+        OrcFile.readerOptions(conf).filesystem(fs));
+    RecordReader rows = reader.rows();
+    batch = reader.getSchema().createRowBatch();
+    int idx = 0;
+    while (rows.nextBatch(batch)) {
+      for(int r=0; r < batch.size; ++r) {
+        assertEquals(input.get(idx++).longValue(),
+            ((LongColumnVector) batch.cols[0]).vector[r]);
+      }
+    }
+  }
+
+  @Test
+  public void testPatchedBaseMax2() throws Exception {
+    TypeDescription schema = TypeDescription.createLong();
+
+    List<Long> input = Lists.newArrayList();
+    Random rand = new Random();
+    for (int i = 0; i < 5120; i++) {
+      input.add((long) rand.nextInt(60));
+    }
+    input.set(128, Long.MAX_VALUE);
+    input.set(256, Long.MAX_VALUE);
+    input.set(511, Long.MAX_VALUE);
+
+    Writer writer = OrcFile.createWriter(testFilePath,
+        OrcFile.writerOptions(conf)
+            .setSchema(schema)
+            .stripeSize(100000)
+            .bufferSize(10000)
+            .encodingStrategy(encodingStrategy));
+    VectorizedRowBatch batch = schema.createRowBatch(5120);
+    for (Long l : input) {
+      appendLong(batch, l);
+    }
+    writer.addRowBatch(batch);
+    writer.close();
+
+    Reader reader = OrcFile.createReader(testFilePath,
+        OrcFile.readerOptions(conf).filesystem(fs));
+    RecordReader rows = reader.rows();
+    batch = reader.getSchema().createRowBatch();
+    int idx = 0;
+    while (rows.nextBatch(batch)) {
+      for(int r=0; r < batch.size; ++r) {
+        assertEquals(input.get(idx++).longValue(),
+            ((LongColumnVector) batch.cols[0]).vector[r]);
+      }
+    }
+  }
+
+  @Test
+  public void testPatchedBaseMax3() throws Exception {
+    TypeDescription schema = TypeDescription.createLong();
+
+    List<Long> input = Lists.newArrayList();
+    input.add(371946367L);
+    input.add(11963367L);
+    input.add(68639400007L);
+    input.add(100233367L);
+    input.add(6367L);
+    input.add(10026367L);
+    input.add(3670000L);
+    input.add(3602367L);
+    input.add(4719226367L);
+    input.add(7196367L);
+    input.add(444442L);
+    input.add(210267L);
+    input.add(21033L);
+    input.add(160267L);
+    input.add(400267L);
+    input.add(23634347L);
+    input.add(16027L);
+    input.add(46026367L);
+    input.add(Long.MAX_VALUE);
+    input.add(33333L);
+
+    Writer writer = OrcFile.createWriter(testFilePath,
+        OrcFile.writerOptions(conf)
+            .setSchema(schema)
+            .stripeSize(100000)
+            .bufferSize(10000)
+            .encodingStrategy(encodingStrategy));
+    VectorizedRowBatch batch = schema.createRowBatch();
+    for (Long l : input) {
+      appendLong(batch, l);
+    }
+    writer.addRowBatch(batch);
+    writer.close();
+
+    Reader reader = OrcFile.createReader(testFilePath,
+        OrcFile.readerOptions(conf).filesystem(fs));
+    RecordReader rows = reader.rows();
+    int idx = 0;
+    while (rows.nextBatch(batch)) {
+      for(int r=0; r < batch.size; ++r) {
+        assertEquals(input.get(idx++).longValue(),
+            ((LongColumnVector) batch.cols[0]).vector[r]);
+      }
+    }
+  }
+
+  @Test
+  public void testPatchedBaseMax4() throws Exception {
+    TypeDescription schema = TypeDescription.createLong();
+
+    List<Long> input = Lists.newArrayList();
+    for (int i = 0; i < 25; i++) {
+      input.add(371292224226367L);
+      input.add(119622332222267L);
+      input.add(686329400222007L);
+      input.add(100233333222367L);
+      input.add(636272333322222L);
+      input.add(10202633223267L);
+      input.add(36700222022230L);
+      input.add(36023226224227L);
+      input.add(47192226364427L);
+      input.add(71963622222447L);
+      input.add(22244444222222L);
+      input.add(21220263327442L);
+      input.add(21032233332232L);
+      input.add(16026322232227L);
+      input.add(40022262272212L);
+      input.add(23634342227222L);
+      input.add(16022222222227L);
+      input.add(46026362222227L);
+      input.add(46026362222227L);
+      input.add(33322222222323L);
+    }
+    input.add(Long.MAX_VALUE);
+
+    Writer writer = OrcFile.createWriter(testFilePath,
+        OrcFile.writerOptions(conf)
+            .setSchema(schema)
+            .stripeSize(100000)
+            .bufferSize(10000)
+            .encodingStrategy(encodingStrategy));
+    VectorizedRowBatch batch = schema.createRowBatch();
+    for (Long l : input) {
+      appendLong(batch, l);
+    }
+    writer.addRowBatch(batch);
+    writer.close();
+
+    Reader reader = OrcFile.createReader(testFilePath,
+        OrcFile.readerOptions(conf).filesystem(fs));
+    RecordReader rows = reader.rows();
+    batch = reader.getSchema().createRowBatch();
+    int idx = 0;
+    while (rows.nextBatch(batch)) {
+      for(int r=0; r < batch.size; ++r) {
+        assertEquals(input.get(idx++).longValue(),
+            ((LongColumnVector) batch.cols[0]).vector[r]);
+      }
+    }
+  }
+
+  @Test
+  public void testPatchedBaseTimestamp() throws Exception {
+    TypeDescription schema = TypeDescription.createStruct()
+        .addField("ts", TypeDescription.createTimestamp());
+
+    Writer writer = OrcFile.createWriter(testFilePath,
+        OrcFile.writerOptions(conf)
+            .setSchema(schema)
+            .stripeSize(100000)
+            .bufferSize(10000)
+            .encodingStrategy(encodingStrategy));
+    VectorizedRowBatch batch = schema.createRowBatch();
+
+    List<Timestamp> tslist = Lists.newArrayList();
+    tslist.add(Timestamp.valueOf("2099-01-01 00:00:00"));
+    tslist.add(Timestamp.valueOf("2003-01-01 00:00:00"));
+    tslist.add(Timestamp.valueOf("1999-01-01 00:00:00"));
+    tslist.add(Timestamp.valueOf("1995-01-01 00:00:00"));
+    tslist.add(Timestamp.valueOf("2002-01-01 00:00:00"));
+    tslist.add(Timestamp.valueOf("2010-03-02 00:00:00"));
+    tslist.add(Timestamp.valueOf("2005-01-01 00:00:00"));
+    tslist.add(Timestamp.valueOf("2006-01-01 00:00:00"));
+    tslist.add(Timestamp.valueOf("2003-01-01 00:00:00"));
+    tslist.add(Timestamp.valueOf("1996-08-02 00:00:00"));
+    tslist.add(Timestamp.valueOf("1998-11-02 00:00:00"));
+    tslist.add(Timestamp.valueOf("2008-10-02 00:00:00"));
+    tslist.add(Timestamp.valueOf("1993-08-02 00:00:00"));
+    tslist.add(Timestamp.valueOf("2008-01-02 00:00:00"));
+    tslist.add(Timestamp.valueOf("2007-01-01 00:00:00"));
+    tslist.add(Timestamp.valueOf("2004-01-01 00:00:00"));
+    tslist.add(Timestamp.valueOf("2008-10-02 00:00:00"));
+    tslist.add(Timestamp.valueOf("2003-01-01 00:00:00"));
+    tslist.add(Timestamp.valueOf("2004-01-01 00:00:00"));
+    tslist.add(Timestamp.valueOf("2008-01-01 00:00:00"));
+    tslist.add(Timestamp.valueOf("2005-01-01 00:00:00"));
+    tslist.add(Timestamp.valueOf("1994-01-01 00:00:00"));
+    tslist.add(Timestamp.valueOf("2006-01-01 00:00:00"));
+    tslist.add(Timestamp.valueOf("2004-01-01 00:00:00"));
+    tslist.add(Timestamp.valueOf("2001-01-01 00:00:00"));
+    tslist.add(Timestamp.valueOf("2000-01-01 00:00:00"));
+    tslist.add(Timestamp.valueOf("2000-01-01 00:00:00"));
+    tslist.add(Timestamp.valueOf("2002-01-01 00:00:00"));
+    tslist.add(Timestamp.valueOf("2006-01-01 00:00:00"));
+    tslist.add(Timestamp.valueOf("2011-01-01 00:00:00"));
+    tslist.add(Timestamp.valueOf("2002-01-01 00:00:00"));
+    tslist.add(Timestamp.valueOf("2005-01-01 00:00:00"));
+    tslist.add(Timestamp.valueOf("1974-01-01 00:00:00"));
+    int idx = 0;
+    for (Timestamp ts : tslist) {
+      ((TimestampColumnVector) batch.cols[0]).set(idx, ts);
+    }
+    writer.addRowBatch(batch);
+    writer.close();
+
+    Reader reader = OrcFile.createReader(testFilePath,
+        OrcFile.readerOptions(conf).filesystem(fs));
+    RecordReader rows = reader.rows();
+    batch = reader.getSchema().createRowBatch();
+    idx = 0;
+    while (rows.nextBatch(batch)) {
+      for(int r=0; r < batch.size; ++r) {
+        assertEquals(tslist.get(idx++),
+            ((TimestampColumnVector) batch.cols[0]).asScratchTimestamp(r));
+      }
+    }
+  }
+
+  @Test
+  public void testDirectLargeNegatives() throws Exception {
+    TypeDescription schema = TypeDescription.createLong();
+
+    Writer writer = OrcFile.createWriter(testFilePath,
+        OrcFile.writerOptions(conf)
+            .setSchema(schema)
+            .stripeSize(100000)
+            .bufferSize(10000)
+            .encodingStrategy(encodingStrategy));
+    VectorizedRowBatch batch = schema.createRowBatch();
+
+    appendLong(batch, -7486502418706614742L);
+    appendLong(batch, 0L);
+    appendLong(batch, 1L);
+    appendLong(batch, 1L);
+    appendLong(batch, -5535739865598783616L);
+    writer.addRowBatch(batch);
+    writer.close();
+
+    Reader reader = OrcFile.createReader(testFilePath,
+        OrcFile.readerOptions(conf).filesystem(fs));
+    RecordReader rows = reader.rows();
+    batch = reader.getSchema().createRowBatch();
+    assertEquals(true, rows.nextBatch(batch));
+    assertEquals(5, batch.size);
+    assertEquals(-7486502418706614742L,
+        ((LongColumnVector) batch.cols[0]).vector[0]);
+    assertEquals(0L,
+        ((LongColumnVector) batch.cols[0]).vector[1]);
+    assertEquals(1L,
+        ((LongColumnVector) batch.cols[0]).vector[2]);
+    assertEquals(1L,
+        ((LongColumnVector) batch.cols[0]).vector[3]);
+    assertEquals(-5535739865598783616L,
+        ((LongColumnVector) batch.cols[0]).vector[4]);
+    assertEquals(false, rows.nextBatch(batch));
+  }
+
+  @Test
+  public void testSeek() throws Exception {
+    TypeDescription schema = TypeDescription.createLong();
+
+    List<Long> input = Lists.newArrayList();
+    Random rand = new Random();
+    for(int i = 0; i < 100000; i++) {
+      input.add((long) rand.nextInt());
+    }
+    Writer writer = OrcFile.createWriter(testFilePath,
+        OrcFile.writerOptions(conf)
+            .setSchema(schema)
+            .compress(CompressionKind.NONE)
+            .stripeSize(100000)
+            .bufferSize(10000)
+            .version(OrcFile.Version.V_0_11)
+            .encodingStrategy(encodingStrategy));
+    VectorizedRowBatch batch = schema.createRowBatch(100000);
+    for(Long l : input) {
+      appendLong(batch, l);
+    }
+    writer.addRowBatch(batch);
+    writer.close();
+
+    Reader reader = OrcFile.createReader(testFilePath,
+        OrcFile.readerOptions(conf).filesystem(fs));
+    RecordReader rows = reader.rows();
+    batch = reader.getSchema().createRowBatch();
+    int idx = 55555;
+    rows.seekToRow(idx);
+    while (rows.nextBatch(batch)) {
+      for(int r=0; r < batch.size; ++r) {
+        assertEquals(input.get(idx++).longValue(),
+            ((LongColumnVector) batch.cols[0]).vector[r]);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/orc/src/test/org/apache/orc/TestOrcNullOptimization.java
----------------------------------------------------------------------
diff --git a/orc/src/test/org/apache/orc/TestOrcNullOptimization.java b/orc/src/test/org/apache/orc/TestOrcNullOptimization.java
new file mode 100644
index 0000000..0b605c9
--- /dev/null
+++ b/orc/src/test/org/apache/orc/TestOrcNullOptimization.java
@@ -0,0 +1,415 @@
+/**
+ * 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.orc;
+
+import static junit.framework.Assert.assertEquals;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.List;
+import java.util.Random;
+
+import junit.framework.Assert;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.ListColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.StructColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+
+import org.apache.orc.impl.RecordReaderImpl;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+
+import com.google.common.collect.Lists;
+
+public class TestOrcNullOptimization {
+
+  TypeDescription createMyStruct() {
+    return TypeDescription.createStruct()
+        .addField("a", TypeDescription.createInt())
+        .addField("b", TypeDescription.createString())
+        .addField("c", TypeDescription.createBoolean())
+        .addField("d", TypeDescription.createList(
+            TypeDescription.createStruct()
+                .addField("z", TypeDescription.createInt())));
+  }
+
+  void addRow(Writer writer, VectorizedRowBatch batch,
+              Integer a, String b, Boolean c,
+              Integer... d) throws IOException {
+    if (batch.size == batch.getMaxSize()) {
+      writer.addRowBatch(batch);
+      batch.reset();
+    }
+    int row = batch.size++;
+    LongColumnVector aColumn = (LongColumnVector) batch.cols[0];
+    BytesColumnVector bColumn = (BytesColumnVector) batch.cols[1];
+    LongColumnVector cColumn = (LongColumnVector) batch.cols[2];
+    ListColumnVector dColumn = (ListColumnVector) batch.cols[3];
+    StructColumnVector dStruct = (StructColumnVector) dColumn.child;
+    LongColumnVector dInt = (LongColumnVector) dStruct.fields[0];
+    if (a == null) {
+      aColumn.noNulls = false;
+      aColumn.isNull[row] = true;
+    } else {
+      aColumn.vector[row] = a;
+    }
+    if (b == null) {
+      bColumn.noNulls = false;
+      bColumn.isNull[row] = true;
+    } else {
+      bColumn.setVal(row, b.getBytes());
+    }
+    if (c == null) {
+      cColumn.noNulls = false;
+      cColumn.isNull[row] = true;
+    } else {
+      cColumn.vector[row] = c ? 1 : 0;
+    }
+    if (d == null) {
+      dColumn.noNulls = false;
+      dColumn.isNull[row] = true;
+    } else {
+      dColumn.offsets[row] = dColumn.childCount;
+      dColumn.lengths[row] = d.length;
+      dColumn.childCount += d.length;
+      for(int e=0; e < d.length; ++e) {
+        dInt.vector[(int) dColumn.offsets[row] + e] = d[e];
+      }
+    }
+  }
+
+  Path workDir = new Path(System.getProperty("test.tmp.dir",
+      "target" + File.separator + "test" + File.separator + "tmp"));
+
+  Configuration conf;
+  FileSystem fs;
+  Path testFilePath;
+
+  @Rule
+  public TestName testCaseName = new TestName();
+
+  @Before
+  public void openFileSystem() throws Exception {
+    conf = new Configuration();
+    fs = FileSystem.getLocal(conf);
+    testFilePath = new Path(workDir, "TestOrcNullOptimization." +
+        testCaseName.getMethodName() + ".orc");
+    fs.delete(testFilePath, false);
+  }
+
+  @Test
+  public void testMultiStripeWithNull() throws Exception {
+    TypeDescription schema = createMyStruct();
+    Writer writer = OrcFile.createWriter(testFilePath,
+                                         OrcFile.writerOptions(conf)
+                                         .setSchema(schema)
+                                         .stripeSize(100000)
+                                         .compress(CompressionKind.NONE)
+                                         .bufferSize(10000));
+    Random rand = new Random(100);
+    VectorizedRowBatch batch = schema.createRowBatch();
+    addRow(writer, batch, null, null, true, 100);
+    for (int i = 2; i < 20000; i++) {
+      addRow(writer, batch, rand.nextInt(1), "a", true, 100);
+    }
+    addRow(writer, batch, null, null, true, 100);
+    writer.addRowBatch(batch);
+    writer.close();
+
+    Reader reader = OrcFile.createReader(testFilePath,
+        OrcFile.readerOptions(conf).filesystem(fs));
+    // check the stats
+    ColumnStatistics[] stats = reader.getStatistics();
+    assertEquals(20000, reader.getNumberOfRows());
+    assertEquals(20000, stats[0].getNumberOfValues());
+
+    assertEquals(0, ((IntegerColumnStatistics) stats[1]).getMaximum());
+    assertEquals(0, ((IntegerColumnStatistics) stats[1]).getMinimum());
+    assertEquals(true, ((IntegerColumnStatistics) stats[1]).isSumDefined());
+    assertEquals(0, ((IntegerColumnStatistics) stats[1]).getSum());
+    assertEquals("count: 19998 hasNull: true min: 0 max: 0 sum: 0",
+        stats[1].toString());
+
+    assertEquals("a", ((StringColumnStatistics) stats[2]).getMaximum());
+    assertEquals("a", ((StringColumnStatistics) stats[2]).getMinimum());
+    assertEquals(19998, stats[2].getNumberOfValues());
+    assertEquals("count: 19998 hasNull: true min: a max: a sum: 19998",
+        stats[2].toString());
+
+    // check the inspectors
+    assertEquals("struct<a:int,b:string,c:boolean,d:array<struct<z:int>>>",
+        reader.getSchema().toString());
+
+    RecordReader rows = reader.rows();
+
+    List<Boolean> expected = Lists.newArrayList();
+    for (StripeInformation sinfo : reader.getStripes()) {
+      expected.add(false);
+    }
+    // only the first and last stripe will have PRESENT stream
+    expected.set(0, true);
+    expected.set(expected.size() - 1, true);
+
+    List<Boolean> got = Lists.newArrayList();
+    // check if the strip footer contains PRESENT stream
+    for (StripeInformation sinfo : reader.getStripes()) {
+      OrcProto.StripeFooter sf =
+        ((RecordReaderImpl) rows).readStripeFooter(sinfo);
+      got.add(sf.toString().indexOf(OrcProto.Stream.Kind.PRESENT.toString())
+              != -1);
+    }
+    assertEquals(expected, got);
+
+    batch = reader.getSchema().createRowBatch();
+    LongColumnVector aColumn = (LongColumnVector) batch.cols[0];
+    BytesColumnVector bColumn = (BytesColumnVector) batch.cols[1];
+    LongColumnVector cColumn = (LongColumnVector) batch.cols[2];
+    ListColumnVector dColumn = (ListColumnVector) batch.cols[3];
+    LongColumnVector dElements =
+        (LongColumnVector)(((StructColumnVector) dColumn.child).fields[0]);
+    assertEquals(true , rows.nextBatch(batch));
+    assertEquals(1024, batch.size);
+
+    // row 1
+    assertEquals(true, aColumn.isNull[0]);
+    assertEquals(true, bColumn.isNull[0]);
+    assertEquals(1, cColumn.vector[0]);
+    assertEquals(0, dColumn.offsets[0]);
+    assertEquals(1, dColumn.lengths[1]);
+    assertEquals(100, dElements.vector[0]);
+
+    rows.seekToRow(19998);
+    rows.nextBatch(batch);
+    assertEquals(2, batch.size);
+
+    // last-1 row
+    assertEquals(0, aColumn.vector[0]);
+    assertEquals("a", bColumn.toString(0));
+    assertEquals(1, cColumn.vector[0]);
+    assertEquals(0, dColumn.offsets[0]);
+    assertEquals(1, dColumn.lengths[0]);
+    assertEquals(100, dElements.vector[0]);
+
+    // last row
+    assertEquals(true, aColumn.isNull[1]);
+    assertEquals(true, bColumn.isNull[1]);
+    assertEquals(1, cColumn.vector[1]);
+    assertEquals(1, dColumn.offsets[1]);
+    assertEquals(1, dColumn.lengths[1]);
+    assertEquals(100, dElements.vector[1]);
+
+    assertEquals(false, rows.nextBatch(batch));
+    rows.close();
+  }
+
+  @Test
+  public void testMultiStripeWithoutNull() throws Exception {
+    TypeDescription schema = createMyStruct();
+    Writer writer = OrcFile.createWriter(testFilePath,
+                                         OrcFile.writerOptions(conf)
+                                         .setSchema(schema)
+                                         .stripeSize(100000)
+                                         .compress(CompressionKind.NONE)
+                                         .bufferSize(10000));
+    Random rand = new Random(100);
+    VectorizedRowBatch batch = schema.createRowBatch();
+    for (int i = 1; i < 20000; i++) {
+      addRow(writer, batch, rand.nextInt(1), "a", true, 100);
+    }
+    addRow(writer, batch, 0, "b", true, 100);
+    writer.addRowBatch(batch);
+    writer.close();
+
+    Reader reader = OrcFile.createReader(testFilePath,
+        OrcFile.readerOptions(conf).filesystem(fs));
+    // check the stats
+    ColumnStatistics[] stats = reader.getStatistics();
+    assertEquals(20000, reader.getNumberOfRows());
+    assertEquals(20000, stats[0].getNumberOfValues());
+
+    assertEquals(0, ((IntegerColumnStatistics) stats[1]).getMaximum());
+    assertEquals(0, ((IntegerColumnStatistics) stats[1]).getMinimum());
+    assertEquals(true, ((IntegerColumnStatistics) stats[1]).isSumDefined());
+    assertEquals(0, ((IntegerColumnStatistics) stats[1]).getSum());
+    assertEquals("count: 20000 hasNull: false min: 0 max: 0 sum: 0",
+        stats[1].toString());
+
+    assertEquals("b", ((StringColumnStatistics) stats[2]).getMaximum());
+    assertEquals("a", ((StringColumnStatistics) stats[2]).getMinimum());
+    assertEquals(20000, stats[2].getNumberOfValues());
+    assertEquals("count: 20000 hasNull: false min: a max: b sum: 20000",
+        stats[2].toString());
+
+    // check the inspectors
+    Assert.assertEquals("struct<a:int,b:string,c:boolean,d:array<struct<z:int>>>",
+        reader.getSchema().toString());
+
+    RecordReader rows = reader.rows();
+
+    // none of the stripes will have PRESENT stream
+    List<Boolean> expected = Lists.newArrayList();
+    for (StripeInformation sinfo : reader.getStripes()) {
+      expected.add(false);
+    }
+
+    List<Boolean> got = Lists.newArrayList();
+    // check if the strip footer contains PRESENT stream
+    for (StripeInformation sinfo : reader.getStripes()) {
+      OrcProto.StripeFooter sf =
+        ((RecordReaderImpl) rows).readStripeFooter(sinfo);
+      got.add(sf.toString().indexOf(OrcProto.Stream.Kind.PRESENT.toString())
+              != -1);
+    }
+    assertEquals(expected, got);
+
+    rows.seekToRow(19998);
+
+    batch = reader.getSchema().createRowBatch();
+    LongColumnVector aColumn = (LongColumnVector) batch.cols[0];
+    BytesColumnVector bColumn = (BytesColumnVector) batch.cols[1];
+    LongColumnVector cColumn = (LongColumnVector) batch.cols[2];
+    ListColumnVector dColumn = (ListColumnVector) batch.cols[3];
+    LongColumnVector dElements =
+        (LongColumnVector)(((StructColumnVector) dColumn.child).fields[0]);
+
+    assertEquals(true, rows.nextBatch(batch));
+    assertEquals(2, batch.size);
+
+    // last-1 row
+    assertEquals(0, aColumn.vector[0]);
+    assertEquals("a", bColumn.toString(0));
+    assertEquals(1, cColumn.vector[0]);
+    assertEquals(0, dColumn.offsets[0]);
+    assertEquals(1, dColumn.lengths[0]);
+    assertEquals(100, dElements.vector[0]);
+
+    // last row
+    assertEquals(0, aColumn.vector[1]);
+    assertEquals("b", bColumn.toString(1));
+    assertEquals(1, cColumn.vector[1]);
+    assertEquals(1, dColumn.offsets[1]);
+    assertEquals(1, dColumn.lengths[1]);
+    assertEquals(100, dElements.vector[1]);
+    rows.close();
+  }
+
+  @Test
+  public void testColumnsWithNullAndCompression() throws Exception {
+    TypeDescription schema = createMyStruct();
+    Writer writer = OrcFile.createWriter(testFilePath,
+                                         OrcFile.writerOptions(conf)
+                                         .setSchema(schema)
+                                         .stripeSize(100000)
+                                         .bufferSize(10000));
+    VectorizedRowBatch batch = schema.createRowBatch();
+    addRow(writer, batch, 3, "a", true, 100);
+    addRow(writer, batch, null, "b", true, 100);
+    addRow(writer, batch, 3, null, false, 100);
+    addRow(writer, batch, 3, "d", true, 100);
+    addRow(writer, batch, 2, "e", true, 100);
+    addRow(writer, batch, 2, "f", true, 100);
+    addRow(writer, batch, 2, "g", true, 100);
+    addRow(writer, batch, 2, "h", true, 100);
+    writer.addRowBatch(batch);
+    writer.close();
+
+    Reader reader = OrcFile.createReader(testFilePath,
+        OrcFile.readerOptions(conf).filesystem(fs));
+    // check the stats
+    ColumnStatistics[] stats = reader.getStatistics();
+    assertEquals(8, reader.getNumberOfRows());
+    assertEquals(8, stats[0].getNumberOfValues());
+
+    assertEquals(3, ((IntegerColumnStatistics) stats[1]).getMaximum());
+    assertEquals(2, ((IntegerColumnStatistics) stats[1]).getMinimum());
+    assertEquals(true, ((IntegerColumnStatistics) stats[1]).isSumDefined());
+    assertEquals(17, ((IntegerColumnStatistics) stats[1]).getSum());
+    assertEquals("count: 7 hasNull: true min: 2 max: 3 sum: 17",
+        stats[1].toString());
+
+    assertEquals("h", ((StringColumnStatistics) stats[2]).getMaximum());
+    assertEquals("a", ((StringColumnStatistics) stats[2]).getMinimum());
+    assertEquals(7, stats[2].getNumberOfValues());
+    assertEquals("count: 7 hasNull: true min: a max: h sum: 7",
+        stats[2].toString());
+
+    // check the inspectors
+    batch = reader.getSchema().createRowBatch();
+    LongColumnVector aColumn = (LongColumnVector) batch.cols[0];
+    BytesColumnVector bColumn = (BytesColumnVector) batch.cols[1];
+    LongColumnVector cColumn = (LongColumnVector) batch.cols[2];
+    ListColumnVector dColumn = (ListColumnVector) batch.cols[3];
+    LongColumnVector dElements =
+        (LongColumnVector)(((StructColumnVector) dColumn.child).fields[0]);
+    Assert.assertEquals("struct<a:int,b:string,c:boolean,d:array<struct<z:int>>>",
+        reader.getSchema().toString());
+
+    RecordReader rows = reader.rows();
+    // only the last strip will have PRESENT stream
+    List<Boolean> expected = Lists.newArrayList();
+    for (StripeInformation sinfo : reader.getStripes()) {
+      expected.add(false);
+    }
+    expected.set(expected.size() - 1, true);
+
+    List<Boolean> got = Lists.newArrayList();
+    // check if the strip footer contains PRESENT stream
+    for (StripeInformation sinfo : reader.getStripes()) {
+      OrcProto.StripeFooter sf =
+        ((RecordReaderImpl) rows).readStripeFooter(sinfo);
+      got.add(sf.toString().indexOf(OrcProto.Stream.Kind.PRESENT.toString())
+              != -1);
+    }
+    assertEquals(expected, got);
+
+    assertEquals(true, rows.nextBatch(batch));
+    assertEquals(8, batch.size);
+
+    // row 1
+    assertEquals(3, aColumn.vector[0]);
+    assertEquals("a", bColumn.toString(0));
+    assertEquals(1, cColumn.vector[0]);
+    assertEquals(0, dColumn.offsets[0]);
+    assertEquals(1, dColumn.lengths[0]);
+    assertEquals(100, dElements.vector[0]);
+
+    // row 2
+    assertEquals(true, aColumn.isNull[1]);
+    assertEquals("b", bColumn.toString(1));
+    assertEquals(1, cColumn.vector[1]);
+    assertEquals(1, dColumn.offsets[1]);
+    assertEquals(1, dColumn.lengths[1]);
+    assertEquals(100, dElements.vector[1]);
+
+    // row 3
+    assertEquals(3, aColumn.vector[2]);
+    assertEquals(true, bColumn.isNull[2]);
+    assertEquals(0, cColumn.vector[2]);
+    assertEquals(2, dColumn.offsets[2]);
+    assertEquals(1, dColumn.lengths[2]);
+    assertEquals(100, dElements.vector[2]);
+
+    rows.close();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/orc/src/test/org/apache/orc/TestOrcTimezone1.java
----------------------------------------------------------------------
diff --git a/orc/src/test/org/apache/orc/TestOrcTimezone1.java b/orc/src/test/org/apache/orc/TestOrcTimezone1.java
new file mode 100644
index 0000000..72dc455
--- /dev/null
+++ b/orc/src/test/org/apache/orc/TestOrcTimezone1.java
@@ -0,0 +1,189 @@
+/**
+ * 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.orc;
+
+import static junit.framework.Assert.assertEquals;
+import static junit.framework.Assert.assertNotNull;
+
+import java.io.File;
+import java.sql.Timestamp;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.TimeZone;
+
+import junit.framework.Assert;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import com.google.common.collect.Lists;
+
+/**
+ *
+ */
+@RunWith(Parameterized.class)
+public class TestOrcTimezone1 {
+  Path workDir = new Path(System.getProperty("test.tmp.dir",
+      "target" + File.separator + "test" + File.separator + "tmp"));
+  Configuration conf;
+  FileSystem fs;
+  Path testFilePath;
+  String writerTimeZone;
+  String readerTimeZone;
+  static TimeZone defaultTimeZone = TimeZone.getDefault();
+
+  public TestOrcTimezone1(String writerTZ, String readerTZ) {
+    this.writerTimeZone = writerTZ;
+    this.readerTimeZone = readerTZ;
+  }
+
+  @Parameterized.Parameters
+  public static Collection<Object[]> data() {
+    List<Object[]> result = Arrays.asList(new Object[][]{
+        /* Extreme timezones */
+        {"GMT-12:00", "GMT+14:00"},
+        /* No difference in DST */
+        {"America/Los_Angeles", "America/Los_Angeles"}, /* same timezone both with DST */
+        {"Europe/Berlin", "Europe/Berlin"}, /* same as above but europe */
+        {"America/Phoenix", "Asia/Kolkata"} /* Writer no DST, Reader no DST */,
+        {"Europe/Berlin", "America/Los_Angeles"} /* Writer DST, Reader DST */,
+        {"Europe/Berlin", "America/Chicago"} /* Writer DST, Reader DST */,
+        /* With DST difference */
+        {"Europe/Berlin", "UTC"},
+        {"UTC", "Europe/Berlin"} /* Writer no DST, Reader DST */,
+        {"America/Los_Angeles", "Asia/Kolkata"} /* Writer DST, Reader no DST */,
+        {"Europe/Berlin", "Asia/Kolkata"} /* Writer DST, Reader no DST */,
+        /* Timezone offsets for the reader has changed historically */
+        {"Asia/Saigon", "Pacific/Enderbury"},
+        {"UTC", "Asia/Jerusalem"},
+
+        // NOTE:
+        // "1995-01-01 03:00:00.688888888" this is not a valid time in Pacific/Enderbury timezone.
+        // On 1995-01-01 00:00:00 GMT offset moved from -11:00 hr to +13:00 which makes all values
+        // on 1995-01-01 invalid. Try this with joda time
+        // new MutableDateTime("1995-01-01", DateTimeZone.forTimeZone(readerTimeZone));
+    });
+    return result;
+  }
+
+  @Rule
+  public TestName testCaseName = new TestName();
+
+  @Before
+  public void openFileSystem() throws Exception {
+    conf = new Configuration();
+    fs = FileSystem.getLocal(conf);
+    testFilePath = new Path(workDir, "TestOrcFile." +
+        testCaseName.getMethodName() + ".orc");
+    fs.delete(testFilePath, false);
+  }
+
+  @After
+  public void restoreTimeZone() {
+    TimeZone.setDefault(defaultTimeZone);
+  }
+
+  @Test
+  public void testTimestampWriter() throws Exception {
+    TypeDescription schema = TypeDescription.createTimestamp();
+
+    TimeZone.setDefault(TimeZone.getTimeZone(writerTimeZone));
+    Writer writer = OrcFile.createWriter(testFilePath,
+        OrcFile.writerOptions(conf).setSchema(schema).stripeSize(100000)
+            .bufferSize(10000));
+    assertEquals(writerTimeZone, TimeZone.getDefault().getID());
+    List<String> ts = Lists.newArrayList();
+    ts.add("2003-01-01 01:00:00.000000222");
+    ts.add("1996-08-02 09:00:00.723100809");
+    ts.add("1999-01-01 02:00:00.999999999");
+    ts.add("1995-01-02 03:00:00.688888888");
+    ts.add("2002-01-01 04:00:00.1");
+    ts.add("2010-03-02 05:00:00.000009001");
+    ts.add("2005-01-01 06:00:00.000002229");
+    ts.add("2006-01-01 07:00:00.900203003");
+    ts.add("2003-01-01 08:00:00.800000007");
+    ts.add("1998-11-02 10:00:00.857340643");
+    ts.add("2008-10-02 11:00:00.0");
+    ts.add("2037-01-01 00:00:00.000999");
+    ts.add("2014-03-28 00:00:00.0");
+    VectorizedRowBatch batch = schema.createRowBatch();
+    TimestampColumnVector times = (TimestampColumnVector) batch.cols[0];
+    for (String t : ts) {
+      times.set(batch.size++, Timestamp.valueOf(t));
+    }
+    writer.addRowBatch(batch);
+    writer.close();
+
+    TimeZone.setDefault(TimeZone.getTimeZone(readerTimeZone));
+    Reader reader = OrcFile.createReader(testFilePath,
+        OrcFile.readerOptions(conf).filesystem(fs));
+    assertEquals(readerTimeZone, TimeZone.getDefault().getID());
+    RecordReader rows = reader.rows();
+    batch = reader.getSchema().createRowBatch();
+    times = (TimestampColumnVector) batch.cols[0];
+    int idx = 0;
+    while (rows.nextBatch(batch)) {
+      for(int r=0; r < batch.size; ++r) {
+        assertEquals(ts.get(idx++), times.asScratchTimestamp(r).toString());
+      }
+    }
+    rows.close();
+  }
+
+  @Test
+  public void testReadTimestampFormat_0_11() throws Exception {
+    TimeZone.setDefault(TimeZone.getTimeZone(readerTimeZone));
+    Path oldFilePath = new Path(getClass().getClassLoader().
+        getSystemResource("orc-file-11-format.orc").getPath());
+    Reader reader = OrcFile.createReader(oldFilePath,
+        OrcFile.readerOptions(conf).filesystem(fs));
+    TypeDescription schema = reader.getSchema();
+    int col = schema.getFieldNames().indexOf("ts");
+    VectorizedRowBatch batch = schema.createRowBatch(10);
+    TimestampColumnVector ts = (TimestampColumnVector) batch.cols[col];
+
+    boolean[] include = new boolean[schema.getMaximumId() + 1];
+    include[schema.getChildren().get(col).getId()] = true;
+    RecordReader rows = reader.rows
+        (new Reader.Options().include(include));
+    assertEquals(true, rows.nextBatch(batch));
+    assertEquals(Timestamp.valueOf("2000-03-12 15:00:00"),
+        ts.asScratchTimestamp(0));
+
+    // check the contents of second row
+    rows.seekToRow(7499);
+    assertEquals(true, rows.nextBatch(batch));
+    assertEquals(1, batch.size);
+    assertEquals(Timestamp.valueOf("2000-03-12 15:00:01"),
+        ts.asScratchTimestamp(0));
+
+    // handle the close up
+    Assert.assertEquals(false, rows.nextBatch(batch));
+    rows.close();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/orc/src/test/org/apache/orc/TestOrcTimezone2.java
----------------------------------------------------------------------
diff --git a/orc/src/test/org/apache/orc/TestOrcTimezone2.java b/orc/src/test/org/apache/orc/TestOrcTimezone2.java
new file mode 100644
index 0000000..4a02855
--- /dev/null
+++ b/orc/src/test/org/apache/orc/TestOrcTimezone2.java
@@ -0,0 +1,143 @@
+/**
+ * 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.orc;
+
+import static junit.framework.Assert.assertEquals;
+
+import java.io.File;
+import java.sql.Timestamp;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Random;
+import java.util.TimeZone;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import com.google.common.collect.Lists;
+
+/**
+ *
+ */
+@RunWith(Parameterized.class)
+public class TestOrcTimezone2 {
+  Path workDir = new Path(System.getProperty("test.tmp.dir",
+      "target" + File.separator + "test" + File.separator + "tmp"));
+  Configuration conf;
+  FileSystem fs;
+  Path testFilePath;
+  String writerTimeZone;
+  String readerTimeZone;
+  static TimeZone defaultTimeZone = TimeZone.getDefault();
+
+  public TestOrcTimezone2(String writerTZ, String readerTZ) {
+    this.writerTimeZone = writerTZ;
+    this.readerTimeZone = readerTZ;
+  }
+
+  @Parameterized.Parameters
+  public static Collection<Object[]> data() {
+    String[] allTimeZones = TimeZone.getAvailableIDs();
+    Random rand = new Random(123);
+    int len = allTimeZones.length;
+    int n = 500;
+    Object[][] data = new Object[n][];
+    for (int i = 0; i < n; i++) {
+      int wIdx = rand.nextInt(len);
+      int rIdx = rand.nextInt(len);
+      data[i] = new Object[2];
+      data[i][0] = allTimeZones[wIdx];
+      data[i][1] = allTimeZones[rIdx];
+    }
+    return Arrays.asList(data);
+  }
+
+  @Rule
+  public TestName testCaseName = new TestName();
+
+  @Before
+  public void openFileSystem() throws Exception {
+    conf = new Configuration();
+    fs = FileSystem.getLocal(conf);
+    testFilePath = new Path(workDir, "TestOrcFile." +
+        testCaseName.getMethodName() + ".orc");
+    fs.delete(testFilePath, false);
+  }
+
+  @After
+  public void restoreTimeZone() {
+    TimeZone.setDefault(defaultTimeZone);
+  }
+
+  @Test
+  public void testTimestampWriter() throws Exception {
+    TypeDescription schema = TypeDescription.createTimestamp();
+
+    TimeZone.setDefault(TimeZone.getTimeZone(writerTimeZone));
+    Writer writer = OrcFile.createWriter(testFilePath,
+        OrcFile.writerOptions(conf).setSchema(schema)
+            .stripeSize(100000).bufferSize(10000));
+    assertEquals(writerTimeZone, TimeZone.getDefault().getID());
+    List<String> ts = Lists.newArrayList();
+    ts.add("2003-01-01 01:00:00.000000222");
+    ts.add("1999-01-01 02:00:00.999999999");
+    ts.add("1995-01-02 03:00:00.688888888");
+    ts.add("2002-01-01 04:00:00.1");
+    ts.add("2010-03-02 05:00:00.000009001");
+    ts.add("2005-01-01 06:00:00.000002229");
+    ts.add("2006-01-01 07:00:00.900203003");
+    ts.add("2003-01-01 08:00:00.800000007");
+    ts.add("1996-08-02 09:00:00.723100809");
+    ts.add("1998-11-02 10:00:00.857340643");
+    ts.add("2008-10-02 11:00:00.0");
+    ts.add("2037-01-01 00:00:00.000999");
+    VectorizedRowBatch batch = schema.createRowBatch();
+    TimestampColumnVector tsc = (TimestampColumnVector) batch.cols[0];
+    for (String t : ts) {
+      tsc.set(batch.size++, Timestamp.valueOf(t));
+    }
+    writer.addRowBatch(batch);
+    writer.close();
+
+    TimeZone.setDefault(TimeZone.getTimeZone(readerTimeZone));
+    Reader reader = OrcFile.createReader(testFilePath,
+        OrcFile.readerOptions(conf).filesystem(fs));
+    assertEquals(readerTimeZone, TimeZone.getDefault().getID());
+    RecordReader rows = reader.rows();
+    int idx = 0;
+    batch = reader.getSchema().createRowBatch();
+    tsc = (TimestampColumnVector) batch.cols[0];
+    while (rows.nextBatch(batch)) {
+      for (int r=0; r < batch.size; ++r) {
+        assertEquals(ts.get(idx++), tsc.asScratchTimestamp(r).toString());
+      }
+    }
+    rows.close();
+  }
+}


[26/27] hive git commit: HIVE-11417. Move the ReaderImpl and RowReaderImpl to the ORC module, by making shims for the row by row reader. (omalley reviewed by prasanth_j)

Posted by om...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/orc/src/java/org/apache/orc/impl/ConvertTreeReaderFactory.java
----------------------------------------------------------------------
diff --git a/orc/src/java/org/apache/orc/impl/ConvertTreeReaderFactory.java b/orc/src/java/org/apache/orc/impl/ConvertTreeReaderFactory.java
new file mode 100644
index 0000000..3ba56f7
--- /dev/null
+++ b/orc/src/java/org/apache/orc/impl/ConvertTreeReaderFactory.java
@@ -0,0 +1,2840 @@
+/**
+ * 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.orc.impl;
+
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.EnumMap;
+import java.util.Map;
+
+import org.apache.hadoop.hive.common.type.HiveDecimal;
+import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.StringExpr;
+import org.apache.hadoop.hive.ql.util.TimestampUtils;
+import org.apache.hadoop.hive.serde2.io.DateWritable;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.FloatWritable;
+import org.apache.orc.OrcProto;
+import org.apache.orc.TypeDescription;
+import org.apache.orc.TypeDescription.Category;
+
+/**
+ * Convert ORC tree readers.
+ */
+public class ConvertTreeReaderFactory extends TreeReaderFactory {
+
+  /**
+   * Override methods like checkEncoding to pass-thru to the convert TreeReader.
+   */
+  public static class ConvertTreeReader extends TreeReader {
+
+    private TreeReader convertTreeReader;
+
+    ConvertTreeReader(int columnId) throws IOException {
+      super(columnId);
+    }
+
+    // The ordering of types here is used to determine which numeric types
+    // are common/convertible to one another. Probably better to rely on the
+    // ordering explicitly defined here than to assume that the enum values
+    // that were arbitrarily assigned in PrimitiveCategory work for our purposes.
+    private static EnumMap<TypeDescription.Category, Integer> numericTypes =
+        new EnumMap<>(TypeDescription.Category.class);
+
+    static {
+      registerNumericType(TypeDescription.Category.BOOLEAN, 1);
+      registerNumericType(TypeDescription.Category.BYTE, 2);
+      registerNumericType(TypeDescription.Category.SHORT, 3);
+      registerNumericType(TypeDescription.Category.INT, 4);
+      registerNumericType(TypeDescription.Category.LONG, 5);
+      registerNumericType(TypeDescription.Category.FLOAT, 6);
+      registerNumericType(TypeDescription.Category.DOUBLE, 7);
+      registerNumericType(TypeDescription.Category.DECIMAL, 8);
+    }
+
+    private static void registerNumericType(TypeDescription.Category kind, int level) {
+      numericTypes.put(kind, level);
+    }
+
+    protected void setConvertTreeReader(TreeReader convertTreeReader) {
+      this.convertTreeReader = convertTreeReader;
+    }
+
+    protected TreeReader getStringGroupTreeReader(int columnId,
+        TypeDescription fileType) throws IOException {
+      switch (fileType.getCategory()) {
+      case STRING:
+        return new StringTreeReader(columnId);
+      case CHAR:
+        return new CharTreeReader(columnId, fileType.getMaxLength());
+      case VARCHAR:
+        return new VarcharTreeReader(columnId, fileType.getMaxLength());
+      default:
+        throw new RuntimeException("Unexpected type kind " + fileType.getCategory().name());
+      }
+    }
+
+    protected void assignStringGroupVectorEntry(BytesColumnVector bytesColVector,
+        int elementNum, TypeDescription readerType, byte[] bytes) {
+      assignStringGroupVectorEntry(bytesColVector,
+          elementNum, readerType, bytes, 0, bytes.length);
+    }
+
+    /*
+     * Assign a BytesColumnVector entry when we have a byte array, start, and
+     * length for the string group which can be (STRING, CHAR, VARCHAR).
+     */
+    protected void assignStringGroupVectorEntry(BytesColumnVector bytesColVector,
+        int elementNum, TypeDescription readerType, byte[] bytes, int start, int length) {
+      switch (readerType.getCategory()) {
+      case STRING:
+        bytesColVector.setVal(elementNum, bytes, start, length);
+        break;
+      case CHAR:
+        {
+          int adjustedDownLen =
+              StringExpr.rightTrimAndTruncate(bytes, start, length, readerType.getMaxLength());
+          bytesColVector.setVal(elementNum, bytes, start, adjustedDownLen);
+        }
+        break;
+      case VARCHAR:
+        {
+          int adjustedDownLen =
+              StringExpr.truncate(bytes, start, length, readerType.getMaxLength());
+          bytesColVector.setVal(elementNum, bytes, start, adjustedDownLen);
+        }
+        break;
+      default:
+        throw new RuntimeException("Unexpected type kind " + readerType.getCategory().name());
+      }
+    }
+
+    protected void convertStringGroupVectorElement(BytesColumnVector bytesColVector,
+        int elementNum, TypeDescription readerType) {
+      switch (readerType.getCategory()) {
+      case STRING:
+        // No conversion needed.
+        break;
+      case CHAR:
+        {
+          int length = bytesColVector.length[elementNum];
+          int adjustedDownLen = StringExpr
+            .rightTrimAndTruncate(bytesColVector.vector[elementNum],
+                bytesColVector.start[elementNum], length,
+                readerType.getMaxLength());
+          if (adjustedDownLen < length) {
+            bytesColVector.length[elementNum] = adjustedDownLen;
+          }
+        }
+        break;
+      case VARCHAR:
+        {
+          int length = bytesColVector.length[elementNum];
+          int adjustedDownLen = StringExpr
+            .truncate(bytesColVector.vector[elementNum],
+                bytesColVector.start[elementNum], length,
+                readerType.getMaxLength());
+          if (adjustedDownLen < length) {
+            bytesColVector.length[elementNum] = adjustedDownLen;
+          }
+        }
+        break;
+      default:
+        throw new RuntimeException("Unexpected type kind " + readerType.getCategory().name());
+      }
+    }
+
+    private boolean isParseError;
+
+    /*
+     * We do this because we want the various parse methods return a primitive.
+     *
+     * @return true if there was a parse error in the last call to
+     * parseLongFromString, etc.
+     */
+    protected boolean getIsParseError() {
+      return isParseError;
+    }
+
+    protected long parseLongFromString(String string) {
+      try {
+        long longValue = Long.parseLong(string);
+        isParseError = false;
+        return longValue;
+      } catch (NumberFormatException e) {
+        isParseError = true;
+        return 0;
+      }
+    }
+
+    protected float parseFloatFromString(String string) {
+      try {
+        float floatValue = Float.parseFloat(string);
+        isParseError = false;
+        return floatValue;
+      } catch (NumberFormatException e) {
+        isParseError = true;
+        return Float.NaN;
+      }
+    }
+
+    protected double parseDoubleFromString(String string) {
+      try {
+        double value = Double.parseDouble(string);
+        isParseError = false;
+        return value;
+      } catch (NumberFormatException e) {
+        isParseError = true;
+        return Double.NaN;
+      }
+    }
+
+    /**
+     * @param string
+     * @return the HiveDecimal parsed, or null if there was a parse error.
+     */
+    protected HiveDecimal parseDecimalFromString(String string) {
+      try {
+        HiveDecimal value = HiveDecimal.create(string);
+        return value;
+      } catch (NumberFormatException e) {
+        return null;
+      }
+    }
+
+    /**
+     * @param string
+     * @return the Timestamp parsed, or null if there was a parse error.
+     */
+    protected Timestamp parseTimestampFromString(String string) {
+      try {
+        Timestamp value = Timestamp.valueOf(string);
+        return value;
+      } catch (IllegalArgumentException e) {
+        return null;
+      }
+    }
+
+    /**
+     * @param string
+     * @return the Date parsed, or null if there was a parse error.
+     */
+    protected Date parseDateFromString(String string) {
+      try {
+        Date value = Date.valueOf(string);
+        return value;
+      } catch (IllegalArgumentException e) {
+        return null;
+      }
+    }
+
+    protected String stringFromBytesColumnVectorEntry(
+        BytesColumnVector bytesColVector, int elementNum) {
+      String string;
+
+      string = new String(
+          bytesColVector.vector[elementNum],
+          bytesColVector.start[elementNum], bytesColVector.length[elementNum],
+          StandardCharsets.UTF_8);
+ 
+      return string;
+    }
+
+    @Override
+    void checkEncoding(OrcProto.ColumnEncoding encoding) throws IOException {
+      // Pass-thru.
+      convertTreeReader.checkEncoding(encoding);
+    }
+
+    @Override
+    void startStripe(Map<StreamName, InStream> streams,
+        OrcProto.StripeFooter stripeFooter
+    ) throws IOException {
+      // Pass-thru.
+      convertTreeReader.startStripe(streams, stripeFooter);
+    }
+
+    @Override
+    void seek(PositionProvider[] index) throws IOException {
+     // Pass-thru.
+      convertTreeReader.seek(index);
+    }
+
+    @Override
+    public void seek(PositionProvider index) throws IOException {
+      // Pass-thru.
+      convertTreeReader.seek(index);
+    }
+
+    @Override
+    void skipRows(long items) throws IOException {
+      // Pass-thru.
+      convertTreeReader.skipRows(items);
+    }
+
+    /**
+     * Override this to use convertVector.
+     * Source and result are member variables in the subclass with the right
+     * type.
+     * @param elementNum
+     * @throws IOException
+     */
+    // Override this to use convertVector.
+    public void setConvertVectorElement(int elementNum) throws IOException {
+      throw new RuntimeException("Expected this method to be overriden");
+    }
+
+    // Common code used by the conversion.
+    public void convertVector(ColumnVector fromColVector,
+        ColumnVector resultColVector, final int batchSize) throws IOException {
+
+      resultColVector.reset();
+      if (fromColVector.isRepeating) {
+        resultColVector.isRepeating = true;
+        if (fromColVector.noNulls || !fromColVector.isNull[0]) {
+          setConvertVectorElement(0);
+        } else {
+          resultColVector.noNulls = false;
+          resultColVector.isNull[0] = true;
+        }
+      } else if (fromColVector.noNulls){
+        for (int i = 0; i < batchSize; i++) {
+          setConvertVectorElement(i);
+        }
+      } else {
+        for (int i = 0; i < batchSize; i++) {
+          if (!fromColVector.isNull[i]) {
+            setConvertVectorElement(i);
+          } else {
+            resultColVector.noNulls = false;
+            resultColVector.isNull[i] = true;
+          }
+        }
+      }
+    }
+
+    public long downCastAnyInteger(long input, TypeDescription readerType) {
+      switch (readerType.getCategory()) {
+      case BOOLEAN:
+        return input == 0 ? 0 : 1;
+      case BYTE:
+        return (byte) input;
+      case SHORT:
+        return (short) input;
+      case INT:
+        return (int) input;
+      case LONG:
+        return input;
+      default:
+        throw new RuntimeException("Unexpected type kind " + readerType.getCategory().name());
+      }
+    }
+
+    protected boolean integerDownCastNeeded(TypeDescription fileType, TypeDescription readerType) {
+      Integer fileLevel = numericTypes.get(fileType.getCategory());
+      Integer schemaLevel = numericTypes.get(readerType.getCategory());
+      return (schemaLevel.intValue() < fileLevel.intValue());
+    }
+  }
+
+  public static class AnyIntegerTreeReader extends ConvertTreeReader {
+
+    private TypeDescription.Category fileTypeCategory;
+    private TreeReader anyIntegerTreeReader;
+
+    private long longValue;
+
+    AnyIntegerTreeReader(int columnId, TypeDescription fileType,
+        boolean skipCorrupt) throws IOException {
+      super(columnId);
+      this.fileTypeCategory = fileType.getCategory();
+      switch (fileTypeCategory) {
+      case BOOLEAN:
+        anyIntegerTreeReader = new BooleanTreeReader(columnId);
+        break;
+      case BYTE:
+        anyIntegerTreeReader = new ByteTreeReader(columnId);
+        break;
+      case SHORT:
+        anyIntegerTreeReader = new ShortTreeReader(columnId);
+        break;
+      case INT:
+        anyIntegerTreeReader = new IntTreeReader(columnId);
+        break;
+      case LONG:
+        anyIntegerTreeReader = new LongTreeReader(columnId, skipCorrupt);
+        break;
+      default:
+        throw new RuntimeException("Unexpected type kind " + fileType.getCategory().name());
+      }
+      setConvertTreeReader(anyIntegerTreeReader);
+    }
+
+    protected long getLong() throws IOException {
+      return longValue;
+    }
+
+    protected String getString(long longValue) {
+      if (fileTypeCategory == TypeDescription.Category.BOOLEAN) {
+        return longValue == 0 ? "FALSE" : "TRUE";
+      } else {
+        return Long.toString(longValue);
+      }
+    }
+
+    protected String getString() {
+      return getString(longValue);
+    }
+
+    @Override
+    public void nextVector(ColumnVector previousVector,
+                           boolean[] isNull,
+                           final int batchSize) throws IOException {
+      anyIntegerTreeReader.nextVector(previousVector, isNull, batchSize);
+    }
+  }
+
+  public static class AnyIntegerFromAnyIntegerTreeReader extends ConvertTreeReader {
+
+    private AnyIntegerTreeReader anyIntegerAsLongTreeReader;
+
+    private final TypeDescription readerType;
+    private final boolean downCastNeeded;
+
+    AnyIntegerFromAnyIntegerTreeReader(int columnId, TypeDescription fileType, TypeDescription readerType, boolean skipCorrupt) throws IOException {
+      super(columnId);
+      this.readerType = readerType;
+      anyIntegerAsLongTreeReader = new AnyIntegerTreeReader(columnId, fileType, skipCorrupt);
+      setConvertTreeReader(anyIntegerAsLongTreeReader);
+      downCastNeeded = integerDownCastNeeded(fileType, readerType);
+    }
+
+    @Override
+    public void nextVector(ColumnVector previousVector,
+                           boolean[] isNull,
+                           final int batchSize) throws IOException {
+      anyIntegerAsLongTreeReader.nextVector(previousVector, isNull, batchSize);
+      LongColumnVector resultColVector = (LongColumnVector) previousVector;
+      if (downCastNeeded) {
+        long[] resultVector = resultColVector.vector;
+        if (resultColVector.isRepeating) {
+          if (resultColVector.noNulls || !resultColVector.isNull[0]) {
+            resultVector[0] = downCastAnyInteger(resultVector[0], readerType);
+          } else {
+            resultColVector.noNulls = false;
+            resultColVector.isNull[0] = true;
+          }
+        } else if (resultColVector.noNulls){
+          for (int i = 0; i < batchSize; i++) {
+            resultVector[i] = downCastAnyInteger(resultVector[i], readerType);
+          }
+        } else {
+          for (int i = 0; i < batchSize; i++) {
+            if (!resultColVector.isNull[i]) {
+              resultVector[i] = downCastAnyInteger(resultVector[i], readerType);
+            } else {
+              resultColVector.noNulls = false;
+              resultColVector.isNull[i] = true;
+            }
+          }
+        }
+      }
+    }
+  }
+
+  public static class AnyIntegerFromFloatTreeReader extends ConvertTreeReader {
+
+    private FloatTreeReader floatTreeReader;
+
+    private final TypeDescription readerType;
+    private FloatWritable floatResult;
+    private DoubleColumnVector doubleColVector;
+    private LongColumnVector longColVector;
+
+    AnyIntegerFromFloatTreeReader(int columnId, TypeDescription readerType)
+        throws IOException {
+      super(columnId);
+      this.readerType = readerType;
+      floatTreeReader = new FloatTreeReader(columnId);
+      setConvertTreeReader(floatTreeReader);
+      floatResult = new FloatWritable();
+    }
+
+    @Override
+    public void setConvertVectorElement(int elementNum) throws IOException {
+      float floatValue = (float) doubleColVector.vector[elementNum];
+      longColVector.vector[elementNum] =
+          downCastAnyInteger(
+              (long) floatValue, readerType);
+    }
+
+    @Override
+    public void nextVector(ColumnVector previousVector,
+                           boolean[] isNull,
+                           final int batchSize) throws IOException {
+      if (doubleColVector == null) {
+        // Allocate column vector for file; cast column vector for reader.
+        doubleColVector = new DoubleColumnVector();
+        longColVector = (LongColumnVector) previousVector;
+      }
+      // Read present/isNull stream
+      floatTreeReader.nextVector(doubleColVector, isNull, batchSize);
+
+      convertVector(doubleColVector, longColVector, batchSize);
+    }
+  }
+
+  public static class AnyIntegerFromDoubleTreeReader extends ConvertTreeReader {
+
+    private DoubleTreeReader doubleTreeReader;
+
+    private final TypeDescription readerType;
+    private DoubleColumnVector doubleColVector;
+    private LongColumnVector longColVector;
+
+    AnyIntegerFromDoubleTreeReader(int columnId, TypeDescription readerType)
+        throws IOException {
+      super(columnId);
+      this.readerType = readerType;
+      doubleTreeReader = new DoubleTreeReader(columnId);
+      setConvertTreeReader(doubleTreeReader);
+    }
+
+    @Override
+    public void setConvertVectorElement(int elementNum) throws IOException {
+      longColVector.vector[elementNum] =
+          downCastAnyInteger(
+              (long) doubleColVector.vector[elementNum], readerType);
+    }
+
+    @Override
+    public void nextVector(ColumnVector previousVector,
+                           boolean[] isNull,
+                           final int batchSize) throws IOException {
+      if (doubleColVector == null) {
+        // Allocate column vector for file; cast column vector for reader.
+        doubleColVector = new DoubleColumnVector();
+        longColVector = (LongColumnVector) previousVector;
+      }
+      // Read present/isNull stream
+      doubleTreeReader.nextVector(doubleColVector, isNull, batchSize);
+
+      convertVector(doubleColVector, longColVector, batchSize);
+    }
+  }
+
+  public static class AnyIntegerFromDecimalTreeReader extends ConvertTreeReader {
+
+    private DecimalTreeReader decimalTreeReader;
+
+    private final int precision;
+    private final int scale;
+    private final TypeDescription readerType;
+    private HiveDecimalWritable hiveDecimalResult;
+    private DecimalColumnVector decimalColVector;
+    private LongColumnVector longColVector;
+
+    AnyIntegerFromDecimalTreeReader(int columnId, TypeDescription fileType,
+        TypeDescription readerType) throws IOException {
+      super(columnId);
+      this.precision = fileType.getPrecision();
+      this.scale = fileType.getScale();
+      this.readerType = readerType;
+      decimalTreeReader = new DecimalTreeReader(columnId, precision, scale);
+      setConvertTreeReader(decimalTreeReader);
+      hiveDecimalResult = new HiveDecimalWritable();
+    }
+
+    @Override
+    public void setConvertVectorElement(int elementNum) throws IOException {
+      longColVector.vector[elementNum] =
+          downCastAnyInteger(
+              decimalColVector.vector[elementNum].getHiveDecimal().longValue(),
+              readerType);
+    }
+
+    @Override
+    public void nextVector(ColumnVector previousVector,
+                           boolean[] isNull,
+                           final int batchSize) throws IOException {
+      if (decimalColVector == null) {
+        // Allocate column vector for file; cast column vector for reader.
+        decimalColVector = new DecimalColumnVector(precision, scale);
+        longColVector = (LongColumnVector) previousVector;
+      }
+      // Read present/isNull stream
+      decimalTreeReader.nextVector(decimalColVector, isNull, batchSize);
+
+      convertVector(decimalColVector, longColVector, batchSize);
+    }
+  }
+
+  public static class AnyIntegerFromStringGroupTreeReader extends ConvertTreeReader {
+
+    private TreeReader stringGroupTreeReader;
+
+    private final TypeDescription fileType;
+    private final TypeDescription readerType;
+    private BytesColumnVector bytesColVector;
+    private LongColumnVector longColVector;
+
+    AnyIntegerFromStringGroupTreeReader(int columnId, TypeDescription fileType,
+        TypeDescription readerType) throws IOException {
+      super(columnId);
+      this.fileType = fileType;
+      this.readerType = readerType;
+      stringGroupTreeReader = getStringGroupTreeReader(columnId, fileType);
+      setConvertTreeReader(stringGroupTreeReader);
+    }
+
+    @Override
+    public void setConvertVectorElement(int elementNum) throws IOException {
+      String string = stringFromBytesColumnVectorEntry(bytesColVector, elementNum);
+      long longValue = parseLongFromString(string);
+      if (!getIsParseError()) {
+        longColVector.vector[elementNum] =
+            downCastAnyInteger(longValue, readerType);
+      } else {
+        longColVector.noNulls = false;
+        longColVector.isNull[elementNum] = true;
+      }
+    }
+
+    @Override
+    public void nextVector(ColumnVector previousVector,
+                           boolean[] isNull,
+                           final int batchSize) throws IOException {
+      if (bytesColVector == null) {
+        // Allocate column vector for file; cast column vector for reader.
+        bytesColVector = new BytesColumnVector();
+        longColVector = (LongColumnVector) previousVector;
+      }
+      // Read present/isNull stream
+      stringGroupTreeReader.nextVector(bytesColVector, isNull, batchSize);
+
+      convertVector(bytesColVector, longColVector, batchSize);
+    }
+  }
+
+  public static class AnyIntegerFromTimestampTreeReader extends ConvertTreeReader {
+
+    private TimestampTreeReader timestampTreeReader;
+
+    private final TypeDescription readerType;
+    private TimestampColumnVector timestampColVector;
+    private LongColumnVector longColVector;
+
+    AnyIntegerFromTimestampTreeReader(int columnId, TypeDescription readerType,
+        boolean skipCorrupt) throws IOException {
+      super(columnId);
+      this.readerType = readerType;
+      timestampTreeReader = new TimestampTreeReader(columnId, skipCorrupt);
+      setConvertTreeReader(timestampTreeReader);
+    }
+
+    @Override
+    public void setConvertVectorElement(int elementNum) throws IOException {
+      // Use TimestampWritable's getSeconds.
+      long longValue = TimestampUtils.millisToSeconds(
+          timestampColVector.asScratchTimestamp(elementNum).getTime());
+      longColVector.vector[elementNum] =
+          downCastAnyInteger(longValue, readerType);
+    }
+
+    @Override
+    public void nextVector(ColumnVector previousVector,
+                           boolean[] isNull,
+                           final int batchSize) throws IOException {
+      if (timestampColVector == null) {
+        // Allocate column vector for file; cast column vector for reader.
+        timestampColVector = new TimestampColumnVector();
+        longColVector = (LongColumnVector) previousVector;
+      }
+      // Read present/isNull stream
+      timestampTreeReader.nextVector(timestampColVector, isNull, batchSize);
+
+      convertVector(timestampColVector, longColVector, batchSize);
+    }
+  }
+
+  public static class FloatFromAnyIntegerTreeReader extends ConvertTreeReader {
+
+    private AnyIntegerTreeReader anyIntegerAsLongTreeReader;
+
+    private LongColumnVector longColVector;
+    private DoubleColumnVector doubleColVector;
+
+    FloatFromAnyIntegerTreeReader(int columnId, TypeDescription fileType,
+        boolean skipCorrupt) throws IOException {
+      super(columnId);
+      anyIntegerAsLongTreeReader =
+          new AnyIntegerTreeReader(columnId, fileType, skipCorrupt);
+      setConvertTreeReader(anyIntegerAsLongTreeReader);
+    }
+
+    @Override
+    public void setConvertVectorElement(int elementNum) throws IOException {
+      float floatValue = (float) longColVector.vector[elementNum];
+      if (!Float.isNaN(floatValue)) {
+        doubleColVector.vector[elementNum] = floatValue;
+      } else {
+        doubleColVector.vector[elementNum] = Double.NaN;
+        doubleColVector.noNulls = false;
+        doubleColVector.isNull[elementNum] = true;
+      }
+    }
+
+    @Override
+    public void nextVector(ColumnVector previousVector,
+                           boolean[] isNull,
+                           final int batchSize) throws IOException {
+      if (longColVector == null) {
+        // Allocate column vector for file; cast column vector for reader.
+        longColVector = new LongColumnVector();
+        doubleColVector = (DoubleColumnVector) previousVector;
+      }
+      // Read present/isNull stream
+      anyIntegerAsLongTreeReader.nextVector(longColVector, isNull, batchSize);
+
+      convertVector(longColVector, doubleColVector, batchSize);
+    }
+  }
+
+  public static class FloatFromDoubleTreeReader extends ConvertTreeReader {
+
+    private DoubleTreeReader doubleTreeReader;
+
+    FloatFromDoubleTreeReader(int columnId) throws IOException {
+      super(columnId);
+      doubleTreeReader = new DoubleTreeReader(columnId);
+      setConvertTreeReader(doubleTreeReader);
+    }
+
+    @Override
+    public void nextVector(ColumnVector previousVector,
+                           boolean[] isNull,
+                           final int batchSize) throws IOException {
+      doubleTreeReader.nextVector(previousVector, isNull, batchSize);
+
+      DoubleColumnVector resultColVector = (DoubleColumnVector) previousVector;
+      double[] resultVector = resultColVector.vector;
+      if (resultColVector.isRepeating) {
+        if (resultColVector.noNulls || !resultColVector.isNull[0]) {
+          resultVector[0] = (float) resultVector[0];
+        } else {
+          resultColVector.noNulls = false;
+          resultColVector.isNull[0] = true;
+        }
+      } else if (resultColVector.noNulls){
+        for (int i = 0; i < batchSize; i++) {
+          resultVector[i] = (float) resultVector[i];
+        }
+      } else {
+        for (int i = 0; i < batchSize; i++) {
+          if (!resultColVector.isNull[i]) {
+            resultVector[i] = (float) resultVector[i];
+          } else {
+            resultColVector.noNulls = false;
+            resultColVector.isNull[i] = true;
+          }
+        }
+      }
+    }
+  }
+
+  public static class FloatFromDecimalTreeReader extends ConvertTreeReader {
+
+    private DecimalTreeReader decimalTreeReader;
+
+    private final int precision;
+    private final int scale;
+    private final TypeDescription readerType;
+    private HiveDecimalWritable hiveDecimalResult;
+    private DecimalColumnVector decimalColVector;
+    private DoubleColumnVector doubleColVector;
+
+    FloatFromDecimalTreeReader(int columnId, TypeDescription fileType,
+        TypeDescription readerType) throws IOException {
+      super(columnId);
+      this.precision = fileType.getPrecision();
+      this.scale = fileType.getScale();
+      this.readerType = readerType;
+      decimalTreeReader = new DecimalTreeReader(columnId, precision, scale);
+      setConvertTreeReader(decimalTreeReader);
+      hiveDecimalResult = new HiveDecimalWritable();
+    }
+
+    @Override
+    public void setConvertVectorElement(int elementNum) throws IOException {
+      doubleColVector.vector[elementNum] =
+          (float) decimalColVector.vector[elementNum].getHiveDecimal().doubleValue();
+    }
+
+    @Override
+    public void nextVector(ColumnVector previousVector,
+                           boolean[] isNull,
+                           final int batchSize) throws IOException {
+      if (decimalColVector == null) {
+        // Allocate column vector for file; cast column vector for reader.
+        decimalColVector = new DecimalColumnVector(precision, scale);
+        doubleColVector = (DoubleColumnVector) previousVector;
+      }
+      // Read present/isNull stream
+      decimalTreeReader.nextVector(decimalColVector, isNull, batchSize);
+
+      convertVector(decimalColVector, doubleColVector, batchSize);
+    }
+  }
+
+  public static class FloatFromStringGroupTreeReader extends ConvertTreeReader {
+
+    private TreeReader stringGroupTreeReader;
+
+    private final TypeDescription fileType;
+    private BytesColumnVector bytesColVector;
+    private DoubleColumnVector doubleColVector;
+
+    FloatFromStringGroupTreeReader(int columnId, TypeDescription fileType)
+        throws IOException {
+      super(columnId);
+      this.fileType = fileType;
+      stringGroupTreeReader = getStringGroupTreeReader(columnId, fileType);
+      setConvertTreeReader(stringGroupTreeReader);
+    }
+
+    @Override
+    public void setConvertVectorElement(int elementNum) throws IOException {
+      String string = stringFromBytesColumnVectorEntry(bytesColVector, elementNum);
+      float floatValue = parseFloatFromString(string);
+      if (!getIsParseError()) {
+        doubleColVector.vector[elementNum] = floatValue;
+      } else {
+        doubleColVector.vector[elementNum] = Double.NaN;
+        doubleColVector.noNulls = false;
+        doubleColVector.isNull[elementNum] = true;
+      }
+    }
+
+    @Override
+    public void nextVector(ColumnVector previousVector,
+                           boolean[] isNull,
+                           final int batchSize) throws IOException {
+      if (bytesColVector == null) {
+        // Allocate column vector for file; cast column vector for reader.
+        bytesColVector = new BytesColumnVector();
+        doubleColVector = (DoubleColumnVector) previousVector;
+      }
+      // Read present/isNull stream
+      stringGroupTreeReader.nextVector(bytesColVector, isNull, batchSize);
+
+      convertVector(bytesColVector, doubleColVector, batchSize);
+    }
+  }
+
+  public static class FloatFromTimestampTreeReader extends ConvertTreeReader {
+
+    private TimestampTreeReader timestampTreeReader;
+
+    private final TypeDescription readerType;
+    private TimestampColumnVector timestampColVector;
+    private DoubleColumnVector doubleColVector;
+
+    FloatFromTimestampTreeReader(int columnId, TypeDescription readerType,
+        boolean skipCorrupt) throws IOException {
+      super(columnId);
+      this.readerType = readerType;
+      timestampTreeReader = new TimestampTreeReader(columnId, skipCorrupt);
+      setConvertTreeReader(timestampTreeReader);
+    }
+
+    @Override
+    public void setConvertVectorElement(int elementNum) throws IOException {
+      doubleColVector.vector[elementNum] = (float) TimestampUtils.getDouble(
+          timestampColVector.asScratchTimestamp(elementNum));
+    }
+
+    @Override
+    public void nextVector(ColumnVector previousVector,
+                           boolean[] isNull,
+                           final int batchSize) throws IOException {
+      if (timestampColVector == null) {
+        // Allocate column vector for file; cast column vector for reader.
+        timestampColVector = new TimestampColumnVector();
+        doubleColVector = (DoubleColumnVector) previousVector;
+      }
+      // Read present/isNull stream
+      timestampTreeReader.nextVector(timestampColVector, isNull, batchSize);
+
+      convertVector(timestampColVector, doubleColVector, batchSize);
+    }
+  }
+
+  public static class DoubleFromAnyIntegerTreeReader extends ConvertTreeReader {
+
+    private AnyIntegerTreeReader anyIntegerAsLongTreeReader;
+
+    private LongColumnVector longColVector;
+    private DoubleColumnVector doubleColVector;
+
+    DoubleFromAnyIntegerTreeReader(int columnId, TypeDescription fileType,
+        boolean skipCorrupt) throws IOException {
+      super(columnId);
+      anyIntegerAsLongTreeReader =
+          new AnyIntegerTreeReader(columnId, fileType, skipCorrupt);
+      setConvertTreeReader(anyIntegerAsLongTreeReader);
+    }
+
+    @Override
+    public void setConvertVectorElement(int elementNum) {
+
+      double doubleValue = (double) longColVector.vector[elementNum];
+      if (!Double.isNaN(doubleValue)) {
+        doubleColVector.vector[elementNum] = doubleValue;
+      } else {
+        doubleColVector.vector[elementNum] = Double.NaN;
+        doubleColVector.noNulls = false;
+        doubleColVector.isNull[elementNum] = true;
+      }
+    }
+
+    @Override
+    public void nextVector(ColumnVector previousVector,
+                           boolean[] isNull,
+                           final int batchSize) throws IOException {
+      if (longColVector == null) {
+        // Allocate column vector for file; cast column vector for reader.
+        longColVector = new LongColumnVector();
+        doubleColVector = (DoubleColumnVector) previousVector;
+      }
+      // Read present/isNull stream
+      anyIntegerAsLongTreeReader.nextVector(longColVector, isNull, batchSize);
+
+      convertVector(longColVector, doubleColVector, batchSize);
+    }
+  }
+
+  public static class DoubleFromFloatTreeReader extends ConvertTreeReader {
+
+    private FloatTreeReader floatTreeReader;
+
+    private FloatWritable floatResult;
+
+    DoubleFromFloatTreeReader(int columnId) throws IOException {
+      super(columnId);
+      floatTreeReader = new FloatTreeReader(columnId);
+      setConvertTreeReader(floatTreeReader);
+      floatResult = new FloatWritable();
+    }
+
+    @Override
+    public void nextVector(ColumnVector previousVector,
+                           boolean[] isNull,
+                           final int batchSize) throws IOException {
+      // The DoubleColumnVector produced by FloatTreeReader is what we want.
+      floatTreeReader.nextVector(previousVector, isNull, batchSize);
+    }
+  }
+
+  public static class DoubleFromDecimalTreeReader extends ConvertTreeReader {
+
+    private DecimalTreeReader decimalTreeReader;
+
+    private final int precision;
+    private final int scale;
+    private final TypeDescription readerType;
+    private HiveDecimalWritable hiveDecimalResult;
+    private DecimalColumnVector decimalColVector;
+    private DoubleColumnVector doubleColVector;
+
+    DoubleFromDecimalTreeReader(int columnId, TypeDescription fileType,
+        TypeDescription readerType) throws IOException {
+      super(columnId);
+      this.precision = fileType.getPrecision();
+      this.scale = fileType.getScale();
+      this.readerType = readerType;
+      decimalTreeReader = new DecimalTreeReader(columnId, precision, scale);
+      setConvertTreeReader(decimalTreeReader);
+      hiveDecimalResult = new HiveDecimalWritable();
+    }
+
+    @Override
+    public void setConvertVectorElement(int elementNum) throws IOException {
+      doubleColVector.vector[elementNum] =
+          decimalColVector.vector[elementNum].getHiveDecimal().doubleValue();
+    }
+
+    @Override
+    public void nextVector(ColumnVector previousVector,
+                           boolean[] isNull,
+                           final int batchSize) throws IOException {
+      if (decimalColVector == null) {
+        // Allocate column vector for file; cast column vector for reader.
+        decimalColVector = new DecimalColumnVector(precision, scale);
+        doubleColVector = (DoubleColumnVector) previousVector;
+      }
+      // Read present/isNull stream
+      decimalTreeReader.nextVector(decimalColVector, isNull, batchSize);
+
+      convertVector(decimalColVector, doubleColVector, batchSize);
+    }
+  }
+
+  public static class DoubleFromStringGroupTreeReader extends ConvertTreeReader {
+
+    private TreeReader stringGroupTreeReader;
+
+    private final TypeDescription fileType;
+    private BytesColumnVector bytesColVector;
+    private DoubleColumnVector doubleColVector;
+
+    DoubleFromStringGroupTreeReader(int columnId, TypeDescription fileType)
+        throws IOException {
+      super(columnId);
+      this.fileType = fileType;
+      stringGroupTreeReader = getStringGroupTreeReader(columnId, fileType);
+      setConvertTreeReader(stringGroupTreeReader);
+    }
+
+    @Override
+    public void setConvertVectorElement(int elementNum) throws IOException {
+      String string = stringFromBytesColumnVectorEntry(bytesColVector, elementNum);
+      double doubleValue = parseDoubleFromString(string);
+      if (!getIsParseError()) {
+        doubleColVector.vector[elementNum] = doubleValue;
+      } else {
+        doubleColVector.noNulls = false;
+        doubleColVector.isNull[elementNum] = true;
+      }
+    }
+
+    @Override
+    public void nextVector(ColumnVector previousVector,
+                           boolean[] isNull,
+                           final int batchSize) throws IOException {
+      if (bytesColVector == null) {
+        // Allocate column vector for file; cast column vector for reader.
+        bytesColVector = new BytesColumnVector();
+        doubleColVector = (DoubleColumnVector) previousVector;
+      }
+      // Read present/isNull stream
+      stringGroupTreeReader.nextVector(bytesColVector, isNull, batchSize);
+
+      convertVector(bytesColVector, doubleColVector, batchSize);
+    }
+  }
+
+  public static class DoubleFromTimestampTreeReader extends ConvertTreeReader {
+
+    private TimestampTreeReader timestampTreeReader;
+
+    private final TypeDescription readerType;
+    private TimestampColumnVector timestampColVector;
+    private DoubleColumnVector doubleColVector;
+
+    DoubleFromTimestampTreeReader(int columnId, TypeDescription readerType,
+        boolean skipCorrupt) throws IOException {
+      super(columnId);
+      this.readerType = readerType;
+      timestampTreeReader = new TimestampTreeReader(columnId, skipCorrupt);
+      setConvertTreeReader(timestampTreeReader);
+    }
+
+    @Override
+    public void setConvertVectorElement(int elementNum) throws IOException {
+      doubleColVector.vector[elementNum] = TimestampUtils.getDouble(
+          timestampColVector.asScratchTimestamp(elementNum));
+    }
+
+    @Override
+    public void nextVector(ColumnVector previousVector,
+                           boolean[] isNull,
+                           final int batchSize) throws IOException {
+      if (timestampColVector == null) {
+        // Allocate column vector for file; cast column vector for reader.
+        timestampColVector = new TimestampColumnVector();
+        doubleColVector = (DoubleColumnVector) previousVector;
+      }
+      // Read present/isNull stream
+      timestampTreeReader.nextVector(timestampColVector, isNull, batchSize);
+
+      convertVector(timestampColVector, doubleColVector, batchSize);
+    }
+  }
+
+  public static class DecimalFromAnyIntegerTreeReader extends ConvertTreeReader {
+
+    private AnyIntegerTreeReader anyIntegerAsLongTreeReader;
+
+    private int precision;
+    private int scale;
+    private LongColumnVector longColVector;
+    private DecimalColumnVector decimalColVector;
+
+    DecimalFromAnyIntegerTreeReader(int columnId, TypeDescription fileType,
+        TypeDescription readerType, boolean skipCorrupt) throws IOException {
+      super(columnId);
+      this.precision = readerType.getPrecision();
+      this.scale = readerType.getScale();
+      anyIntegerAsLongTreeReader =
+          new AnyIntegerTreeReader(columnId, fileType, skipCorrupt);
+      setConvertTreeReader(anyIntegerAsLongTreeReader);
+    }
+
+    @Override
+    public void setConvertVectorElement(int elementNum) {
+      long longValue = longColVector.vector[elementNum];
+      HiveDecimalWritable hiveDecimalWritable =
+          new HiveDecimalWritable(longValue);
+      decimalColVector.set(elementNum, hiveDecimalWritable);
+    }
+
+    @Override
+    public void nextVector(ColumnVector previousVector,
+        boolean[] isNull,
+        final int batchSize) throws IOException {
+      if (longColVector == null) {
+        // Allocate column vector for file; cast column vector for reader.
+        longColVector = new LongColumnVector();
+        decimalColVector = (DecimalColumnVector) previousVector;
+      }
+      // Read present/isNull stream
+      anyIntegerAsLongTreeReader.nextVector(longColVector, isNull, batchSize);
+
+      convertVector(longColVector, decimalColVector, batchSize);
+    }
+  }
+
+  public static class DecimalFromFloatTreeReader extends ConvertTreeReader {
+
+    private FloatTreeReader floatTreeReader;
+
+    private int precision;
+    private int scale;
+    private FloatWritable floatResult;
+    private DoubleColumnVector doubleColVector;
+    private DecimalColumnVector decimalColVector;
+
+    DecimalFromFloatTreeReader(int columnId, TypeDescription readerType)
+        throws IOException {
+      super(columnId);
+      this.precision = readerType.getPrecision();
+      this.scale = readerType.getScale();
+      floatTreeReader = new FloatTreeReader(columnId);
+      setConvertTreeReader(floatTreeReader);
+      floatResult = new FloatWritable();
+    }
+
+    @Override
+    public void setConvertVectorElement(int elementNum) throws IOException {
+      float floatValue = (float) doubleColVector.vector[elementNum];
+      if (!Float.isNaN(floatValue)) {
+        HiveDecimal value =
+            HiveDecimal.create(Float.toString(floatValue));
+        if (value != null) {
+          decimalColVector.set(elementNum, value);
+        } else {
+          decimalColVector.noNulls = false;
+          decimalColVector.isNull[elementNum] = true;
+        }
+      } else {
+        decimalColVector.noNulls = false;
+        decimalColVector.isNull[elementNum] = true;
+      }
+    }
+
+    @Override
+    public void nextVector(ColumnVector previousVector,
+                           boolean[] isNull,
+                           final int batchSize) throws IOException {
+      if (doubleColVector == null) {
+        // Allocate column vector for file; cast column vector for reader.
+        doubleColVector = new DoubleColumnVector();
+        decimalColVector = (DecimalColumnVector) previousVector;
+      }
+      // Read present/isNull stream
+      floatTreeReader.nextVector(doubleColVector, isNull, batchSize);
+
+      convertVector(doubleColVector, decimalColVector, batchSize);
+    }
+  }
+
+  public static class DecimalFromDoubleTreeReader extends ConvertTreeReader {
+
+    private DoubleTreeReader doubleTreeReader;
+
+    private DoubleColumnVector doubleColVector;
+    private DecimalColumnVector decimalColVector;
+
+    DecimalFromDoubleTreeReader(int columnId, TypeDescription readerType)
+        throws IOException {
+      super(columnId);
+      doubleTreeReader = new DoubleTreeReader(columnId);
+      setConvertTreeReader(doubleTreeReader);
+    }
+
+    @Override
+    public void setConvertVectorElement(int elementNum) throws IOException {
+      HiveDecimal value =
+          HiveDecimal.create(Double.toString(doubleColVector.vector[elementNum]));
+      if (value != null) {
+        decimalColVector.set(elementNum, value);
+      } else {
+        decimalColVector.noNulls = false;
+        decimalColVector.isNull[elementNum] = true;
+      }
+    }
+
+    @Override
+    public void nextVector(ColumnVector previousVector,
+                           boolean[] isNull,
+                           final int batchSize) throws IOException {
+      if (doubleColVector == null) {
+        // Allocate column vector for file; cast column vector for reader.
+        doubleColVector = new DoubleColumnVector();
+        decimalColVector = (DecimalColumnVector) previousVector;
+      }
+      // Read present/isNull stream
+      doubleTreeReader.nextVector(doubleColVector, isNull, batchSize);
+
+      convertVector(doubleColVector, decimalColVector, batchSize);
+    }
+  }
+
+  public static class DecimalFromStringGroupTreeReader extends ConvertTreeReader {
+
+    private TreeReader stringGroupTreeReader;
+
+    private final TypeDescription fileType;
+    private BytesColumnVector bytesColVector;
+    private DecimalColumnVector decimalColVector;
+
+    DecimalFromStringGroupTreeReader(int columnId, TypeDescription fileType,
+        TypeDescription readerType) throws IOException {
+      super(columnId);
+      this.fileType = fileType;
+      stringGroupTreeReader = getStringGroupTreeReader(columnId, fileType);
+      setConvertTreeReader(stringGroupTreeReader);
+    }
+
+    @Override
+    public void setConvertVectorElement(int elementNum) throws IOException {
+      String string = stringFromBytesColumnVectorEntry(bytesColVector, elementNum);
+      HiveDecimal value = parseDecimalFromString(string);
+      if (value != null) {
+        decimalColVector.set(elementNum, value);
+      } else {
+        decimalColVector.noNulls = false;
+        decimalColVector.isNull[elementNum] = true;
+      }
+    }
+
+    @Override
+    public void nextVector(ColumnVector previousVector,
+                           boolean[] isNull,
+                           final int batchSize) throws IOException {
+      if (bytesColVector == null) {
+        // Allocate column vector for file; cast column vector for reader.
+        bytesColVector = new BytesColumnVector();
+        decimalColVector = (DecimalColumnVector) previousVector;
+      }
+      // Read present/isNull stream
+      stringGroupTreeReader.nextVector(bytesColVector, isNull, batchSize);
+
+      convertVector(bytesColVector, decimalColVector, batchSize);
+    }
+  }
+
+  public static class DecimalFromTimestampTreeReader extends ConvertTreeReader {
+
+    private TimestampTreeReader timestampTreeReader;
+
+    private final TypeDescription readerType;
+    private TimestampColumnVector timestampColVector;
+    private int precision;
+    private int scale;
+    private DecimalColumnVector decimalColVector;
+
+    DecimalFromTimestampTreeReader(int columnId, TypeDescription readerType,
+        boolean skipCorrupt) throws IOException {
+      super(columnId);
+      this.readerType = readerType;
+      this.precision = readerType.getPrecision();
+      this.scale = readerType.getScale();
+      timestampTreeReader = new TimestampTreeReader(columnId, skipCorrupt);
+      setConvertTreeReader(timestampTreeReader);
+    }
+
+    @Override
+    public void setConvertVectorElement(int elementNum) throws IOException {
+      double doubleValue = TimestampUtils.getDouble(
+          timestampColVector.asScratchTimestamp(elementNum));
+      HiveDecimal value = HiveDecimal.create(Double.toString(doubleValue));
+      if (value != null) {
+        decimalColVector.set(elementNum, value);
+      } else {
+        decimalColVector.noNulls = false;
+        decimalColVector.isNull[elementNum] = true;
+      }
+    }
+
+    @Override
+    public void nextVector(ColumnVector previousVector,
+                           boolean[] isNull,
+                           final int batchSize) throws IOException {
+      if (timestampColVector == null) {
+        // Allocate column vector for file; cast column vector for reader.
+        timestampColVector = new TimestampColumnVector();
+        decimalColVector = (DecimalColumnVector) previousVector;
+      }
+      // Read present/isNull stream
+      timestampTreeReader.nextVector(timestampColVector, isNull, batchSize);
+
+      convertVector(timestampColVector, decimalColVector, batchSize);
+    }
+  }
+
+  public static class StringGroupFromAnyIntegerTreeReader extends ConvertTreeReader {
+
+    private AnyIntegerTreeReader anyIntegerAsLongTreeReader;
+
+    private final TypeDescription fileType;
+    private final TypeDescription readerType;
+    private LongColumnVector longColVector;
+    private BytesColumnVector bytesColVector;
+
+    StringGroupFromAnyIntegerTreeReader(int columnId, TypeDescription fileType,
+        TypeDescription readerType, boolean skipCorrupt) throws IOException {
+      super(columnId);
+      this.fileType = fileType;
+      this.readerType = readerType;
+      anyIntegerAsLongTreeReader =
+          new AnyIntegerTreeReader(columnId, fileType, skipCorrupt);
+      setConvertTreeReader(anyIntegerAsLongTreeReader);
+    }
+
+    @Override
+    public void setConvertVectorElement(int elementNum) {
+      long longValue = longColVector.vector[elementNum];
+      String string = anyIntegerAsLongTreeReader.getString(longValue);
+      byte[] bytes = string.getBytes();
+      assignStringGroupVectorEntry(bytesColVector, elementNum, readerType, bytes);
+    }
+
+    @Override
+    public void nextVector(ColumnVector previousVector,
+                           boolean[] isNull,
+                           final int batchSize) throws IOException {
+      if (longColVector == null) {
+        // Allocate column vector for file; cast column vector for reader.
+        longColVector = new LongColumnVector();
+        bytesColVector = (BytesColumnVector) previousVector;
+      }
+      // Read present/isNull stream
+      anyIntegerAsLongTreeReader.nextVector(longColVector, isNull, batchSize);
+
+      convertVector(longColVector, bytesColVector, batchSize);
+    }
+  }
+
+  public static class StringGroupFromFloatTreeReader extends ConvertTreeReader {
+
+    private FloatTreeReader floatTreeReader;
+
+    private final TypeDescription readerType;
+    private FloatWritable floatResult;
+    private DoubleColumnVector doubleColVector;
+    private BytesColumnVector bytesColVector;
+
+
+    StringGroupFromFloatTreeReader(int columnId, TypeDescription readerType,
+        boolean skipCorrupt) throws IOException {
+      super(columnId);
+      this.readerType = readerType;
+      floatTreeReader = new FloatTreeReader(columnId);
+      setConvertTreeReader(floatTreeReader);
+      floatResult = new FloatWritable();
+    }
+
+    @Override
+    public void setConvertVectorElement(int elementNum) {
+      float floatValue = (float) doubleColVector.vector[elementNum];
+      if (!Float.isNaN(floatValue)) {
+        String string = String.valueOf(floatValue);
+        byte[] bytes = string.getBytes();
+        assignStringGroupVectorEntry(bytesColVector, elementNum, readerType, bytes);
+      } else {
+        bytesColVector.noNulls = false;
+        bytesColVector.isNull[elementNum] = true;
+      }
+    }
+
+    @Override
+    public void nextVector(ColumnVector previousVector,
+                           boolean[] isNull,
+                           final int batchSize) throws IOException {
+      if (doubleColVector == null) {
+        // Allocate column vector for file; cast column vector for reader.
+        doubleColVector = new DoubleColumnVector();
+        bytesColVector = (BytesColumnVector) previousVector;
+      }
+      // Read present/isNull stream
+      floatTreeReader.nextVector(doubleColVector, isNull, batchSize);
+
+      convertVector(doubleColVector, bytesColVector, batchSize);
+    }
+  }
+
+  public static class StringGroupFromDoubleTreeReader extends ConvertTreeReader {
+
+    private DoubleTreeReader doubleTreeReader;
+
+    private final TypeDescription readerType;
+    private DoubleColumnVector doubleColVector;
+    private BytesColumnVector bytesColVector;
+
+    StringGroupFromDoubleTreeReader(int columnId, TypeDescription readerType,
+        boolean skipCorrupt) throws IOException {
+      super(columnId);
+      this.readerType = readerType;
+      doubleTreeReader = new DoubleTreeReader(columnId);
+      setConvertTreeReader(doubleTreeReader);
+    }
+
+    @Override
+    public void setConvertVectorElement(int elementNum) {
+      double doubleValue = doubleColVector.vector[elementNum];
+      if (!Double.isNaN(doubleValue)) {
+        String string = String.valueOf(doubleValue);
+        byte[] bytes = string.getBytes();
+        assignStringGroupVectorEntry(bytesColVector, elementNum, readerType, bytes);
+      } else {
+        bytesColVector.noNulls = false;
+        bytesColVector.isNull[elementNum] = true;
+      }
+    }
+
+    @Override
+    public void nextVector(ColumnVector previousVector,
+                           boolean[] isNull,
+                           final int batchSize) throws IOException {
+      if (doubleColVector == null) {
+        // Allocate column vector for file; cast column vector for reader.
+        doubleColVector = new DoubleColumnVector();
+        bytesColVector = (BytesColumnVector) previousVector;
+      }
+      // Read present/isNull stream
+      doubleTreeReader.nextVector(doubleColVector, isNull, batchSize);
+
+      convertVector(doubleColVector, bytesColVector, batchSize);
+    }
+  }
+
+
+
+  public static class StringGroupFromDecimalTreeReader extends ConvertTreeReader {
+
+    private DecimalTreeReader decimalTreeReader;
+
+    private int precision;
+    private int scale;
+    private final TypeDescription readerType;
+    private DecimalColumnVector decimalColVector;
+    private BytesColumnVector bytesColVector;
+
+    StringGroupFromDecimalTreeReader(int columnId, TypeDescription fileType,
+        TypeDescription readerType, boolean skipCorrupt) throws IOException {
+      super(columnId);
+      this.precision = fileType.getPrecision();
+      this.scale = fileType.getScale();
+      this.readerType = readerType;
+      decimalTreeReader = new DecimalTreeReader(columnId, precision, scale);
+      setConvertTreeReader(decimalTreeReader);
+    }
+
+    @Override
+    public void setConvertVectorElement(int elementNum) {
+      String string = decimalColVector.vector[elementNum].getHiveDecimal().toString();
+      byte[] bytes = string.getBytes();
+      assignStringGroupVectorEntry(bytesColVector, elementNum, readerType, bytes);
+    }
+
+    @Override
+    public void nextVector(ColumnVector previousVector,
+                           boolean[] isNull,
+                           final int batchSize) throws IOException {
+      if (decimalColVector == null) {
+        // Allocate column vector for file; cast column vector for reader.
+        decimalColVector = new DecimalColumnVector(precision, scale);
+        bytesColVector = (BytesColumnVector) previousVector;
+      }
+      // Read present/isNull stream
+      decimalTreeReader.nextVector(decimalColVector, isNull, batchSize);
+
+      convertVector(decimalColVector, bytesColVector, batchSize);
+    }
+  }
+
+  public static class StringGroupFromTimestampTreeReader extends ConvertTreeReader {
+
+    private TimestampTreeReader timestampTreeReader;
+
+    private final TypeDescription readerType;
+    private TimestampColumnVector timestampColVector;
+    private BytesColumnVector bytesColVector;
+
+    StringGroupFromTimestampTreeReader(int columnId, TypeDescription readerType,
+        boolean skipCorrupt) throws IOException {
+      super(columnId);
+      this.readerType = readerType;
+      timestampTreeReader = new TimestampTreeReader(columnId, skipCorrupt);
+      setConvertTreeReader(timestampTreeReader);
+    }
+
+    @Override
+    public void setConvertVectorElement(int elementNum) throws IOException {
+      String string =
+          timestampColVector.asScratchTimestamp(elementNum).toString();
+      byte[] bytes = string.getBytes();
+      assignStringGroupVectorEntry(bytesColVector, elementNum, readerType, bytes);
+    }
+
+    @Override
+    public void nextVector(ColumnVector previousVector,
+                           boolean[] isNull,
+                           final int batchSize) throws IOException {
+      if (timestampColVector == null) {
+        // Allocate column vector for file; cast column vector for reader.
+        timestampColVector = new TimestampColumnVector();
+        bytesColVector = (BytesColumnVector) previousVector;
+      }
+      // Read present/isNull stream
+      timestampTreeReader.nextVector(timestampColVector, isNull, batchSize);
+
+      convertVector(timestampColVector, bytesColVector, batchSize);
+    }
+  }
+
+  public static class StringGroupFromDateTreeReader extends ConvertTreeReader {
+
+    private DateTreeReader dateTreeReader;
+
+    private final TypeDescription readerType;
+    private LongColumnVector longColVector;
+    private BytesColumnVector bytesColVector;
+    private DateWritable dateWritableResult;
+    private Date date;
+
+    StringGroupFromDateTreeReader(int columnId, TypeDescription readerType,
+        boolean skipCorrupt) throws IOException {
+      super(columnId);
+      this.readerType = readerType;
+      dateTreeReader = new DateTreeReader(columnId);
+      setConvertTreeReader(dateTreeReader);
+      dateWritableResult = new DateWritable();
+      date = new Date(0);
+    }
+
+    @Override
+    public void setConvertVectorElement(int elementNum) throws IOException {
+      date.setTime(DateWritable.daysToMillis((int) longColVector.vector[elementNum]));
+      String string = date.toString();
+      byte[] bytes = string.getBytes();
+      assignStringGroupVectorEntry(bytesColVector, elementNum, readerType, bytes);
+    }
+
+    @Override
+    public void nextVector(ColumnVector previousVector,
+                           boolean[] isNull,
+                           final int batchSize) throws IOException {
+      if (longColVector == null) {
+        // Allocate column vector for file; cast column vector for reader.
+        longColVector = new LongColumnVector();
+        bytesColVector = (BytesColumnVector) previousVector;
+      }
+      // Read present/isNull stream
+      dateTreeReader.nextVector(longColVector, isNull, batchSize);
+
+      convertVector(longColVector, bytesColVector, batchSize);
+    }
+  }
+
+  public static class StringGroupFromStringGroupTreeReader extends ConvertTreeReader {
+
+    private TreeReader stringGroupTreeReader;
+
+    private final TypeDescription fileType;
+    private final TypeDescription readerType;
+
+    StringGroupFromStringGroupTreeReader(int columnId, TypeDescription fileType,
+        TypeDescription readerType) throws IOException {
+      super(columnId);
+      this.fileType = fileType;
+      this.readerType = readerType;
+      stringGroupTreeReader = getStringGroupTreeReader(columnId, fileType);
+      setConvertTreeReader(stringGroupTreeReader);
+    }
+
+    @Override
+    public void nextVector(ColumnVector previousVector,
+                           boolean[] isNull,
+                           final int batchSize) throws IOException {
+      stringGroupTreeReader.nextVector(previousVector, isNull, batchSize);
+
+      BytesColumnVector resultColVector = (BytesColumnVector) previousVector;
+
+      if (resultColVector.isRepeating) {
+        if (resultColVector.noNulls || !resultColVector.isNull[0]) {
+          convertStringGroupVectorElement(resultColVector, 0, readerType);
+        } else {
+          resultColVector.noNulls = false;
+          resultColVector.isNull[0] = true;
+        }
+      } else if (resultColVector.noNulls){
+        for (int i = 0; i < batchSize; i++) {
+          convertStringGroupVectorElement(resultColVector, i, readerType);
+        }
+      } else {
+        for (int i = 0; i < batchSize; i++) {
+          if (!resultColVector.isNull[i]) {
+            convertStringGroupVectorElement(resultColVector, i, readerType);
+          } else {
+            resultColVector.noNulls = false;
+            resultColVector.isNull[i] = true;
+          }
+        }
+      }
+    }
+  }
+
+  public static class StringGroupFromBinaryTreeReader extends ConvertTreeReader {
+
+    private BinaryTreeReader binaryTreeReader;
+
+    private final TypeDescription readerType;
+    private BytesWritable binaryWritableResult;
+    private BytesColumnVector inBytesColVector;
+    private BytesColumnVector outBytesColVector;
+
+    StringGroupFromBinaryTreeReader(int columnId, TypeDescription readerType,
+        boolean skipCorrupt) throws IOException {
+      super(columnId);
+      this.readerType = readerType;
+      binaryTreeReader = new BinaryTreeReader(columnId);
+      setConvertTreeReader(binaryTreeReader);
+      binaryWritableResult = new BytesWritable();
+    }
+
+    @Override
+    public void setConvertVectorElement(int elementNum) throws IOException {
+      byte[] bytes = inBytesColVector.vector[elementNum];
+      int start = inBytesColVector.start[elementNum];
+      int length = inBytesColVector.length[elementNum];
+      byte[] string = new byte[length == 0 ? 0 : 3 * length - 1];
+      for(int p = 0; p < string.length; p += 2) {
+        if (p != 0) {
+          string[p++] = ' ';
+        }
+        int num = 0xff & bytes[start++];
+        int digit = num / 16;
+        string[p] = (byte)((digit) + (digit < 10 ? '0' : 'a' - 10));
+        digit = num % 16;
+        string[p + 1] = (byte)((digit) + (digit < 10 ? '0' : 'a' - 10));
+      }
+      assignStringGroupVectorEntry(outBytesColVector, elementNum, readerType,
+          string, 0, string.length);
+    }
+
+    @Override
+    public void nextVector(ColumnVector previousVector,
+                           boolean[] isNull,
+                           final int batchSize) throws IOException {
+      if (inBytesColVector == null) {
+        // Allocate column vector for file; cast column vector for reader.
+        inBytesColVector = new BytesColumnVector();
+        outBytesColVector = (BytesColumnVector) previousVector;
+      }
+      // Read present/isNull stream
+      binaryTreeReader.nextVector(inBytesColVector, isNull, batchSize);
+
+      convertVector(inBytesColVector, outBytesColVector, batchSize);
+    }
+  }
+
+  public static class TimestampFromAnyIntegerTreeReader extends ConvertTreeReader {
+
+    private AnyIntegerTreeReader anyIntegerAsLongTreeReader;
+
+    private LongColumnVector longColVector;
+    private TimestampColumnVector timestampColVector;
+
+    TimestampFromAnyIntegerTreeReader(int columnId, TypeDescription fileType,
+        boolean skipCorrupt) throws IOException {
+      super(columnId);
+      anyIntegerAsLongTreeReader =
+          new AnyIntegerTreeReader(columnId, fileType, skipCorrupt);
+      setConvertTreeReader(anyIntegerAsLongTreeReader);
+    }
+
+    @Override
+    public void setConvertVectorElement(int elementNum) {
+      long longValue = longColVector.vector[elementNum];
+      // UNDONE: What does the boolean setting need to be?
+      timestampColVector.set(elementNum, new Timestamp(longValue));
+    }
+
+    @Override
+    public void nextVector(ColumnVector previousVector,
+                           boolean[] isNull,
+                           final int batchSize) throws IOException {
+      if (longColVector == null) {
+        // Allocate column vector for file; cast column vector for reader.
+        longColVector = new LongColumnVector();
+        timestampColVector = (TimestampColumnVector) previousVector;
+      }
+      // Read present/isNull stream
+      anyIntegerAsLongTreeReader.nextVector(longColVector, isNull, batchSize);
+
+      convertVector(longColVector, timestampColVector, batchSize);
+    }
+  }
+
+  public static class TimestampFromFloatTreeReader extends ConvertTreeReader {
+
+    private FloatTreeReader floatTreeReader;
+
+    private FloatWritable floatResult;
+    private DoubleColumnVector doubleColVector;
+    private TimestampColumnVector timestampColVector;
+
+    TimestampFromFloatTreeReader(int columnId, TypeDescription fileType,
+        boolean skipCorrupt) throws IOException {
+      super(columnId);
+      floatTreeReader = new FloatTreeReader(columnId);
+      setConvertTreeReader(floatTreeReader);
+      floatResult = new FloatWritable();
+    }
+
+    @Override
+    public void setConvertVectorElement(int elementNum) {
+      float floatValue = (float) doubleColVector.vector[elementNum];
+      timestampColVector.set(elementNum,
+          TimestampUtils.doubleToTimestamp(floatValue));
+    }
+
+    @Override
+    public void nextVector(ColumnVector previousVector,
+                           boolean[] isNull,
+                           final int batchSize) throws IOException {
+      if (doubleColVector == null) {
+        // Allocate column vector for file; cast column vector for reader.
+        doubleColVector = new DoubleColumnVector();
+        timestampColVector = (TimestampColumnVector) previousVector;
+      }
+      // Read present/isNull stream
+      floatTreeReader.nextVector(doubleColVector, isNull, batchSize);
+
+      convertVector(doubleColVector, timestampColVector, batchSize);
+    }
+  }
+
+  public static class TimestampFromDoubleTreeReader extends ConvertTreeReader {
+
+    private DoubleTreeReader doubleTreeReader;
+
+    private DoubleColumnVector doubleColVector;
+    private TimestampColumnVector timestampColVector;
+
+    TimestampFromDoubleTreeReader(int columnId, TypeDescription fileType,
+        boolean skipCorrupt) throws IOException {
+      super(columnId);
+      doubleTreeReader = new DoubleTreeReader(columnId);
+      setConvertTreeReader(doubleTreeReader);
+    }
+
+    @Override
+    public void setConvertVectorElement(int elementNum) {
+      double doubleValue = doubleColVector.vector[elementNum];
+      timestampColVector.set(elementNum,
+          TimestampUtils.doubleToTimestamp(doubleValue));
+    }
+
+    @Override
+    public void nextVector(ColumnVector previousVector,
+                           boolean[] isNull,
+                           final int batchSize) throws IOException {
+      if (doubleColVector == null) {
+        // Allocate column vector for file; cast column vector for reader.
+        doubleColVector = new DoubleColumnVector();
+        timestampColVector = (TimestampColumnVector) previousVector;
+      }
+      // Read present/isNull stream
+      doubleTreeReader.nextVector(doubleColVector, isNull, batchSize);
+
+      convertVector(doubleColVector, timestampColVector, batchSize);
+    }
+  }
+
+  public static class TimestampFromDecimalTreeReader extends ConvertTreeReader {
+
+    private DecimalTreeReader decimalTreeReader;
+
+    private final int precision;
+    private final int scale;
+    private HiveDecimalWritable hiveDecimalResult;
+    private DecimalColumnVector decimalColVector;
+    private TimestampColumnVector timestampColVector;
+
+    TimestampFromDecimalTreeReader(int columnId, TypeDescription fileType,
+        boolean skipCorrupt) throws IOException {
+      super(columnId);
+      this.precision = fileType.getPrecision();
+      this.scale = fileType.getScale();
+      decimalTreeReader = new DecimalTreeReader(columnId, precision, scale);
+      setConvertTreeReader(decimalTreeReader);
+      hiveDecimalResult = new HiveDecimalWritable();
+    }
+
+    @Override
+    public void setConvertVectorElement(int elementNum) {
+      Timestamp timestampValue =
+          TimestampUtils.decimalToTimestamp(
+              decimalColVector.vector[elementNum].getHiveDecimal());
+      timestampColVector.set(elementNum, timestampValue);
+    }
+
+    @Override
+    public void nextVector(ColumnVector previousVector,
+                           boolean[] isNull,
+                           final int batchSize) throws IOException {
+      if (decimalColVector == null) {
+        // Allocate column vector for file; cast column vector for reader.
+        decimalColVector = new DecimalColumnVector(precision, scale);
+        timestampColVector = (TimestampColumnVector) previousVector;
+      }
+      // Read present/isNull stream
+      decimalTreeReader.nextVector(decimalColVector, isNull, batchSize);
+
+      convertVector(decimalColVector, timestampColVector, batchSize);
+    }
+  }
+
+  public static class TimestampFromStringGroupTreeReader extends ConvertTreeReader {
+
+    private TreeReader stringGroupTreeReader;
+
+    private final TypeDescription fileType;
+    private BytesColumnVector bytesColVector;
+    private TimestampColumnVector timestampColVector;
+
+    TimestampFromStringGroupTreeReader(int columnId, TypeDescription fileType)
+        throws IOException {
+      super(columnId);
+      this.fileType = fileType;
+      stringGroupTreeReader = getStringGroupTreeReader(columnId, fileType);
+      setConvertTreeReader(stringGroupTreeReader);
+    }
+
+    @Override
+    public void setConvertVectorElement(int elementNum) throws IOException {
+      String stringValue =
+          stringFromBytesColumnVectorEntry(bytesColVector, elementNum);
+      Timestamp timestampValue = parseTimestampFromString(stringValue);
+      if (timestampValue != null) {
+        timestampColVector.set(elementNum, timestampValue);
+      } else {
+        timestampColVector.noNulls = false;
+        timestampColVector.isNull[elementNum] = true;
+      }
+    }
+
+    @Override
+    public void nextVector(ColumnVector previousVector,
+                           boolean[] isNull,
+                           final int batchSize) throws IOException {
+      if (bytesColVector == null) {
+        // Allocate column vector for file; cast column vector for reader.
+        bytesColVector = new BytesColumnVector();
+        timestampColVector = (TimestampColumnVector) previousVector;
+      }
+      // Read present/isNull stream
+      stringGroupTreeReader.nextVector(bytesColVector, isNull, batchSize);
+
+      convertVector(bytesColVector, timestampColVector, batchSize);
+    }
+  }
+
+  public static class TimestampFromDateTreeReader extends ConvertTreeReader {
+
+    private DateTreeReader dateTreeReader;
+
+    private DateWritable doubleResult;
+    private LongColumnVector longColVector;
+    private TimestampColumnVector timestampColVector;
+
+    TimestampFromDateTreeReader(int columnId, TypeDescription fileType,
+        boolean skipCorrupt) throws IOException {
+      super(columnId);
+      dateTreeReader = new DateTreeReader(columnId);
+      setConvertTreeReader(dateTreeReader);
+      doubleResult = new DateWritable();
+    }
+
+    @Override
+    public void setConvertVectorElement(int elementNum) {
+      long millis =
+          DateWritable.daysToMillis((int) longColVector.vector[elementNum]);
+      timestampColVector.set(elementNum, new Timestamp(millis));
+    }
+
+    @Override
+    public void nextVector(ColumnVector previousVector,
+                           boolean[] isNull,
+                           final int batchSize) throws IOException {
+      if (longColVector == null) {
+        // Allocate column vector for file; cast column vector for reader.
+        longColVector = new LongColumnVector();
+        timestampColVector = (TimestampColumnVector) previousVector;
+      }
+      // Read present/isNull stream
+      dateTreeReader.nextVector(longColVector, isNull, batchSize);
+
+      convertVector(longColVector, timestampColVector, batchSize);
+    }
+  }
+
+  public static class DateFromStringGroupTreeReader extends ConvertTreeReader {
+
+    private TreeReader stringGroupTreeReader;
+
+    private final TypeDescription fileType;
+    private BytesColumnVector bytesColVector;
+    private LongColumnVector longColVector;
+
+    DateFromStringGroupTreeReader(int columnId, TypeDescription fileType)
+        throws IOException {
+      super(columnId);
+      this.fileType = fileType;
+      stringGroupTreeReader = getStringGroupTreeReader(columnId, fileType);
+      setConvertTreeReader(stringGroupTreeReader);
+    }
+
+    @Override
+    public void setConvertVectorElement(int elementNum) throws IOException {
+      String stringValue =
+          stringFromBytesColumnVectorEntry(bytesColVector, elementNum);
+      Date dateValue = parseDateFromString(stringValue);
+      if (dateValue != null) {
+        longColVector.vector[elementNum] = DateWritable.dateToDays(dateValue);
+      } else {
+        longColVector.noNulls = false;
+        longColVector.isNull[elementNum] = true;
+      }
+    }
+
+    @Override
+    public void nextVector(ColumnVector previousVector,
+                           boolean[] isNull,
+                           final int batchSize) throws IOException {
+      if (bytesColVector == null) {
+        // Allocate column vector for file; cast column vector for reader.
+        bytesColVector = new BytesColumnVector();
+        longColVector = (LongColumnVector) previousVector;
+      }
+      // Read present/isNull stream
+      stringGroupTreeReader.nextVector(bytesColVector, isNull, batchSize);
+
+      convertVector(bytesColVector, longColVector, batchSize);
+    }
+  }
+
+  public static class DateFromTimestampTreeReader extends ConvertTreeReader {
+
+    private TimestampTreeReader timestampTreeReader;
+
+    private final TypeDescription readerType;
+    private TimestampColumnVector timestampColVector;
+    private LongColumnVector longColVector;
+
+    DateFromTimestampTreeReader(int columnId, TypeDescription readerType,
+        boolean skipCorrupt) throws IOException {
+      super(columnId);
+      this.readerType = readerType;
+      timestampTreeReader = new TimestampTreeReader(columnId, skipCorrupt);
+      setConvertTreeReader(timestampTreeReader);
+    }
+
+    @Override
+    public void setConvertVectorElement(int elementNum) throws IOException {
+      Date dateValue =
+          DateWritable.timeToDate(TimestampUtils.millisToSeconds(
+              timestampColVector.asScratchTimestamp(elementNum).getTime()));
+      longColVector.vector[elementNum] = DateWritable.dateToDays(dateValue);
+    }
+
+    @Override
+    public void nextVector(ColumnVector previousVector,
+                           boolean[] isNull,
+                           final int batchSize) throws IOException {
+      if (timestampColVector == null) {
+        // Allocate column vector for file; cast column vector for reader.
+        timestampColVector = new TimestampColumnVector();
+        longColVector = (LongColumnVector) previousVector;
+      }
+      // Read present/isNull stream
+      timestampTreeReader.nextVector(timestampColVector, isNull, batchSize);
+
+      convertVector(timestampColVector, longColVector, batchSize);
+    }
+  }
+
+  public static class BinaryFromStringGroupTreeReader extends ConvertTreeReader {
+
+    private TreeReader stringGroupTreeReader;
+
+    private final TypeDescription fileType;
+
+    BinaryFromStringGroupTreeReader(int columnId, TypeDescription fileType)
+        throws IOException {
+      super(columnId);
+      this.fileType = fileType;
+      stringGroupTreeReader = getStringGroupTreeReader(columnId, fileType);
+      setConvertTreeReader(stringGroupTreeReader);
+    }
+
+    @Override
+    public void nextVector(ColumnVector previousVector,
+                           boolean[] isNull,
+                           final int batchSize) throws IOException {
+      super.nextVector(previousVector, isNull, batchSize);
+    }
+  }
+
+  private static TreeReader createAnyIntegerConvertTreeReader(int columnId,
+                                                              TypeDescription fileType,
+                                                              TypeDescription readerType,
+                                                              SchemaEvolution evolution,
+                                                              boolean[] included,
+                                                              boolean skipCorrupt) throws IOException {
+
+    // CONVERT from (BOOLEAN, BYTE, SHORT, INT, LONG) to schema type.
+    //
+    switch (readerType.getCategory()) {
+
+    case BOOLEAN:
+    case BYTE:
+    case SHORT:
+    case INT:
+    case LONG:
+      if (fileType.getCategory() == readerType.getCategory()) {
+        throw new IllegalArgumentException("No conversion of type " +
+            readerType.getCategory() + " to self needed");
+      }
+      return new AnyIntegerFromAnyIntegerTreeReader(columnId, fileType, readerType,
+          skipCorrupt);
+
+    case FLOAT:
+      return new FloatFromAnyIntegerTreeReader(columnId, fileType,
+          skipCorrupt);
+
+    case DOUBLE:
+      return new DoubleFromAnyIntegerTreeReader(columnId, fileType,
+          skipCorrupt);
+
+    case DECIMAL:
+      return new DecimalFromAnyIntegerTreeReader(columnId, fileType, readerType, skipCorrupt);
+
+    case STRING:
+    case CHAR:
+    case VARCHAR:
+      return new StringGroupFromAnyIntegerTreeReader(columnId, fileType, readerType,
+          skipCorrupt);
+
+    case TIMESTAMP:
+      return new TimestampFromAnyIntegerTreeReader(columnId, fileType, skipCorrupt);
+
+    // Not currently supported conversion(s):
+    case BINARY:
+    case DATE:
+
+    case STRUCT:
+    case LIST:
+    case MAP:
+    case UNION:
+    default:
+      throw new IllegalArgumentException("Unsupported type " +
+          readerType.getCategory());
+    }
+  }
+
+  private static TreeReader createFloatConvertTreeReader(int columnId,
+                                                         TypeDescription fileType,
+                                                         TypeDescription readerType,
+                                                         SchemaEvolution evolution,
+                                                         boolean[] included,
+                                                         boolean skipCorrupt) throws IOException {
+
+    // CONVERT from FLOAT to schema type.
+    switch (readerType.getCategory()) {
+
+    case BOOLEAN:
+    case BYTE:
+    case SHORT:
+    case INT:
+    case LONG:
+      return new AnyIntegerFromFloatTreeReader(columnId, readerType);
+
+    case FLOAT:
+      throw new IllegalArgumentException("No conversion of type " +
+        readerType.getCategory() + " to self needed");
+
+    case DOUBLE:
+      return new DoubleFromFloatTreeReader(columnId);
+
+    case DECIMAL:
+      return new DecimalFromFloatTreeReader(columnId, readerType);
+
+    case STRING:
+    case CHAR:
+    case VARCHAR:
+      return new StringGroupFromFloatTreeReader(columnId, readerType, skipCorrupt);
+
+    case TIMESTAMP:
+      return new TimestampFromFloatTreeReader(columnId, readerType, skipCorrupt);
+
+    // Not currently supported conversion(s):
+    case BINARY:
+    case DATE:
+
+    case STRUCT:
+    case LIST:
+    case MAP:
+    case UNION:
+    default:
+      throw new IllegalArgumentException("Unsupported type " +
+          readerType.getCategory());
+    }
+  }
+
+  private static TreeReader createDoubleConvertTreeReader(int columnId,
+                                                          TypeDescription fileType,
+                                                          TypeDescription readerType,
+                                                          SchemaEvolution evolution,
+                                                          boolean[] included,
+                                                          boolean skipCorrupt) throws IOException {
+
+    // CONVERT from DOUBLE to schema type.
+    switch (readerType.getCategory()) {
+
+    case BOOLEAN:
+    case BYTE:
+    case SHORT:
+    case INT:
+    case LONG:
+      return new AnyIntegerFromDoubleTreeReader(columnId, readerType);
+
+    case FLOAT:
+      return new FloatFromDoubleTreeReader(columnId);
+
+    case DOUBLE:
+      throw new IllegalArgumentException("No conversion of type " +
+        readerType.getCategory() + " to self needed");
+
+    case DECIMAL:
+      return new DecimalFromDoubleTreeReader(columnId, readerType);
+
+    case STRING:
+    case CHAR:
+    case VARCHAR:
+      return new StringGroupFromDoubleTreeReader(columnId, readerType, skipCorrupt);
+
+    case TIMESTAMP:
+      return new TimestampFromDoubleTreeReader(columnId, readerType, skipCorrupt);
+
+    // Not currently supported conversion(s):
+    case BINARY:
+    case DATE:
+
+    case STRUCT:
+    case LIST:
+    case MAP:
+    case UNION:
+    default:
+      throw new IllegalArgumentException("Unsupported type " +
+          readerType.getCategory());
+    }
+  }
+
+  private static TreeReader createDecimalConvertTreeReader(int columnId,
+                                                           TypeDescription fileType,
+                                                           TypeDescription readerType,
+                                                           SchemaEvolution evolution,
+                                                           boolean[] included,
+                                                           boolean skipCorrupt) throws IOException {
+
+    // CONVERT from DECIMAL to schema type.
+    switch (readerType.getCategory()) {
+
+    case BOOLEAN:
+    case BYTE:
+    case SHORT:
+    case INT:
+    case LONG:
+      return new AnyIntegerFromDecimalTreeReader(columnId, fileType, readerType);
+
+    case FLOAT:
+      return new FloatFromDecimalTreeReader(columnId, fileType, readerType);
+
+    case DOUBLE:
+      return new DoubleFromDecimalTreeReader(columnId, fileType, readerType);
+
+    case STRING:
+    case CHAR:
+    case VARCHAR:
+      return new StringGroupFromDecimalTreeReader(columnId, fileType, readerType, skipCorrupt);
+
+    case TIMESTAMP:
+      return new TimestampFromDecimalTreeReader(columnId, fileType, skipCorrupt);
+
+    case DECIMAL:
+      // UNDONE: Decimal to Decimal conversion????
+
+    // Not currently supported conversion(s):
+    case BINARY:
+    case DATE:
+
+    case STRUCT:
+    case LIST:
+    case MAP:
+    case UNION:
+    default:
+      throw new IllegalArgumentException("Unsupported type " +
+          readerType.getCategory());
+    }
+  }
+
+  private static TreeReader createStringConvertTreeReader(int columnId,
+                                                          TypeDescription fileType,
+                                                          TypeDescription readerType,
+                                                          SchemaEvolution evolution,
+                                                          boolean[] included,
+                                                          boolean skipCorrupt) throws IOException {
+
+    // CONVERT from STRING to schema type.
+    switch (readerType.getCategory()) {
+
+    case BOOLEAN:
+    case BYTE:
+    case SHORT:
+    case INT:
+    case LONG:
+      return new AnyIntegerFromStringGroupTreeReader(columnId, fileType, readerType);
+
+    case FLOAT:
+      return new FloatFromStringGroupTreeReader(columnId, fileType);
+
+    case DOUBLE:
+      return new DoubleFromStringGroupTreeReader(columnId, fileType);
+
+    case DECIMAL:
+      return new DecimalFromStringGroupTreeReader(columnId, fileType, readerType);
+
+    case CHAR:
+      return new StringGroupFromStringGroupTreeReader(columnId, fileType, readerType);
+
+    case VARCHAR:
+      return new StringGroupFromStringGroupTreeReader(columnId, fileType, readerType);
+
+    case STRING:
+      throw new IllegalArgumentException("No conversion of type " +
+          readerType.getCategory() + " to self needed");
+
+    case BINARY:
+      return new BinaryFromStringGroupTreeReader(columnId, fileType);
+
+    case TIMESTAMP:
+      return new TimestampFromStringGroupTreeReader(columnId, fileType);
+
+    case DATE:
+      return new DateFromStringGroupTreeReader(columnId, fileType);
+
+    // Not currently supported conversion(s):
+
+    case STRUCT:
+    case LIST:
+    case MAP:
+    case UNION:
+    default:
+      throw new IllegalArgumentException("Unsupported type " +
+          readerType.getCategory());
+    }
+  }
+
+  private static TreeReader createCharConvertTreeReader(int columnId,
+                                                        TypeDescription fileType,
+                                                        TypeDescription readerType,
+                                                        SchemaEvolution evolution,
+                                                        boolean[] included,
+                                                        boolean skipCorrupt) throws IOException {
+
+    // CONVERT from CHAR to schema type.
+    switch (readerType.getCategory()) {
+
+    case BOOLEAN:
+    case BYTE:
+    case SHORT:
+    case INT:
+    case LONG:
+      return new AnyIntegerFromStringGroupTreeReader(columnId, fileType, readerType);
+
+    case FLOAT:
+      return new FloatFromStringGroupTreeReader(columnId, fileType);
+
+    case DOUBLE:
+      return new DoubleFromStringGroupTreeReader(columnId, fileType);
+
+    case DECIMAL:
+      return new DecimalFromStringGroupTreeReader(columnId, fileType, readerType);
+
+    case STRING:
+      return new StringGroupFromStringGroupTreeReader(columnId, fileType, readerType);
+
+    case VARCHAR:
+      return new StringGroupFromStringGroupTreeReader(columnId, fileType, readerType);
+
+    case CHAR:
+      throw new IllegalArgumentException("No conversion of type " +
+          readerType.getCategory() + " to self needed");
+
+    case BINARY:
+      return new BinaryFromStringGroupTreeReader(columnId, fileType);
+
+    case TIMESTAMP:
+      return new TimestampFromStringGroupTreeReader(columnId, fileType);
+
+    case DATE:
+      return new DateFromStringGroupTreeReader(columnId, fileType);
+
+    // Not currently supported conversion(s):
+
+    case STRUCT:
+    case LIST:
+    case MAP:
+    case UNION:
+    default:
+      throw new IllegalArgumentException("Unsupported type " +
+          readerType.getCategory());
+    }
+  }
+
+  private static TreeReader createVarcharConvertTreeReader(int columnId,
+                                                           TypeDescription fileType,
+                                                           TypeDescription readerType,
+                                                           SchemaEvolution evolution,
+                                                           boolean[] included,
+                                                           boolean skipCorrupt) throws IOException {
+
+    // CONVERT from VARCHAR to schema type.
+    switch (readerType.getCategory()) {
+
+    case BOOLEAN:
+    case BYTE:
+    case SHORT:
+    case INT:
+    case LONG:
+      return new AnyIntegerFromStringGroupTreeReader(columnId, fileType, readerType);
+
+    case FLOAT:
+      return new FloatFromStringGroupTreeReader(columnId, fileType);
+
+    case DOUBLE:
+      return new DoubleFromStringGroupTreeReader(columnId, fileType);
+
+    case DECIMAL:
+      return new DecimalFromStringGroupTreeReader(columnId, fileType, readerType);
+
+    case STRING:
+      return new StringGroupFromStringGroupTreeReader(columnId, fileType, readerType);
+
+    case CHAR:
+      return new StringGroupFromStringGroupTreeReader(columnId, fileType, readerType);
+
+    case VARCHAR:
+      throw new IllegalArgumentException("No conversion of type " +
+          readerType.getCategory() + " to self needed");
+
+    case BINARY:
+      return new BinaryFromStringGroupTreeReader(columnId, fileType);
+
+    case TIMESTAMP:
+      return new TimestampFromStringGroupTreeReader(columnId, fileType);
+
+    case DATE:
+      return new DateFromStringGroupTreeReader(columnId, fileType);
+
+    // Not currently supported conversion(s):
+
+    case STRUCT:
+    case LIST:
+    case MAP:
+    case UNION:
+    default:
+      throw new IllegalArgumentException("Unsupported type " +
+          readerType.getCategory());
+    }
+  }
+
+  private static TreeReader createTimestampConvertTreeReader(int columnId,
+                                                             TypeDescription fileType,
+                                                             TypeDescription readerType,
+                                                             SchemaEvolution evolution,
+                                                             boolean[] included,
+                                                             boolean skipCorrupt) throws IOException {
+
+    // CONVERT from TIMESTAMP to schema type.
+    switch (readerType.getCategory()) {
+
+    case BOOLEAN:
+    case BYTE:
+    case SHORT:
+    case INT:
+    case LONG:
+      return new AnyInte

<TRUNCATED>

[07/27] hive git commit: HIVE-11417. Move the ReaderImpl and RowReaderImpl to the ORC module, by making shims for the row by row reader. (omalley reviewed by prasanth_j)

Posted by om...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcWideTable.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcWideTable.java b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcWideTable.java
deleted file mode 100644
index da2c681..0000000
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcWideTable.java
+++ /dev/null
@@ -1,64 +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.hive.ql.io.orc;
-
-import static org.junit.Assert.assertEquals;
-
-import java.io.IOException;
-
-import org.junit.Test;
-
-public class TestOrcWideTable {
-
-  @Test
-  public void testBufferSizeFor1Col() throws IOException {
-    assertEquals(128 * 1024, WriterImpl.getEstimatedBufferSize(512 * 1024 * 1024,
-        1, 128*1024));
-  }
-
-  @Test
-  public void testBufferSizeFor50Col() throws IOException {
-    assertEquals(256 * 1024, WriterImpl.getEstimatedBufferSize(256 * 1024 * 1024,
-        50, 256*1024));
-  }
-
-  @Test
-  public void testBufferSizeFor1000Col() throws IOException {
-    assertEquals(32 * 1024, WriterImpl.getEstimatedBufferSize(512 * 1024 * 1024,
-        1000, 128*1024));
-  }
-
-  @Test
-  public void testBufferSizeFor2000Col() throws IOException {
-    assertEquals(16 * 1024, WriterImpl.getEstimatedBufferSize(512 * 1024 * 1024,
-        2000, 256*1024));
-  }
-
-  @Test
-  public void testBufferSizeFor4000Col() throws IOException {
-    assertEquals(8 * 1024, WriterImpl.getEstimatedBufferSize(512 * 1024 * 1024,
-        4000, 256*1024));
-  }
-
-  @Test
-  public void testBufferSizeFor25000Col() throws IOException {
-    assertEquals(4 * 1024, WriterImpl.getEstimatedBufferSize(512 * 1024 * 1024,
-        25000, 256*1024));
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestRLEv2.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestRLEv2.java b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestRLEv2.java
deleted file mode 100644
index 1a3559e..0000000
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestRLEv2.java
+++ /dev/null
@@ -1,297 +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.hive.ql.io.orc;
-
-import static org.junit.Assert.assertEquals;
-
-import java.io.ByteArrayOutputStream;
-import java.io.File;
-import java.io.PrintStream;
-import java.util.Random;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestName;
-
-public class TestRLEv2 {
-  Path workDir = new Path(System.getProperty("test.tmp.dir",
-      "target" + File.separator + "test" + File.separator + "tmp"));
-  Path testFilePath;
-  Configuration conf;
-  FileSystem fs;
-
-  @Rule
-  public TestName testCaseName = new TestName();
-
-  @Before
-  public void openFileSystem () throws Exception {
-    conf = new Configuration();
-    fs = FileSystem.getLocal(conf);
-    testFilePath = new Path(workDir, "TestRLEv2." +
-        testCaseName.getMethodName() + ".orc");
-    fs.delete(testFilePath, false);
-  }
-
-  @Test
-  public void testFixedDeltaZero() throws Exception {
-    ObjectInspector inspector = ObjectInspectorFactory.getReflectionObjectInspector(
-        Integer.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    Writer w = OrcFile.createWriter(testFilePath,
-        OrcFile.writerOptions(conf)
-            .compress(CompressionKind.NONE)
-            .inspector(inspector)
-            .rowIndexStride(0)
-            .encodingStrategy(OrcFile.EncodingStrategy.COMPRESSION)
-            .version(OrcFile.Version.V_0_12)
-    );
-
-    for (int i = 0; i < 5120; ++i) {
-      w.addRow(123);
-    }
-    w.close();
-
-    PrintStream origOut = System.out;
-    ByteArrayOutputStream myOut = new ByteArrayOutputStream();
-    System.setOut(new PrintStream(myOut));
-    FileDump.main(new String[]{testFilePath.toUri().toString()});
-    System.out.flush();
-    String outDump = new String(myOut.toByteArray());
-    // 10 runs of 512 elements. Each run has 2 bytes header, 2 bytes base (base = 123,
-    // zigzag encoded varint) and 1 byte delta (delta = 0). In total, 5 bytes per run.
-    assertEquals(true, outDump.contains("Stream: column 0 section DATA start: 3 length 50"));
-    System.setOut(origOut);
-  }
-
-  @Test
-  public void testFixedDeltaOne() throws Exception {
-    ObjectInspector inspector = ObjectInspectorFactory.getReflectionObjectInspector(
-        Integer.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    Writer w = OrcFile.createWriter(testFilePath,
-        OrcFile.writerOptions(conf)
-            .compress(CompressionKind.NONE)
-            .inspector(inspector)
-            .rowIndexStride(0)
-            .encodingStrategy(OrcFile.EncodingStrategy.COMPRESSION)
-            .version(OrcFile.Version.V_0_12)
-    );
-
-    for (int i = 0; i < 5120; ++i) {
-      w.addRow(i % 512);
-    }
-    w.close();
-
-    PrintStream origOut = System.out;
-    ByteArrayOutputStream myOut = new ByteArrayOutputStream();
-    System.setOut(new PrintStream(myOut));
-    FileDump.main(new String[]{testFilePath.toUri().toString()});
-    System.out.flush();
-    String outDump = new String(myOut.toByteArray());
-    // 10 runs of 512 elements. Each run has 2 bytes header, 1 byte base (base = 0)
-    // and 1 byte delta (delta = 1). In total, 4 bytes per run.
-    assertEquals(true, outDump.contains("Stream: column 0 section DATA start: 3 length 40"));
-    System.setOut(origOut);
-  }
-
-  @Test
-  public void testFixedDeltaOneDescending() throws Exception {
-    ObjectInspector inspector = ObjectInspectorFactory.getReflectionObjectInspector(
-        Integer.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    Writer w = OrcFile.createWriter(testFilePath,
-        OrcFile.writerOptions(conf)
-            .compress(CompressionKind.NONE)
-            .inspector(inspector)
-            .rowIndexStride(0)
-            .encodingStrategy(OrcFile.EncodingStrategy.COMPRESSION)
-            .version(OrcFile.Version.V_0_12)
-    );
-
-    for (int i = 0; i < 5120; ++i) {
-      w.addRow(512 - (i % 512));
-    }
-    w.close();
-
-    PrintStream origOut = System.out;
-    ByteArrayOutputStream myOut = new ByteArrayOutputStream();
-    System.setOut(new PrintStream(myOut));
-    FileDump.main(new String[]{testFilePath.toUri().toString()});
-    System.out.flush();
-    String outDump = new String(myOut.toByteArray());
-    // 10 runs of 512 elements. Each run has 2 bytes header, 2 byte base (base = 512, zigzag + varint)
-    // and 1 byte delta (delta = 1). In total, 5 bytes per run.
-    assertEquals(true, outDump.contains("Stream: column 0 section DATA start: 3 length 50"));
-    System.setOut(origOut);
-  }
-
-  @Test
-  public void testFixedDeltaLarge() throws Exception {
-    ObjectInspector inspector = ObjectInspectorFactory.getReflectionObjectInspector(
-        Integer.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    Writer w = OrcFile.createWriter(testFilePath,
-        OrcFile.writerOptions(conf)
-            .compress(CompressionKind.NONE)
-            .inspector(inspector)
-            .rowIndexStride(0)
-            .encodingStrategy(OrcFile.EncodingStrategy.COMPRESSION)
-            .version(OrcFile.Version.V_0_12)
-    );
-
-    for (int i = 0; i < 5120; ++i) {
-      w.addRow(i % 512 + ((i % 512 ) * 100));
-    }
-    w.close();
-
-    PrintStream origOut = System.out;
-    ByteArrayOutputStream myOut = new ByteArrayOutputStream();
-    System.setOut(new PrintStream(myOut));
-    FileDump.main(new String[]{testFilePath.toUri().toString()});
-    System.out.flush();
-    String outDump = new String(myOut.toByteArray());
-    // 10 runs of 512 elements. Each run has 2 bytes header, 1 byte base (base = 0)
-    // and 2 bytes delta (delta = 100, zigzag encoded varint). In total, 5 bytes per run.
-    assertEquals(true, outDump.contains("Stream: column 0 section DATA start: 3 length 50"));
-    System.setOut(origOut);
-  }
-
-  @Test
-  public void testFixedDeltaLargeDescending() throws Exception {
-    ObjectInspector inspector = ObjectInspectorFactory.getReflectionObjectInspector(
-        Integer.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    Writer w = OrcFile.createWriter(testFilePath,
-        OrcFile.writerOptions(conf)
-            .compress(CompressionKind.NONE)
-            .inspector(inspector)
-            .rowIndexStride(0)
-            .encodingStrategy(OrcFile.EncodingStrategy.COMPRESSION)
-            .version(OrcFile.Version.V_0_12)
-    );
-
-    for (int i = 0; i < 5120; ++i) {
-      w.addRow((512 - i % 512) + ((i % 512 ) * 100));
-    }
-    w.close();
-
-    PrintStream origOut = System.out;
-    ByteArrayOutputStream myOut = new ByteArrayOutputStream();
-    System.setOut(new PrintStream(myOut));
-    FileDump.main(new String[]{testFilePath.toUri().toString()});
-    System.out.flush();
-    String outDump = new String(myOut.toByteArray());
-    // 10 runs of 512 elements. Each run has 2 bytes header, 2 byte base (base = 512, zigzag + varint)
-    // and 2 bytes delta (delta = 100, zigzag encoded varint). In total, 6 bytes per run.
-    assertEquals(true, outDump.contains("Stream: column 0 section DATA start: 3 length 60"));
-    System.setOut(origOut);
-  }
-
-  @Test
-  public void testShortRepeat() throws Exception {
-    ObjectInspector inspector = ObjectInspectorFactory.getReflectionObjectInspector(
-        Integer.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    Writer w = OrcFile.createWriter(testFilePath,
-        OrcFile.writerOptions(conf)
-            .compress(CompressionKind.NONE)
-            .inspector(inspector)
-            .rowIndexStride(0)
-            .encodingStrategy(OrcFile.EncodingStrategy.COMPRESSION)
-            .version(OrcFile.Version.V_0_12)
-    );
-
-    for (int i = 0; i < 5; ++i) {
-      w.addRow(10);
-    }
-    w.close();
-
-    PrintStream origOut = System.out;
-    ByteArrayOutputStream myOut = new ByteArrayOutputStream();
-    System.setOut(new PrintStream(myOut));
-    FileDump.main(new String[]{testFilePath.toUri().toString()});
-    System.out.flush();
-    String outDump = new String(myOut.toByteArray());
-    // 1 byte header + 1 byte value
-    assertEquals(true, outDump.contains("Stream: column 0 section DATA start: 3 length 2"));
-    System.setOut(origOut);
-  }
-
-  @Test
-  public void testDeltaUnknownSign() throws Exception {
-    ObjectInspector inspector = ObjectInspectorFactory.getReflectionObjectInspector(
-        Integer.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    Writer w = OrcFile.createWriter(testFilePath,
-        OrcFile.writerOptions(conf)
-            .compress(CompressionKind.NONE)
-            .inspector(inspector)
-            .rowIndexStride(0)
-            .encodingStrategy(OrcFile.EncodingStrategy.COMPRESSION)
-            .version(OrcFile.Version.V_0_12)
-    );
-
-    w.addRow(0);
-    for (int i = 0; i < 511; ++i) {
-      w.addRow(i);
-    }
-    w.close();
-
-    PrintStream origOut = System.out;
-    ByteArrayOutputStream myOut = new ByteArrayOutputStream();
-    System.setOut(new PrintStream(myOut));
-    FileDump.main(new String[]{testFilePath.toUri().toString()});
-    System.out.flush();
-    String outDump = new String(myOut.toByteArray());
-    // monotonicity will be undetermined for this sequence 0,0,1,2,3,...510. Hence DIRECT encoding
-    // will be used. 2 bytes for header and 640 bytes for data (512 values with fixed bit of 10 bits
-    // each, 5120/8 = 640). Total bytes 642
-    assertEquals(true, outDump.contains("Stream: column 0 section DATA start: 3 length 642"));
-    System.setOut(origOut);
-  }
-
-  @Test
-  public void testPatchedBase() throws Exception {
-    ObjectInspector inspector = ObjectInspectorFactory.getReflectionObjectInspector(
-        Integer.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    Writer w = OrcFile.createWriter(testFilePath,
-        OrcFile.writerOptions(conf)
-            .compress(CompressionKind.NONE)
-            .inspector(inspector)
-            .rowIndexStride(0)
-            .encodingStrategy(OrcFile.EncodingStrategy.COMPRESSION)
-            .version(OrcFile.Version.V_0_12)
-    );
-
-    Random rand = new Random(123);
-    w.addRow(10000000);
-    for (int i = 0; i < 511; ++i) {
-      w.addRow(rand.nextInt(i+1));
-    }
-    w.close();
-
-    PrintStream origOut = System.out;
-    ByteArrayOutputStream myOut = new ByteArrayOutputStream();
-    System.setOut(new PrintStream(myOut));
-    FileDump.main(new String[]{testFilePath.toUri().toString()});
-    System.out.flush();
-    String outDump = new String(myOut.toByteArray());
-    // use PATCHED_BASE encoding
-    assertEquals(true, outDump.contains("Stream: column 0 section DATA start: 3 length 583"));
-    System.setOut(origOut);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestReaderImpl.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestReaderImpl.java b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestReaderImpl.java
deleted file mode 100644
index e0199d6..0000000
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestReaderImpl.java
+++ /dev/null
@@ -1,151 +0,0 @@
-/*
- * Copyright 2016 The Apache Software Foundation.
- *
- * Licensed 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.hive.ql.io.orc;
-
-import java.io.ByteArrayInputStream;
-import java.io.EOFException;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.charset.CharacterCodingException;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.PositionedReadable;
-import org.apache.hadoop.fs.Seekable;
-import org.apache.hadoop.hive.ql.io.FileFormatException;
-import org.apache.hadoop.io.Text;
-import org.junit.Test;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.rules.ExpectedException;
-
-public class TestReaderImpl {
-
-  @Rule
-  public ExpectedException thrown = ExpectedException.none();
-
-  private final Path path = new Path("test-file.orc");
-  private FSDataInputStream in;
-  private int psLen;
-  private ByteBuffer buffer;
-
-  @Before
-  public void setup() {
-    in = null;
-  }
-
-  @Test
-  public void testEnsureOrcFooterSmallTextFile() throws IOException {
-    prepareTestCase("1".getBytes());
-    thrown.expect(FileFormatException.class);
-    ReaderImpl.ensureOrcFooter(in, path, psLen, buffer);
-  }
-
-  @Test
-  public void testEnsureOrcFooterLargeTextFile() throws IOException {
-    prepareTestCase("This is Some Text File".getBytes());
-    thrown.expect(FileFormatException.class);
-    ReaderImpl.ensureOrcFooter(in, path, psLen, buffer);
-  }
-
-  @Test
-  public void testEnsureOrcFooter011ORCFile() throws IOException {
-    prepareTestCase(composeContent(OrcFile.MAGIC, "FOOTER"));
-    ReaderImpl.ensureOrcFooter(in, path, psLen, buffer);
-  }
-
-  @Test
-  public void testEnsureOrcFooterCorrectORCFooter() throws IOException {
-    prepareTestCase(composeContent("",OrcFile.MAGIC));
-    ReaderImpl.ensureOrcFooter(in, path, psLen, buffer);
-  }
-
-  private void prepareTestCase(byte[] bytes) {
-    buffer = ByteBuffer.wrap(bytes);
-    psLen = buffer.get(bytes.length - 1) & 0xff;
-    in = new FSDataInputStream(new SeekableByteArrayInputStream(bytes));
-  }
-
-  private byte[] composeContent(String headerStr, String footerStr) throws CharacterCodingException {
-    ByteBuffer header = Text.encode(headerStr);
-    ByteBuffer footer = Text.encode(footerStr);
-    int headerLen = header.remaining();
-    int footerLen = footer.remaining() + 1;
-
-    ByteBuffer buf = ByteBuffer.allocate(headerLen + footerLen);
-
-    buf.put(header);
-    buf.put(footer);
-    buf.put((byte) footerLen);
-    return buf.array();
-  }
-
-  private static final class SeekableByteArrayInputStream extends ByteArrayInputStream
-          implements Seekable, PositionedReadable {
-
-    public SeekableByteArrayInputStream(byte[] buf) {
-      super(buf);
-    }
-
-    @Override
-    public void seek(long pos) throws IOException {
-      this.reset();
-      this.skip(pos);
-    }
-
-    @Override
-    public long getPos() throws IOException {
-      return pos;
-    }
-
-    @Override
-    public boolean seekToNewSource(long targetPos) throws IOException {
-      return false;
-    }
-
-    @Override
-    public int read(long position, byte[] buffer, int offset, int length)
-            throws IOException {
-      long oldPos = getPos();
-      int nread = -1;
-      try {
-        seek(position);
-        nread = read(buffer, offset, length);
-      } finally {
-        seek(oldPos);
-      }
-      return nread;
-    }
-
-    @Override
-    public void readFully(long position, byte[] buffer, int offset, int length)
-            throws IOException {
-      int nread = 0;
-      while (nread < length) {
-        int nbytes = read(position + nread, buffer, offset + nread, length - nread);
-        if (nbytes < 0) {
-          throw new EOFException("End of file reached before reading fully.");
-        }
-        nread += nbytes;
-      }
-    }
-
-    @Override
-    public void readFully(long position, byte[] buffer)
-            throws IOException {
-      readFully(position, buffer, 0, buffer.length);
-    }
-  }
-}


[05/27] hive git commit: HIVE-11417. Move the ReaderImpl and RowReaderImpl to the ORC module, by making shims for the row by row reader. (omalley reviewed by prasanth_j)

Posted by om...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestStringDictionary.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestStringDictionary.java b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestStringDictionary.java
deleted file mode 100644
index 41a211b..0000000
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestStringDictionary.java
+++ /dev/null
@@ -1,261 +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.hive.ql.io.orc;
-
-import static org.junit.Assert.assertEquals;
-
-import java.io.File;
-import java.util.Random;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
-import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
-import org.apache.hadoop.io.Text;
-import org.apache.orc.CompressionKind;
-import org.apache.orc.OrcProto;
-
-import org.apache.orc.StripeInformation;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestName;
-
-public class TestStringDictionary {
-
-  Path workDir = new Path(System.getProperty("test.tmp.dir", "target" + File.separator + "test"
-      + File.separator + "tmp"));
-
-  Configuration conf;
-  FileSystem fs;
-  Path testFilePath;
-
-  @Rule
-  public TestName testCaseName = new TestName();
-
-  @Before
-  public void openFileSystem() throws Exception {
-    conf = new Configuration();
-    fs = FileSystem.getLocal(conf);
-    testFilePath = new Path(workDir, "TestOrcFile." + testCaseName.getMethodName() + ".orc");
-    fs.delete(testFilePath, false);
-  }
-
-  @Test
-  public void testTooManyDistinct() throws Exception {
-    ObjectInspector inspector;
-    synchronized (TestOrcFile.class) {
-      inspector = ObjectInspectorFactory.getReflectionObjectInspector(Text.class,
-          ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    }
-
-    Writer writer = OrcFile.createWriter(
-        testFilePath,
-        OrcFile.writerOptions(conf).inspector(inspector).compress(CompressionKind.NONE)
-            .bufferSize(10000));
-    for (int i = 0; i < 20000; i++) {
-      writer.addRow(new Text(String.valueOf(i)));
-    }
-    writer.close();
-
-    Reader reader = OrcFile.createReader(testFilePath, OrcFile.readerOptions(conf).filesystem(fs));
-    RecordReader rows = reader.rows();
-    int idx = 0;
-    while (rows.hasNext()) {
-      Object row = rows.next(null);
-      assertEquals(new Text(String.valueOf(idx++)), row);
-    }
-
-    // make sure the encoding type is correct
-    for (StripeInformation stripe : reader.getStripes()) {
-      // hacky but does the job, this casting will work as long this test resides
-      // within the same package as ORC reader
-      OrcProto.StripeFooter footer = ((RecordReaderImpl) rows).readStripeFooter(stripe);
-      for (int i = 0; i < footer.getColumnsCount(); ++i) {
-        OrcProto.ColumnEncoding encoding = footer.getColumns(i);
-        assertEquals(OrcProto.ColumnEncoding.Kind.DIRECT_V2, encoding.getKind());
-      }
-    }
-  }
-
-  @Test
-  public void testHalfDistinct() throws Exception {
-    ObjectInspector inspector;
-    synchronized (TestOrcFile.class) {
-      inspector = ObjectInspectorFactory.getReflectionObjectInspector(Text.class,
-          ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    }
-
-    Writer writer = OrcFile.createWriter(
-        testFilePath,
-        OrcFile.writerOptions(conf).inspector(inspector).compress(CompressionKind.NONE)
-            .bufferSize(10000));
-    Random rand = new Random(123);
-    int[] input = new int[20000];
-    for (int i = 0; i < 20000; i++) {
-      input[i] = rand.nextInt(10000);
-    }
-
-    for (int i = 0; i < 20000; i++) {
-      writer.addRow(new Text(String.valueOf(input[i])));
-    }
-    writer.close();
-
-    Reader reader = OrcFile.createReader(testFilePath, OrcFile.readerOptions(conf).filesystem(fs));
-    RecordReader rows = reader.rows();
-    int idx = 0;
-    while (rows.hasNext()) {
-      Object row = rows.next(null);
-      assertEquals(new Text(String.valueOf(input[idx++])), row);
-    }
-
-    // make sure the encoding type is correct
-    for (StripeInformation stripe : reader.getStripes()) {
-      // hacky but does the job, this casting will work as long this test resides
-      // within the same package as ORC reader
-      OrcProto.StripeFooter footer = ((RecordReaderImpl) rows).readStripeFooter(stripe);
-      for (int i = 0; i < footer.getColumnsCount(); ++i) {
-        OrcProto.ColumnEncoding encoding = footer.getColumns(i);
-        assertEquals(OrcProto.ColumnEncoding.Kind.DICTIONARY_V2, encoding.getKind());
-      }
-    }
-  }
-
-  @Test
-  public void testTooManyDistinctCheckDisabled() throws Exception {
-    ObjectInspector inspector;
-    synchronized (TestOrcFile.class) {
-      inspector = ObjectInspectorFactory.getReflectionObjectInspector(Text.class,
-          ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    }
-
-    conf.setBoolean(ConfVars.HIVE_ORC_ROW_INDEX_STRIDE_DICTIONARY_CHECK.varname, false);
-    Writer writer = OrcFile.createWriter(
-        testFilePath,
-        OrcFile.writerOptions(conf).inspector(inspector).compress(CompressionKind.NONE)
-            .bufferSize(10000));
-    for (int i = 0; i < 20000; i++) {
-      writer.addRow(new Text(String.valueOf(i)));
-    }
-    writer.close();
-
-    Reader reader = OrcFile.createReader(testFilePath, OrcFile.readerOptions(conf).filesystem(fs));
-    RecordReader rows = reader.rows();
-    int idx = 0;
-    while (rows.hasNext()) {
-      Object row = rows.next(null);
-      assertEquals(new Text(String.valueOf(idx++)), row);
-    }
-
-    // make sure the encoding type is correct
-    for (StripeInformation stripe : reader.getStripes()) {
-      // hacky but does the job, this casting will work as long this test resides
-      // within the same package as ORC reader
-      OrcProto.StripeFooter footer = ((RecordReaderImpl) rows).readStripeFooter(stripe);
-      for (int i = 0; i < footer.getColumnsCount(); ++i) {
-        OrcProto.ColumnEncoding encoding = footer.getColumns(i);
-        assertEquals(OrcProto.ColumnEncoding.Kind.DIRECT_V2, encoding.getKind());
-      }
-    }
-  }
-
-  @Test
-  public void testHalfDistinctCheckDisabled() throws Exception {
-    ObjectInspector inspector;
-    synchronized (TestOrcFile.class) {
-      inspector = ObjectInspectorFactory.getReflectionObjectInspector(Text.class,
-          ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    }
-
-    conf.setBoolean(ConfVars.HIVE_ORC_ROW_INDEX_STRIDE_DICTIONARY_CHECK.varname, false);
-    Writer writer = OrcFile.createWriter(
-        testFilePath,
-        OrcFile.writerOptions(conf).inspector(inspector).compress(CompressionKind.NONE)
-            .bufferSize(10000));
-    Random rand = new Random(123);
-    int[] input = new int[20000];
-    for (int i = 0; i < 20000; i++) {
-      input[i] = rand.nextInt(10000);
-    }
-
-    for (int i = 0; i < 20000; i++) {
-      writer.addRow(new Text(String.valueOf(input[i])));
-    }
-    writer.close();
-
-    Reader reader = OrcFile.createReader(testFilePath, OrcFile.readerOptions(conf).filesystem(fs));
-    RecordReader rows = reader.rows();
-    int idx = 0;
-    while (rows.hasNext()) {
-      Object row = rows.next(null);
-      assertEquals(new Text(String.valueOf(input[idx++])), row);
-    }
-
-    // make sure the encoding type is correct
-    for (StripeInformation stripe : reader.getStripes()) {
-      // hacky but does the job, this casting will work as long this test resides
-      // within the same package as ORC reader
-      OrcProto.StripeFooter footer = ((RecordReaderImpl) rows).readStripeFooter(stripe);
-      for (int i = 0; i < footer.getColumnsCount(); ++i) {
-        OrcProto.ColumnEncoding encoding = footer.getColumns(i);
-        assertEquals(OrcProto.ColumnEncoding.Kind.DICTIONARY_V2, encoding.getKind());
-      }
-    }
-  }
-
-  @Test
-  public void testTooManyDistinctV11AlwaysDictionary() throws Exception {
-    ObjectInspector inspector;
-    synchronized (TestOrcFile.class) {
-      inspector = ObjectInspectorFactory.getReflectionObjectInspector(Text.class,
-          ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    }
-
-    Writer writer = OrcFile.createWriter(
-        testFilePath,
-        OrcFile.writerOptions(conf).inspector(inspector).compress(CompressionKind.NONE)
-            .version(OrcFile.Version.V_0_11).bufferSize(10000));
-    for (int i = 0; i < 20000; i++) {
-      writer.addRow(new Text(String.valueOf(i)));
-    }
-    writer.close();
-
-    Reader reader = OrcFile.createReader(testFilePath, OrcFile.readerOptions(conf).filesystem(fs));
-    RecordReader rows = reader.rows();
-    int idx = 0;
-    while (rows.hasNext()) {
-      Object row = rows.next(null);
-      assertEquals(new Text(String.valueOf(idx++)), row);
-    }
-
-    // make sure the encoding type is correct
-    for (StripeInformation stripe : reader.getStripes()) {
-      // hacky but does the job, this casting will work as long this test resides
-      // within the same package as ORC reader
-      OrcProto.StripeFooter footer = ((RecordReaderImpl) rows).readStripeFooter(stripe);
-      for (int i = 0; i < footer.getColumnsCount(); ++i) {
-        OrcProto.ColumnEncoding encoding = footer.getColumns(i);
-        assertEquals(OrcProto.ColumnEncoding.Kind.DICTIONARY, encoding.getKind());
-      }
-    }
-
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestTypeDescription.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestTypeDescription.java b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestTypeDescription.java
deleted file mode 100644
index 96af65a..0000000
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestTypeDescription.java
+++ /dev/null
@@ -1,68 +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.hive.ql.io.orc;
-
-import static org.junit.Assert.assertEquals;
-
-import org.apache.orc.TypeDescription;
-import org.junit.Test;
-
-public class TestTypeDescription {
-
-  @Test
-  public void testJson() {
-    TypeDescription bin = TypeDescription.createBinary();
-    assertEquals("{\"category\": \"binary\", \"id\": 0, \"max\": 0}",
-        bin.toJson());
-    assertEquals("binary", bin.toString());
-    TypeDescription struct = TypeDescription.createStruct()
-        .addField("f1", TypeDescription.createInt())
-        .addField("f2", TypeDescription.createString())
-        .addField("f3", TypeDescription.createDecimal());
-    assertEquals("struct<f1:int,f2:string,f3:decimal(38,10)>",
-        struct.toString());
-    assertEquals("{\"category\": \"struct\", \"id\": 0, \"max\": 3, \"fields\": [\n"
-            + "  \"f1\": {\"category\": \"int\", \"id\": 1, \"max\": 1},\n"
-            + "  \"f2\": {\"category\": \"string\", \"id\": 2, \"max\": 2},\n"
-            + "  \"f3\": {\"category\": \"decimal\", \"id\": 3, \"max\": 3, \"precision\": 38, \"scale\": 10}]}",
-        struct.toJson());
-    struct = TypeDescription.createStruct()
-        .addField("f1", TypeDescription.createUnion()
-            .addUnionChild(TypeDescription.createByte())
-            .addUnionChild(TypeDescription.createDecimal()
-                .withPrecision(20).withScale(10)))
-        .addField("f2", TypeDescription.createStruct()
-            .addField("f3", TypeDescription.createDate())
-            .addField("f4", TypeDescription.createDouble())
-            .addField("f5", TypeDescription.createBoolean()))
-        .addField("f6", TypeDescription.createChar().withMaxLength(100));
-    assertEquals("struct<f1:uniontype<tinyint,decimal(20,10)>,f2:struct<f3:date,f4:double,f5:boolean>,f6:char(100)>",
-        struct.toString());
-    assertEquals(
-        "{\"category\": \"struct\", \"id\": 0, \"max\": 8, \"fields\": [\n" +
-            "  \"f1\": {\"category\": \"uniontype\", \"id\": 1, \"max\": 3, \"children\": [\n" +
-            "    {\"category\": \"tinyint\", \"id\": 2, \"max\": 2},\n" +
-            "    {\"category\": \"decimal\", \"id\": 3, \"max\": 3, \"precision\": 20, \"scale\": 10}]},\n" +
-            "  \"f2\": {\"category\": \"struct\", \"id\": 4, \"max\": 7, \"fields\": [\n" +
-            "    \"f3\": {\"category\": \"date\", \"id\": 5, \"max\": 5},\n" +
-            "    \"f4\": {\"category\": \"double\", \"id\": 6, \"max\": 6},\n" +
-            "    \"f5\": {\"category\": \"boolean\", \"id\": 7, \"max\": 7}]},\n" +
-            "  \"f6\": {\"category\": \"char\", \"id\": 8, \"max\": 8, \"length\": 100}]}",
-        struct.toJson());
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestUnrolledBitPack.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestUnrolledBitPack.java b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestUnrolledBitPack.java
deleted file mode 100644
index 3251731..0000000
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestUnrolledBitPack.java
+++ /dev/null
@@ -1,114 +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.hive.ql.io.orc;
-
-import static org.junit.Assert.assertEquals;
-
-import java.io.File;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.List;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.orc.CompressionKind;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestName;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.junit.runners.Parameterized.Parameters;
-
-import com.google.common.collect.Lists;
-import com.google.common.primitives.Longs;
-
-@RunWith(value = Parameterized.class)
-public class TestUnrolledBitPack {
-
-  private long val;
-
-  public TestUnrolledBitPack(long val) {
-    this.val = val;
-  }
-
-  @Parameters
-  public static Collection<Object[]> data() {
-    Object[][] data = new Object[][] { { -1 }, { 1 }, { 7 }, { -128 }, { 32000 }, { 8300000 },
-        { Integer.MAX_VALUE }, { 540000000000L }, { 140000000000000L }, { 36000000000000000L },
-        { Long.MAX_VALUE } };
-    return Arrays.asList(data);
-  }
-
-  Path workDir = new Path(System.getProperty("test.tmp.dir", "target" + File.separator + "test"
-      + File.separator + "tmp"));
-
-  Configuration conf;
-  FileSystem fs;
-  Path testFilePath;
-
-  @Rule
-  public TestName testCaseName = new TestName();
-
-  @Before
-  public void openFileSystem() throws Exception {
-    conf = new Configuration();
-    fs = FileSystem.getLocal(conf);
-    testFilePath = new Path(workDir, "TestOrcFile." + testCaseName.getMethodName() + ".orc");
-    fs.delete(testFilePath, false);
-  }
-
-  @Test
-  public void testBitPacking() throws Exception {
-    ObjectInspector inspector;
-    synchronized (TestOrcFile.class) {
-      inspector = ObjectInspectorFactory.getReflectionObjectInspector(Long.class,
-          ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    }
-
-    long[] inp = new long[] { val, 0, val, val, 0, val, 0, val, val, 0, val, 0, val, val, 0, 0,
-        val, val, 0, val, 0, 0, val, 0, val, 0, val, 0, 0, val, 0, val, 0, val, 0, 0, val, 0, val,
-        0, val, 0, 0, val, 0, val, 0, val, 0, 0, val, 0, val, 0, val, 0, 0, val, 0, val, 0, val, 0,
-        0, val, 0, val, 0, val, 0, 0, val, 0, val, 0, val, 0, 0, val, 0, val, 0, val, 0, 0, val, 0,
-        val, 0, val, 0, 0, val, 0, val, 0, 0, val, val };
-    List<Long> input = Lists.newArrayList(Longs.asList(inp));
-
-    Writer writer = OrcFile.createWriter(
-        testFilePath,
-        OrcFile.writerOptions(conf).inspector(inspector).stripeSize(100000)
-            .compress(CompressionKind.NONE).bufferSize(10000));
-    for (Long l : input) {
-      writer.addRow(l);
-    }
-    writer.close();
-
-    Reader reader = OrcFile.createReader(testFilePath, OrcFile.readerOptions(conf).filesystem(fs));
-    RecordReader rows = reader.rows();
-    int idx = 0;
-    while (rows.hasNext()) {
-      Object row = rows.next(null);
-      assertEquals(input.get(idx++).longValue(), ((LongWritable) row).get());
-    }
-  }
-
-}


[08/27] hive git commit: HIVE-11417. Move the ReaderImpl and RowReaderImpl to the ORC module, by making shims for the row by row reader. (omalley reviewed by prasanth_j)

Posted by om...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestNewIntegerEncoding.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestNewIntegerEncoding.java b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestNewIntegerEncoding.java
deleted file mode 100644
index f41a7ba..0000000
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestNewIntegerEncoding.java
+++ /dev/null
@@ -1,1342 +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.hive.ql.io.orc;
-
-import static junit.framework.Assert.assertEquals;
-
-import java.io.File;
-import java.sql.Timestamp;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.List;
-import java.util.Random;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hive.serde2.io.TimestampWritable;
-import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
-import org.apache.hadoop.io.IntWritable;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.orc.CompressionKind;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestName;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.junit.runners.Parameterized.Parameters;
-
-import com.google.common.collect.Lists;
-import com.google.common.primitives.Longs;
-
-@RunWith(value = Parameterized.class)
-public class TestNewIntegerEncoding {
-
-  private OrcFile.EncodingStrategy encodingStrategy;
-
-  public TestNewIntegerEncoding( OrcFile.EncodingStrategy es) {
-    this.encodingStrategy = es;
-  }
-
-  @Parameters
-  public static Collection<Object[]> data() {
-    Object[][] data = new Object[][] { {  OrcFile.EncodingStrategy.COMPRESSION },
-        {  OrcFile.EncodingStrategy.SPEED } };
-    return Arrays.asList(data);
-  }
-
-  public static class TSRow {
-    Timestamp ts;
-
-    public TSRow(Timestamp ts) {
-      this.ts = ts;
-    }
-  }
-
-  public static class Row {
-    Integer int1;
-    Long long1;
-
-    public Row(int val, long l) {
-      this.int1 = val;
-      this.long1 = l;
-    }
-  }
-
-  Path workDir = new Path(System.getProperty("test.tmp.dir", "target"
-      + File.separator + "test" + File.separator + "tmp"));
-
-  Configuration conf;
-  FileSystem fs;
-  Path testFilePath;
-
-  @Rule
-  public TestName testCaseName = new TestName();
-
-  @Before
-  public void openFileSystem() throws Exception {
-    conf = new Configuration();
-    fs = FileSystem.getLocal(conf);
-    testFilePath = new Path(workDir, "TestOrcFile."
-        + testCaseName.getMethodName() + ".orc");
-    fs.delete(testFilePath, false);
-  }
-
-  @Test
-  public void testBasicRow() throws Exception {
-    ObjectInspector inspector;
-    synchronized (TestOrcFile.class) {
-      inspector = ObjectInspectorFactory.getReflectionObjectInspector(
-          Row.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    }
-
-    Writer writer = OrcFile.createWriter(testFilePath,
-                                         OrcFile.writerOptions(conf)
-                                         .inspector(inspector)
-                                         .stripeSize(100000)
-                                         .compress(CompressionKind.NONE)
-                                         .bufferSize(10000)
-                                         .encodingStrategy(encodingStrategy));
-    writer.addRow(new Row(111, 1111L));
-    writer.addRow(new Row(111, 1111L));
-    writer.addRow(new Row(111, 1111L));
-    writer.close();
-
-    Reader reader = OrcFile.createReader(testFilePath,
-        OrcFile.readerOptions(conf).filesystem(fs));
-    RecordReader rows = reader.rows();
-    while (rows.hasNext()) {
-      Object row = rows.next(null);
-      assertEquals(new IntWritable(111), ((OrcStruct) row).getFieldValue(0));
-      assertEquals(new LongWritable(1111), ((OrcStruct) row).getFieldValue(1));
-    }
-  }
-
-  @Test
-  public void testBasicOld() throws Exception {
-    ObjectInspector inspector;
-    synchronized (TestOrcFile.class) {
-      inspector = ObjectInspectorFactory.getReflectionObjectInspector(
-          Long.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    }
-
-    long[] inp = new long[] { 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 2, 3, 4, 5, 6,
-        7, 8, 9, 10, 1, 1, 1, 1, 1, 1, 10, 9, 7, 6, 5, 4, 3, 2, 1, 1, 1, 1, 1,
-        2, 5, 1, 3, 7, 1, 9, 2, 6, 3, 7, 1, 9, 2, 6, 3, 7, 1, 9, 2, 6, 3, 7, 1,
-        9, 2, 6, 3, 7, 1, 9, 2, 6, 2000, 2, 1, 1, 1, 1, 1, 3, 7, 1, 9, 2, 6, 1,
-        1, 1, 1, 1 };
-    List<Long> input = Lists.newArrayList(Longs.asList(inp));
-    Writer writer = OrcFile.createWriter(testFilePath,
-                                         OrcFile.writerOptions(conf)
-                                         .inspector(inspector)
-                                         .compress(CompressionKind.NONE)
-                                         .version(OrcFile.Version.V_0_11)
-                                         .bufferSize(10000)
-                                         .encodingStrategy(encodingStrategy));
-    for(Long l : input) {
-      writer.addRow(l);
-    }
-    writer.close();
-
-    Reader reader = OrcFile.createReader(testFilePath,
-        OrcFile.readerOptions(conf).filesystem(fs));
-    RecordReader rows = reader.rows();
-    int idx = 0;
-    while (rows.hasNext()) {
-      Object row = rows.next(null);
-      assertEquals(input.get(idx++).longValue(), ((LongWritable) row).get());
-    }
-  }
-
-  @Test
-  public void testBasicNew() throws Exception {
-    ObjectInspector inspector;
-    synchronized (TestOrcFile.class) {
-      inspector = ObjectInspectorFactory.getReflectionObjectInspector(
-          Long.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    }
-
-    long[] inp = new long[] { 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 2, 3, 4, 5, 6,
-        7, 8, 9, 10, 1, 1, 1, 1, 1, 1, 10, 9, 7, 6, 5, 4, 3, 2, 1, 1, 1, 1, 1,
-        2, 5, 1, 3, 7, 1, 9, 2, 6, 3, 7, 1, 9, 2, 6, 3, 7, 1, 9, 2, 6, 3, 7, 1,
-        9, 2, 6, 3, 7, 1, 9, 2, 6, 2000, 2, 1, 1, 1, 1, 1, 3, 7, 1, 9, 2, 6, 1,
-        1, 1, 1, 1 };
-    List<Long> input = Lists.newArrayList(Longs.asList(inp));
-
-    Writer writer = OrcFile.createWriter(testFilePath,
-                                         OrcFile.writerOptions(conf)
-                                         .inspector(inspector)
-                                         .stripeSize(100000)
-                                         .compress(CompressionKind.NONE)
-                                         .bufferSize(10000)
-                                         .encodingStrategy(encodingStrategy));
-    for(Long l : input) {
-      writer.addRow(l);
-    }
-    writer.close();
-
-    Reader reader = OrcFile.createReader(testFilePath,
-        OrcFile.readerOptions(conf).filesystem(fs));
-    RecordReader rows = reader.rows();
-    int idx = 0;
-    while (rows.hasNext()) {
-      Object row = rows.next(null);
-      assertEquals(input.get(idx++).longValue(), ((LongWritable) row).get());
-    }
-  }
-  
-  @Test
-  public void testBasicDelta1() throws Exception {
-    ObjectInspector inspector;
-    synchronized (TestOrcFile.class) {
-      inspector = ObjectInspectorFactory.getReflectionObjectInspector(
-          Long.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    }
-
-    long[] inp = new long[] { -500, -400, -350, -325, -310 };
-    List<Long> input = Lists.newArrayList(Longs.asList(inp));
-
-    Writer writer = OrcFile.createWriter(testFilePath,
-                                         OrcFile.writerOptions(conf)
-                                         .inspector(inspector)
-                                         .stripeSize(100000)
-                                         .compress(CompressionKind.NONE)
-                                         .bufferSize(10000)
-                                         .encodingStrategy(encodingStrategy));
-    for(Long l : input) {
-      writer.addRow(l);
-    }
-    writer.close();
-
-    Reader reader = OrcFile.createReader(testFilePath,
-        OrcFile.readerOptions(conf).filesystem(fs));
-    RecordReader rows = reader.rows();
-    int idx = 0;
-    while (rows.hasNext()) {
-      Object row = rows.next(null);
-      assertEquals(input.get(idx++).longValue(), ((LongWritable) row).get());
-    }
-  }
-
-  @Test
-  public void testBasicDelta2() throws Exception {
-    ObjectInspector inspector;
-    synchronized (TestOrcFile.class) {
-      inspector = ObjectInspectorFactory.getReflectionObjectInspector(
-          Long.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    }
-
-    long[] inp = new long[] { -500, -600, -650, -675, -710 };
-    List<Long> input = Lists.newArrayList(Longs.asList(inp));
-
-    Writer writer = OrcFile.createWriter(testFilePath,
-                                         OrcFile.writerOptions(conf)
-                                         .inspector(inspector)
-                                         .stripeSize(100000)
-                                         .compress(CompressionKind.NONE)
-                                         .bufferSize(10000)
-                                         .encodingStrategy(encodingStrategy));
-    for(Long l : input) {
-      writer.addRow(l);
-    }
-    writer.close();
-
-    Reader reader = OrcFile.createReader(testFilePath,
-        OrcFile.readerOptions(conf).filesystem(fs));
-    RecordReader rows = reader.rows();
-    int idx = 0;
-    while (rows.hasNext()) {
-      Object row = rows.next(null);
-      assertEquals(input.get(idx++).longValue(), ((LongWritable) row).get());
-    }
-  }
-
-  @Test
-  public void testBasicDelta3() throws Exception {
-    ObjectInspector inspector;
-    synchronized (TestOrcFile.class) {
-      inspector = ObjectInspectorFactory.getReflectionObjectInspector(
-          Long.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    }
-
-    long[] inp = new long[] { 500, 400, 350, 325, 310 };
-    List<Long> input = Lists.newArrayList(Longs.asList(inp));
-
-    Writer writer = OrcFile.createWriter(testFilePath,
-                                         OrcFile.writerOptions(conf)
-                                         .inspector(inspector)
-                                         .stripeSize(100000)
-                                         .compress(CompressionKind.NONE)
-                                         .bufferSize(10000)
-                                         .encodingStrategy(encodingStrategy));
-    for(Long l : input) {
-      writer.addRow(l);
-    }
-    writer.close();
-
-    Reader reader = OrcFile.createReader(testFilePath,
-        OrcFile.readerOptions(conf).filesystem(fs));
-    RecordReader rows = reader.rows();
-    int idx = 0;
-    while (rows.hasNext()) {
-      Object row = rows.next(null);
-      assertEquals(input.get(idx++).longValue(), ((LongWritable) row).get());
-    }
-  }
-
-  @Test
-  public void testBasicDelta4() throws Exception {
-    ObjectInspector inspector;
-    synchronized (TestOrcFile.class) {
-      inspector = ObjectInspectorFactory.getReflectionObjectInspector(
-          Long.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    }
-
-    long[] inp = new long[] { 500, 600, 650, 675, 710 };
-    List<Long> input = Lists.newArrayList(Longs.asList(inp));
-
-    Writer writer = OrcFile.createWriter(testFilePath,
-                                         OrcFile.writerOptions(conf)
-                                         .inspector(inspector)
-                                         .stripeSize(100000)
-                                         .compress(CompressionKind.NONE)
-                                         .bufferSize(10000)
-                                         .encodingStrategy(encodingStrategy));
-    for(Long l : input) {
-      writer.addRow(l);
-    }
-    writer.close();
-
-    Reader reader = OrcFile.createReader(testFilePath,
-        OrcFile.readerOptions(conf).filesystem(fs));
-    RecordReader rows = reader.rows();
-    int idx = 0;
-    while (rows.hasNext()) {
-      Object row = rows.next(null);
-      assertEquals(input.get(idx++).longValue(), ((LongWritable) row).get());
-    }
-  }
-
-  @Test
-  public void testDeltaOverflow() throws Exception {
-    ObjectInspector inspector;
-    synchronized (TestOrcFile.class) {
-      inspector = ObjectInspectorFactory
-          .getReflectionObjectInspector(Long.class,
-              ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    }
-
-    long[] inp = new long[]{4513343538618202719l, 4513343538618202711l,
-        2911390882471569739l,
-        -9181829309989854913l};
-    List<Long> input = Lists.newArrayList(Longs.asList(inp));
-
-    Writer writer = OrcFile.createWriter(
-        testFilePath,
-        OrcFile.writerOptions(conf).inspector(inspector).stripeSize(100000)
-            .compress(CompressionKind.NONE).bufferSize(10000));
-    for (Long l : input) {
-      writer.addRow(l);
-    }
-    writer.close();
-
-    Reader reader = OrcFile
-        .createReader(testFilePath, OrcFile.readerOptions(conf).filesystem(fs));
-    RecordReader rows = reader.rows();
-    int idx = 0;
-    while (rows.hasNext()) {
-      Object row = rows.next(null);
-      assertEquals(input.get(idx++).longValue(), ((LongWritable) row).get());
-    }
-  }
-
-  @Test
-  public void testDeltaOverflow2() throws Exception {
-    ObjectInspector inspector;
-    synchronized (TestOrcFile.class) {
-      inspector = ObjectInspectorFactory
-          .getReflectionObjectInspector(Long.class,
-              ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    }
-
-    long[] inp = new long[]{Long.MAX_VALUE, 4513343538618202711l,
-        2911390882471569739l,
-        Long.MIN_VALUE};
-    List<Long> input = Lists.newArrayList(Longs.asList(inp));
-
-    Writer writer = OrcFile.createWriter(
-        testFilePath,
-        OrcFile.writerOptions(conf).inspector(inspector).stripeSize(100000)
-            .compress(CompressionKind.NONE).bufferSize(10000));
-    for (Long l : input) {
-      writer.addRow(l);
-    }
-    writer.close();
-
-    Reader reader = OrcFile
-        .createReader(testFilePath, OrcFile.readerOptions(conf).filesystem(fs));
-    RecordReader rows = reader.rows();
-    int idx = 0;
-    while (rows.hasNext()) {
-      Object row = rows.next(null);
-      assertEquals(input.get(idx++).longValue(), ((LongWritable) row).get());
-    }
-  }
-
-  @Test
-  public void testDeltaOverflow3() throws Exception {
-    ObjectInspector inspector;
-    synchronized (TestOrcFile.class) {
-      inspector = ObjectInspectorFactory
-          .getReflectionObjectInspector(Long.class,
-              ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    }
-
-    long[] inp = new long[]{-4513343538618202711l, -2911390882471569739l, -2,
-        Long.MAX_VALUE};
-    List<Long> input = Lists.newArrayList(Longs.asList(inp));
-
-    Writer writer = OrcFile.createWriter(
-        testFilePath,
-        OrcFile.writerOptions(conf).inspector(inspector).stripeSize(100000)
-            .compress(CompressionKind.NONE).bufferSize(10000));
-    for (Long l : input) {
-      writer.addRow(l);
-    }
-    writer.close();
-
-    Reader reader = OrcFile
-        .createReader(testFilePath, OrcFile.readerOptions(conf).filesystem(fs));
-    RecordReader rows = reader.rows();
-    int idx = 0;
-    while (rows.hasNext()) {
-      Object row = rows.next(null);
-      assertEquals(input.get(idx++).longValue(), ((LongWritable) row).get());
-    }
-  }
-
-  @Test
-  public void testIntegerMin() throws Exception {
-    ObjectInspector inspector;
-    synchronized (TestOrcFile.class) {
-      inspector = ObjectInspectorFactory.getReflectionObjectInspector(
-          Long.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    }
-
-    List<Long> input = Lists.newArrayList();
-    input.add((long) Integer.MIN_VALUE);
-
-    Writer writer = OrcFile.createWriter(testFilePath,
-                                         OrcFile.writerOptions(conf)
-                                         .inspector(inspector)
-                                         .stripeSize(100000)
-                                         .bufferSize(10000)
-                                         .encodingStrategy(encodingStrategy));
-    for(Long l : input) {
-      writer.addRow(l);
-    }
-    writer.close();
-
-    Reader reader = OrcFile.createReader(testFilePath,
-        OrcFile.readerOptions(conf).filesystem(fs));
-    RecordReader rows = reader.rows();
-    int idx = 0;
-    while (rows.hasNext()) {
-      Object row = rows.next(null);
-      assertEquals(input.get(idx++).longValue(), ((LongWritable) row).get());
-    }
-  }
-
-  @Test
-  public void testIntegerMax() throws Exception {
-    ObjectInspector inspector;
-    synchronized (TestOrcFile.class) {
-      inspector = ObjectInspectorFactory.getReflectionObjectInspector(
-          Long.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    }
-
-    List<Long> input = Lists.newArrayList();
-    input.add((long) Integer.MAX_VALUE);
-
-    Writer writer = OrcFile.createWriter(testFilePath,
-                                         OrcFile.writerOptions(conf)
-                                         .inspector(inspector)
-                                         .stripeSize(100000)
-                                         .compress(CompressionKind.NONE)
-                                         .bufferSize(10000)
-                                         .encodingStrategy(encodingStrategy));
-    for(Long l : input) {
-      writer.addRow(l);
-    }
-    writer.close();
-
-    Reader reader = OrcFile.createReader(testFilePath,
-        OrcFile.readerOptions(conf).filesystem(fs));
-    RecordReader rows = reader.rows();
-    int idx = 0;
-    while (rows.hasNext()) {
-      Object row = rows.next(null);
-      assertEquals(input.get(idx++).longValue(), ((LongWritable) row).get());
-    }
-  }
-
-  @Test
-  public void testLongMin() throws Exception {
-    ObjectInspector inspector;
-    synchronized (TestOrcFile.class) {
-      inspector = ObjectInspectorFactory.getReflectionObjectInspector(
-          Long.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    }
-
-    List<Long> input = Lists.newArrayList();
-    input.add(Long.MIN_VALUE);
-
-    Writer writer = OrcFile.createWriter(testFilePath,
-                                         OrcFile.writerOptions(conf)
-                                         .inspector(inspector)
-                                         .stripeSize(100000)
-                                         .compress(CompressionKind.NONE)
-                                         .bufferSize(10000)
-                                         .encodingStrategy(encodingStrategy));
-    for(Long l : input) {
-      writer.addRow(l);
-    }
-    writer.close();
-
-    Reader reader = OrcFile.createReader(testFilePath,
-        OrcFile.readerOptions(conf).filesystem(fs));
-    RecordReader rows = reader.rows();
-    int idx = 0;
-    while (rows.hasNext()) {
-      Object row = rows.next(null);
-      assertEquals(input.get(idx++).longValue(), ((LongWritable) row).get());
-    }
-  }
-
-  @Test
-  public void testLongMax() throws Exception {
-    ObjectInspector inspector;
-    synchronized (TestOrcFile.class) {
-      inspector = ObjectInspectorFactory.getReflectionObjectInspector(
-          Long.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    }
-
-    List<Long> input = Lists.newArrayList();
-    input.add(Long.MAX_VALUE);
-
-    Writer writer = OrcFile.createWriter(testFilePath,
-                                         OrcFile.writerOptions(conf)
-                                         .inspector(inspector)
-                                         .stripeSize(100000)
-                                         .compress(CompressionKind.NONE)
-                                         .bufferSize(10000)
-                                         .encodingStrategy(encodingStrategy));
-    for(Long l : input) {
-      writer.addRow(l);
-    }
-    writer.close();
-
-    Reader reader = OrcFile.createReader(testFilePath,
-        OrcFile.readerOptions(conf).filesystem(fs));
-    RecordReader rows = reader.rows();
-    int idx = 0;
-    while (rows.hasNext()) {
-      Object row = rows.next(null);
-      assertEquals(input.get(idx++).longValue(), ((LongWritable) row).get());
-    }
-  }
-
-  @Test
-  public void testRandomInt() throws Exception {
-    ObjectInspector inspector;
-    synchronized (TestOrcFile.class) {
-      inspector = ObjectInspectorFactory.getReflectionObjectInspector(
-          Long.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    }
-
-    List<Long> input = Lists.newArrayList();
-    Random rand = new Random();
-    for(int i = 0; i < 100000; i++) {
-      input.add((long) rand.nextInt());
-    }
-
-    Writer writer = OrcFile.createWriter(testFilePath,
-                                         OrcFile.writerOptions(conf)
-                                         .inspector(inspector)
-                                         .stripeSize(100000)
-                                         .compress(CompressionKind.NONE)
-                                         .bufferSize(10000)
-                                         .encodingStrategy(encodingStrategy));
-    for(Long l : input) {
-      writer.addRow(l);
-    }
-    writer.close();
-
-    Reader reader = OrcFile.createReader(testFilePath,
-        OrcFile.readerOptions(conf).filesystem(fs));
-    RecordReader rows = reader.rows();
-    int idx = 0;
-    while (rows.hasNext()) {
-      Object row = rows.next(null);
-      assertEquals(input.get(idx++).longValue(), ((LongWritable) row).get());
-    }
-  }
-
-  @Test
-  public void testRandomLong() throws Exception {
-    ObjectInspector inspector;
-    synchronized (TestOrcFile.class) {
-      inspector = ObjectInspectorFactory.getReflectionObjectInspector(
-          Long.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    }
-
-    List<Long> input = Lists.newArrayList();
-    Random rand = new Random();
-    for(int i = 0; i < 100000; i++) {
-      input.add(rand.nextLong());
-    }
-
-    Writer writer = OrcFile.createWriter(testFilePath,
-                                         OrcFile.writerOptions(conf)
-                                         .inspector(inspector)
-                                         .stripeSize(100000)
-                                         .compress(CompressionKind.NONE)
-                                         .bufferSize(10000)
-                                         .encodingStrategy(encodingStrategy));
-    for(Long l : input) {
-      writer.addRow(l);
-    }
-    writer.close();
-
-    Reader reader = OrcFile.createReader(testFilePath,
-        OrcFile.readerOptions(conf).filesystem(fs));
-    RecordReader rows = reader.rows();
-    int idx = 0;
-    while (rows.hasNext()) {
-      Object row = rows.next(null);
-      assertEquals(input.get(idx++).longValue(), ((LongWritable) row).get());
-    }
-  }
-
-  @Test
-  public void testPatchedBaseNegativeMin() throws Exception {
-    ObjectInspector inspector;
-    synchronized (TestOrcFile.class) {
-      inspector = ObjectInspectorFactory.getReflectionObjectInspector(
-          Long.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    }
-
-    long[] inp = new long[] { 20, 2, 3, 2, 1, 3, 17, 71, 35, 2, 1, 139, 2, 2,
-        3, 1783, 475, 2, 1, 1, 3, 1, 3, 2, 32, 1, 2, 3, 1, 8, 30, 1, 3, 414, 1,
-        1, 135, 3, 3, 1, 414, 2, 1, 2, 2, 594, 2, 5, 6, 4, 11, 1, 2, 2, 1, 1,
-        52, 4, 1, 2, 7, 1, 17, 334, 1, 2, 1, 2, 2, 6, 1, 266, 1, 2, 217, 2, 6,
-        2, 13, 2, 2, 1, 2, 3, 5, 1, 2, 1, 7244, 11813, 1, 33, 2, -13, 1, 2, 3,
-        13, 1, 92, 3, 13, 5, 14, 9, 141, 12, 6, 15, 25, 1, 1, 1, 46, 2, 1, 1,
-        141, 3, 1, 1, 1, 1, 2, 1, 4, 34, 5, 78, 8, 1, 2, 2, 1, 9, 10, 2, 1, 4,
-        13, 1, 5, 4, 4, 19, 5, 1, 1, 1, 68, 33, 399, 1, 1885, 25, 5, 2, 4, 1,
-        1, 2, 16, 1, 2966, 3, 1, 1, 25501, 1, 1, 1, 66, 1, 3, 8, 131, 14, 5, 1,
-        2, 2, 1, 1, 8, 1, 1, 2, 1, 5, 9, 2, 3, 112, 13, 2, 2, 1, 5, 10, 3, 1,
-        1, 13, 2, 3, 4, 1, 3, 1, 1, 2, 1, 1, 2, 4, 2, 207, 1, 1, 2, 4, 3, 3, 2,
-        2, 16 };
-    List<Long> input = Lists.newArrayList(Longs.asList(inp));
-
-    Writer writer = OrcFile.createWriter(testFilePath,
-                                         OrcFile.writerOptions(conf)
-                                         .inspector(inspector)
-                                         .stripeSize(100000)
-                                         .compress(CompressionKind.NONE)
-                                         .bufferSize(10000)
-                                         .encodingStrategy(encodingStrategy));
-    for(Long l : input) {
-      writer.addRow(l);
-    }
-    writer.close();
-
-    Reader reader = OrcFile.createReader(testFilePath,
-        OrcFile.readerOptions(conf).filesystem(fs));
-    RecordReader rows = reader.rows();
-    int idx = 0;
-    while (rows.hasNext()) {
-      Object row = rows.next(null);
-      assertEquals(input.get(idx++).longValue(), ((LongWritable) row).get());
-    }
-  }
-
-  @Test
-  public void testPatchedBaseNegativeMin2() throws Exception {
-    ObjectInspector inspector;
-    synchronized (TestOrcFile.class) {
-      inspector = ObjectInspectorFactory.getReflectionObjectInspector(
-          Long.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    }
-
-    long[] inp = new long[] { 20, 2, 3, 2, 1, 3, 17, 71, 35, 2, 1, 139, 2, 2,
-        3, 1783, 475, 2, 1, 1, 3, 1, 3, 2, 32, 1, 2, 3, 1, 8, 30, 1, 3, 414, 1,
-        1, 135, 3, 3, 1, 414, 2, 1, 2, 2, 594, 2, 5, 6, 4, 11, 1, 2, 2, 1, 1,
-        52, 4, 1, 2, 7, 1, 17, 334, 1, 2, 1, 2, 2, 6, 1, 266, 1, 2, 217, 2, 6,
-        2, 13, 2, 2, 1, 2, 3, 5, 1, 2, 1, 7244, 11813, 1, 33, 2, -1, 1, 2, 3,
-        13, 1, 92, 3, 13, 5, 14, 9, 141, 12, 6, 15, 25, 1, 1, 1, 46, 2, 1, 1,
-        141, 3, 1, 1, 1, 1, 2, 1, 4, 34, 5, 78, 8, 1, 2, 2, 1, 9, 10, 2, 1, 4,
-        13, 1, 5, 4, 4, 19, 5, 1, 1, 1, 68, 33, 399, 1, 1885, 25, 5, 2, 4, 1,
-        1, 2, 16, 1, 2966, 3, 1, 1, 25501, 1, 1, 1, 66, 1, 3, 8, 131, 14, 5, 1,
-        2, 2, 1, 1, 8, 1, 1, 2, 1, 5, 9, 2, 3, 112, 13, 2, 2, 1, 5, 10, 3, 1,
-        1, 13, 2, 3, 4, 1, 3, 1, 1, 2, 1, 1, 2, 4, 2, 207, 1, 1, 2, 4, 3, 3, 2,
-        2, 16 };
-    List<Long> input = Lists.newArrayList(Longs.asList(inp));
-
-    Writer writer = OrcFile.createWriter(testFilePath,
-                                         OrcFile.writerOptions(conf)
-                                         .inspector(inspector)
-                                         .stripeSize(100000)
-                                         .compress(CompressionKind.NONE)
-                                         .bufferSize(10000)
-                                         .encodingStrategy(encodingStrategy));
-    for(Long l : input) {
-      writer.addRow(l);
-    }
-    writer.close();
-
-    Reader reader = OrcFile.createReader(testFilePath,
-        OrcFile.readerOptions(conf).filesystem(fs));
-    RecordReader rows = reader.rows();
-    int idx = 0;
-    while (rows.hasNext()) {
-      Object row = rows.next(null);
-      assertEquals(input.get(idx++).longValue(), ((LongWritable) row).get());
-    }
-  }
-
-  @Test
-  public void testPatchedBaseNegativeMin3() throws Exception {
-    ObjectInspector inspector;
-    synchronized (TestOrcFile.class) {
-      inspector = ObjectInspectorFactory.getReflectionObjectInspector(
-          Long.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    }
-
-    long[] inp = new long[] { 20, 2, 3, 2, 1, 3, 17, 71, 35, 2, 1, 139, 2, 2,
-        3, 1783, 475, 2, 1, 1, 3, 1, 3, 2, 32, 1, 2, 3, 1, 8, 30, 1, 3, 414, 1,
-        1, 135, 3, 3, 1, 414, 2, 1, 2, 2, 594, 2, 5, 6, 4, 11, 1, 2, 2, 1, 1,
-        52, 4, 1, 2, 7, 1, 17, 334, 1, 2, 1, 2, 2, 6, 1, 266, 1, 2, 217, 2, 6,
-        2, 13, 2, 2, 1, 2, 3, 5, 1, 2, 1, 7244, 11813, 1, 33, 2, 0, 1, 2, 3,
-        13, 1, 92, 3, 13, 5, 14, 9, 141, 12, 6, 15, 25, 1, 1, 1, 46, 2, 1, 1,
-        141, 3, 1, 1, 1, 1, 2, 1, 4, 34, 5, 78, 8, 1, 2, 2, 1, 9, 10, 2, 1, 4,
-        13, 1, 5, 4, 4, 19, 5, 1, 1, 1, 68, 33, 399, 1, 1885, 25, 5, 2, 4, 1,
-        1, 2, 16, 1, 2966, 3, 1, 1, 25501, 1, 1, 1, 66, 1, 3, 8, 131, 14, 5, 1,
-        2, 2, 1, 1, 8, 1, 1, 2, 1, 5, 9, 2, 3, 112, 13, 2, 2, 1, 5, 10, 3, 1,
-        1, 13, 2, 3, 4, 1, 3, 1, 1, 2, 1, 1, 2, 4, 2, 207, 1, 1, 2, 4, 3, 3, 2,
-        2, 16 };
-    List<Long> input = Lists.newArrayList(Longs.asList(inp));
-
-    Writer writer = OrcFile.createWriter(testFilePath,
-                                         OrcFile.writerOptions(conf)
-                                         .inspector(inspector)
-                                         .stripeSize(100000)
-                                         .compress(CompressionKind.NONE)
-                                         .bufferSize(10000)
-                                         .encodingStrategy(encodingStrategy));
-    for(Long l : input) {
-      writer.addRow(l);
-    }
-    writer.close();
-
-    Reader reader = OrcFile.createReader(testFilePath,
-        OrcFile.readerOptions(conf).filesystem(fs));
-    RecordReader rows = reader.rows();
-    int idx = 0;
-    while (rows.hasNext()) {
-      Object row = rows.next(null);
-      assertEquals(input.get(idx++).longValue(), ((LongWritable) row).get());
-    }
-  }
-
-  @Test
-  public void testPatchedBaseNegativeMin4() throws Exception {
-    ObjectInspector inspector;
-    synchronized (TestOrcFile.class) {
-      inspector = ObjectInspectorFactory.getReflectionObjectInspector(
-          Long.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    }
-
-    long[] inp = new long[] { 13, 13, 11, 8, 13, 10, 10, 11, 11, 14, 11, 7, 13,
-        12, 12, 11, 15, 12, 12, 9, 8, 10, 13, 11, 8, 6, 5, 6, 11, 7, 15, 10, 7,
-        6, 8, 7, 9, 9, 11, 33, 11, 3, 7, 4, 6, 10, 14, 12, 5, 14, 7, 6 };
-    List<Long> input = Lists.newArrayList(Longs.asList(inp));
-
-    Writer writer = OrcFile.createWriter(testFilePath,
-                                         OrcFile.writerOptions(conf)
-                                         .inspector(inspector)
-                                         .stripeSize(100000)
-                                         .compress(CompressionKind.NONE)
-                                         .bufferSize(10000)
-                                         .encodingStrategy(encodingStrategy));
-    for(Long l : input) {
-      writer.addRow(l);
-    }
-    writer.close();
-
-    Reader reader = OrcFile.createReader(testFilePath,
-        OrcFile.readerOptions(conf).filesystem(fs));
-    RecordReader rows = reader.rows();
-    int idx = 0;
-    while (rows.hasNext()) {
-      Object row = rows.next(null);
-      assertEquals(input.get(idx++).longValue(), ((LongWritable) row).get());
-    }
-  }
-
-  @Test
-  public void testPatchedBaseAt0() throws Exception {
-    ObjectInspector inspector;
-    synchronized (TestOrcFile.class) {
-      inspector = ObjectInspectorFactory.getReflectionObjectInspector(
-          Long.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    }
-
-    List<Long> input = Lists.newArrayList();
-    Random rand = new Random();
-    for(int i = 0; i < 5120; i++) {
-      input.add((long) rand.nextInt(100));
-    }
-    input.set(0, 20000L);
-
-    Writer writer = OrcFile.createWriter(testFilePath,
-                                         OrcFile.writerOptions(conf)
-                                         .inspector(inspector)
-                                         .stripeSize(100000)
-                                         .compress(CompressionKind.NONE)
-                                         .bufferSize(10000)
-                                         .encodingStrategy(encodingStrategy));
-    for(Long l : input) {
-      writer.addRow(l);
-    }
-    writer.close();
-
-    Reader reader = OrcFile.createReader(testFilePath,
-        OrcFile.readerOptions(conf).filesystem(fs));
-    RecordReader rows = reader.rows();
-    int idx = 0;
-    while (rows.hasNext()) {
-      Object row = rows.next(null);
-      assertEquals(input.get(idx++).longValue(), ((LongWritable) row).get());
-    }
-  }
-
-  @Test
-  public void testPatchedBaseAt1() throws Exception {
-    ObjectInspector inspector;
-    synchronized (TestOrcFile.class) {
-      inspector = ObjectInspectorFactory.getReflectionObjectInspector(
-          Long.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    }
-
-    List<Long> input = Lists.newArrayList();
-    Random rand = new Random();
-    for(int i = 0; i < 5120; i++) {
-      input.add((long) rand.nextInt(100));
-    }
-    input.set(1, 20000L);
-
-    Writer writer = OrcFile.createWriter(testFilePath,
-                                         OrcFile.writerOptions(conf)
-                                         .inspector(inspector)
-                                         .stripeSize(100000)
-                                         .compress(CompressionKind.NONE)
-                                         .bufferSize(10000)
-                                         .encodingStrategy(encodingStrategy));
-    for(Long l : input) {
-      writer.addRow(l);
-    }
-    writer.close();
-
-    Reader reader = OrcFile.createReader(testFilePath,
-        OrcFile.readerOptions(conf).filesystem(fs));
-    RecordReader rows = reader.rows();
-    int idx = 0;
-    while (rows.hasNext()) {
-      Object row = rows.next(null);
-      assertEquals(input.get(idx++).longValue(), ((LongWritable) row).get());
-    }
-  }
-
-  @Test
-  public void testPatchedBaseAt255() throws Exception {
-    ObjectInspector inspector;
-    synchronized (TestOrcFile.class) {
-      inspector = ObjectInspectorFactory.getReflectionObjectInspector(
-          Long.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    }
-
-    List<Long> input = Lists.newArrayList();
-    Random rand = new Random();
-    for(int i = 0; i < 5120; i++) {
-      input.add((long) rand.nextInt(100));
-    }
-    input.set(255, 20000L);
-
-    Writer writer = OrcFile.createWriter(testFilePath,
-                                         OrcFile.writerOptions(conf)
-                                         .inspector(inspector)
-                                         .stripeSize(100000)
-                                         .bufferSize(10000)
-                                         .encodingStrategy(encodingStrategy));
-    for(Long l : input) {
-      writer.addRow(l);
-    }
-    writer.close();
-
-    Reader reader = OrcFile.createReader(testFilePath,
-        OrcFile.readerOptions(conf).filesystem(fs));
-    RecordReader rows = reader.rows();
-    int idx = 0;
-    while (rows.hasNext()) {
-      Object row = rows.next(null);
-      assertEquals(input.get(idx++).longValue(), ((LongWritable) row).get());
-    }
-  }
-
-  @Test
-  public void testPatchedBaseAt256() throws Exception {
-    ObjectInspector inspector;
-    synchronized (TestOrcFile.class) {
-      inspector = ObjectInspectorFactory.getReflectionObjectInspector(
-          Long.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    }
-
-    List<Long> input = Lists.newArrayList();
-    Random rand = new Random();
-    for(int i = 0; i < 5120; i++) {
-      input.add((long) rand.nextInt(100));
-    }
-    input.set(256, 20000L);
-
-    Writer writer = OrcFile.createWriter(testFilePath,
-                                         OrcFile.writerOptions(conf)
-                                         .inspector(inspector)
-                                         .stripeSize(100000)
-                                         .bufferSize(10000)
-                                         .encodingStrategy(encodingStrategy));
-    for(Long l : input) {
-      writer.addRow(l);
-    }
-    writer.close();
-
-    Reader reader = OrcFile.createReader(testFilePath,
-        OrcFile.readerOptions(conf).filesystem(fs));
-    RecordReader rows = reader.rows();
-    int idx = 0;
-    while (rows.hasNext()) {
-      Object row = rows.next(null);
-      assertEquals(input.get(idx++).longValue(), ((LongWritable) row).get());
-    }
-  }
-
-  @Test
-  public void testPatchedBase510() throws Exception {
-    ObjectInspector inspector;
-    synchronized (TestOrcFile.class) {
-      inspector = ObjectInspectorFactory.getReflectionObjectInspector(
-          Long.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    }
-
-    List<Long> input = Lists.newArrayList();
-    Random rand = new Random();
-    for(int i = 0; i < 5120; i++) {
-      input.add((long) rand.nextInt(100));
-    }
-    input.set(510, 20000L);
-
-    Writer writer = OrcFile.createWriter(testFilePath,
-                                         OrcFile.writerOptions(conf)
-                                         .inspector(inspector)
-                                         .stripeSize(100000)
-                                         .bufferSize(10000)
-                                         .encodingStrategy(encodingStrategy));
-    for(Long l : input) {
-      writer.addRow(l);
-    }
-    writer.close();
-
-    Reader reader = OrcFile.createReader(testFilePath,
-        OrcFile.readerOptions(conf).filesystem(fs));
-    RecordReader rows = reader.rows();
-    int idx = 0;
-    while (rows.hasNext()) {
-      Object row = rows.next(null);
-      assertEquals(input.get(idx++).longValue(), ((LongWritable) row).get());
-    }
-  }
-
-  @Test
-  public void testPatchedBase511() throws Exception {
-    ObjectInspector inspector;
-    synchronized (TestOrcFile.class) {
-      inspector = ObjectInspectorFactory.getReflectionObjectInspector(
-          Long.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    }
-
-    List<Long> input = Lists.newArrayList();
-    Random rand = new Random();
-    for(int i = 0; i < 5120; i++) {
-      input.add((long) rand.nextInt(100));
-    }
-    input.set(511, 20000L);
-
-    Writer writer = OrcFile.createWriter(testFilePath,
-                                         OrcFile.writerOptions(conf)
-                                         .inspector(inspector)
-                                         .stripeSize(100000)
-                                         .bufferSize(10000)
-                                         .encodingStrategy(encodingStrategy));
-    for(Long l : input) {
-      writer.addRow(l);
-    }
-    writer.close();
-
-    Reader reader = OrcFile.createReader(testFilePath,
-        OrcFile.readerOptions(conf).filesystem(fs));
-    RecordReader rows = reader.rows();
-    int idx = 0;
-    while (rows.hasNext()) {
-      Object row = rows.next(null);
-      assertEquals(input.get(idx++).longValue(), ((LongWritable) row).get());
-    }
-  }
-
-  @Test
-  public void testPatchedBaseMax1() throws Exception {
-    ObjectInspector inspector;
-    synchronized (TestOrcFile.class) {
-      inspector = ObjectInspectorFactory.getReflectionObjectInspector(Long.class,
-          ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    }
-
-    List<Long> input = Lists.newArrayList();
-    Random rand = new Random();
-    for (int i = 0; i < 5120; i++) {
-      input.add((long) rand.nextInt(60));
-    }
-    input.set(511, Long.MAX_VALUE);
-
-    Writer writer = OrcFile.createWriter(testFilePath,
-        OrcFile.writerOptions(conf)
-        .inspector(inspector)
-        .stripeSize(100000)
-        .bufferSize(10000)
-        .encodingStrategy(encodingStrategy));
-    for (Long l : input) {
-      writer.addRow(l);
-    }
-    writer.close();
-
-    Reader reader = OrcFile.createReader(testFilePath,
-        OrcFile.readerOptions(conf).filesystem(fs));
-    RecordReader rows = reader.rows();
-    int idx = 0;
-    while (rows.hasNext()) {
-      Object row = rows.next(null);
-      assertEquals(input.get(idx++).longValue(), ((LongWritable) row).get());
-    }
-  }
-
-  @Test
-  public void testPatchedBaseMax2() throws Exception {
-    ObjectInspector inspector;
-    synchronized (TestOrcFile.class) {
-      inspector = ObjectInspectorFactory.getReflectionObjectInspector(Long.class,
-          ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    }
-
-    List<Long> input = Lists.newArrayList();
-    Random rand = new Random();
-    for (int i = 0; i < 5120; i++) {
-      input.add((long) rand.nextInt(60));
-    }
-    input.set(128, Long.MAX_VALUE);
-    input.set(256, Long.MAX_VALUE);
-    input.set(511, Long.MAX_VALUE);
-
-    Writer writer = OrcFile.createWriter(testFilePath,
-        OrcFile.writerOptions(conf)
-        .inspector(inspector)
-        .stripeSize(100000)
-        .bufferSize(10000)
-        .encodingStrategy(encodingStrategy));
-    for (Long l : input) {
-      writer.addRow(l);
-    }
-    writer.close();
-
-    Reader reader = OrcFile.createReader(testFilePath,
-        OrcFile.readerOptions(conf).filesystem(fs));
-    RecordReader rows = reader.rows();
-    int idx = 0;
-    while (rows.hasNext()) {
-      Object row = rows.next(null);
-      assertEquals(input.get(idx++).longValue(), ((LongWritable) row).get());
-    }
-  }
-
-  @Test
-  public void testPatchedBaseMax3() throws Exception {
-    ObjectInspector inspector;
-    synchronized (TestOrcFile.class) {
-      inspector = ObjectInspectorFactory.getReflectionObjectInspector(Long.class,
-          ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    }
-
-    List<Long> input = Lists.newArrayList();
-    input.add(371946367L);
-    input.add(11963367L);
-    input.add(68639400007L);
-    input.add(100233367L);
-    input.add(6367L);
-    input.add(10026367L);
-    input.add(3670000L);
-    input.add(3602367L);
-    input.add(4719226367L);
-    input.add(7196367L);
-    input.add(444442L);
-    input.add(210267L);
-    input.add(21033L);
-    input.add(160267L);
-    input.add(400267L);
-    input.add(23634347L);
-    input.add(16027L);
-    input.add(46026367L);
-    input.add(Long.MAX_VALUE);
-    input.add(33333L);
-
-    Writer writer = OrcFile.createWriter(testFilePath,
-        OrcFile.writerOptions(conf)
-        .inspector(inspector)
-        .stripeSize(100000)
-        .bufferSize(10000)
-        .encodingStrategy(encodingStrategy));
-    for (Long l : input) {
-      writer.addRow(l);
-    }
-    writer.close();
-
-    Reader reader = OrcFile.createReader(testFilePath,
-        OrcFile.readerOptions(conf).filesystem(fs));
-    RecordReader rows = reader.rows();
-    int idx = 0;
-    while (rows.hasNext()) {
-      Object row = rows.next(null);
-      assertEquals(input.get(idx++).longValue(), ((LongWritable) row).get());
-    }
-  }
-
-  @Test
-  public void testPatchedBaseMax4() throws Exception {
-    ObjectInspector inspector;
-    synchronized (TestOrcFile.class) {
-      inspector = ObjectInspectorFactory.getReflectionObjectInspector(Long.class,
-          ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    }
-
-    List<Long> input = Lists.newArrayList();
-    for (int i = 0; i < 25; i++) {
-      input.add(371292224226367L);
-      input.add(119622332222267L);
-      input.add(686329400222007L);
-      input.add(100233333222367L);
-      input.add(636272333322222L);
-      input.add(10202633223267L);
-      input.add(36700222022230L);
-      input.add(36023226224227L);
-      input.add(47192226364427L);
-      input.add(71963622222447L);
-      input.add(22244444222222L);
-      input.add(21220263327442L);
-      input.add(21032233332232L);
-      input.add(16026322232227L);
-      input.add(40022262272212L);
-      input.add(23634342227222L);
-      input.add(16022222222227L);
-      input.add(46026362222227L);
-      input.add(46026362222227L);
-      input.add(33322222222323L);
-    }
-    input.add(Long.MAX_VALUE);
-
-    Writer writer = OrcFile.createWriter(testFilePath,
-        OrcFile.writerOptions(conf)
-        .inspector(inspector)
-        .stripeSize(100000)
-        .bufferSize(10000)
-        .encodingStrategy(encodingStrategy));
-    for (Long l : input) {
-      writer.addRow(l);
-    }
-    writer.close();
-
-    Reader reader = OrcFile.createReader(testFilePath,
-        OrcFile.readerOptions(conf).filesystem(fs));
-    RecordReader rows = reader.rows();
-    int idx = 0;
-    while (rows.hasNext()) {
-      Object row = rows.next(null);
-      assertEquals(input.get(idx++).longValue(), ((LongWritable) row).get());
-    }
-  }
-
-  @Test
-  public void testPatchedBaseTimestamp() throws Exception {
-    ObjectInspector inspector;
-    synchronized (TestOrcFile.class) {
-      inspector = ObjectInspectorFactory.getReflectionObjectInspector(TSRow.class,
-          ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    }
-
-    Writer writer = OrcFile.createWriter(testFilePath,
-        OrcFile.writerOptions(conf)
-        .inspector(inspector)
-        .stripeSize(100000)
-        .bufferSize(10000)
-        .encodingStrategy(encodingStrategy));
-
-    List<Timestamp> tslist = Lists.newArrayList();
-    tslist.add(Timestamp.valueOf("2099-01-01 00:00:00"));
-    tslist.add(Timestamp.valueOf("2003-01-01 00:00:00"));
-    tslist.add(Timestamp.valueOf("1999-01-01 00:00:00"));
-    tslist.add(Timestamp.valueOf("1995-01-01 00:00:00"));
-    tslist.add(Timestamp.valueOf("2002-01-01 00:00:00"));
-    tslist.add(Timestamp.valueOf("2010-03-02 00:00:00"));
-    tslist.add(Timestamp.valueOf("2005-01-01 00:00:00"));
-    tslist.add(Timestamp.valueOf("2006-01-01 00:00:00"));
-    tslist.add(Timestamp.valueOf("2003-01-01 00:00:00"));
-    tslist.add(Timestamp.valueOf("1996-08-02 00:00:00"));
-    tslist.add(Timestamp.valueOf("1998-11-02 00:00:00"));
-    tslist.add(Timestamp.valueOf("2008-10-02 00:00:00"));
-    tslist.add(Timestamp.valueOf("1993-08-02 00:00:00"));
-    tslist.add(Timestamp.valueOf("2008-01-02 00:00:00"));
-    tslist.add(Timestamp.valueOf("2007-01-01 00:00:00"));
-    tslist.add(Timestamp.valueOf("2004-01-01 00:00:00"));
-    tslist.add(Timestamp.valueOf("2008-10-02 00:00:00"));
-    tslist.add(Timestamp.valueOf("2003-01-01 00:00:00"));
-    tslist.add(Timestamp.valueOf("2004-01-01 00:00:00"));
-    tslist.add(Timestamp.valueOf("2008-01-01 00:00:00"));
-    tslist.add(Timestamp.valueOf("2005-01-01 00:00:00"));
-    tslist.add(Timestamp.valueOf("1994-01-01 00:00:00"));
-    tslist.add(Timestamp.valueOf("2006-01-01 00:00:00"));
-    tslist.add(Timestamp.valueOf("2004-01-01 00:00:00"));
-    tslist.add(Timestamp.valueOf("2001-01-01 00:00:00"));
-    tslist.add(Timestamp.valueOf("2000-01-01 00:00:00"));
-    tslist.add(Timestamp.valueOf("2000-01-01 00:00:00"));
-    tslist.add(Timestamp.valueOf("2002-01-01 00:00:00"));
-    tslist.add(Timestamp.valueOf("2006-01-01 00:00:00"));
-    tslist.add(Timestamp.valueOf("2011-01-01 00:00:00"));
-    tslist.add(Timestamp.valueOf("2002-01-01 00:00:00"));
-    tslist.add(Timestamp.valueOf("2005-01-01 00:00:00"));
-    tslist.add(Timestamp.valueOf("1974-01-01 00:00:00"));
-
-    for (Timestamp ts : tslist) {
-      writer.addRow(new TSRow(ts));
-    }
-
-    writer.close();
-
-    Reader reader = OrcFile.createReader(testFilePath,
-        OrcFile.readerOptions(conf).filesystem(fs));
-    RecordReader rows = reader.rows();
-    int idx = 0;
-    while (rows.hasNext()) {
-      Object row = rows.next(null);
-      assertEquals(tslist.get(idx++).getNanos(),
-          ((TimestampWritable) ((OrcStruct) row).getFieldValue(0)).getNanos());
-    }
-  }
-
-  @Test
-  public void testDirectLargeNegatives() throws Exception {
-    ObjectInspector inspector;
-    synchronized (TestOrcFile.class) {
-      inspector = ObjectInspectorFactory.getReflectionObjectInspector(Long.class,
-          ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    }
-
-    Writer writer = OrcFile.createWriter(testFilePath,
-        OrcFile.writerOptions(conf)
-        .inspector(inspector)
-        .stripeSize(100000)
-        .bufferSize(10000)
-        .encodingStrategy(encodingStrategy));
-
-    writer.addRow(-7486502418706614742L);
-    writer.addRow(0L);
-    writer.addRow(1L);
-    writer.addRow(1L);
-    writer.addRow(-5535739865598783616L);
-    writer.close();
-
-    Reader reader = OrcFile.createReader(testFilePath,
-        OrcFile.readerOptions(conf).filesystem(fs));
-    RecordReader rows = reader.rows();
-    Object row = rows.next(null);
-    assertEquals(-7486502418706614742L, ((LongWritable) row).get());
-    row = rows.next(row);
-    assertEquals(0L, ((LongWritable) row).get());
-    row = rows.next(row);
-    assertEquals(1L, ((LongWritable) row).get());
-    row = rows.next(row);
-    assertEquals(1L, ((LongWritable) row).get());
-    row = rows.next(row);
-    assertEquals(-5535739865598783616L, ((LongWritable) row).get());
-  }
-
-  @Test
-  public void testSeek() throws Exception {
-    ObjectInspector inspector;
-    synchronized (TestOrcFile.class) {
-      inspector = ObjectInspectorFactory.getReflectionObjectInspector(
-          Long.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    }
-
-    List<Long> input = Lists.newArrayList();
-    Random rand = new Random();
-    for(int i = 0; i < 100000; i++) {
-      input.add((long) rand.nextInt());
-    }
-    Writer writer = OrcFile.createWriter(testFilePath,
-                                         OrcFile.writerOptions(conf)
-                                         .inspector(inspector)
-                                         .compress(CompressionKind.NONE)
-                                         .stripeSize(100000)
-                                         .bufferSize(10000)
-                                         .version(OrcFile.Version.V_0_11)
-                                         .encodingStrategy(encodingStrategy));
-    for(Long l : input) {
-      writer.addRow(l);
-    }
-    writer.close();
-
-    Reader reader = OrcFile.createReader(testFilePath,
-        OrcFile.readerOptions(conf).filesystem(fs));
-    RecordReader rows = reader.rows();
-    int idx = 55555;
-    rows.seekToRow(idx);
-    while (rows.hasNext()) {
-      Object row = rows.next(null);
-      assertEquals(input.get(idx++).longValue(), ((LongWritable) row).get());
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcFile.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcFile.java b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcFile.java
index 1a97a6d..c7c2c9d 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcFile.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcFile.java
@@ -45,6 +45,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.llap.TypeDesc;
 import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
 import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
 import org.apache.hadoop.hive.ql.io.sarg.SearchArgumentFactory;
@@ -537,7 +538,7 @@ public class TestOrcFile {
 
     Reader reader = OrcFile.createReader(testFilePath,
         OrcFile.readerOptions(conf).filesystem(fs));
-    RecordReader rows = reader.rows(null);
+    RecordReader rows = reader.rows();
     int idx = 0;
     while (rows.hasNext()) {
       Object row = rows.next(null);
@@ -574,7 +575,7 @@ public class TestOrcFile {
     List<? extends StructField> fields = readerInspector.getAllStructFieldRefs();
     HiveDecimalObjectInspector doi = (HiveDecimalObjectInspector) readerInspector.
         getStructFieldRef("dec").getFieldObjectInspector();
-    RecordReader rows = reader.rows(null);
+    RecordReader rows = reader.rows();
     while (rows.hasNext()) {
       Object row = rows.next(null);
       assertEquals(null, doi.getPrimitiveWritableObject(readerInspector.getStructFieldData(row,
@@ -617,7 +618,7 @@ public class TestOrcFile {
     List<? extends StructField> fields = readerInspector.getAllStructFieldRefs();
     HiveDecimalObjectInspector doi = (HiveDecimalObjectInspector) readerInspector.
         getStructFieldRef("dec").getFieldObjectInspector();
-    RecordReader rows = reader.rows(null);
+    RecordReader rows = reader.rows();
     int idx = 0;
     while (rows.hasNext()) {
       Object row = rows.next(null);
@@ -1702,6 +1703,11 @@ public class TestOrcFile {
     RecordReader rows = reader.rows();
     OrcStruct row = null;
     for(int i=COUNT-1; i >= 0; --i) {
+      // since we are walking backwards, seek back a buffer width so that
+      // we load the previous buffer of rows
+      if (i % COUNT == COUNT - 1) {
+        rows.seekToRow(i - (COUNT - 1));
+      }
       rows.seekToRow(i);
       row = (OrcStruct) rows.next(row);
       BigRow expected = createRandomRow(intValues, doubleValues,
@@ -1816,6 +1822,11 @@ public class TestOrcFile {
     /* all tests are identical to the other seek() tests */
     OrcStruct row = null;
     for(int i=COUNT-1; i >= 0; --i) {
+      // since we are walking backwards, seek back a buffer width so that
+      // we load the previous buffer of rows
+      if (i % COUNT == COUNT - 1) {
+        rows.seekToRow(i - (COUNT - 1));
+      }
       rows.seekToRow(i);
       row = (OrcStruct) rows.next(row);
       BigRow expected = createRandomRow(intValues, doubleValues,
@@ -2067,10 +2078,11 @@ public class TestOrcFile {
         .range(0L, Long.MAX_VALUE)
         .include(new boolean[]{true, true, true})
         .searchArgument(sarg, new String[]{null, "int1", "string1"}));
-    assertEquals(1000L, rows.getRowNumber());
+    assertEquals(0L, rows.getRowNumber());
     OrcStruct row = null;
     for(int i=1000; i < 2000; ++i) {
       assertTrue(rows.hasNext());
+      assertEquals(i, rows.getRowNumber());
       row = (OrcStruct) rows.next(row);
       assertEquals(300 * i, ((IntWritable) row.getFieldValue(0)).get());
       assertEquals(Integer.toHexString(10*i), row.getFieldValue(1).toString());
@@ -2088,7 +2100,6 @@ public class TestOrcFile {
         .range(0L, Long.MAX_VALUE)
         .include(new boolean[]{true, true, true})
         .searchArgument(sarg, new String[]{null, "int1", "string1"}));
-    assertEquals(3500L, rows.getRowNumber());
     assertTrue(!rows.hasNext());
 
     // select first 100 and last 100 rows
@@ -2154,4 +2165,53 @@ public class TestOrcFile {
       Assert.assertEquals(input.get(idx++).longValue(), ((LongWritable) row).get());
     }
   }
+
+  static class MyList {
+    List<Integer> list = new ArrayList<>();
+  }
+
+  @Test
+  public void testListExpansion() throws Exception {
+    ObjectInspector inspector;
+    synchronized (TestOrcFile.class) {
+      inspector = ObjectInspectorFactory.getReflectionObjectInspector
+          (MyList.class,
+              ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
+    }
+    Writer writer = OrcFile.createWriter(testFilePath,
+        OrcFile.writerOptions(conf).inspector(inspector));
+    MyList row = new MyList();
+    row.list.add(1);
+    row.list.add(2);
+    row.list.add(3);
+    writer.addRow(row);
+    row.list.clear();
+    writer.addRow(row);
+    row.list.add(11);
+    row.list.add(12);
+    writer.addRow(row);
+    row.list = null;
+    writer.addRow(row);
+    row.list = new ArrayList<>();
+    row.list.add(21);
+    row.list.add(22);
+    row.list.add(23);
+    row.list.add(24);
+    writer.addRow(row);
+    writer.close();
+    RecordReader reader = OrcFile.createReader(testFilePath,
+        OrcFile.readerOptions(conf)).rows();
+    assertEquals(true, reader.hasNext());
+    OrcStruct orcrow = (OrcStruct) reader.next(null);
+    assertEquals(3, ((List<IntWritable>) orcrow.getFieldValue(0)).size());
+    orcrow = (OrcStruct) reader.next(row);
+    assertEquals(0, ((List<IntWritable>) orcrow.getFieldValue(0)).size());
+    orcrow = (OrcStruct) reader.next(row);
+    assertEquals(2, ((List<IntWritable>) orcrow.getFieldValue(0)).size());
+    assertEquals(null, ((OrcStruct) reader.next(row)).getFieldValue(0));
+    orcrow = (OrcStruct) reader.next(row);
+    assertEquals(4, ((List<IntWritable>) orcrow.getFieldValue(0)).size());
+    assertEquals(false, reader.hasNext());
+    reader.close();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcNullOptimization.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcNullOptimization.java b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcNullOptimization.java
deleted file mode 100644
index e96c809..0000000
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcNullOptimization.java
+++ /dev/null
@@ -1,400 +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.hive.ql.io.orc;
-
-import static junit.framework.Assert.assertEquals;
-import static junit.framework.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-
-import java.io.File;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Random;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
-import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
-import org.apache.hadoop.io.BooleanWritable;
-import org.apache.hadoop.io.IntWritable;
-import org.apache.orc.ColumnStatistics;
-import org.apache.orc.CompressionKind;
-import org.apache.orc.IntegerColumnStatistics;
-import org.apache.orc.OrcProto;
-
-import org.apache.orc.StringColumnStatistics;
-import org.apache.orc.StripeInformation;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestName;
-
-import com.google.common.collect.Lists;
-
-public class TestOrcNullOptimization {
-
-  public static class MyStruct {
-    Integer a;
-    String b;
-    Boolean c;
-    List<InnerStruct> list = new ArrayList<InnerStruct>();
-
-    public MyStruct(Integer a, String b, Boolean c, List<InnerStruct> l) {
-      this.a = a;
-      this.b = b;
-      this.c = c;
-      this.list = l;
-    }
-  }
-
-  public static class InnerStruct {
-    Integer z;
-
-    public InnerStruct(int z) {
-      this.z = z;
-    }
-  }
-
-  Path workDir = new Path(System.getProperty("test.tmp.dir",
-      "target" + File.separator + "test" + File.separator + "tmp"));
-
-  Configuration conf;
-  FileSystem fs;
-  Path testFilePath;
-
-  @Rule
-  public TestName testCaseName = new TestName();
-
-  @Before
-  public void openFileSystem() throws Exception {
-    conf = new Configuration();
-    fs = FileSystem.getLocal(conf);
-    testFilePath = new Path(workDir, "TestOrcNullOptimization." +
-        testCaseName.getMethodName() + ".orc");
-    fs.delete(testFilePath, false);
-  }
-
-  @Test
-  public void testMultiStripeWithNull() throws Exception {
-    ObjectInspector inspector;
-    synchronized (TestOrcNullOptimization.class) {
-      inspector = ObjectInspectorFactory.getReflectionObjectInspector
-          (MyStruct.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    }
-    Writer writer = OrcFile.createWriter(testFilePath,
-                                         OrcFile.writerOptions(conf)
-                                         .inspector(inspector)
-                                         .stripeSize(100000)
-                                         .compress(CompressionKind.NONE)
-                                         .bufferSize(10000));
-    Random rand = new Random(100);
-    writer.addRow(new MyStruct(null, null, true,
-                               Lists.newArrayList(new InnerStruct(100))));
-    for (int i = 2; i < 20000; i++) {
-      writer.addRow(new MyStruct(rand.nextInt(1), "a", true, Lists
-          .newArrayList(new InnerStruct(100))));
-    }
-    writer.addRow(new MyStruct(null, null, true,
-                               Lists.newArrayList(new InnerStruct(100))));
-    writer.close();
-
-    Reader reader = OrcFile.createReader(testFilePath,
-        OrcFile.readerOptions(conf).filesystem(fs));
-    // check the stats
-    ColumnStatistics[] stats = reader.getStatistics();
-    assertEquals(20000, reader.getNumberOfRows());
-    assertEquals(20000, stats[0].getNumberOfValues());
-
-    assertEquals(0, ((IntegerColumnStatistics) stats[1]).getMaximum());
-    assertEquals(0, ((IntegerColumnStatistics) stats[1]).getMinimum());
-    assertEquals(true, ((IntegerColumnStatistics) stats[1]).isSumDefined());
-    assertEquals(0, ((IntegerColumnStatistics) stats[1]).getSum());
-    assertEquals("count: 19998 hasNull: true min: 0 max: 0 sum: 0",
-        stats[1].toString());
-
-    assertEquals("a", ((StringColumnStatistics) stats[2]).getMaximum());
-    assertEquals("a", ((StringColumnStatistics) stats[2]).getMinimum());
-    assertEquals(19998, stats[2].getNumberOfValues());
-    assertEquals("count: 19998 hasNull: true min: a max: a sum: 19998",
-        stats[2].toString());
-
-    // check the inspectors
-    StructObjectInspector readerInspector =
-        (StructObjectInspector) reader.getObjectInspector();
-    assertEquals(ObjectInspector.Category.STRUCT,
-        readerInspector.getCategory());
-    assertEquals("struct<a:int,b:string,c:boolean,list:array<struct<z:int>>>",
-        readerInspector.getTypeName());
-
-    RecordReader rows = reader.rows();
-
-    List<Boolean> expected = Lists.newArrayList();
-    for (StripeInformation sinfo : reader.getStripes()) {
-      expected.add(false);
-    }
-    // only the first and last stripe will have PRESENT stream
-    expected.set(0, true);
-    expected.set(expected.size() - 1, true);
-
-    List<Boolean> got = Lists.newArrayList();
-    // check if the strip footer contains PRESENT stream
-    for (StripeInformation sinfo : reader.getStripes()) {
-      OrcProto.StripeFooter sf =
-        ((RecordReaderImpl) rows).readStripeFooter(sinfo);
-      got.add(sf.toString().indexOf(OrcProto.Stream.Kind.PRESENT.toString())
-              != -1);
-    }
-    assertEquals(expected, got);
-
-    // row 1
-    OrcStruct row = (OrcStruct) rows.next(null);
-    assertNotNull(row);
-    assertNull(row.getFieldValue(0));
-    assertNull(row.getFieldValue(1));
-    assertEquals(new BooleanWritable(true), row.getFieldValue(2));
-    assertEquals(new IntWritable(100),
-        ((OrcStruct) ((ArrayList<?>) row.getFieldValue(3)).get(0)).
-                 getFieldValue(0));
-
-    rows.seekToRow(19998);
-    // last-1 row
-    row = (OrcStruct) rows.next(null);
-    assertNotNull(row);
-    assertNotNull(row.getFieldValue(1));
-    assertEquals(new IntWritable(0), row.getFieldValue(0));
-    assertEquals(new BooleanWritable(true), row.getFieldValue(2));
-    assertEquals(new IntWritable(100),
-        ((OrcStruct) ((ArrayList<?>) row.getFieldValue(3)).get(0)).
-                 getFieldValue(0));
-
-    // last row
-    row = (OrcStruct) rows.next(row);
-    assertNotNull(row);
-    assertNull(row.getFieldValue(0));
-    assertNull(row.getFieldValue(1));
-    assertEquals(new BooleanWritable(true), row.getFieldValue(2));
-    assertEquals(new IntWritable(100),
-        ((OrcStruct) ((ArrayList<?>) row.getFieldValue(3)).get(0)).
-                 getFieldValue(0));
-
-    rows.close();
-  }
-
-  @Test
-  public void testMultiStripeWithoutNull() throws Exception {
-    ObjectInspector inspector;
-    synchronized (TestOrcNullOptimization.class) {
-      inspector = ObjectInspectorFactory.getReflectionObjectInspector
-          (MyStruct.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    }
-    Writer writer = OrcFile.createWriter(testFilePath,
-                                         OrcFile.writerOptions(conf)
-                                         .inspector(inspector)
-                                         .stripeSize(100000)
-                                         .compress(CompressionKind.NONE)
-                                         .bufferSize(10000));
-    Random rand = new Random(100);
-    for (int i = 1; i < 20000; i++) {
-      writer.addRow(new MyStruct(rand.nextInt(1), "a", true, Lists
-          .newArrayList(new InnerStruct(100))));
-    }
-    writer.addRow(new MyStruct(0, "b", true,
-                               Lists.newArrayList(new InnerStruct(100))));
-    writer.close();
-
-    Reader reader = OrcFile.createReader(testFilePath,
-        OrcFile.readerOptions(conf).filesystem(fs));
-    // check the stats
-    ColumnStatistics[] stats = reader.getStatistics();
-    assertEquals(20000, reader.getNumberOfRows());
-    assertEquals(20000, stats[0].getNumberOfValues());
-
-    assertEquals(0, ((IntegerColumnStatistics) stats[1]).getMaximum());
-    assertEquals(0, ((IntegerColumnStatistics) stats[1]).getMinimum());
-    assertEquals(true, ((IntegerColumnStatistics) stats[1]).isSumDefined());
-    assertEquals(0, ((IntegerColumnStatistics) stats[1]).getSum());
-    assertEquals("count: 20000 hasNull: false min: 0 max: 0 sum: 0",
-        stats[1].toString());
-
-    assertEquals("b", ((StringColumnStatistics) stats[2]).getMaximum());
-    assertEquals("a", ((StringColumnStatistics) stats[2]).getMinimum());
-    assertEquals(20000, stats[2].getNumberOfValues());
-    assertEquals("count: 20000 hasNull: false min: a max: b sum: 20000",
-        stats[2].toString());
-
-    // check the inspectors
-    StructObjectInspector readerInspector =
-        (StructObjectInspector) reader.getObjectInspector();
-    assertEquals(ObjectInspector.Category.STRUCT,
-        readerInspector.getCategory());
-    assertEquals("struct<a:int,b:string,c:boolean,list:array<struct<z:int>>>",
-        readerInspector.getTypeName());
-
-    RecordReader rows = reader.rows();
-
-    // none of the stripes will have PRESENT stream
-    List<Boolean> expected = Lists.newArrayList();
-    for (StripeInformation sinfo : reader.getStripes()) {
-      expected.add(false);
-    }
-
-    List<Boolean> got = Lists.newArrayList();
-    // check if the strip footer contains PRESENT stream
-    for (StripeInformation sinfo : reader.getStripes()) {
-      OrcProto.StripeFooter sf =
-        ((RecordReaderImpl) rows).readStripeFooter(sinfo);
-      got.add(sf.toString().indexOf(OrcProto.Stream.Kind.PRESENT.toString())
-              != -1);
-    }
-    assertEquals(expected, got);
-
-    rows.seekToRow(19998);
-    // last-1 row
-    OrcStruct row = (OrcStruct) rows.next(null);
-    assertNotNull(row);
-    assertNotNull(row.getFieldValue(1));
-    assertEquals(new IntWritable(0), row.getFieldValue(0));
-    assertEquals("a", row.getFieldValue(1).toString());
-    assertEquals(new BooleanWritable(true), row.getFieldValue(2));
-    assertEquals(new IntWritable(100),
-                 ((OrcStruct) ((ArrayList<?>) row.getFieldValue(3)).get(0)).
-                   getFieldValue(0));
-
-    // last row
-    row = (OrcStruct) rows.next(row);
-    assertNotNull(row);
-    assertNotNull(row.getFieldValue(0));
-    assertNotNull(row.getFieldValue(1));
-    assertEquals("b", row.getFieldValue(1).toString());
-    assertEquals(new BooleanWritable(true), row.getFieldValue(2));
-    assertEquals(new IntWritable(100),
-                 ((OrcStruct) ((ArrayList<?>) row.getFieldValue(3)).get(0)).
-                   getFieldValue(0));
-    rows.close();
-  }
-
-  @Test
-  public void testColumnsWithNullAndCompression() throws Exception {
-    ObjectInspector inspector;
-    synchronized (TestOrcNullOptimization.class) {
-      inspector = ObjectInspectorFactory.getReflectionObjectInspector
-          (MyStruct.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    }
-    Writer writer = OrcFile.createWriter(testFilePath,
-                                         OrcFile.writerOptions(conf)
-                                         .inspector(inspector)
-                                         .stripeSize(100000)
-                                         .bufferSize(10000));
-    writer.addRow(new MyStruct(3, "a", true,
-                               Lists.newArrayList(new InnerStruct(100))));
-    writer.addRow(new MyStruct(null, "b", true,
-                               Lists.newArrayList(new InnerStruct(100))));
-    writer.addRow(new MyStruct(3, null, false,
-                               Lists.newArrayList(new InnerStruct(100))));
-    writer.addRow(new MyStruct(3, "d", true,
-                               Lists.newArrayList(new InnerStruct(100))));
-    writer.addRow(new MyStruct(2, "e", true,
-                               Lists.newArrayList(new InnerStruct(100))));
-    writer.addRow(new MyStruct(2, "f", true,
-                               Lists.newArrayList(new InnerStruct(100))));
-    writer.addRow(new MyStruct(2, "g", true,
-                               Lists.newArrayList(new InnerStruct(100))));
-    writer.addRow(new MyStruct(2, "h", true,
-                               Lists.newArrayList(new InnerStruct(100))));
-    writer.close();
-
-    Reader reader = OrcFile.createReader(testFilePath,
-        OrcFile.readerOptions(conf).filesystem(fs));
-    // check the stats
-    ColumnStatistics[] stats = reader.getStatistics();
-    assertEquals(8, reader.getNumberOfRows());
-    assertEquals(8, stats[0].getNumberOfValues());
-
-    assertEquals(3, ((IntegerColumnStatistics) stats[1]).getMaximum());
-    assertEquals(2, ((IntegerColumnStatistics) stats[1]).getMinimum());
-    assertEquals(true, ((IntegerColumnStatistics) stats[1]).isSumDefined());
-    assertEquals(17, ((IntegerColumnStatistics) stats[1]).getSum());
-    assertEquals("count: 7 hasNull: true min: 2 max: 3 sum: 17",
-        stats[1].toString());
-
-    assertEquals("h", ((StringColumnStatistics) stats[2]).getMaximum());
-    assertEquals("a", ((StringColumnStatistics) stats[2]).getMinimum());
-    assertEquals(7, stats[2].getNumberOfValues());
-    assertEquals("count: 7 hasNull: true min: a max: h sum: 7",
-        stats[2].toString());
-
-    // check the inspectors
-    StructObjectInspector readerInspector =
-        (StructObjectInspector) reader.getObjectInspector();
-    assertEquals(ObjectInspector.Category.STRUCT,
-        readerInspector.getCategory());
-    assertEquals("struct<a:int,b:string,c:boolean,list:array<struct<z:int>>>",
-        readerInspector.getTypeName());
-
-    RecordReader rows = reader.rows();
-    // only the last strip will have PRESENT stream
-    List<Boolean> expected = Lists.newArrayList();
-    for (StripeInformation sinfo : reader.getStripes()) {
-      expected.add(false);
-    }
-    expected.set(expected.size() - 1, true);
-
-    List<Boolean> got = Lists.newArrayList();
-    // check if the strip footer contains PRESENT stream
-    for (StripeInformation sinfo : reader.getStripes()) {
-      OrcProto.StripeFooter sf =
-        ((RecordReaderImpl) rows).readStripeFooter(sinfo);
-      got.add(sf.toString().indexOf(OrcProto.Stream.Kind.PRESENT.toString())
-              != -1);
-    }
-    assertEquals(expected, got);
-
-    // row 1
-    OrcStruct row = (OrcStruct) rows.next(null);
-    assertNotNull(row);
-    assertEquals(new IntWritable(3), row.getFieldValue(0));
-    assertEquals("a", row.getFieldValue(1).toString());
-    assertEquals(new BooleanWritable(true), row.getFieldValue(2));
-    assertEquals(new IntWritable(100),
-        ((OrcStruct) ((ArrayList<?>) row.getFieldValue(3)).get(0)).
-                 getFieldValue(0));
-
-    // row 2
-    row = (OrcStruct) rows.next(row);
-    assertNotNull(row);
-    assertNull(row.getFieldValue(0));
-    assertEquals("b", row.getFieldValue(1).toString());
-    assertEquals(new BooleanWritable(true), row.getFieldValue(2));
-    assertEquals(new IntWritable(100),
-        ((OrcStruct) ((ArrayList<?>) row.getFieldValue(3)).get(0)).
-                 getFieldValue(0));
-
-    // row 3
-    row = (OrcStruct) rows.next(row);
-    assertNotNull(row);
-    assertNull(row.getFieldValue(1));
-    assertEquals(new IntWritable(3), row.getFieldValue(0));
-    assertEquals(new BooleanWritable(false), row.getFieldValue(2));
-    assertEquals(new IntWritable(100),
-                 ((OrcStruct) ((ArrayList<?>) row.getFieldValue(3)).get(0)).
-                 getFieldValue(0));
-    rows.close();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcRecordUpdater.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcRecordUpdater.java b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcRecordUpdater.java
index 973cc40..0a61fb8 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcRecordUpdater.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcRecordUpdater.java
@@ -40,6 +40,8 @@ import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.Reporter;
+import org.apache.orc.impl.OrcAcidUtils;
+import org.apache.orc.tools.FileDump;
 import org.junit.Test;
 
 public class TestOrcRecordUpdater {
@@ -115,7 +117,7 @@ public class TestOrcRecordUpdater {
     assertEquals(5L, updater.getStats().getRowCount());
 
     Path bucketPath = AcidUtils.createFilename(root, options);
-    Path sidePath = OrcRecordUpdater.getSideFile(bucketPath);
+    Path sidePath = OrcAcidUtils.getSideFile(bucketPath);
     DataInputStream side = fs.open(sidePath);
 
     // read the stopping point for the first flush and make sure we only see

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcTimezone1.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcTimezone1.java b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcTimezone1.java
deleted file mode 100644
index 526c357..0000000
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcTimezone1.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.hive.ql.io.orc;
-
-import static junit.framework.Assert.assertEquals;
-import static junit.framework.Assert.assertNotNull;
-
-import java.io.File;
-import java.sql.Timestamp;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.List;
-import java.util.TimeZone;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hive.serde2.io.TimestampWritable;
-import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
-import org.apache.hadoop.hive.serde2.objectinspector.StructField;
-import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.TimestampObjectInspector;
-import org.apache.hive.common.util.HiveTestUtils;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestName;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-import com.google.common.collect.Lists;
-
-/**
- *
- */
-@RunWith(Parameterized.class)
-public class TestOrcTimezone1 {
-  Path workDir = new Path(System.getProperty("test.tmp.dir",
-      "target" + File.separator + "test" + File.separator + "tmp"));
-  Configuration conf;
-  FileSystem fs;
-  Path testFilePath;
-  String writerTimeZone;
-  String readerTimeZone;
-  static TimeZone defaultTimeZone = TimeZone.getDefault();
-
-  public TestOrcTimezone1(String writerTZ, String readerTZ) {
-    this.writerTimeZone = writerTZ;
-    this.readerTimeZone = readerTZ;
-  }
-
-  @Parameterized.Parameters
-  public static Collection<Object[]> data() {
-    List<Object[]> result = Arrays.asList(new Object[][]{
-        /* Extreme timezones */
-        {"GMT-12:00", "GMT+14:00"},
-        /* No difference in DST */
-        {"America/Los_Angeles", "America/Los_Angeles"}, /* same timezone both with DST */
-        {"Europe/Berlin", "Europe/Berlin"}, /* same as above but europe */
-        {"America/Phoenix", "Asia/Kolkata"} /* Writer no DST, Reader no DST */,
-        {"Europe/Berlin", "America/Los_Angeles"} /* Writer DST, Reader DST */,
-        {"Europe/Berlin", "America/Chicago"} /* Writer DST, Reader DST */,
-        /* With DST difference */
-        {"Europe/Berlin", "UTC"},
-        {"UTC", "Europe/Berlin"} /* Writer no DST, Reader DST */,
-        {"America/Los_Angeles", "Asia/Kolkata"} /* Writer DST, Reader no DST */,
-        {"Europe/Berlin", "Asia/Kolkata"} /* Writer DST, Reader no DST */,
-        /* Timezone offsets for the reader has changed historically */
-        {"Asia/Saigon", "Pacific/Enderbury"},
-        {"UTC", "Asia/Jerusalem"},
-
-        // NOTE:
-        // "1995-01-01 03:00:00.688888888" this is not a valid time in Pacific/Enderbury timezone.
-        // On 1995-01-01 00:00:00 GMT offset moved from -11:00 hr to +13:00 which makes all values
-        // on 1995-01-01 invalid. Try this with joda time
-        // new MutableDateTime("1995-01-01", DateTimeZone.forTimeZone(readerTimeZone));
-    });
-    return result;
-  }
-
-  @Rule
-  public TestName testCaseName = new TestName();
-
-  @Before
-  public void openFileSystem() throws Exception {
-    conf = new Configuration();
-    fs = FileSystem.getLocal(conf);
-    testFilePath = new Path(workDir, "TestOrcFile." +
-        testCaseName.getMethodName() + ".orc");
-    fs.delete(testFilePath, false);
-  }
-
-  @After
-  public void restoreTimeZone() {
-    TimeZone.setDefault(defaultTimeZone);
-  }
-
-  @Test
-  public void testTimestampWriter() throws Exception {
-    ObjectInspector inspector;
-    synchronized (TestOrcFile.class) {
-      inspector = ObjectInspectorFactory.getReflectionObjectInspector(Timestamp.class,
-          ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    }
-
-    TimeZone.setDefault(TimeZone.getTimeZone(writerTimeZone));
-    Writer writer = OrcFile.createWriter(testFilePath,
-        OrcFile.writerOptions(conf).inspector(inspector).stripeSize(100000).bufferSize(10000));
-    assertEquals(writerTimeZone, TimeZone.getDefault().getID());
-    List<String> ts = Lists.newArrayList();
-    ts.add("2003-01-01 01:00:00.000000222");
-    ts.add("1996-08-02 09:00:00.723100809");
-    ts.add("1999-01-01 02:00:00.999999999");
-    ts.add("1995-01-02 03:00:00.688888888");
-    ts.add("2002-01-01 04:00:00.1");
-    ts.add("2010-03-02 05:00:00.000009001");
-    ts.add("2005-01-01 06:00:00.000002229");
-    ts.add("2006-01-01 07:00:00.900203003");
-    ts.add("2003-01-01 08:00:00.800000007");
-    ts.add("1998-11-02 10:00:00.857340643");
-    ts.add("2008-10-02 11:00:00.0");
-    ts.add("2037-01-01 00:00:00.000999");
-    ts.add("2014-03-28 00:00:00.0");
-    for (String t : ts) {
-      writer.addRow(Timestamp.valueOf(t));
-    }
-    writer.close();
-
-    TimeZone.setDefault(TimeZone.getTimeZone(readerTimeZone));
-    Reader reader = OrcFile.createReader(testFilePath,
-        OrcFile.readerOptions(conf).filesystem(fs));
-    assertEquals(readerTimeZone, TimeZone.getDefault().getID());
-    RecordReader rows = reader.rows(null);
-    int idx = 0;
-    while (rows.hasNext()) {
-      Object row = rows.next(null);
-      Timestamp got = ((TimestampWritable) row).getTimestamp();
-      assertEquals(ts.get(idx++), got.toString());
-    }
-    rows.close();
-  }
-
-  @Test
-  public void testReadTimestampFormat_0_11() throws Exception {
-    TimeZone.setDefault(TimeZone.getTimeZone(readerTimeZone));
-    Path oldFilePath =
-        new Path(HiveTestUtils.getFileFromClasspath("orc-file-11-format.orc"));
-    Reader reader = OrcFile.createReader(oldFilePath,
-        OrcFile.readerOptions(conf).filesystem(fs));
-
-    StructObjectInspector readerInspector = (StructObjectInspector) reader
-        .getObjectInspector();
-    List<? extends StructField> fields = readerInspector
-        .getAllStructFieldRefs();
-    TimestampObjectInspector tso = (TimestampObjectInspector) readerInspector
-        .getStructFieldRef("ts").getFieldObjectInspector();
-    
-    RecordReader rows = reader.rows();
-    Object row = rows.next(null);
-    assertNotNull(row);
-    assertEquals(Timestamp.valueOf("2000-03-12 15:00:00"),
-        tso.getPrimitiveJavaObject(readerInspector.getStructFieldData(row,
-            fields.get(12))));
-    
-    // check the contents of second row
-    assertEquals(true, rows.hasNext());
-    rows.seekToRow(7499);
-    row = rows.next(null);
-    assertEquals(Timestamp.valueOf("2000-03-12 15:00:01"),
-        tso.getPrimitiveJavaObject(readerInspector.getStructFieldData(row,
-            fields.get(12))));
-    
-    // handle the close up
-    assertEquals(false, rows.hasNext());
-    rows.close();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcTimezone2.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcTimezone2.java b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcTimezone2.java
deleted file mode 100644
index 3eae4a9..0000000
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcTimezone2.java
+++ /dev/null
@@ -1,142 +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.hive.ql.io.orc;
-
-import static junit.framework.Assert.assertEquals;
-
-import java.io.File;
-import java.sql.Timestamp;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.List;
-import java.util.Random;
-import java.util.TimeZone;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hive.serde2.io.TimestampWritable;
-import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestName;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-import com.google.common.collect.Lists;
-
-/**
- *
- */
-@RunWith(Parameterized.class)
-public class TestOrcTimezone2 {
-  Path workDir = new Path(System.getProperty("test.tmp.dir",
-      "target" + File.separator + "test" + File.separator + "tmp"));
-  Configuration conf;
-  FileSystem fs;
-  Path testFilePath;
-  String writerTimeZone;
-  String readerTimeZone;
-  static TimeZone defaultTimeZone = TimeZone.getDefault();
-
-  public TestOrcTimezone2(String writerTZ, String readerTZ) {
-    this.writerTimeZone = writerTZ;
-    this.readerTimeZone = readerTZ;
-  }
-
-  @Parameterized.Parameters
-  public static Collection<Object[]> data() {
-    String[] allTimeZones = TimeZone.getAvailableIDs();
-    Random rand = new Random(123);
-    int len = allTimeZones.length;
-    int n = 500;
-    Object[][] data = new Object[n][];
-    for (int i = 0; i < n; i++) {
-      int wIdx = rand.nextInt(len);
-      int rIdx = rand.nextInt(len);
-      data[i] = new Object[2];
-      data[i][0] = allTimeZones[wIdx];
-      data[i][1] = allTimeZones[rIdx];
-    }
-    return Arrays.asList(data);
-  }
-
-  @Rule
-  public TestName testCaseName = new TestName();
-
-  @Before
-  public void openFileSystem() throws Exception {
-    conf = new Configuration();
-    fs = FileSystem.getLocal(conf);
-    testFilePath = new Path(workDir, "TestOrcFile." +
-        testCaseName.getMethodName() + ".orc");
-    fs.delete(testFilePath, false);
-  }
-
-  @After
-  public void restoreTimeZone() {
-    TimeZone.setDefault(defaultTimeZone);
-  }
-
-  @Test
-  public void testTimestampWriter() throws Exception {
-    ObjectInspector inspector;
-    synchronized (TestOrcFile.class) {
-      inspector = ObjectInspectorFactory.getReflectionObjectInspector(Timestamp.class,
-          ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    }
-
-    TimeZone.setDefault(TimeZone.getTimeZone(writerTimeZone));
-    Writer writer = OrcFile.createWriter(testFilePath,
-        OrcFile.writerOptions(conf).inspector(inspector).stripeSize(100000).bufferSize(10000));
-    assertEquals(writerTimeZone, TimeZone.getDefault().getID());
-    List<String> ts = Lists.newArrayList();
-    ts.add("2003-01-01 01:00:00.000000222");
-    ts.add("1999-01-01 02:00:00.999999999");
-    ts.add("1995-01-02 03:00:00.688888888");
-    ts.add("2002-01-01 04:00:00.1");
-    ts.add("2010-03-02 05:00:00.000009001");
-    ts.add("2005-01-01 06:00:00.000002229");
-    ts.add("2006-01-01 07:00:00.900203003");
-    ts.add("2003-01-01 08:00:00.800000007");
-    ts.add("1996-08-02 09:00:00.723100809");
-    ts.add("1998-11-02 10:00:00.857340643");
-    ts.add("2008-10-02 11:00:00.0");
-    ts.add("2037-01-01 00:00:00.000999");
-    for (String t : ts) {
-      writer.addRow(Timestamp.valueOf(t));
-    }
-    writer.close();
-
-    TimeZone.setDefault(TimeZone.getTimeZone(readerTimeZone));
-    Reader reader = OrcFile.createReader(testFilePath,
-        OrcFile.readerOptions(conf).filesystem(fs));
-    assertEquals(readerTimeZone, TimeZone.getDefault().getID());
-    RecordReader rows = reader.rows(null);
-    int idx = 0;
-    while (rows.hasNext()) {
-      Object row = rows.next(null);
-      Timestamp got = ((TimestampWritable) row).getTimestamp();
-      assertEquals(ts.get(idx++), got.toString());
-    }
-    rows.close();
-  }
-}


[23/27] hive git commit: HIVE-11417. Move the ReaderImpl and RowReaderImpl to the ORC module, by making shims for the row by row reader. (omalley reviewed by prasanth_j)

Posted by om...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/orc/src/java/org/apache/orc/impl/TreeReaderFactory.java
----------------------------------------------------------------------
diff --git a/orc/src/java/org/apache/orc/impl/TreeReaderFactory.java b/orc/src/java/org/apache/orc/impl/TreeReaderFactory.java
new file mode 100644
index 0000000..6c8ecfd
--- /dev/null
+++ b/orc/src/java/org/apache/orc/impl/TreeReaderFactory.java
@@ -0,0 +1,2093 @@
+/**
+ * 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.orc.impl;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.math.BigInteger;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.TimeZone;
+
+import org.apache.hadoop.hive.common.type.HiveDecimal;
+import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.ListColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.MapColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.StructColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.UnionColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.StringExpr;
+import org.apache.orc.TypeDescription;
+import org.apache.orc.OrcProto;
+
+/**
+ * Factory for creating ORC tree readers.
+ */
+public class TreeReaderFactory {
+
+  public abstract static class TreeReader {
+    protected final int columnId;
+    protected BitFieldReader present = null;
+    protected boolean valuePresent = false;
+    protected int vectorColumnCount;
+
+    TreeReader(int columnId) throws IOException {
+      this(columnId, null);
+    }
+
+    protected TreeReader(int columnId, InStream in) throws IOException {
+      this.columnId = columnId;
+      if (in == null) {
+        present = null;
+        valuePresent = true;
+      } else {
+        present = new BitFieldReader(in, 1);
+      }
+      vectorColumnCount = -1;
+    }
+
+    void setVectorColumnCount(int vectorColumnCount) {
+      this.vectorColumnCount = vectorColumnCount;
+    }
+
+    void checkEncoding(OrcProto.ColumnEncoding encoding) throws IOException {
+      if (encoding.getKind() != OrcProto.ColumnEncoding.Kind.DIRECT) {
+        throw new IOException("Unknown encoding " + encoding + " in column " +
+            columnId);
+      }
+    }
+
+    static IntegerReader createIntegerReader(OrcProto.ColumnEncoding.Kind kind,
+        InStream in,
+        boolean signed, boolean skipCorrupt) throws IOException {
+      switch (kind) {
+        case DIRECT_V2:
+        case DICTIONARY_V2:
+          return new RunLengthIntegerReaderV2(in, signed, skipCorrupt);
+        case DIRECT:
+        case DICTIONARY:
+          return new RunLengthIntegerReader(in, signed);
+        default:
+          throw new IllegalArgumentException("Unknown encoding " + kind);
+      }
+    }
+
+    void startStripe(Map<StreamName, InStream> streams,
+        OrcProto.StripeFooter stripeFooter
+    ) throws IOException {
+      checkEncoding(stripeFooter.getColumnsList().get(columnId));
+      InStream in = streams.get(new StreamName(columnId,
+          OrcProto.Stream.Kind.PRESENT));
+      if (in == null) {
+        present = null;
+        valuePresent = true;
+      } else {
+        present = new BitFieldReader(in, 1);
+      }
+    }
+
+    /**
+     * Seek to the given position.
+     *
+     * @param index the indexes loaded from the file
+     * @throws IOException
+     */
+    void seek(PositionProvider[] index) throws IOException {
+      seek(index[columnId]);
+    }
+
+    public void seek(PositionProvider index) throws IOException {
+      if (present != null) {
+        present.seek(index);
+      }
+    }
+
+    protected long countNonNulls(long rows) throws IOException {
+      if (present != null) {
+        long result = 0;
+        for (long c = 0; c < rows; ++c) {
+          if (present.next() == 1) {
+            result += 1;
+          }
+        }
+        return result;
+      } else {
+        return rows;
+      }
+    }
+
+    abstract void skipRows(long rows) throws IOException;
+
+    /**
+     * Called at the top level to read into the given batch.
+     * @param batch the batch to read into
+     * @param batchSize the number of rows to read
+     * @throws IOException
+     */
+    public void nextBatch(VectorizedRowBatch batch,
+                          int batchSize) throws IOException {
+      batch.cols[0].reset();
+      batch.cols[0].ensureSize(batchSize, false);
+      nextVector(batch.cols[0], null, batchSize);
+    }
+
+    /**
+     * Populates the isNull vector array in the previousVector object based on
+     * the present stream values. This function is called from all the child
+     * readers, and they all set the values based on isNull field value.
+     *
+     * @param previous The columnVector object whose isNull value is populated
+     * @param isNull Whether the each value was null at a higher level. If
+     *               isNull is null, all values are non-null.
+     * @param batchSize      Size of the column vector
+     * @throws IOException
+     */
+    public void nextVector(ColumnVector previous,
+                           boolean[] isNull,
+                           final int batchSize) throws IOException {
+      if (present != null || isNull != null) {
+        // Set noNulls and isNull vector of the ColumnVector based on
+        // present stream
+        previous.noNulls = true;
+        boolean allNull = true;
+        for (int i = 0; i < batchSize; i++) {
+          if (isNull == null || !isNull[i]) {
+            if (present != null && present.next() != 1) {
+              previous.noNulls = false;
+              previous.isNull[i] = true;
+            } else {
+              previous.isNull[i] = false;
+              allNull = false;
+            }
+          } else {
+            previous.noNulls = false;
+            previous.isNull[i] = true;
+          }
+        }
+        previous.isRepeating = !previous.noNulls && allNull;
+      } else {
+        // There is no present stream, this means that all the values are
+        // present.
+        previous.noNulls = true;
+        for (int i = 0; i < batchSize; i++) {
+          previous.isNull[i] = false;
+        }
+      }
+    }
+
+    public BitFieldReader getPresent() {
+      return present;
+    }
+  }
+
+  public static class NullTreeReader extends TreeReader {
+
+    public NullTreeReader(int columnId) throws IOException {
+      super(columnId);
+    }
+
+    @Override
+    public void startStripe(Map<StreamName, InStream> streams,
+                            OrcProto.StripeFooter footer) {
+      // PASS
+    }
+
+    @Override
+    void skipRows(long rows) {
+      // PASS
+    }
+
+    @Override
+    public void seek(PositionProvider position) {
+      // PASS
+    }
+
+    @Override
+    public void seek(PositionProvider[] position) {
+      // PASS
+    }
+
+    @Override
+    public void nextVector(ColumnVector vector, boolean[] isNull, int size) {
+      vector.noNulls = false;
+      vector.isNull[0] = true;
+      vector.isRepeating = true;
+    }
+  }
+
+  public static class BooleanTreeReader extends TreeReader {
+    protected BitFieldReader reader = null;
+
+    BooleanTreeReader(int columnId) throws IOException {
+      this(columnId, null, null);
+    }
+
+    protected BooleanTreeReader(int columnId, InStream present, InStream data) throws IOException {
+      super(columnId, present);
+      if (data != null) {
+        reader = new BitFieldReader(data, 1);
+      }
+    }
+
+    @Override
+    void startStripe(Map<StreamName, InStream> streams,
+        OrcProto.StripeFooter stripeFooter
+    ) throws IOException {
+      super.startStripe(streams, stripeFooter);
+      reader = new BitFieldReader(streams.get(new StreamName(columnId,
+          OrcProto.Stream.Kind.DATA)), 1);
+    }
+
+    @Override
+    void seek(PositionProvider[] index) throws IOException {
+      seek(index[columnId]);
+    }
+
+    @Override
+    public void seek(PositionProvider index) throws IOException {
+      super.seek(index);
+      reader.seek(index);
+    }
+
+    @Override
+    void skipRows(long items) throws IOException {
+      reader.skip(countNonNulls(items));
+    }
+
+    @Override
+    public void nextVector(ColumnVector previousVector,
+                           boolean[] isNull,
+                           final int batchSize) throws IOException {
+      LongColumnVector result = (LongColumnVector) previousVector;
+
+      // Read present/isNull stream
+      super.nextVector(result, isNull, batchSize);
+
+      // Read value entries based on isNull entries
+      reader.nextVector(result, batchSize);
+    }
+  }
+
+  public static class ByteTreeReader extends TreeReader {
+    protected RunLengthByteReader reader = null;
+
+    ByteTreeReader(int columnId) throws IOException {
+      this(columnId, null, null);
+    }
+
+    protected ByteTreeReader(int columnId, InStream present, InStream data) throws IOException {
+      super(columnId, present);
+      this.reader = new RunLengthByteReader(data);
+    }
+
+    @Override
+    void startStripe(Map<StreamName, InStream> streams,
+        OrcProto.StripeFooter stripeFooter
+    ) throws IOException {
+      super.startStripe(streams, stripeFooter);
+      reader = new RunLengthByteReader(streams.get(new StreamName(columnId,
+          OrcProto.Stream.Kind.DATA)));
+    }
+
+    @Override
+    void seek(PositionProvider[] index) throws IOException {
+      seek(index[columnId]);
+    }
+
+    @Override
+    public void seek(PositionProvider index) throws IOException {
+      super.seek(index);
+      reader.seek(index);
+    }
+
+    @Override
+    public void nextVector(ColumnVector previousVector,
+                           boolean[] isNull,
+                           final int batchSize) throws IOException {
+      final LongColumnVector result = (LongColumnVector) previousVector;
+
+      // Read present/isNull stream
+      super.nextVector(result, isNull, batchSize);
+
+      // Read value entries based on isNull entries
+      reader.nextVector(result, result.vector, batchSize);
+    }
+
+    @Override
+    void skipRows(long items) throws IOException {
+      reader.skip(countNonNulls(items));
+    }
+  }
+
+  public static class ShortTreeReader extends TreeReader {
+    protected IntegerReader reader = null;
+
+    ShortTreeReader(int columnId) throws IOException {
+      this(columnId, null, null, null);
+    }
+
+    protected ShortTreeReader(int columnId, InStream present, InStream data,
+        OrcProto.ColumnEncoding encoding)
+        throws IOException {
+      super(columnId, present);
+      if (data != null && encoding != null) {
+        checkEncoding(encoding);
+        this.reader = createIntegerReader(encoding.getKind(), data, true, false);
+      }
+    }
+
+    @Override
+    void checkEncoding(OrcProto.ColumnEncoding encoding) throws IOException {
+      if ((encoding.getKind() != OrcProto.ColumnEncoding.Kind.DIRECT) &&
+          (encoding.getKind() != OrcProto.ColumnEncoding.Kind.DIRECT_V2)) {
+        throw new IOException("Unknown encoding " + encoding + " in column " +
+            columnId);
+      }
+    }
+
+    @Override
+    void startStripe(Map<StreamName, InStream> streams,
+        OrcProto.StripeFooter stripeFooter
+    ) throws IOException {
+      super.startStripe(streams, stripeFooter);
+      StreamName name = new StreamName(columnId,
+          OrcProto.Stream.Kind.DATA);
+      reader = createIntegerReader(stripeFooter.getColumnsList().get(columnId).getKind(),
+          streams.get(name), true, false);
+    }
+
+    @Override
+    void seek(PositionProvider[] index) throws IOException {
+      seek(index[columnId]);
+    }
+
+    @Override
+    public void seek(PositionProvider index) throws IOException {
+      super.seek(index);
+      reader.seek(index);
+    }
+
+    @Override
+    public void nextVector(ColumnVector previousVector,
+                           boolean[] isNull,
+                           final int batchSize) throws IOException {
+      final LongColumnVector result = (LongColumnVector) previousVector;
+
+      // Read present/isNull stream
+      super.nextVector(result, isNull, batchSize);
+
+      // Read value entries based on isNull entries
+      reader.nextVector(result, result.vector, batchSize);
+    }
+
+    @Override
+    void skipRows(long items) throws IOException {
+      reader.skip(countNonNulls(items));
+    }
+  }
+
+  public static class IntTreeReader extends TreeReader {
+    protected IntegerReader reader = null;
+
+    IntTreeReader(int columnId) throws IOException {
+      this(columnId, null, null, null);
+    }
+
+    protected IntTreeReader(int columnId, InStream present, InStream data,
+        OrcProto.ColumnEncoding encoding)
+        throws IOException {
+      super(columnId, present);
+      if (data != null && encoding != null) {
+        checkEncoding(encoding);
+        this.reader = createIntegerReader(encoding.getKind(), data, true, false);
+      }
+    }
+
+    @Override
+    void checkEncoding(OrcProto.ColumnEncoding encoding) throws IOException {
+      if ((encoding.getKind() != OrcProto.ColumnEncoding.Kind.DIRECT) &&
+          (encoding.getKind() != OrcProto.ColumnEncoding.Kind.DIRECT_V2)) {
+        throw new IOException("Unknown encoding " + encoding + " in column " +
+            columnId);
+      }
+    }
+
+    @Override
+    void startStripe(Map<StreamName, InStream> streams,
+        OrcProto.StripeFooter stripeFooter
+    ) throws IOException {
+      super.startStripe(streams, stripeFooter);
+      StreamName name = new StreamName(columnId,
+          OrcProto.Stream.Kind.DATA);
+      reader = createIntegerReader(stripeFooter.getColumnsList().get(columnId).getKind(),
+          streams.get(name), true, false);
+    }
+
+    @Override
+    void seek(PositionProvider[] index) throws IOException {
+      seek(index[columnId]);
+    }
+
+    @Override
+    public void seek(PositionProvider index) throws IOException {
+      super.seek(index);
+      reader.seek(index);
+    }
+
+    @Override
+    public void nextVector(ColumnVector previousVector,
+                           boolean[] isNull,
+                           final int batchSize) throws IOException {
+      final LongColumnVector result = (LongColumnVector) previousVector;
+
+      // Read present/isNull stream
+      super.nextVector(result, isNull, batchSize);
+
+      // Read value entries based on isNull entries
+      reader.nextVector(result, result.vector, batchSize);
+    }
+
+    @Override
+    void skipRows(long items) throws IOException {
+      reader.skip(countNonNulls(items));
+    }
+  }
+
+  public static class LongTreeReader extends TreeReader {
+    protected IntegerReader reader = null;
+
+    LongTreeReader(int columnId, boolean skipCorrupt) throws IOException {
+      this(columnId, null, null, null, skipCorrupt);
+    }
+
+    protected LongTreeReader(int columnId, InStream present, InStream data,
+        OrcProto.ColumnEncoding encoding,
+        boolean skipCorrupt)
+        throws IOException {
+      super(columnId, present);
+      if (data != null && encoding != null) {
+        checkEncoding(encoding);
+        this.reader = createIntegerReader(encoding.getKind(), data, true, skipCorrupt);
+      }
+    }
+
+    @Override
+    void checkEncoding(OrcProto.ColumnEncoding encoding) throws IOException {
+      if ((encoding.getKind() != OrcProto.ColumnEncoding.Kind.DIRECT) &&
+          (encoding.getKind() != OrcProto.ColumnEncoding.Kind.DIRECT_V2)) {
+        throw new IOException("Unknown encoding " + encoding + " in column " +
+            columnId);
+      }
+    }
+
+    @Override
+    void startStripe(Map<StreamName, InStream> streams,
+        OrcProto.StripeFooter stripeFooter
+    ) throws IOException {
+      super.startStripe(streams, stripeFooter);
+      StreamName name = new StreamName(columnId,
+          OrcProto.Stream.Kind.DATA);
+      reader = createIntegerReader(stripeFooter.getColumnsList().get(columnId).getKind(),
+          streams.get(name), true, false);
+    }
+
+    @Override
+    void seek(PositionProvider[] index) throws IOException {
+      seek(index[columnId]);
+    }
+
+    @Override
+    public void seek(PositionProvider index) throws IOException {
+      super.seek(index);
+      reader.seek(index);
+    }
+
+    @Override
+    public void nextVector(ColumnVector previousVector,
+                           boolean[] isNull,
+                           final int batchSize) throws IOException {
+      final LongColumnVector result = (LongColumnVector) previousVector;
+
+      // Read present/isNull stream
+      super.nextVector(result, isNull, batchSize);
+
+      // Read value entries based on isNull entries
+      reader.nextVector(result, result.vector, batchSize);
+    }
+
+    @Override
+    void skipRows(long items) throws IOException {
+      reader.skip(countNonNulls(items));
+    }
+  }
+
+  public static class FloatTreeReader extends TreeReader {
+    protected InStream stream;
+    private final SerializationUtils utils;
+
+    FloatTreeReader(int columnId) throws IOException {
+      this(columnId, null, null);
+    }
+
+    protected FloatTreeReader(int columnId, InStream present, InStream data) throws IOException {
+      super(columnId, present);
+      this.utils = new SerializationUtils();
+      this.stream = data;
+    }
+
+    @Override
+    void startStripe(Map<StreamName, InStream> streams,
+        OrcProto.StripeFooter stripeFooter
+    ) throws IOException {
+      super.startStripe(streams, stripeFooter);
+      StreamName name = new StreamName(columnId,
+          OrcProto.Stream.Kind.DATA);
+      stream = streams.get(name);
+    }
+
+    @Override
+    void seek(PositionProvider[] index) throws IOException {
+      seek(index[columnId]);
+    }
+
+    @Override
+    public void seek(PositionProvider index) throws IOException {
+      super.seek(index);
+      stream.seek(index);
+    }
+
+    @Override
+    public void nextVector(ColumnVector previousVector,
+                           boolean[] isNull,
+                           final int batchSize) throws IOException {
+      final DoubleColumnVector result = (DoubleColumnVector) previousVector;
+
+      // Read present/isNull stream
+      super.nextVector(result, isNull, batchSize);
+
+      final boolean hasNulls = !result.noNulls;
+      boolean allNulls = hasNulls;
+
+      if (hasNulls) {
+        // conditions to ensure bounds checks skips
+        for (int i = 0; batchSize <= result.isNull.length && i < batchSize; i++) {
+          allNulls = allNulls & result.isNull[i];
+        }
+        if (allNulls) {
+          result.vector[0] = Double.NaN;
+          result.isRepeating = true;
+        } else {
+          // some nulls
+          result.isRepeating = false;
+          // conditions to ensure bounds checks skips
+          for (int i = 0; batchSize <= result.isNull.length
+              && batchSize <= result.vector.length && i < batchSize; i++) {
+            if (!result.isNull[i]) {
+              result.vector[i] = utils.readFloat(stream);
+            } else {
+              // If the value is not present then set NaN
+              result.vector[i] = Double.NaN;
+            }
+          }
+        }
+      } else {
+        // no nulls & > 1 row (check repeating)
+        boolean repeating = (batchSize > 1);
+        final float f1 = utils.readFloat(stream);
+        result.vector[0] = f1;
+        // conditions to ensure bounds checks skips
+        for (int i = 1; i < batchSize && batchSize <= result.vector.length; i++) {
+          final float f2 = utils.readFloat(stream);
+          repeating = repeating && (f1 == f2);
+          result.vector[i] = f2;
+        }
+        result.isRepeating = repeating;
+      }
+    }
+
+    @Override
+    protected void skipRows(long items) throws IOException {
+      items = countNonNulls(items);
+      for (int i = 0; i < items; ++i) {
+        utils.readFloat(stream);
+      }
+    }
+  }
+
+  public static class DoubleTreeReader extends TreeReader {
+    protected InStream stream;
+    private final SerializationUtils utils;
+
+    DoubleTreeReader(int columnId) throws IOException {
+      this(columnId, null, null);
+    }
+
+    protected DoubleTreeReader(int columnId, InStream present, InStream data) throws IOException {
+      super(columnId, present);
+      this.utils = new SerializationUtils();
+      this.stream = data;
+    }
+
+    @Override
+    void startStripe(Map<StreamName, InStream> streams,
+        OrcProto.StripeFooter stripeFooter
+    ) throws IOException {
+      super.startStripe(streams, stripeFooter);
+      StreamName name =
+          new StreamName(columnId,
+              OrcProto.Stream.Kind.DATA);
+      stream = streams.get(name);
+    }
+
+    @Override
+    void seek(PositionProvider[] index) throws IOException {
+      seek(index[columnId]);
+    }
+
+    @Override
+    public void seek(PositionProvider index) throws IOException {
+      super.seek(index);
+      stream.seek(index);
+    }
+
+    @Override
+    public void nextVector(ColumnVector previousVector,
+                           boolean[] isNull,
+                           final int batchSize) throws IOException {
+      final DoubleColumnVector result = (DoubleColumnVector) previousVector;
+
+      // Read present/isNull stream
+      super.nextVector(result, isNull, batchSize);
+
+      final boolean hasNulls = !result.noNulls;
+      boolean allNulls = hasNulls;
+
+      if (hasNulls) {
+        // conditions to ensure bounds checks skips
+        for (int i = 0; i < batchSize && batchSize <= result.isNull.length; i++) {
+          allNulls = allNulls & result.isNull[i];
+        }
+        if (allNulls) {
+          result.vector[0] = Double.NaN;
+          result.isRepeating = true;
+        } else {
+          // some nulls
+          result.isRepeating = false;
+          // conditions to ensure bounds checks skips
+          for (int i = 0; batchSize <= result.isNull.length
+              && batchSize <= result.vector.length && i < batchSize; i++) {
+            if (!result.isNull[i]) {
+              result.vector[i] = utils.readDouble(stream);
+            } else {
+              // If the value is not present then set NaN
+              result.vector[i] = Double.NaN;
+            }
+          }
+        }
+      } else {
+        // no nulls
+        boolean repeating = (batchSize > 1);
+        final double d1 = utils.readDouble(stream);
+        result.vector[0] = d1;
+        // conditions to ensure bounds checks skips
+        for (int i = 1; i < batchSize && batchSize <= result.vector.length; i++) {
+          final double d2 = utils.readDouble(stream);
+          repeating = repeating && (d1 == d2);
+          result.vector[i] = d2;
+        }
+        result.isRepeating = repeating;
+      }
+    }
+
+    @Override
+    void skipRows(long items) throws IOException {
+      items = countNonNulls(items);
+      long len = items * 8;
+      while (len > 0) {
+        len -= stream.skip(len);
+      }
+    }
+  }
+
+  public static class BinaryTreeReader extends TreeReader {
+    protected InStream stream;
+    protected IntegerReader lengths = null;
+    protected final LongColumnVector scratchlcv;
+
+    BinaryTreeReader(int columnId) throws IOException {
+      this(columnId, null, null, null, null);
+    }
+
+    protected BinaryTreeReader(int columnId, InStream present, InStream data, InStream length,
+        OrcProto.ColumnEncoding encoding) throws IOException {
+      super(columnId, present);
+      scratchlcv = new LongColumnVector();
+      this.stream = data;
+      if (length != null && encoding != null) {
+        checkEncoding(encoding);
+        this.lengths = createIntegerReader(encoding.getKind(), length, false, false);
+      }
+    }
+
+    @Override
+    void checkEncoding(OrcProto.ColumnEncoding encoding) throws IOException {
+      if ((encoding.getKind() != OrcProto.ColumnEncoding.Kind.DIRECT) &&
+          (encoding.getKind() != OrcProto.ColumnEncoding.Kind.DIRECT_V2)) {
+        throw new IOException("Unknown encoding " + encoding + " in column " +
+            columnId);
+      }
+    }
+
+    @Override
+    void startStripe(Map<StreamName, InStream> streams,
+        OrcProto.StripeFooter stripeFooter
+    ) throws IOException {
+      super.startStripe(streams, stripeFooter);
+      StreamName name = new StreamName(columnId,
+          OrcProto.Stream.Kind.DATA);
+      stream = streams.get(name);
+      lengths = createIntegerReader(stripeFooter.getColumnsList().get(columnId).getKind(),
+          streams.get(new StreamName(columnId, OrcProto.Stream.Kind.LENGTH)), false, false);
+    }
+
+    @Override
+    void seek(PositionProvider[] index) throws IOException {
+      seek(index[columnId]);
+    }
+
+    @Override
+    public void seek(PositionProvider index) throws IOException {
+      super.seek(index);
+      stream.seek(index);
+      lengths.seek(index);
+    }
+
+    @Override
+    public void nextVector(ColumnVector previousVector,
+                           boolean[] isNull,
+                           final int batchSize) throws IOException {
+      final BytesColumnVector result = (BytesColumnVector) previousVector;
+
+      // Read present/isNull stream
+      super.nextVector(result, isNull, batchSize);
+
+      BytesColumnVectorUtil.readOrcByteArrays(stream, lengths, scratchlcv, result, batchSize);
+    }
+
+    @Override
+    void skipRows(long items) throws IOException {
+      items = countNonNulls(items);
+      long lengthToSkip = 0;
+      for (int i = 0; i < items; ++i) {
+        lengthToSkip += lengths.next();
+      }
+      while (lengthToSkip > 0) {
+        lengthToSkip -= stream.skip(lengthToSkip);
+      }
+    }
+  }
+
+  public static class TimestampTreeReader extends TreeReader {
+    protected IntegerReader data = null;
+    protected IntegerReader nanos = null;
+    private final boolean skipCorrupt;
+    private Map<String, Long> baseTimestampMap;
+    private long base_timestamp;
+    private final TimeZone readerTimeZone;
+    private TimeZone writerTimeZone;
+    private boolean hasSameTZRules;
+
+    TimestampTreeReader(int columnId, boolean skipCorrupt) throws IOException {
+      this(columnId, null, null, null, null, skipCorrupt);
+    }
+
+    protected TimestampTreeReader(int columnId, InStream presentStream, InStream dataStream,
+        InStream nanosStream, OrcProto.ColumnEncoding encoding, boolean skipCorrupt)
+        throws IOException {
+      super(columnId, presentStream);
+      this.skipCorrupt = skipCorrupt;
+      this.baseTimestampMap = new HashMap<>();
+      this.readerTimeZone = TimeZone.getDefault();
+      this.writerTimeZone = readerTimeZone;
+      this.hasSameTZRules = writerTimeZone.hasSameRules(readerTimeZone);
+      this.base_timestamp = getBaseTimestamp(readerTimeZone.getID());
+      if (encoding != null) {
+        checkEncoding(encoding);
+
+        if (dataStream != null) {
+          this.data = createIntegerReader(encoding.getKind(), dataStream, true, skipCorrupt);
+        }
+
+        if (nanosStream != null) {
+          this.nanos = createIntegerReader(encoding.getKind(), nanosStream, false, skipCorrupt);
+        }
+      }
+    }
+
+    @Override
+    void checkEncoding(OrcProto.ColumnEncoding encoding) throws IOException {
+      if ((encoding.getKind() != OrcProto.ColumnEncoding.Kind.DIRECT) &&
+          (encoding.getKind() != OrcProto.ColumnEncoding.Kind.DIRECT_V2)) {
+        throw new IOException("Unknown encoding " + encoding + " in column " +
+            columnId);
+      }
+    }
+
+    @Override
+    void startStripe(Map<StreamName, InStream> streams,
+        OrcProto.StripeFooter stripeFooter
+    ) throws IOException {
+      super.startStripe(streams, stripeFooter);
+      data = createIntegerReader(stripeFooter.getColumnsList().get(columnId).getKind(),
+          streams.get(new StreamName(columnId,
+              OrcProto.Stream.Kind.DATA)), true, skipCorrupt);
+      nanos = createIntegerReader(stripeFooter.getColumnsList().get(columnId).getKind(),
+          streams.get(new StreamName(columnId,
+              OrcProto.Stream.Kind.SECONDARY)), false, skipCorrupt);
+      base_timestamp = getBaseTimestamp(stripeFooter.getWriterTimezone());
+    }
+
+    private long getBaseTimestamp(String timeZoneId) throws IOException {
+      // to make sure new readers read old files in the same way
+      if (timeZoneId == null || timeZoneId.isEmpty()) {
+        timeZoneId = readerTimeZone.getID();
+      }
+
+      if (!baseTimestampMap.containsKey(timeZoneId)) {
+        writerTimeZone = TimeZone.getTimeZone(timeZoneId);
+        hasSameTZRules = writerTimeZone.hasSameRules(readerTimeZone);
+        SimpleDateFormat sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
+        sdf.setTimeZone(writerTimeZone);
+        try {
+          long epoch =
+              sdf.parse(WriterImpl.BASE_TIMESTAMP_STRING).getTime() / WriterImpl.MILLIS_PER_SECOND;
+          baseTimestampMap.put(timeZoneId, epoch);
+          return epoch;
+        } catch (ParseException e) {
+          throw new IOException("Unable to create base timestamp", e);
+        } finally {
+          sdf.setTimeZone(readerTimeZone);
+        }
+      }
+
+      return baseTimestampMap.get(timeZoneId);
+    }
+
+    @Override
+    void seek(PositionProvider[] index) throws IOException {
+      seek(index[columnId]);
+    }
+
+    @Override
+    public void seek(PositionProvider index) throws IOException {
+      super.seek(index);
+      data.seek(index);
+      nanos.seek(index);
+    }
+
+    @Override
+    public void nextVector(ColumnVector previousVector,
+                           boolean[] isNull,
+                           final int batchSize) throws IOException {
+      TimestampColumnVector result = (TimestampColumnVector) previousVector;
+      super.nextVector(previousVector, isNull, batchSize);
+
+      for (int i = 0; i < batchSize; i++) {
+        if (result.noNulls || !result.isNull[i]) {
+          long millis = data.next() + base_timestamp;
+          int newNanos = parseNanos(nanos.next());
+          if (millis < 0 && newNanos != 0) {
+            millis -= 1;
+          }
+          millis *= WriterImpl.MILLIS_PER_SECOND;
+          long offset = 0;
+          // If reader and writer time zones have different rules, adjust the timezone difference
+          // between reader and writer taking day light savings into account.
+          if (!hasSameTZRules) {
+            offset = writerTimeZone.getOffset(millis) - readerTimeZone.getOffset(millis);
+          }
+          long adjustedMillis = millis + offset;
+          // Sometimes the reader timezone might have changed after adding the adjustedMillis.
+          // To account for that change, check for any difference in reader timezone after
+          // adding adjustedMillis. If so use the new offset (offset at adjustedMillis point of time).
+          if (!hasSameTZRules &&
+              (readerTimeZone.getOffset(millis) != readerTimeZone.getOffset(adjustedMillis))) {
+            long newOffset =
+                writerTimeZone.getOffset(millis) - readerTimeZone.getOffset(adjustedMillis);
+            adjustedMillis = millis + newOffset;
+          }
+          result.time[i] = adjustedMillis;
+          result.nanos[i] = newNanos;
+          if (result.isRepeating && i != 0 &&
+              (result.time[0] != result.time[i] ||
+                  result.nanos[0] != result.nanos[i])) {
+            result.isRepeating = false;
+          }
+        }
+      }
+    }
+
+    private static int parseNanos(long serialized) {
+      int zeros = 7 & (int) serialized;
+      int result = (int) (serialized >>> 3);
+      if (zeros != 0) {
+        for (int i = 0; i <= zeros; ++i) {
+          result *= 10;
+        }
+      }
+      return result;
+    }
+
+    @Override
+    void skipRows(long items) throws IOException {
+      items = countNonNulls(items);
+      data.skip(items);
+      nanos.skip(items);
+    }
+  }
+
+  public static class DateTreeReader extends TreeReader {
+    protected IntegerReader reader = null;
+
+    DateTreeReader(int columnId) throws IOException {
+      this(columnId, null, null, null);
+    }
+
+    protected DateTreeReader(int columnId, InStream present, InStream data,
+        OrcProto.ColumnEncoding encoding) throws IOException {
+      super(columnId, present);
+      if (data != null && encoding != null) {
+        checkEncoding(encoding);
+        reader = createIntegerReader(encoding.getKind(), data, true, false);
+      }
+    }
+
+    @Override
+    void checkEncoding(OrcProto.ColumnEncoding encoding) throws IOException {
+      if ((encoding.getKind() != OrcProto.ColumnEncoding.Kind.DIRECT) &&
+          (encoding.getKind() != OrcProto.ColumnEncoding.Kind.DIRECT_V2)) {
+        throw new IOException("Unknown encoding " + encoding + " in column " +
+            columnId);
+      }
+    }
+
+    @Override
+    void startStripe(Map<StreamName, InStream> streams,
+        OrcProto.StripeFooter stripeFooter
+    ) throws IOException {
+      super.startStripe(streams, stripeFooter);
+      StreamName name = new StreamName(columnId,
+          OrcProto.Stream.Kind.DATA);
+      reader = createIntegerReader(stripeFooter.getColumnsList().get(columnId).getKind(),
+          streams.get(name), true, false);
+    }
+
+    @Override
+    void seek(PositionProvider[] index) throws IOException {
+      seek(index[columnId]);
+    }
+
+    @Override
+    public void seek(PositionProvider index) throws IOException {
+      super.seek(index);
+      reader.seek(index);
+    }
+
+    @Override
+    public void nextVector(ColumnVector previousVector,
+                           boolean[] isNull,
+                           final int batchSize) throws IOException {
+      final LongColumnVector result = (LongColumnVector) previousVector;
+
+      // Read present/isNull stream
+      super.nextVector(result, isNull, batchSize);
+
+      // Read value entries based on isNull entries
+      reader.nextVector(result, result.vector, batchSize);
+    }
+
+    @Override
+    void skipRows(long items) throws IOException {
+      reader.skip(countNonNulls(items));
+    }
+  }
+
+  public static class DecimalTreeReader extends TreeReader {
+    protected InStream valueStream;
+    protected IntegerReader scaleReader = null;
+    private int[] scratchScaleVector;
+
+    private final int precision;
+    private final int scale;
+
+    DecimalTreeReader(int columnId, int precision, int scale) throws IOException {
+      this(columnId, precision, scale, null, null, null, null);
+    }
+
+    protected DecimalTreeReader(int columnId, int precision, int scale, InStream present,
+        InStream valueStream, InStream scaleStream, OrcProto.ColumnEncoding encoding)
+        throws IOException {
+      super(columnId, present);
+      this.precision = precision;
+      this.scale = scale;
+      this.scratchScaleVector = new int[VectorizedRowBatch.DEFAULT_SIZE];
+      this.valueStream = valueStream;
+      if (scaleStream != null && encoding != null) {
+        checkEncoding(encoding);
+        this.scaleReader = createIntegerReader(encoding.getKind(), scaleStream, true, false);
+      }
+    }
+
+    @Override
+    void checkEncoding(OrcProto.ColumnEncoding encoding) throws IOException {
+      if ((encoding.getKind() != OrcProto.ColumnEncoding.Kind.DIRECT) &&
+          (encoding.getKind() != OrcProto.ColumnEncoding.Kind.DIRECT_V2)) {
+        throw new IOException("Unknown encoding " + encoding + " in column " +
+            columnId);
+      }
+    }
+
+    @Override
+    void startStripe(Map<StreamName, InStream> streams,
+        OrcProto.StripeFooter stripeFooter
+    ) throws IOException {
+      super.startStripe(streams, stripeFooter);
+      valueStream = streams.get(new StreamName(columnId,
+          OrcProto.Stream.Kind.DATA));
+      scaleReader = createIntegerReader(stripeFooter.getColumnsList().get(columnId).getKind(),
+          streams.get(new StreamName(columnId, OrcProto.Stream.Kind.SECONDARY)), true, false);
+    }
+
+    @Override
+    void seek(PositionProvider[] index) throws IOException {
+      seek(index[columnId]);
+    }
+
+    @Override
+    public void seek(PositionProvider index) throws IOException {
+      super.seek(index);
+      valueStream.seek(index);
+      scaleReader.seek(index);
+    }
+
+    @Override
+    public void nextVector(ColumnVector previousVector,
+                           boolean[] isNull,
+                           final int batchSize) throws IOException {
+      final DecimalColumnVector result = (DecimalColumnVector) previousVector;
+      // Read present/isNull stream
+      super.nextVector(result, isNull, batchSize);
+
+      if (batchSize > scratchScaleVector.length) {
+        scratchScaleVector = new int[(int) batchSize];
+      }
+      // read the scales
+      scaleReader.nextVector(result, scratchScaleVector, batchSize);
+      // Read value entries based on isNull entries
+      if (result.noNulls) {
+        for (int r=0; r < batchSize; ++r) {
+          BigInteger bInt = SerializationUtils.readBigInteger(valueStream);
+          HiveDecimal dec = HiveDecimal.create(bInt, scratchScaleVector[r]);
+          result.set(r, dec);
+        }
+      } else if (!result.isRepeating || !result.isNull[0]) {
+        for (int r=0; r < batchSize; ++r) {
+          if (!result.isNull[r]) {
+            BigInteger bInt = SerializationUtils.readBigInteger(valueStream);
+            HiveDecimal dec = HiveDecimal.create(bInt, scratchScaleVector[r]);
+            result.set(r, dec);
+          }
+        }
+      }
+    }
+
+    @Override
+    void skipRows(long items) throws IOException {
+      items = countNonNulls(items);
+      for (int i = 0; i < items; i++) {
+        SerializationUtils.readBigInteger(valueStream);
+      }
+      scaleReader.skip(items);
+    }
+  }
+
+  /**
+   * A tree reader that will read string columns. At the start of the
+   * stripe, it creates an internal reader based on whether a direct or
+   * dictionary encoding was used.
+   */
+  public static class StringTreeReader extends TreeReader {
+    protected TreeReader reader;
+
+    StringTreeReader(int columnId) throws IOException {
+      super(columnId);
+    }
+
+    protected StringTreeReader(int columnId, InStream present, InStream data, InStream length,
+        InStream dictionary, OrcProto.ColumnEncoding encoding) throws IOException {
+      super(columnId, present);
+      if (encoding != null) {
+        switch (encoding.getKind()) {
+          case DIRECT:
+          case DIRECT_V2:
+            reader = new StringDirectTreeReader(columnId, present, data, length,
+                encoding.getKind());
+            break;
+          case DICTIONARY:
+          case DICTIONARY_V2:
+            reader = new StringDictionaryTreeReader(columnId, present, data, length, dictionary,
+                encoding);
+            break;
+          default:
+            throw new IllegalArgumentException("Unsupported encoding " +
+                encoding.getKind());
+        }
+      }
+    }
+
+    @Override
+    void checkEncoding(OrcProto.ColumnEncoding encoding) throws IOException {
+      reader.checkEncoding(encoding);
+    }
+
+    @Override
+    void startStripe(Map<StreamName, InStream> streams,
+        OrcProto.StripeFooter stripeFooter
+    ) throws IOException {
+      // For each stripe, checks the encoding and initializes the appropriate
+      // reader
+      switch (stripeFooter.getColumnsList().get(columnId).getKind()) {
+        case DIRECT:
+        case DIRECT_V2:
+          reader = new StringDirectTreeReader(columnId);
+          break;
+        case DICTIONARY:
+        case DICTIONARY_V2:
+          reader = new StringDictionaryTreeReader(columnId);
+          break;
+        default:
+          throw new IllegalArgumentException("Unsupported encoding " +
+              stripeFooter.getColumnsList().get(columnId).getKind());
+      }
+      reader.startStripe(streams, stripeFooter);
+    }
+
+    @Override
+    void seek(PositionProvider[] index) throws IOException {
+      reader.seek(index);
+    }
+
+    @Override
+    public void seek(PositionProvider index) throws IOException {
+      reader.seek(index);
+    }
+
+    @Override
+    public void nextVector(ColumnVector previousVector,
+                           boolean[] isNull,
+                           final int batchSize) throws IOException {
+      reader.nextVector(previousVector, isNull, batchSize);
+    }
+
+    @Override
+    void skipRows(long items) throws IOException {
+      reader.skipRows(items);
+    }
+  }
+
+  // This class collects together very similar methods for reading an ORC vector of byte arrays and
+  // creating the BytesColumnVector.
+  //
+  public static class BytesColumnVectorUtil {
+
+    private static byte[] commonReadByteArrays(InStream stream, IntegerReader lengths,
+        LongColumnVector scratchlcv,
+        BytesColumnVector result, final int batchSize) throws IOException {
+      // Read lengths
+      scratchlcv.isNull = result.isNull;  // Notice we are replacing the isNull vector here...
+      lengths.nextVector(scratchlcv, scratchlcv.vector, batchSize);
+      int totalLength = 0;
+      if (!scratchlcv.isRepeating) {
+        for (int i = 0; i < batchSize; i++) {
+          if (!scratchlcv.isNull[i]) {
+            totalLength += (int) scratchlcv.vector[i];
+          }
+        }
+      } else {
+        if (!scratchlcv.isNull[0]) {
+          totalLength = (int) (batchSize * scratchlcv.vector[0]);
+        }
+      }
+
+      // Read all the strings for this batch
+      byte[] allBytes = new byte[totalLength];
+      int offset = 0;
+      int len = totalLength;
+      while (len > 0) {
+        int bytesRead = stream.read(allBytes, offset, len);
+        if (bytesRead < 0) {
+          throw new EOFException("Can't finish byte read from " + stream);
+        }
+        len -= bytesRead;
+        offset += bytesRead;
+      }
+
+      return allBytes;
+    }
+
+    // This method has the common code for reading in bytes into a BytesColumnVector.
+    public static void readOrcByteArrays(InStream stream,
+                                         IntegerReader lengths,
+                                         LongColumnVector scratchlcv,
+                                         BytesColumnVector result,
+                                         final int batchSize) throws IOException {
+      if (result.noNulls || !(result.isRepeating && result.isNull[0])) {
+        byte[] allBytes = commonReadByteArrays(stream, lengths, scratchlcv,
+            result, (int) batchSize);
+
+        // Too expensive to figure out 'repeating' by comparisons.
+        result.isRepeating = false;
+        int offset = 0;
+        if (!scratchlcv.isRepeating) {
+          for (int i = 0; i < batchSize; i++) {
+            if (!scratchlcv.isNull[i]) {
+              result.setRef(i, allBytes, offset, (int) scratchlcv.vector[i]);
+              offset += scratchlcv.vector[i];
+            } else {
+              result.setRef(i, allBytes, 0, 0);
+            }
+          }
+        } else {
+          for (int i = 0; i < batchSize; i++) {
+            if (!scratchlcv.isNull[i]) {
+              result.setRef(i, allBytes, offset, (int) scratchlcv.vector[0]);
+              offset += scratchlcv.vector[0];
+            } else {
+              result.setRef(i, allBytes, 0, 0);
+            }
+          }
+        }
+      }
+    }
+  }
+
+  /**
+   * A reader for string columns that are direct encoded in the current
+   * stripe.
+   */
+  public static class StringDirectTreeReader extends TreeReader {
+    private static final HadoopShims SHIMS = HadoopShims.Factory.get();
+    protected InStream stream;
+    protected HadoopShims.TextReaderShim data;
+    protected IntegerReader lengths;
+    private final LongColumnVector scratchlcv;
+
+    StringDirectTreeReader(int columnId) throws IOException {
+      this(columnId, null, null, null, null);
+    }
+
+    protected StringDirectTreeReader(int columnId, InStream present, InStream data,
+        InStream length, OrcProto.ColumnEncoding.Kind encoding) throws IOException {
+      super(columnId, present);
+      this.scratchlcv = new LongColumnVector();
+      this.stream = data;
+      if (length != null && encoding != null) {
+        this.lengths = createIntegerReader(encoding, length, false, false);
+        this.data = SHIMS.getTextReaderShim(this.stream);
+      }
+    }
+
+    @Override
+    void checkEncoding(OrcProto.ColumnEncoding encoding) throws IOException {
+      if (encoding.getKind() != OrcProto.ColumnEncoding.Kind.DIRECT &&
+          encoding.getKind() != OrcProto.ColumnEncoding.Kind.DIRECT_V2) {
+        throw new IOException("Unknown encoding " + encoding + " in column " +
+            columnId);
+      }
+    }
+
+    @Override
+    void startStripe(Map<StreamName, InStream> streams,
+        OrcProto.StripeFooter stripeFooter
+    ) throws IOException {
+      super.startStripe(streams, stripeFooter);
+      StreamName name = new StreamName(columnId,
+          OrcProto.Stream.Kind.DATA);
+      stream = streams.get(name);
+      data = SHIMS.getTextReaderShim(this.stream);
+      lengths = createIntegerReader(stripeFooter.getColumnsList().get(columnId).getKind(),
+          streams.get(new StreamName(columnId, OrcProto.Stream.Kind.LENGTH)),
+          false, false);
+    }
+
+    @Override
+    void seek(PositionProvider[] index) throws IOException {
+      seek(index[columnId]);
+    }
+
+    @Override
+    public void seek(PositionProvider index) throws IOException {
+      super.seek(index);
+      stream.seek(index);
+      // don't seek data stream
+      lengths.seek(index);
+    }
+
+    @Override
+    public void nextVector(ColumnVector previousVector,
+                           boolean[] isNull,
+                           final int batchSize) throws IOException {
+      final BytesColumnVector result = (BytesColumnVector) previousVector;
+
+      // Read present/isNull stream
+      super.nextVector(result, isNull, batchSize);
+
+      BytesColumnVectorUtil.readOrcByteArrays(stream, lengths, scratchlcv,
+          result, batchSize);
+    }
+
+    @Override
+    void skipRows(long items) throws IOException {
+      items = countNonNulls(items);
+      long lengthToSkip = 0;
+      for (int i = 0; i < items; ++i) {
+        lengthToSkip += lengths.next();
+      }
+
+      while (lengthToSkip > 0) {
+        lengthToSkip -= stream.skip(lengthToSkip);
+      }
+    }
+
+    public IntegerReader getLengths() {
+      return lengths;
+    }
+
+    public InStream getStream() {
+      return stream;
+    }
+  }
+
+  /**
+   * A reader for string columns that are dictionary encoded in the current
+   * stripe.
+   */
+  public static class StringDictionaryTreeReader extends TreeReader {
+    private static final byte[] EMPTY_BYTE_ARRAY = new byte[0];
+    private DynamicByteArray dictionaryBuffer;
+    private int[] dictionaryOffsets;
+    protected IntegerReader reader;
+
+    private byte[] dictionaryBufferInBytesCache = null;
+    private final LongColumnVector scratchlcv;
+
+    StringDictionaryTreeReader(int columnId) throws IOException {
+      this(columnId, null, null, null, null, null);
+    }
+
+    protected StringDictionaryTreeReader(int columnId, InStream present, InStream data,
+        InStream length, InStream dictionary, OrcProto.ColumnEncoding encoding)
+        throws IOException {
+      super(columnId, present);
+      scratchlcv = new LongColumnVector();
+      if (data != null && encoding != null) {
+        this.reader = createIntegerReader(encoding.getKind(), data, false, false);
+      }
+
+      if (dictionary != null && encoding != null) {
+        readDictionaryStream(dictionary);
+      }
+
+      if (length != null && encoding != null) {
+        readDictionaryLengthStream(length, encoding);
+      }
+    }
+
+    @Override
+    void checkEncoding(OrcProto.ColumnEncoding encoding) throws IOException {
+      if (encoding.getKind() != OrcProto.ColumnEncoding.Kind.DICTIONARY &&
+          encoding.getKind() != OrcProto.ColumnEncoding.Kind.DICTIONARY_V2) {
+        throw new IOException("Unknown encoding " + encoding + " in column " +
+            columnId);
+      }
+    }
+
+    @Override
+    void startStripe(Map<StreamName, InStream> streams,
+        OrcProto.StripeFooter stripeFooter
+    ) throws IOException {
+      super.startStripe(streams, stripeFooter);
+
+      // read the dictionary blob
+      StreamName name = new StreamName(columnId,
+          OrcProto.Stream.Kind.DICTIONARY_DATA);
+      InStream in = streams.get(name);
+      readDictionaryStream(in);
+
+      // read the lengths
+      name = new StreamName(columnId, OrcProto.Stream.Kind.LENGTH);
+      in = streams.get(name);
+      readDictionaryLengthStream(in, stripeFooter.getColumnsList().get(columnId));
+
+      // set up the row reader
+      name = new StreamName(columnId, OrcProto.Stream.Kind.DATA);
+      reader = createIntegerReader(stripeFooter.getColumnsList().get(columnId).getKind(),
+          streams.get(name), false, false);
+    }
+
+    private void readDictionaryLengthStream(InStream in, OrcProto.ColumnEncoding encoding)
+        throws IOException {
+      int dictionarySize = encoding.getDictionarySize();
+      if (in != null) { // Guard against empty LENGTH stream.
+        IntegerReader lenReader = createIntegerReader(encoding.getKind(), in, false, false);
+        int offset = 0;
+        if (dictionaryOffsets == null ||
+            dictionaryOffsets.length < dictionarySize + 1) {
+          dictionaryOffsets = new int[dictionarySize + 1];
+        }
+        for (int i = 0; i < dictionarySize; ++i) {
+          dictionaryOffsets[i] = offset;
+          offset += (int) lenReader.next();
+        }
+        dictionaryOffsets[dictionarySize] = offset;
+        in.close();
+      }
+
+    }
+
+    private void readDictionaryStream(InStream in) throws IOException {
+      if (in != null) { // Guard against empty dictionary stream.
+        if (in.available() > 0) {
+          dictionaryBuffer = new DynamicByteArray(64, in.available());
+          dictionaryBuffer.readAll(in);
+          // Since its start of strip invalidate the cache.
+          dictionaryBufferInBytesCache = null;
+        }
+        in.close();
+      } else {
+        dictionaryBuffer = null;
+      }
+    }
+
+    @Override
+    void seek(PositionProvider[] index) throws IOException {
+      seek(index[columnId]);
+    }
+
+    @Override
+    public void seek(PositionProvider index) throws IOException {
+      super.seek(index);
+      reader.seek(index);
+    }
+
+    @Override
+    public void nextVector(ColumnVector previousVector,
+                           boolean[] isNull,
+                           final int batchSize) throws IOException {
+      final BytesColumnVector result = (BytesColumnVector) previousVector;
+      int offset;
+      int length;
+
+      // Read present/isNull stream
+      super.nextVector(result, isNull, batchSize);
+
+      if (dictionaryBuffer != null) {
+
+        // Load dictionaryBuffer into cache.
+        if (dictionaryBufferInBytesCache == null) {
+          dictionaryBufferInBytesCache = dictionaryBuffer.get();
+        }
+
+        // Read string offsets
+        scratchlcv.isNull = result.isNull;
+        scratchlcv.ensureSize((int) batchSize, false);
+        reader.nextVector(scratchlcv, scratchlcv.vector, batchSize);
+        if (!scratchlcv.isRepeating) {
+
+          // The vector has non-repeating strings. Iterate thru the batch
+          // and set strings one by one
+          for (int i = 0; i < batchSize; i++) {
+            if (!scratchlcv.isNull[i]) {
+              offset = dictionaryOffsets[(int) scratchlcv.vector[i]];
+              length = getDictionaryEntryLength((int) scratchlcv.vector[i], offset);
+              result.setRef(i, dictionaryBufferInBytesCache, offset, length);
+            } else {
+              // If the value is null then set offset and length to zero (null string)
+              result.setRef(i, dictionaryBufferInBytesCache, 0, 0);
+            }
+          }
+        } else {
+          // If the value is repeating then just set the first value in the
+          // vector and set the isRepeating flag to true. No need to iterate thru and
+          // set all the elements to the same value
+          offset = dictionaryOffsets[(int) scratchlcv.vector[0]];
+          length = getDictionaryEntryLength((int) scratchlcv.vector[0], offset);
+          result.setRef(0, dictionaryBufferInBytesCache, offset, length);
+        }
+        result.isRepeating = scratchlcv.isRepeating;
+      } else {
+        if (dictionaryOffsets == null) {
+          // Entire stripe contains null strings.
+          result.isRepeating = true;
+          result.noNulls = false;
+          result.isNull[0] = true;
+          result.setRef(0, EMPTY_BYTE_ARRAY, 0, 0);
+        } else {
+          // stripe contains nulls and empty strings
+          for (int i = 0; i < batchSize; i++) {
+            if (!result.isNull[i]) {
+              result.setRef(i, EMPTY_BYTE_ARRAY, 0, 0);
+            }
+          }
+        }
+      }
+    }
+
+    int getDictionaryEntryLength(int entry, int offset) {
+      final int length;
+      // if it isn't the last entry, subtract the offsets otherwise use
+      // the buffer length.
+      if (entry < dictionaryOffsets.length - 1) {
+        length = dictionaryOffsets[entry + 1] - offset;
+      } else {
+        length = dictionaryBuffer.size() - offset;
+      }
+      return length;
+    }
+
+    @Override
+    void skipRows(long items) throws IOException {
+      reader.skip(countNonNulls(items));
+    }
+
+    public IntegerReader getReader() {
+      return reader;
+    }
+  }
+
+  public static class CharTreeReader extends StringTreeReader {
+    int maxLength;
+
+    CharTreeReader(int columnId, int maxLength) throws IOException {
+      this(columnId, maxLength, null, null, null, null, null);
+    }
+
+    protected CharTreeReader(int columnId, int maxLength, InStream present, InStream data,
+        InStream length, InStream dictionary, OrcProto.ColumnEncoding encoding) throws IOException {
+      super(columnId, present, data, length, dictionary, encoding);
+      this.maxLength = maxLength;
+    }
+
+    @Override
+    public void nextVector(ColumnVector previousVector,
+                           boolean[] isNull,
+                           final int batchSize) throws IOException {
+      // Get the vector of strings from StringTreeReader, then make a 2nd pass to
+      // adjust down the length (right trim and truncate) if necessary.
+      super.nextVector(previousVector, isNull, batchSize);
+      BytesColumnVector result = (BytesColumnVector) previousVector;
+      int adjustedDownLen;
+      if (result.isRepeating) {
+        if (result.noNulls || !result.isNull[0]) {
+          adjustedDownLen = StringExpr
+              .rightTrimAndTruncate(result.vector[0], result.start[0], result.length[0], maxLength);
+          if (adjustedDownLen < result.length[0]) {
+            result.setRef(0, result.vector[0], result.start[0], adjustedDownLen);
+          }
+        }
+      } else {
+        if (result.noNulls) {
+          for (int i = 0; i < batchSize; i++) {
+            adjustedDownLen = StringExpr
+                .rightTrimAndTruncate(result.vector[i], result.start[i], result.length[i],
+                    maxLength);
+            if (adjustedDownLen < result.length[i]) {
+              result.setRef(i, result.vector[i], result.start[i], adjustedDownLen);
+            }
+          }
+        } else {
+          for (int i = 0; i < batchSize; i++) {
+            if (!result.isNull[i]) {
+              adjustedDownLen = StringExpr
+                  .rightTrimAndTruncate(result.vector[i], result.start[i], result.length[i],
+                      maxLength);
+              if (adjustedDownLen < result.length[i]) {
+                result.setRef(i, result.vector[i], result.start[i], adjustedDownLen);
+              }
+            }
+          }
+        }
+      }
+    }
+  }
+
+  public static class VarcharTreeReader extends StringTreeReader {
+    int maxLength;
+
+    VarcharTreeReader(int columnId, int maxLength) throws IOException {
+      this(columnId, maxLength, null, null, null, null, null);
+    }
+
+    protected VarcharTreeReader(int columnId, int maxLength, InStream present, InStream data,
+        InStream length, InStream dictionary, OrcProto.ColumnEncoding encoding) throws IOException {
+      super(columnId, present, data, length, dictionary, encoding);
+      this.maxLength = maxLength;
+    }
+
+    @Override
+    public void nextVector(ColumnVector previousVector,
+                           boolean[] isNull,
+                           final int batchSize) throws IOException {
+      // Get the vector of strings from StringTreeReader, then make a 2nd pass to
+      // adjust down the length (truncate) if necessary.
+      super.nextVector(previousVector, isNull, batchSize);
+      BytesColumnVector result = (BytesColumnVector) previousVector;
+
+      int adjustedDownLen;
+      if (result.isRepeating) {
+        if (result.noNulls || !result.isNull[0]) {
+          adjustedDownLen = StringExpr
+              .truncate(result.vector[0], result.start[0], result.length[0], maxLength);
+          if (adjustedDownLen < result.length[0]) {
+            result.setRef(0, result.vector[0], result.start[0], adjustedDownLen);
+          }
+        }
+      } else {
+        if (result.noNulls) {
+          for (int i = 0; i < batchSize; i++) {
+            adjustedDownLen = StringExpr
+                .truncate(result.vector[i], result.start[i], result.length[i], maxLength);
+            if (adjustedDownLen < result.length[i]) {
+              result.setRef(i, result.vector[i], result.start[i], adjustedDownLen);
+            }
+          }
+        } else {
+          for (int i = 0; i < batchSize; i++) {
+            if (!result.isNull[i]) {
+              adjustedDownLen = StringExpr
+                  .truncate(result.vector[i], result.start[i], result.length[i], maxLength);
+              if (adjustedDownLen < result.length[i]) {
+                result.setRef(i, result.vector[i], result.start[i], adjustedDownLen);
+              }
+            }
+          }
+        }
+      }
+    }
+  }
+
+  protected static class StructTreeReader extends TreeReader {
+    protected final TreeReader[] fields;
+
+    protected StructTreeReader(int columnId,
+                               TypeDescription readerSchema,
+                               SchemaEvolution evolution,
+                               boolean[] included,
+                               boolean skipCorrupt) throws IOException {
+      super(columnId);
+
+      List<TypeDescription> childrenTypes = readerSchema.getChildren();
+      this.fields = new TreeReader[childrenTypes.size()];
+      for (int i = 0; i < fields.length; ++i) {
+        TypeDescription subtype = childrenTypes.get(i);
+        this.fields[i] = createTreeReader(subtype, evolution, included, skipCorrupt);
+      }
+    }
+
+    @Override
+    void seek(PositionProvider[] index) throws IOException {
+      super.seek(index);
+      for (TreeReader kid : fields) {
+        if (kid != null) {
+          kid.seek(index);
+        }
+      }
+    }
+
+    @Override
+    public void nextBatch(VectorizedRowBatch batch,
+                          int batchSize) throws IOException {
+      for(int i=0; i < fields.length &&
+          (vectorColumnCount == -1 || i < vectorColumnCount); ++i) {
+        batch.cols[i].reset();
+        batch.cols[i].ensureSize((int) batchSize, false);
+        fields[i].nextVector(batch.cols[i], null, batchSize);
+      }
+    }
+
+    @Override
+    public void nextVector(ColumnVector previousVector,
+                           boolean[] isNull,
+                           final int batchSize) throws IOException {
+      super.nextVector(previousVector, isNull, batchSize);
+      StructColumnVector result = (StructColumnVector) previousVector;
+      if (result.noNulls || !(result.isRepeating && result.isNull[0])) {
+        result.isRepeating = false;
+
+        // Read all the members of struct as column vectors
+        boolean[] mask = result.noNulls ? null : result.isNull;
+        for (int f = 0; f < fields.length; f++) {
+          if (fields[f] != null) {
+            fields[f].nextVector(result.fields[f], mask, batchSize);
+          }
+        }
+      }
+    }
+
+    @Override
+    void startStripe(Map<StreamName, InStream> streams,
+        OrcProto.StripeFooter stripeFooter
+    ) throws IOException {
+      super.startStripe(streams, stripeFooter);
+      for (TreeReader field : fields) {
+        if (field != null) {
+          field.startStripe(streams, stripeFooter);
+        }
+      }
+    }
+
+    @Override
+    void skipRows(long items) throws IOException {
+      items = countNonNulls(items);
+      for (TreeReader field : fields) {
+        if (field != null) {
+          field.skipRows(items);
+        }
+      }
+    }
+  }
+
+  public static class UnionTreeReader extends TreeReader {
+    protected final TreeReader[] fields;
+    protected RunLengthByteReader tags;
+
+    protected UnionTreeReader(int fileColumn,
+                              TypeDescription readerSchema,
+                              SchemaEvolution evolution,
+                              boolean[] included,
+                              boolean skipCorrupt) throws IOException {
+      super(fileColumn);
+      List<TypeDescription> childrenTypes = readerSchema.getChildren();
+      int fieldCount = childrenTypes.size();
+      this.fields = new TreeReader[fieldCount];
+      for (int i = 0; i < fieldCount; ++i) {
+        TypeDescription subtype = childrenTypes.get(i);
+        this.fields[i] = createTreeReader(subtype, evolution, included, skipCorrupt);
+      }
+    }
+
+    @Override
+    void seek(PositionProvider[] index) throws IOException {
+      super.seek(index);
+      tags.seek(index[columnId]);
+      for (TreeReader kid : fields) {
+        kid.seek(index);
+      }
+    }
+
+    @Override
+    public void nextVector(ColumnVector previousVector,
+                           boolean[] isNull,
+                           final int batchSize) throws IOException {
+      UnionColumnVector result = (UnionColumnVector) previousVector;
+      super.nextVector(result, isNull, batchSize);
+      if (result.noNulls || !(result.isRepeating && result.isNull[0])) {
+        result.isRepeating = false;
+        tags.nextVector(result.noNulls ? null : result.isNull, result.tags,
+            batchSize);
+        boolean[] ignore = new boolean[(int) batchSize];
+        for (int f = 0; f < result.fields.length; ++f) {
+          // build the ignore list for this tag
+          for (int r = 0; r < batchSize; ++r) {
+            ignore[r] = (!result.noNulls && result.isNull[r]) ||
+                result.tags[r] != f;
+          }
+          fields[f].nextVector(result.fields[f], ignore, batchSize);
+        }
+      }
+    }
+
+    @Override
+    void startStripe(Map<StreamName, InStream> streams,
+        OrcProto.StripeFooter stripeFooter
+    ) throws IOException {
+      super.startStripe(streams, stripeFooter);
+      tags = new RunLengthByteReader(streams.get(new StreamName(columnId,
+          OrcProto.Stream.Kind.DATA)));
+      for (TreeReader field : fields) {
+        if (field != null) {
+          field.startStripe(streams, stripeFooter);
+        }
+      }
+    }
+
+    @Override
+    void skipRows(long items) throws IOException {
+      items = countNonNulls(items);
+      long[] counts = new long[fields.length];
+      for (int i = 0; i < items; ++i) {
+        counts[tags.next()] += 1;
+      }
+      for (int i = 0; i < counts.length; ++i) {
+        fields[i].skipRows(counts[i]);
+      }
+    }
+  }
+
+  public static class ListTreeReader extends TreeReader {
+    protected final TreeReader elementReader;
+    protected IntegerReader lengths = null;
+
+    protected ListTreeReader(int fileColumn,
+                             TypeDescription readerSchema,
+                             SchemaEvolution evolution,
+                             boolean[] included,
+                             boolean skipCorrupt) throws IOException {
+      super(fileColumn);
+      TypeDescription elementType = readerSchema.getChildren().get(0);
+      elementReader = createTreeReader(elementType, evolution, included,
+          skipCorrupt);
+    }
+
+    @Override
+    void seek(PositionProvider[] index) throws IOException {
+      super.seek(index);
+      lengths.seek(index[columnId]);
+      elementReader.seek(index);
+    }
+
+    @Override
+    public void nextVector(ColumnVector previous,
+                           boolean[] isNull,
+                           final int batchSize) throws IOException {
+      ListColumnVector result = (ListColumnVector) previous;
+      super.nextVector(result, isNull, batchSize);
+      // if we have some none-null values, then read them
+      if (result.noNulls || !(result.isRepeating && result.isNull[0])) {
+        lengths.nextVector(result, result.lengths, batchSize);
+        // even with repeating lengths, the list doesn't repeat
+        result.isRepeating = false;
+        // build the offsets vector and figure out how many children to read
+        result.childCount = 0;
+        for (int r = 0; r < batchSize; ++r) {
+          if (result.noNulls || !result.isNull[r]) {
+            result.offsets[r] = result.childCount;
+            result.childCount += result.lengths[r];
+          }
+        }
+        result.child.ensureSize(result.childCount, false);
+        elementReader.nextVector(result.child, null, result.childCount);
+      }
+    }
+
+    @Override
+    void checkEncoding(OrcProto.ColumnEncoding encoding) throws IOException {
+      if ((encoding.getKind() != OrcProto.ColumnEncoding.Kind.DIRECT) &&
+          (encoding.getKind() != OrcProto.ColumnEncoding.Kind.DIRECT_V2)) {
+        throw new IOException("Unknown encoding " + encoding + " in column " +
+            columnId);
+      }
+    }
+
+    @Override
+    void startStripe(Map<StreamName, InStream> streams,
+        OrcProto.StripeFooter stripeFooter
+    ) throws IOException {
+      super.startStripe(streams, stripeFooter);
+      lengths = createIntegerReader(stripeFooter.getColumnsList().get(columnId).getKind(),
+          streams.get(new StreamName(columnId,
+              OrcProto.Stream.Kind.LENGTH)), false, false);
+      if (elementReader != null) {
+        elementReader.startStripe(streams, stripeFooter);
+      }
+    }
+
+    @Override
+    void skipRows(long items) throws IOException {
+      items = countNonNulls(items);
+      long childSkip = 0;
+      for (long i = 0; i < items; ++i) {
+        childSkip += lengths.next();
+      }
+      elementReader.skipRows(childSkip);
+    }
+  }
+
+  public static class MapTreeReader extends TreeReader {
+    protected final TreeReader keyReader;
+    protected final TreeReader valueReader;
+    protected IntegerReader lengths = null;
+
+    protected MapTreeReader(int fileColumn,
+                            TypeDescription readerSchema,
+                            SchemaEvolution evolution,
+                            boolean[] included,
+                            boolean skipCorrupt) throws IOException {
+      super(fileColumn);
+      TypeDescription keyType = readerSchema.getChildren().get(0);
+      TypeDescription valueType = readerSchema.getChildren().get(1);
+      keyReader = createTreeReader(keyType, evolution, included, skipCorrupt);
+      valueReader = createTreeReader(valueType, evolution, included, skipCorrupt);
+    }
+
+    @Override
+    void seek(PositionProvider[] index) throws IOException {
+      super.seek(index);
+      lengths.seek(index[columnId]);
+      keyReader.seek(index);
+      valueReader.seek(index);
+    }
+
+    @Override
+    public void nextVector(ColumnVector previous,
+                           boolean[] isNull,
+                           final int batchSize) throws IOException {
+      MapColumnVector result = (MapColumnVector) previous;
+      super.nextVector(result, isNull, batchSize);
+      if (result.noNulls || !(result.isRepeating && result.isNull[0])) {
+        lengths.nextVector(result, result.lengths, batchSize);
+        // even with repeating lengths, the map doesn't repeat
+        result.isRepeating = false;
+        // build the offsets vector and figure out how many children to read
+        result.childCount = 0;
+        for (int r = 0; r < batchSize; ++r) {
+          if (result.noNulls || !result.isNull[r]) {
+            result.offsets[r] = result.childCount;
+            result.childCount += result.lengths[r];
+          }
+        }
+        result.keys.ensureSize(result.childCount, false);
+        result.values.ensureSize(result.childCount, false);
+        keyReader.nextVector(result.keys, null, result.childCount);
+        valueReader.nextVector(result.values, null, result.childCount);
+      }
+    }
+
+    @Override
+    void checkEncoding(OrcProto.ColumnEncoding encoding) throws IOException {
+      if ((encoding.getKind() != OrcProto.ColumnEncoding.Kind.DIRECT) &&
+          (encoding.getKind() != OrcProto.ColumnEncoding.Kind.DIRECT_V2)) {
+        throw new IOException("Unknown encoding " + encoding + " in column " +
+            columnId);
+      }
+    }
+
+    @Override
+    void startStripe(Map<StreamName, InStream> streams,
+        OrcProto.StripeFooter stripeFooter
+    ) throws IOException {
+      super.startStripe(streams, stripeFooter);
+      lengths = createIntegerReader(stripeFooter.getColumnsList().get(columnId).getKind(),
+          streams.get(new StreamName(columnId,
+              OrcProto.Stream.Kind.LENGTH)), false, false);
+      if (keyReader != null) {
+        keyReader.startStripe(streams, stripeFooter);
+      }
+      if (valueReader != null) {
+        valueReader.startStripe(streams, stripeFooter);
+      }
+    }
+
+    @Override
+    void skipRows(long items) throws IOException {
+      items = countNonNulls(items);
+      long childSkip = 0;
+      for (long i = 0; i < items; ++i) {
+        childSkip += lengths.next();
+      }
+      keyReader.skipRows(childSkip);
+      valueReader.skipRows(childSkip);
+    }
+  }
+
+  public static TreeReader createTreeReader(TypeDescription readerType,
+                                            SchemaEvolution evolution,
+                                            boolean[] included,
+                                            boolean skipCorrupt
+                                            ) throws IOException {
+    TypeDescription fileType = evolution.getFileType(readerType);
+    if (fileType == null ||
+        (included != null && !included[readerType.getId()])) {
+      return new NullTreeReader(0);
+    }
+    TypeDescription.Category readerTypeCategory = readerType.getCategory();
+    if (!fileType.getCategory().equals(readerTypeCategory) &&
+        (readerTypeCategory != TypeDescription.Category.STRUCT &&
+         readerTypeCategory != TypeDescription.Category.MAP &&
+         readerTypeCategory != TypeDescription.Category.LIST &&
+         readerTypeCategory != TypeDescription.Category.UNION)) {
+      // We only convert complex children.
+      return ConvertTreeReaderFactory.createConvertTreeReader(readerType, evolution,
+          included, skipCorrupt);
+    }
+    switch (readerTypeCategory) {
+      case BOOLEAN:
+        return new BooleanTreeReader(fileType.getId());
+      case BYTE:
+        return new ByteTreeReader(fileType.getId());
+      case DOUBLE:
+        return new DoubleTreeReader(fileType.getId());
+      case FLOAT:
+        return new FloatTreeReader(fileType.getId());
+      case SHORT:
+        return new ShortTreeReader(fileType.getId());
+      case INT:
+        return new IntTreeReader(fileType.getId());
+      case LONG:
+        return new LongTreeReader(fileType.getId(), skipCorrupt);
+      case STRING:
+        return new StringTreeReader(fileType.getId());
+      case CHAR:
+        return new CharTreeReader(fileType.getId(), readerType.getMaxLength());
+      case VARCHAR:
+        return new VarcharTreeReader(fileType.getId(), readerType.getMaxLength());
+      case BINARY:
+        return new BinaryTreeReader(fileType.getId());
+      case TIMESTAMP:
+        return new TimestampTreeReader(fileType.getId(), skipCorrupt);
+      case DATE:
+        return new DateTreeReader(fileType.getId());
+      case DECIMAL:
+        return new DecimalTreeReader(fileType.getId(), readerType.getPrecision(),
+            readerType.getScale());
+      case STRUCT:
+        return new StructTreeReader(fileType.getId(), readerType,
+            evolution, included, skipCorrupt);
+      case LIST:
+        return new ListTreeReader(fileType.getId(), readerType,
+            evolution, included, skipCorrupt);
+      case MAP:
+        return new MapTreeReader(fileType.getId(), readerType, evolution,
+            included, skipCorrupt);
+      case UNION:
+        return new UnionTreeReader(fileType.getId(), readerType,
+            evolution, included, skipCorrupt);
+      default:
+        throw new IllegalArgumentException("Unsupported type " +
+            readerTypeCategory);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/orc/src/java/org/apache/orc/impl/ZeroCopyShims.java
----------------------------------------------------------------------
diff --git a/orc/src/java/org/apache/orc/impl/ZeroCopyShims.java b/orc/src/java/org/apache/orc/impl/ZeroCopyShims.java
new file mode 100644
index 0000000..de02c8b
--- /dev/null
+++ b/orc/src/java/org/apache/orc/impl/ZeroCopyShims.java
@@ -0,0 +1,89 @@
+/**
+ * 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.orc.impl;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.EnumSet;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.ReadOption;
+import org.apache.hadoop.io.ByteBufferPool;
+
+class ZeroCopyShims {
+  private static final class ByteBufferPoolAdapter implements ByteBufferPool {
+    private HadoopShims.ByteBufferPoolShim pool;
+
+    public ByteBufferPoolAdapter(HadoopShims.ByteBufferPoolShim pool) {
+      this.pool = pool;
+    }
+
+    @Override
+    public final ByteBuffer getBuffer(boolean direct, int length) {
+      return this.pool.getBuffer(direct, length);
+    }
+
+    @Override
+    public final void putBuffer(ByteBuffer buffer) {
+      this.pool.putBuffer(buffer);
+    }
+  }
+
+  private static final class ZeroCopyAdapter implements HadoopShims.ZeroCopyReaderShim {
+    private final FSDataInputStream in;
+    private final ByteBufferPoolAdapter pool;
+    private final static EnumSet<ReadOption> CHECK_SUM = EnumSet
+        .noneOf(ReadOption.class);
+    private final static EnumSet<ReadOption> NO_CHECK_SUM = EnumSet
+        .of(ReadOption.SKIP_CHECKSUMS);
+
+    public ZeroCopyAdapter(FSDataInputStream in,
+                           HadoopShims.ByteBufferPoolShim poolshim) {
+      this.in = in;
+      if (poolshim != null) {
+        pool = new ByteBufferPoolAdapter(poolshim);
+      } else {
+        pool = null;
+      }
+    }
+
+    public final ByteBuffer readBuffer(int maxLength, boolean verifyChecksums)
+        throws IOException {
+      EnumSet<ReadOption> options = NO_CHECK_SUM;
+      if (verifyChecksums) {
+        options = CHECK_SUM;
+      }
+      return this.in.read(this.pool, maxLength, options);
+    }
+
+    public final void releaseBuffer(ByteBuffer buffer) {
+      this.in.releaseBuffer(buffer);
+    }
+
+    @Override
+    public final void close() throws IOException {
+      this.in.close();
+    }
+  }
+
+  public static HadoopShims.ZeroCopyReaderShim getZeroCopyReader(FSDataInputStream in,
+                                                                 HadoopShims.ByteBufferPoolShim pool) throws IOException {
+    return new ZeroCopyAdapter(in, pool);
+  }
+
+}


[13/27] hive git commit: HIVE-11417. Move the ReaderImpl and RowReaderImpl to the ORC module, by making shims for the row by row reader. (omalley reviewed by prasanth_j)

Posted by om...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/ql/src/java/org/apache/hadoop/hive/ql/io/orc/FileDump.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/FileDump.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/FileDump.java
deleted file mode 100644
index 9c2f88f..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/FileDump.java
+++ /dev/null
@@ -1,884 +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
- * <p/>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p/>
- * 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.hive.ql.io.orc;
-
-import java.io.IOException;
-import java.io.OutputStreamWriter;
-import java.io.PrintStream;
-import java.text.DecimalFormat;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.GnuParser;
-import org.apache.commons.cli.HelpFormatter;
-import org.apache.commons.cli.OptionBuilder;
-import org.apache.commons.cli.Options;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.PathFilter;
-import org.apache.hadoop.hdfs.DistributedFileSystem;
-import org.apache.hadoop.hive.ql.io.AcidUtils;
-import org.apache.orc.BloomFilterIO;
-import org.apache.hadoop.hive.serde2.io.ByteWritable;
-import org.apache.hadoop.hive.serde2.io.DoubleWritable;
-import org.apache.hadoop.hive.serde2.io.ShortWritable;
-import org.apache.hadoop.io.BooleanWritable;
-import org.apache.hadoop.io.FloatWritable;
-import org.apache.hadoop.io.IntWritable;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.orc.ColumnStatistics;
-import org.apache.orc.TypeDescription;
-import org.apache.orc.impl.ColumnStatisticsImpl;
-import org.apache.orc.impl.OrcIndex;
-import org.apache.orc.OrcProto;
-import org.apache.orc.StripeInformation;
-import org.apache.orc.StripeStatistics;
-import org.codehaus.jettison.json.JSONException;
-import org.codehaus.jettison.json.JSONWriter;
-
-import com.google.common.base.Joiner;
-import com.google.common.base.Strings;
-import com.google.common.collect.Lists;
-
-/**
- * A tool for printing out the file structure of ORC files.
- */
-public final class FileDump {
-  public static final String UNKNOWN = "UNKNOWN";
-  public static final String SEPARATOR = Strings.repeat("_", 120) + "\n";
-  public static final int DEFAULT_BLOCK_SIZE = 256 * 1024 * 1024;
-  public static final String DEFAULT_BACKUP_PATH = System.getProperty("java.io.tmpdir");
-  public static final PathFilter HIDDEN_AND_SIDE_FILE_FILTER = new PathFilter() {
-    public boolean accept(Path p) {
-      String name = p.getName();
-      return !name.startsWith("_") && !name.startsWith(".") && !name.endsWith(
-          AcidUtils.DELTA_SIDE_FILE_SUFFIX);
-    }
-  };
-
-  // not used
-  private FileDump() {
-  }
-
-  public static void main(String[] args) throws Exception {
-    Configuration conf = new Configuration();
-
-    List<Integer> rowIndexCols = null;
-    Options opts = createOptions();
-    CommandLine cli = new GnuParser().parse(opts, args);
-
-    if (cli.hasOption('h')) {
-      HelpFormatter formatter = new HelpFormatter();
-      formatter.printHelp("orcfiledump", opts);
-      return;
-    }
-
-    boolean dumpData = cli.hasOption('d');
-    boolean recover = cli.hasOption("recover");
-    boolean skipDump = cli.hasOption("skip-dump");
-    String backupPath = DEFAULT_BACKUP_PATH;
-    if (cli.hasOption("backup-path")) {
-      backupPath = cli.getOptionValue("backup-path");
-    }
-
-    if (cli.hasOption("r")) {
-      String[] colStrs = cli.getOptionValue("r").split(",");
-      rowIndexCols = new ArrayList<Integer>(colStrs.length);
-      for (String colStr : colStrs) {
-        rowIndexCols.add(Integer.parseInt(colStr));
-      }
-    }
-
-    boolean printTimeZone = cli.hasOption('t');
-    boolean jsonFormat = cli.hasOption('j');
-    String[] files = cli.getArgs();
-    if (files.length == 0) {
-      System.err.println("Error : ORC files are not specified");
-      return;
-    }
-
-    // if the specified path is directory, iterate through all files and print the file dump
-    List<String> filesInPath = Lists.newArrayList();
-    for (String filename : files) {
-      Path path = new Path(filename);
-      filesInPath.addAll(getAllFilesInPath(path, conf));
-    }
-
-    if (dumpData) {
-      printData(filesInPath, conf);
-    } else if (recover && skipDump) {
-      recoverFiles(filesInPath, conf, backupPath);
-    } else {
-      if (jsonFormat) {
-        boolean prettyPrint = cli.hasOption('p');
-        JsonFileDump.printJsonMetaData(filesInPath, conf, rowIndexCols, prettyPrint, printTimeZone);
-      } else {
-        printMetaData(filesInPath, conf, rowIndexCols, printTimeZone, recover, backupPath);
-      }
-    }
-  }
-
-  /**
-   * This method returns an ORC reader object if the specified file is readable. If the specified
-   * file has side file (_flush_length) file, then max footer offset will be read from the side
-   * file and orc reader will be created from that offset. Since both data file and side file
-   * use hflush() for flushing the data, there could be some inconsistencies and both files could be
-   * out-of-sync. Following are the cases under which null will be returned
-   *
-   * 1) If the file specified by path or its side file is still open for writes
-   * 2) If *_flush_length file does not return any footer offset
-   * 3) If *_flush_length returns a valid footer offset but the data file is not readable at that
-   *    position (incomplete data file)
-   * 4) If *_flush_length file length is not a multiple of 8, then reader will be created from
-   *    previous valid footer. If there is no such footer (file length > 0 and < 8), then null will
-   *    be returned
-   *
-   * Also, if this method detects any file corruption (mismatch between data file and side file)
-   * then it will add the corresponding file to the specified input list for corrupted files.
-   *
-   * In all other cases, where the file is readable this method will return a reader object.
-   *
-   * @param path - file to get reader for
-   * @param conf - configuration object
-   * @param corruptFiles - fills this list with all possible corrupted files
-   * @return - reader for the specified file or null
-   * @throws IOException
-   */
-  static Reader getReader(final Path path, final Configuration conf,
-      final List<String> corruptFiles) throws IOException {
-    FileSystem fs = path.getFileSystem(conf);
-    long dataFileLen = fs.getFileStatus(path).getLen();
-    System.err.println("Processing data file " + path + " [length: " + dataFileLen + "]");
-    Path sideFile = OrcRecordUpdater.getSideFile(path);
-    final boolean sideFileExists = fs.exists(sideFile);
-    boolean openDataFile = false;
-    boolean openSideFile = false;
-    if (fs instanceof DistributedFileSystem) {
-      DistributedFileSystem dfs = (DistributedFileSystem) fs;
-      openDataFile = !dfs.isFileClosed(path);
-      openSideFile = sideFileExists && !dfs.isFileClosed(sideFile);
-    }
-
-    if (openDataFile || openSideFile) {
-      if (openDataFile && openSideFile) {
-        System.err.println("Unable to perform file dump as " + path + " and " + sideFile +
-            " are still open for writes.");
-      } else if (openSideFile) {
-        System.err.println("Unable to perform file dump as " + sideFile +
-            " is still open for writes.");
-      } else {
-        System.err.println("Unable to perform file dump as " + path +
-            " is still open for writes.");
-      }
-
-      return null;
-    }
-
-    Reader reader = null;
-    if (sideFileExists) {
-      final long maxLen = OrcRawRecordMerger.getLastFlushLength(fs, path);
-      final long sideFileLen = fs.getFileStatus(sideFile).getLen();
-      System.err.println("Found flush length file " + sideFile
-          + " [length: " + sideFileLen + ", maxFooterOffset: " + maxLen + "]");
-      // no offsets read from side file
-      if (maxLen == -1) {
-
-        // if data file is larger than last flush length, then additional data could be recovered
-        if (dataFileLen > maxLen) {
-          System.err.println("Data file has more data than max footer offset:" + maxLen +
-              ". Adding data file to recovery list.");
-          if (corruptFiles != null) {
-            corruptFiles.add(path.toUri().toString());
-          }
-        }
-        return null;
-      }
-
-      try {
-        reader = OrcFile.createReader(path, OrcFile.readerOptions(conf).maxLength(maxLen));
-
-        // if data file is larger than last flush length, then additional data could be recovered
-        if (dataFileLen > maxLen) {
-          System.err.println("Data file has more data than max footer offset:" + maxLen +
-              ". Adding data file to recovery list.");
-          if (corruptFiles != null) {
-            corruptFiles.add(path.toUri().toString());
-          }
-        }
-      } catch (Exception e) {
-        if (corruptFiles != null) {
-          corruptFiles.add(path.toUri().toString());
-        }
-        System.err.println("Unable to read data from max footer offset." +
-            " Adding data file to recovery list.");
-        return null;
-      }
-    } else {
-      reader = OrcFile.createReader(path, OrcFile.readerOptions(conf));
-    }
-
-    return reader;
-  }
-
-  public static Collection<String> getAllFilesInPath(final Path path,
-      final Configuration conf) throws IOException {
-    List<String> filesInPath = Lists.newArrayList();
-    FileSystem fs = path.getFileSystem(conf);
-    FileStatus fileStatus = fs.getFileStatus(path);
-    if (fileStatus.isDir()) {
-      FileStatus[] fileStatuses = fs.listStatus(path, HIDDEN_AND_SIDE_FILE_FILTER);
-      for (FileStatus fileInPath : fileStatuses) {
-        if (fileInPath.isDir()) {
-          filesInPath.addAll(getAllFilesInPath(fileInPath.getPath(), conf));
-        } else {
-          filesInPath.add(fileInPath.getPath().toString());
-        }
-      }
-    } else {
-      filesInPath.add(path.toString());
-    }
-
-    return filesInPath;
-  }
-
-  private static void printData(List<String> files,
-      Configuration conf) throws IOException,
-      JSONException {
-    for (String file : files) {
-      try {
-        Path path = new Path(file);
-        Reader reader = getReader(path, conf, Lists.<String>newArrayList());
-        if (reader == null) {
-          continue;
-        }
-        printJsonData(reader);
-        System.out.println(SEPARATOR);
-      } catch (Exception e) {
-        System.err.println("Unable to dump data for file: " + file);
-        continue;
-      }
-    }
-  }
-
-  private static void printMetaData(List<String> files, Configuration conf,
-      List<Integer> rowIndexCols, boolean printTimeZone, final boolean recover,
-      final String backupPath)
-      throws IOException {
-    List<String> corruptFiles = Lists.newArrayList();
-    for (String filename : files) {
-      printMetaDataImpl(filename, conf, rowIndexCols, printTimeZone, corruptFiles);
-      System.out.println(SEPARATOR);
-    }
-
-    if (!corruptFiles.isEmpty()) {
-      if (recover) {
-        recoverFiles(corruptFiles, conf, backupPath);
-      } else {
-        System.err.println(corruptFiles.size() + " file(s) are corrupted." +
-            " Run the following command to recover corrupted files.\n");
-        String fileNames = Joiner.on(" ").skipNulls().join(corruptFiles);
-        System.err.println("hive --orcfiledump --recover --skip-dump " + fileNames);
-        System.out.println(SEPARATOR);
-      }
-    }
-  }
-
-  private static void printMetaDataImpl(final String filename,
-      final Configuration conf, final List<Integer> rowIndexCols, final boolean printTimeZone,
-      final List<String> corruptFiles) throws IOException {
-    Path file = new Path(filename);
-    Reader reader = getReader(file, conf, corruptFiles);
-    // if we can create reader then footer is not corrupt and file will readable
-    if (reader == null) {
-      return;
-    }
-
-    System.out.println("Structure for " + filename);
-    System.out.println("File Version: " + reader.getFileVersion().getName() +
-        " with " + reader.getWriterVersion());
-    RecordReaderImpl rows = (RecordReaderImpl) reader.rows();
-    System.out.println("Rows: " + reader.getNumberOfRows());
-    System.out.println("Compression: " + reader.getCompression());
-    if (reader.getCompression() != CompressionKind.NONE) {
-      System.out.println("Compression size: " + reader.getCompressionSize());
-    }
-    System.out.println("Type: " + reader.getObjectInspector().getTypeName());
-    System.out.println("\nStripe Statistics:");
-    List<StripeStatistics> stripeStats = reader.getStripeStatistics();
-    for (int n = 0; n < stripeStats.size(); n++) {
-      System.out.println("  Stripe " + (n + 1) + ":");
-      StripeStatistics ss = stripeStats.get(n);
-      for (int i = 0; i < ss.getColumnStatistics().length; ++i) {
-        System.out.println("    Column " + i + ": " +
-            ss.getColumnStatistics()[i].toString());
-      }
-    }
-    ColumnStatistics[] stats = reader.getStatistics();
-    int colCount = stats.length;
-    System.out.println("\nFile Statistics:");
-    for (int i = 0; i < stats.length; ++i) {
-      System.out.println("  Column " + i + ": " + stats[i].toString());
-    }
-    System.out.println("\nStripes:");
-    int stripeIx = -1;
-    for (StripeInformation stripe : reader.getStripes()) {
-      ++stripeIx;
-      long stripeStart = stripe.getOffset();
-      OrcProto.StripeFooter footer = rows.readStripeFooter(stripe);
-      if (printTimeZone) {
-        String tz = footer.getWriterTimezone();
-        if (tz == null || tz.isEmpty()) {
-          tz = UNKNOWN;
-        }
-        System.out.println("  Stripe: " + stripe.toString() + " timezone: " + tz);
-      } else {
-        System.out.println("  Stripe: " + stripe.toString());
-      }
-      long sectionStart = stripeStart;
-      for (OrcProto.Stream section : footer.getStreamsList()) {
-        String kind = section.hasKind() ? section.getKind().name() : UNKNOWN;
-        System.out.println("    Stream: column " + section.getColumn() +
-            " section " + kind + " start: " + sectionStart +
-            " length " + section.getLength());
-        sectionStart += section.getLength();
-      }
-      for (int i = 0; i < footer.getColumnsCount(); ++i) {
-        OrcProto.ColumnEncoding encoding = footer.getColumns(i);
-        StringBuilder buf = new StringBuilder();
-        buf.append("    Encoding column ");
-        buf.append(i);
-        buf.append(": ");
-        buf.append(encoding.getKind());
-        if (encoding.getKind() == OrcProto.ColumnEncoding.Kind.DICTIONARY ||
-            encoding.getKind() == OrcProto.ColumnEncoding.Kind.DICTIONARY_V2) {
-          buf.append("[");
-          buf.append(encoding.getDictionarySize());
-          buf.append("]");
-        }
-        System.out.println(buf);
-      }
-      if (rowIndexCols != null && !rowIndexCols.isEmpty()) {
-        // include the columns that are specified, only if the columns are included, bloom filter
-        // will be read
-        boolean[] sargColumns = new boolean[colCount];
-        for (int colIdx : rowIndexCols) {
-          sargColumns[colIdx] = true;
-        }
-        OrcIndex indices = rows
-            .readRowIndex(stripeIx, null, null, null, sargColumns);
-        for (int col : rowIndexCols) {
-          StringBuilder buf = new StringBuilder();
-          String rowIdxString = getFormattedRowIndices(col, indices.getRowGroupIndex());
-          buf.append(rowIdxString);
-          String bloomFilString = getFormattedBloomFilters(col, indices.getBloomFilterIndex());
-          buf.append(bloomFilString);
-          System.out.println(buf);
-        }
-      }
-    }
-
-    FileSystem fs = file.getFileSystem(conf);
-    long fileLen = fs.getFileStatus(file).getLen();
-    long paddedBytes = getTotalPaddingSize(reader);
-    // empty ORC file is ~45 bytes. Assumption here is file length always >0
-    double percentPadding = ((double) paddedBytes / (double) fileLen) * 100;
-    DecimalFormat format = new DecimalFormat("##.##");
-    System.out.println("\nFile length: " + fileLen + " bytes");
-    System.out.println("Padding length: " + paddedBytes + " bytes");
-    System.out.println("Padding ratio: " + format.format(percentPadding) + "%");
-    OrcRecordUpdater.AcidStats acidStats = OrcRecordUpdater.parseAcidStats(reader);
-    if (acidStats != null) {
-      System.out.println("ACID stats:" + acidStats);
-    }
-    rows.close();
-  }
-
-  private static void recoverFiles(final List<String> corruptFiles, final Configuration conf,
-      final String backup)
-      throws IOException {
-    for (String corruptFile : corruptFiles) {
-      System.err.println("Recovering file " + corruptFile);
-      Path corruptPath = new Path(corruptFile);
-      FileSystem fs = corruptPath.getFileSystem(conf);
-      FSDataInputStream fdis = fs.open(corruptPath);
-      try {
-        long corruptFileLen = fs.getFileStatus(corruptPath).getLen();
-        long remaining = corruptFileLen;
-        List<Long> footerOffsets = Lists.newArrayList();
-
-        // start reading the data file form top to bottom and record the valid footers
-        while (remaining > 0) {
-          int toRead = (int) Math.min(DEFAULT_BLOCK_SIZE, remaining);
-          byte[] data = new byte[toRead];
-          long startPos = corruptFileLen - remaining;
-          fdis.readFully(startPos, data, 0, toRead);
-
-          // find all MAGIC string and see if the file is readable from there
-          int index = 0;
-          long nextFooterOffset;
-
-          while (index != -1) {
-            index = indexOf(data, OrcFile.MAGIC.getBytes(), index + 1);
-            if (index != -1) {
-              nextFooterOffset = startPos + index + OrcFile.MAGIC.length() + 1;
-              if (isReadable(corruptPath, conf, nextFooterOffset)) {
-                footerOffsets.add(nextFooterOffset);
-              }
-            }
-          }
-
-          System.err.println("Scanning for valid footers - startPos: " + startPos +
-              " toRead: " + toRead + " remaining: " + remaining);
-          remaining = remaining - toRead;
-        }
-
-        System.err.println("Readable footerOffsets: " + footerOffsets);
-        recoverFile(corruptPath, fs, conf, footerOffsets, backup);
-      } catch (Exception e) {
-        Path recoveryFile = getRecoveryFile(corruptPath);
-        if (fs.exists(recoveryFile)) {
-          fs.delete(recoveryFile, false);
-        }
-        System.err.println("Unable to recover file " + corruptFile);
-        e.printStackTrace();
-        System.err.println(SEPARATOR);
-        continue;
-      } finally {
-        fdis.close();
-      }
-      System.err.println(corruptFile + " recovered successfully!");
-      System.err.println(SEPARATOR);
-    }
-  }
-
-  private static void recoverFile(final Path corruptPath, final FileSystem fs,
-      final Configuration conf, final List<Long> footerOffsets, final String backup)
-      throws IOException {
-
-    // first recover the file to .recovered file and then once successful rename it to actual file
-    Path recoveredPath = getRecoveryFile(corruptPath);
-
-    // make sure that file does not exist
-    if (fs.exists(recoveredPath)) {
-      fs.delete(recoveredPath, false);
-    }
-
-    // if there are no valid footers, the file should still be readable so create an empty orc file
-    if (footerOffsets == null || footerOffsets.isEmpty()) {
-      System.err.println("No readable footers found. Creating empty orc file.");
-      TypeDescription schema = TypeDescription.createStruct();
-      Writer writer = OrcFile.createWriter(recoveredPath,
-          OrcFile.writerOptions(conf).setSchema(schema));
-      writer.close();
-    } else {
-      FSDataInputStream fdis = fs.open(corruptPath);
-      FileStatus fileStatus = fs.getFileStatus(corruptPath);
-      // read corrupt file and copy it to recovered file until last valid footer
-      FSDataOutputStream fdos = fs.create(recoveredPath, true,
-          conf.getInt("io.file.buffer.size", 4096),
-          fileStatus.getReplication(),
-          fileStatus.getBlockSize());
-      try {
-        long fileLen = footerOffsets.get(footerOffsets.size() - 1);
-        long remaining = fileLen;
-
-        while (remaining > 0) {
-          int toRead = (int) Math.min(DEFAULT_BLOCK_SIZE, remaining);
-          byte[] data = new byte[toRead];
-          long startPos = fileLen - remaining;
-          fdis.readFully(startPos, data, 0, toRead);
-          fdos.write(data);
-          System.err.println("Copying data to recovery file - startPos: " + startPos +
-              " toRead: " + toRead + " remaining: " + remaining);
-          remaining = remaining - toRead;
-        }
-      } catch (Exception e) {
-        fs.delete(recoveredPath, false);
-        throw new IOException(e);
-      } finally {
-        fdis.close();
-        fdos.close();
-      }
-    }
-
-    // validate the recovered file once again and start moving corrupt files to backup folder
-    if (isReadable(recoveredPath, conf, Long.MAX_VALUE)) {
-      Path backupDataPath;
-      String scheme = corruptPath.toUri().getScheme();
-      String authority = corruptPath.toUri().getAuthority();
-      String filePath = corruptPath.toUri().getPath();
-
-      // use the same filesystem as corrupt file if backup-path is not explicitly specified
-      if (backup.equals(DEFAULT_BACKUP_PATH)) {
-        backupDataPath = new Path(scheme, authority, DEFAULT_BACKUP_PATH + filePath);
-      } else {
-        backupDataPath = Path.mergePaths(new Path(backup), corruptPath);
-      }
-
-      // Move data file to backup path
-      moveFiles(fs, corruptPath, backupDataPath);
-
-      // Move side file to backup path
-      Path sideFilePath = OrcRecordUpdater.getSideFile(corruptPath);
-      Path backupSideFilePath = new Path(backupDataPath.getParent(), sideFilePath.getName());
-      moveFiles(fs, sideFilePath, backupSideFilePath);
-
-      // finally move recovered file to actual file
-      moveFiles(fs, recoveredPath, corruptPath);
-
-      // we are done recovering, backing up and validating
-      System.err.println("Validation of recovered file successful!");
-    }
-  }
-
-  private static void moveFiles(final FileSystem fs, final Path src, final Path dest)
-      throws IOException {
-    try {
-      // create the dest directory if not exist
-      if (!fs.exists(dest.getParent())) {
-        fs.mkdirs(dest.getParent());
-      }
-
-      // if the destination file exists for some reason delete it
-      fs.delete(dest, false);
-
-      if (fs.rename(src, dest)) {
-        System.err.println("Moved " + src + " to " + dest);
-      } else {
-        throw new IOException("Unable to move " + src + " to " + dest);
-      }
-
-    } catch (Exception e) {
-      throw new IOException("Unable to move " + src + " to " + dest, e);
-    }
-  }
-
-  private static Path getRecoveryFile(final Path corruptPath) {
-    return new Path(corruptPath.getParent(), corruptPath.getName() + ".recovered");
-  }
-
-  private static boolean isReadable(final Path corruptPath, final Configuration conf,
-      final long maxLen) {
-    try {
-      OrcFile.createReader(corruptPath, OrcFile.readerOptions(conf).maxLength(maxLen));
-      return true;
-    } catch (Exception e) {
-      // ignore this exception as maxLen is unreadable
-      return false;
-    }
-  }
-
-  // search for byte pattern in another byte array
-  private static int indexOf(final byte[] data, final byte[] pattern, final int index) {
-    if (data == null || data.length == 0 || pattern == null || pattern.length == 0 ||
-        index > data.length || index < 0) {
-      return -1;
-    }
-
-    int j = 0;
-    for (int i = index; i < data.length; i++) {
-      if (pattern[j] == data[i]) {
-        j++;
-      } else {
-        j = 0;
-      }
-
-      if (j == pattern.length) {
-        return i - pattern.length + 1;
-      }
-    }
-
-    return -1;
-  }
-
-  private static String getFormattedBloomFilters(int col,
-      OrcProto.BloomFilterIndex[] bloomFilterIndex) {
-    StringBuilder buf = new StringBuilder();
-    BloomFilterIO stripeLevelBF = null;
-    if (bloomFilterIndex != null && bloomFilterIndex[col] != null) {
-      int idx = 0;
-      buf.append("\n    Bloom filters for column ").append(col).append(":");
-      for (OrcProto.BloomFilter bf : bloomFilterIndex[col].getBloomFilterList()) {
-        BloomFilterIO toMerge = new BloomFilterIO(bf);
-        buf.append("\n      Entry ").append(idx++).append(":").append(getBloomFilterStats(toMerge));
-        if (stripeLevelBF == null) {
-          stripeLevelBF = toMerge;
-        } else {
-          stripeLevelBF.merge(toMerge);
-        }
-      }
-      String bloomFilterStats = getBloomFilterStats(stripeLevelBF);
-      buf.append("\n      Stripe level merge:").append(bloomFilterStats);
-    }
-    return buf.toString();
-  }
-
-  private static String getBloomFilterStats(BloomFilterIO bf) {
-    StringBuilder sb = new StringBuilder();
-    int bitCount = bf.getBitSize();
-    int popCount = 0;
-    for (long l : bf.getBitSet()) {
-      popCount += Long.bitCount(l);
-    }
-    int k = bf.getNumHashFunctions();
-    float loadFactor = (float) popCount / (float) bitCount;
-    float expectedFpp = (float) Math.pow(loadFactor, k);
-    DecimalFormat df = new DecimalFormat("###.####");
-    sb.append(" numHashFunctions: ").append(k);
-    sb.append(" bitCount: ").append(bitCount);
-    sb.append(" popCount: ").append(popCount);
-    sb.append(" loadFactor: ").append(df.format(loadFactor));
-    sb.append(" expectedFpp: ").append(expectedFpp);
-    return sb.toString();
-  }
-
-  private static String getFormattedRowIndices(int col,
-                                               OrcProto.RowIndex[] rowGroupIndex) {
-    StringBuilder buf = new StringBuilder();
-    OrcProto.RowIndex index;
-    buf.append("    Row group indices for column ").append(col).append(":");
-    if (rowGroupIndex == null || (col >= rowGroupIndex.length) ||
-        ((index = rowGroupIndex[col]) == null)) {
-      buf.append(" not found\n");
-      return buf.toString();
-    }
-
-    for (int entryIx = 0; entryIx < index.getEntryCount(); ++entryIx) {
-      buf.append("\n      Entry ").append(entryIx).append(": ");
-      OrcProto.RowIndexEntry entry = index.getEntry(entryIx);
-      if (entry == null) {
-        buf.append("unknown\n");
-        continue;
-      }
-      OrcProto.ColumnStatistics colStats = entry.getStatistics();
-      if (colStats == null) {
-        buf.append("no stats at ");
-      } else {
-        ColumnStatistics cs = ColumnStatisticsImpl.deserialize(colStats);
-        buf.append(cs.toString());
-      }
-      buf.append(" positions: ");
-      for (int posIx = 0; posIx < entry.getPositionsCount(); ++posIx) {
-        if (posIx != 0) {
-          buf.append(",");
-        }
-        buf.append(entry.getPositions(posIx));
-      }
-    }
-    return buf.toString();
-  }
-
-  public static long getTotalPaddingSize(Reader reader) throws IOException {
-    long paddedBytes = 0;
-    List<StripeInformation> stripes = reader.getStripes();
-    for (int i = 1; i < stripes.size(); i++) {
-      long prevStripeOffset = stripes.get(i - 1).getOffset();
-      long prevStripeLen = stripes.get(i - 1).getLength();
-      paddedBytes += stripes.get(i).getOffset() - (prevStripeOffset + prevStripeLen);
-    }
-    return paddedBytes;
-  }
-
-  static Options createOptions() {
-    Options result = new Options();
-
-    // add -d and --data to print the rows
-    result.addOption(OptionBuilder
-        .withLongOpt("data")
-        .withDescription("Should the data be printed")
-        .create('d'));
-
-    // to avoid breaking unit tests (when run in different time zones) for file dump, printing
-    // of timezone is made optional
-    result.addOption(OptionBuilder
-        .withLongOpt("timezone")
-        .withDescription("Print writer's time zone")
-        .create('t'));
-
-    result.addOption(OptionBuilder
-        .withLongOpt("help")
-        .withDescription("print help message")
-        .create('h'));
-
-    result.addOption(OptionBuilder
-        .withLongOpt("rowindex")
-        .withArgName("comma separated list of column ids for which row index should be printed")
-        .withDescription("Dump stats for column number(s)")
-        .hasArg()
-        .create('r'));
-
-    result.addOption(OptionBuilder
-        .withLongOpt("json")
-        .withDescription("Print metadata in JSON format")
-        .create('j'));
-
-    result.addOption(OptionBuilder
-        .withLongOpt("pretty")
-        .withDescription("Pretty print json metadata output")
-        .create('p'));
-
-    result.addOption(OptionBuilder
-        .withLongOpt("recover")
-        .withDescription("recover corrupted orc files generated by streaming")
-        .create());
-
-    result.addOption(OptionBuilder
-        .withLongOpt("skip-dump")
-        .withDescription("used along with --recover to directly recover files without dumping")
-        .create());
-
-    result.addOption(OptionBuilder
-        .withLongOpt("backup-path")
-        .withDescription("specify a backup path to store the corrupted files (default: /tmp)")
-        .hasArg()
-        .create());
-    return result;
-  }
-
-  private static void printMap(JSONWriter writer,
-      Map<Object, Object> obj,
-      List<OrcProto.Type> types,
-      OrcProto.Type type
-  ) throws IOException, JSONException {
-    writer.array();
-    int keyType = type.getSubtypes(0);
-    int valueType = type.getSubtypes(1);
-    for (Map.Entry<Object, Object> item : obj.entrySet()) {
-      writer.object();
-      writer.key("_key");
-      printObject(writer, item.getKey(), types, keyType);
-      writer.key("_value");
-      printObject(writer, item.getValue(), types, valueType);
-      writer.endObject();
-    }
-    writer.endArray();
-  }
-
-  private static void printList(JSONWriter writer,
-      List<Object> obj,
-      List<OrcProto.Type> types,
-      OrcProto.Type type
-  ) throws IOException, JSONException {
-    int subtype = type.getSubtypes(0);
-    writer.array();
-    for (Object item : obj) {
-      printObject(writer, item, types, subtype);
-    }
-    writer.endArray();
-  }
-
-  private static void printUnion(JSONWriter writer,
-      OrcUnion obj,
-      List<OrcProto.Type> types,
-      OrcProto.Type type
-  ) throws IOException, JSONException {
-    int subtype = type.getSubtypes(obj.getTag());
-    printObject(writer, obj.getObject(), types, subtype);
-  }
-
-  static void printStruct(JSONWriter writer,
-      OrcStruct obj,
-      List<OrcProto.Type> types,
-      OrcProto.Type type) throws IOException, JSONException {
-    writer.object();
-    List<Integer> fieldTypes = type.getSubtypesList();
-    for (int i = 0; i < fieldTypes.size(); ++i) {
-      writer.key(type.getFieldNames(i));
-      printObject(writer, obj.getFieldValue(i), types, fieldTypes.get(i));
-    }
-    writer.endObject();
-  }
-
-  static void printObject(JSONWriter writer,
-      Object obj,
-      List<OrcProto.Type> types,
-      int typeId) throws IOException, JSONException {
-    OrcProto.Type type = types.get(typeId);
-    if (obj == null) {
-      writer.value(null);
-    } else {
-      switch (type.getKind()) {
-        case STRUCT:
-          printStruct(writer, (OrcStruct) obj, types, type);
-          break;
-        case UNION:
-          printUnion(writer, (OrcUnion) obj, types, type);
-          break;
-        case LIST:
-          printList(writer, (List<Object>) obj, types, type);
-          break;
-        case MAP:
-          printMap(writer, (Map<Object, Object>) obj, types, type);
-          break;
-        case BYTE:
-          writer.value(((ByteWritable) obj).get());
-          break;
-        case SHORT:
-          writer.value(((ShortWritable) obj).get());
-          break;
-        case INT:
-          writer.value(((IntWritable) obj).get());
-          break;
-        case LONG:
-          writer.value(((LongWritable) obj).get());
-          break;
-        case FLOAT:
-          writer.value(((FloatWritable) obj).get());
-          break;
-        case DOUBLE:
-          writer.value(((DoubleWritable) obj).get());
-          break;
-        case BOOLEAN:
-          writer.value(((BooleanWritable) obj).get());
-          break;
-        default:
-          writer.value(obj.toString());
-          break;
-      }
-    }
-  }
-
-  static void printJsonData(final Reader reader) throws IOException, JSONException {
-    PrintStream printStream = System.out;
-    OutputStreamWriter out = new OutputStreamWriter(printStream, "UTF-8");
-    RecordReader rows = reader.rows(null);
-    Object row = null;
-    try {
-      List<OrcProto.Type> types = reader.getTypes();
-      while (rows.hasNext()) {
-        row = rows.next(row);
-        JSONWriter writer = new JSONWriter(out);
-        printObject(writer, row, types, 0);
-        out.write("\n");
-        out.flush();
-        if (printStream.checkError()) {
-          throw new IOException("Error encountered when writing to stdout.");
-        }
-      }
-    } finally {
-      rows.close();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/ql/src/java/org/apache/hadoop/hive/ql/io/orc/JsonFileDump.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/JsonFileDump.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/JsonFileDump.java
deleted file mode 100644
index 00de545..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/JsonFileDump.java
+++ /dev/null
@@ -1,401 +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
- * <p/>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p/>
- * 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.hive.ql.io.orc;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.Set;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.codehaus.jettison.json.JSONArray;
-import org.apache.orc.BloomFilterIO;
-import org.apache.orc.BinaryColumnStatistics;
-import org.apache.orc.BooleanColumnStatistics;
-import org.apache.orc.ColumnStatistics;
-import org.apache.orc.impl.ColumnStatisticsImpl;
-import org.apache.orc.DateColumnStatistics;
-import org.apache.orc.DecimalColumnStatistics;
-import org.apache.orc.DoubleColumnStatistics;
-import org.apache.orc.IntegerColumnStatistics;
-import org.apache.orc.impl.OrcIndex;
-import org.apache.orc.OrcProto;
-import org.apache.orc.StringColumnStatistics;
-import org.apache.orc.StripeInformation;
-import org.apache.orc.StripeStatistics;
-import org.apache.orc.TimestampColumnStatistics;
-import org.codehaus.jettison.json.JSONException;
-import org.codehaus.jettison.json.JSONObject;
-import org.codehaus.jettison.json.JSONStringer;
-import org.codehaus.jettison.json.JSONWriter;
-
-/**
- * File dump tool with json formatted output.
- */
-public class JsonFileDump {
-
-  public static void printJsonMetaData(List<String> files,
-      Configuration conf,
-      List<Integer> rowIndexCols, boolean prettyPrint, boolean printTimeZone)
-      throws JSONException, IOException {
-    if (files.isEmpty()) {
-      return;
-    }
-    JSONStringer writer = new JSONStringer();
-    boolean multiFile = files.size() > 1;
-    if (multiFile) {
-      writer.array();
-    } else {
-      writer.object();
-    }
-    for (String filename : files) {
-      try {
-        if (multiFile) {
-          writer.object();
-        }
-        writer.key("fileName").value(filename);
-        Path path = new Path(filename);
-        Reader reader = FileDump.getReader(path, conf, null);
-        if (reader == null) {
-          writer.key("status").value("FAILED");
-          continue;
-        }
-        writer.key("fileVersion").value(reader.getFileVersion().getName());
-        writer.key("writerVersion").value(reader.getWriterVersion());
-        RecordReaderImpl rows = (RecordReaderImpl) reader.rows();
-        writer.key("numberOfRows").value(reader.getNumberOfRows());
-        writer.key("compression").value(reader.getCompression());
-        if (reader.getCompression() != CompressionKind.NONE) {
-          writer.key("compressionBufferSize").value(reader.getCompressionSize());
-        }
-        writer.key("schemaString").value(reader.getObjectInspector().getTypeName());
-        writer.key("schema").array();
-        writeSchema(writer, reader.getTypes());
-        writer.endArray();
-
-        writer.key("stripeStatistics").array();
-        List<StripeStatistics> stripeStatistics = reader.getStripeStatistics();
-        for (int n = 0; n < stripeStatistics.size(); n++) {
-          writer.object();
-          writer.key("stripeNumber").value(n + 1);
-          StripeStatistics ss = stripeStatistics.get(n);
-          writer.key("columnStatistics").array();
-          for (int i = 0; i < ss.getColumnStatistics().length; i++) {
-            writer.object();
-            writer.key("columnId").value(i);
-            writeColumnStatistics(writer, ss.getColumnStatistics()[i]);
-            writer.endObject();
-          }
-          writer.endArray();
-          writer.endObject();
-        }
-        writer.endArray();
-
-        ColumnStatistics[] stats = reader.getStatistics();
-        int colCount = stats.length;
-        writer.key("fileStatistics").array();
-        for (int i = 0; i < stats.length; ++i) {
-          writer.object();
-          writer.key("columnId").value(i);
-          writeColumnStatistics(writer, stats[i]);
-          writer.endObject();
-        }
-        writer.endArray();
-
-        writer.key("stripes").array();
-        int stripeIx = -1;
-        for (StripeInformation stripe : reader.getStripes()) {
-          ++stripeIx;
-          long stripeStart = stripe.getOffset();
-          OrcProto.StripeFooter footer = rows.readStripeFooter(stripe);
-          writer.object(); // start of stripe information
-          writer.key("stripeNumber").value(stripeIx + 1);
-          writer.key("stripeInformation");
-          writeStripeInformation(writer, stripe);
-          if (printTimeZone) {
-            writer.key("writerTimezone").value(
-                footer.hasWriterTimezone() ? footer.getWriterTimezone() : FileDump.UNKNOWN);
-          }
-          long sectionStart = stripeStart;
-
-          writer.key("streams").array();
-          for (OrcProto.Stream section : footer.getStreamsList()) {
-            writer.object();
-            String kind = section.hasKind() ? section.getKind().name() : FileDump.UNKNOWN;
-            writer.key("columnId").value(section.getColumn());
-            writer.key("section").value(kind);
-            writer.key("startOffset").value(sectionStart);
-            writer.key("length").value(section.getLength());
-            sectionStart += section.getLength();
-            writer.endObject();
-          }
-          writer.endArray();
-
-          writer.key("encodings").array();
-          for (int i = 0; i < footer.getColumnsCount(); ++i) {
-            writer.object();
-            OrcProto.ColumnEncoding encoding = footer.getColumns(i);
-            writer.key("columnId").value(i);
-            writer.key("kind").value(encoding.getKind());
-            if (encoding.getKind() == OrcProto.ColumnEncoding.Kind.DICTIONARY ||
-                encoding.getKind() == OrcProto.ColumnEncoding.Kind.DICTIONARY_V2) {
-              writer.key("dictionarySize").value(encoding.getDictionarySize());
-            }
-            writer.endObject();
-          }
-          writer.endArray();
-
-          if (rowIndexCols != null && !rowIndexCols.isEmpty()) {
-            // include the columns that are specified, only if the columns are included, bloom filter
-            // will be read
-            boolean[] sargColumns = new boolean[colCount];
-            for (int colIdx : rowIndexCols) {
-              sargColumns[colIdx] = true;
-            }
-            OrcIndex indices = rows.readRowIndex(stripeIx, null, sargColumns);
-            writer.key("indexes").array();
-            for (int col : rowIndexCols) {
-              writer.object();
-              writer.key("columnId").value(col);
-              writeRowGroupIndexes(writer, col, indices.getRowGroupIndex());
-              writeBloomFilterIndexes(writer, col, indices.getBloomFilterIndex());
-              writer.endObject();
-            }
-            writer.endArray();
-          }
-          writer.endObject(); // end of stripe information
-        }
-        writer.endArray();
-
-        FileSystem fs = path.getFileSystem(conf);
-        long fileLen = fs.getContentSummary(path).getLength();
-        long paddedBytes = FileDump.getTotalPaddingSize(reader);
-        // empty ORC file is ~45 bytes. Assumption here is file length always >0
-        double percentPadding = ((double) paddedBytes / (double) fileLen) * 100;
-        writer.key("fileLength").value(fileLen);
-        writer.key("paddingLength").value(paddedBytes);
-        writer.key("paddingRatio").value(percentPadding);
-        OrcRecordUpdater.AcidStats acidStats = OrcRecordUpdater.parseAcidStats(reader);
-        if (acidStats != null) {
-          writer.key("numInserts").value(acidStats.inserts);
-          writer.key("numDeletes").value(acidStats.deletes);
-          writer.key("numUpdates").value(acidStats.updates);
-        }
-        writer.key("status").value("OK");
-        rows.close();
-
-        writer.endObject();
-      } catch (Exception e) {
-        writer.key("status").value("FAILED");
-        throw e;
-      }
-    }
-    if (multiFile) {
-      writer.endArray();
-    }
-
-    if (prettyPrint) {
-      final String prettyJson;
-      if (multiFile) {
-        JSONArray jsonArray = new JSONArray(writer.toString());
-        prettyJson = jsonArray.toString(2);
-      } else {
-        JSONObject jsonObject = new JSONObject(writer.toString());
-        prettyJson = jsonObject.toString(2);
-      }
-      System.out.println(prettyJson);
-    } else {
-      System.out.println(writer.toString());
-    }
-  }
-
-  private static void writeSchema(JSONStringer writer, List<OrcProto.Type> types)
-      throws JSONException {
-    int i = 0;
-    for(OrcProto.Type type : types) {
-      writer.object();
-      writer.key("columnId").value(i++);
-      writer.key("columnType").value(type.getKind());
-      if (type.getFieldNamesCount() > 0) {
-        writer.key("childColumnNames").array();
-        for (String field : type.getFieldNamesList()) {
-          writer.value(field);
-        }
-        writer.endArray();
-        writer.key("childColumnIds").array();
-        for (Integer colId : type.getSubtypesList()) {
-          writer.value(colId);
-        }
-        writer.endArray();
-      }
-      if (type.hasPrecision()) {
-        writer.key("precision").value(type.getPrecision());
-      }
-
-      if (type.hasScale()) {
-        writer.key("scale").value(type.getScale());
-      }
-
-      if (type.hasMaximumLength()) {
-        writer.key("maxLength").value(type.getMaximumLength());
-      }
-      writer.endObject();
-    }
-  }
-
-  private static void writeStripeInformation(JSONWriter writer, StripeInformation stripe)
-      throws JSONException {
-    writer.object();
-    writer.key("offset").value(stripe.getOffset());
-    writer.key("indexLength").value(stripe.getIndexLength());
-    writer.key("dataLength").value(stripe.getDataLength());
-    writer.key("footerLength").value(stripe.getFooterLength());
-    writer.key("rowCount").value(stripe.getNumberOfRows());
-    writer.endObject();
-  }
-
-  private static void writeColumnStatistics(JSONWriter writer, ColumnStatistics cs)
-      throws JSONException {
-    if (cs != null) {
-      writer.key("count").value(cs.getNumberOfValues());
-      writer.key("hasNull").value(cs.hasNull());
-      if (cs instanceof BinaryColumnStatistics) {
-        writer.key("totalLength").value(((BinaryColumnStatistics) cs).getSum());
-        writer.key("type").value(OrcProto.Type.Kind.BINARY);
-      } else if (cs instanceof BooleanColumnStatistics) {
-        writer.key("trueCount").value(((BooleanColumnStatistics) cs).getTrueCount());
-        writer.key("falseCount").value(((BooleanColumnStatistics) cs).getFalseCount());
-        writer.key("type").value(OrcProto.Type.Kind.BOOLEAN);
-      } else if (cs instanceof IntegerColumnStatistics) {
-        writer.key("min").value(((IntegerColumnStatistics) cs).getMinimum());
-        writer.key("max").value(((IntegerColumnStatistics) cs).getMaximum());
-        if (((IntegerColumnStatistics) cs).isSumDefined()) {
-          writer.key("sum").value(((IntegerColumnStatistics) cs).getSum());
-        }
-        writer.key("type").value(OrcProto.Type.Kind.LONG);
-      } else if (cs instanceof DoubleColumnStatistics) {
-        writer.key("min").value(((DoubleColumnStatistics) cs).getMinimum());
-        writer.key("max").value(((DoubleColumnStatistics) cs).getMaximum());
-        writer.key("sum").value(((DoubleColumnStatistics) cs).getSum());
-        writer.key("type").value(OrcProto.Type.Kind.DOUBLE);
-      } else if (cs instanceof StringColumnStatistics) {
-        writer.key("min").value(((StringColumnStatistics) cs).getMinimum());
-        writer.key("max").value(((StringColumnStatistics) cs).getMaximum());
-        writer.key("totalLength").value(((StringColumnStatistics) cs).getSum());
-        writer.key("type").value(OrcProto.Type.Kind.STRING);
-      } else if (cs instanceof DateColumnStatistics) {
-        if (((DateColumnStatistics) cs).getMaximum() != null) {
-          writer.key("min").value(((DateColumnStatistics) cs).getMinimum());
-          writer.key("max").value(((DateColumnStatistics) cs).getMaximum());
-        }
-        writer.key("type").value(OrcProto.Type.Kind.DATE);
-      } else if (cs instanceof TimestampColumnStatistics) {
-        if (((TimestampColumnStatistics) cs).getMaximum() != null) {
-          writer.key("min").value(((TimestampColumnStatistics) cs).getMinimum());
-          writer.key("max").value(((TimestampColumnStatistics) cs).getMaximum());
-        }
-        writer.key("type").value(OrcProto.Type.Kind.TIMESTAMP);
-      } else if (cs instanceof DecimalColumnStatistics) {
-        if (((DecimalColumnStatistics) cs).getMaximum() != null) {
-          writer.key("min").value(((DecimalColumnStatistics) cs).getMinimum());
-          writer.key("max").value(((DecimalColumnStatistics) cs).getMaximum());
-          writer.key("sum").value(((DecimalColumnStatistics) cs).getSum());
-        }
-        writer.key("type").value(OrcProto.Type.Kind.DECIMAL);
-      }
-    }
-  }
-
-  private static void writeBloomFilterIndexes(JSONWriter writer, int col,
-      OrcProto.BloomFilterIndex[] bloomFilterIndex) throws JSONException {
-
-    BloomFilterIO stripeLevelBF = null;
-    if (bloomFilterIndex != null && bloomFilterIndex[col] != null) {
-      int entryIx = 0;
-      writer.key("bloomFilterIndexes").array();
-      for (OrcProto.BloomFilter bf : bloomFilterIndex[col].getBloomFilterList()) {
-        writer.object();
-        writer.key("entryId").value(entryIx++);
-        BloomFilterIO toMerge = new BloomFilterIO(bf);
-        writeBloomFilterStats(writer, toMerge);
-        if (stripeLevelBF == null) {
-          stripeLevelBF = toMerge;
-        } else {
-          stripeLevelBF.merge(toMerge);
-        }
-        writer.endObject();
-      }
-      writer.endArray();
-    }
-    if (stripeLevelBF != null) {
-      writer.key("stripeLevelBloomFilter");
-      writer.object();
-      writeBloomFilterStats(writer, stripeLevelBF);
-      writer.endObject();
-    }
-  }
-
-  private static void writeBloomFilterStats(JSONWriter writer, BloomFilterIO bf)
-      throws JSONException {
-    int bitCount = bf.getBitSize();
-    int popCount = 0;
-    for (long l : bf.getBitSet()) {
-      popCount += Long.bitCount(l);
-    }
-    int k = bf.getNumHashFunctions();
-    float loadFactor = (float) popCount / (float) bitCount;
-    float expectedFpp = (float) Math.pow(loadFactor, k);
-    writer.key("numHashFunctions").value(k);
-    writer.key("bitCount").value(bitCount);
-    writer.key("popCount").value(popCount);
-    writer.key("loadFactor").value(loadFactor);
-    writer.key("expectedFpp").value(expectedFpp);
-  }
-
-  private static void writeRowGroupIndexes(JSONWriter writer, int col,
-      OrcProto.RowIndex[] rowGroupIndex)
-      throws JSONException {
-
-    OrcProto.RowIndex index;
-    if (rowGroupIndex == null || (col >= rowGroupIndex.length) ||
-        ((index = rowGroupIndex[col]) == null)) {
-      return;
-    }
-
-    writer.key("rowGroupIndexes").array();
-    for (int entryIx = 0; entryIx < index.getEntryCount(); ++entryIx) {
-      writer.object();
-      writer.key("entryId").value(entryIx);
-      OrcProto.RowIndexEntry entry = index.getEntry(entryIx);
-      if (entry == null) {
-        continue;
-      }
-      OrcProto.ColumnStatistics colStats = entry.getStatistics();
-      writeColumnStatistics(writer, ColumnStatisticsImpl.deserialize(colStats));
-      writer.key("positions").array();
-      for (int posIx = 0; posIx < entry.getPositionsCount(); ++posIx) {
-        writer.value(entry.getPositions(posIx));
-      }
-      writer.endArray();
-      writer.endObject();
-    }
-    writer.endArray();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcRawRecordMerger.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcRawRecordMerger.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcRawRecordMerger.java
index 0dd58b7..b9094bf 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcRawRecordMerger.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcRawRecordMerger.java
@@ -18,10 +18,7 @@
 package org.apache.hadoop.hive.ql.io.orc;
 
 import java.io.IOException;
-import java.util.ArrayDeque;
-import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Deque;
 import java.util.List;
 import java.util.Map;
 import java.util.TreeMap;
@@ -29,22 +26,20 @@ import java.util.TreeMap;
 import org.apache.orc.OrcUtils;
 import org.apache.orc.StripeInformation;
 import org.apache.orc.TypeDescription;
+import org.apache.orc.impl.AcidStats;
+import org.apache.orc.impl.OrcAcidUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.common.ValidTxnList;
-import org.apache.hadoop.hive.ql.ErrorMsg;
 import org.apache.hadoop.hive.ql.io.AcidInputFormat;
 import org.apache.hadoop.hive.ql.io.AcidUtils;
 import org.apache.hadoop.hive.ql.io.RecordIdentifier;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.LongWritable;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 import com.google.common.annotations.VisibleForTesting;
 
@@ -494,7 +489,7 @@ public class OrcRawRecordMerger implements AcidInputFormat.RawReader<OrcStruct>{
         Path deltaFile = AcidUtils.createBucketFile(delta, bucket);
         AcidUtils.ParsedDelta deltaDir = AcidUtils.parsedDelta(delta);
         FileSystem fs = deltaFile.getFileSystem(conf);
-        long length = getLastFlushLength(fs, deltaFile);
+        long length = OrcAcidUtils.getLastFlushLength(fs, deltaFile);
         if (length != -1 && fs.exists(deltaFile)) {
           Reader deltaReader = OrcFile.createReader(deltaFile,
               OrcFile.readerOptions(conf).maxLength(length));
@@ -504,7 +499,7 @@ public class OrcRawRecordMerger implements AcidInputFormat.RawReader<OrcStruct>{
             // it can produce wrong results (if the latest valid version of the record is filtered out by
             // the sarg) or ArrayOutOfBounds errors (when the sarg is applied to a delete record)
             // unless the delta only has insert events
-            OrcRecordUpdater.AcidStats acidStats = OrcRecordUpdater.parseAcidStats(deltaReader);
+            AcidStats acidStats = OrcAcidUtils.parseAcidStats(deltaReader);
             if(acidStats.deletes > 0 || acidStats.updates > 0) {
               deltaEventOptions = eventOptions.clone().searchArgument(null, null);
             }
@@ -536,28 +531,6 @@ public class OrcRawRecordMerger implements AcidInputFormat.RawReader<OrcStruct>{
     }
   }
 
-  /**
-   * Read the side file to get the last flush length.
-   * @param fs the file system to use
-   * @param deltaFile the path of the delta file
-   * @return the maximum size of the file to use
-   * @throws IOException
-   */
-  static long getLastFlushLength(FileSystem fs,
-                                         Path deltaFile) throws IOException {
-    Path lengths = OrcRecordUpdater.getSideFile(deltaFile);
-    long result = Long.MAX_VALUE;
-    try (FSDataInputStream stream = fs.open(lengths)) {
-      result = -1;
-      while (stream.available() > 0) {
-        result = stream.readLong();
-      }
-      return result;
-    } catch (IOException ioe) {
-      return result;
-    }
-  }
-
   @VisibleForTesting
   RecordIdentifier getMinKey() {
     return minKey;

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcRecordUpdater.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcRecordUpdater.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcRecordUpdater.java
index d085c58..4bf2403 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcRecordUpdater.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcRecordUpdater.java
@@ -25,6 +25,8 @@ import java.nio.charset.CharsetDecoder;
 import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.orc.impl.AcidStats;
+import org.apache.orc.impl.OrcAcidUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -55,7 +57,6 @@ public class OrcRecordUpdater implements RecordUpdater {
 
   public static final String ACID_KEY_INDEX_NAME = "hive.acid.key.index";
   public static final String ACID_FORMAT = "_orc_acid_version";
-  public static final String ACID_STATS = "hive.acid.stats";
   public static final int ORC_ACID_VERSION = 0;
 
 
@@ -102,46 +103,6 @@ public class OrcRecordUpdater implements RecordUpdater {
   private LongObjectInspector origTxnInspector; // OI for the original txn inside the record
   // identifer
 
-  static class AcidStats {
-    long inserts;
-    long updates;
-    long deletes;
-
-    AcidStats() {
-      // nothing
-    }
-
-    AcidStats(String serialized) {
-      String[] parts = serialized.split(",");
-      inserts = Long.parseLong(parts[0]);
-      updates = Long.parseLong(parts[1]);
-      deletes = Long.parseLong(parts[2]);
-    }
-
-    String serialize() {
-      StringBuilder builder = new StringBuilder();
-      builder.append(inserts);
-      builder.append(",");
-      builder.append(updates);
-      builder.append(",");
-      builder.append(deletes);
-      return builder.toString();
-    }
-
-    @Override
-    public String toString() {
-      StringBuilder builder = new StringBuilder();
-      builder.append(" inserts: ").append(inserts);
-      builder.append(" updates: ").append(updates);
-      builder.append(" deletes: ").append(deletes);
-      return builder.toString();
-    }
-  }
-
-  public static Path getSideFile(Path main) {
-    return new Path(main + AcidUtils.DELTA_SIDE_FILE_SUFFIX);
-  }
-
   static int getOperation(OrcStruct struct) {
     return ((IntWritable) struct.getFieldValue(OPERATION)).get();
   }
@@ -237,7 +198,7 @@ public class OrcRecordUpdater implements RecordUpdater {
     }
     if (options.getMinimumTransactionId() != options.getMaximumTransactionId()
         && !options.isWritingBase()){
-      flushLengths = fs.create(getSideFile(this.path), true, 8,
+      flushLengths = fs.create(OrcAcidUtils.getSideFile(this.path), true, 8,
           options.getReporter());
     } else {
       flushLengths = null;
@@ -297,7 +258,7 @@ public class OrcRecordUpdater implements RecordUpdater {
       }
       Reader reader = OrcFile.createReader(matchingBucket, OrcFile.readerOptions(options.getConfiguration()));
       //no close() on Reader?!
-      AcidStats acidStats = parseAcidStats(reader);
+      AcidStats acidStats = OrcAcidUtils.parseAcidStats(reader);
       if(acidStats.inserts > 0) {
         return acidStats.inserts;
       }
@@ -412,7 +373,7 @@ public class OrcRecordUpdater implements RecordUpdater {
     }
     if (flushLengths != null) {
       flushLengths.close();
-      fs.delete(getSideFile(path), false);
+      fs.delete(OrcAcidUtils.getSideFile(path), false);
     }
     writer = null;
   }
@@ -456,26 +417,6 @@ public class OrcRecordUpdater implements RecordUpdater {
     }
     return result;
   }
-  /**
-   * {@link KeyIndexBuilder} creates these
-   */
-  static AcidStats parseAcidStats(Reader reader) {
-    if (reader.hasMetadataValue(OrcRecordUpdater.ACID_STATS)) {
-      String statsSerialized;
-      try {
-        ByteBuffer val =
-            reader.getMetadataValue(OrcRecordUpdater.ACID_STATS)
-                .duplicate();
-        statsSerialized = utf8Decoder.decode(val).toString();
-      } catch (CharacterCodingException e) {
-        throw new IllegalArgumentException("Bad string encoding for " +
-            OrcRecordUpdater.ACID_STATS, e);
-      }
-      return new AcidStats(statsSerialized);
-    } else {
-      return null;
-    }
-  }
 
   static class KeyIndexBuilder implements OrcFile.WriterCallback {
     StringBuilder lastKey = new StringBuilder();
@@ -500,7 +441,7 @@ public class OrcRecordUpdater implements RecordUpdater {
                                ) throws IOException {
       context.getWriter().addUserMetadata(ACID_KEY_INDEX_NAME,
           UTF8.encode(lastKey.toString()));
-      context.getWriter().addUserMetadata(ACID_STATS,
+      context.getWriter().addUserMetadata(OrcAcidUtils.ACID_STATS,
           UTF8.encode(acidStats.serialize()));
     }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/ql/src/java/org/apache/hadoop/hive/ql/io/orc/ReaderImpl.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/ReaderImpl.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/ReaderImpl.java
index b7437be..3a2e7d8 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/ReaderImpl.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/ReaderImpl.java
@@ -22,17 +22,9 @@ import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashSet;
 import java.util.List;
-import java.util.Set;
 
-import com.google.common.collect.Lists;
-import org.apache.orc.OrcUtils;
-import org.apache.orc.TypeDescription;
 import org.apache.orc.impl.BufferChunk;
-import org.apache.orc.ColumnStatistics;
-import org.apache.orc.impl.ColumnStatisticsImpl;
 import org.apache.orc.CompressionCodec;
 import org.apache.orc.FileMetaInfo;
 import org.apache.orc.FileMetadata;
@@ -41,47 +33,25 @@ import org.apache.orc.StripeInformation;
 import org.apache.orc.StripeStatistics;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.common.io.DiskRange;
-import org.apache.hadoop.hive.ql.io.FileFormatException;
 import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
-import org.apache.hadoop.hive.ql.util.JavaDataModel;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
-import org.apache.hadoop.io.Text;
 import org.apache.orc.OrcProto;
 
+import com.google.common.collect.Lists;
 import com.google.protobuf.CodedInputStream;
 
-public class ReaderImpl implements Reader {
+public class ReaderImpl extends org.apache.orc.impl.ReaderImpl
+                        implements Reader {
 
   private static final Logger LOG = LoggerFactory.getLogger(ReaderImpl.class);
 
   private static final int DIRECTORY_SIZE_GUESS = 16 * 1024;
 
-  protected final FileSystem fileSystem;
-  private final long maxLength;
-  protected final Path path;
-  protected final org.apache.orc.CompressionKind compressionKind;
-  protected final CompressionCodec codec;
-  protected final int bufferSize;
-  private final List<OrcProto.StripeStatistics> stripeStats;
-  private final int metadataSize;
-  protected final List<OrcProto.Type> types;
-  private final TypeDescription schema;
-  private final List<OrcProto.UserMetadataItem> userMetadata;
-  private final List<OrcProto.ColumnStatistics> fileStats;
-  private final List<StripeInformation> stripes;
-  protected final int rowIndexStride;
-  private final long contentLength, numberOfRows;
-
   private final ObjectInspector inspector;
-  private long deserializedSize = -1;
-  protected final Configuration conf;
-  private final List<Integer> versionList;
-  private final OrcFile.WriterVersion writerVersion;
 
   //serialized footer - Keeping this around for use by getFileMetaInfo()
   // will help avoid cpu cycles spend in deserializing at cost of increased
@@ -91,83 +61,9 @@ public class ReaderImpl implements Reader {
   // This will only be set if the file footer/metadata was read from disk.
   private final ByteBuffer footerMetaAndPsBuffer;
 
-  public static class StripeInformationImpl
-      implements StripeInformation {
-    private final OrcProto.StripeInformation stripe;
-
-    public StripeInformationImpl(OrcProto.StripeInformation stripe) {
-      this.stripe = stripe;
-    }
-
-    @Override
-    public long getOffset() {
-      return stripe.getOffset();
-    }
-
-    @Override
-    public long getLength() {
-      return stripe.getDataLength() + getIndexLength() + getFooterLength();
-    }
-
-    @Override
-    public long getDataLength() {
-      return stripe.getDataLength();
-    }
-
-    @Override
-    public long getFooterLength() {
-      return stripe.getFooterLength();
-    }
-
-    @Override
-    public long getIndexLength() {
-      return stripe.getIndexLength();
-    }
-
-    @Override
-    public long getNumberOfRows() {
-      return stripe.getNumberOfRows();
-    }
-
-    @Override
-    public String toString() {
-      return "offset: " + getOffset() + " data: " + getDataLength() +
-        " rows: " + getNumberOfRows() + " tail: " + getFooterLength() +
-        " index: " + getIndexLength();
-    }
-  }
-
   @Override
-  public long getNumberOfRows() {
-    return numberOfRows;
-  }
-
-  @Override
-  public List<String> getMetadataKeys() {
-    List<String> result = new ArrayList<String>();
-    for(OrcProto.UserMetadataItem item: userMetadata) {
-      result.add(item.getName());
-    }
-    return result;
-  }
-
-  @Override
-  public ByteBuffer getMetadataValue(String key) {
-    for(OrcProto.UserMetadataItem item: userMetadata) {
-      if (item.hasName() && item.getName().equals(key)) {
-        return item.getValue().asReadOnlyByteBuffer();
-      }
-    }
-    throw new IllegalArgumentException("Can't find user metadata " + key);
-  }
-
-  public boolean hasMetadataValue(String key) {
-    for(OrcProto.UserMetadataItem item: userMetadata) {
-      if (item.hasName() && item.getName().equals(key)) {
-        return true;
-      }
-    }
-    return false;
+  public ObjectInspector getObjectInspector() {
+    return inspector;
   }
 
   @Override
@@ -181,181 +77,19 @@ public class ReaderImpl implements Reader {
         compressionKind);
   }
 
-  @Override
-  public org.apache.orc.CompressionKind getCompressionKind() {
-    return compressionKind;
-  }
-
-  @Override
-  public int getCompressionSize() {
-    return bufferSize;
-  }
-
-  @Override
-  public List<StripeInformation> getStripes() {
-    return stripes;
-  }
-
-  @Override
-  public ObjectInspector getObjectInspector() {
-    return inspector;
-  }
-
-  @Override
-  public long getContentLength() {
-    return contentLength;
-  }
-
-  @Override
-  public List<OrcProto.Type> getTypes() {
-    return types;
-  }
-
-  @Override
-  public OrcFile.Version getFileVersion() {
-    for (OrcFile.Version version: OrcFile.Version.values()) {
-      if ((versionList != null && !versionList.isEmpty()) &&
-          version.getMajor() == versionList.get(0) &&
-          version.getMinor() == versionList.get(1)) {
-        return version;
-      }
-    }
-    return OrcFile.Version.V_0_11;
-  }
-
-  @Override
-  public OrcFile.WriterVersion getWriterVersion() {
-    return writerVersion;
-  }
-
-  @Override
-  public int getRowIndexStride() {
-    return rowIndexStride;
-  }
-
-  @Override
-  public ColumnStatistics[] getStatistics() {
-    ColumnStatistics[] result = new ColumnStatistics[types.size()];
-    for(int i=0; i < result.length; ++i) {
-      result[i] = ColumnStatisticsImpl.deserialize(fileStats.get(i));
-    }
-    return result;
-  }
-
-  @Override
-  public TypeDescription getSchema() {
-    return schema;
-  }
-
-  /**
-   * Ensure this is an ORC file to prevent users from trying to read text
-   * files or RC files as ORC files.
-   * @param in the file being read
-   * @param path the filename for error messages
-   * @param psLen the postscript length
-   * @param buffer the tail of the file
-   * @throws IOException
-   */
-  static void ensureOrcFooter(FSDataInputStream in,
-                                      Path path,
-                                      int psLen,
-                                      ByteBuffer buffer) throws IOException {
-    int magicLength = OrcFile.MAGIC.length();
-    int fullLength = magicLength + 1;
-    if (psLen < fullLength || buffer.remaining() < fullLength) {
-      throw new FileFormatException("Malformed ORC file " + path +
-          ". Invalid postscript length " + psLen);
-    }
-    int offset = buffer.arrayOffset() + buffer.position() + buffer.limit() - fullLength;
-    byte[] array = buffer.array();
-    // now look for the magic string at the end of the postscript.
-    if (!Text.decode(array, offset, magicLength).equals(OrcFile.MAGIC)) {
-      // If it isn't there, this may be the 0.11.0 version of ORC.
-      // Read the first 3 bytes of the file to check for the header
-      byte[] header = new byte[magicLength];
-      in.readFully(0, header, 0, magicLength);
-      // if it isn't there, this isn't an ORC file
-      if (!Text.decode(header, 0 , magicLength).equals(OrcFile.MAGIC)) {
-        throw new FileFormatException("Malformed ORC file " + path +
-            ". Invalid postscript.");
-      }
-    }
-  }
-
-  /**
-   * Build a version string out of an array.
-   * @param version the version number as a list
-   * @return the human readable form of the version string
-   */
-  private static String versionString(List<Integer> version) {
-    StringBuilder buffer = new StringBuilder();
-    for(int i=0; i < version.size(); ++i) {
-      if (i != 0) {
-        buffer.append('.');
-      }
-      buffer.append(version.get(i));
-    }
-    return buffer.toString();
-  }
-
-  /**
-   * Check to see if this ORC file is from a future version and if so,
-   * warn the user that we may not be able to read all of the column encodings.
-   * @param log the logger to write any error message to
-   * @param path the data source path for error messages
-   * @param version the version of hive that wrote the file.
-   */
-  static void checkOrcVersion(Logger log, Path path, List<Integer> version) {
-    if (version.size() >= 1) {
-      int major = version.get(0);
-      int minor = 0;
-      if (version.size() >= 2) {
-        minor = version.get(1);
-      }
-      if (major > OrcFile.Version.CURRENT.getMajor() ||
-          (major == OrcFile.Version.CURRENT.getMajor() &&
-           minor > OrcFile.Version.CURRENT.getMinor())) {
-        log.warn(path + " was written by a future Hive version " +
-                 versionString(version) +
-                 ". This file may not be readable by this version of Hive.");
-      }
-    }
-  }
-
   /**
   * Constructor that let's the user specify additional options.
    * @param path pathname for file
    * @param options options for reading
    * @throws IOException
    */
-  public ReaderImpl(Path path, OrcFile.ReaderOptions options) throws IOException {
-    FileSystem fs = options.getFilesystem();
-    if (fs == null) {
-      fs = path.getFileSystem(options.getConfiguration());
-    }
-    this.fileSystem = fs;
-    this.path = path;
-    this.conf = options.getConfiguration();
-    this.maxLength = options.getMaxLength();
-
+  public ReaderImpl(Path path,
+                    OrcFile.ReaderOptions options) throws IOException {
+    super(path, options);
     FileMetadata fileMetadata = options.getFileMetadata();
     if (fileMetadata != null) {
-      this.compressionKind = fileMetadata.getCompressionKind();
-      this.bufferSize = fileMetadata.getCompressionBufferSize();
-      this.codec = WriterImpl.createCodec(compressionKind);
-      this.metadataSize = fileMetadata.getMetadataSize();
-      this.stripeStats = fileMetadata.getStripeStats();
-      this.versionList = fileMetadata.getVersionList();
-      this.writerVersion = OrcFile.WriterVersion.from(fileMetadata.getWriterVersionNum());
-      this.types = fileMetadata.getTypes();
-      this.rowIndexStride = fileMetadata.getRowIndexStride();
-      this.contentLength = fileMetadata.getContentLength();
-      this.numberOfRows = fileMetadata.getNumberOfRows();
-      this.fileStats = fileMetadata.getFileStats();
-      this.stripes = fileMetadata.getStripes();
       this.inspector =  OrcStruct.createObjectInspector(0, fileMetadata.getTypes());
       this.footerByteBuffer = null; // not cached and not needed here
-      this.userMetadata = null; // not cached and not needed here
       this.footerMetaAndPsBuffer = null;
     } else {
       FileMetaInfo footerMetaData;
@@ -363,7 +97,7 @@ public class ReaderImpl implements Reader {
         footerMetaData = options.getFileMetaInfo();
         this.footerMetaAndPsBuffer = null;
       } else {
-        footerMetaData = extractMetaInfoFromFooter(fs, path,
+        footerMetaData = extractMetaInfoFromFooter(fileSystem, path,
             options.getMaxLength());
         this.footerMetaAndPsBuffer = footerMetaData.footerMetaAndPsBuffer;
       }
@@ -374,37 +108,8 @@ public class ReaderImpl implements Reader {
                                    footerMetaData.footerBuffer
                                    );
       this.footerByteBuffer = footerMetaData.footerBuffer;
-      this.compressionKind = rInfo.compressionKind;
-      this.codec = rInfo.codec;
-      this.bufferSize = rInfo.bufferSize;
-      this.metadataSize = rInfo.metadataSize;
-      this.stripeStats = rInfo.metadata.getStripeStatsList();
-      this.types = rInfo.footer.getTypesList();
-      this.rowIndexStride = rInfo.footer.getRowIndexStride();
-      this.contentLength = rInfo.footer.getContentLength();
-      this.numberOfRows = rInfo.footer.getNumberOfRows();
-      this.userMetadata = rInfo.footer.getMetadataList();
-      this.fileStats = rInfo.footer.getStatisticsList();
       this.inspector = rInfo.inspector;
-      this.versionList = footerMetaData.versionList;
-      this.writerVersion = footerMetaData.writerVersion;
-      this.stripes = convertProtoStripesToStripes(rInfo.footer.getStripesList());
     }
-    this.schema = OrcUtils.convertTypeFromProtobuf(this.types, 0);
-  }
-
-  /**
-   * Get the WriterVersion based on the ORC file postscript.
-   * @param writerVersion the integer writer version
-   * @return the writer version of the file
-   */
-  static OrcFile.WriterVersion getWriterVersion(int writerVersion) {
-    for(OrcFile.WriterVersion version: OrcFile.WriterVersion.values()) {
-      if (version.getId() == writerVersion) {
-        return version;
-      }
-    }
-    return OrcFile.WriterVersion.FUTURE;
   }
 
   /** Extracts the necessary metadata from an externally store buffer (fullFooterBuffer). */
@@ -565,20 +270,6 @@ public class ReaderImpl implements Reader {
         );
   }
 
-  private static OrcFile.WriterVersion extractWriterVersion(OrcProto.PostScript ps) {
-    return (ps.hasWriterVersion()
-        ? getWriterVersion(ps.getWriterVersion()) : OrcFile.WriterVersion.ORIGINAL);
-  }
-
-  private static List<StripeInformation> convertProtoStripesToStripes(
-      List<OrcProto.StripeInformation> stripes) {
-    List<StripeInformation> result = new ArrayList<StripeInformation>(stripes.size());
-    for (OrcProto.StripeInformation info : stripes) {
-      result.add(new StripeInformationImpl(info));
-    }
-    return result;
-  }
-
   /**
    * MetaInfoObjExtractor - has logic to create the values for the fields in ReaderImpl
    *  from serialized fields.
@@ -617,7 +308,8 @@ public class ReaderImpl implements Reader {
 
   public FileMetaInfo getFileMetaInfo() {
     return new FileMetaInfo(compressionKind.toString(), bufferSize,
-        metadataSize, footerByteBuffer, versionList, writerVersion, footerMetaAndPsBuffer);
+        getMetadataSize(), footerByteBuffer, getVersionList(),
+        getWriterVersion(), footerMetaAndPsBuffer);
   }
 
   /** Same as FileMetaInfo, but with extra fields. FileMetaInfo is serialized for splits
@@ -697,184 +389,7 @@ public class ReaderImpl implements Reader {
   }
 
   @Override
-  public long getRawDataSize() {
-    // if the deserializedSize is not computed, then compute it, else
-    // return the already computed size. since we are reading from the footer
-    // we don't have to compute deserialized size repeatedly
-    if (deserializedSize == -1) {
-      List<Integer> indices = Lists.newArrayList();
-      for (int i = 0; i < fileStats.size(); ++i) {
-        indices.add(i);
-      }
-      deserializedSize = getRawDataSizeFromColIndices(indices);
-    }
-    return deserializedSize;
-  }
-
-  @Override
-  public long getRawDataSizeFromColIndices(List<Integer> colIndices) {
-    return getRawDataSizeFromColIndices(colIndices, types, fileStats);
-  }
-
-  public static long getRawDataSizeFromColIndices(
-      List<Integer> colIndices, List<OrcProto.Type> types,
-      List<OrcProto.ColumnStatistics> stats) {
-    long result = 0;
-    for (int colIdx : colIndices) {
-      result += getRawDataSizeOfColumn(colIdx, types, stats);
-    }
-    return result;
-  }
-
-  private static long getRawDataSizeOfColumn(int colIdx, List<OrcProto.Type> types,
-      List<OrcProto.ColumnStatistics> stats) {
-    OrcProto.ColumnStatistics colStat = stats.get(colIdx);
-    long numVals = colStat.getNumberOfValues();
-    OrcProto.Type type = types.get(colIdx);
-
-    switch (type.getKind()) {
-    case BINARY:
-      // old orc format doesn't support binary statistics. checking for binary
-      // statistics is not required as protocol buffers takes care of it.
-      return colStat.getBinaryStatistics().getSum();
-    case STRING:
-    case CHAR:
-    case VARCHAR:
-      // old orc format doesn't support sum for string statistics. checking for
-      // existence is not required as protocol buffers takes care of it.
-
-      // ORC strings are deserialized to java strings. so use java data model's
-      // string size
-      numVals = numVals == 0 ? 1 : numVals;
-      int avgStrLen = (int) (colStat.getStringStatistics().getSum() / numVals);
-      return numVals * JavaDataModel.get().lengthForStringOfLength(avgStrLen);
-    case TIMESTAMP:
-      return numVals * JavaDataModel.get().lengthOfTimestamp();
-    case DATE:
-      return numVals * JavaDataModel.get().lengthOfDate();
-    case DECIMAL:
-      return numVals * JavaDataModel.get().lengthOfDecimal();
-    case DOUBLE:
-    case LONG:
-      return numVals * JavaDataModel.get().primitive2();
-    case FLOAT:
-    case INT:
-    case SHORT:
-    case BOOLEAN:
-    case BYTE:
-      return numVals * JavaDataModel.get().primitive1();
-    default:
-      LOG.debug("Unknown primitive category: " + type.getKind());
-      break;
-    }
-
-    return 0;
-  }
-
-  @Override
-  public long getRawDataSizeOfColumns(List<String> colNames) {
-    List<Integer> colIndices = getColumnIndicesFromNames(colNames);
-    return getRawDataSizeFromColIndices(colIndices);
-  }
-
-  private List<Integer> getColumnIndicesFromNames(List<String> colNames) {
-    // top level struct
-    OrcProto.Type type = types.get(0);
-    List<Integer> colIndices = Lists.newArrayList();
-    List<String> fieldNames = type.getFieldNamesList();
-    int fieldIdx = 0;
-    for (String colName : colNames) {
-      if (fieldNames.contains(colName)) {
-        fieldIdx = fieldNames.indexOf(colName);
-      } else {
-        String s = "Cannot find field for: " + colName + " in ";
-        for (String fn : fieldNames) {
-          s += fn + ", ";
-        }
-        LOG.warn(s);
-        continue;
-      }
-
-      // a single field may span multiple columns. find start and end column
-      // index for the requested field
-      int idxStart = type.getSubtypes(fieldIdx);
-
-      int idxEnd;
-
-      // if the specified is the last field and then end index will be last
-      // column index
-      if (fieldIdx + 1 > fieldNames.size() - 1) {
-        idxEnd = getLastIdx() + 1;
-      } else {
-        idxEnd = type.getSubtypes(fieldIdx + 1);
-      }
-
-      // if start index and end index are same then the field is a primitive
-      // field else complex field (like map, list, struct, union)
-      if (idxStart == idxEnd) {
-        // simple field
-        colIndices.add(idxStart);
-      } else {
-        // complex fields spans multiple columns
-        for (int i = idxStart; i < idxEnd; i++) {
-          colIndices.add(i);
-        }
-      }
-    }
-    return colIndices;
-  }
-
-  private int getLastIdx() {
-    Set<Integer> indices = new HashSet<>();
-    for (OrcProto.Type type : types) {
-      indices.addAll(type.getSubtypesList());
-    }
-    return Collections.max(indices);
-  }
-
-  @Override
-  public List<OrcProto.StripeStatistics> getOrcProtoStripeStatistics() {
-    return stripeStats;
-  }
-
-  @Override
-  public List<OrcProto.ColumnStatistics> getOrcProtoFileStatistics() {
-    return fileStats;
-  }
-
-  @Override
-  public List<StripeStatistics> getStripeStatistics() {
-    List<StripeStatistics> result = new ArrayList<>();
-    for (OrcProto.StripeStatistics ss : stripeStats) {
-      result.add(new StripeStatistics(ss.getColStatsList()));
-    }
-    return result;
-  }
-
-  public List<OrcProto.UserMetadataItem> getOrcProtoUserMetadata() {
-    return userMetadata;
-  }
-
-  @Override
-  public List<Integer> getVersionList() {
-    return versionList;
-  }
-
-  @Override
-  public int getMetadataSize() {
-    return metadataSize;
-  }
-
-  @Override
   public String toString() {
-    StringBuilder buffer = new StringBuilder();
-    buffer.append("ORC Reader(");
-    buffer.append(path);
-    if (maxLength != -1) {
-      buffer.append(", ");
-      buffer.append(maxLength);
-    }
-    buffer.append(")");
-    return buffer.toString();
+    return "Hive " + super.toString();
   }
 }


[27/27] hive git commit: HIVE-11417. Move the ReaderImpl and RowReaderImpl to the ORC module, by making shims for the row by row reader. (omalley reviewed by prasanth_j)

Posted by om...@apache.org.
HIVE-11417. Move the ReaderImpl and RowReaderImpl to the ORC module,
by making shims for the row by row reader. (omalley reviewed by prasanth_j)

Fixes #72.


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/ffb79509
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/ffb79509
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/ffb79509

Branch: refs/heads/master
Commit: ffb79509bcaefb9e7f916930edb022371b9d810f
Parents: fd06601
Author: Owen O'Malley <om...@apache.org>
Authored: Fri May 20 14:20:00 2016 -0700
Committer: Owen O'Malley <om...@apache.org>
Committed: Fri May 20 14:21:39 2016 -0700

----------------------------------------------------------------------
 bin/ext/orcfiledump.cmd                         |    2 +-
 bin/ext/orcfiledump.sh                          |    2 +-
 .../hive/hcatalog/streaming/TestStreaming.java  |    9 +-
 .../llap/io/decode/OrcEncodedDataConsumer.java  |    2 +-
 .../llap/io/encoded/OrcEncodedDataReader.java   |   15 +-
 .../hive/llap/io/metadata/OrcFileMetadata.java  |    4 +-
 orc/pom.xml                                     |   27 +
 .../org/apache/orc/FileFormatException.java     |   30 +
 orc/src/java/org/apache/orc/OrcFile.java        |    6 +
 orc/src/java/org/apache/orc/Reader.java         |    2 +-
 .../java/org/apache/orc/TypeDescription.java    |   18 +-
 orc/src/java/org/apache/orc/impl/AcidStats.java |   60 +
 .../orc/impl/ConvertTreeReaderFactory.java      | 2840 +++++++++++++
 .../java/org/apache/orc/impl/HadoopShims.java   |   79 +
 .../org/apache/orc/impl/HadoopShimsCurrent.java |   30 +
 .../org/apache/orc/impl/HadoopShims_2_2.java    |   71 +-
 .../java/org/apache/orc/impl/IntegerReader.java |    3 +-
 .../java/org/apache/orc/impl/OrcAcidUtils.java  |   85 +
 .../java/org/apache/orc/impl/ReaderImpl.java    |  758 ++++
 .../org/apache/orc/impl/RecordReaderImpl.java   | 1215 ++++++
 .../org/apache/orc/impl/RecordReaderUtils.java  |  578 +++
 .../org/apache/orc/impl/SchemaEvolution.java    |  190 +
 .../org/apache/orc/impl/TreeReaderFactory.java  | 2093 ++++++++++
 .../java/org/apache/orc/impl/ZeroCopyShims.java |   89 +
 orc/src/java/org/apache/orc/tools/FileDump.java |  934 +++++
 .../java/org/apache/orc/tools/JsonFileDump.java |  406 ++
 .../org/apache/orc/TestColumnStatistics.java    |  364 ++
 .../org/apache/orc/TestNewIntegerEncoding.java  | 1373 +++++++
 .../org/apache/orc/TestOrcNullOptimization.java |  415 ++
 .../test/org/apache/orc/TestOrcTimezone1.java   |  189 +
 .../test/org/apache/orc/TestOrcTimezone2.java   |  143 +
 .../org/apache/orc/TestStringDictionary.java    |  290 ++
 .../org/apache/orc/TestTypeDescription.java     |   68 +
 .../org/apache/orc/TestUnrolledBitPack.java     |  114 +
 .../test/org/apache/orc/TestVectorOrcFile.java  | 2782 +++++++++++++
 .../org/apache/orc/impl/TestOrcWideTable.java   |   64 +
 orc/src/test/org/apache/orc/impl/TestRLEv2.java |  307 ++
 .../org/apache/orc/impl/TestReaderImpl.java     |  152 +
 .../apache/orc/impl/TestRecordReaderImpl.java   | 1691 ++++++++
 .../org/apache/orc/impl/TestStreamName.java     |   49 +
 .../test/org/apache/orc/tools/TestFileDump.java |  486 +++
 .../org/apache/orc/tools/TestJsonFileDump.java  |  150 +
 orc/src/test/resources/orc-file-11-format.orc   |  Bin 0 -> 373336 bytes
 .../resources/orc-file-dump-bloomfilter.out     |  179 +
 .../resources/orc-file-dump-bloomfilter2.out    |  179 +
 .../orc-file-dump-dictionary-threshold.out      |  190 +
 orc/src/test/resources/orc-file-dump.json       | 1355 +++++++
 orc/src/test/resources/orc-file-dump.out        |  195 +
 orc/src/test/resources/orc-file-has-null.out    |  112 +
 .../expressions/CastDecimalToTimestamp.java     |    8 +-
 .../expressions/CastDoubleToTimestamp.java      |   13 +-
 .../vector/expressions/CastLongToTimestamp.java |    4 +-
 .../CastMillisecondsLongToTimestamp.java        |    7 +-
 .../ql/exec/vector/expressions/StringExpr.java  |  354 --
 .../hive/ql/hooks/PostExecOrcFileDump.java      |    4 +-
 .../hadoop/hive/ql/io/FileFormatException.java  |   30 -
 .../ql/io/orc/ConvertTreeReaderFactory.java     | 3750 ------------------
 .../apache/hadoop/hive/ql/io/orc/FileDump.java  |  884 -----
 .../hadoop/hive/ql/io/orc/JsonFileDump.java     |  401 --
 .../hive/ql/io/orc/OrcRawRecordMerger.java      |   35 +-
 .../hadoop/hive/ql/io/orc/OrcRecordUpdater.java |   71 +-
 .../hadoop/hive/ql/io/orc/ReaderImpl.java       |  509 +--
 .../hadoop/hive/ql/io/orc/RecordReaderImpl.java | 1823 ++++-----
 .../hive/ql/io/orc/RecordReaderUtils.java       |  586 ---
 .../hadoop/hive/ql/io/orc/SchemaEvolution.java  |  190 -
 .../hive/ql/io/orc/TreeReaderFactory.java       | 2525 ------------
 .../ql/io/orc/encoded/EncodedReaderImpl.java    |    2 +-
 .../orc/encoded/EncodedTreeReaderFactory.java   |    2 +-
 .../apache/hadoop/hive/ql/TestTxnCommands.java  |    2 -
 .../TestTimestampWritableAndColumnVector.java   |    7 +-
 .../vector/expressions/TestVectorTypeCasts.java |   10 +-
 .../exec/vector/udf/TestVectorUDFAdaptor.java   |    2 -
 .../hive/ql/io/orc/TestColumnStatistics.java    |  352 --
 .../hadoop/hive/ql/io/orc/TestFileDump.java     |  418 --
 .../hadoop/hive/ql/io/orc/TestJsonFileDump.java |  139 -
 .../hive/ql/io/orc/TestNewIntegerEncoding.java  | 1342 -------
 .../hadoop/hive/ql/io/orc/TestOrcFile.java      |   70 +-
 .../hive/ql/io/orc/TestOrcNullOptimization.java |  400 --
 .../hive/ql/io/orc/TestOrcRecordUpdater.java    |    4 +-
 .../hadoop/hive/ql/io/orc/TestOrcTimezone1.java |  194 -
 .../hadoop/hive/ql/io/orc/TestOrcTimezone2.java |  142 -
 .../hadoop/hive/ql/io/orc/TestOrcWideTable.java |   64 -
 .../apache/hadoop/hive/ql/io/orc/TestRLEv2.java |  297 --
 .../hadoop/hive/ql/io/orc/TestReaderImpl.java   |  151 -
 .../hive/ql/io/orc/TestRecordReaderImpl.java    | 1678 --------
 .../hadoop/hive/ql/io/orc/TestStreamName.java   |   50 -
 .../hive/ql/io/orc/TestStringDictionary.java    |  261 --
 .../hive/ql/io/orc/TestTypeDescription.java     |   68 -
 .../hive/ql/io/orc/TestUnrolledBitPack.java     |  114 -
 .../hive/ql/io/orc/TestVectorOrcFile.java       | 2791 -------------
 .../resources/orc-file-dump-bloomfilter.out     |  179 -
 .../resources/orc-file-dump-bloomfilter2.out    |  179 -
 .../orc-file-dump-dictionary-threshold.out      |  190 -
 ql/src/test/resources/orc-file-dump.json        | 1355 -------
 ql/src/test/resources/orc-file-dump.out         |  195 -
 ql/src/test/resources/orc-file-has-null.out     |  112 -
 .../results/clientpositive/orc_create.q.out     |   12 +-
 .../clientpositive/orc_int_type_promotion.q.out |   12 +-
 ...vol_orc_vec_mapwork_part_all_primitive.q.out |   40 +-
 ...vol_orc_vec_mapwork_part_all_primitive.q.out |   40 +-
 .../clientpositive/vector_complex_all.q.out     |    6 +-
 .../hive/serde2/io/TimestampWritable.java       |  114 +-
 .../PrimitiveObjectInspectorUtils.java          |    7 +-
 .../hive/serde2/io/TestTimestampWritable.java   |   41 +-
 .../apache/hadoop/hive/shims/Hadoop23Shims.java |   63 +-
 .../apache/hadoop/hive/shims/ZeroCopyShims.java |   86 -
 .../apache/hadoop/hive/shims/HadoopShims.java   |   70 -
 .../hadoop/hive/shims/HadoopShimsSecure.java    |   29 -
 .../ql/exec/vector/TimestampColumnVector.java   |    9 +-
 .../ql/exec/vector/expressions/StringExpr.java  |  354 ++
 .../hive/ql/io/sarg/SearchArgumentImpl.java     |   16 +-
 .../hadoop/hive/ql/util/TimestampUtils.java     |   94 +
 112 files changed, 21796 insertions(+), 21556 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/bin/ext/orcfiledump.cmd
----------------------------------------------------------------------
diff --git a/bin/ext/orcfiledump.cmd b/bin/ext/orcfiledump.cmd
index f78ed7f..ff4b410 100644
--- a/bin/ext/orcfiledump.cmd
+++ b/bin/ext/orcfiledump.cmd
@@ -14,7 +14,7 @@
 @rem See the License for the specific language governing permissions and
 @rem limitations under the License.
 
-set CLASS=org.apache.hadoop.hive.ql.io.orc.FileDump
+set CLASS=org.apache.orc.tools.FileDump
 set HIVE_OPTS=
 set HADOOP_CLASSPATH=
 

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/bin/ext/orcfiledump.sh
----------------------------------------------------------------------
diff --git a/bin/ext/orcfiledump.sh b/bin/ext/orcfiledump.sh
index 74f1a1e..c84e61c 100644
--- a/bin/ext/orcfiledump.sh
+++ b/bin/ext/orcfiledump.sh
@@ -17,7 +17,7 @@ THISSERVICE=orcfiledump
 export SERVICE_LIST="${SERVICE_LIST}${THISSERVICE} "
 
 orcfiledump () {
-  CLASS=org.apache.hadoop.hive.ql.io.orc.FileDump
+  CLASS=org.apache.orc.tools.FileDump
   HIVE_OPTS=''
   execHiveCmd $CLASS "$@"
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/TestStreaming.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/TestStreaming.java b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/TestStreaming.java
index 6016425..4d2a2ee 100644
--- a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/TestStreaming.java
+++ b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/TestStreaming.java
@@ -57,16 +57,15 @@ import org.apache.hadoop.hive.metastore.api.ShowLocksResponseElement;
 import org.apache.hadoop.hive.metastore.api.TxnAbortedException;
 import org.apache.hadoop.hive.metastore.api.TxnInfo;
 import org.apache.hadoop.hive.metastore.api.TxnState;
-import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
 import org.apache.hadoop.hive.metastore.txn.TxnDbUtil;
 import org.apache.hadoop.hive.ql.CommandNeedRetryException;
 import org.apache.hadoop.hive.ql.Driver;
 import org.apache.hadoop.hive.ql.io.AcidUtils;
 import org.apache.hadoop.hive.ql.io.IOConstants;
-import org.apache.hadoop.hive.ql.io.orc.FileDump;
+import org.apache.orc.impl.OrcAcidUtils;
+import org.apache.orc.tools.FileDump;
 import org.apache.hadoop.hive.ql.io.orc.OrcFile;
 import org.apache.hadoop.hive.ql.io.orc.OrcInputFormat;
-import org.apache.hadoop.hive.ql.io.orc.OrcRecordUpdater;
 import org.apache.hadoop.hive.ql.io.orc.OrcStruct;
 import org.apache.hadoop.hive.ql.io.orc.Reader;
 import org.apache.hadoop.hive.ql.io.orc.RecordReader;
@@ -1089,7 +1088,7 @@ public class TestStreaming {
     Reader reader = OrcFile.createReader(orcFile,
             OrcFile.readerOptions(conf).filesystem(fs));
 
-    RecordReader rows = reader.rows(null);
+    RecordReader rows = reader.rows();
     StructObjectInspector inspector = (StructObjectInspector) reader
             .getObjectInspector();
 
@@ -1561,7 +1560,7 @@ public class TestStreaming {
       final Map<String, List<Long>> offsetMap, final String key, final int numEntries)
       throws IOException {
     Path dataPath = new Path(file);
-    Path sideFilePath = OrcRecordUpdater.getSideFile(dataPath);
+    Path sideFilePath = OrcAcidUtils.getSideFile(dataPath);
     Path cPath = new Path(sideFilePath.getParent(), sideFilePath.getName() + ".corrupt");
     FileSystem fs = sideFilePath.getFileSystem(conf);
     List<Long> offsets = offsetMap.get(key);

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/llap-server/src/java/org/apache/hadoop/hive/llap/io/decode/OrcEncodedDataConsumer.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/io/decode/OrcEncodedDataConsumer.java b/llap-server/src/java/org/apache/hadoop/hive/llap/io/decode/OrcEncodedDataConsumer.java
index a689f10..619d1a4 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/io/decode/OrcEncodedDataConsumer.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/io/decode/OrcEncodedDataConsumer.java
@@ -42,7 +42,7 @@ import org.apache.hadoop.hive.ql.io.orc.encoded.EncodedTreeReaderFactory.Settabl
 import org.apache.hadoop.hive.ql.io.orc.encoded.OrcBatchKey;
 import org.apache.hadoop.hive.ql.io.orc.encoded.Reader.OrcEncodedColumnBatch;
 import org.apache.hadoop.hive.ql.io.orc.RecordReaderImpl;
-import org.apache.hadoop.hive.ql.io.orc.TreeReaderFactory;
+import org.apache.orc.impl.TreeReaderFactory;
 import org.apache.hadoop.hive.ql.io.orc.WriterImpl;
 import org.apache.orc.OrcProto;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/OrcEncodedDataReader.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/OrcEncodedDataReader.java b/llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/OrcEncodedDataReader.java
index 7effe69..69c0647 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/OrcEncodedDataReader.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/OrcEncodedDataReader.java
@@ -67,13 +67,12 @@ import org.apache.hadoop.hive.ql.io.orc.OrcInputFormat;
 import org.apache.hadoop.hive.ql.io.orc.OrcSplit;
 import org.apache.hadoop.hive.ql.io.orc.encoded.Reader;
 import org.apache.hadoop.hive.ql.io.orc.RecordReaderImpl;
-import org.apache.hadoop.hive.ql.io.orc.RecordReaderImpl.SargApplier;
 import org.apache.hadoop.hive.ql.io.orc.encoded.EncodedOrcFile;
 import org.apache.hadoop.hive.ql.io.orc.encoded.EncodedReader;
 import org.apache.hadoop.hive.ql.io.orc.encoded.OrcBatchKey;
 import org.apache.hadoop.hive.ql.io.orc.encoded.Reader.OrcEncodedColumnBatch;
 import org.apache.hadoop.hive.ql.io.orc.encoded.Reader.PoolFactory;
-import org.apache.hadoop.hive.ql.io.orc.RecordReaderUtils;
+import org.apache.orc.impl.RecordReaderUtils;
 import org.apache.orc.StripeInformation;
 import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
 import org.apache.hadoop.mapred.FileSplit;
@@ -343,7 +342,8 @@ public class OrcEncodedDataReader extends CallableWithNdc<Void>
         // intermediate changes for individual columns will unset values in the array.
         // Skip this case for 0-column read. We could probably special-case it just like we do
         // in EncodedReaderImpl, but for now it's not that important.
-        if (colRgs.length > 0 && colRgs[0] == SargApplier.READ_NO_RGS) continue;
+        if (colRgs.length > 0 && colRgs[0] ==
+            RecordReaderImpl.SargApplier.READ_NO_RGS) continue;
 
         // 6.1. Determine the columns to read (usually the same as requested).
         if (cols == null || cols.size() == colRgs.length) {
@@ -691,12 +691,13 @@ public class OrcEncodedDataReader extends CallableWithNdc<Void>
    */
   private boolean determineRgsToRead(boolean[] globalIncludes, int rowIndexStride,
       ArrayList<OrcStripeMetadata> metadata) throws IOException {
-    SargApplier sargApp = null;
+    RecordReaderImpl.SargApplier sargApp = null;
     if (sarg != null && rowIndexStride != 0) {
       List<OrcProto.Type> types = fileMetadata.getTypes();
       String[] colNamesForSarg = OrcInputFormat.getSargColumnNames(
           columnNames, types, globalIncludes, fileMetadata.isOriginalFormat());
-      sargApp = new SargApplier(sarg, colNamesForSarg, rowIndexStride, types, globalIncludes.length);
+      sargApp = new RecordReaderImpl.SargApplier(sarg, colNamesForSarg,
+          rowIndexStride, types, globalIncludes.length);
     }
     boolean hasAnyData = false;
     // readState should have been initialized by this time with an empty array.
@@ -710,8 +711,8 @@ public class OrcEncodedDataReader extends CallableWithNdc<Void>
         rgsToRead = sargApp.pickRowGroups(stripe, stripeMetadata.getRowIndexes(),
             stripeMetadata.getBloomFilterIndexes(), true);
       }
-      boolean isNone = rgsToRead == SargApplier.READ_NO_RGS,
-          isAll = rgsToRead == SargApplier.READ_ALL_RGS;
+      boolean isNone = rgsToRead == RecordReaderImpl.SargApplier.READ_NO_RGS,
+          isAll = rgsToRead == RecordReaderImpl.SargApplier.READ_ALL_RGS;
       hasAnyData = hasAnyData || !isNone;
       if (LlapIoImpl.ORC_LOGGER.isTraceEnabled()) {
         if (isNone) {

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/llap-server/src/java/org/apache/hadoop/hive/llap/io/metadata/OrcFileMetadata.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/io/metadata/OrcFileMetadata.java b/llap-server/src/java/org/apache/hadoop/hive/llap/io/metadata/OrcFileMetadata.java
index 4e42a0f..c9b0a4d 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/io/metadata/OrcFileMetadata.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/io/metadata/OrcFileMetadata.java
@@ -29,11 +29,11 @@ import org.apache.hadoop.hive.llap.cache.LlapCacheableBuffer;
 import org.apache.hadoop.hive.ql.io.SyntheticFileId;
 import org.apache.hadoop.hive.ql.io.orc.OrcInputFormat;
 import org.apache.hadoop.hive.ql.io.orc.Reader;
-import org.apache.hadoop.hive.ql.io.orc.ReaderImpl.StripeInformationImpl;
 import org.apache.orc.CompressionKind;
 import org.apache.orc.FileMetadata;
 import org.apache.orc.OrcProto;
 import org.apache.orc.StripeInformation;
+import org.apache.orc.impl.ReaderImpl;
 
 /** ORC file metadata. Currently contains some duplicate info due to how different parts
  * of ORC use different info. Ideally we would get rid of protobuf structs in code beyond reading,
@@ -72,7 +72,7 @@ public final class OrcFileMetadata extends LlapCacheableBuffer implements FileMe
   @VisibleForTesting
   public static OrcFileMetadata createDummy(Object fileKey) {
     OrcFileMetadata ofm = new OrcFileMetadata(fileKey);
-    ofm.stripes.add(new StripeInformationImpl(
+    ofm.stripes.add(new ReaderImpl.StripeInformationImpl(
         OrcProto.StripeInformation.getDefaultInstance()));
     ofm.fileStats.add(OrcProto.ColumnStatistics.getDefaultInstance());
     ofm.stripeStats.add(OrcProto.StripeStatistics.newBuilder().addColStats(createStatsDummy()).build());

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/orc/pom.xml
----------------------------------------------------------------------
diff --git a/orc/pom.xml b/orc/pom.xml
index 2d80c97..cc27077 100644
--- a/orc/pom.xml
+++ b/orc/pom.xml
@@ -72,6 +72,33 @@
       </exclusions>
     </dependency>
     <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-hdfs</artifactId>
+      <version>${hadoop.version}</version>
+      <exclusions>
+        <exclusion>
+          <groupId>javax.servlet</groupId>
+          <artifactId>servlet-api</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>javax.servlet.jsp</groupId>
+          <artifactId>jsp-api</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.mortbay.jetty</groupId>
+          <artifactId>jetty</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.mortbay.jetty</groupId>
+          <artifactId>jetty-util</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.apache.avro</groupId>
+          <artifactId>avro</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
       <groupId>org.iq80.snappy</groupId>
       <artifactId>snappy</artifactId>
       <version>${snappy.version}</version>

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/orc/src/java/org/apache/orc/FileFormatException.java
----------------------------------------------------------------------
diff --git a/orc/src/java/org/apache/orc/FileFormatException.java b/orc/src/java/org/apache/orc/FileFormatException.java
new file mode 100644
index 0000000..2cebea7
--- /dev/null
+++ b/orc/src/java/org/apache/orc/FileFormatException.java
@@ -0,0 +1,30 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.orc;
+
+import java.io.IOException;
+
+/**
+ * Thrown when an invalid file format is encountered.
+ */
+public class FileFormatException extends IOException {
+
+  public FileFormatException(String errMsg) {
+    super(errMsg);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/orc/src/java/org/apache/orc/OrcFile.java
----------------------------------------------------------------------
diff --git a/orc/src/java/org/apache/orc/OrcFile.java b/orc/src/java/org/apache/orc/OrcFile.java
index 85506ff..7dd7333 100644
--- a/orc/src/java/org/apache/orc/OrcFile.java
+++ b/orc/src/java/org/apache/orc/OrcFile.java
@@ -25,6 +25,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.orc.impl.MemoryManager;
+import org.apache.orc.impl.ReaderImpl;
 import org.apache.orc.impl.WriterImpl;
 
 /**
@@ -212,6 +213,11 @@ public class OrcFile {
     return new ReaderOptions(conf);
   }
 
+  public static Reader createReader(Path path,
+                                    ReaderOptions options) throws IOException {
+    return new ReaderImpl(path, options);
+  }
+
   public interface WriterContext {
     Writer getWriter();
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/orc/src/java/org/apache/orc/Reader.java
----------------------------------------------------------------------
diff --git a/orc/src/java/org/apache/orc/Reader.java b/orc/src/java/org/apache/orc/Reader.java
index 39de763..87f3293 100644
--- a/orc/src/java/org/apache/orc/Reader.java
+++ b/orc/src/java/org/apache/orc/Reader.java
@@ -334,7 +334,7 @@ public interface Reader {
    * @return a new RecordReader
    * @throws IOException
    */
-  RecordReader rowsOptions(Options options) throws IOException;
+  RecordReader rows(Options options) throws IOException;
 
   /**
    * @return List of integers representing version of the file, in order from major to minor.

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/orc/src/java/org/apache/orc/TypeDescription.java
----------------------------------------------------------------------
diff --git a/orc/src/java/org/apache/orc/TypeDescription.java b/orc/src/java/org/apache/orc/TypeDescription.java
index b8e057e..ffe3c1f 100644
--- a/orc/src/java/org/apache/orc/TypeDescription.java
+++ b/orc/src/java/org/apache/orc/TypeDescription.java
@@ -344,25 +344,25 @@ public class TypeDescription {
       case INT:
       case LONG:
       case DATE:
-        return new LongColumnVector();
+        return new LongColumnVector(maxSize);
       case TIMESTAMP:
-        return new TimestampColumnVector();
+        return new TimestampColumnVector(maxSize);
       case FLOAT:
       case DOUBLE:
-        return new DoubleColumnVector();
+        return new DoubleColumnVector(maxSize);
       case DECIMAL:
-        return new DecimalColumnVector(precision, scale);
+        return new DecimalColumnVector(maxSize, precision, scale);
       case STRING:
       case BINARY:
       case CHAR:
       case VARCHAR:
-        return new BytesColumnVector();
+        return new BytesColumnVector(maxSize);
       case STRUCT: {
         ColumnVector[] fieldVector = new ColumnVector[children.size()];
         for(int i=0; i < fieldVector.length; ++i) {
           fieldVector[i] = children.get(i).createColumn(maxSize);
         }
-        return new StructColumnVector(VectorizedRowBatch.DEFAULT_SIZE,
+        return new StructColumnVector(maxSize,
                 fieldVector);
       }
       case UNION: {
@@ -370,14 +370,14 @@ public class TypeDescription {
         for(int i=0; i < fieldVector.length; ++i) {
           fieldVector[i] = children.get(i).createColumn(maxSize);
         }
-        return new UnionColumnVector(VectorizedRowBatch.DEFAULT_SIZE,
+        return new UnionColumnVector(maxSize,
             fieldVector);
       }
       case LIST:
-        return new ListColumnVector(VectorizedRowBatch.DEFAULT_SIZE,
+        return new ListColumnVector(maxSize,
             children.get(0).createColumn(maxSize));
       case MAP:
-        return new MapColumnVector(VectorizedRowBatch.DEFAULT_SIZE,
+        return new MapColumnVector(maxSize,
             children.get(0).createColumn(maxSize),
             children.get(1).createColumn(maxSize));
       default:

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/orc/src/java/org/apache/orc/impl/AcidStats.java
----------------------------------------------------------------------
diff --git a/orc/src/java/org/apache/orc/impl/AcidStats.java b/orc/src/java/org/apache/orc/impl/AcidStats.java
new file mode 100644
index 0000000..6657fe9
--- /dev/null
+++ b/orc/src/java/org/apache/orc/impl/AcidStats.java
@@ -0,0 +1,60 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.orc.impl;
+
+/**
+ * Statistics about the ACID operations in an ORC file
+ */
+public class AcidStats {
+  public long inserts;
+  public long updates;
+  public long deletes;
+
+  public AcidStats() {
+    inserts = 0;
+    updates = 0;
+    deletes = 0;
+  }
+
+  public AcidStats(String serialized) {
+    String[] parts = serialized.split(",");
+    inserts = Long.parseLong(parts[0]);
+    updates = Long.parseLong(parts[1]);
+    deletes = Long.parseLong(parts[2]);
+  }
+
+  public String serialize() {
+    StringBuilder builder = new StringBuilder();
+    builder.append(inserts);
+    builder.append(",");
+    builder.append(updates);
+    builder.append(",");
+    builder.append(deletes);
+    return builder.toString();
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder builder = new StringBuilder();
+    builder.append(" inserts: ").append(inserts);
+    builder.append(" updates: ").append(updates);
+    builder.append(" deletes: ").append(deletes);
+    return builder.toString();
+  }
+}


[12/27] hive git commit: HIVE-11417. Move the ReaderImpl and RowReaderImpl to the ORC module, by making shims for the row by row reader. (omalley reviewed by prasanth_j)

Posted by om...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java
index 2199b11..e46ca51 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java
@@ -18,1218 +18,923 @@
 package org.apache.hadoop.hive.ql.io.orc;
 
 import java.io.IOException;
-import java.math.BigDecimal;
-import java.sql.Date;
-import java.sql.Timestamp;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.HashMap;
 import java.util.List;
-import java.util.Map;
 
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.orc.BooleanColumnStatistics;
-import org.apache.orc.impl.BufferChunk;
-import org.apache.orc.ColumnStatistics;
-import org.apache.orc.impl.ColumnStatisticsImpl;
-import org.apache.orc.CompressionCodec;
-import org.apache.orc.DataReader;
-import org.apache.orc.DateColumnStatistics;
-import org.apache.orc.DecimalColumnStatistics;
-import org.apache.orc.DoubleColumnStatistics;
-import org.apache.orc.impl.DataReaderProperties;
-import org.apache.orc.impl.InStream;
-import org.apache.orc.IntegerColumnStatistics;
-import org.apache.orc.OrcConf;
-import org.apache.orc.impl.OrcIndex;
-import org.apache.orc.impl.PositionProvider;
-import org.apache.orc.impl.StreamName;
-import org.apache.orc.StringColumnStatistics;
-import org.apache.orc.StripeInformation;
-import org.apache.orc.TimestampColumnStatistics;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hive.common.io.DiskRange;
-import org.apache.hadoop.hive.common.io.DiskRangeList;
-import org.apache.hadoop.hive.common.io.DiskRangeList.CreateHelper;
-import org.apache.hadoop.hive.common.type.HiveDecimal;
-import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
-import org.apache.orc.BloomFilterIO;
-import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
-import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
-import org.apache.hadoop.hive.ql.io.sarg.SearchArgument.TruthValue;
+import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.ListColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.MapColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.StructColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.UnionColumnVector;
+import org.apache.hadoop.hive.serde2.io.ByteWritable;
 import org.apache.hadoop.hive.serde2.io.DateWritable;
+import org.apache.hadoop.hive.serde2.io.DoubleWritable;
+import org.apache.hadoop.hive.serde2.io.HiveCharWritable;
 import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.hadoop.hive.serde2.io.HiveVarcharWritable;
+import org.apache.hadoop.hive.serde2.io.ShortWritable;
 import org.apache.hadoop.hive.serde2.io.TimestampWritable;
+import org.apache.hadoop.io.BooleanWritable;
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
-import org.apache.orc.OrcProto;
+import org.apache.orc.TypeDescription;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 
-public class RecordReaderImpl implements RecordReader {
+public class RecordReaderImpl extends org.apache.orc.impl.RecordReaderImpl
+                              implements RecordReader {
   static final Logger LOG = LoggerFactory.getLogger(RecordReaderImpl.class);
-  private static final boolean isLogDebugEnabled = LOG.isDebugEnabled();
-  private static final Object UNKNOWN_VALUE = new Object();
-  private final Path path;
-  private final long firstRow;
-  private final List<StripeInformation> stripes =
-      new ArrayList<StripeInformation>();
-  private OrcProto.StripeFooter stripeFooter;
-  private final long totalRowCount;
-  private final CompressionCodec codec;
-  private final List<OrcProto.Type> types;
-  private final int bufferSize;
-  private final boolean[] included;
-  private final long rowIndexStride;
-  private long rowInStripe = 0;
-  private int currentStripe = -1;
-  private long rowBaseInStripe = 0;
-  private long rowCountInStripe = 0;
-  private final Map<StreamName, InStream> streams =
-      new HashMap<StreamName, InStream>();
-  DiskRangeList bufferChunks = null;
-  private final TreeReaderFactory.TreeReader reader;
-  private final OrcProto.RowIndex[] indexes;
-  private final OrcProto.BloomFilterIndex[] bloomFilterIndices;
-  private final SargApplier sargApp;
-  // an array about which row groups aren't skipped
-  private boolean[] includedRowGroups = null;
-  private final DataReader dataReader;
+  private final VectorizedRowBatch batch;
+  private int rowInBatch;
+  private long baseRow;
 
-  /**
-   * Given a list of column names, find the given column and return the index.
-   *
-   * @param columnNames the list of potential column names
-   * @param columnName  the column name to look for
-   * @param rootColumn  offset the result with the rootColumn
-   * @return the column number or -1 if the column wasn't found
-   */
-  static int findColumns(String[] columnNames,
-                         String columnName,
-                         int rootColumn) {
-    for(int i=0; i < columnNames.length; ++i) {
-      if (columnName.equals(columnNames[i])) {
-        return i + rootColumn;
-      }
-    }
-    return -1;
+  protected RecordReaderImpl(ReaderImpl fileReader,
+                             Reader.Options options) throws IOException {
+    super(fileReader, options);
+    batch = this.schema.createRowBatch();
+    rowInBatch = 0;
   }
 
   /**
-   * Find the mapping from predicate leaves to columns.
-   * @param sargLeaves the search argument that we need to map
-   * @param columnNames the names of the columns
-   * @param rootColumn the offset of the top level row, which offsets the
-   *                   result
-   * @return an array mapping the sarg leaves to concrete column numbers
+   * If the current batch is empty, get a new one.
+   * @return true if we have rows available.
+   * @throws IOException
    */
-  public static int[] mapSargColumnsToOrcInternalColIdx(List<PredicateLeaf> sargLeaves,
-                             String[] columnNames,
-                             int rootColumn) {
-    int[] result = new int[sargLeaves.size()];
-    Arrays.fill(result, -1);
-    for(int i=0; i < result.length; ++i) {
-      String colName = sargLeaves.get(i).getColumnName();
-      result[i] = findColumns(columnNames, colName, rootColumn);
+  boolean ensureBatch() throws IOException {
+    if (rowInBatch >= batch.size) {
+      baseRow = super.getRowNumber();
+      rowInBatch = 0;
+      return super.nextBatch(batch);
     }
-    return result;
+    return true;
   }
 
-  protected RecordReaderImpl(ReaderImpl fileReader,
-                             Reader.Options options) throws IOException {
-    SchemaEvolution treeReaderSchema;
-    this.included = options.getInclude();
-    included[0] = true;
-    if (options.getSchema() == null) {
-      if (LOG.isInfoEnabled()) {
-        LOG.info("Schema on read not provided -- using file schema " +
-            fileReader.getSchema());
-      }
-      treeReaderSchema = new SchemaEvolution(fileReader.getSchema(), included);
-    } else {
+  @Override
+  public long getRowNumber() {
+    return baseRow + rowInBatch;
+  }
 
-      // Now that we are creating a record reader for a file, validate that the schema to read
-      // is compatible with the file schema.
-      //
-      treeReaderSchema = new SchemaEvolution(fileReader.getSchema(),
-          options.getSchema(),
-          included);
-    }
-    this.path = fileReader.path;
-    this.codec = fileReader.codec;
-    this.types = fileReader.types;
-    this.bufferSize = fileReader.bufferSize;
-    this.rowIndexStride = fileReader.rowIndexStride;
-    FileSystem fileSystem = fileReader.fileSystem;
-    SearchArgument sarg = options.getSearchArgument();
-    if (sarg != null && rowIndexStride != 0) {
-      sargApp = new SargApplier(
-          sarg, options.getColumnNames(), rowIndexStride, types, included.length);
-    } else {
-      sargApp = null;
-    }
-    long rows = 0;
-    long skippedRows = 0;
-    long offset = options.getOffset();
-    long maxOffset = options.getMaxOffset();
-    for(StripeInformation stripe: fileReader.getStripes()) {
-      long stripeStart = stripe.getOffset();
-      if (offset > stripeStart) {
-        skippedRows += stripe.getNumberOfRows();
-      } else if (stripeStart < maxOffset) {
-        this.stripes.add(stripe);
-        rows += stripe.getNumberOfRows();
-      }
-    }
+  @Override
+  public boolean hasNext() throws IOException {
+    return ensureBatch();
+  }
 
-    Boolean zeroCopy = options.getUseZeroCopy();
-    if (zeroCopy == null) {
-      zeroCopy = OrcConf.USE_ZEROCOPY.getBoolean(fileReader.conf);
-    }
-    if (options.getDataReader() == null) {
-      dataReader = RecordReaderUtils.createDefaultDataReader(
-          DataReaderProperties.builder()
-              .withBufferSize(bufferSize)
-              .withCompression(fileReader.compressionKind)
-              .withFileSystem(fileSystem)
-              .withPath(path)
-              .withTypeCount(types.size())
-              .withZeroCopy(zeroCopy)
-              .build());
+  @Override
+  public void seekToRow(long row) throws IOException {
+    if (row >= baseRow && row < baseRow + batch.size) {
+      rowInBatch = (int) (row - baseRow);
     } else {
-      dataReader = options.getDataReader();
+      super.seekToRow(row);
+      batch.size = 0;
+      ensureBatch();
     }
-    firstRow = skippedRows;
-    totalRowCount = rows;
-    Boolean skipCorrupt = options.getSkipCorruptRecords();
-    if (skipCorrupt == null) {
-      skipCorrupt = OrcConf.SKIP_CORRUPT_DATA.getBoolean(fileReader.conf);
-    }
-
-    reader = TreeReaderFactory.createTreeReader(treeReaderSchema.getReaderSchema(),
-        treeReaderSchema, included, skipCorrupt);
-    indexes = new OrcProto.RowIndex[types.size()];
-    bloomFilterIndices = new OrcProto.BloomFilterIndex[types.size()];
-    advanceToNextRow(reader, 0L, true);
   }
 
-  public static final class PositionProviderImpl implements PositionProvider {
-    private final OrcProto.RowIndexEntry entry;
-    private int index;
-
-    public PositionProviderImpl(OrcProto.RowIndexEntry entry) {
-      this(entry, 0);
+  @Override
+  public Object next(Object previous) throws IOException {
+    if (!ensureBatch()) {
+      return null;
     }
-
-    public PositionProviderImpl(OrcProto.RowIndexEntry entry, int startPos) {
-      this.entry = entry;
-      this.index = startPos;
+    if (schema.getCategory() == TypeDescription.Category.STRUCT) {
+      OrcStruct result;
+      List<TypeDescription> children = schema.getChildren();
+      int numberOfChildren = children.size();
+      if (previous == null || previous.getClass() != OrcStruct.class) {
+        result = new OrcStruct(numberOfChildren);
+        previous = result;
+      } else {
+        result = (OrcStruct) previous;
+        if (result.getNumFields() != numberOfChildren) {
+          result.setNumFields(numberOfChildren);
+        }
+      }
+      for(int i=0; i < numberOfChildren; ++i) {
+        result.setFieldValue(i, nextValue(batch.cols[i], rowInBatch,
+            children.get(i), result.getFieldValue(i)));
+      }
+    } else {
+      previous = nextValue(batch.cols[0], rowInBatch, schema, previous);
     }
+    rowInBatch += 1;
+    return previous;
+  }
 
-    @Override
-    public long getNext() {
-      return entry.getPositions(index++);
+  public boolean nextBatch(VectorizedRowBatch theirBatch) throws IOException {
+    // If the user hasn't been reading by row, use the fast path.
+    if (rowInBatch >= batch.size) {
+      return super.nextBatch(theirBatch);
     }
+    copyIntoBatch(theirBatch, batch, rowInBatch);
+    rowInBatch += theirBatch.size;
+    return theirBatch.size > 0;
   }
 
-  OrcProto.StripeFooter readStripeFooter(StripeInformation stripe) throws IOException {
-    return dataReader.readStripeFooter(stripe);
+  @Override
+  public void close() throws IOException {
+    super.close();
+    // free the memory for the column vectors
+    batch.cols = null;
   }
 
-  enum Location {
-    BEFORE, MIN, MIDDLE, MAX, AFTER
-  }
+  /* Routines for stubbing into Writables */
 
-  /**
-   * Given a point and min and max, determine if the point is before, at the
-   * min, in the middle, at the max, or after the range.
-   * @param point the point to test
-   * @param min the minimum point
-   * @param max the maximum point
-   * @param <T> the type of the comparision
-   * @return the location of the point
-   */
-  static <T> Location compareToRange(Comparable<T> point, T min, T max) {
-    int minCompare = point.compareTo(min);
-    if (minCompare < 0) {
-      return Location.BEFORE;
-    } else if (minCompare == 0) {
-      return Location.MIN;
+  static BooleanWritable nextBoolean(ColumnVector vector,
+                                     int row,
+                                     Object previous) {
+    if (vector.isRepeating) {
+      row = 0;
     }
-    int maxCompare = point.compareTo(max);
-    if (maxCompare > 0) {
-      return Location.AFTER;
-    } else if (maxCompare == 0) {
-      return Location.MAX;
+    if (vector.noNulls || !vector.isNull[row]) {
+      BooleanWritable result;
+      if (previous == null || previous.getClass() != BooleanWritable.class) {
+        result = new BooleanWritable();
+      } else {
+        result = (BooleanWritable) previous;
+      }
+      result.set(((LongColumnVector) vector).vector[row] != 0);
+      return result;
+    } else {
+      return null;
     }
-    return Location.MIDDLE;
   }
 
-  /**
-   * Get the maximum value out of an index entry.
-   * @param index
-   *          the index entry
-   * @return the object for the maximum value or null if there isn't one
-   */
-  static Object getMax(ColumnStatistics index) {
-    if (index instanceof IntegerColumnStatistics) {
-      return ((IntegerColumnStatistics) index).getMaximum();
-    } else if (index instanceof DoubleColumnStatistics) {
-      return ((DoubleColumnStatistics) index).getMaximum();
-    } else if (index instanceof StringColumnStatistics) {
-      return ((StringColumnStatistics) index).getMaximum();
-    } else if (index instanceof DateColumnStatistics) {
-      return ((DateColumnStatistics) index).getMaximum();
-    } else if (index instanceof DecimalColumnStatistics) {
-      return ((DecimalColumnStatistics) index).getMaximum();
-    } else if (index instanceof TimestampColumnStatistics) {
-      return ((TimestampColumnStatistics) index).getMaximum();
-    } else if (index instanceof BooleanColumnStatistics) {
-      if (((BooleanColumnStatistics)index).getTrueCount()!=0) {
-        return Boolean.TRUE;
+  static ByteWritable nextByte(ColumnVector vector,
+                               int row,
+                               Object previous) {
+    if (vector.isRepeating) {
+      row = 0;
+    }
+    if (vector.noNulls || !vector.isNull[row]) {
+      ByteWritable result;
+      if (previous == null || previous.getClass() != ByteWritable.class) {
+        result = new ByteWritable();
       } else {
-        return Boolean.FALSE;
+        result = (ByteWritable) previous;
       }
+      result.set((byte) ((LongColumnVector) vector).vector[row]);
+      return result;
     } else {
       return null;
     }
   }
 
-  /**
-   * Get the minimum value out of an index entry.
-   * @param index
-   *          the index entry
-   * @return the object for the minimum value or null if there isn't one
-   */
-  static Object getMin(ColumnStatistics index) {
-    if (index instanceof IntegerColumnStatistics) {
-      return ((IntegerColumnStatistics) index).getMinimum();
-    } else if (index instanceof DoubleColumnStatistics) {
-      return ((DoubleColumnStatistics) index).getMinimum();
-    } else if (index instanceof StringColumnStatistics) {
-      return ((StringColumnStatistics) index).getMinimum();
-    } else if (index instanceof DateColumnStatistics) {
-      return ((DateColumnStatistics) index).getMinimum();
-    } else if (index instanceof DecimalColumnStatistics) {
-      return ((DecimalColumnStatistics) index).getMinimum();
-    } else if (index instanceof TimestampColumnStatistics) {
-      return ((TimestampColumnStatistics) index).getMinimum();
-    } else if (index instanceof BooleanColumnStatistics) {
-      if (((BooleanColumnStatistics)index).getFalseCount()!=0) {
-        return Boolean.FALSE;
+  static ShortWritable nextShort(ColumnVector vector,
+                                 int row,
+                                 Object previous) {
+    if (vector.isRepeating) {
+      row = 0;
+    }
+    if (vector.noNulls || !vector.isNull[row]) {
+      ShortWritable result;
+      if (previous == null || previous.getClass() != ShortWritable.class) {
+        result = new ShortWritable();
       } else {
-        return Boolean.TRUE;
+        result = (ShortWritable) previous;
       }
+      result.set((short) ((LongColumnVector) vector).vector[row]);
+      return result;
     } else {
-      return UNKNOWN_VALUE; // null is not safe here
+      return null;
     }
   }
 
-  /**
-   * Evaluate a predicate with respect to the statistics from the column
-   * that is referenced in the predicate.
-   * @param statsProto the statistics for the column mentioned in the predicate
-   * @param predicate the leaf predicate we need to evaluation
-   * @param bloomFilter
-   * @return the set of truth values that may be returned for the given
-   *   predicate.
-   */
-  static TruthValue evaluatePredicateProto(OrcProto.ColumnStatistics statsProto,
-      PredicateLeaf predicate, OrcProto.BloomFilter bloomFilter) {
-    ColumnStatistics cs = ColumnStatisticsImpl.deserialize(statsProto);
-    Object minValue = getMin(cs);
-    Object maxValue = getMax(cs);
-    BloomFilterIO bf = null;
-    if (bloomFilter != null) {
-      bf = new BloomFilterIO(bloomFilter);
+  static IntWritable nextInt(ColumnVector vector,
+                             int row,
+                             Object previous) {
+    if (vector.isRepeating) {
+      row = 0;
     }
-    return evaluatePredicateRange(predicate, minValue, maxValue, cs.hasNull(), bf);
-  }
-
-  /**
-   * Evaluate a predicate with respect to the statistics from the column
-   * that is referenced in the predicate.
-   * @param stats the statistics for the column mentioned in the predicate
-   * @param predicate the leaf predicate we need to evaluation
-   * @return the set of truth values that may be returned for the given
-   *   predicate.
-   */
-  static TruthValue evaluatePredicate(ColumnStatistics stats,
-      PredicateLeaf predicate, BloomFilterIO bloomFilter) {
-    Object minValue = getMin(stats);
-    Object maxValue = getMax(stats);
-    return evaluatePredicateRange(predicate, minValue, maxValue, stats.hasNull(), bloomFilter);
-  }
-
-  static TruthValue evaluatePredicateRange(PredicateLeaf predicate, Object min,
-      Object max, boolean hasNull, BloomFilterIO bloomFilter) {
-    // if we didn't have any values, everything must have been null
-    if (min == null) {
-      if (predicate.getOperator() == PredicateLeaf.Operator.IS_NULL) {
-        return TruthValue.YES;
+    if (vector.noNulls || !vector.isNull[row]) {
+      IntWritable result;
+      if (previous == null || previous.getClass() != IntWritable.class) {
+        result = new IntWritable();
       } else {
-        return TruthValue.NULL;
+        result = (IntWritable) previous;
       }
-    } else if (min == UNKNOWN_VALUE) {
-      return TruthValue.YES_NO_NULL;
+      result.set((int) ((LongColumnVector) vector).vector[row]);
+      return result;
+    } else {
+      return null;
     }
+  }
 
-    TruthValue result;
-    Object baseObj = predicate.getLiteral();
-    try {
-      // Predicate object and stats objects are converted to the type of the predicate object.
-      Object minValue = getBaseObjectForComparison(predicate.getType(), min);
-      Object maxValue = getBaseObjectForComparison(predicate.getType(), max);
-      Object predObj = getBaseObjectForComparison(predicate.getType(), baseObj);
-
-      result = evaluatePredicateMinMax(predicate, predObj, minValue, maxValue, hasNull);
-      if (shouldEvaluateBloomFilter(predicate, result, bloomFilter)) {
-        result = evaluatePredicateBloomFilter(predicate, predObj, bloomFilter, hasNull);
-      }
-      // in case failed conversion, return the default YES_NO_NULL truth value
-    } catch (Exception e) {
-      if (LOG.isWarnEnabled()) {
-        final String statsType = min == null ?
-            (max == null ? "null" : max.getClass().getSimpleName()) :
-            min.getClass().getSimpleName();
-        final String predicateType = baseObj == null ? "null" : baseObj.getClass().getSimpleName();
-        final String reason = e.getClass().getSimpleName() + " when evaluating predicate." +
-            " Skipping ORC PPD." +
-            " Exception: " + e.getMessage() +
-            " StatsType: " + statsType +
-            " PredicateType: " + predicateType;
-        LOG.warn(reason);
-        LOG.debug(reason, e);
-      }
-      if (predicate.getOperator().equals(PredicateLeaf.Operator.NULL_SAFE_EQUALS) || !hasNull) {
-        result = TruthValue.YES_NO;
+  static LongWritable nextLong(ColumnVector vector,
+                               int row,
+                               Object previous) {
+    if (vector.isRepeating) {
+      row = 0;
+    }
+    if (vector.noNulls || !vector.isNull[row]) {
+      LongWritable result;
+      if (previous == null || previous.getClass() != LongWritable.class) {
+        result = new LongWritable();
       } else {
-        result = TruthValue.YES_NO_NULL;
+        result = (LongWritable) previous;
       }
+      result.set(((LongColumnVector) vector).vector[row]);
+      return result;
+    } else {
+      return null;
     }
-    return result;
   }
 
-  private static boolean shouldEvaluateBloomFilter(PredicateLeaf predicate,
-      TruthValue result, BloomFilterIO bloomFilter) {
-    // evaluate bloom filter only when
-    // 1) Bloom filter is available
-    // 2) Min/Max evaluation yield YES or MAYBE
-    // 3) Predicate is EQUALS or IN list
-    if (bloomFilter != null
-        && result != TruthValue.NO_NULL && result != TruthValue.NO
-        && (predicate.getOperator().equals(PredicateLeaf.Operator.EQUALS)
-            || predicate.getOperator().equals(PredicateLeaf.Operator.NULL_SAFE_EQUALS)
-            || predicate.getOperator().equals(PredicateLeaf.Operator.IN))) {
-      return true;
+  static FloatWritable nextFloat(ColumnVector vector,
+                                 int row,
+                                 Object previous) {
+    if (vector.isRepeating) {
+      row = 0;
     }
-    return false;
-  }
-
-  private static TruthValue evaluatePredicateMinMax(PredicateLeaf predicate, Object predObj,
-      Object minValue,
-      Object maxValue,
-      boolean hasNull) {
-    Location loc;
-
-    switch (predicate.getOperator()) {
-      case NULL_SAFE_EQUALS:
-        loc = compareToRange((Comparable) predObj, minValue, maxValue);
-        if (loc == Location.BEFORE || loc == Location.AFTER) {
-          return TruthValue.NO;
-        } else {
-          return TruthValue.YES_NO;
-        }
-      case EQUALS:
-        loc = compareToRange((Comparable) predObj, minValue, maxValue);
-        if (minValue.equals(maxValue) && loc == Location.MIN) {
-          return hasNull ? TruthValue.YES_NULL : TruthValue.YES;
-        } else if (loc == Location.BEFORE || loc == Location.AFTER) {
-          return hasNull ? TruthValue.NO_NULL : TruthValue.NO;
-        } else {
-          return hasNull ? TruthValue.YES_NO_NULL : TruthValue.YES_NO;
-        }
-      case LESS_THAN:
-        loc = compareToRange((Comparable) predObj, minValue, maxValue);
-        if (loc == Location.AFTER) {
-          return hasNull ? TruthValue.YES_NULL : TruthValue.YES;
-        } else if (loc == Location.BEFORE || loc == Location.MIN) {
-          return hasNull ? TruthValue.NO_NULL : TruthValue.NO;
-        } else {
-          return hasNull ? TruthValue.YES_NO_NULL : TruthValue.YES_NO;
-        }
-      case LESS_THAN_EQUALS:
-        loc = compareToRange((Comparable) predObj, minValue, maxValue);
-        if (loc == Location.AFTER || loc == Location.MAX) {
-          return hasNull ? TruthValue.YES_NULL : TruthValue.YES;
-        } else if (loc == Location.BEFORE) {
-          return hasNull ? TruthValue.NO_NULL : TruthValue.NO;
-        } else {
-          return hasNull ? TruthValue.YES_NO_NULL : TruthValue.YES_NO;
-        }
-      case IN:
-        if (minValue.equals(maxValue)) {
-          // for a single value, look through to see if that value is in the
-          // set
-          for (Object arg : predicate.getLiteralList()) {
-            predObj = getBaseObjectForComparison(predicate.getType(), arg);
-            loc = compareToRange((Comparable) predObj, minValue, maxValue);
-            if (loc == Location.MIN) {
-              return hasNull ? TruthValue.YES_NULL : TruthValue.YES;
-            }
-          }
-          return hasNull ? TruthValue.NO_NULL : TruthValue.NO;
-        } else {
-          // are all of the values outside of the range?
-          for (Object arg : predicate.getLiteralList()) {
-            predObj = getBaseObjectForComparison(predicate.getType(), arg);
-            loc = compareToRange((Comparable) predObj, minValue, maxValue);
-            if (loc == Location.MIN || loc == Location.MIDDLE ||
-                loc == Location.MAX) {
-              return hasNull ? TruthValue.YES_NO_NULL : TruthValue.YES_NO;
-            }
-          }
-          return hasNull ? TruthValue.NO_NULL : TruthValue.NO;
-        }
-      case BETWEEN:
-        List<Object> args = predicate.getLiteralList();
-        Object predObj1 = getBaseObjectForComparison(predicate.getType(), args.get(0));
-
-        loc = compareToRange((Comparable) predObj1, minValue, maxValue);
-        if (loc == Location.BEFORE || loc == Location.MIN) {
-          Object predObj2 = getBaseObjectForComparison(predicate.getType(), args.get(1));
-
-          Location loc2 = compareToRange((Comparable) predObj2, minValue, maxValue);
-          if (loc2 == Location.AFTER || loc2 == Location.MAX) {
-            return hasNull ? TruthValue.YES_NULL : TruthValue.YES;
-          } else if (loc2 == Location.BEFORE) {
-            return hasNull ? TruthValue.NO_NULL : TruthValue.NO;
-          } else {
-            return hasNull ? TruthValue.YES_NO_NULL : TruthValue.YES_NO;
-          }
-        } else if (loc == Location.AFTER) {
-          return hasNull ? TruthValue.NO_NULL : TruthValue.NO;
-        } else {
-          return hasNull ? TruthValue.YES_NO_NULL : TruthValue.YES_NO;
-        }
-      case IS_NULL:
-        // min = null condition above handles the all-nulls YES case
-        return hasNull ? TruthValue.YES_NO : TruthValue.NO;
-      default:
-        return hasNull ? TruthValue.YES_NO_NULL : TruthValue.YES_NO;
+    if (vector.noNulls || !vector.isNull[row]) {
+      FloatWritable result;
+      if (previous == null || previous.getClass() != FloatWritable.class) {
+        result = new FloatWritable();
+      } else {
+        result = (FloatWritable) previous;
+      }
+      result.set((float) ((DoubleColumnVector) vector).vector[row]);
+      return result;
+    } else {
+      return null;
     }
   }
 
-  private static TruthValue evaluatePredicateBloomFilter(PredicateLeaf predicate,
-      final Object predObj, BloomFilterIO bloomFilter, boolean hasNull) {
-    switch (predicate.getOperator()) {
-      case NULL_SAFE_EQUALS:
-        // null safe equals does not return *_NULL variant. So set hasNull to false
-        return checkInBloomFilter(bloomFilter, predObj, false);
-      case EQUALS:
-        return checkInBloomFilter(bloomFilter, predObj, hasNull);
-      case IN:
-        for (Object arg : predicate.getLiteralList()) {
-          // if atleast one value in IN list exist in bloom filter, qualify the row group/stripe
-          Object predObjItem = getBaseObjectForComparison(predicate.getType(), arg);
-          TruthValue result = checkInBloomFilter(bloomFilter, predObjItem, hasNull);
-          if (result == TruthValue.YES_NO_NULL || result == TruthValue.YES_NO) {
-            return result;
-          }
-        }
-        return hasNull ? TruthValue.NO_NULL : TruthValue.NO;
-      default:
-        return hasNull ? TruthValue.YES_NO_NULL : TruthValue.YES_NO;
+  static DoubleWritable nextDouble(ColumnVector vector,
+                                   int row,
+                                   Object previous) {
+    if (vector.isRepeating) {
+      row = 0;
     }
-  }
-
-  private static TruthValue checkInBloomFilter(BloomFilterIO bf, Object predObj, boolean hasNull) {
-    TruthValue result = hasNull ? TruthValue.NO_NULL : TruthValue.NO;
-
-    if (predObj instanceof Long) {
-      if (bf.testLong(((Long) predObj).longValue())) {
-        result = TruthValue.YES_NO_NULL;
-      }
-    } else if (predObj instanceof Double) {
-      if (bf.testDouble(((Double) predObj).doubleValue())) {
-        result = TruthValue.YES_NO_NULL;
-      }
-    } else if (predObj instanceof String || predObj instanceof Text ||
-        predObj instanceof HiveDecimalWritable ||
-        predObj instanceof BigDecimal) {
-      if (bf.testString(predObj.toString())) {
-        result = TruthValue.YES_NO_NULL;
-      }
-    } else if (predObj instanceof Timestamp) {
-      if (bf.testLong(((Timestamp) predObj).getTime())) {
-        result = TruthValue.YES_NO_NULL;
-      }
-    } else if (predObj instanceof TimestampWritable) {
-      if (bf.testLong(((TimestampWritable) predObj).getTimestamp().getTime())) {
-        result = TruthValue.YES_NO_NULL;
-      }
-    } else if (predObj instanceof Date) {
-      if (bf.testLong(DateWritable.dateToDays((Date) predObj))) {
-        result = TruthValue.YES_NO_NULL;
+    if (vector.noNulls || !vector.isNull[row]) {
+      DoubleWritable result;
+      if (previous == null || previous.getClass() != DoubleWritable.class) {
+        result = new DoubleWritable();
+      } else {
+        result = (DoubleWritable) previous;
       }
+      result.set(((DoubleColumnVector) vector).vector[row]);
+      return result;
     } else {
-        // if the predicate object is null and if hasNull says there are no nulls then return NO
-        if (predObj == null && !hasNull) {
-          result = TruthValue.NO;
-        } else {
-          result = TruthValue.YES_NO_NULL;
-        }
-      }
-
-    if (result == TruthValue.YES_NO_NULL && !hasNull) {
-      result = TruthValue.YES_NO;
-    }
-
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Bloom filter evaluation: " + result.toString());
+      return null;
     }
-
-    return result;
   }
 
-  private static Object getBaseObjectForComparison(PredicateLeaf.Type type, Object obj) {
-    if (obj == null) {
-      return null;
+  static Text nextString(ColumnVector vector,
+                         int row,
+                         Object previous) {
+    if (vector.isRepeating) {
+      row = 0;
     }
-    switch (type) {
-      case BOOLEAN:
-        if (obj instanceof Boolean) {
-          return obj;
-        } else {
-          // will only be true if the string conversion yields "true", all other values are
-          // considered false
-          return Boolean.valueOf(obj.toString());
-        }
-      case DATE:
-        if (obj instanceof Date) {
-          return obj;
-        } else if (obj instanceof String) {
-          return Date.valueOf((String) obj);
-        } else if (obj instanceof Timestamp) {
-          return DateWritable.timeToDate(((Timestamp) obj).getTime() / 1000L);
-        }
-        // always string, but prevent the comparison to numbers (are they days/seconds/milliseconds?)
-        break;
-      case DECIMAL:
-        if (obj instanceof Boolean) {
-          return new HiveDecimalWritable(((Boolean) obj).booleanValue() ?
-              HiveDecimal.ONE : HiveDecimal.ZERO);
-        } else if (obj instanceof Integer) {
-          return new HiveDecimalWritable(((Integer) obj).intValue());
-        } else if (obj instanceof Long) {
-          return new HiveDecimalWritable(((Long) obj));
-        } else if (obj instanceof Float || obj instanceof Double ||
-            obj instanceof String) {
-          return new HiveDecimalWritable(obj.toString());
-        } else if (obj instanceof BigDecimal) {
-          return new HiveDecimalWritable(HiveDecimal.create((BigDecimal) obj));
-        } else if (obj instanceof HiveDecimal) {
-          return new HiveDecimalWritable((HiveDecimal) obj);
-        } else if (obj instanceof HiveDecimalWritable) {
-          return obj;
-        } else if (obj instanceof Timestamp) {
-          return new HiveDecimalWritable(
-              new Double(new TimestampWritable((Timestamp) obj).getDouble()).toString());
-        }
-        break;
-      case FLOAT:
-        if (obj instanceof Number) {
-          // widening conversion
-          return ((Number) obj).doubleValue();
-        } else if (obj instanceof HiveDecimal) {
-          return ((HiveDecimal) obj).doubleValue();
-        } else if (obj instanceof String) {
-          return Double.valueOf(obj.toString());
-        } else if (obj instanceof Timestamp) {
-          return new TimestampWritable((Timestamp)obj).getDouble();
-        } else if (obj instanceof HiveDecimal) {
-          return ((HiveDecimal) obj).doubleValue();
-        } else if (obj instanceof BigDecimal) {
-          return ((BigDecimal) obj).doubleValue();
-        }
-        break;
-      case LONG:
-        if (obj instanceof Number) {
-          // widening conversion
-          return ((Number) obj).longValue();
-        } else if (obj instanceof HiveDecimal) {
-          return ((HiveDecimal) obj).longValue();
-        } else if (obj instanceof String) {
-          return Long.valueOf(obj.toString());
-        }
-        break;
-      case STRING:
-        if (obj != null) {
-          return (obj.toString());
-        }
-        break;
-      case TIMESTAMP:
-        if (obj instanceof Timestamp) {
-          return obj;
-        } else if (obj instanceof Integer) {
-          return TimestampWritable.longToTimestamp(((Number) obj).longValue(), false);
-        } else if (obj instanceof Float) {
-          return TimestampWritable.doubleToTimestamp(((Float) obj).doubleValue());
-        } else if (obj instanceof Double) {
-          return TimestampWritable.doubleToTimestamp(((Double) obj).doubleValue());
-        } else if (obj instanceof HiveDecimal) {
-          return TimestampWritable.decimalToTimestamp((HiveDecimal) obj);
-        } else if (obj instanceof HiveDecimalWritable) {
-          return TimestampWritable.decimalToTimestamp(((HiveDecimalWritable) obj).getHiveDecimal());
-        } else if (obj instanceof Date) {
-          return new Timestamp(((Date) obj).getTime());
-        }
-        // float/double conversion to timestamp is interpreted as seconds whereas integer conversion
-        // to timestamp is interpreted as milliseconds by default. The integer to timestamp casting
-        // is also config driven. The filter operator changes its promotion based on config:
-        // "int.timestamp.conversion.in.seconds". Disable PPD for integer cases.
-        break;
-      default:
-        break;
+    if (vector.noNulls || !vector.isNull[row]) {
+      Text result;
+      if (previous == null || previous.getClass() != Text.class) {
+        result = new Text();
+      } else {
+        result = (Text) previous;
+      }
+      BytesColumnVector bytes = (BytesColumnVector) vector;
+      result.set(bytes.vector[row], bytes.start[row], bytes.length[row]);
+      return result;
+    } else {
+      return null;
     }
-
-    throw new IllegalArgumentException(String.format(
-        "ORC SARGS could not convert from %s to %s", obj == null ? "(null)" : obj.getClass()
-            .getSimpleName(), type));
   }
 
-  public static class SargApplier {
-    public final static boolean[] READ_ALL_RGS = null;
-    public final static boolean[] READ_NO_RGS = new boolean[0];
-
-    private final SearchArgument sarg;
-    private final List<PredicateLeaf> sargLeaves;
-    private final int[] filterColumns;
-    private final long rowIndexStride;
-    // same as the above array, but indices are set to true
-    private final boolean[] sargColumns;
-
-    public SargApplier(SearchArgument sarg, String[] columnNames, long rowIndexStride,
-        List<OrcProto.Type> types, int includedCount) {
-      this.sarg = sarg;
-      sargLeaves = sarg.getLeaves();
-      filterColumns = mapSargColumnsToOrcInternalColIdx(sargLeaves, columnNames, 0);
-      this.rowIndexStride = rowIndexStride;
-      // included will not be null, row options will fill the array with trues if null
-      sargColumns = new boolean[includedCount];
-      for (int i : filterColumns) {
-        // filter columns may have -1 as index which could be partition column in SARG.
-        if (i > 0) {
-          sargColumns[i] = true;
-        }
-      }
+  static HiveCharWritable nextChar(ColumnVector vector,
+                                   int row,
+                                   int size,
+                                   Object previous) {
+    if (vector.isRepeating) {
+      row = 0;
     }
-
-    /**
-     * Pick the row groups that we need to load from the current stripe.
-     *
-     * @return an array with a boolean for each row group or null if all of the
-     * row groups must be read.
-     * @throws IOException
-     */
-    public boolean[] pickRowGroups(StripeInformation stripe, OrcProto.RowIndex[] indexes,
-        OrcProto.BloomFilterIndex[] bloomFilterIndices, boolean returnNone) throws IOException {
-      long rowsInStripe = stripe.getNumberOfRows();
-      int groupsInStripe = (int) ((rowsInStripe + rowIndexStride - 1) / rowIndexStride);
-      boolean[] result = new boolean[groupsInStripe]; // TODO: avoid alloc?
-      TruthValue[] leafValues = new TruthValue[sargLeaves.size()];
-      boolean hasSelected = false, hasSkipped = false;
-      for (int rowGroup = 0; rowGroup < result.length; ++rowGroup) {
-        for (int pred = 0; pred < leafValues.length; ++pred) {
-          int columnIx = filterColumns[pred];
-          if (columnIx != -1) {
-            if (indexes[columnIx] == null) {
-              throw new AssertionError("Index is not populated for " + columnIx);
-            }
-            OrcProto.RowIndexEntry entry = indexes[columnIx].getEntry(rowGroup);
-            if (entry == null) {
-              throw new AssertionError("RG is not populated for " + columnIx + " rg " + rowGroup);
-            }
-            OrcProto.ColumnStatistics stats = entry.getStatistics();
-            OrcProto.BloomFilter bf = null;
-            if (bloomFilterIndices != null && bloomFilterIndices[filterColumns[pred]] != null) {
-              bf = bloomFilterIndices[filterColumns[pred]].getBloomFilter(rowGroup);
-            }
-            leafValues[pred] = evaluatePredicateProto(stats, sargLeaves.get(pred), bf);
-            if (LOG.isTraceEnabled()) {
-              LOG.trace("Stats = " + stats);
-              LOG.trace("Setting " + sargLeaves.get(pred) + " to " + leafValues[pred]);
-            }
-          } else {
-            // the column is a virtual column
-            leafValues[pred] = TruthValue.YES_NO_NULL;
-          }
-        }
-        result[rowGroup] = sarg.evaluate(leafValues).isNeeded();
-        hasSelected = hasSelected || result[rowGroup];
-        hasSkipped = hasSkipped || (!result[rowGroup]);
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Row group " + (rowIndexStride * rowGroup) + " to " +
-              (rowIndexStride * (rowGroup + 1) - 1) + " is " +
-              (result[rowGroup] ? "" : "not ") + "included.");
-        }
+    if (vector.noNulls || !vector.isNull[row]) {
+      HiveCharWritable result;
+      if (previous == null || previous.getClass() != HiveCharWritable.class) {
+        result = new HiveCharWritable();
+      } else {
+        result = (HiveCharWritable) previous;
       }
-
-      return hasSkipped ? ((hasSelected || !returnNone) ? result : READ_NO_RGS) : READ_ALL_RGS;
+      BytesColumnVector bytes = (BytesColumnVector) vector;
+      result.set(bytes.toString(row), size);
+      return result;
+    } else {
+      return null;
     }
   }
 
-  /**
-   * Pick the row groups that we need to load from the current stripe.
-   *
-   * @return an array with a boolean for each row group or null if all of the
-   * row groups must be read.
-   * @throws IOException
-   */
-  protected boolean[] pickRowGroups() throws IOException {
-    // if we don't have a sarg or indexes, we read everything
-    if (sargApp == null) {
+  static HiveVarcharWritable nextVarchar(ColumnVector vector,
+                                         int row,
+                                         int size,
+                                         Object previous) {
+    if (vector.isRepeating) {
+      row = 0;
+    }
+    if (vector.noNulls || !vector.isNull[row]) {
+      HiveVarcharWritable result;
+      if (previous == null || previous.getClass() != HiveVarcharWritable.class) {
+        result = new HiveVarcharWritable();
+      } else {
+        result = (HiveVarcharWritable) previous;
+      }
+      BytesColumnVector bytes = (BytesColumnVector) vector;
+      result.set(bytes.toString(row), size);
+      return result;
+    } else {
       return null;
     }
-    readRowIndex(currentStripe, included, sargApp.sargColumns);
-    return sargApp.pickRowGroups(stripes.get(currentStripe), indexes, bloomFilterIndices, false);
   }
 
-  private void clearStreams()  {
-    // explicit close of all streams to de-ref ByteBuffers
-    for (InStream is : streams.values()) {
-      is.close();
+  static BytesWritable nextBinary(ColumnVector vector,
+                                  int row,
+                                  Object previous) {
+    if (vector.isRepeating) {
+      row = 0;
     }
-    if (bufferChunks != null) {
-      if (dataReader.isTrackingDiskRanges()) {
-        for (DiskRangeList range = bufferChunks; range != null; range = range.next) {
-          if (!(range instanceof BufferChunk)) {
-            continue;
-          }
-          dataReader.releaseBuffer(((BufferChunk) range).getChunk());
-        }
+    if (vector.noNulls || !vector.isNull[row]) {
+      BytesWritable result;
+      if (previous == null || previous.getClass() != BytesWritable.class) {
+        result = new BytesWritable();
+      } else {
+        result = (BytesWritable) previous;
       }
+      BytesColumnVector bytes = (BytesColumnVector) vector;
+      result.set(bytes.vector[row], bytes.start[row], bytes.length[row]);
+      return result;
+    } else {
+      return null;
     }
-    bufferChunks = null;
-    streams.clear();
   }
 
-  /**
-   * Read the current stripe into memory.
-   *
-   * @throws IOException
-   */
-  private void readStripe() throws IOException {
-    StripeInformation stripe = beginReadStripe();
-    includedRowGroups = pickRowGroups();
-
-    // move forward to the first unskipped row
-    if (includedRowGroups != null) {
-      while (rowInStripe < rowCountInStripe &&
-          !includedRowGroups[(int) (rowInStripe / rowIndexStride)]) {
-        rowInStripe = Math.min(rowCountInStripe, rowInStripe + rowIndexStride);
-      }
+  static HiveDecimalWritable nextDecimal(ColumnVector vector,
+                                         int row,
+                                         Object previous) {
+    if (vector.isRepeating) {
+      row = 0;
     }
-
-    // if we haven't skipped the whole stripe, read the data
-    if (rowInStripe < rowCountInStripe) {
-      // if we aren't projecting columns or filtering rows, just read it all
-      if (included == null && includedRowGroups == null) {
-        readAllDataStreams(stripe);
+    if (vector.noNulls || !vector.isNull[row]) {
+      HiveDecimalWritable result;
+      if (previous == null || previous.getClass() != HiveDecimalWritable.class) {
+        result = new HiveDecimalWritable();
       } else {
-        readPartialDataStreams(stripe);
-      }
-      reader.startStripe(streams, stripeFooter);
-      // if we skipped the first row group, move the pointers forward
-      if (rowInStripe != 0) {
-        seekToRowEntry(reader, (int) (rowInStripe / rowIndexStride));
+        result = (HiveDecimalWritable) previous;
       }
+      result.set(((DecimalColumnVector) vector).vector[row]);
+      return result;
+    } else {
+      return null;
     }
   }
 
-  private StripeInformation beginReadStripe() throws IOException {
-    StripeInformation stripe = stripes.get(currentStripe);
-    stripeFooter = readStripeFooter(stripe);
-    clearStreams();
-    // setup the position in the stripe
-    rowCountInStripe = stripe.getNumberOfRows();
-    rowInStripe = 0;
-    rowBaseInStripe = 0;
-    for (int i = 0; i < currentStripe; ++i) {
-      rowBaseInStripe += stripes.get(i).getNumberOfRows();
+  static DateWritable nextDate(ColumnVector vector,
+                               int row,
+                               Object previous) {
+    if (vector.isRepeating) {
+      row = 0;
     }
-    // reset all of the indexes
-    for (int i = 0; i < indexes.length; ++i) {
-      indexes[i] = null;
+    if (vector.noNulls || !vector.isNull[row]) {
+      DateWritable result;
+      if (previous == null || previous.getClass() != DateWritable.class) {
+        result = new DateWritable();
+      } else {
+        result = (DateWritable) previous;
+      }
+      int date = (int) ((LongColumnVector) vector).vector[row];
+      result.set(date);
+      return result;
+    } else {
+      return null;
     }
-    return stripe;
   }
 
-  private void readAllDataStreams(StripeInformation stripe) throws IOException {
-    long start = stripe.getIndexLength();
-    long end = start + stripe.getDataLength();
-    // explicitly trigger 1 big read
-    DiskRangeList toRead = new DiskRangeList(start, end);
-    bufferChunks = dataReader.readFileData(toRead, stripe.getOffset(), false);
-    List<OrcProto.Stream> streamDescriptions = stripeFooter.getStreamsList();
-    createStreams(streamDescriptions, bufferChunks, null, codec, bufferSize, streams);
-  }
-
-  /**
-   * Plan the ranges of the file that we need to read given the list of
-   * columns and row groups.
-   *
-   * @param streamList        the list of streams available
-   * @param indexes           the indexes that have been loaded
-   * @param includedColumns   which columns are needed
-   * @param includedRowGroups which row groups are needed
-   * @param isCompressed      does the file have generic compression
-   * @param encodings         the encodings for each column
-   * @param types             the types of the columns
-   * @param compressionSize   the compression block size
-   * @return the list of disk ranges that will be loaded
-   */
-  static DiskRangeList planReadPartialDataStreams
-  (List<OrcProto.Stream> streamList,
-      OrcProto.RowIndex[] indexes,
-      boolean[] includedColumns,
-      boolean[] includedRowGroups,
-      boolean isCompressed,
-      List<OrcProto.ColumnEncoding> encodings,
-      List<OrcProto.Type> types,
-      int compressionSize,
-      boolean doMergeBuffers) {
-    long offset = 0;
-    // figure out which columns have a present stream
-    boolean[] hasNull = RecordReaderUtils.findPresentStreamsByColumn(streamList, types);
-    CreateHelper list = new CreateHelper();
-    for (OrcProto.Stream stream : streamList) {
-      long length = stream.getLength();
-      int column = stream.getColumn();
-      OrcProto.Stream.Kind streamKind = stream.getKind();
-      // since stream kind is optional, first check if it exists
-      if (stream.hasKind() &&
-          (StreamName.getArea(streamKind) == StreamName.Area.DATA) &&
-          (column < includedColumns.length && includedColumns[column])) {
-        // if we aren't filtering or it is a dictionary, load it.
-        if (includedRowGroups == null
-            || RecordReaderUtils.isDictionary(streamKind, encodings.get(column))) {
-          RecordReaderUtils.addEntireStreamToRanges(offset, length, list, doMergeBuffers);
-        } else {
-          RecordReaderUtils.addRgFilteredStreamToRanges(stream, includedRowGroups,
-              isCompressed, indexes[column], encodings.get(column), types.get(column),
-              compressionSize, hasNull[column], offset, length, list, doMergeBuffers);
-        }
+  static TimestampWritable nextTimestamp(ColumnVector vector,
+                                         int row,
+                                         Object previous) {
+    if (vector.isRepeating) {
+      row = 0;
+    }
+    if (vector.noNulls || !vector.isNull[row]) {
+      TimestampWritable result;
+      if (previous == null || previous.getClass() != TimestampWritable.class) {
+        result = new TimestampWritable();
+      } else {
+        result = (TimestampWritable) previous;
       }
-      offset += length;
+      TimestampColumnVector tcv = (TimestampColumnVector) vector;
+      result.setInternal(tcv.time[row], tcv.nanos[row]);
+      return result;
+    } else {
+      return null;
     }
-    return list.extract();
   }
 
-  void createStreams(List<OrcProto.Stream> streamDescriptions,
-      DiskRangeList ranges,
-      boolean[] includeColumn,
-      CompressionCodec codec,
-      int bufferSize,
-      Map<StreamName, InStream> streams) throws IOException {
-    long streamOffset = 0;
-    for (OrcProto.Stream streamDesc : streamDescriptions) {
-      int column = streamDesc.getColumn();
-      if ((includeColumn != null &&
-          (column < included.length && !includeColumn[column])) ||
-          streamDesc.hasKind() &&
-              (StreamName.getArea(streamDesc.getKind()) != StreamName.Area.DATA)) {
-        streamOffset += streamDesc.getLength();
-        continue;
+  static OrcStruct nextStruct(ColumnVector vector,
+                              int row,
+                              TypeDescription schema,
+                              Object previous) {
+    if (vector.isRepeating) {
+      row = 0;
+    }
+    if (vector.noNulls || !vector.isNull[row]) {
+      OrcStruct result;
+      List<TypeDescription> childrenTypes = schema.getChildren();
+      int numChildren = childrenTypes.size();
+      if (previous == null || previous.getClass() != OrcStruct.class) {
+        result = new OrcStruct(numChildren);
+      } else {
+        result = (OrcStruct) previous;
+        result.setNumFields(numChildren);
+      }
+      StructColumnVector struct = (StructColumnVector) vector;
+      for(int f=0; f < numChildren; ++f) {
+        result.setFieldValue(f, nextValue(struct.fields[f], row,
+            childrenTypes.get(f), result.getFieldValue(f)));
       }
-      List<DiskRange> buffers = RecordReaderUtils.getStreamBuffers(
-          ranges, streamOffset, streamDesc.getLength());
-      StreamName name = new StreamName(column, streamDesc.getKind());
-      streams.put(name, InStream.create(name.toString(), buffers,
-          streamDesc.getLength(), codec, bufferSize));
-      streamOffset += streamDesc.getLength();
+      return result;
+    } else {
+      return null;
     }
   }
 
-  private void readPartialDataStreams(StripeInformation stripe) throws IOException {
-    List<OrcProto.Stream> streamList = stripeFooter.getStreamsList();
-    DiskRangeList toRead = planReadPartialDataStreams(streamList,
-        indexes, included, includedRowGroups, codec != null,
-        stripeFooter.getColumnsList(), types, bufferSize, true);
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("chunks = " + RecordReaderUtils.stringifyDiskRanges(toRead));
+  static OrcUnion nextUnion(ColumnVector vector,
+                            int row,
+                            TypeDescription schema,
+                            Object previous) {
+    if (vector.isRepeating) {
+      row = 0;
     }
-    bufferChunks = dataReader.readFileData(toRead, stripe.getOffset(), false);
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("merge = " + RecordReaderUtils.stringifyDiskRanges(bufferChunks));
+    if (vector.noNulls || !vector.isNull[row]) {
+      OrcUnion result;
+      List<TypeDescription> childrenTypes = schema.getChildren();
+      if (previous == null || previous.getClass() != OrcUnion.class) {
+        result = new OrcUnion();
+      } else {
+        result = (OrcUnion) previous;
+      }
+      UnionColumnVector union = (UnionColumnVector) vector;
+      byte tag = (byte) union.tags[row];
+      result.set(tag, nextValue(union.fields[tag], row, childrenTypes.get(tag),
+          result.getObject()));
+      return result;
+    } else {
+      return null;
     }
-
-    createStreams(streamList, bufferChunks, included, codec, bufferSize, streams);
-  }
-
-  @Override
-  public boolean hasNext() throws IOException {
-    return rowInStripe < rowCountInStripe;
   }
 
-  /**
-   * Read the next stripe until we find a row that we don't skip.
-   *
-   * @throws IOException
-   */
-  private void advanceStripe() throws IOException {
-    rowInStripe = rowCountInStripe;
-    while (rowInStripe >= rowCountInStripe &&
-        currentStripe < stripes.size() - 1) {
-      currentStripe += 1;
-      readStripe();
+  static ArrayList<Object> nextList(ColumnVector vector,
+                                    int row,
+                                    TypeDescription schema,
+                                    Object previous) {
+    if (vector.isRepeating) {
+      row = 0;
     }
-  }
-
-  /**
-   * Skip over rows that we aren't selecting, so that the next row is
-   * one that we will read.
-   *
-   * @param nextRow the row we want to go to
-   * @throws IOException
-   */
-  private boolean advanceToNextRow(
-      TreeReaderFactory.TreeReader reader, long nextRow, boolean canAdvanceStripe)
-      throws IOException {
-    long nextRowInStripe = nextRow - rowBaseInStripe;
-    // check for row skipping
-    if (rowIndexStride != 0 &&
-        includedRowGroups != null &&
-        nextRowInStripe < rowCountInStripe) {
-      int rowGroup = (int) (nextRowInStripe / rowIndexStride);
-      if (!includedRowGroups[rowGroup]) {
-        while (rowGroup < includedRowGroups.length && !includedRowGroups[rowGroup]) {
-          rowGroup += 1;
-        }
-        if (rowGroup >= includedRowGroups.length) {
-          if (canAdvanceStripe) {
-            advanceStripe();
-          }
-          return canAdvanceStripe;
+    if (vector.noNulls || !vector.isNull[row]) {
+      ArrayList<Object> result;
+      if (previous == null || previous.getClass() != ArrayList.class) {
+        result = new ArrayList<>();
+      } else {
+        result = (ArrayList<Object>) previous;
+      }
+      ListColumnVector list = (ListColumnVector) vector;
+      int length = (int) list.lengths[row];
+      int offset = (int) list.offsets[row];
+      result.ensureCapacity(length);
+      int oldLength = result.size();
+      int idx = 0;
+      TypeDescription childType = schema.getChildren().get(0);
+      while (idx < length && idx < oldLength) {
+        result.set(idx, nextValue(list.child, offset + idx, childType,
+            result.get(idx)));
+        idx += 1;
+      }
+      if (length < oldLength) {
+        result.subList(length,result.size()).clear();
+      } else if (oldLength < length) {
+        while (idx < length) {
+          result.add(nextValue(list.child, offset + idx, childType, null));
+          idx += 1;
         }
-        nextRowInStripe = Math.min(rowCountInStripe, rowGroup * rowIndexStride);
       }
+      return result;
+    } else {
+      return null;
     }
-    if (nextRowInStripe >= rowCountInStripe) {
-      if (canAdvanceStripe) {
-        advanceStripe();
-      }
-      return canAdvanceStripe;
+  }
+
+  static HashMap<Object,Object> nextMap(ColumnVector vector,
+                                        int row,
+                                        TypeDescription schema,
+                                        Object previous) {
+    if (vector.isRepeating) {
+      row = 0;
     }
-    if (nextRowInStripe != rowInStripe) {
-      if (rowIndexStride != 0) {
-        int rowGroup = (int) (nextRowInStripe / rowIndexStride);
-        seekToRowEntry(reader, rowGroup);
-        reader.skipRows(nextRowInStripe - rowGroup * rowIndexStride);
+    if (vector.noNulls || !vector.isNull[row]) {
+      MapColumnVector map = (MapColumnVector) vector;
+      int length = (int) map.lengths[row];
+      int offset = (int) map.offsets[row];
+      TypeDescription keyType = schema.getChildren().get(0);
+      TypeDescription valueType = schema.getChildren().get(1);
+      HashMap<Object,Object> result;
+      if (previous == null || previous.getClass() != HashMap.class) {
+        result = new HashMap<Object,Object>(length);
       } else {
-        reader.skipRows(nextRowInStripe - rowInStripe);
+        result = (HashMap<Object,Object>) previous;
+        // I couldn't think of a good way to reuse the keys and value objects
+        // without even more allocations, so take the easy and safe approach.
+        result.clear();
       }
-      rowInStripe = nextRowInStripe;
+      for(int e=0; e < length; ++e) {
+        result.put(nextValue(map.keys, e + offset, keyType, null),
+                   nextValue(map.values, e + offset, valueType, null));
+      }
+      return result;
+    } else {
+      return null;
     }
-    return true;
   }
 
-  @Override
-  public Object next(Object previous) throws IOException {
-    try {
-      final Object result = reader.next(previous);
-      // find the next row
-      rowInStripe += 1;
-      advanceToNextRow(reader, rowInStripe + rowBaseInStripe, true);
-      return result;
-    } catch (IOException e) {
-      // Rethrow exception with file name in log message
-      throw new IOException("Error reading file: " + path, e);
+  static Object nextValue(ColumnVector vector,
+                          int row,
+                          TypeDescription schema,
+                          Object previous) {
+    switch (schema.getCategory()) {
+      case BOOLEAN:
+        return nextBoolean(vector, row, previous);
+      case BYTE:
+        return nextByte(vector, row, previous);
+      case SHORT:
+        return nextShort(vector, row, previous);
+      case INT:
+        return nextInt(vector, row, previous);
+      case LONG:
+        return nextLong(vector, row, previous);
+      case FLOAT:
+        return nextFloat(vector, row, previous);
+      case DOUBLE:
+        return nextDouble(vector, row, previous);
+      case STRING:
+        return nextString(vector, row, previous);
+      case CHAR:
+        return nextChar(vector, row, schema.getMaxLength(), previous);
+      case VARCHAR:
+        return nextVarchar(vector, row, schema.getMaxLength(), previous);
+      case BINARY:
+        return nextBinary(vector, row, previous);
+      case DECIMAL:
+        return nextDecimal(vector, row, previous);
+      case DATE:
+        return nextDate(vector, row, previous);
+      case TIMESTAMP:
+        return nextTimestamp(vector, row, previous);
+      case STRUCT:
+        return nextStruct(vector, row, schema, previous);
+      case UNION:
+        return nextUnion(vector, row, schema, previous);
+      case LIST:
+        return nextList(vector, row, schema, previous);
+      case MAP:
+        return nextMap(vector, row, schema, previous);
+      default:
+        throw new IllegalArgumentException("Unknown type " + schema);
     }
   }
 
-  @Override
-  public boolean nextBatch(VectorizedRowBatch batch) throws IOException {
-    try {
-      if (rowInStripe >= rowCountInStripe) {
-        currentStripe += 1;
-        if (currentStripe >= stripes.size()) {
-          batch.size = 0;
-          return false;
+  /* Routines for copying between VectorizedRowBatches */
+
+  void copyLongColumn(ColumnVector destination,
+                      ColumnVector source,
+                      int sourceOffset,
+                      int length) {
+    LongColumnVector lsource = (LongColumnVector) source;
+    LongColumnVector ldest = (LongColumnVector) destination;
+    ldest.isRepeating = lsource.isRepeating;
+    ldest.noNulls = lsource.noNulls;
+    if (source.isRepeating) {
+      ldest.isNull[0] = lsource.isNull[0];
+      ldest.vector[0] = lsource.vector[0];
+    } else {
+      if (!lsource.noNulls) {
+        for(int r=0; r < length; ++r) {
+          ldest.isNull[r] = lsource.isNull[sourceOffset + r];
+          ldest.vector[r] = lsource.vector[sourceOffset + r];
+        }
+      } else {
+        for (int r = 0; r < length; ++r) {
+          ldest.vector[r] = lsource.vector[sourceOffset + r];
         }
-        readStripe();
       }
-
-      int batchSize = computeBatchSize(batch.getMaxSize());
-
-      rowInStripe += batchSize;
-      reader.setVectorColumnCount(batch.getDataColumnCount());
-      reader.nextBatch(batch, batchSize);
-
-      batch.size = (int) batchSize;
-      batch.selectedInUse = false;
-      advanceToNextRow(reader, rowInStripe + rowBaseInStripe, true);
-      return batch.size  != 0;
-    } catch (IOException e) {
-      // Rethrow exception with file name in log message
-      throw new IOException("Error reading file: " + path, e);
     }
   }
 
-  private int computeBatchSize(long targetBatchSize) {
-    final int batchSize;
-    // In case of PPD, batch size should be aware of row group boundaries. If only a subset of row
-    // groups are selected then marker position is set to the end of range (subset of row groups
-    // within strip). Batch size computed out of marker position makes sure that batch size is
-    // aware of row group boundary and will not cause overflow when reading rows
-    // illustration of this case is here https://issues.apache.org/jira/browse/HIVE-6287
-    if (rowIndexStride != 0 && includedRowGroups != null && rowInStripe < rowCountInStripe) {
-      int startRowGroup = (int) (rowInStripe / rowIndexStride);
-      if (!includedRowGroups[startRowGroup]) {
-        while (startRowGroup < includedRowGroups.length && !includedRowGroups[startRowGroup]) {
-          startRowGroup += 1;
+  void copyDoubleColumn(ColumnVector destination,
+                        ColumnVector source,
+                        int sourceOffset,
+                        int length) {
+    DoubleColumnVector castedSource = (DoubleColumnVector) source;
+    DoubleColumnVector castedDestination = (DoubleColumnVector) destination;
+    if (source.isRepeating) {
+      castedDestination.isRepeating = true;
+      castedDestination.noNulls = castedSource.noNulls;
+      castedDestination.isNull[0] = castedSource.isNull[0];
+      castedDestination.vector[0] = castedSource.vector[0];
+    } else {
+      if (!castedSource.noNulls) {
+        castedDestination.noNulls = true;
+        for(int r=0; r < length; ++r) {
+          castedDestination.isNull[r] = castedSource.isNull[sourceOffset + r];
         }
       }
-
-      int endRowGroup = startRowGroup;
-      while (endRowGroup < includedRowGroups.length && includedRowGroups[endRowGroup]) {
-        endRowGroup += 1;
-      }
-
-      final long markerPosition =
-          (endRowGroup * rowIndexStride) < rowCountInStripe ? (endRowGroup * rowIndexStride)
-              : rowCountInStripe;
-      batchSize = (int) Math.min(targetBatchSize, (markerPosition - rowInStripe));
-
-      if (isLogDebugEnabled && batchSize < targetBatchSize) {
-        LOG.debug("markerPosition: " + markerPosition + " batchSize: " + batchSize);
+      for(int r=0; r < length; ++r) {
+        castedDestination.vector[r] = castedSource.vector[sourceOffset + r];
       }
-    } else {
-      batchSize = (int) Math.min(targetBatchSize, (rowCountInStripe - rowInStripe));
     }
-    return batchSize;
-  }
-
-  @Override
-  public void close() throws IOException {
-    clearStreams();
-    dataReader.close();
-  }
-
-  @Override
-  public long getRowNumber() {
-    return rowInStripe + rowBaseInStripe + firstRow;
-  }
-
-  /**
-   * Return the fraction of rows that have been read from the selected.
-   * section of the file
-   *
-   * @return fraction between 0.0 and 1.0 of rows consumed
-   */
-  @Override
-  public float getProgress() {
-    return ((float) rowBaseInStripe + rowInStripe) / totalRowCount;
   }
 
-  private int findStripe(long rowNumber) {
-    for (int i = 0; i < stripes.size(); i++) {
-      StripeInformation stripe = stripes.get(i);
-      if (stripe.getNumberOfRows() > rowNumber) {
-        return i;
+  void copyTimestampColumn(ColumnVector destination,
+                           ColumnVector source,
+                           int sourceOffset,
+                           int length) {
+    TimestampColumnVector castedSource = (TimestampColumnVector) source;
+    TimestampColumnVector castedDestination = (TimestampColumnVector) destination;
+    castedDestination.isRepeating = castedSource.isRepeating;
+    castedDestination.noNulls = castedSource.noNulls;
+    if (source.isRepeating) {
+      castedDestination.isNull[0] = castedSource.isNull[0];
+      castedDestination.time[0] = castedSource.time[0];
+      castedDestination.nanos[0] = castedSource.nanos[0];
+    } else {
+      if (!castedSource.noNulls) {
+        castedDestination.noNulls = true;
+        for(int r=0; r < length; ++r) {
+          castedDestination.isNull[r] = castedSource.isNull[sourceOffset + r];
+          castedDestination.time[r] = castedSource.time[sourceOffset + r];
+          castedDestination.nanos[r] = castedSource.nanos[sourceOffset + r];
+        }
+      } else {
+        for (int r = 0; r < length; ++r) {
+          castedDestination.time[r] = castedSource.time[sourceOffset + r];
+          castedDestination.nanos[r] = castedSource.nanos[sourceOffset + r];
+        }
       }
-      rowNumber -= stripe.getNumberOfRows();
     }
-    throw new IllegalArgumentException("Seek after the end of reader range");
   }
 
-  OrcIndex readRowIndex(
-      int stripeIndex, boolean[] included, boolean[] sargColumns) throws IOException {
-    return readRowIndex(stripeIndex, included, null, null, sargColumns);
+  void copyDecimalColumn(ColumnVector destination,
+                         ColumnVector source,
+                         int sourceOffset,
+                         int length) {
+    DecimalColumnVector castedSource = (DecimalColumnVector) source;
+    DecimalColumnVector castedDestination = (DecimalColumnVector) destination;
+    castedDestination.isRepeating = castedSource.isRepeating;
+    castedDestination.noNulls = castedSource.noNulls;
+    if (source.isRepeating) {
+      castedDestination.isNull[0] = castedSource.isNull[0];
+      if (!castedSource.isNull[0]) {
+        castedDestination.set(0, castedSource.vector[0]);
+      }
+    } else {
+      if (!castedSource.noNulls) {
+        for(int r=0; r < length; ++r) {
+          castedDestination.isNull[r] = castedSource.isNull[sourceOffset + r];
+          if (!castedDestination.isNull[r]) {
+            castedDestination.set(r, castedSource.vector[r]);
+          }
+        }
+      } else {
+        for (int r = 0; r < length; ++r) {
+          castedDestination.set(r, castedSource.vector[r]);
+        }
+      }
+    }
   }
 
-  OrcIndex readRowIndex(int stripeIndex, boolean[] included, OrcProto.RowIndex[] indexes,
-      OrcProto.BloomFilterIndex[] bloomFilterIndex, boolean[] sargColumns) throws IOException {
-    StripeInformation stripe = stripes.get(stripeIndex);
-    OrcProto.StripeFooter stripeFooter = null;
-    // if this is the current stripe, use the cached objects.
-    if (stripeIndex == currentStripe) {
-      stripeFooter = this.stripeFooter;
-      indexes = indexes == null ? this.indexes : indexes;
-      bloomFilterIndex = bloomFilterIndex == null ? this.bloomFilterIndices : bloomFilterIndex;
-      sargColumns = sargColumns == null ?
-          (sargApp == null ? null : sargApp.sargColumns) : sargColumns;
+  void copyBytesColumn(ColumnVector destination,
+                       ColumnVector source,
+                       int sourceOffset,
+                       int length) {
+    BytesColumnVector castedSource = (BytesColumnVector) source;
+    BytesColumnVector castedDestination = (BytesColumnVector) destination;
+    castedDestination.isRepeating = castedSource.isRepeating;
+    castedDestination.noNulls = castedSource.noNulls;
+    if (source.isRepeating) {
+      castedDestination.isNull[0] = castedSource.isNull[0];
+      if (!castedSource.isNull[0]) {
+        castedDestination.setVal(0, castedSource.vector[0],
+            castedSource.start[0], castedSource.length[0]);
+      }
+    } else {
+      if (!castedSource.noNulls) {
+        for(int r=0; r < length; ++r) {
+          castedDestination.isNull[r] = castedSource.isNull[sourceOffset + r];
+          if (!castedDestination.isNull[r]) {
+            castedDestination.setVal(r, castedSource.vector[sourceOffset + r],
+                castedSource.start[sourceOffset + r],
+                castedSource.length[sourceOffset + r]);
+          }
+        }
+      } else {
+        for (int r = 0; r < length; ++r) {
+          castedDestination.setVal(r, castedSource.vector[sourceOffset + r],
+              castedSource.start[sourceOffset + r],
+              castedSource.length[sourceOffset + r]);
+        }
+      }
     }
-    return dataReader.readRowIndex(stripe, stripeFooter, included, indexes,
-        sargColumns, bloomFilterIndex);
   }
 
-  private void seekToRowEntry(TreeReaderFactory.TreeReader reader, int rowEntry)
-      throws IOException {
-    PositionProvider[] index = new PositionProvider[indexes.length];
-    for (int i = 0; i < indexes.length; ++i) {
-      if (indexes[i] != null) {
-        index[i] = new PositionProviderImpl(indexes[i].getEntry(rowEntry));
+  void copyStructColumn(ColumnVector destination,
+                        ColumnVector source,
+                        int sourceOffset,
+                        int length) {
+    StructColumnVector castedSource = (StructColumnVector) source;
+    StructColumnVector castedDestination = (StructColumnVector) destination;
+    castedDestination.isRepeating = castedSource.isRepeating;
+    castedDestination.noNulls = castedSource.noNulls;
+    if (source.isRepeating) {
+      castedDestination.isNull[0] = castedSource.isNull[0];
+      for(int c=0; c > castedSource.fields.length; ++c) {
+        copyColumn(castedDestination.fields[c], castedSource.fields[c], 0, 1);
+      }
+    } else {
+      if (!castedSource.noNulls) {
+        for (int r = 0; r < length; ++r) {
+          castedDestination.isNull[r] = castedSource.isNull[sourceOffset + r];
+        }
+      } else {
+        for (int c = 0; c > castedSource.fields.length; ++c) {
+          copyColumn(castedDestination.fields[c], castedSource.fields[c],
+              sourceOffset, length);
+        }
       }
     }
-    reader.seek(index);
   }
 
-  @Override
-  public void seekToRow(long rowNumber) throws IOException {
-    if (rowNumber < 0) {
-      throw new IllegalArgumentException("Seek to a negative row number " +
-          rowNumber);
-    } else if (rowNumber < firstRow) {
-      throw new IllegalArgumentException("Seek before reader range " +
-          rowNumber);
+  void copyUnionColumn(ColumnVector destination,
+                       ColumnVector source,
+                       int sourceOffset,
+                       int length) {
+    UnionColumnVector castedSource = (UnionColumnVector) source;
+    UnionColumnVector castedDestination = (UnionColumnVector) destination;
+    castedDestination.isRepeating = castedSource.isRepeating;
+    castedDestination.noNulls = castedSource.noNulls;
+    if (source.isRepeating) {
+      castedDestination.isNull[0] = castedSource.isNull[0];
+      int tag = castedSource.tags[0];
+      castedDestination.tags[0] = tag;
+      if (!castedDestination.isNull[0]) {
+        copyColumn(castedDestination.fields[tag], castedSource.fields[tag], 0,
+            1);
+      }
+    } else {
+      if (!castedSource.noNulls) {
+        for (int r = 0; r < length; ++r) {
+          castedDestination.isNull[r] = castedSource.isNull[sourceOffset + r];
+          castedDestination.tags[r] = castedSource.tags[sourceOffset + r];
+        }
+      } else {
+        for(int r=0; r < length; ++r) {
+          castedDestination.tags[r] = castedSource.tags[sourceOffset + r];
+        }
+      }
+      for(int c=0; c > castedSource.fields.length; ++c) {
+        copyColumn(castedDestination.fields[c], castedSource.fields[c],
+            sourceOffset, length);
+      }
     }
-    // convert to our internal form (rows from the beginning of slice)
-    rowNumber -= firstRow;
+  }
 
-    // move to the right stripe
-    int rightStripe = findStripe(rowNumber);
-    if (rightStripe != currentStripe) {
-      currentStripe = rightStripe;
-      readStripe();
+  void copyListColumn(ColumnVector destination,
+                       ColumnVector source,
+                       int sourceOffset,
+                       int length) {
+    ListColumnVector castedSource = (ListColumnVector) source;
+    ListColumnVector castedDestination = (ListColumnVector) destination;
+    castedDestination.isRepeating = castedSource.noNulls;
+    castedDestination.noNulls = castedSource.noNulls;
+    if (source.isRepeating) {
+      castedDestination.isNull[0] = castedSource.isNull[0];
+      castedDestination.offsets[0] = 0;
+      castedDestination.lengths[0] = castedSource.lengths[0];
+      copyColumn(castedDestination.child, castedSource.child,
+          (int) castedSource.offsets[0], (int) castedSource.lengths[0]);
+    } else {
+      if (!castedSource.noNulls) {
+        for (int r = 0; r < length; ++r) {
+          castedDestination.isNull[r] = castedSource.isNull[sourceOffset + r];
+        }
+      }
+      int minOffset = Integer.MAX_VALUE;
+      int maxOffset = Integer.MIN_VALUE;
+      for(int r=0; r < length; ++r) {
+        int childOffset = (int) castedSource.offsets[r + sourceOffset];
+        int childLength = (int) castedSource.lengths[r + sourceOffset];
+        castedDestination.offsets[r] = childOffset;
+        castedDestination.lengths[r] = childLength;
+        minOffset = Math.min(minOffset, childOffset);
+        maxOffset = Math.max(maxOffset, childOffset + childLength);
+      }
+      if (minOffset <= maxOffset) {
+        castedDestination.childCount = maxOffset - minOffset + 1;
+        copyColumn(castedDestination.child, castedSource.child,
+            minOffset, castedDestination.childCount);
+      } else {
+        castedDestination.childCount = 0;
+      }
+    }
+  }
+
+  void copyMapColumn(ColumnVector destination,
+                     ColumnVector source,
+                     int sourceOffset,
+                     int length) {
+    MapColumnVector castedSource = (MapColumnVector) source;
+    MapColumnVector castedDestination = (MapColumnVector) destination;
+    castedDestination.isRepeating = castedSource.noNulls;
+    castedDestination.noNulls = castedSource.noNulls;
+    if (source.isRepeating) {
+      castedDestination.isNull[0] = castedSource.isNull[0];
+      castedDestination.offsets[0] = 0;
+      castedDestination.lengths[0] = castedSource.lengths[0];
+      copyColumn(castedDestination.keys, castedSource.keys,
+          (int) castedSource.offsets[0], (int) castedSource.lengths[0]);
+      copyColumn(castedDestination.values, castedSource.values,
+          (int) castedSource.offsets[0], (int) castedSource.lengths[0]);
+    } else {
+      if (!castedSource.noNulls) {
+        for (int r = 0; r < length; ++r) {
+          castedDestination.isNull[r] = castedSource.isNull[sourceOffset + r];
+        }
+      }
+      int minOffset = Integer.MAX_VALUE;
+      int maxOffset = Integer.MIN_VALUE;
+      for(int r=0; r < length; ++r) {
+        int childOffset = (int) castedSource.offsets[r + sourceOffset];
+        int childLength = (int) castedSource.lengths[r + sourceOffset];
+        castedDestination.offsets[r] = childOffset;
+        castedDestination.lengths[r] = childLength;
+        minOffset = Math.min(minOffset, childOffset);
+        maxOffset = Math.max(maxOffset, childOffset + childLength);
+      }
+      if (minOffset <= maxOffset) {
+        castedDestination.childCount = maxOffset - minOffset + 1;
+        copyColumn(castedDestination.keys, castedSource.keys,
+            minOffset, castedDestination.childCount);
+        copyColumn(castedDestination.values, castedSource.values,
+            minOffset, castedDestination.childCount);
+      } else {
+        castedDestination.childCount = 0;
+      }
     }
-    readRowIndex(currentStripe, included, sargApp == null ? null : sargApp.sargColumns);
-
-    // if we aren't to the right row yet, advance in the stripe.
-    advanceToNextRow(reader, rowNumber, true);
   }
 
-  private static final String TRANSLATED_SARG_SEPARATOR = "_";
-  public static String encodeTranslatedSargColumn(int rootColumn, Integer indexInSourceTable) {
-    return rootColumn + TRANSLATED_SARG_SEPARATOR
-        + ((indexInSourceTable == null) ? -1 : indexInSourceTable);
+  void copyColumn(ColumnVector destination,
+                  ColumnVector source,
+                  int sourceOffset,
+                  int length) {
+    if (source.getClass() == LongColumnVector.class) {
+      copyLongColumn(destination, source, sourceOffset, length);
+    } else if (source.getClass() == DoubleColumnVector.class) {
+      copyDoubleColumn(destination, source, sourceOffset, length);
+    } else if (source.getClass() == BytesColumnVector.class) {
+      copyBytesColumn(destination, source, sourceOffset, length);
+    } else if (source.getClass() == TimestampColumnVector.class) {
+      copyTimestampColumn(destination, source, sourceOffset, length);
+    } else if (source.getClass() == DecimalColumnVector.class) {
+      copyDecimalColumn(destination, source, sourceOffset, length);
+    } else if (source.getClass() == StructColumnVector.class) {
+      copyStructColumn(destination, source, sourceOffset, length);
+    } else if (source.getClass() == UnionColumnVector.class) {
+      copyUnionColumn(destination, source, sourceOffset, length);
+    } else if (source.getClass() == ListColumnVector.class) {
+      copyListColumn(destination, source, sourceOffset, length);
+    } else if (source.getClass() == MapColumnVector.class) {
+      copyMapColumn(destination, source, sourceOffset, length);
+    }
   }
 
-  public static int[] mapTranslatedSargColumns(
-      List<OrcProto.Type> types, List<PredicateLeaf> sargLeaves) {
-    int[] result = new int[sargLeaves.size()];
-    OrcProto.Type lastRoot = null; // Root will be the same for everyone as of now.
-    String lastRootStr = null;
-    for (int i = 0; i < result.length; ++i) {
-      String[] rootAndIndex = sargLeaves.get(i).getColumnName().split(TRANSLATED_SARG_SEPARATOR);
-      assert rootAndIndex.length == 2;
-      String rootStr = rootAndIndex[0], indexStr = rootAndIndex[1];
-      int index = Integer.parseInt(indexStr);
-      // First, check if the column even maps to anything.
-      if (index == -1) {
-        result[i] = -1;
-        continue;
-      }
-      assert index >= 0;
-      // Then, find the root type if needed.
-      if (!rootStr.equals(lastRootStr)) {
-        lastRoot = types.get(Integer.parseInt(rootStr));
-        lastRootStr = rootStr;
-      }
-      // Subtypes of the root types correspond, in order, to the columns in the table schema
-      // (disregarding schema evolution that doesn't presently work). Get the index for the
-      // corresponding subtype.
-      result[i] = lastRoot.getSubtypes(index);
-    }
-    return result;
+  /**
+   * Copy part of a batch into the destination batch.
+   * @param destination the batch to copy into
+   * @param source the batch to copy from
+   * @param sourceStart the row number to start from in the source
+   * @return the number of rows copied
+   */
+  void copyIntoBatch(VectorizedRowBatch destination,
+                     VectorizedRowBatch source,
+                     int sourceStart) {
+    int rows = Math.min(source.size - sourceStart, destination.getMaxSize());
+    for(int c=0; c < source.cols.length; ++c) {
+      destination.cols[c].reset();
+      copyColumn(destination.cols[c], source.cols[c], sourceStart, rows);
+    }
+    destination.size = rows;
   }
 }


[25/27] hive git commit: HIVE-11417. Move the ReaderImpl and RowReaderImpl to the ORC module, by making shims for the row by row reader. (omalley reviewed by prasanth_j)

Posted by om...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/orc/src/java/org/apache/orc/impl/HadoopShims.java
----------------------------------------------------------------------
diff --git a/orc/src/java/org/apache/orc/impl/HadoopShims.java b/orc/src/java/org/apache/orc/impl/HadoopShims.java
index 2980d71..ef7d70f 100644
--- a/orc/src/java/org/apache/orc/impl/HadoopShims.java
+++ b/orc/src/java/org/apache/orc/impl/HadoopShims.java
@@ -18,9 +18,13 @@
 
 package org.apache.orc.impl;
 
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.io.Text;
 import org.apache.hadoop.util.VersionInfo;
 
+import java.io.Closeable;
 import java.io.IOException;
+import java.io.InputStream;
 import java.nio.ByteBuffer;
 
 public interface HadoopShims {
@@ -43,6 +47,81 @@ public interface HadoopShims {
    */
   DirectDecompressor getDirectDecompressor(DirectCompressionType codec);
 
+  /**
+   * a hadoop.io ByteBufferPool shim.
+   */
+  public interface ByteBufferPoolShim {
+    /**
+     * Get a new ByteBuffer from the pool.  The pool can provide this from
+     * removing a buffer from its internal cache, or by allocating a
+     * new buffer.
+     *
+     * @param direct     Whether the buffer should be direct.
+     * @param length     The minimum length the buffer will have.
+     * @return           A new ByteBuffer. Its capacity can be less
+     *                   than what was requested, but must be at
+     *                   least 1 byte.
+     */
+    ByteBuffer getBuffer(boolean direct, int length);
+
+    /**
+     * Release a buffer back to the pool.
+     * The pool may choose to put this buffer into its cache/free it.
+     *
+     * @param buffer    a direct bytebuffer
+     */
+    void putBuffer(ByteBuffer buffer);
+  }
+
+  /**
+   * Provides an HDFS ZeroCopyReader shim.
+   * @param in FSDataInputStream to read from (where the cached/mmap buffers are tied to)
+   * @param in ByteBufferPoolShim to allocate fallback buffers with
+   *
+   * @return returns null if not supported
+   */
+  public ZeroCopyReaderShim getZeroCopyReader(FSDataInputStream in, ByteBufferPoolShim pool) throws IOException;
+
+  public interface ZeroCopyReaderShim extends Closeable {
+    /**
+     * Get a ByteBuffer from the FSDataInputStream - this can be either a HeapByteBuffer or an MappedByteBuffer.
+     * Also move the in stream by that amount. The data read can be small than maxLength.
+     *
+     * @return ByteBuffer read from the stream,
+     */
+    public ByteBuffer readBuffer(int maxLength, boolean verifyChecksums) throws IOException;
+    /**
+     * Release a ByteBuffer obtained from a read on the
+     * Also move the in stream by that amount. The data read can be small than maxLength.
+     *
+     */
+    public void releaseBuffer(ByteBuffer buffer);
+
+    /**
+     * Close the underlying stream.
+     * @throws IOException
+     */
+    public void close() throws IOException;
+  }
+  /**
+   * Read data into a Text object in the fastest way possible
+   */
+  public interface TextReaderShim {
+    /**
+     * @param txt
+     * @param size
+     * @return bytes read
+     * @throws IOException
+     */
+    void read(Text txt, int size) throws IOException;
+  }
+
+  /**
+   * Wrap a TextReaderShim around an input stream. The reader shim will not
+   * buffer any reads from the underlying stream and will only consume bytes
+   * which are required for TextReaderShim.read() input.
+   */
+  public TextReaderShim getTextReaderShim(InputStream input) throws IOException;
 
   class Factory {
     private static HadoopShims SHIMS = null;

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/orc/src/java/org/apache/orc/impl/HadoopShimsCurrent.java
----------------------------------------------------------------------
diff --git a/orc/src/java/org/apache/orc/impl/HadoopShimsCurrent.java b/orc/src/java/org/apache/orc/impl/HadoopShimsCurrent.java
index 3b9371d..5c53f74 100644
--- a/orc/src/java/org/apache/orc/impl/HadoopShimsCurrent.java
+++ b/orc/src/java/org/apache/orc/impl/HadoopShimsCurrent.java
@@ -18,10 +18,14 @@
 
 package org.apache.orc.impl;
 
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.compress.snappy.SnappyDecompressor;
 import org.apache.hadoop.io.compress.zlib.ZlibDecompressor;
 
+import java.io.DataInputStream;
 import java.io.IOException;
+import java.io.InputStream;
 import java.nio.ByteBuffer;
 
 /**
@@ -59,4 +63,30 @@ public class HadoopShimsCurrent implements HadoopShims {
         return null;
     }
   }
+
+  @Override
+  public ZeroCopyReaderShim getZeroCopyReader(FSDataInputStream in,
+                                              ByteBufferPoolShim pool
+                                              ) throws IOException {
+    return ZeroCopyShims.getZeroCopyReader(in, pool);
+  }
+
+  private final class FastTextReaderShim implements TextReaderShim {
+    private final DataInputStream din;
+
+    public FastTextReaderShim(InputStream in) {
+      this.din = new DataInputStream(in);
+    }
+
+    @Override
+    public void read(Text txt, int len) throws IOException {
+      txt.readWithKnownLength(din, len);
+    }
+  }
+
+  @Override
+  public TextReaderShim getTextReaderShim(InputStream in) throws IOException {
+    return new FastTextReaderShim(in);
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/orc/src/java/org/apache/orc/impl/HadoopShims_2_2.java
----------------------------------------------------------------------
diff --git a/orc/src/java/org/apache/orc/impl/HadoopShims_2_2.java b/orc/src/java/org/apache/orc/impl/HadoopShims_2_2.java
index ac46836..3f65e74 100644
--- a/orc/src/java/org/apache/orc/impl/HadoopShims_2_2.java
+++ b/orc/src/java/org/apache/orc/impl/HadoopShims_2_2.java
@@ -18,19 +18,84 @@
 
 package org.apache.orc.impl;
 
-import org.apache.hadoop.io.compress.snappy.SnappyDecompressor;
-import org.apache.hadoop.io.compress.zlib.ZlibDecompressor;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.io.Text;
 
+import java.io.EOFException;
 import java.io.IOException;
-import java.nio.ByteBuffer;
+import java.io.InputStream;
+import java.lang.reflect.Method;
 
 /**
  * Shims for versions of Hadoop up to and including 2.2.x
  */
 public class HadoopShims_2_2 implements HadoopShims {
 
+  final boolean zeroCopy;
+  final boolean fastRead;
+
+  HadoopShims_2_2() {
+    boolean zcr = false;
+    try {
+      Class.forName("org.apache.hadoop.fs.CacheFlag", false,
+        HadoopShims_2_2.class.getClassLoader());
+      zcr = true;
+    } catch (ClassNotFoundException ce) {
+    }
+    zeroCopy = zcr;
+    boolean fastRead = false;
+    if (zcr) {
+      for (Method m : Text.class.getMethods()) {
+        if ("readWithKnownLength".equals(m.getName())) {
+          fastRead = true;
+        }
+      }
+    }
+    this.fastRead = fastRead;
+  }
+
   public DirectDecompressor getDirectDecompressor(
       DirectCompressionType codec) {
     return null;
   }
+
+  @Override
+  public ZeroCopyReaderShim getZeroCopyReader(FSDataInputStream in,
+                                              ByteBufferPoolShim pool
+                                              ) throws IOException {
+    if(zeroCopy) {
+      return ZeroCopyShims.getZeroCopyReader(in, pool);
+    }
+    /* not supported */
+    return null;
+  }
+
+  private final class BasicTextReaderShim implements TextReaderShim {
+    private final InputStream in;
+
+    public BasicTextReaderShim(InputStream in) {
+      this.in = in;
+    }
+
+    @Override
+    public void read(Text txt, int len) throws IOException {
+      int offset = 0;
+      byte[] bytes = new byte[len];
+      while (len > 0) {
+        int written = in.read(bytes, offset, len);
+        if (written < 0) {
+          throw new EOFException("Can't finish read from " + in + " read "
+              + (offset) + " bytes out of " + bytes.length);
+        }
+        len -= written;
+        offset += written;
+      }
+      txt.set(bytes);
+    }
+  }
+
+  @Override
+  public TextReaderShim getTextReaderShim(InputStream in) throws IOException {
+    return new BasicTextReaderShim(in);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/orc/src/java/org/apache/orc/impl/IntegerReader.java
----------------------------------------------------------------------
diff --git a/orc/src/java/org/apache/orc/impl/IntegerReader.java b/orc/src/java/org/apache/orc/impl/IntegerReader.java
index 8bef0f1..3e64d54 100644
--- a/orc/src/java/org/apache/orc/impl/IntegerReader.java
+++ b/orc/src/java/org/apache/orc/impl/IntegerReader.java
@@ -78,4 +78,5 @@ public interface IntegerReader {
   void nextVector(ColumnVector column,
                   int[] data,
                   int length
-                  ) throws IOException;}
+                  ) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/orc/src/java/org/apache/orc/impl/OrcAcidUtils.java
----------------------------------------------------------------------
diff --git a/orc/src/java/org/apache/orc/impl/OrcAcidUtils.java b/orc/src/java/org/apache/orc/impl/OrcAcidUtils.java
new file mode 100644
index 0000000..72c7f54
--- /dev/null
+++ b/orc/src/java/org/apache/orc/impl/OrcAcidUtils.java
@@ -0,0 +1,85 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.orc.impl;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.orc.Reader;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetDecoder;
+
+public class OrcAcidUtils {
+  public static final String ACID_STATS = "hive.acid.stats";
+  public static final String DELTA_SIDE_FILE_SUFFIX = "_flush_length";
+
+  /**
+   * Get the filename of the ORC ACID side file that contains the lengths
+   * of the intermediate footers.
+   * @param main the main ORC filename
+   * @return the name of the side file
+   */
+  public static Path getSideFile(Path main) {
+    return new Path(main + DELTA_SIDE_FILE_SUFFIX);
+  }
+
+  /**
+   * Read the side file to get the last flush length.
+   * @param fs the file system to use
+   * @param deltaFile the path of the delta file
+   * @return the maximum size of the file to use
+   * @throws IOException
+   */
+  public static long getLastFlushLength(FileSystem fs,
+                                        Path deltaFile) throws IOException {
+    Path lengths = getSideFile(deltaFile);
+    long result = Long.MAX_VALUE;
+    try (FSDataInputStream stream = fs.open(lengths)) {
+      result = -1;
+      while (stream.available() > 0) {
+        result = stream.readLong();
+      }
+      return result;
+    } catch (IOException ioe) {
+      return result;
+    }
+  }
+
+  private static final Charset utf8 = Charset.forName("UTF-8");
+  private static final CharsetDecoder utf8Decoder = utf8.newDecoder();
+
+  public static AcidStats parseAcidStats(Reader reader) {
+    if (reader.hasMetadataValue(ACID_STATS)) {
+      try {
+        ByteBuffer val = reader.getMetadataValue(ACID_STATS).duplicate();
+        return new AcidStats(utf8Decoder.decode(val).toString());
+      } catch (CharacterCodingException e) {
+        throw new IllegalArgumentException("Bad string encoding for " +
+            ACID_STATS, e);
+      }
+    } else {
+      return null;
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/orc/src/java/org/apache/orc/impl/ReaderImpl.java
----------------------------------------------------------------------
diff --git a/orc/src/java/org/apache/orc/impl/ReaderImpl.java b/orc/src/java/org/apache/orc/impl/ReaderImpl.java
new file mode 100644
index 0000000..2da590e
--- /dev/null
+++ b/orc/src/java/org/apache/orc/impl/ReaderImpl.java
@@ -0,0 +1,758 @@
+/**
+ * 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.orc.impl;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.orc.OrcFile;
+import org.apache.orc.OrcUtils;
+import org.apache.orc.Reader;
+import org.apache.orc.RecordReader;
+import org.apache.orc.TypeDescription;
+import org.apache.orc.ColumnStatistics;
+import org.apache.orc.CompressionCodec;
+import org.apache.orc.FileFormatException;
+import org.apache.orc.FileMetaInfo;
+import org.apache.orc.FileMetadata;
+import org.apache.orc.StripeInformation;
+import org.apache.orc.StripeStatistics;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.common.io.DiskRange;
+import org.apache.hadoop.hive.ql.util.JavaDataModel;
+import org.apache.hadoop.io.Text;
+import org.apache.orc.OrcProto;
+
+import com.google.common.collect.Lists;
+import com.google.protobuf.CodedInputStream;
+
+public class ReaderImpl implements Reader {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ReaderImpl.class);
+
+  private static final int DIRECTORY_SIZE_GUESS = 16 * 1024;
+
+  protected final FileSystem fileSystem;
+  private final long maxLength;
+  protected final Path path;
+  protected final org.apache.orc.CompressionKind compressionKind;
+  protected final CompressionCodec codec;
+  protected final int bufferSize;
+  private final List<OrcProto.StripeStatistics> stripeStats;
+  private final int metadataSize;
+  protected final List<OrcProto.Type> types;
+  private final TypeDescription schema;
+  private final List<OrcProto.UserMetadataItem> userMetadata;
+  private final List<OrcProto.ColumnStatistics> fileStats;
+  private final List<StripeInformation> stripes;
+  protected final int rowIndexStride;
+  private final long contentLength, numberOfRows;
+
+
+  private long deserializedSize = -1;
+  protected final Configuration conf;
+  private final List<Integer> versionList;
+  private final OrcFile.WriterVersion writerVersion;
+
+  // Same for metastore cache - maintains the same background buffer, but includes postscript.
+  // This will only be set if the file footer/metadata was read from disk.
+  private final ByteBuffer footerMetaAndPsBuffer;
+
+  public static class StripeInformationImpl
+      implements StripeInformation {
+    private final OrcProto.StripeInformation stripe;
+
+    public StripeInformationImpl(OrcProto.StripeInformation stripe) {
+      this.stripe = stripe;
+    }
+
+    @Override
+    public long getOffset() {
+      return stripe.getOffset();
+    }
+
+    @Override
+    public long getLength() {
+      return stripe.getDataLength() + getIndexLength() + getFooterLength();
+    }
+
+    @Override
+    public long getDataLength() {
+      return stripe.getDataLength();
+    }
+
+    @Override
+    public long getFooterLength() {
+      return stripe.getFooterLength();
+    }
+
+    @Override
+    public long getIndexLength() {
+      return stripe.getIndexLength();
+    }
+
+    @Override
+    public long getNumberOfRows() {
+      return stripe.getNumberOfRows();
+    }
+
+    @Override
+    public String toString() {
+      return "offset: " + getOffset() + " data: " + getDataLength() +
+        " rows: " + getNumberOfRows() + " tail: " + getFooterLength() +
+        " index: " + getIndexLength();
+    }
+  }
+
+  @Override
+  public long getNumberOfRows() {
+    return numberOfRows;
+  }
+
+  @Override
+  public List<String> getMetadataKeys() {
+    List<String> result = new ArrayList<String>();
+    for(OrcProto.UserMetadataItem item: userMetadata) {
+      result.add(item.getName());
+    }
+    return result;
+  }
+
+  @Override
+  public ByteBuffer getMetadataValue(String key) {
+    for(OrcProto.UserMetadataItem item: userMetadata) {
+      if (item.hasName() && item.getName().equals(key)) {
+        return item.getValue().asReadOnlyByteBuffer();
+      }
+    }
+    throw new IllegalArgumentException("Can't find user metadata " + key);
+  }
+
+  public boolean hasMetadataValue(String key) {
+    for(OrcProto.UserMetadataItem item: userMetadata) {
+      if (item.hasName() && item.getName().equals(key)) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  @Override
+  public org.apache.orc.CompressionKind getCompressionKind() {
+    return compressionKind;
+  }
+
+  @Override
+  public int getCompressionSize() {
+    return bufferSize;
+  }
+
+  @Override
+  public List<StripeInformation> getStripes() {
+    return stripes;
+  }
+
+  @Override
+  public long getContentLength() {
+    return contentLength;
+  }
+
+  @Override
+  public List<OrcProto.Type> getTypes() {
+    return types;
+  }
+
+  @Override
+  public OrcFile.Version getFileVersion() {
+    for (OrcFile.Version version: OrcFile.Version.values()) {
+      if ((versionList != null && !versionList.isEmpty()) &&
+          version.getMajor() == versionList.get(0) &&
+          version.getMinor() == versionList.get(1)) {
+        return version;
+      }
+    }
+    return OrcFile.Version.V_0_11;
+  }
+
+  @Override
+  public OrcFile.WriterVersion getWriterVersion() {
+    return writerVersion;
+  }
+
+  @Override
+  public int getRowIndexStride() {
+    return rowIndexStride;
+  }
+
+  @Override
+  public ColumnStatistics[] getStatistics() {
+    ColumnStatistics[] result = new ColumnStatistics[types.size()];
+    for(int i=0; i < result.length; ++i) {
+      result[i] = ColumnStatisticsImpl.deserialize(fileStats.get(i));
+    }
+    return result;
+  }
+
+  @Override
+  public TypeDescription getSchema() {
+    return schema;
+  }
+
+  /**
+   * Ensure this is an ORC file to prevent users from trying to read text
+   * files or RC files as ORC files.
+   * @param in the file being read
+   * @param path the filename for error messages
+   * @param psLen the postscript length
+   * @param buffer the tail of the file
+   * @throws IOException
+   */
+  protected static void ensureOrcFooter(FSDataInputStream in,
+                                        Path path,
+                                        int psLen,
+                                        ByteBuffer buffer) throws IOException {
+    int magicLength = OrcFile.MAGIC.length();
+    int fullLength = magicLength + 1;
+    if (psLen < fullLength || buffer.remaining() < fullLength) {
+      throw new FileFormatException("Malformed ORC file " + path +
+          ". Invalid postscript length " + psLen);
+    }
+    int offset = buffer.arrayOffset() + buffer.position() + buffer.limit() - fullLength;
+    byte[] array = buffer.array();
+    // now look for the magic string at the end of the postscript.
+    if (!Text.decode(array, offset, magicLength).equals(OrcFile.MAGIC)) {
+      // If it isn't there, this may be the 0.11.0 version of ORC.
+      // Read the first 3 bytes of the file to check for the header
+      byte[] header = new byte[magicLength];
+      in.readFully(0, header, 0, magicLength);
+      // if it isn't there, this isn't an ORC file
+      if (!Text.decode(header, 0 , magicLength).equals(OrcFile.MAGIC)) {
+        throw new FileFormatException("Malformed ORC file " + path +
+            ". Invalid postscript.");
+      }
+    }
+  }
+
+  /**
+   * Build a version string out of an array.
+   * @param version the version number as a list
+   * @return the human readable form of the version string
+   */
+  private static String versionString(List<Integer> version) {
+    StringBuilder buffer = new StringBuilder();
+    for(int i=0; i < version.size(); ++i) {
+      if (i != 0) {
+        buffer.append('.');
+      }
+      buffer.append(version.get(i));
+    }
+    return buffer.toString();
+  }
+
+  /**
+   * Check to see if this ORC file is from a future version and if so,
+   * warn the user that we may not be able to read all of the column encodings.
+   * @param log the logger to write any error message to
+   * @param path the data source path for error messages
+   * @param version the version of hive that wrote the file.
+   */
+  protected static void checkOrcVersion(Logger log, Path path,
+                                        List<Integer> version) {
+    if (version.size() >= 1) {
+      int major = version.get(0);
+      int minor = 0;
+      if (version.size() >= 2) {
+        minor = version.get(1);
+      }
+      if (major > OrcFile.Version.CURRENT.getMajor() ||
+          (major == OrcFile.Version.CURRENT.getMajor() &&
+           minor > OrcFile.Version.CURRENT.getMinor())) {
+        log.warn(path + " was written by a future Hive version " +
+                 versionString(version) +
+                 ". This file may not be readable by this version of Hive.");
+      }
+    }
+  }
+
+  /**
+  * Constructor that let's the user specify additional options.
+   * @param path pathname for file
+   * @param options options for reading
+   * @throws IOException
+   */
+  public ReaderImpl(Path path, OrcFile.ReaderOptions options) throws IOException {
+    FileSystem fs = options.getFilesystem();
+    if (fs == null) {
+      fs = path.getFileSystem(options.getConfiguration());
+    }
+    this.fileSystem = fs;
+    this.path = path;
+    this.conf = options.getConfiguration();
+    this.maxLength = options.getMaxLength();
+
+    FileMetadata fileMetadata = options.getFileMetadata();
+    if (fileMetadata != null) {
+      this.compressionKind = fileMetadata.getCompressionKind();
+      this.bufferSize = fileMetadata.getCompressionBufferSize();
+      this.codec = WriterImpl.createCodec(compressionKind);
+      this.metadataSize = fileMetadata.getMetadataSize();
+      this.stripeStats = fileMetadata.getStripeStats();
+      this.versionList = fileMetadata.getVersionList();
+      this.writerVersion =
+          OrcFile.WriterVersion.from(fileMetadata.getWriterVersionNum());
+      this.types = fileMetadata.getTypes();
+      this.rowIndexStride = fileMetadata.getRowIndexStride();
+      this.contentLength = fileMetadata.getContentLength();
+      this.numberOfRows = fileMetadata.getNumberOfRows();
+      this.fileStats = fileMetadata.getFileStats();
+      this.stripes = fileMetadata.getStripes();
+      this.userMetadata = null; // not cached and not needed here
+      this.footerMetaAndPsBuffer = null;
+    } else {
+      FileMetaInfo footerMetaData;
+      if (options.getFileMetaInfo() != null) {
+        footerMetaData = options.getFileMetaInfo();
+        this.footerMetaAndPsBuffer = null;
+      } else {
+        footerMetaData = extractMetaInfoFromFooter(fs, path,
+            options.getMaxLength());
+        this.footerMetaAndPsBuffer = footerMetaData.footerMetaAndPsBuffer;
+      }
+      MetaInfoObjExtractor rInfo =
+          new MetaInfoObjExtractor(footerMetaData.compressionType,
+                                   footerMetaData.bufferSize,
+                                   footerMetaData.metadataSize,
+                                   footerMetaData.footerBuffer
+                                   );
+      this.compressionKind = rInfo.compressionKind;
+      this.codec = rInfo.codec;
+      this.bufferSize = rInfo.bufferSize;
+      this.metadataSize = rInfo.metadataSize;
+      this.stripeStats = rInfo.metadata.getStripeStatsList();
+      this.types = rInfo.footer.getTypesList();
+      this.rowIndexStride = rInfo.footer.getRowIndexStride();
+      this.contentLength = rInfo.footer.getContentLength();
+      this.numberOfRows = rInfo.footer.getNumberOfRows();
+      this.userMetadata = rInfo.footer.getMetadataList();
+      this.fileStats = rInfo.footer.getStatisticsList();
+      this.versionList = footerMetaData.versionList;
+      this.writerVersion = footerMetaData.writerVersion;
+      this.stripes = convertProtoStripesToStripes(rInfo.footer.getStripesList());
+    }
+    this.schema = OrcUtils.convertTypeFromProtobuf(this.types, 0);
+  }
+
+  /**
+   * Get the WriterVersion based on the ORC file postscript.
+   * @param writerVersion the integer writer version
+   * @return the version of the software that produced the file
+   */
+  public static OrcFile.WriterVersion getWriterVersion(int writerVersion) {
+    for(OrcFile.WriterVersion version: OrcFile.WriterVersion.values()) {
+      if (version.getId() == writerVersion) {
+        return version;
+      }
+    }
+    return OrcFile.WriterVersion.FUTURE;
+  }
+
+  private static OrcProto.Footer extractFooter(ByteBuffer bb, int footerAbsPos,
+      int footerSize, CompressionCodec codec, int bufferSize) throws IOException {
+    bb.position(footerAbsPos);
+    bb.limit(footerAbsPos + footerSize);
+    return OrcProto.Footer.parseFrom(InStream.createCodedInputStream("footer",
+        Lists.<DiskRange>newArrayList(new BufferChunk(bb, 0)), footerSize, codec, bufferSize));
+  }
+
+  private static OrcProto.Metadata extractMetadata(ByteBuffer bb, int metadataAbsPos,
+      int metadataSize, CompressionCodec codec, int bufferSize) throws IOException {
+    bb.position(metadataAbsPos);
+    bb.limit(metadataAbsPos + metadataSize);
+    return OrcProto.Metadata.parseFrom(InStream.createCodedInputStream("metadata",
+        Lists.<DiskRange>newArrayList(new BufferChunk(bb, 0)), metadataSize, codec, bufferSize));
+  }
+
+  private static OrcProto.PostScript extractPostScript(ByteBuffer bb, Path path,
+      int psLen, int psAbsOffset) throws IOException {
+    // TODO: when PB is upgraded to 2.6, newInstance(ByteBuffer) method should be used here.
+    assert bb.hasArray();
+    CodedInputStream in = CodedInputStream.newInstance(
+        bb.array(), bb.arrayOffset() + psAbsOffset, psLen);
+    OrcProto.PostScript ps = OrcProto.PostScript.parseFrom(in);
+    checkOrcVersion(LOG, path, ps.getVersionList());
+
+    // Check compression codec.
+    switch (ps.getCompression()) {
+      case NONE:
+        break;
+      case ZLIB:
+        break;
+      case SNAPPY:
+        break;
+      case LZO:
+        break;
+      default:
+        throw new IllegalArgumentException("Unknown compression");
+    }
+    return ps;
+  }
+
+  private static FileMetaInfo extractMetaInfoFromFooter(FileSystem fs,
+                                                        Path path,
+                                                        long maxFileLength
+                                                        ) throws IOException {
+    FSDataInputStream file = fs.open(path);
+    ByteBuffer buffer = null, fullFooterBuffer = null;
+    OrcProto.PostScript ps = null;
+    OrcFile.WriterVersion writerVersion = null;
+    try {
+      // figure out the size of the file using the option or filesystem
+      long size;
+      if (maxFileLength == Long.MAX_VALUE) {
+        size = fs.getFileStatus(path).getLen();
+      } else {
+        size = maxFileLength;
+      }
+
+      //read last bytes into buffer to get PostScript
+      int readSize = (int) Math.min(size, DIRECTORY_SIZE_GUESS);
+      buffer = ByteBuffer.allocate(readSize);
+      assert buffer.position() == 0;
+      file.readFully((size - readSize),
+          buffer.array(), buffer.arrayOffset(), readSize);
+      buffer.position(0);
+
+      //read the PostScript
+      //get length of PostScript
+      int psLen = buffer.get(readSize - 1) & 0xff;
+      ensureOrcFooter(file, path, psLen, buffer);
+      int psOffset = readSize - 1 - psLen;
+      ps = extractPostScript(buffer, path, psLen, psOffset);
+
+      int footerSize = (int) ps.getFooterLength();
+      int metadataSize = (int) ps.getMetadataLength();
+      writerVersion = extractWriterVersion(ps);
+
+      //check if extra bytes need to be read
+      int extra = Math.max(0, psLen + 1 + footerSize + metadataSize - readSize);
+      if (extra > 0) {
+        //more bytes need to be read, seek back to the right place and read extra bytes
+        ByteBuffer extraBuf = ByteBuffer.allocate(extra + readSize);
+        file.readFully((size - readSize - extra), extraBuf.array(),
+            extraBuf.arrayOffset() + extraBuf.position(), extra);
+        extraBuf.position(extra);
+        //append with already read bytes
+        extraBuf.put(buffer);
+        buffer = extraBuf;
+        buffer.position(0);
+        fullFooterBuffer = buffer.slice();
+        buffer.limit(footerSize + metadataSize);
+      } else {
+        //footer is already in the bytes in buffer, just adjust position, length
+        buffer.position(psOffset - footerSize - metadataSize);
+        fullFooterBuffer = buffer.slice();
+        buffer.limit(psOffset);
+      }
+
+      // remember position for later TODO: what later? this comment is useless
+      buffer.mark();
+    } finally {
+      try {
+        file.close();
+      } catch (IOException ex) {
+        LOG.error("Failed to close the file after another error", ex);
+      }
+    }
+
+    return new FileMetaInfo(
+        ps.getCompression().toString(),
+        (int) ps.getCompressionBlockSize(),
+        (int) ps.getMetadataLength(),
+        buffer,
+        ps.getVersionList(),
+        writerVersion,
+        fullFooterBuffer
+        );
+  }
+
+  protected static OrcFile.WriterVersion extractWriterVersion(OrcProto.PostScript ps) {
+    return (ps.hasWriterVersion()
+        ? getWriterVersion(ps.getWriterVersion()) : OrcFile.WriterVersion.ORIGINAL);
+  }
+
+  protected static List<StripeInformation> convertProtoStripesToStripes(
+      List<OrcProto.StripeInformation> stripes) {
+    List<StripeInformation> result = new ArrayList<StripeInformation>(stripes.size());
+    for (OrcProto.StripeInformation info : stripes) {
+      result.add(new StripeInformationImpl(info));
+    }
+    return result;
+  }
+
+  /**
+   * MetaInfoObjExtractor - has logic to create the values for the fields in ReaderImpl
+   *  from serialized fields.
+   * As the fields are final, the fields need to be initialized in the constructor and
+   *  can't be done in some helper function. So this helper class is used instead.
+   *
+   */
+  private static class MetaInfoObjExtractor{
+    final org.apache.orc.CompressionKind compressionKind;
+    final CompressionCodec codec;
+    final int bufferSize;
+    final int metadataSize;
+    final OrcProto.Metadata metadata;
+    final OrcProto.Footer footer;
+
+    MetaInfoObjExtractor(String codecStr, int bufferSize, int metadataSize, 
+        ByteBuffer footerBuffer) throws IOException {
+
+      this.compressionKind = org.apache.orc.CompressionKind.valueOf(codecStr.toUpperCase());
+      this.bufferSize = bufferSize;
+      this.codec = WriterImpl.createCodec(compressionKind);
+      this.metadataSize = metadataSize;
+
+      int position = footerBuffer.position();
+      int footerBufferSize = footerBuffer.limit() - footerBuffer.position() - metadataSize;
+
+      this.metadata = extractMetadata(footerBuffer, position, metadataSize, codec, bufferSize);
+      this.footer = extractFooter(
+          footerBuffer, position + metadataSize, footerBufferSize, codec, bufferSize);
+
+      footerBuffer.position(position);
+    }
+  }
+
+  @Override
+  public ByteBuffer getSerializedFileFooter() {
+    return footerMetaAndPsBuffer;
+  }
+
+  @Override
+  public RecordReader rows() throws IOException {
+    return rows(new Options());
+  }
+
+  @Override
+  public RecordReader rows(Options options) throws IOException {
+    LOG.info("Reading ORC rows from " + path + " with " + options);
+    boolean[] include = options.getInclude();
+    // if included columns is null, then include all columns
+    if (include == null) {
+      include = new boolean[types.size()];
+      Arrays.fill(include, true);
+      options.include(include);
+    }
+    return new RecordReaderImpl(this, options);
+  }
+
+
+  @Override
+  public long getRawDataSize() {
+    // if the deserializedSize is not computed, then compute it, else
+    // return the already computed size. since we are reading from the footer
+    // we don't have to compute deserialized size repeatedly
+    if (deserializedSize == -1) {
+      List<Integer> indices = Lists.newArrayList();
+      for (int i = 0; i < fileStats.size(); ++i) {
+        indices.add(i);
+      }
+      deserializedSize = getRawDataSizeFromColIndices(indices);
+    }
+    return deserializedSize;
+  }
+
+  @Override
+  public long getRawDataSizeFromColIndices(List<Integer> colIndices) {
+    return getRawDataSizeFromColIndices(colIndices, types, fileStats);
+  }
+
+  public static long getRawDataSizeFromColIndices(
+      List<Integer> colIndices, List<OrcProto.Type> types,
+      List<OrcProto.ColumnStatistics> stats) {
+    long result = 0;
+    for (int colIdx : colIndices) {
+      result += getRawDataSizeOfColumn(colIdx, types, stats);
+    }
+    return result;
+  }
+
+  private static long getRawDataSizeOfColumn(int colIdx, List<OrcProto.Type> types,
+      List<OrcProto.ColumnStatistics> stats) {
+    OrcProto.ColumnStatistics colStat = stats.get(colIdx);
+    long numVals = colStat.getNumberOfValues();
+    OrcProto.Type type = types.get(colIdx);
+
+    switch (type.getKind()) {
+    case BINARY:
+      // old orc format doesn't support binary statistics. checking for binary
+      // statistics is not required as protocol buffers takes care of it.
+      return colStat.getBinaryStatistics().getSum();
+    case STRING:
+    case CHAR:
+    case VARCHAR:
+      // old orc format doesn't support sum for string statistics. checking for
+      // existence is not required as protocol buffers takes care of it.
+
+      // ORC strings are deserialized to java strings. so use java data model's
+      // string size
+      numVals = numVals == 0 ? 1 : numVals;
+      int avgStrLen = (int) (colStat.getStringStatistics().getSum() / numVals);
+      return numVals * JavaDataModel.get().lengthForStringOfLength(avgStrLen);
+    case TIMESTAMP:
+      return numVals * JavaDataModel.get().lengthOfTimestamp();
+    case DATE:
+      return numVals * JavaDataModel.get().lengthOfDate();
+    case DECIMAL:
+      return numVals * JavaDataModel.get().lengthOfDecimal();
+    case DOUBLE:
+    case LONG:
+      return numVals * JavaDataModel.get().primitive2();
+    case FLOAT:
+    case INT:
+    case SHORT:
+    case BOOLEAN:
+    case BYTE:
+      return numVals * JavaDataModel.get().primitive1();
+    default:
+      LOG.debug("Unknown primitive category: " + type.getKind());
+      break;
+    }
+
+    return 0;
+  }
+
+  @Override
+  public long getRawDataSizeOfColumns(List<String> colNames) {
+    List<Integer> colIndices = getColumnIndicesFromNames(colNames);
+    return getRawDataSizeFromColIndices(colIndices);
+  }
+
+  private List<Integer> getColumnIndicesFromNames(List<String> colNames) {
+    // top level struct
+    OrcProto.Type type = types.get(0);
+    List<Integer> colIndices = Lists.newArrayList();
+    List<String> fieldNames = type.getFieldNamesList();
+    int fieldIdx;
+    for (String colName : colNames) {
+      if (fieldNames.contains(colName)) {
+        fieldIdx = fieldNames.indexOf(colName);
+      } else {
+        String s = "Cannot find field for: " + colName + " in ";
+        for (String fn : fieldNames) {
+          s += fn + ", ";
+        }
+        LOG.warn(s);
+        continue;
+      }
+
+      // a single field may span multiple columns. find start and end column
+      // index for the requested field
+      int idxStart = type.getSubtypes(fieldIdx);
+
+      int idxEnd;
+
+      // if the specified is the last field and then end index will be last
+      // column index
+      if (fieldIdx + 1 > fieldNames.size() - 1) {
+        idxEnd = getLastIdx() + 1;
+      } else {
+        idxEnd = type.getSubtypes(fieldIdx + 1);
+      }
+
+      // if start index and end index are same then the field is a primitive
+      // field else complex field (like map, list, struct, union)
+      if (idxStart == idxEnd) {
+        // simple field
+        colIndices.add(idxStart);
+      } else {
+        // complex fields spans multiple columns
+        for (int i = idxStart; i < idxEnd; i++) {
+          colIndices.add(i);
+        }
+      }
+    }
+    return colIndices;
+  }
+
+  private int getLastIdx() {
+    Set<Integer> indices = new HashSet<>();
+    for (OrcProto.Type type : types) {
+      indices.addAll(type.getSubtypesList());
+    }
+    return Collections.max(indices);
+  }
+
+  @Override
+  public List<OrcProto.StripeStatistics> getOrcProtoStripeStatistics() {
+    return stripeStats;
+  }
+
+  @Override
+  public List<OrcProto.ColumnStatistics> getOrcProtoFileStatistics() {
+    return fileStats;
+  }
+
+  @Override
+  public List<StripeStatistics> getStripeStatistics() {
+    List<StripeStatistics> result = new ArrayList<>();
+    for (OrcProto.StripeStatistics ss : stripeStats) {
+      result.add(new StripeStatistics(ss.getColStatsList()));
+    }
+    return result;
+  }
+
+  public List<OrcProto.UserMetadataItem> getOrcProtoUserMetadata() {
+    return userMetadata;
+  }
+
+  @Override
+  public List<Integer> getVersionList() {
+    return versionList;
+  }
+
+  @Override
+  public int getMetadataSize() {
+    return metadataSize;
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder buffer = new StringBuilder();
+    buffer.append("ORC Reader(");
+    buffer.append(path);
+    if (maxLength != -1) {
+      buffer.append(", ");
+      buffer.append(maxLength);
+    }
+    buffer.append(")");
+    return buffer.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/orc/src/java/org/apache/orc/impl/RecordReaderImpl.java
----------------------------------------------------------------------
diff --git a/orc/src/java/org/apache/orc/impl/RecordReaderImpl.java b/orc/src/java/org/apache/orc/impl/RecordReaderImpl.java
new file mode 100644
index 0000000..36a802e
--- /dev/null
+++ b/orc/src/java/org/apache/orc/impl/RecordReaderImpl.java
@@ -0,0 +1,1215 @@
+/**
+ * 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.orc.impl;
+
+import java.io.IOException;
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.orc.BooleanColumnStatistics;
+import org.apache.orc.Reader;
+import org.apache.orc.RecordReader;
+import org.apache.orc.TypeDescription;
+import org.apache.orc.ColumnStatistics;
+import org.apache.orc.CompressionCodec;
+import org.apache.orc.DataReader;
+import org.apache.orc.DateColumnStatistics;
+import org.apache.orc.DecimalColumnStatistics;
+import org.apache.orc.DoubleColumnStatistics;
+import org.apache.orc.IntegerColumnStatistics;
+import org.apache.orc.OrcConf;
+import org.apache.orc.StringColumnStatistics;
+import org.apache.orc.StripeInformation;
+import org.apache.orc.TimestampColumnStatistics;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.common.io.DiskRange;
+import org.apache.hadoop.hive.common.io.DiskRangeList;
+import org.apache.hadoop.hive.common.io.DiskRangeList.CreateHelper;
+import org.apache.hadoop.hive.common.type.HiveDecimal;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.orc.BloomFilterIO;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument.TruthValue;
+import org.apache.hadoop.hive.serde2.io.DateWritable;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.hadoop.hive.ql.util.TimestampUtils;
+import org.apache.hadoop.io.Text;
+import org.apache.orc.OrcProto;
+
+public class RecordReaderImpl implements RecordReader {
+  static final Logger LOG = LoggerFactory.getLogger(RecordReaderImpl.class);
+  private static final boolean isLogDebugEnabled = LOG.isDebugEnabled();
+  private static final Object UNKNOWN_VALUE = new Object();
+  protected final Path path;
+  private final long firstRow;
+  private final List<StripeInformation> stripes =
+      new ArrayList<StripeInformation>();
+  private OrcProto.StripeFooter stripeFooter;
+  private final long totalRowCount;
+  private final CompressionCodec codec;
+  protected final TypeDescription schema;
+  private final List<OrcProto.Type> types;
+  private final int bufferSize;
+  private final boolean[] included;
+  private final long rowIndexStride;
+  private long rowInStripe = 0;
+  private int currentStripe = -1;
+  private long rowBaseInStripe = 0;
+  private long rowCountInStripe = 0;
+  private final Map<StreamName, InStream> streams =
+      new HashMap<StreamName, InStream>();
+  DiskRangeList bufferChunks = null;
+  private final TreeReaderFactory.TreeReader reader;
+  private final OrcProto.RowIndex[] indexes;
+  private final OrcProto.BloomFilterIndex[] bloomFilterIndices;
+  private final SargApplier sargApp;
+  // an array about which row groups aren't skipped
+  private boolean[] includedRowGroups = null;
+  private final DataReader dataReader;
+
+  /**
+   * Given a list of column names, find the given column and return the index.
+   *
+   * @param columnNames the list of potential column names
+   * @param columnName  the column name to look for
+   * @param rootColumn  offset the result with the rootColumn
+   * @return the column number or -1 if the column wasn't found
+   */
+  static int findColumns(String[] columnNames,
+                         String columnName,
+                         int rootColumn) {
+    for(int i=0; i < columnNames.length; ++i) {
+      if (columnName.equals(columnNames[i])) {
+        return i + rootColumn;
+      }
+    }
+    return -1;
+  }
+
+  /**
+   * Find the mapping from predicate leaves to columns.
+   * @param sargLeaves the search argument that we need to map
+   * @param columnNames the names of the columns
+   * @param rootColumn the offset of the top level row, which offsets the
+   *                   result
+   * @return an array mapping the sarg leaves to concrete column numbers
+   */
+  public static int[] mapSargColumnsToOrcInternalColIdx(List<PredicateLeaf> sargLeaves,
+                             String[] columnNames,
+                             int rootColumn) {
+    int[] result = new int[sargLeaves.size()];
+    Arrays.fill(result, -1);
+    for(int i=0; i < result.length; ++i) {
+      String colName = sargLeaves.get(i).getColumnName();
+      result[i] = findColumns(columnNames, colName, rootColumn);
+    }
+    return result;
+  }
+
+  protected RecordReaderImpl(ReaderImpl fileReader,
+                             Reader.Options options) throws IOException {
+    SchemaEvolution treeReaderSchema;
+    this.included = options.getInclude();
+    included[0] = true;
+    if (options.getSchema() == null) {
+      if (LOG.isInfoEnabled()) {
+        LOG.info("Schema on read not provided -- using file schema " +
+            fileReader.getSchema());
+      }
+      treeReaderSchema = new SchemaEvolution(fileReader.getSchema(), included);
+    } else {
+
+      // Now that we are creating a record reader for a file, validate that the schema to read
+      // is compatible with the file schema.
+      //
+      treeReaderSchema = new SchemaEvolution(fileReader.getSchema(),
+          options.getSchema(),included);
+    }
+    this.schema = treeReaderSchema.getReaderSchema();
+    this.path = fileReader.path;
+    this.codec = fileReader.codec;
+    this.types = fileReader.types;
+    this.bufferSize = fileReader.bufferSize;
+    this.rowIndexStride = fileReader.rowIndexStride;
+    SearchArgument sarg = options.getSearchArgument();
+    if (sarg != null && rowIndexStride != 0) {
+      sargApp = new SargApplier(
+          sarg, options.getColumnNames(), rowIndexStride, types,
+          included.length);
+    } else {
+      sargApp = null;
+    }
+    long rows = 0;
+    long skippedRows = 0;
+    long offset = options.getOffset();
+    long maxOffset = options.getMaxOffset();
+    for(StripeInformation stripe: fileReader.getStripes()) {
+      long stripeStart = stripe.getOffset();
+      if (offset > stripeStart) {
+        skippedRows += stripe.getNumberOfRows();
+      } else if (stripeStart < maxOffset) {
+        this.stripes.add(stripe);
+        rows += stripe.getNumberOfRows();
+      }
+    }
+
+    Boolean zeroCopy = options.getUseZeroCopy();
+    if (zeroCopy == null) {
+      zeroCopy = OrcConf.USE_ZEROCOPY.getBoolean(fileReader.conf);
+    }
+    if (options.getDataReader() != null) {
+      this.dataReader = options.getDataReader();
+    } else {
+      this.dataReader = RecordReaderUtils.createDefaultDataReader(
+          DataReaderProperties.builder()
+              .withBufferSize(bufferSize)
+              .withCompression(fileReader.compressionKind)
+              .withFileSystem(fileReader.fileSystem)
+              .withPath(fileReader.path)
+              .withTypeCount(types.size())
+              .withZeroCopy(zeroCopy)
+              .build());
+    }
+    this.dataReader.open();
+
+    firstRow = skippedRows;
+    totalRowCount = rows;
+    Boolean skipCorrupt = options.getSkipCorruptRecords();
+    if (skipCorrupt == null) {
+      skipCorrupt = OrcConf.SKIP_CORRUPT_DATA.getBoolean(fileReader.conf);
+    }
+
+    reader = TreeReaderFactory.createTreeReader(treeReaderSchema.getReaderSchema(),
+        treeReaderSchema, included, skipCorrupt);
+    indexes = new OrcProto.RowIndex[types.size()];
+    bloomFilterIndices = new OrcProto.BloomFilterIndex[types.size()];
+    advanceToNextRow(reader, 0L, true);
+  }
+
+  public static final class PositionProviderImpl implements PositionProvider {
+    private final OrcProto.RowIndexEntry entry;
+    private int index;
+
+    public PositionProviderImpl(OrcProto.RowIndexEntry entry) {
+      this(entry, 0);
+    }
+
+    public PositionProviderImpl(OrcProto.RowIndexEntry entry, int startPos) {
+      this.entry = entry;
+      this.index = startPos;
+    }
+
+    @Override
+    public long getNext() {
+      return entry.getPositions(index++);
+    }
+  }
+
+  public OrcProto.StripeFooter readStripeFooter(StripeInformation stripe
+                                                ) throws IOException {
+    return dataReader.readStripeFooter(stripe);
+  }
+
+  enum Location {
+    BEFORE, MIN, MIDDLE, MAX, AFTER
+  }
+
+  /**
+   * Given a point and min and max, determine if the point is before, at the
+   * min, in the middle, at the max, or after the range.
+   * @param point the point to test
+   * @param min the minimum point
+   * @param max the maximum point
+   * @param <T> the type of the comparision
+   * @return the location of the point
+   */
+  static <T> Location compareToRange(Comparable<T> point, T min, T max) {
+    int minCompare = point.compareTo(min);
+    if (minCompare < 0) {
+      return Location.BEFORE;
+    } else if (minCompare == 0) {
+      return Location.MIN;
+    }
+    int maxCompare = point.compareTo(max);
+    if (maxCompare > 0) {
+      return Location.AFTER;
+    } else if (maxCompare == 0) {
+      return Location.MAX;
+    }
+    return Location.MIDDLE;
+  }
+
+  /**
+   * Get the maximum value out of an index entry.
+   * @param index
+   *          the index entry
+   * @return the object for the maximum value or null if there isn't one
+   */
+  static Object getMax(ColumnStatistics index) {
+    if (index instanceof IntegerColumnStatistics) {
+      return ((IntegerColumnStatistics) index).getMaximum();
+    } else if (index instanceof DoubleColumnStatistics) {
+      return ((DoubleColumnStatistics) index).getMaximum();
+    } else if (index instanceof StringColumnStatistics) {
+      return ((StringColumnStatistics) index).getMaximum();
+    } else if (index instanceof DateColumnStatistics) {
+      return ((DateColumnStatistics) index).getMaximum();
+    } else if (index instanceof DecimalColumnStatistics) {
+      return ((DecimalColumnStatistics) index).getMaximum();
+    } else if (index instanceof TimestampColumnStatistics) {
+      return ((TimestampColumnStatistics) index).getMaximum();
+    } else if (index instanceof BooleanColumnStatistics) {
+      if (((BooleanColumnStatistics)index).getTrueCount()!=0) {
+        return Boolean.TRUE;
+      } else {
+        return Boolean.FALSE;
+      }
+    } else {
+      return null;
+    }
+  }
+
+  /**
+   * Get the minimum value out of an index entry.
+   * @param index
+   *          the index entry
+   * @return the object for the minimum value or null if there isn't one
+   */
+  static Object getMin(ColumnStatistics index) {
+    if (index instanceof IntegerColumnStatistics) {
+      return ((IntegerColumnStatistics) index).getMinimum();
+    } else if (index instanceof DoubleColumnStatistics) {
+      return ((DoubleColumnStatistics) index).getMinimum();
+    } else if (index instanceof StringColumnStatistics) {
+      return ((StringColumnStatistics) index).getMinimum();
+    } else if (index instanceof DateColumnStatistics) {
+      return ((DateColumnStatistics) index).getMinimum();
+    } else if (index instanceof DecimalColumnStatistics) {
+      return ((DecimalColumnStatistics) index).getMinimum();
+    } else if (index instanceof TimestampColumnStatistics) {
+      return ((TimestampColumnStatistics) index).getMinimum();
+    } else if (index instanceof BooleanColumnStatistics) {
+      if (((BooleanColumnStatistics)index).getFalseCount()!=0) {
+        return Boolean.FALSE;
+      } else {
+        return Boolean.TRUE;
+      }
+    } else {
+      return UNKNOWN_VALUE; // null is not safe here
+    }
+  }
+
+  /**
+   * Evaluate a predicate with respect to the statistics from the column
+   * that is referenced in the predicate.
+   * @param statsProto the statistics for the column mentioned in the predicate
+   * @param predicate the leaf predicate we need to evaluation
+   * @param bloomFilter
+   * @return the set of truth values that may be returned for the given
+   *   predicate.
+   */
+  static TruthValue evaluatePredicateProto(OrcProto.ColumnStatistics statsProto,
+      PredicateLeaf predicate, OrcProto.BloomFilter bloomFilter) {
+    ColumnStatistics cs = ColumnStatisticsImpl.deserialize(statsProto);
+    Object minValue = getMin(cs);
+    Object maxValue = getMax(cs);
+    BloomFilterIO bf = null;
+    if (bloomFilter != null) {
+      bf = new BloomFilterIO(bloomFilter);
+    }
+    return evaluatePredicateRange(predicate, minValue, maxValue, cs.hasNull(), bf);
+  }
+
+  /**
+   * Evaluate a predicate with respect to the statistics from the column
+   * that is referenced in the predicate.
+   * @param stats the statistics for the column mentioned in the predicate
+   * @param predicate the leaf predicate we need to evaluation
+   * @return the set of truth values that may be returned for the given
+   *   predicate.
+   */
+  public static TruthValue evaluatePredicate(ColumnStatistics stats,
+                                             PredicateLeaf predicate,
+                                             BloomFilterIO bloomFilter) {
+    Object minValue = getMin(stats);
+    Object maxValue = getMax(stats);
+    return evaluatePredicateRange(predicate, minValue, maxValue, stats.hasNull(), bloomFilter);
+  }
+
+  static TruthValue evaluatePredicateRange(PredicateLeaf predicate, Object min,
+      Object max, boolean hasNull, BloomFilterIO bloomFilter) {
+    // if we didn't have any values, everything must have been null
+    if (min == null) {
+      if (predicate.getOperator() == PredicateLeaf.Operator.IS_NULL) {
+        return TruthValue.YES;
+      } else {
+        return TruthValue.NULL;
+      }
+    } else if (min == UNKNOWN_VALUE) {
+      return TruthValue.YES_NO_NULL;
+    }
+
+    TruthValue result;
+    Object baseObj = predicate.getLiteral();
+    try {
+      // Predicate object and stats objects are converted to the type of the predicate object.
+      Object minValue = getBaseObjectForComparison(predicate.getType(), min);
+      Object maxValue = getBaseObjectForComparison(predicate.getType(), max);
+      Object predObj = getBaseObjectForComparison(predicate.getType(), baseObj);
+
+      result = evaluatePredicateMinMax(predicate, predObj, minValue, maxValue, hasNull);
+      if (shouldEvaluateBloomFilter(predicate, result, bloomFilter)) {
+        result = evaluatePredicateBloomFilter(predicate, predObj, bloomFilter, hasNull);
+      }
+      // in case failed conversion, return the default YES_NO_NULL truth value
+    } catch (Exception e) {
+      if (LOG.isWarnEnabled()) {
+        final String statsType = min == null ?
+            (max == null ? "null" : max.getClass().getSimpleName()) :
+            min.getClass().getSimpleName();
+        final String predicateType = baseObj == null ? "null" : baseObj.getClass().getSimpleName();
+        final String reason = e.getClass().getSimpleName() + " when evaluating predicate." +
+            " Skipping ORC PPD." +
+            " Exception: " + e.getMessage() +
+            " StatsType: " + statsType +
+            " PredicateType: " + predicateType;
+        LOG.warn(reason);
+        LOG.debug(reason, e);
+      }
+      if (predicate.getOperator().equals(PredicateLeaf.Operator.NULL_SAFE_EQUALS) || !hasNull) {
+        result = TruthValue.YES_NO;
+      } else {
+        result = TruthValue.YES_NO_NULL;
+      }
+    }
+    return result;
+  }
+
+  private static boolean shouldEvaluateBloomFilter(PredicateLeaf predicate,
+      TruthValue result, BloomFilterIO bloomFilter) {
+    // evaluate bloom filter only when
+    // 1) Bloom filter is available
+    // 2) Min/Max evaluation yield YES or MAYBE
+    // 3) Predicate is EQUALS or IN list
+    if (bloomFilter != null
+        && result != TruthValue.NO_NULL && result != TruthValue.NO
+        && (predicate.getOperator().equals(PredicateLeaf.Operator.EQUALS)
+            || predicate.getOperator().equals(PredicateLeaf.Operator.NULL_SAFE_EQUALS)
+            || predicate.getOperator().equals(PredicateLeaf.Operator.IN))) {
+      return true;
+    }
+    return false;
+  }
+
+  private static TruthValue evaluatePredicateMinMax(PredicateLeaf predicate, Object predObj,
+      Object minValue,
+      Object maxValue,
+      boolean hasNull) {
+    Location loc;
+
+    switch (predicate.getOperator()) {
+      case NULL_SAFE_EQUALS:
+        loc = compareToRange((Comparable) predObj, minValue, maxValue);
+        if (loc == Location.BEFORE || loc == Location.AFTER) {
+          return TruthValue.NO;
+        } else {
+          return TruthValue.YES_NO;
+        }
+      case EQUALS:
+        loc = compareToRange((Comparable) predObj, minValue, maxValue);
+        if (minValue.equals(maxValue) && loc == Location.MIN) {
+          return hasNull ? TruthValue.YES_NULL : TruthValue.YES;
+        } else if (loc == Location.BEFORE || loc == Location.AFTER) {
+          return hasNull ? TruthValue.NO_NULL : TruthValue.NO;
+        } else {
+          return hasNull ? TruthValue.YES_NO_NULL : TruthValue.YES_NO;
+        }
+      case LESS_THAN:
+        loc = compareToRange((Comparable) predObj, minValue, maxValue);
+        if (loc == Location.AFTER) {
+          return hasNull ? TruthValue.YES_NULL : TruthValue.YES;
+        } else if (loc == Location.BEFORE || loc == Location.MIN) {
+          return hasNull ? TruthValue.NO_NULL : TruthValue.NO;
+        } else {
+          return hasNull ? TruthValue.YES_NO_NULL : TruthValue.YES_NO;
+        }
+      case LESS_THAN_EQUALS:
+        loc = compareToRange((Comparable) predObj, minValue, maxValue);
+        if (loc == Location.AFTER || loc == Location.MAX) {
+          return hasNull ? TruthValue.YES_NULL : TruthValue.YES;
+        } else if (loc == Location.BEFORE) {
+          return hasNull ? TruthValue.NO_NULL : TruthValue.NO;
+        } else {
+          return hasNull ? TruthValue.YES_NO_NULL : TruthValue.YES_NO;
+        }
+      case IN:
+        if (minValue.equals(maxValue)) {
+          // for a single value, look through to see if that value is in the
+          // set
+          for (Object arg : predicate.getLiteralList()) {
+            predObj = getBaseObjectForComparison(predicate.getType(), arg);
+            loc = compareToRange((Comparable) predObj, minValue, maxValue);
+            if (loc == Location.MIN) {
+              return hasNull ? TruthValue.YES_NULL : TruthValue.YES;
+            }
+          }
+          return hasNull ? TruthValue.NO_NULL : TruthValue.NO;
+        } else {
+          // are all of the values outside of the range?
+          for (Object arg : predicate.getLiteralList()) {
+            predObj = getBaseObjectForComparison(predicate.getType(), arg);
+            loc = compareToRange((Comparable) predObj, minValue, maxValue);
+            if (loc == Location.MIN || loc == Location.MIDDLE ||
+                loc == Location.MAX) {
+              return hasNull ? TruthValue.YES_NO_NULL : TruthValue.YES_NO;
+            }
+          }
+          return hasNull ? TruthValue.NO_NULL : TruthValue.NO;
+        }
+      case BETWEEN:
+        List<Object> args = predicate.getLiteralList();
+        Object predObj1 = getBaseObjectForComparison(predicate.getType(), args.get(0));
+
+        loc = compareToRange((Comparable) predObj1, minValue, maxValue);
+        if (loc == Location.BEFORE || loc == Location.MIN) {
+          Object predObj2 = getBaseObjectForComparison(predicate.getType(), args.get(1));
+
+          Location loc2 = compareToRange((Comparable) predObj2, minValue, maxValue);
+          if (loc2 == Location.AFTER || loc2 == Location.MAX) {
+            return hasNull ? TruthValue.YES_NULL : TruthValue.YES;
+          } else if (loc2 == Location.BEFORE) {
+            return hasNull ? TruthValue.NO_NULL : TruthValue.NO;
+          } else {
+            return hasNull ? TruthValue.YES_NO_NULL : TruthValue.YES_NO;
+          }
+        } else if (loc == Location.AFTER) {
+          return hasNull ? TruthValue.NO_NULL : TruthValue.NO;
+        } else {
+          return hasNull ? TruthValue.YES_NO_NULL : TruthValue.YES_NO;
+        }
+      case IS_NULL:
+        // min = null condition above handles the all-nulls YES case
+        return hasNull ? TruthValue.YES_NO : TruthValue.NO;
+      default:
+        return hasNull ? TruthValue.YES_NO_NULL : TruthValue.YES_NO;
+    }
+  }
+
+  private static TruthValue evaluatePredicateBloomFilter(PredicateLeaf predicate,
+      final Object predObj, BloomFilterIO bloomFilter, boolean hasNull) {
+    switch (predicate.getOperator()) {
+      case NULL_SAFE_EQUALS:
+        // null safe equals does not return *_NULL variant. So set hasNull to false
+        return checkInBloomFilter(bloomFilter, predObj, false);
+      case EQUALS:
+        return checkInBloomFilter(bloomFilter, predObj, hasNull);
+      case IN:
+        for (Object arg : predicate.getLiteralList()) {
+          // if atleast one value in IN list exist in bloom filter, qualify the row group/stripe
+          Object predObjItem = getBaseObjectForComparison(predicate.getType(), arg);
+          TruthValue result = checkInBloomFilter(bloomFilter, predObjItem, hasNull);
+          if (result == TruthValue.YES_NO_NULL || result == TruthValue.YES_NO) {
+            return result;
+          }
+        }
+        return hasNull ? TruthValue.NO_NULL : TruthValue.NO;
+      default:
+        return hasNull ? TruthValue.YES_NO_NULL : TruthValue.YES_NO;
+    }
+  }
+
+  private static TruthValue checkInBloomFilter(BloomFilterIO bf, Object predObj, boolean hasNull) {
+    TruthValue result = hasNull ? TruthValue.NO_NULL : TruthValue.NO;
+
+    if (predObj instanceof Long) {
+      if (bf.testLong(((Long) predObj).longValue())) {
+        result = TruthValue.YES_NO_NULL;
+      }
+    } else if (predObj instanceof Double) {
+      if (bf.testDouble(((Double) predObj).doubleValue())) {
+        result = TruthValue.YES_NO_NULL;
+      }
+    } else if (predObj instanceof String || predObj instanceof Text ||
+        predObj instanceof HiveDecimalWritable ||
+        predObj instanceof BigDecimal) {
+      if (bf.testString(predObj.toString())) {
+        result = TruthValue.YES_NO_NULL;
+      }
+    } else if (predObj instanceof Timestamp) {
+      if (bf.testLong(((Timestamp) predObj).getTime())) {
+        result = TruthValue.YES_NO_NULL;
+      }
+    } else if (predObj instanceof Date) {
+      if (bf.testLong(DateWritable.dateToDays((Date) predObj))) {
+        result = TruthValue.YES_NO_NULL;
+      }
+    } else {
+        // if the predicate object is null and if hasNull says there are no nulls then return NO
+        if (predObj == null && !hasNull) {
+          result = TruthValue.NO;
+        } else {
+          result = TruthValue.YES_NO_NULL;
+        }
+      }
+
+    if (result == TruthValue.YES_NO_NULL && !hasNull) {
+      result = TruthValue.YES_NO;
+    }
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Bloom filter evaluation: " + result.toString());
+    }
+
+    return result;
+  }
+
+  private static Object getBaseObjectForComparison(PredicateLeaf.Type type, Object obj) {
+    if (obj == null) {
+      return null;
+    }
+    switch (type) {
+      case BOOLEAN:
+        if (obj instanceof Boolean) {
+          return obj;
+        } else {
+          // will only be true if the string conversion yields "true", all other values are
+          // considered false
+          return Boolean.valueOf(obj.toString());
+        }
+      case DATE:
+        if (obj instanceof Date) {
+          return obj;
+        } else if (obj instanceof String) {
+          return Date.valueOf((String) obj);
+        } else if (obj instanceof Timestamp) {
+          return DateWritable.timeToDate(((Timestamp) obj).getTime() / 1000L);
+        }
+        // always string, but prevent the comparison to numbers (are they days/seconds/milliseconds?)
+        break;
+      case DECIMAL:
+        if (obj instanceof Boolean) {
+          return new HiveDecimalWritable(((Boolean) obj).booleanValue() ?
+              HiveDecimal.ONE : HiveDecimal.ZERO);
+        } else if (obj instanceof Integer) {
+          return new HiveDecimalWritable(((Integer) obj).intValue());
+        } else if (obj instanceof Long) {
+          return new HiveDecimalWritable(((Long) obj));
+        } else if (obj instanceof Float || obj instanceof Double ||
+            obj instanceof String) {
+          return new HiveDecimalWritable(obj.toString());
+        } else if (obj instanceof BigDecimal) {
+          return new HiveDecimalWritable(HiveDecimal.create((BigDecimal) obj));
+        } else if (obj instanceof HiveDecimal) {
+          return new HiveDecimalWritable((HiveDecimal) obj);
+        } else if (obj instanceof HiveDecimalWritable) {
+          return obj;
+        } else if (obj instanceof Timestamp) {
+          return new HiveDecimalWritable(Double.toString(
+              TimestampUtils.getDouble((Timestamp) obj)));
+        }
+        break;
+      case FLOAT:
+        if (obj instanceof Number) {
+          // widening conversion
+          return ((Number) obj).doubleValue();
+        } else if (obj instanceof HiveDecimal) {
+          return ((HiveDecimal) obj).doubleValue();
+        } else if (obj instanceof String) {
+          return Double.valueOf(obj.toString());
+        } else if (obj instanceof Timestamp) {
+          return TimestampUtils.getDouble((Timestamp) obj);
+        } else if (obj instanceof HiveDecimal) {
+          return ((HiveDecimal) obj).doubleValue();
+        } else if (obj instanceof BigDecimal) {
+          return ((BigDecimal) obj).doubleValue();
+        }
+        break;
+      case LONG:
+        if (obj instanceof Number) {
+          // widening conversion
+          return ((Number) obj).longValue();
+        } else if (obj instanceof HiveDecimal) {
+          return ((HiveDecimal) obj).longValue();
+        } else if (obj instanceof String) {
+          return Long.valueOf(obj.toString());
+        }
+        break;
+      case STRING:
+        if (obj != null) {
+          return (obj.toString());
+        }
+        break;
+      case TIMESTAMP:
+        if (obj instanceof Timestamp) {
+          return obj;
+        } else if (obj instanceof Integer) {
+          return new Timestamp(((Number) obj).longValue());
+        } else if (obj instanceof Float) {
+          return TimestampUtils.doubleToTimestamp(((Float) obj).doubleValue());
+        } else if (obj instanceof Double) {
+          return TimestampUtils.doubleToTimestamp(((Double) obj).doubleValue());
+        } else if (obj instanceof HiveDecimal) {
+          return TimestampUtils.decimalToTimestamp((HiveDecimal) obj);
+        } else if (obj instanceof HiveDecimalWritable) {
+          return TimestampUtils.decimalToTimestamp(((HiveDecimalWritable) obj).getHiveDecimal());
+        } else if (obj instanceof Date) {
+          return new Timestamp(((Date) obj).getTime());
+        }
+        // float/double conversion to timestamp is interpreted as seconds whereas integer conversion
+        // to timestamp is interpreted as milliseconds by default. The integer to timestamp casting
+        // is also config driven. The filter operator changes its promotion based on config:
+        // "int.timestamp.conversion.in.seconds". Disable PPD for integer cases.
+        break;
+      default:
+        break;
+    }
+
+    throw new IllegalArgumentException(String.format(
+        "ORC SARGS could not convert from %s to %s", obj == null ? "(null)" : obj.getClass()
+            .getSimpleName(), type));
+  }
+
+  public static class SargApplier {
+    public final static boolean[] READ_ALL_RGS = null;
+    public final static boolean[] READ_NO_RGS = new boolean[0];
+
+    private final SearchArgument sarg;
+    private final List<PredicateLeaf> sargLeaves;
+    private final int[] filterColumns;
+    private final long rowIndexStride;
+    // same as the above array, but indices are set to true
+    private final boolean[] sargColumns;
+
+    public SargApplier(SearchArgument sarg, String[] columnNames, long rowIndexStride,
+        List<OrcProto.Type> types, int includedCount) {
+      this.sarg = sarg;
+      sargLeaves = sarg.getLeaves();
+      filterColumns = mapSargColumnsToOrcInternalColIdx(sargLeaves, columnNames, 0);
+      this.rowIndexStride = rowIndexStride;
+      // included will not be null, row options will fill the array with trues if null
+      sargColumns = new boolean[includedCount];
+      for (int i : filterColumns) {
+        // filter columns may have -1 as index which could be partition column in SARG.
+        if (i > 0) {
+          sargColumns[i] = true;
+        }
+      }
+    }
+
+    /**
+     * Pick the row groups that we need to load from the current stripe.
+     *
+     * @return an array with a boolean for each row group or null if all of the
+     * row groups must be read.
+     * @throws IOException
+     */
+    public boolean[] pickRowGroups(StripeInformation stripe, OrcProto.RowIndex[] indexes,
+        OrcProto.BloomFilterIndex[] bloomFilterIndices, boolean returnNone) throws IOException {
+      long rowsInStripe = stripe.getNumberOfRows();
+      int groupsInStripe = (int) ((rowsInStripe + rowIndexStride - 1) / rowIndexStride);
+      boolean[] result = new boolean[groupsInStripe]; // TODO: avoid alloc?
+      TruthValue[] leafValues = new TruthValue[sargLeaves.size()];
+      boolean hasSelected = false, hasSkipped = false;
+      for (int rowGroup = 0; rowGroup < result.length; ++rowGroup) {
+        for (int pred = 0; pred < leafValues.length; ++pred) {
+          int columnIx = filterColumns[pred];
+          if (columnIx != -1) {
+            if (indexes[columnIx] == null) {
+              throw new AssertionError("Index is not populated for " + columnIx);
+            }
+            OrcProto.RowIndexEntry entry = indexes[columnIx].getEntry(rowGroup);
+            if (entry == null) {
+              throw new AssertionError("RG is not populated for " + columnIx + " rg " + rowGroup);
+            }
+            OrcProto.ColumnStatistics stats = entry.getStatistics();
+            OrcProto.BloomFilter bf = null;
+            if (bloomFilterIndices != null && bloomFilterIndices[filterColumns[pred]] != null) {
+              bf = bloomFilterIndices[filterColumns[pred]].getBloomFilter(rowGroup);
+            }
+            leafValues[pred] = evaluatePredicateProto(stats, sargLeaves.get(pred), bf);
+            if (LOG.isTraceEnabled()) {
+              LOG.trace("Stats = " + stats);
+              LOG.trace("Setting " + sargLeaves.get(pred) + " to " + leafValues[pred]);
+            }
+          } else {
+            // the column is a virtual column
+            leafValues[pred] = TruthValue.YES_NO_NULL;
+          }
+        }
+        result[rowGroup] = sarg.evaluate(leafValues).isNeeded();
+        hasSelected = hasSelected || result[rowGroup];
+        hasSkipped = hasSkipped || (!result[rowGroup]);
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Row group " + (rowIndexStride * rowGroup) + " to " +
+              (rowIndexStride * (rowGroup + 1) - 1) + " is " +
+              (result[rowGroup] ? "" : "not ") + "included.");
+        }
+      }
+
+      return hasSkipped ? ((hasSelected || !returnNone) ? result : READ_NO_RGS) : READ_ALL_RGS;
+    }
+  }
+
+  /**
+   * Pick the row groups that we need to load from the current stripe.
+   *
+   * @return an array with a boolean for each row group or null if all of the
+   * row groups must be read.
+   * @throws IOException
+   */
+  protected boolean[] pickRowGroups() throws IOException {
+    // if we don't have a sarg or indexes, we read everything
+    if (sargApp == null) {
+      return null;
+    }
+    readRowIndex(currentStripe, included, sargApp.sargColumns);
+    return sargApp.pickRowGroups(stripes.get(currentStripe), indexes, bloomFilterIndices, false);
+  }
+
+  private void clearStreams() {
+    // explicit close of all streams to de-ref ByteBuffers
+    for (InStream is : streams.values()) {
+      is.close();
+    }
+    if (bufferChunks != null) {
+      if (dataReader.isTrackingDiskRanges()) {
+        for (DiskRangeList range = bufferChunks; range != null; range = range.next) {
+          if (!(range instanceof BufferChunk)) {
+            continue;
+          }
+          dataReader.releaseBuffer(((BufferChunk) range).getChunk());
+        }
+      }
+    }
+    bufferChunks = null;
+    streams.clear();
+  }
+
+  /**
+   * Read the current stripe into memory.
+   *
+   * @throws IOException
+   */
+  private void readStripe() throws IOException {
+    StripeInformation stripe = beginReadStripe();
+    includedRowGroups = pickRowGroups();
+
+    // move forward to the first unskipped row
+    if (includedRowGroups != null) {
+      while (rowInStripe < rowCountInStripe &&
+          !includedRowGroups[(int) (rowInStripe / rowIndexStride)]) {
+        rowInStripe = Math.min(rowCountInStripe, rowInStripe + rowIndexStride);
+      }
+    }
+
+    // if we haven't skipped the whole stripe, read the data
+    if (rowInStripe < rowCountInStripe) {
+      // if we aren't projecting columns or filtering rows, just read it all
+      if (included == null && includedRowGroups == null) {
+        readAllDataStreams(stripe);
+      } else {
+        readPartialDataStreams(stripe);
+      }
+      reader.startStripe(streams, stripeFooter);
+      // if we skipped the first row group, move the pointers forward
+      if (rowInStripe != 0) {
+        seekToRowEntry(reader, (int) (rowInStripe / rowIndexStride));
+      }
+    }
+  }
+
+  private StripeInformation beginReadStripe() throws IOException {
+    StripeInformation stripe = stripes.get(currentStripe);
+    stripeFooter = readStripeFooter(stripe);
+    clearStreams();
+    // setup the position in the stripe
+    rowCountInStripe = stripe.getNumberOfRows();
+    rowInStripe = 0;
+    rowBaseInStripe = 0;
+    for (int i = 0; i < currentStripe; ++i) {
+      rowBaseInStripe += stripes.get(i).getNumberOfRows();
+    }
+    // reset all of the indexes
+    for (int i = 0; i < indexes.length; ++i) {
+      indexes[i] = null;
+    }
+    return stripe;
+  }
+
+  private void readAllDataStreams(StripeInformation stripe) throws IOException {
+    long start = stripe.getIndexLength();
+    long end = start + stripe.getDataLength();
+    // explicitly trigger 1 big read
+    DiskRangeList toRead = new DiskRangeList(start, end);
+    bufferChunks = dataReader.readFileData(toRead, stripe.getOffset(), false);
+    List<OrcProto.Stream> streamDescriptions = stripeFooter.getStreamsList();
+    createStreams(streamDescriptions, bufferChunks, null, codec, bufferSize, streams);
+  }
+
+  /**
+   * Plan the ranges of the file that we need to read given the list of
+   * columns and row groups.
+   *
+   * @param streamList        the list of streams available
+   * @param indexes           the indexes that have been loaded
+   * @param includedColumns   which columns are needed
+   * @param includedRowGroups which row groups are needed
+   * @param isCompressed      does the file have generic compression
+   * @param encodings         the encodings for each column
+   * @param types             the types of the columns
+   * @param compressionSize   the compression block size
+   * @return the list of disk ranges that will be loaded
+   */
+  static DiskRangeList planReadPartialDataStreams
+  (List<OrcProto.Stream> streamList,
+      OrcProto.RowIndex[] indexes,
+      boolean[] includedColumns,
+      boolean[] includedRowGroups,
+      boolean isCompressed,
+      List<OrcProto.ColumnEncoding> encodings,
+      List<OrcProto.Type> types,
+      int compressionSize,
+      boolean doMergeBuffers) {
+    long offset = 0;
+    // figure out which columns have a present stream
+    boolean[] hasNull = RecordReaderUtils.findPresentStreamsByColumn(streamList, types);
+    CreateHelper list = new CreateHelper();
+    for (OrcProto.Stream stream : streamList) {
+      long length = stream.getLength();
+      int column = stream.getColumn();
+      OrcProto.Stream.Kind streamKind = stream.getKind();
+      // since stream kind is optional, first check if it exists
+      if (stream.hasKind() &&
+          (StreamName.getArea(streamKind) == StreamName.Area.DATA) &&
+          (column < includedColumns.length && includedColumns[column])) {
+        // if we aren't filtering or it is a dictionary, load it.
+        if (includedRowGroups == null
+            || RecordReaderUtils.isDictionary(streamKind, encodings.get(column))) {
+          RecordReaderUtils.addEntireStreamToRanges(offset, length, list, doMergeBuffers);
+        } else {
+          RecordReaderUtils.addRgFilteredStreamToRanges(stream, includedRowGroups,
+              isCompressed, indexes[column], encodings.get(column), types.get(column),
+              compressionSize, hasNull[column], offset, length, list, doMergeBuffers);
+        }
+      }
+      offset += length;
+    }
+    return list.extract();
+  }
+
+  void createStreams(List<OrcProto.Stream> streamDescriptions,
+      DiskRangeList ranges,
+      boolean[] includeColumn,
+      CompressionCodec codec,
+      int bufferSize,
+      Map<StreamName, InStream> streams) throws IOException {
+    long streamOffset = 0;
+    for (OrcProto.Stream streamDesc : streamDescriptions) {
+      int column = streamDesc.getColumn();
+      if ((includeColumn != null &&
+          (column < included.length && !includeColumn[column])) ||
+          streamDesc.hasKind() &&
+              (StreamName.getArea(streamDesc.getKind()) != StreamName.Area.DATA)) {
+        streamOffset += streamDesc.getLength();
+        continue;
+      }
+      List<DiskRange> buffers = RecordReaderUtils.getStreamBuffers(
+          ranges, streamOffset, streamDesc.getLength());
+      StreamName name = new StreamName(column, streamDesc.getKind());
+      streams.put(name, InStream.create(name.toString(), buffers,
+          streamDesc.getLength(), codec, bufferSize));
+      streamOffset += streamDesc.getLength();
+    }
+  }
+
+  private void readPartialDataStreams(StripeInformation stripe) throws IOException {
+    List<OrcProto.Stream> streamList = stripeFooter.getStreamsList();
+    DiskRangeList toRead = planReadPartialDataStreams(streamList,
+        indexes, included, includedRowGroups, codec != null,
+        stripeFooter.getColumnsList(), types, bufferSize, true);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("chunks = " + RecordReaderUtils.stringifyDiskRanges(toRead));
+    }
+    bufferChunks = dataReader.readFileData(toRead, stripe.getOffset(), false);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("merge = " + RecordReaderUtils.stringifyDiskRanges(bufferChunks));
+    }
+
+    createStreams(streamList, bufferChunks, included, codec, bufferSize, streams);
+  }
+
+  /**
+   * Read the next stripe until we find a row that we don't skip.
+   *
+   * @throws IOException
+   */
+  private void advanceStripe() throws IOException {
+    rowInStripe = rowCountInStripe;
+    while (rowInStripe >= rowCountInStripe &&
+        currentStripe < stripes.size() - 1) {
+      currentStripe += 1;
+      readStripe();
+    }
+  }
+
+  /**
+   * Skip over rows that we aren't selecting, so that the next row is
+   * one that we will read.
+   *
+   * @param nextRow the row we want to go to
+   * @throws IOException
+   */
+  private boolean advanceToNextRow(
+      TreeReaderFactory.TreeReader reader, long nextRow, boolean canAdvanceStripe)
+      throws IOException {
+    long nextRowInStripe = nextRow - rowBaseInStripe;
+    // check for row skipping
+    if (rowIndexStride != 0 &&
+        includedRowGroups != null &&
+        nextRowInStripe < rowCountInStripe) {
+      int rowGroup = (int) (nextRowInStripe / rowIndexStride);
+      if (!includedRowGroups[rowGroup]) {
+        while (rowGroup < includedRowGroups.length && !includedRowGroups[rowGroup]) {
+          rowGroup += 1;
+        }
+        if (rowGroup >= includedRowGroups.length) {
+          if (canAdvanceStripe) {
+            advanceStripe();
+          }
+          return canAdvanceStripe;
+        }
+        nextRowInStripe = Math.min(rowCountInStripe, rowGroup * rowIndexStride);
+      }
+    }
+    if (nextRowInStripe >= rowCountInStripe) {
+      if (canAdvanceStripe) {
+        advanceStripe();
+      }
+      return canAdvanceStripe;
+    }
+    if (nextRowInStripe != rowInStripe) {
+      if (rowIndexStride != 0) {
+        int rowGroup = (int) (nextRowInStripe / rowIndexStride);
+        seekToRowEntry(reader, rowGroup);
+        reader.skipRows(nextRowInStripe - rowGroup * rowIndexStride);
+      } else {
+        reader.skipRows(nextRowInStripe - rowInStripe);
+      }
+      rowInStripe = nextRowInStripe;
+    }
+    return true;
+  }
+
+  @Override
+  public boolean nextBatch(VectorizedRowBatch batch) throws IOException {
+    try {
+      if (rowInStripe >= rowCountInStripe) {
+        currentStripe += 1;
+        if (currentStripe >= stripes.size()) {
+          batch.size = 0;
+          return false;
+        }
+        readStripe();
+      }
+
+      int batchSize = computeBatchSize(batch.getMaxSize());
+
+      rowInStripe += batchSize;
+      reader.setVectorColumnCount(batch.getDataColumnCount());
+      reader.nextBatch(batch, batchSize);
+      batch.selectedInUse = false;
+      batch.size = batchSize;
+      advanceToNextRow(reader, rowInStripe + rowBaseInStripe, true);
+      return batch.size  != 0;
+    } catch (IOException e) {
+      // Rethrow exception with file name in log message
+      throw new IOException("Error reading file: " + path, e);
+    }
+  }
+
+  private int computeBatchSize(long targetBatchSize) {
+    final int batchSize;
+    // In case of PPD, batch size should be aware of row group boundaries. If only a subset of row
+    // groups are selected then marker position is set to the end of range (subset of row groups
+    // within strip). Batch size computed out of marker position makes sure that batch size is
+    // aware of row group boundary and will not cause overflow when reading rows
+    // illustration of this case is here https://issues.apache.org/jira/browse/HIVE-6287
+    if (rowIndexStride != 0 && includedRowGroups != null && rowInStripe < rowCountInStripe) {
+      int startRowGroup = (int) (rowInStripe / rowIndexStride);
+      if (!includedRowGroups[startRowGroup]) {
+        while (startRowGroup < includedRowGroups.length && !includedRowGroups[startRowGroup]) {
+          startRowGroup += 1;
+        }
+      }
+
+      int endRowGroup = startRowGroup;
+      while (endRowGroup < includedRowGroups.length && includedRowGroups[endRowGroup]) {
+        endRowGroup += 1;
+      }
+
+      final long markerPosition =
+          (endRowGroup * rowIndexStride) < rowCountInStripe ? (endRowGroup * rowIndexStride)
+              : rowCountInStripe;
+      batchSize = (int) Math.min(targetBatchSize, (markerPosition - rowInStripe));
+
+      if (isLogDebugEnabled && batchSize < targetBatchSize) {
+        LOG.debug("markerPosition: " + markerPosition + " batchSize: " + batchSize);
+      }
+    } else {
+      batchSize = (int) Math.min(targetBatchSize, (rowCountInStripe - rowInStripe));
+    }
+    return batchSize;
+  }
+
+  @Override
+  public void close() throws IOException {
+    clearStreams();
+    dataReader.close();
+  }
+
+  @Override
+  public long getRowNumber() {
+    return rowInStripe + rowBaseInStripe + firstRow;
+  }
+
+  /**
+   * Return the fraction of rows that have been read from the selected.
+   * section of the file
+   *
+   * @return fraction between 0.0 and 1.0 of rows consumed
+   */
+  @Override
+  public float getProgress() {
+    return ((float) rowBaseInStripe + rowInStripe) / totalRowCount;
+  }
+
+  private int findStripe(long rowNumber) {
+    for (int i = 0; i < stripes.size(); i++) {
+      StripeInformation stripe = stripes.get(i);
+      if (stripe.getNumberOfRows() > rowNumber) {
+        return i;
+      }
+      rowNumber -= stripe.getNumberOfRows();
+    }
+    throw new IllegalArgumentException("Seek after the end of reader range");
+  }
+
+  public OrcIndex readRowIndex(int stripeIndex, boolean[] included,
+                               boolean[] sargColumns) throws IOException {
+    return readRowIndex(stripeIndex, included, null, null, sargColumns);
+  }
+
+  public OrcIndex readRowIndex(int stripeIndex, boolean[] included,
+                               OrcProto.RowIndex[] indexes,
+                               OrcProto.BloomFilterIndex[] bloomFilterIndex,
+                               boolean[] sargColumns) throws IOException {
+    StripeInformation stripe = stripes.get(stripeIndex);
+    OrcProto.StripeFooter stripeFooter = null;
+    // if this is the current stripe, use the cached objects.
+    if (stripeIndex == currentStripe) {
+      stripeFooter = this.stripeFooter;
+      indexes = indexes == null ? this.indexes : indexes;
+      bloomFilterIndex = bloomFilterIndex == null ? this.bloomFilterIndices : bloomFilterIndex;
+      sargColumns = sargColumns == null ?
+          (sargApp == null ? null : sargApp.sargColumns) : sargColumns;
+    }
+    return dataReader.readRowIndex(stripe, stripeFooter, included, indexes, sargColumns,
+        bloomFilterIndex);
+  }
+
+  private void seekToRowEntry(TreeReaderFactory.TreeReader reader, int rowEntry)
+      throws IOException {
+    PositionProvider[] index = new PositionProvider[indexes.length];
+    for (int i = 0; i < indexes.length; ++i) {
+      if (indexes[i] != null) {
+        index[i] = new PositionProviderImpl(indexes[i].getEntry(rowEntry));
+      }
+    }
+    reader.seek(index);
+  }
+
+  @Override
+  public void seekToRow(long rowNumber) throws IOException {
+    if (rowNumber < 0) {
+      throw new IllegalArgumentException("Seek to a negative row number " +
+          rowNumber);
+    } else if (rowNumber < firstRow) {
+      throw new IllegalArgumentException("Seek before reader range " +
+          rowNumber);
+    }
+    // convert to our internal form (rows from the beginning of slice)
+    rowNumber -= firstRow;
+
+    // move to the right stripe
+    int rightStripe = findStripe(rowNumber);
+    if (rightStripe != currentStripe) {
+      currentStripe = rightStripe;
+      readStripe();
+    }
+    readRowIndex(currentStripe, included, sargApp == null ? null : sargApp.sargColumns);
+
+    // if we aren't to the right row yet, advance in the stripe.
+    advanceToNextRow(reader, rowNumber, true);
+  }
+
+  private static final String TRANSLATED_SARG_SEPARATOR = "_";
+  public static String encodeTranslatedSargColumn(int rootColumn, Integer indexInSourceTable) {
+    return rootColumn + TRANSLATED_SARG_SEPARATOR
+        + ((indexInSourceTable == null) ? -1 : indexInSourceTable);
+  }
+
+  public static int[] mapTranslatedSargColumns(
+      List<OrcProto.Type> types, List<PredicateLeaf> sargLeaves) {
+    int[] result = new int[sargLeaves.size()];
+    OrcProto.Type lastRoot = null; // Root will be the same for everyone as of now.
+    String lastRootStr = null;
+    for (int i = 0; i < result.length; ++i) {
+      String[] rootAndIndex = sargLeaves.get(i).getColumnName().split(TRANSLATED_SARG_SEPARATOR);
+      assert rootAndIndex.length == 2;
+      String rootStr = rootAndIndex[0], indexStr = rootAndIndex[1];
+      int index = Integer.parseInt(indexStr);
+      // First, check if the column even maps to anything.
+      if (index == -1) {
+        result[i] = -1;
+        continue;
+      }
+      assert index >= 0;
+      // Then, find the root type if needed.
+      if (!rootStr.equals(lastRootStr)) {
+        lastRoot = types.get(Integer.parseInt(rootStr));
+        lastRootStr = rootStr;
+      }
+      // Subtypes of the root types correspond, in order, to the columns in the table schema
+      // (disregarding schema evolution that doesn't presently work). Get the index for the
+      // corresponding subtype.
+      result[i] = lastRoot.getSubtypes(index);
+    }
+    return result;
+  }
+}


[03/27] hive git commit: HIVE-11417. Move the ReaderImpl and RowReaderImpl to the ORC module, by making shims for the row by row reader. (omalley reviewed by prasanth_j)

Posted by om...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/ql/src/test/resources/orc-file-dump-bloomfilter.out
----------------------------------------------------------------------
diff --git a/ql/src/test/resources/orc-file-dump-bloomfilter.out b/ql/src/test/resources/orc-file-dump-bloomfilter.out
deleted file mode 100644
index 18fd2fb..0000000
--- a/ql/src/test/resources/orc-file-dump-bloomfilter.out
+++ /dev/null
@@ -1,179 +0,0 @@
-Structure for TestFileDump.testDump.orc
-File Version: 0.12 with HIVE_13083
-Rows: 21000
-Compression: ZLIB
-Compression size: 4096
-Type: struct<i:int,l:bigint,s:string>
-
-Stripe Statistics:
-  Stripe 1:
-    Column 0: count: 5000 hasNull: false
-    Column 1: count: 5000 hasNull: false min: -2146021688 max: 2147223299 sum: 515792826
-    Column 2: count: 5000 hasNull: false min: -9218592812243954469 max: 9221614132680747961
-    Column 3: count: 5000 hasNull: false min: Darkness, max: worst sum: 19280
-  Stripe 2:
-    Column 0: count: 5000 hasNull: false
-    Column 1: count: 5000 hasNull: false min: -2146733128 max: 2147001622 sum: 7673427
-    Column 2: count: 5000 hasNull: false min: -9220818777591257749 max: 9222259462014003839
-    Column 3: count: 5000 hasNull: false min: Darkness, max: worst sum: 19504
-  Stripe 3:
-    Column 0: count: 5000 hasNull: false
-    Column 1: count: 5000 hasNull: false min: -2146993718 max: 2147378179 sum: 132660742551
-    Column 2: count: 5000 hasNull: false min: -9218342074710552826 max: 9222303228623055266
-    Column 3: count: 5000 hasNull: false min: Darkness, max: worst sum: 19641
-  Stripe 4:
-    Column 0: count: 5000 hasNull: false
-    Column 1: count: 5000 hasNull: false min: -2146658006 max: 2145520931 sum: 8533549236
-    Column 2: count: 5000 hasNull: false min: -9222758097219661129 max: 9221043130193737406
-    Column 3: count: 5000 hasNull: false min: Darkness, max: worst sum: 19470
-  Stripe 5:
-    Column 0: count: 1000 hasNull: false
-    Column 1: count: 1000 hasNull: false min: -2146245500 max: 2146378640 sum: 51299706363
-    Column 2: count: 1000 hasNull: false min: -9208193203370316142 max: 9218567213558056476
-    Column 3: count: 1000 hasNull: false min: Darkness, max: worst sum: 3866
-
-File Statistics:
-  Column 0: count: 21000 hasNull: false
-  Column 1: count: 21000 hasNull: false min: -2146993718 max: 2147378179 sum: 193017464403
-  Column 2: count: 21000 hasNull: false min: -9222758097219661129 max: 9222303228623055266
-  Column 3: count: 21000 hasNull: false min: Darkness, max: worst sum: 81761
-
-Stripes:
-  Stripe: offset: 3 data: 63786 rows: 5000 tail: 86 index: 951
-    Stream: column 0 section ROW_INDEX start: 3 length 17
-    Stream: column 1 section ROW_INDEX start: 20 length 166
-    Stream: column 2 section ROW_INDEX start: 186 length 169
-    Stream: column 3 section ROW_INDEX start: 355 length 87
-    Stream: column 3 section BLOOM_FILTER start: 442 length 512
-    Stream: column 1 section DATA start: 954 length 20035
-    Stream: column 2 section DATA start: 20989 length 40050
-    Stream: column 3 section DATA start: 61039 length 3543
-    Stream: column 3 section LENGTH start: 64582 length 25
-    Stream: column 3 section DICTIONARY_DATA start: 64607 length 133
-    Encoding column 0: DIRECT
-    Encoding column 1: DIRECT_V2
-    Encoding column 2: DIRECT_V2
-    Encoding column 3: DICTIONARY_V2[35]
-    Row group indices for column 3:
-      Entry 0: count: 1000 hasNull: false min: Darkness, max: worst sum: 3862 positions: 0,0,0
-      Entry 1: count: 1000 hasNull: false min: Darkness, max: worst sum: 3884 positions: 0,659,149
-      Entry 2: count: 1000 hasNull: false min: Darkness, max: worst sum: 3893 positions: 0,1531,3
-      Entry 3: count: 1000 hasNull: false min: Darkness, max: worst sum: 3798 positions: 0,2281,32
-      Entry 4: count: 1000 hasNull: false min: Darkness, max: worst sum: 3843 positions: 0,3033,45
-    Bloom filters for column 3:
-      Entry 0: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
-      Entry 1: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
-      Entry 2: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
-      Entry 3: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
-      Entry 4: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
-      Stripe level merge: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
-  Stripe: offset: 64826 data: 63775 rows: 5000 tail: 86 index: 944
-    Stream: column 0 section ROW_INDEX start: 64826 length 17
-    Stream: column 1 section ROW_INDEX start: 64843 length 164
-    Stream: column 2 section ROW_INDEX start: 65007 length 168
-    Stream: column 3 section ROW_INDEX start: 65175 length 83
-    Stream: column 3 section BLOOM_FILTER start: 65258 length 512
-    Stream: column 1 section DATA start: 65770 length 20035
-    Stream: column 2 section DATA start: 85805 length 40050
-    Stream: column 3 section DATA start: 125855 length 3532
-    Stream: column 3 section LENGTH start: 129387 length 25
-    Stream: column 3 section DICTIONARY_DATA start: 129412 length 133
-    Encoding column 0: DIRECT
-    Encoding column 1: DIRECT_V2
-    Encoding column 2: DIRECT_V2
-    Encoding column 3: DICTIONARY_V2[35]
-    Row group indices for column 3:
-      Entry 0: count: 1000 hasNull: false min: Darkness, max: worst sum: 3923 positions: 0,0,0
-      Entry 1: count: 1000 hasNull: false min: Darkness, max: worst sum: 3869 positions: 0,761,12
-      Entry 2: count: 1000 hasNull: false min: Darkness, max: worst sum: 3817 positions: 0,1472,70
-      Entry 3: count: 1000 hasNull: false min: Darkness, max: worst sum: 3931 positions: 0,2250,43
-      Entry 4: count: 1000 hasNull: false min: Darkness, max: worst sum: 3964 positions: 0,2978,88
-    Bloom filters for column 3:
-      Entry 0: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
-      Entry 1: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
-      Entry 2: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
-      Entry 3: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
-      Entry 4: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
-      Stripe level merge: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
-  Stripe: offset: 129631 data: 63787 rows: 5000 tail: 86 index: 950
-    Stream: column 0 section ROW_INDEX start: 129631 length 17
-    Stream: column 1 section ROW_INDEX start: 129648 length 163
-    Stream: column 2 section ROW_INDEX start: 129811 length 168
-    Stream: column 3 section ROW_INDEX start: 129979 length 90
-    Stream: column 3 section BLOOM_FILTER start: 130069 length 512
-    Stream: column 1 section DATA start: 130581 length 20035
-    Stream: column 2 section DATA start: 150616 length 40050
-    Stream: column 3 section DATA start: 190666 length 3544
-    Stream: column 3 section LENGTH start: 194210 length 25
-    Stream: column 3 section DICTIONARY_DATA start: 194235 length 133
-    Encoding column 0: DIRECT
-    Encoding column 1: DIRECT_V2
-    Encoding column 2: DIRECT_V2
-    Encoding column 3: DICTIONARY_V2[35]
-    Row group indices for column 3:
-      Entry 0: count: 1000 hasNull: false min: Darkness, max: worst sum: 3817 positions: 0,0,0
-      Entry 1: count: 1000 hasNull: false min: Darkness, max: worst sum: 4008 positions: 0,634,174
-      Entry 2: count: 1000 hasNull: false min: Darkness, max: worst sum: 3999 positions: 0,1469,69
-      Entry 3: count: 1000 hasNull: false min: Darkness, max: worst sum: 3817 positions: 0,2133,194
-      Entry 4: count: 1000 hasNull: false min: Darkness, max: worst sum: 4000 positions: 0,3005,43
-    Bloom filters for column 3:
-      Entry 0: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
-      Entry 1: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
-      Entry 2: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
-      Entry 3: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
-      Entry 4: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
-      Stripe level merge: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
-  Stripe: offset: 194454 data: 63817 rows: 5000 tail: 86 index: 952
-    Stream: column 0 section ROW_INDEX start: 194454 length 17
-    Stream: column 1 section ROW_INDEX start: 194471 length 165
-    Stream: column 2 section ROW_INDEX start: 194636 length 167
-    Stream: column 3 section ROW_INDEX start: 194803 length 91
-    Stream: column 3 section BLOOM_FILTER start: 194894 length 512
-    Stream: column 1 section DATA start: 195406 length 20035
-    Stream: column 2 section DATA start: 215441 length 40050
-    Stream: column 3 section DATA start: 255491 length 3574
-    Stream: column 3 section LENGTH start: 259065 length 25
-    Stream: column 3 section DICTIONARY_DATA start: 259090 length 133
-    Encoding column 0: DIRECT
-    Encoding column 1: DIRECT_V2
-    Encoding column 2: DIRECT_V2
-    Encoding column 3: DICTIONARY_V2[35]
-    Row group indices for column 3:
-      Entry 0: count: 1000 hasNull: false min: Darkness, max: worst sum: 3901 positions: 0,0,0
-      Entry 1: count: 1000 hasNull: false min: Darkness, max: worst sum: 3900 positions: 0,431,431
-      Entry 2: count: 1000 hasNull: false min: Darkness, max: worst sum: 3909 positions: 0,1485,52
-      Entry 3: count: 1000 hasNull: false min: Darkness, max: worst sum: 3947 positions: 0,2196,104
-      Entry 4: count: 1000 hasNull: false min: Darkness, max: worst sum: 3813 positions: 0,2934,131
-    Bloom filters for column 3:
-      Entry 0: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
-      Entry 1: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
-      Entry 2: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
-      Entry 3: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
-      Entry 4: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
-      Stripe level merge: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
-  Stripe: offset: 259309 data: 12943 rows: 1000 tail: 78 index: 432
-    Stream: column 0 section ROW_INDEX start: 259309 length 12
-    Stream: column 1 section ROW_INDEX start: 259321 length 38
-    Stream: column 2 section ROW_INDEX start: 259359 length 41
-    Stream: column 3 section ROW_INDEX start: 259400 length 40
-    Stream: column 3 section BLOOM_FILTER start: 259440 length 301
-    Stream: column 1 section DATA start: 259741 length 4007
-    Stream: column 2 section DATA start: 263748 length 8010
-    Stream: column 3 section DATA start: 271758 length 768
-    Stream: column 3 section LENGTH start: 272526 length 25
-    Stream: column 3 section DICTIONARY_DATA start: 272551 length 133
-    Encoding column 0: DIRECT
-    Encoding column 1: DIRECT_V2
-    Encoding column 2: DIRECT_V2
-    Encoding column 3: DICTIONARY_V2[35]
-    Row group indices for column 3:
-      Entry 0: count: 1000 hasNull: false min: Darkness, max: worst sum: 3866 positions: 0,0,0
-    Bloom filters for column 3:
-      Entry 0: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
-      Stripe level merge: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
-
-File length: 273307 bytes
-Padding length: 0 bytes
-Padding ratio: 0%
-________________________________________________________________________________________________________________________
-

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/ql/src/test/resources/orc-file-dump-bloomfilter2.out
----------------------------------------------------------------------
diff --git a/ql/src/test/resources/orc-file-dump-bloomfilter2.out b/ql/src/test/resources/orc-file-dump-bloomfilter2.out
deleted file mode 100644
index fa5cc2d..0000000
--- a/ql/src/test/resources/orc-file-dump-bloomfilter2.out
+++ /dev/null
@@ -1,179 +0,0 @@
-Structure for TestFileDump.testDump.orc
-File Version: 0.12 with HIVE_13083
-Rows: 21000
-Compression: ZLIB
-Compression size: 4096
-Type: struct<i:int,l:bigint,s:string>
-
-Stripe Statistics:
-  Stripe 1:
-    Column 0: count: 5000 hasNull: false
-    Column 1: count: 5000 hasNull: false min: -2146021688 max: 2147223299 sum: 515792826
-    Column 2: count: 5000 hasNull: false min: -9218592812243954469 max: 9221614132680747961
-    Column 3: count: 5000 hasNull: false min: Darkness, max: worst sum: 19280
-  Stripe 2:
-    Column 0: count: 5000 hasNull: false
-    Column 1: count: 5000 hasNull: false min: -2146733128 max: 2147001622 sum: 7673427
-    Column 2: count: 5000 hasNull: false min: -9220818777591257749 max: 9222259462014003839
-    Column 3: count: 5000 hasNull: false min: Darkness, max: worst sum: 19504
-  Stripe 3:
-    Column 0: count: 5000 hasNull: false
-    Column 1: count: 5000 hasNull: false min: -2146993718 max: 2147378179 sum: 132660742551
-    Column 2: count: 5000 hasNull: false min: -9218342074710552826 max: 9222303228623055266
-    Column 3: count: 5000 hasNull: false min: Darkness, max: worst sum: 19641
-  Stripe 4:
-    Column 0: count: 5000 hasNull: false
-    Column 1: count: 5000 hasNull: false min: -2146658006 max: 2145520931 sum: 8533549236
-    Column 2: count: 5000 hasNull: false min: -9222758097219661129 max: 9221043130193737406
-    Column 3: count: 5000 hasNull: false min: Darkness, max: worst sum: 19470
-  Stripe 5:
-    Column 0: count: 1000 hasNull: false
-    Column 1: count: 1000 hasNull: false min: -2146245500 max: 2146378640 sum: 51299706363
-    Column 2: count: 1000 hasNull: false min: -9208193203370316142 max: 9218567213558056476
-    Column 3: count: 1000 hasNull: false min: Darkness, max: worst sum: 3866
-
-File Statistics:
-  Column 0: count: 21000 hasNull: false
-  Column 1: count: 21000 hasNull: false min: -2146993718 max: 2147378179 sum: 193017464403
-  Column 2: count: 21000 hasNull: false min: -9222758097219661129 max: 9222303228623055266
-  Column 3: count: 21000 hasNull: false min: Darkness, max: worst sum: 81761
-
-Stripes:
-  Stripe: offset: 3 data: 63786 rows: 5000 tail: 85 index: 6974
-    Stream: column 0 section ROW_INDEX start: 3 length 17
-    Stream: column 1 section ROW_INDEX start: 20 length 166
-    Stream: column 2 section ROW_INDEX start: 186 length 169
-    Stream: column 2 section BLOOM_FILTER start: 355 length 6535
-    Stream: column 3 section ROW_INDEX start: 6890 length 87
-    Stream: column 1 section DATA start: 6977 length 20035
-    Stream: column 2 section DATA start: 27012 length 40050
-    Stream: column 3 section DATA start: 67062 length 3543
-    Stream: column 3 section LENGTH start: 70605 length 25
-    Stream: column 3 section DICTIONARY_DATA start: 70630 length 133
-    Encoding column 0: DIRECT
-    Encoding column 1: DIRECT_V2
-    Encoding column 2: DIRECT_V2
-    Encoding column 3: DICTIONARY_V2[35]
-    Row group indices for column 2:
-      Entry 0: count: 1000 hasNull: false min: -9200577545527640566 max: 9175500305011173751 positions: 0,0,0
-      Entry 1: count: 1000 hasNull: false min: -9203618157670445774 max: 9208123824411178101 positions: 4099,2,488
-      Entry 2: count: 1000 hasNull: false min: -9218592812243954469 max: 9221351515892923972 positions: 12297,6,464
-      Entry 3: count: 1000 hasNull: false min: -9206585617947511272 max: 9167703224425685487 positions: 20495,10,440
-      Entry 4: count: 1000 hasNull: false min: -9206645795733282496 max: 9221614132680747961 positions: 28693,14,416
-    Bloom filters for column 2:
-      Entry 0: numHashFunctions: 7 bitCount: 9600 popCount: 4931 loadFactor: 0.5136 expectedFpp: 0.009432924
-      Entry 1: numHashFunctions: 7 bitCount: 9600 popCount: 4956 loadFactor: 0.5163 expectedFpp: 0.009772834
-      Entry 2: numHashFunctions: 7 bitCount: 9600 popCount: 4971 loadFactor: 0.5178 expectedFpp: 0.009981772
-      Entry 3: numHashFunctions: 7 bitCount: 9600 popCount: 4971 loadFactor: 0.5178 expectedFpp: 0.009981772
-      Entry 4: numHashFunctions: 7 bitCount: 9600 popCount: 4949 loadFactor: 0.5155 expectedFpp: 0.009676614
-      Stripe level merge: numHashFunctions: 7 bitCount: 9600 popCount: 9347 loadFactor: 0.9736 expectedFpp: 0.829482
-  Stripe: offset: 70848 data: 63775 rows: 5000 tail: 85 index: 6965
-    Stream: column 0 section ROW_INDEX start: 70848 length 17
-    Stream: column 1 section ROW_INDEX start: 70865 length 164
-    Stream: column 2 section ROW_INDEX start: 71029 length 168
-    Stream: column 2 section BLOOM_FILTER start: 71197 length 6533
-    Stream: column 3 section ROW_INDEX start: 77730 length 83
-    Stream: column 1 section DATA start: 77813 length 20035
-    Stream: column 2 section DATA start: 97848 length 40050
-    Stream: column 3 section DATA start: 137898 length 3532
-    Stream: column 3 section LENGTH start: 141430 length 25
-    Stream: column 3 section DICTIONARY_DATA start: 141455 length 133
-    Encoding column 0: DIRECT
-    Encoding column 1: DIRECT_V2
-    Encoding column 2: DIRECT_V2
-    Encoding column 3: DICTIONARY_V2[35]
-    Row group indices for column 2:
-      Entry 0: count: 1000 hasNull: false min: -9218450653857701562 max: 9189819526332228512 positions: 0,0,0
-      Entry 1: count: 1000 hasNull: false min: -9220818777591257749 max: 9178821722829648113 positions: 4099,2,488
-      Entry 2: count: 1000 hasNull: false min: -9220031433030423388 max: 9210838931786956852 positions: 12297,6,464
-      Entry 3: count: 1000 hasNull: false min: -9208195729739635607 max: 9222259462014003839 positions: 20495,10,440
-      Entry 4: count: 1000 hasNull: false min: -9174271499932339698 max: 9212277876771676916 positions: 28693,14,416
-    Bloom filters for column 2:
-      Entry 0: numHashFunctions: 7 bitCount: 9600 popCount: 4971 loadFactor: 0.5178 expectedFpp: 0.009981772
-      Entry 1: numHashFunctions: 7 bitCount: 9600 popCount: 4988 loadFactor: 0.5196 expectedFpp: 0.010223193
-      Entry 2: numHashFunctions: 7 bitCount: 9600 popCount: 5002 loadFactor: 0.521 expectedFpp: 0.01042575
-      Entry 3: numHashFunctions: 7 bitCount: 9600 popCount: 4962 loadFactor: 0.5169 expectedFpp: 0.009855959
-      Entry 4: numHashFunctions: 7 bitCount: 9600 popCount: 4966 loadFactor: 0.5173 expectedFpp: 0.009911705
-      Stripe level merge: numHashFunctions: 7 bitCount: 9600 popCount: 9344 loadFactor: 0.9733 expectedFpp: 0.8276205
-  Stripe: offset: 141673 data: 63787 rows: 5000 tail: 85 index: 6971
-    Stream: column 0 section ROW_INDEX start: 141673 length 17
-    Stream: column 1 section ROW_INDEX start: 141690 length 163
-    Stream: column 2 section ROW_INDEX start: 141853 length 168
-    Stream: column 2 section BLOOM_FILTER start: 142021 length 6533
-    Stream: column 3 section ROW_INDEX start: 148554 length 90
-    Stream: column 1 section DATA start: 148644 length 20035
-    Stream: column 2 section DATA start: 168679 length 40050
-    Stream: column 3 section DATA start: 208729 length 3544
-    Stream: column 3 section LENGTH start: 212273 length 25
-    Stream: column 3 section DICTIONARY_DATA start: 212298 length 133
-    Encoding column 0: DIRECT
-    Encoding column 1: DIRECT_V2
-    Encoding column 2: DIRECT_V2
-    Encoding column 3: DICTIONARY_V2[35]
-    Row group indices for column 2:
-      Entry 0: count: 1000 hasNull: false min: -9211978436552246208 max: 9179058898902097152 positions: 0,0,0
-      Entry 1: count: 1000 hasNull: false min: -9195645160817780503 max: 9189147759444307708 positions: 4099,2,488
-      Entry 2: count: 1000 hasNull: false min: -9202888157616520823 max: 9193561362676960747 positions: 12297,6,464
-      Entry 3: count: 1000 hasNull: false min: -9216318198067839390 max: 9221286760675829363 positions: 20495,10,440
-      Entry 4: count: 1000 hasNull: false min: -9218342074710552826 max: 9222303228623055266 positions: 28693,14,416
-    Bloom filters for column 2:
-      Entry 0: numHashFunctions: 7 bitCount: 9600 popCount: 4967 loadFactor: 0.5174 expectedFpp: 0.009925688
-      Entry 1: numHashFunctions: 7 bitCount: 9600 popCount: 5002 loadFactor: 0.521 expectedFpp: 0.01042575
-      Entry 2: numHashFunctions: 7 bitCount: 9600 popCount: 4964 loadFactor: 0.5171 expectedFpp: 0.009883798
-      Entry 3: numHashFunctions: 7 bitCount: 9600 popCount: 4943 loadFactor: 0.5149 expectedFpp: 0.009594797
-      Entry 4: numHashFunctions: 7 bitCount: 9600 popCount: 4930 loadFactor: 0.5135 expectedFpp: 0.009419539
-      Stripe level merge: numHashFunctions: 7 bitCount: 9600 popCount: 9333 loadFactor: 0.9722 expectedFpp: 0.82082444
-  Stripe: offset: 212516 data: 63817 rows: 5000 tail: 85 index: 6964
-    Stream: column 0 section ROW_INDEX start: 212516 length 17
-    Stream: column 1 section ROW_INDEX start: 212533 length 165
-    Stream: column 2 section ROW_INDEX start: 212698 length 167
-    Stream: column 2 section BLOOM_FILTER start: 212865 length 6524
-    Stream: column 3 section ROW_INDEX start: 219389 length 91
-    Stream: column 1 section DATA start: 219480 length 20035
-    Stream: column 2 section DATA start: 239515 length 40050
-    Stream: column 3 section DATA start: 279565 length 3574
-    Stream: column 3 section LENGTH start: 283139 length 25
-    Stream: column 3 section DICTIONARY_DATA start: 283164 length 133
-    Encoding column 0: DIRECT
-    Encoding column 1: DIRECT_V2
-    Encoding column 2: DIRECT_V2
-    Encoding column 3: DICTIONARY_V2[35]
-    Row group indices for column 2:
-      Entry 0: count: 1000 hasNull: false min: -9222731174895935707 max: 9214167447015056056 positions: 0,0,0
-      Entry 1: count: 1000 hasNull: false min: -9222758097219661129 max: 9221043130193737406 positions: 4099,2,488
-      Entry 2: count: 1000 hasNull: false min: -9174483776261243438 max: 9208134757538374043 positions: 12297,6,464
-      Entry 3: count: 1000 hasNull: false min: -9174329712613510612 max: 9197412874152820822 positions: 20495,10,440
-      Entry 4: count: 1000 hasNull: false min: -9221162005892422758 max: 9220625004936875965 positions: 28693,14,416
-    Bloom filters for column 2:
-      Entry 0: numHashFunctions: 7 bitCount: 9600 popCount: 4951 loadFactor: 0.5157 expectedFpp: 0.009704026
-      Entry 1: numHashFunctions: 7 bitCount: 9600 popCount: 4969 loadFactor: 0.5176 expectedFpp: 0.009953696
-      Entry 2: numHashFunctions: 7 bitCount: 9600 popCount: 4994 loadFactor: 0.5202 expectedFpp: 0.010309587
-      Entry 3: numHashFunctions: 7 bitCount: 9600 popCount: 4941 loadFactor: 0.5147 expectedFpp: 0.009567649
-      Entry 4: numHashFunctions: 7 bitCount: 9600 popCount: 4993 loadFactor: 0.5201 expectedFpp: 0.010295142
-      Stripe level merge: numHashFunctions: 7 bitCount: 9600 popCount: 9353 loadFactor: 0.9743 expectedFpp: 0.8332165
-  Stripe: offset: 283382 data: 12943 rows: 1000 tail: 78 index: 1468
-    Stream: column 0 section ROW_INDEX start: 283382 length 12
-    Stream: column 1 section ROW_INDEX start: 283394 length 38
-    Stream: column 2 section ROW_INDEX start: 283432 length 41
-    Stream: column 2 section BLOOM_FILTER start: 283473 length 1337
-    Stream: column 3 section ROW_INDEX start: 284810 length 40
-    Stream: column 1 section DATA start: 284850 length 4007
-    Stream: column 2 section DATA start: 288857 length 8010
-    Stream: column 3 section DATA start: 296867 length 768
-    Stream: column 3 section LENGTH start: 297635 length 25
-    Stream: column 3 section DICTIONARY_DATA start: 297660 length 133
-    Encoding column 0: DIRECT
-    Encoding column 1: DIRECT_V2
-    Encoding column 2: DIRECT_V2
-    Encoding column 3: DICTIONARY_V2[35]
-    Row group indices for column 2:
-      Entry 0: count: 1000 hasNull: false min: -9208193203370316142 max: 9218567213558056476 positions: 0,0,0
-    Bloom filters for column 2:
-      Entry 0: numHashFunctions: 7 bitCount: 9600 popCount: 4948 loadFactor: 0.5154 expectedFpp: 0.00966294
-      Stripe level merge: numHashFunctions: 7 bitCount: 9600 popCount: 4948 loadFactor: 0.5154 expectedFpp: 0.00966294
-
-File length: 298416 bytes
-Padding length: 0 bytes
-Padding ratio: 0%
-________________________________________________________________________________________________________________________
-

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/ql/src/test/resources/orc-file-dump-dictionary-threshold.out
----------------------------------------------------------------------
diff --git a/ql/src/test/resources/orc-file-dump-dictionary-threshold.out b/ql/src/test/resources/orc-file-dump-dictionary-threshold.out
deleted file mode 100644
index 17a964b..0000000
--- a/ql/src/test/resources/orc-file-dump-dictionary-threshold.out
+++ /dev/null
@@ -1,190 +0,0 @@
-Structure for TestFileDump.testDump.orc
-File Version: 0.12 with HIVE_13083
-Rows: 21000
-Compression: ZLIB
-Compression size: 4096
-Type: struct<i:int,l:bigint,s:string>
-
-Stripe Statistics:
-  Stripe 1:
-    Column 0: count: 5000 hasNull: false
-    Column 1: count: 5000 hasNull: false min: -2147115959 max: 2145911404 sum: 159677169195
-    Column 2: count: 5000 hasNull: false min: -9216505819108477308 max: 9217851628057711416
-    Column 3: count: 5000 hasNull: false min: Darkness,-230 max: worst-54-290-346-648-908-996-1038-1080-1560-1584-1620-1744-1770-1798-1852-1966-2162-2244-2286-2296-2534-2660-3114-3676-3788-4068-4150-4706-4744 sum: 381254
-  Stripe 2:
-    Column 0: count: 5000 hasNull: false
-    Column 1: count: 5000 hasNull: false min: -2147390285 max: 2147224606 sum: -14961457759
-    Column 2: count: 5000 hasNull: false min: -9222178666167296739 max: 9221301751385928177
-    Column 3: count: 5000 hasNull: false min: Darkness,-230-368-488-586-862-930-1686-2044-2636-2652-2872-3108-3162-3192-3404-3442-3508-3542-3550-3712-3980-4146-4204-4336-4390-4418-4424-4490-4512-4650-4768-4924-4950-5210 max: worst-54-290-346-648-908-996-1038-1080-1560-1584-1620-1744-1770-1798-1852-1966-2162-2244-2286-2296-2534-2660-3114-3676-3788-4068-4150-4706-4744-5350-5420-5582-5696-5726-6006-6020-6024-6098-6184-6568-6636-6802-6994-7004-7318-7498-7758-7780-7798-7920-7952-7960-7988-8232-8256-8390-8416-8478-8620-8840-8984-9038-9128-9236-9248-9344-9594-9650-9714-9928-9938 sum: 1117994
-  Stripe 3:
-    Column 0: count: 5000 hasNull: false
-    Column 1: count: 5000 hasNull: false min: -2145842720 max: 2146718321 sum: 141092475520
-    Column 2: count: 5000 hasNull: false min: -9221963099397084326 max: 9222722740629726770
-    Column 3: count: 5000 hasNull: false min: Darkness,-230-368-488-586-862-930-1686-2044-2636-2652-2872-3108-3162-3192-3404-3442-3508-3542-3550-3712-3980-4146-4204-4336-4390-4418-4424-4490-4512-4650-4768-4924-4950-5210-5524-5630-5678-5710-5758-5952-6238-6252-6300-6366-6668-6712-6926-6942-7100-7194-7802-8030-8452-8608-8640-8862-8868-9134-9234-9412-9602-9608-9642-9678-9740-9780-10426 max: worst-54-290-346-648-908-996-1038-1080-1560-1584-1620-1744-1770-1798-1852-1966-2162-2244-2286-2296-2534-2660-3114-3676-3788-4068-4150-4706-4744-5350-5420-5582-5696-5726-6006-6020-6024-6098-6184-6568-6636-6802-6994-7004-7318-7498-7758-7780-7798-7920-7952-7960-7988-8232-8256-8390-8416-8478-8620-8840-8984-9038-9128-9236-9248-9344-9594-9650-9714-9928-9938-10178-10368-10414-10502-10732-10876-11008-11158-11410-11722-11836-11964-12054-12096-12126-12136-12202-12246-12298-12616-12774-12782-12790-12802-12976-13216-13246-13502-13766-14454-14974 sum: 1925226
-  Stripe 4:
-    Column 0: count: 5000 hasNull: false
-    Column 1: count: 5000 hasNull: false min: -2145378214 max: 2147453086 sum: -153680004530
-    Column 2: count: 5000 hasNull: false min: -9222731174895935707 max: 9222919052987871506
-    Column 3: count: 5000 hasNull: false min: Darkness,-230-368-488-586-862-930-1686-2044-2636-2652-2872-3108-3162-3192-3404-3442-3508-3542-3550-3712-3980-4146-4204-4336-4390-4418-4424-4490-4512-4650-4768-4924-4950-5210-5524-5630-5678-5710-5758-5952-6238-6252-6300-6366-6668-6712-6926-6942-7100-7194-7802-8030-8452-8608-8640-8862-8868-9134-9234-9412-9602-9608-9642-9678-9740-9780-10426-10510-10514-10706-10814-10870-10942-11028-11244-11326-11462-11496-11656-11830-12022-12178-12418-12832-13304-13448-13590-13618-13908-14188-14246-14340-14364-14394-14762-14850-14964-15048 max: worst-54-290-346-648-908-996-1038-1080-1560-1584-1620-1744-1770-1798-1852-1966-2162-2244-2286-2296-2534-2660-3114-3676-3788-4068-4150-4706-4744-5350-5420-5582-5696-5726-6006-6020-6024-6098-6184-6568-6636-6802-6994-7004-7318-7498-7758-7780-7798-7920-7952-7960-7988-8232-8256-8390-8416-8478-8620-8840-8984-9038-9128-9236-9248-9344-9594-9650-9714-9928-9938-10178-10368-10414-10502-10732-10876-11008-11158-11410-11722-11836-
 11964-12054-12096-12126-12136-12202-12246-12298-12616-12774-12782-12790-12802-12976-13216-13246-13502-13766-14454-14974-15004-15124-15252-15294-15356-15530-15610-16316-16936-17024-17122-17214-17310-17528-17682-17742-17870-17878-18010-18410-18524-18788-19204-19254-19518-19596-19786-19874-19904 sum: 2815002
-  Stripe 5:
-    Column 0: count: 1000 hasNull: false
-    Column 1: count: 1000 hasNull: false min: -2143595397 max: 2136858458 sum: -22999664100
-    Column 2: count: 1000 hasNull: false min: -9212379634781416464 max: 9197412874152820822
-    Column 3: count: 1000 hasNull: false min: Darkness,-230-368-488-586-862-930-1686-2044-2636-2652-2872-3108-3162-3192-3404-3442-3508-3542-3550-3712-3980-4146-4204-4336-4390-4418-4424-4490-4512-4650-4768-4924-4950-5210-5524-5630-5678-5710-5758-5952-6238-6252-6300-6366-6668-6712-6926-6942-7100-7194-7802-8030-8452-8608-8640-8862-8868-9134-9234-9412-9602-9608-9642-9678-9740-9780-10426-10510-10514-10706-10814-10870-10942-11028-11244-11326-11462-11496-11656-11830-12022-12178-12418-12832-13304-13448-13590-13618-13908-14188-14246-14340-14364-14394-14762-14850-14964-15048-15494-15674-15726-16006-16056-16180-16304-16332-16452-16598-16730-16810-16994-17210-17268-17786-17962-18214-18444-18446-18724-18912-18952-19164-19348-19400-19546-19776-19896-20084 max: worst-54-290-346-648-908-996-1038-1080-1560-1584-1620-1744-1770-1798-1852-1966-2162-2244-2286-2296-2534-2660-3114-3676-3788-4068-4150-4706-4744-5350-5420-5582-5696-5726-6006-6020-6024-6098-6184-6568-6636-6802-6994-7004-7318-7498-7758-7780-7
 798-7920-7952-7960-7988-8232-8256-8390-8416-8478-8620-8840-8984-9038-9128-9236-9248-9344-9594-9650-9714-9928-9938-10178-10368-10414-10502-10732-10876-11008-11158-11410-11722-11836-11964-12054-12096-12126-12136-12202-12246-12298-12616-12774-12782-12790-12802-12976-13216-13246-13502-13766-14454-14974-15004-15124-15252-15294-15356-15530-15610-16316-16936-17024-17122-17214-17310-17528-17682-17742-17870-17878-18010-18410-18524-18788-19204-19254-19518-19596-19786-19874-19904-20390-20752-20936 sum: 670762
-
-File Statistics:
-  Column 0: count: 21000 hasNull: false
-  Column 1: count: 21000 hasNull: false min: -2147390285 max: 2147453086 sum: 109128518326
-  Column 2: count: 21000 hasNull: false min: -9222731174895935707 max: 9222919052987871506
-  Column 3: count: 21000 hasNull: false min: Darkness,-230 max: worst-54-290-346-648-908-996-1038-1080-1560-1584-1620-1744-1770-1798-1852-1966-2162-2244-2286-2296-2534-2660-3114-3676-3788-4068-4150-4706-4744-5350-5420-5582-5696-5726-6006-6020-6024-6098-6184-6568-6636-6802-6994-7004-7318-7498-7758-7780-7798-7920-7952-7960-7988-8232-8256-8390-8416-8478-8620-8840-8984-9038-9128-9236-9248-9344-9594-9650-9714-9928-9938-10178-10368-10414-10502-10732-10876-11008-11158-11410-11722-11836-11964-12054-12096-12126-12136-12202-12246-12298-12616-12774-12782-12790-12802-12976-13216-13246-13502-13766-14454-14974-15004-15124-15252-15294-15356-15530-15610-16316-16936-17024-17122-17214-17310-17528-17682-17742-17870-17878-18010-18410-18524-18788-19204-19254-19518-19596-19786-19874-19904-20390-20752-20936 sum: 6910238
-
-Stripes:
-  Stripe: offset: 3 data: 163602 rows: 5000 tail: 68 index: 720
-    Stream: column 0 section ROW_INDEX start: 3 length 17
-    Stream: column 1 section ROW_INDEX start: 20 length 166
-    Stream: column 2 section ROW_INDEX start: 186 length 171
-    Stream: column 3 section ROW_INDEX start: 357 length 366
-    Stream: column 1 section DATA start: 723 length 20035
-    Stream: column 2 section DATA start: 20758 length 40050
-    Stream: column 3 section DATA start: 60808 length 99226
-    Stream: column 3 section LENGTH start: 160034 length 4291
-    Encoding column 0: DIRECT
-    Encoding column 1: DIRECT_V2
-    Encoding column 2: DIRECT_V2
-    Encoding column 3: DIRECT_V2
-    Row group indices for column 1:
-      Entry 0: count: 1000 hasNull: false min: -2132329551 max: 2145911404 sum: 61941331718 positions: 0,0,0
-      Entry 1: count: 1000 hasNull: false min: -2138433136 max: 2145210552 sum: 14574030042 positions: 0,2050,488
-      Entry 2: count: 1000 hasNull: false min: -2147115959 max: 2137805337 sum: -2032493169 positions: 4099,2054,464
-      Entry 3: count: 1000 hasNull: false min: -2137828953 max: 2145877119 sum: -3167202608 positions: 8198,2058,440
-      Entry 4: count: 1000 hasNull: false min: -2146452517 max: 2142394906 sum: 88361503212 positions: 12297,2062,416
-    Row group indices for column 2:
-      Entry 0: count: 1000 hasNull: false min: -9206837518492372266 max: 9169230975203934579 positions: 0,0,0
-      Entry 1: count: 1000 hasNull: false min: -9188878639954124284 max: 9213664245516510068 positions: 4099,2,488
-      Entry 2: count: 1000 hasNull: false min: -9211329013123260308 max: 9217851628057711416 positions: 12297,6,464
-      Entry 3: count: 1000 hasNull: false min: -9185745718227889962 max: 9181722705210917931 positions: 20495,10,440
-      Entry 4: count: 1000 hasNull: false min: -9216505819108477308 max: 9196474183833079923 positions: 28693,14,416
-    Row group indices for column 3:
-      Entry 0: count: 1000 hasNull: false min: Darkness,-230 max: worst-54-290-346-648-908-996 sum: 18442 positions: 0,0,0,0,0
-      Entry 1: count: 1000 hasNull: false min: Darkness,-230-368-488-586-862-930-1686 max: worst-54-290-346-648-908-996-1038-1080-1560-1584-1620-1744-1770-1798-1852-1966 sum: 46338 positions: 4767,2058,0,695,18
-      Entry 2: count: 1000 hasNull: false min: Darkness,-230-368-488-586-862-930-1686-2044 max: worst-54-290-346-648-908-996-1038-1080-1560-1584-1620-1744-1770-1798-1852-1966-2162-2244-2286-2296-2534-2660 sum: 75448 positions: 16464,3340,0,1554,14
-      Entry 3: count: 1000 hasNull: false min: Darkness,-230-368-488-586-862-930-1686-2044-2636-2652-2872-3108 max: worst-54-290-346-648-908-996-1038-1080-1560-1584-1620-1744-1770-1798-1852-1966-2162-2244-2286-2296-2534-2660-3114-3676-3788 sum: 104868 positions: 36532,964,0,2372,90
-      Entry 4: count: 1000 hasNull: false min: Darkness,-230-368-488-586-862-930-1686-2044-2636-2652-2872-3108-3162-3192-3404-3442-3508-3542-3550-3712-3980-4146 max: worst-54-290-346-648-908-996-1038-1080-1560-1584-1620-1744-1770-1798-1852-1966-2162-2244-2286-2296-2534-2660-3114-3676-3788-4068-4150-4706-4744 sum: 136158 positions: 63067,3432,0,3354,108
-  Stripe: offset: 164393 data: 368335 rows: 5000 tail: 69 index: 956
-    Stream: column 0 section ROW_INDEX start: 164393 length 17
-    Stream: column 1 section ROW_INDEX start: 164410 length 157
-    Stream: column 2 section ROW_INDEX start: 164567 length 166
-    Stream: column 3 section ROW_INDEX start: 164733 length 616
-    Stream: column 1 section DATA start: 165349 length 20035
-    Stream: column 2 section DATA start: 185384 length 40050
-    Stream: column 3 section DATA start: 225434 length 302715
-    Stream: column 3 section LENGTH start: 528149 length 5535
-    Encoding column 0: DIRECT
-    Encoding column 1: DIRECT_V2
-    Encoding column 2: DIRECT_V2
-    Encoding column 3: DIRECT_V2
-    Row group indices for column 1:
-      Entry 0: count: 1000 hasNull: false min: -2146021688 max: 2146838901 sum: -50979197646 positions: 0,0,0
-      Entry 1: count: 1000 hasNull: false min: -2143569489 max: 2141223179 sum: 22810066834 positions: 0,2050,488
-      Entry 2: count: 1000 hasNull: false min: -2140649392 max: 2146301701 sum: -31694882346 positions: 4099,2054,464
-      Entry 3: count: 1000 hasNull: false min: -2147390285 max: 2146299933 sum: 79371934221 positions: 8198,2058,440
-      Entry 4: count: 1000 hasNull: false min: -2145928262 max: 2147224606 sum: -34469378822 positions: 12297,2062,416
-    Row group indices for column 2:
-      Entry 0: count: 1000 hasNull: false min: -9222178666167296739 max: 9191250610515369723 positions: 0,0,0
-      Entry 1: count: 1000 hasNull: false min: -9220148577547102875 max: 9213945522531717278 positions: 4099,2,488
-      Entry 2: count: 1000 hasNull: false min: -9220818777591257749 max: 9221301751385928177 positions: 12297,6,464
-      Entry 3: count: 1000 hasNull: false min: -9220031433030423388 max: 9207856144487414148 positions: 20495,10,440
-      Entry 4: count: 1000 hasNull: false min: -9201438531577205959 max: 9212462124593119846 positions: 28693,14,416
-    Row group indices for column 3:
-      Entry 0: count: 1000 hasNull: false min: Darkness,-230-368-488-586-862-930-1686-2044-2636-2652-2872-3108-3162-3192-3404-3442-3508-3542-3550-3712-3980-4146-4204-4336-4390-4418-4424-4490-4512-4650-4768-4924-4950-5210 max: worst-54-290-346-648-908-996-1038-1080-1560-1584-1620-1744-1770-1798-1852-1966-2162-2244-2286-2296-2534-2660-3114-3676-3788-4068-4150-4706-4744-5350-5420-5582-5696-5726 sum: 166320 positions: 0,0,0,0,0
-      Entry 1: count: 1000 hasNull: false min: Darkness,-230-368-488-586-862-930-1686-2044-2636-2652-2872-3108-3162-3192-3404-3442-3508-3542-3550-3712-3980-4146-4204-4336-4390-4418-4424-4490-4512-4650-4768-4924-4950-5210-5524-5630-5678-5710-5758-5952-6238 max: worst-54-290-346-648-908-996-1038-1080-1560-1584-1620-1744-1770-1798-1852-1966-2162-2244-2286-2296-2534-2660-3114-3676-3788-4068-4150-4706-4744-5350-5420-5582-5696-5726-6006-6020-6024-6098-6184-6568-6636-6802-6994 sum: 193436 positions: 43833,2480,0,967,90
-      Entry 2: count: 1000 hasNull: false min: Darkness,-230-368-488-586-862-930-1686-2044-2636-2652-2872-3108-3162-3192-3404-3442-3508-3542-3550-3712-3980-4146-4204-4336-4390-4418-4424-4490-4512-4650-4768-4924-4950-5210-5524-5630-5678-5710-5758-5952-6238-6252-6300-6366-6668-6712-6926-6942-7100 max: worst-54-290-346-648-908-996-1038-1080-1560-1584-1620-1744-1770-1798-1852-1966-2162-2244-2286-2296-2534-2660-3114-3676-3788-4068-4150-4706-4744-5350-5420-5582-5696-5726-6006-6020-6024-6098-6184-6568-6636-6802-6994-7004-7318-7498-7758-7780-7798-7920-7952-7960-7988 sum: 224740 positions: 94117,3404,0,1945,222
-      Entry 3: count: 1000 hasNull: false min: Darkness,-230-368-488-586-862-930-1686-2044-2636-2652-2872-3108-3162-3192-3404-3442-3508-3542-3550-3712-3980-4146-4204-4336-4390-4418-4424-4490-4512-4650-4768-4924-4950-5210-5524-5630-5678-5710-5758-5952-6238-6252-6300-6366-6668-6712-6926-6942-7100-7194-7802-8030 max: worst-54-290-346-648-908-996-1038-1080-1560-1584-1620-1744-1770-1798-1852-1966-2162-2244-2286-2296-2534-2660-3114-3676-3788-4068-4150-4706-4744-5350-5420-5582-5696-5726-6006-6020-6024-6098-6184-6568-6636-6802-6994-7004-7318-7498-7758-7780-7798-7920-7952-7960-7988-8232-8256-8390-8416-8478-8620-8840-8984 sum: 252094 positions: 155111,2864,0,3268,48
-      Entry 4: count: 1000 hasNull: false min: Darkness,-230-368-488-586-862-930-1686-2044-2636-2652-2872-3108-3162-3192-3404-3442-3508-3542-3550-3712-3980-4146-4204-4336-4390-4418-4424-4490-4512-4650-4768-4924-4950-5210-5524-5630-5678-5710-5758-5952-6238-6252-6300-6366-6668-6712-6926-6942-7100-7194-7802-8030-8452-8608-8640-8862-8868-9134 max: worst-54-290-346-648-908-996-1038-1080-1560-1584-1620-1744-1770-1798-1852-1966-2162-2244-2286-2296-2534-2660-3114-3676-3788-4068-4150-4706-4744-5350-5420-5582-5696-5726-6006-6020-6024-6098-6184-6568-6636-6802-6994-7004-7318-7498-7758-7780-7798-7920-7952-7960-7988-8232-8256-8390-8416-8478-8620-8840-8984-9038-9128-9236-9248-9344-9594-9650-9714-9928-9938 sum: 281404 positions: 224570,1006,0,4064,342
-  Stripe: offset: 533753 data: 606074 rows: 5000 tail: 69 index: 1427
-    Stream: column 0 section ROW_INDEX start: 533753 length 17
-    Stream: column 1 section ROW_INDEX start: 533770 length 167
-    Stream: column 2 section ROW_INDEX start: 533937 length 168
-    Stream: column 3 section ROW_INDEX start: 534105 length 1075
-    Stream: column 1 section DATA start: 535180 length 20035
-    Stream: column 2 section DATA start: 555215 length 40050
-    Stream: column 3 section DATA start: 595265 length 540210
-    Stream: column 3 section LENGTH start: 1135475 length 5779
-    Encoding column 0: DIRECT
-    Encoding column 1: DIRECT_V2
-    Encoding column 2: DIRECT_V2
-    Encoding column 3: DIRECT_V2
-    Row group indices for column 1:
-      Entry 0: count: 1000 hasNull: false min: -2138229212 max: 2144818981 sum: -22823642812 positions: 0,0,0
-      Entry 1: count: 1000 hasNull: false min: -2145842720 max: 2144179881 sum: -12562754334 positions: 0,2050,488
-      Entry 2: count: 1000 hasNull: false min: -2143045885 max: 2146718321 sum: 82993638644 positions: 4099,2054,464
-      Entry 3: count: 1000 hasNull: false min: -2144745617 max: 2146570474 sum: 25138722367 positions: 8198,2058,440
-      Entry 4: count: 1000 hasNull: false min: -2140127150 max: 2135081620 sum: 68346511655 positions: 12297,2062,416
-    Row group indices for column 2:
-      Entry 0: count: 1000 hasNull: false min: -9204340807292138409 max: 9208698732685326961 positions: 0,0,0
-      Entry 1: count: 1000 hasNull: false min: -9221963099397084326 max: 9222722740629726770 positions: 4099,2,488
-      Entry 2: count: 1000 hasNull: false min: -9210480084701091299 max: 9207767402467343058 positions: 12297,6,464
-      Entry 3: count: 1000 hasNull: false min: -9195038026813631215 max: 9199201928563274421 positions: 20495,10,440
-      Entry 4: count: 1000 hasNull: false min: -9215483580266514322 max: 9220102792864959501 positions: 28693,14,416
-    Row group indices for column 3:
-      Entry 0: count: 1000 hasNull: false min: Darkness,-230-368-488-586-862-930-1686-2044-2636-2652-2872-3108-3162-3192-3404-3442-3508-3542-3550-3712-3980-4146-4204-4336-4390-4418-4424-4490-4512-4650-4768-4924-4950-5210-5524-5630-5678-5710-5758-5952-6238-6252-6300-6366-6668-6712-6926-6942-7100-7194-7802-8030-8452-8608-8640-8862-8868-9134-9234-9412-9602-9608-9642-9678-9740-9780-10426 max: worst-54-290-346-648-908-996-1038-1080-1560-1584-1620-1744-1770-1798-1852-1966-2162-2244-2286-2296-2534-2660-3114-3676-3788-4068-4150-4706-4744-5350-5420-5582-5696-5726-6006-6020-6024-6098-6184-6568-6636-6802-6994-7004-7318-7498-7758-7780-7798-7920-7952-7960-7988-8232-8256-8390-8416-8478-8620-8840-8984-9038-9128-9236-9248-9344-9594-9650-9714-9928-9938-10178-10368-10414-10502-10732-10876 sum: 313880 positions: 0,0,0,0,0
-      Entry 1: count: 1000 hasNull: false min: Darkness,-230-368-488-586-862-930-1686-2044-2636-2652-2872-3108-3162-3192-3404-3442-3508-3542-3550-3712-3980-4146-4204-4336-4390-4418-4424-4490-4512-4650-4768-4924-4950-5210-5524-5630-5678-5710-5758-5952-6238-6252-6300-6366-6668-6712-6926-6942-7100-7194-7802-8030-8452-8608-8640-8862-8868-9134-9234-9412-9602-9608-9642-9678-9740-9780-10426-10510-10514-10706-10814-10870-10942-11028 max: worst-54-290-346-648-908-996-1038-1080-1560-1584-1620-1744-1770-1798-1852-1966-2162-2244-2286-2296-2534-2660-3114-3676-3788-4068-4150-4706-4744-5350-5420-5582-5696-5726-6006-6020-6024-6098-6184-6568-6636-6802-6994-7004-7318-7498-7758-7780-7798-7920-7952-7960-7988-8232-8256-8390-8416-8478-8620-8840-8984-9038-9128-9236-9248-9344-9594-9650-9714-9928-9938-10178-10368-10414-10502-10732-10876-11008-11158-11410-11722-11836-11964 sum: 349542 positions: 87800,2584,0,1097,28
-      Entry 2: count: 1000 hasNull: false min: Darkness,-230-368-488-586-862-930-1686-2044-2636-2652-2872-3108-3162-3192-3404-3442-3508-3542-3550-3712-3980-4146-4204-4336-4390-4418-4424-4490-4512-4650-4768-4924-4950-5210-5524-5630-5678-5710-5758-5952-6238-6252-6300-6366-6668-6712-6926-6942-7100-7194-7802-8030-8452-8608-8640-8862-8868-9134-9234-9412-9602-9608-9642-9678-9740-9780-10426-10510-10514-10706-10814-10870-10942-11028-11244-11326-11462-11496-11656-11830-12022 max: worst-54-290-346-648-908-996-1038-1080-1560-1584-1620-1744-1770-1798-1852-1966-2162-2244-2286-2296-2534-2660-3114-3676-3788-4068-4150-4706-4744-5350-5420-5582-5696-5726-6006-6020-6024-6098-6184-6568-6636-6802-6994-7004-7318-7498-7758-7780-7798-7920-7952-7960-7988-8232-8256-8390-8416-8478-8620-8840-8984-9038-9128-9236-9248-9344-9594-9650-9714-9928-9938-10178-10368-10414-10502-10732-10876-11008-11158-11410-11722-11836-11964-12054-12096-12126-12136-12202-12246-12298-12616-12774-12782-12790-12802-12976 sum: 386538 posit
 ions: 185635,3966,0,2077,162
-      Entry 3: count: 1000 hasNull: false min: Darkness,-230-368-488-586-862-930-1686-2044-2636-2652-2872-3108-3162-3192-3404-3442-3508-3542-3550-3712-3980-4146-4204-4336-4390-4418-4424-4490-4512-4650-4768-4924-4950-5210-5524-5630-5678-5710-5758-5952-6238-6252-6300-6366-6668-6712-6926-6942-7100-7194-7802-8030-8452-8608-8640-8862-8868-9134-9234-9412-9602-9608-9642-9678-9740-9780-10426-10510-10514-10706-10814-10870-10942-11028-11244-11326-11462-11496-11656-11830-12022-12178-12418-12832-13304 max: worst-54-290-346-648-908-996-1038-1080-1560-1584-1620-1744-1770-1798-1852-1966-2162-2244-2286-2296-2534-2660-3114-3676-3788-4068-4150-4706-4744-5350-5420-5582-5696-5726-6006-6020-6024-6098-6184-6568-6636-6802-6994-7004-7318-7498-7758-7780-7798-7920-7952-7960-7988-8232-8256-8390-8416-8478-8620-8840-8984-9038-9128-9236-9248-9344-9594-9650-9714-9928-9938-10178-10368-10414-10502-10732-10876-11008-11158-11410-11722-11836-11964-12054-12096-12126-12136-12202-12246-12298-12616-12774-12782-12790-12802
 -12976-13216-13246-13502-13766 sum: 421660 positions: 295550,1384,0,3369,16
-      Entry 4: count: 1000 hasNull: false min: Darkness,-230-368-488-586-862-930-1686-2044-2636-2652-2872-3108-3162-3192-3404-3442-3508-3542-3550-3712-3980-4146-4204-4336-4390-4418-4424-4490-4512-4650-4768-4924-4950-5210-5524-5630-5678-5710-5758-5952-6238-6252-6300-6366-6668-6712-6926-6942-7100-7194-7802-8030-8452-8608-8640-8862-8868-9134-9234-9412-9602-9608-9642-9678-9740-9780-10426-10510-10514-10706-10814-10870-10942-11028-11244-11326-11462-11496-11656-11830-12022-12178-12418-12832-13304-13448-13590-13618-13908-14188 max: worst-54-290-346-648-908-996-1038-1080-1560-1584-1620-1744-1770-1798-1852-1966-2162-2244-2286-2296-2534-2660-3114-3676-3788-4068-4150-4706-4744-5350-5420-5582-5696-5726-6006-6020-6024-6098-6184-6568-6636-6802-6994-7004-7318-7498-7758-7780-7798-7920-7952-7960-7988-8232-8256-8390-8416-8478-8620-8840-8984-9038-9128-9236-9248-9344-9594-9650-9714-9928-9938-10178-10368-10414-10502-10732-10876-11008-11158-11410-11722-11836-11964-12054-12096-12126-12136-12202-12246-12298
 -12616-12774-12782-12790-12802-12976-13216-13246-13502-13766-14454-14974 sum: 453606 positions: 412768,1156,0,4041,470
-  Stripe: offset: 1141323 data: 864001 rows: 5000 tail: 69 index: 1975
-    Stream: column 0 section ROW_INDEX start: 1141323 length 17
-    Stream: column 1 section ROW_INDEX start: 1141340 length 156
-    Stream: column 2 section ROW_INDEX start: 1141496 length 168
-    Stream: column 3 section ROW_INDEX start: 1141664 length 1634
-    Stream: column 1 section DATA start: 1143298 length 20035
-    Stream: column 2 section DATA start: 1163333 length 40050
-    Stream: column 3 section DATA start: 1203383 length 798014
-    Stream: column 3 section LENGTH start: 2001397 length 5902
-    Encoding column 0: DIRECT
-    Encoding column 1: DIRECT_V2
-    Encoding column 2: DIRECT_V2
-    Encoding column 3: DIRECT_V2
-    Row group indices for column 1:
-      Entry 0: count: 1000 hasNull: false min: -2145319330 max: 2146998132 sum: -50856753363 positions: 0,0,0
-      Entry 1: count: 1000 hasNull: false min: -2134288866 max: 2147453086 sum: -17911019023 positions: 0,2050,488
-      Entry 2: count: 1000 hasNull: false min: -2139010804 max: 2144727593 sum: -24993151857 positions: 4099,2054,464
-      Entry 3: count: 1000 hasNull: false min: -2145378214 max: 2144098933 sum: -18055164052 positions: 8198,2058,440
-      Entry 4: count: 1000 hasNull: false min: -2140494429 max: 2144595861 sum: -41863916235 positions: 12297,2062,416
-    Row group indices for column 2:
-      Entry 0: count: 1000 hasNull: false min: -9172774601303513941 max: 9212917101275642143 positions: 0,0,0
-      Entry 1: count: 1000 hasNull: false min: -9218164880949195469 max: 9222919052987871506 positions: 4099,2,488
-      Entry 2: count: 1000 hasNull: false min: -9222731174895935707 max: 9214167447015056056 positions: 12297,6,464
-      Entry 3: count: 1000 hasNull: false min: -9196276654247395117 max: 9210639275226058005 positions: 20495,10,440
-      Entry 4: count: 1000 hasNull: false min: -9197393848859294562 max: 9208134757538374043 positions: 28693,14,416
-    Row group indices for column 3:
-      Entry 0: count: 1000 hasNull: false min: Darkness,-230-368-488-586-862-930-1686-2044-2636-2652-2872-3108-3162-3192-3404-3442-3508-3542-3550-3712-3980-4146-4204-4336-4390-4418-4424-4490-4512-4650-4768-4924-4950-5210-5524-5630-5678-5710-5758-5952-6238-6252-6300-6366-6668-6712-6926-6942-7100-7194-7802-8030-8452-8608-8640-8862-8868-9134-9234-9412-9602-9608-9642-9678-9740-9780-10426-10510-10514-10706-10814-10870-10942-11028-11244-11326-11462-11496-11656-11830-12022-12178-12418-12832-13304-13448-13590-13618-13908-14188-14246-14340-14364-14394-14762-14850-14964-15048 max: worst-54-290-346-648-908-996-1038-1080-1560-1584-1620-1744-1770-1798-1852-1966-2162-2244-2286-2296-2534-2660-3114-3676-3788-4068-4150-4706-4744-5350-5420-5582-5696-5726-6006-6020-6024-6098-6184-6568-6636-6802-6994-7004-7318-7498-7758-7780-7798-7920-7952-7960-7988-8232-8256-8390-8416-8478-8620-8840-8984-9038-9128-9236-9248-9344-9594-9650-9714-9928-9938-10178-10368-10414-10502-10732-10876-11008-11158-11410-11722-11836
 -11964-12054-12096-12126-12136-12202-12246-12298-12616-12774-12782-12790-12802-12976-13216-13246-13502-13766-14454-14974-15004-15124-15252-15294-15356-15530-15610 sum: 492916 positions: 0,0,0,0,0
-      Entry 1: count: 1000 hasNull: false min: Darkness,-230-368-488-586-862-930-1686-2044-2636-2652-2872-3108-3162-3192-3404-3442-3508-3542-3550-3712-3980-4146-4204-4336-4390-4418-4424-4490-4512-4650-4768-4924-4950-5210-5524-5630-5678-5710-5758-5952-6238-6252-6300-6366-6668-6712-6926-6942-7100-7194-7802-8030-8452-8608-8640-8862-8868-9134-9234-9412-9602-9608-9642-9678-9740-9780-10426-10510-10514-10706-10814-10870-10942-11028-11244-11326-11462-11496-11656-11830-12022-12178-12418-12832-13304-13448-13590-13618-13908-14188-14246-14340-14364-14394-14762-14850-14964-15048-15494-15674-15726-16006 max: worst-54-290-346-648-908-996-1038-1080-1560-1584-1620-1744-1770-1798-1852-1966-2162-2244-2286-2296-2534-2660-3114-3676-3788-4068-4150-4706-4744-5350-5420-5582-5696-5726-6006-6020-6024-6098-6184-6568-6636-6802-6994-7004-7318-7498-7758-7780-7798-7920-7952-7960-7988-8232-8256-8390-8416-8478-8620-8840-8984-9038-9128-9236-9248-9344-9594-9650-9714-9928-9938-10178-10368-10414-10502-10732-10876-11008
 -11158-11410-11722-11836-11964-12054-12096-12126-12136-12202-12246-12298-12616-12774-12782-12790-12802-12976-13216-13246-13502-13766-14454-14974-15004-15124-15252-15294-15356-15530-15610-16316-16936 sum: 527290 positions: 139298,1396,0,1077,140
-      Entry 2: count: 1000 hasNull: false min: Darkness,-230-368-488-586-862-930-1686-2044-2636-2652-2872-3108-3162-3192-3404-3442-3508-3542-3550-3712-3980-4146-4204-4336-4390-4418-4424-4490-4512-4650-4768-4924-4950-5210-5524-5630-5678-5710-5758-5952-6238-6252-6300-6366-6668-6712-6926-6942-7100-7194-7802-8030-8452-8608-8640-8862-8868-9134-9234-9412-9602-9608-9642-9678-9740-9780-10426-10510-10514-10706-10814-10870-10942-11028-11244-11326-11462-11496-11656-11830-12022-12178-12418-12832-13304-13448-13590-13618-13908-14188-14246-14340-14364-14394-14762-14850-14964-15048-15494-15674-15726-16006-16056-16180-16304-16332-16452-16598-16730-16810-16994-17210 max: worst-54-290-346-648-908-996-1038-1080-1560-1584-1620-1744-1770-1798-1852-1966-2162-2244-2286-2296-2534-2660-3114-3676-3788-4068-4150-4706-4744-5350-5420-5582-5696-5726-6006-6020-6024-6098-6184-6568-6636-6802-6994-7004-7318-7498-7758-7780-7798-7920-7952-7960-7988-8232-8256-8390-8416-8478-8620-8840-8984-9038-9128-9236-9248-9344-9594-9
 650-9714-9928-9938-10178-10368-10414-10502-10732-10876-11008-11158-11410-11722-11836-11964-12054-12096-12126-12136-12202-12246-12298-12616-12774-12782-12790-12802-12976-13216-13246-13502-13766-14454-14974-15004-15124-15252-15294-15356-15530-15610-16316-16936-17024-17122-17214-17310-17528-17682-17742-17870-17878 sum: 568274 positions: 286457,302,0,1926,462
-      Entry 3: count: 1000 hasNull: false min: Darkness,-230-368-488-586-862-930-1686-2044-2636-2652-2872-3108-3162-3192-3404-3442-3508-3542-3550-3712-3980-4146-4204-4336-4390-4418-4424-4490-4512-4650-4768-4924-4950-5210-5524-5630-5678-5710-5758-5952-6238-6252-6300-6366-6668-6712-6926-6942-7100-7194-7802-8030-8452-8608-8640-8862-8868-9134-9234-9412-9602-9608-9642-9678-9740-9780-10426-10510-10514-10706-10814-10870-10942-11028-11244-11326-11462-11496-11656-11830-12022-12178-12418-12832-13304-13448-13590-13618-13908-14188-14246-14340-14364-14394-14762-14850-14964-15048-15494-15674-15726-16006-16056-16180-16304-16332-16452-16598-16730-16810-16994-17210-17268-17786-17962-18214 max: worst-54-290-346-648-908-996-1038-1080-1560-1584-1620-1744-1770-1798-1852-1966-2162-2244-2286-2296-2534-2660-3114-3676-3788-4068-4150-4706-4744-5350-5420-5582-5696-5726-6006-6020-6024-6098-6184-6568-6636-6802-6994-7004-7318-7498-7758-7780-7798-7920-7952-7960-7988-8232-8256-8390-8416-8478-8620-8840-8984-9038-91
 28-9236-9248-9344-9594-9650-9714-9928-9938-10178-10368-10414-10502-10732-10876-11008-11158-11410-11722-11836-11964-12054-12096-12126-12136-12202-12246-12298-12616-12774-12782-12790-12802-12976-13216-13246-13502-13766-14454-14974-15004-15124-15252-15294-15356-15530-15610-16316-16936-17024-17122-17214-17310-17528-17682-17742-17870-17878-18010-18410-18524-18788 sum: 594578 positions: 447943,3328,0,3444,250
-      Entry 4: count: 1000 hasNull: false min: Darkness,-230-368-488-586-862-930-1686-2044-2636-2652-2872-3108-3162-3192-3404-3442-3508-3542-3550-3712-3980-4146-4204-4336-4390-4418-4424-4490-4512-4650-4768-4924-4950-5210-5524-5630-5678-5710-5758-5952-6238-6252-6300-6366-6668-6712-6926-6942-7100-7194-7802-8030-8452-8608-8640-8862-8868-9134-9234-9412-9602-9608-9642-9678-9740-9780-10426-10510-10514-10706-10814-10870-10942-11028-11244-11326-11462-11496-11656-11830-12022-12178-12418-12832-13304-13448-13590-13618-13908-14188-14246-14340-14364-14394-14762-14850-14964-15048-15494-15674-15726-16006-16056-16180-16304-16332-16452-16598-16730-16810-16994-17210-17268-17786-17962-18214-18444-18446-18724-18912-18952-19164 max: worst-54-290-346-648-908-996-1038-1080-1560-1584-1620-1744-1770-1798-1852-1966-2162-2244-2286-2296-2534-2660-3114-3676-3788-4068-4150-4706-4744-5350-5420-5582-5696-5726-6006-6020-6024-6098-6184-6568-6636-6802-6994-7004-7318-7498-7758-7780-7798-7920-7952-7960-7988-8232-8256-8
 390-8416-8478-8620-8840-8984-9038-9128-9236-9248-9344-9594-9650-9714-9928-9938-10178-10368-10414-10502-10732-10876-11008-11158-11410-11722-11836-11964-12054-12096-12126-12136-12202-12246-12298-12616-12774-12782-12790-12802-12976-13216-13246-13502-13766-14454-14974-15004-15124-15252-15294-15356-15530-15610-16316-16936-17024-17122-17214-17310-17528-17682-17742-17870-17878-18010-18410-18524-18788-19204-19254-19518-19596-19786-19874-19904 sum: 631944 positions: 616471,3986,3778,547,292
-  Stripe: offset: 2007368 data: 207295 rows: 1000 tail: 67 index: 841
-    Stream: column 0 section ROW_INDEX start: 2007368 length 12
-    Stream: column 1 section ROW_INDEX start: 2007380 length 38
-    Stream: column 2 section ROW_INDEX start: 2007418 length 41
-    Stream: column 3 section ROW_INDEX start: 2007459 length 750
-    Stream: column 1 section DATA start: 2008209 length 4007
-    Stream: column 2 section DATA start: 2012216 length 8010
-    Stream: column 3 section DATA start: 2020226 length 194018
-    Stream: column 3 section LENGTH start: 2214244 length 1260
-    Encoding column 0: DIRECT
-    Encoding column 1: DIRECT_V2
-    Encoding column 2: DIRECT_V2
-    Encoding column 3: DIRECT_V2
-    Row group indices for column 1:
-      Entry 0: count: 1000 hasNull: false min: -2143595397 max: 2136858458 sum: -22999664100 positions: 0,0,0
-    Row group indices for column 2:
-      Entry 0: count: 1000 hasNull: false min: -9212379634781416464 max: 9197412874152820822 positions: 0,0,0
-    Row group indices for column 3:
-      Entry 0: count: 1000 hasNull: false min: Darkness,-230-368-488-586-862-930-1686-2044-2636-2652-2872-3108-3162-3192-3404-3442-3508-3542-3550-3712-3980-4146-4204-4336-4390-4418-4424-4490-4512-4650-4768-4924-4950-5210-5524-5630-5678-5710-5758-5952-6238-6252-6300-6366-6668-6712-6926-6942-7100-7194-7802-8030-8452-8608-8640-8862-8868-9134-9234-9412-9602-9608-9642-9678-9740-9780-10426-10510-10514-10706-10814-10870-10942-11028-11244-11326-11462-11496-11656-11830-12022-12178-12418-12832-13304-13448-13590-13618-13908-14188-14246-14340-14364-14394-14762-14850-14964-15048-15494-15674-15726-16006-16056-16180-16304-16332-16452-16598-16730-16810-16994-17210-17268-17786-17962-18214-18444-18446-18724-18912-18952-19164-19348-19400-19546-19776-19896-20084 max: worst-54-290-346-648-908-996-1038-1080-1560-1584-1620-1744-1770-1798-1852-1966-2162-2244-2286-2296-2534-2660-3114-3676-3788-4068-4150-4706-4744-5350-5420-5582-5696-5726-6006-6020-6024-6098-6184-6568-6636-6802-6994-7004-7318-7498-7758-7780-
 7798-7920-7952-7960-7988-8232-8256-8390-8416-8478-8620-8840-8984-9038-9128-9236-9248-9344-9594-9650-9714-9928-9938-10178-10368-10414-10502-10732-10876-11008-11158-11410-11722-11836-11964-12054-12096-12126-12136-12202-12246-12298-12616-12774-12782-12790-12802-12976-13216-13246-13502-13766-14454-14974-15004-15124-15252-15294-15356-15530-15610-16316-16936-17024-17122-17214-17310-17528-17682-17742-17870-17878-18010-18410-18524-18788-19204-19254-19518-19596-19786-19874-19904-20390-20752-20936 sum: 670762 positions: 0,0,0,0,0
-
-File length: 2217685 bytes
-Padding length: 0 bytes
-Padding ratio: 0%
-________________________________________________________________________________________________________________________
-


[09/27] hive git commit: HIVE-11417. Move the ReaderImpl and RowReaderImpl to the ORC module, by making shims for the row by row reader. (omalley reviewed by prasanth_j)

Posted by om...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedReaderImpl.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedReaderImpl.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedReaderImpl.java
index 40cc86f..dad35e3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedReaderImpl.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedReaderImpl.java
@@ -38,7 +38,7 @@ import org.apache.orc.CompressionCodec;
 import org.apache.orc.DataReader;
 import org.apache.orc.OrcConf;
 import org.apache.orc.impl.OutStream;
-import org.apache.hadoop.hive.ql.io.orc.RecordReaderUtils;
+import org.apache.orc.impl.RecordReaderUtils;
 import org.apache.orc.impl.StreamName;
 import org.apache.orc.StripeInformation;
 import org.apache.orc.impl.BufferChunk;

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedTreeReaderFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedTreeReaderFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedTreeReaderFactory.java
index fe46446..b44da06 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedTreeReaderFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedTreeReaderFactory.java
@@ -25,7 +25,7 @@ import org.apache.hadoop.hive.common.io.encoded.EncodedColumnBatch.ColumnStreamD
 import org.apache.orc.CompressionCodec;
 import org.apache.orc.impl.PositionProvider;
 import org.apache.orc.impl.SettableUncompressedStream;
-import org.apache.hadoop.hive.ql.io.orc.TreeReaderFactory;
+import org.apache.orc.impl.TreeReaderFactory;
 import org.apache.orc.OrcProto;
 
 public class EncodedTreeReaderFactory extends TreeReaderFactory {

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands.java b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands.java
index b20ce28..e4cbd5f 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands.java
@@ -23,7 +23,6 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.txn.TxnDbUtil;
 import org.apache.hadoop.hive.ql.io.AcidUtils;
-import org.apache.hadoop.hive.ql.io.orc.FileDump;
 import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hadoop.hive.ql.txn.AcidHouseKeeperService;
@@ -36,7 +35,6 @@ import org.junit.Test;
 import org.junit.rules.TestName;
 
 import java.io.File;
-import java.io.FileNotFoundException;
 import java.io.FileOutputStream;
 import java.util.ArrayList;
 import java.util.Arrays;

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestTimestampWritableAndColumnVector.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestTimestampWritableAndColumnVector.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestTimestampWritableAndColumnVector.java
index 6c46257..2fa9ab2 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestTimestampWritableAndColumnVector.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestTimestampWritableAndColumnVector.java
@@ -20,14 +20,11 @@ package org.apache.hadoop.hive.ql.exec.vector;
 
 import org.junit.Test;
 
-import java.math.BigDecimal;
-import java.math.RoundingMode;
 import java.sql.Timestamp;
-import java.util.Date;
 import java.util.Random;
 
 import org.apache.hadoop.hive.common.type.RandomTypeUtil;
-import org.apache.hadoop.hive.serde2.io.TimestampWritable;
+import org.apache.hadoop.hive.ql.util.TimestampUtils;
 
 import static org.junit.Assert.*;
 
@@ -58,7 +55,7 @@ public class TestTimestampWritableAndColumnVector {
       if (!retrievedTimestamp.equals(randTimestamp)) {
         assertTrue(false);
       }
-      double randDouble = TimestampWritable.getDouble(randTimestamp);
+      double randDouble = TimestampUtils.getDouble(randTimestamp);
       double retrievedDouble = timestampColVector.getDouble(i);
       if (randDouble != retrievedDouble) {
         assertTrue(false);

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorTypeCasts.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorTypeCasts.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorTypeCasts.java
index 1e41fce..e7a044e 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorTypeCasts.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorTypeCasts.java
@@ -33,7 +33,6 @@ import java.util.concurrent.TimeUnit;
 
 import junit.framework.Assert;
 
-import org.apache.hadoop.hive.common.type.Decimal128;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
 import org.apache.hadoop.hive.common.type.RandomTypeUtil;
 import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
@@ -44,6 +43,7 @@ import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.*;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.*;
+import org.apache.hadoop.hive.ql.util.TimestampUtils;
 import org.apache.hadoop.hive.serde2.io.TimestampWritable;
 import org.apache.hadoop.hive.serde2.typeinfo.HiveDecimalUtils;
 import org.junit.Test;
@@ -91,8 +91,8 @@ public class TestVectorTypeCasts {
     b.cols[0].noNulls = true;
     VectorExpression expr = new CastDoubleToTimestamp(0, 1);
     expr.evaluate(b);
-    Assert.assertEquals(0.0, TimestampWritable.getDouble(resultV.asScratchTimestamp(3)));
-    Assert.assertEquals(0.5d, TimestampWritable.getDouble(resultV.asScratchTimestamp(4)));
+    Assert.assertEquals(0.0, TimestampUtils.getDouble(resultV.asScratchTimestamp(3)));
+    Assert.assertEquals(0.5d, TimestampUtils.getDouble(resultV.asScratchTimestamp(4)));
   }
 
   @Test
@@ -152,7 +152,7 @@ public class TestVectorTypeCasts {
     expr.evaluate(b);
     for (int i = 0; i < doubleValues.length; i++) {
       double actual = resultV.vector[i];
-      double doubleValue = TimestampWritable.getDouble(inV.asScratchTimestamp(i));
+      double doubleValue = TimestampUtils.getDouble(inV.asScratchTimestamp(i));
       assertEquals(actual, doubleValue, 0.000000001F);
     }
   }
@@ -382,7 +382,7 @@ public class TestVectorTypeCasts {
     TimestampColumnVector r = (TimestampColumnVector) b.cols[1];
     for (int i = 0; i < doubleValues.length; i++) {
       Timestamp timestamp = r.asScratchTimestamp(i);
-      double asDouble = TimestampWritable.getDouble(timestamp);
+      double asDouble = TimestampUtils.getDouble(timestamp);
       double expectedDouble = doubleValues[i];
       if (expectedDouble != asDouble) {
         assertTrue(false);

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/udf/TestVectorUDFAdaptor.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/udf/TestVectorUDFAdaptor.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/udf/TestVectorUDFAdaptor.java
index a7567b7..b78c1f2 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/udf/TestVectorUDFAdaptor.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/udf/TestVectorUDFAdaptor.java
@@ -27,8 +27,6 @@ import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.StringExpr;
-import org.apache.hadoop.hive.ql.exec.vector.udf.VectorUDFAdaptor;
-import org.apache.hadoop.hive.ql.exec.vector.udf.VectorUDFArgDesc;
 import org.apache.hadoop.hive.ql.exec.vector.udf.generic.GenericUDFIsNull;
 import org.apache.hadoop.hive.ql.exec.vector.udf.legacy.ConcatTextLongDoubleUDF;
 import org.apache.hadoop.hive.ql.exec.vector.udf.legacy.LongUDF;

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestColumnStatistics.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestColumnStatistics.java b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestColumnStatistics.java
deleted file mode 100644
index 5f0146f..0000000
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestColumnStatistics.java
+++ /dev/null
@@ -1,352 +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.hive.ql.io.orc;
-
-import static junit.framework.Assert.assertEquals;
-import static org.junit.Assume.assumeTrue;
-
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.PrintStream;
-import java.sql.Timestamp;
-import java.util.List;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hive.common.type.HiveDecimal;
-import org.apache.hadoop.hive.serde2.io.DateWritable;
-import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
-import org.apache.hadoop.io.BytesWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.orc.ColumnStatistics;
-import org.apache.orc.impl.ColumnStatisticsImpl;
-import org.apache.orc.DateColumnStatistics;
-import org.apache.orc.DecimalColumnStatistics;
-import org.apache.orc.DoubleColumnStatistics;
-import org.apache.orc.IntegerColumnStatistics;
-import org.apache.orc.StringColumnStatistics;
-import org.apache.orc.StripeStatistics;
-import org.apache.orc.TimestampColumnStatistics;
-import org.apache.orc.TypeDescription;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestName;
-
-/**
- * Test ColumnStatisticsImpl for ORC.
- */
-public class TestColumnStatistics {
-
-  @Test
-  public void testLongMerge() throws Exception {
-    TypeDescription schema = TypeDescription.createInt();
-
-    ColumnStatisticsImpl stats1 = ColumnStatisticsImpl.create(schema);
-    ColumnStatisticsImpl stats2 = ColumnStatisticsImpl.create(schema);
-    stats1.updateInteger(10, 2);
-    stats2.updateInteger(1, 1);
-    stats2.updateInteger(1000, 1);
-    stats1.merge(stats2);
-    IntegerColumnStatistics typed = (IntegerColumnStatistics) stats1;
-    assertEquals(1, typed.getMinimum());
-    assertEquals(1000, typed.getMaximum());
-    stats1.reset();
-    stats1.updateInteger(-10, 1);
-    stats1.updateInteger(10000, 1);
-    stats1.merge(stats2);
-    assertEquals(-10, typed.getMinimum());
-    assertEquals(10000, typed.getMaximum());
-  }
-
-  @Test
-  public void testDoubleMerge() throws Exception {
-    TypeDescription schema = TypeDescription.createDouble();
-
-    ColumnStatisticsImpl stats1 = ColumnStatisticsImpl.create(schema);
-    ColumnStatisticsImpl stats2 = ColumnStatisticsImpl.create(schema);
-    stats1.updateDouble(10.0);
-    stats1.updateDouble(100.0);
-    stats2.updateDouble(1.0);
-    stats2.updateDouble(1000.0);
-    stats1.merge(stats2);
-    DoubleColumnStatistics typed = (DoubleColumnStatistics) stats1;
-    assertEquals(1.0, typed.getMinimum(), 0.001);
-    assertEquals(1000.0, typed.getMaximum(), 0.001);
-    stats1.reset();
-    stats1.updateDouble(-10);
-    stats1.updateDouble(10000);
-    stats1.merge(stats2);
-    assertEquals(-10, typed.getMinimum(), 0.001);
-    assertEquals(10000, typed.getMaximum(), 0.001);
-  }
-
-
-  @Test
-  public void testStringMerge() throws Exception {
-    TypeDescription schema = TypeDescription.createString();
-
-    ColumnStatisticsImpl stats1 = ColumnStatisticsImpl.create(schema);
-    ColumnStatisticsImpl stats2 = ColumnStatisticsImpl.create(schema);
-    stats1.updateString(new Text("bob"));
-    stats1.updateString(new Text("david"));
-    stats1.updateString(new Text("charles"));
-    stats2.updateString(new Text("anne"));
-    byte[] erin = new byte[]{0, 1, 2, 3, 4, 5, 101, 114, 105, 110};
-    stats2.updateString(erin, 6, 4, 5);
-    assertEquals(24, ((StringColumnStatistics)stats2).getSum());
-    stats1.merge(stats2);
-    StringColumnStatistics typed = (StringColumnStatistics) stats1;
-    assertEquals("anne", typed.getMinimum());
-    assertEquals("erin", typed.getMaximum());
-    assertEquals(39, typed.getSum());
-    stats1.reset();
-    stats1.updateString(new Text("aaa"));
-    stats1.updateString(new Text("zzz"));
-    stats1.merge(stats2);
-    assertEquals("aaa", typed.getMinimum());
-    assertEquals("zzz", typed.getMaximum());
-  }
-
-  @Test
-  public void testDateMerge() throws Exception {
-    TypeDescription schema = TypeDescription.createDate();
-
-    ColumnStatisticsImpl stats1 = ColumnStatisticsImpl.create(schema);
-    ColumnStatisticsImpl stats2 = ColumnStatisticsImpl.create(schema);
-    stats1.updateDate(new DateWritable(1000));
-    stats1.updateDate(new DateWritable(100));
-    stats2.updateDate(new DateWritable(10));
-    stats2.updateDate(new DateWritable(2000));
-    stats1.merge(stats2);
-    DateColumnStatistics typed = (DateColumnStatistics) stats1;
-    assertEquals(new DateWritable(10).get(), typed.getMinimum());
-    assertEquals(new DateWritable(2000).get(), typed.getMaximum());
-    stats1.reset();
-    stats1.updateDate(new DateWritable(-10));
-    stats1.updateDate(new DateWritable(10000));
-    stats1.merge(stats2);
-    assertEquals(new DateWritable(-10).get(), typed.getMinimum());
-    assertEquals(new DateWritable(10000).get(), typed.getMaximum());
-  }
-
-  @Test
-  public void testTimestampMerge() throws Exception {
-    TypeDescription schema = TypeDescription.createTimestamp();
-
-    ColumnStatisticsImpl stats1 = ColumnStatisticsImpl.create(schema);
-    ColumnStatisticsImpl stats2 = ColumnStatisticsImpl.create(schema);
-    stats1.updateTimestamp(new Timestamp(10));
-    stats1.updateTimestamp(new Timestamp(100));
-    stats2.updateTimestamp(new Timestamp(1));
-    stats2.updateTimestamp(new Timestamp(1000));
-    stats1.merge(stats2);
-    TimestampColumnStatistics typed = (TimestampColumnStatistics) stats1;
-    assertEquals(1, typed.getMinimum().getTime());
-    assertEquals(1000, typed.getMaximum().getTime());
-    stats1.reset();
-    stats1.updateTimestamp(new Timestamp(-10));
-    stats1.updateTimestamp(new Timestamp(10000));
-    stats1.merge(stats2);
-    assertEquals(-10, typed.getMinimum().getTime());
-    assertEquals(10000, typed.getMaximum().getTime());
-  }
-
-  @Test
-  public void testDecimalMerge() throws Exception {
-    TypeDescription schema = TypeDescription.createDecimal()
-        .withPrecision(38).withScale(16);
-
-    ColumnStatisticsImpl stats1 = ColumnStatisticsImpl.create(schema);
-    ColumnStatisticsImpl stats2 = ColumnStatisticsImpl.create(schema);
-    stats1.updateDecimal(HiveDecimal.create(10));
-    stats1.updateDecimal(HiveDecimal.create(100));
-    stats2.updateDecimal(HiveDecimal.create(1));
-    stats2.updateDecimal(HiveDecimal.create(1000));
-    stats1.merge(stats2);
-    DecimalColumnStatistics typed = (DecimalColumnStatistics) stats1;
-    assertEquals(1, typed.getMinimum().longValue());
-    assertEquals(1000, typed.getMaximum().longValue());
-    stats1.reset();
-    stats1.updateDecimal(HiveDecimal.create(-10));
-    stats1.updateDecimal(HiveDecimal.create(10000));
-    stats1.merge(stats2);
-    assertEquals(-10, typed.getMinimum().longValue());
-    assertEquals(10000, typed.getMaximum().longValue());
-  }
-
-
-  public static class SimpleStruct {
-    BytesWritable bytes1;
-    Text string1;
-
-    SimpleStruct(BytesWritable b1, String s1) {
-      this.bytes1 = b1;
-      if (s1 == null) {
-        this.string1 = null;
-      } else {
-        this.string1 = new Text(s1);
-      }
-    }
-  }
-
-  Path workDir = new Path(System.getProperty("test.tmp.dir",
-      "target" + File.separator + "test" + File.separator + "tmp"));
-
-  Configuration conf;
-  FileSystem fs;
-  Path testFilePath;
-
-  @Rule
-  public TestName testCaseName = new TestName();
-
-  @Before
-  public void openFileSystem() throws Exception {
-    conf = new Configuration();
-    fs = FileSystem.getLocal(conf);
-    fs.setWorkingDirectory(workDir);
-    testFilePath = new Path("TestOrcFile." + testCaseName.getMethodName() + ".orc");
-    fs.delete(testFilePath, false);
-  }
-
-  private static BytesWritable bytes(int... items) {
-    BytesWritable result = new BytesWritable();
-    result.setSize(items.length);
-    for (int i = 0; i < items.length; ++i) {
-      result.getBytes()[i] = (byte) items[i];
-    }
-    return result;
-  }
-
-  @Test
-  public void testHasNull() throws Exception {
-
-    ObjectInspector inspector;
-    synchronized (TestOrcFile.class) {
-      inspector = ObjectInspectorFactory.getReflectionObjectInspector
-          (SimpleStruct.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    }
-    Writer writer = OrcFile.createWriter(testFilePath,
-        OrcFile.writerOptions(conf)
-            .inspector(inspector)
-            .rowIndexStride(1000)
-            .stripeSize(10000)
-            .bufferSize(10000));
-    // STRIPE 1
-    // RG1
-    for(int i=0; i<1000; i++) {
-      writer.addRow(new SimpleStruct(bytes(1,2,3), "RG1"));
-    }
-    // RG2
-    for(int i=0; i<1000; i++) {
-      writer.addRow(new SimpleStruct(bytes(1,2,3), null));
-    }
-    // RG3
-    for(int i=0; i<1000; i++) {
-      writer.addRow(new SimpleStruct(bytes(1,2,3), "RG3"));
-    }
-    // RG4
-    for(int i=0; i<1000; i++) {
-      writer.addRow(new SimpleStruct(bytes(1,2,3), null));
-    }
-    // RG5
-    for(int i=0; i<1000; i++) {
-      writer.addRow(new SimpleStruct(bytes(1,2,3), null));
-    }
-    // STRIPE 2
-    for(int i=0; i<5000; i++) {
-      writer.addRow(new SimpleStruct(bytes(1,2,3), null));
-    }
-    // STRIPE 3
-    for(int i=0; i<5000; i++) {
-      writer.addRow(new SimpleStruct(bytes(1,2,3), "STRIPE-3"));
-    }
-    // STRIPE 4
-    for(int i=0; i<5000; i++) {
-      writer.addRow(new SimpleStruct(bytes(1,2,3), null));
-    }
-    writer.close();
-    Reader reader = OrcFile.createReader(testFilePath,
-        OrcFile.readerOptions(conf).filesystem(fs));
-
-    // check the file level stats
-    ColumnStatistics[] stats = reader.getStatistics();
-    assertEquals(20000, stats[0].getNumberOfValues());
-    assertEquals(20000, stats[1].getNumberOfValues());
-    assertEquals(7000, stats[2].getNumberOfValues());
-    assertEquals(false, stats[0].hasNull());
-    assertEquals(false, stats[1].hasNull());
-    assertEquals(true, stats[2].hasNull());
-
-    // check the stripe level stats
-    List<StripeStatistics> stripeStats = reader.getStripeStatistics();
-    // stripe 1 stats
-    StripeStatistics ss1 = stripeStats.get(0);
-    ColumnStatistics ss1_cs1 = ss1.getColumnStatistics()[0];
-    ColumnStatistics ss1_cs2 = ss1.getColumnStatistics()[1];
-    ColumnStatistics ss1_cs3 = ss1.getColumnStatistics()[2];
-    assertEquals(false, ss1_cs1.hasNull());
-    assertEquals(false, ss1_cs2.hasNull());
-    assertEquals(true, ss1_cs3.hasNull());
-
-    // stripe 2 stats
-    StripeStatistics ss2 = stripeStats.get(1);
-    ColumnStatistics ss2_cs1 = ss2.getColumnStatistics()[0];
-    ColumnStatistics ss2_cs2 = ss2.getColumnStatistics()[1];
-    ColumnStatistics ss2_cs3 = ss2.getColumnStatistics()[2];
-    assertEquals(false, ss2_cs1.hasNull());
-    assertEquals(false, ss2_cs2.hasNull());
-    assertEquals(true, ss2_cs3.hasNull());
-
-    // stripe 3 stats
-    StripeStatistics ss3 = stripeStats.get(2);
-    ColumnStatistics ss3_cs1 = ss3.getColumnStatistics()[0];
-    ColumnStatistics ss3_cs2 = ss3.getColumnStatistics()[1];
-    ColumnStatistics ss3_cs3 = ss3.getColumnStatistics()[2];
-    assertEquals(false, ss3_cs1.hasNull());
-    assertEquals(false, ss3_cs2.hasNull());
-    assertEquals(false, ss3_cs3.hasNull());
-
-    // stripe 4 stats
-    StripeStatistics ss4 = stripeStats.get(3);
-    ColumnStatistics ss4_cs1 = ss4.getColumnStatistics()[0];
-    ColumnStatistics ss4_cs2 = ss4.getColumnStatistics()[1];
-    ColumnStatistics ss4_cs3 = ss4.getColumnStatistics()[2];
-    assertEquals(false, ss4_cs1.hasNull());
-    assertEquals(false, ss4_cs2.hasNull());
-    assertEquals(true, ss4_cs3.hasNull());
-
-    // Test file dump
-    PrintStream origOut = System.out;
-    String outputFilename = "orc-file-has-null.out";
-    FileOutputStream myOut = new FileOutputStream(workDir + File.separator + outputFilename);
-
-    // replace stdout and run command
-    System.setOut(new PrintStream(myOut));
-    FileDump.main(new String[]{testFilePath.toString(), "--rowindex=2"});
-    System.out.flush();
-    System.setOut(origOut);
-    // If called with an expression evaluating to false, the test will halt
-    // and be ignored.
-    assumeTrue(!System.getProperty("os.name").startsWith("Windows"));
-    TestFileDump.checkOutput(outputFilename, workDir + File.separator + outputFilename);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestFileDump.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestFileDump.java b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestFileDump.java
deleted file mode 100644
index 554033c..0000000
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestFileDump.java
+++ /dev/null
@@ -1,418 +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.hive.ql.io.orc;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
-
-import java.io.BufferedReader;
-import java.io.ByteArrayOutputStream;
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.FileReader;
-import java.io.PrintStream;
-import java.sql.Date;
-import java.sql.Timestamp;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Random;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hive.common.type.HiveChar;
-import org.apache.hadoop.hive.common.type.HiveDecimal;
-import org.apache.hadoop.hive.common.type.HiveVarchar;
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
-import org.apache.hive.common.util.HiveTestUtils;
-import org.junit.Before;
-import org.junit.Test;
-
-public class TestFileDump {
-
-  Path workDir = new Path(System.getProperty("test.tmp.dir"));
-  Configuration conf;
-  FileSystem fs;
-  Path testFilePath;
-
-  @Before
-  public void openFileSystem () throws Exception {
-    conf = new Configuration();
-    fs = FileSystem.getLocal(conf);
-    fs.setWorkingDirectory(workDir);
-    testFilePath = new Path("TestFileDump.testDump.orc");
-    fs.delete(testFilePath, false);
-  }
-
-  static class MyRecord {
-    int i;
-    long l;
-    String s;
-    MyRecord(int i, long l, String s) {
-      this.i = i;
-      this.l = l;
-      this.s = s;
-    }
-  }
-
-  static class AllTypesRecord {
-    static class Struct {
-      int i;
-      String s;
-
-      Struct(int i, String s) {
-        this.i = i;
-        this.s = s;
-      }
-    }
-    boolean b;
-    byte bt;
-    short s;
-    int i;
-    long l;
-    float f;
-    double d;
-    HiveDecimal de;
-    Timestamp t;
-    Date dt;
-    String str;
-    HiveChar c;
-    HiveVarchar vc;
-    Map<String, String> m;
-    List<Integer> a;
-    Struct st;
-
-    AllTypesRecord(boolean b, byte bt, short s, int i, long l, float f, double d, HiveDecimal de,
-                   Timestamp t, Date dt, String str, HiveChar c, HiveVarchar vc, Map<String,
-                   String> m, List<Integer> a, Struct st) {
-      this.b = b;
-      this.bt = bt;
-      this.s = s;
-      this.i = i;
-      this.l = l;
-      this.f = f;
-      this.d = d;
-      this.de = de;
-      this.t = t;
-      this.dt = dt;
-      this.str = str;
-      this.c = c;
-      this.vc = vc;
-      this.m = m;
-      this.a = a;
-      this.st = st;
-    }
-  }
-
-  static void checkOutput(String expected,
-                                  String actual) throws Exception {
-    BufferedReader eStream =
-        new BufferedReader(new FileReader(HiveTestUtils.getFileFromClasspath(expected)));
-    BufferedReader aStream =
-        new BufferedReader(new FileReader(actual));
-    String expectedLine = eStream.readLine().trim();
-    while (expectedLine != null) {
-      String actualLine = aStream.readLine().trim();
-      System.out.println("actual:   " + actualLine);
-      System.out.println("expected: " + expectedLine);
-      assertEquals(expectedLine, actualLine);
-      expectedLine = eStream.readLine();
-      expectedLine = expectedLine == null ? null : expectedLine.trim();
-    }
-    assertNull(eStream.readLine());
-    assertNull(aStream.readLine());
-    eStream.close();
-    aStream.close();
-  }
-
-  @Test
-  public void testDump() throws Exception {
-    ObjectInspector inspector;
-    synchronized (TestOrcFile.class) {
-      inspector = ObjectInspectorFactory.getReflectionObjectInspector
-          (MyRecord.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    }
-    conf.set(HiveConf.ConfVars.HIVE_ORC_ENCODING_STRATEGY.varname, "COMPRESSION");
-    Writer writer = OrcFile.createWriter(testFilePath,
-        OrcFile.writerOptions(conf)
-            .fileSystem(fs)
-            .inspector(inspector)
-            .batchSize(1000)
-            .compress(CompressionKind.ZLIB)
-            .stripeSize(100000)
-            .rowIndexStride(1000));
-    Random r1 = new Random(1);
-    String[] words = new String[]{"It", "was", "the", "best", "of", "times,",
-        "it", "was", "the", "worst", "of", "times,", "it", "was", "the", "age",
-        "of", "wisdom,", "it", "was", "the", "age", "of", "foolishness,", "it",
-        "was", "the", "epoch", "of", "belief,", "it", "was", "the", "epoch",
-        "of", "incredulity,", "it", "was", "the", "season", "of", "Light,",
-        "it", "was", "the", "season", "of", "Darkness,", "it", "was", "the",
-        "spring", "of", "hope,", "it", "was", "the", "winter", "of", "despair,",
-        "we", "had", "everything", "before", "us,", "we", "had", "nothing",
-        "before", "us,", "we", "were", "all", "going", "direct", "to",
-        "Heaven,", "we", "were", "all", "going", "direct", "the", "other",
-        "way"};
-    for(int i=0; i < 21000; ++i) {
-      writer.addRow(new MyRecord(r1.nextInt(), r1.nextLong(),
-          words[r1.nextInt(words.length)]));
-    }
-    writer.close();
-    PrintStream origOut = System.out;
-    String outputFilename = "orc-file-dump.out";
-    FileOutputStream myOut = new FileOutputStream(workDir + File.separator + outputFilename);
-
-    // replace stdout and run command
-    System.setOut(new PrintStream(myOut));
-    FileDump.main(new String[]{testFilePath.toString(), "--rowindex=1,2,3"});
-    System.out.flush();
-    System.setOut(origOut);
-
-
-    checkOutput(outputFilename, workDir + File.separator + outputFilename);
-  }
-
-  @Test
-  public void testDataDump() throws Exception {
-    ObjectInspector inspector;
-    synchronized (TestOrcFile.class) {
-      inspector = ObjectInspectorFactory.getReflectionObjectInspector
-          (AllTypesRecord.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    }
-    Writer writer = OrcFile.createWriter(fs, testFilePath, conf, inspector,
-        100000, CompressionKind.NONE, 10000, 1000);
-    Map<String, String> m = new HashMap<String, String>(2);
-    m.put("k1", "v1");
-    writer.addRow(new AllTypesRecord(
-        true,
-        (byte) 10,
-        (short) 100,
-        1000,
-        10000L,
-        4.0f,
-        20.0,
-        HiveDecimal.create("4.2222"),
-        new Timestamp(1416967764000L),
-        new Date(1416967764000L),
-        "string",
-        new HiveChar("hello", 5),
-        new HiveVarchar("hello", 10),
-        m,
-        Arrays.asList(100, 200),
-        new AllTypesRecord.Struct(10, "foo")));
-    m.clear();
-    m.put("k3", "v3");
-    writer.addRow(new AllTypesRecord(
-        false,
-        (byte)20,
-        (short)200,
-        2000,
-        20000L,
-        8.0f,
-        40.0,
-        HiveDecimal.create("2.2222"),
-        new Timestamp(1416967364000L),
-        new Date(1411967764000L),
-        "abcd",
-        new HiveChar("world", 5),
-        new HiveVarchar("world", 10),
-        m,
-        Arrays.asList(200, 300),
-        new AllTypesRecord.Struct(20, "bar")));
-
-    writer.close();
-    PrintStream origOut = System.out;
-    ByteArrayOutputStream myOut = new ByteArrayOutputStream();
-
-    // replace stdout and run command
-    System.setOut(new PrintStream(myOut));
-    FileDump.main(new String[]{testFilePath.toString(), "-d"});
-    System.out.flush();
-    System.setOut(origOut);
-
-    String[] lines = myOut.toString().split("\n");
-    // Don't be fooled by the big space in the middle, this line is quite long
-    assertEquals("{\"b\":true,\"bt\":10,\"s\":100,\"i\":1000,\"l\":10000,\"f\":4,\"d\":20,\"de\":\"4.2222\",\"t\":\"2014-11-25 18:09:24\",\"dt\":\"2014-11-25\",\"str\":\"string\",\"c\":\"hello                                                                                                                                                                                                                                                          \",\"vc\":\"hello\",\"m\":[{\"_key\":\"k1\",\"_value\":\"v1\"}],\"a\":[100,200],\"st\":{\"i\":10,\"s\":\"foo\"}}", lines[0]);
-    assertEquals("{\"b\":false,\"bt\":20,\"s\":200,\"i\":2000,\"l\":20000,\"f\":8,\"d\":40,\"de\":\"2.2222\",\"t\":\"2014-11-25 18:02:44\",\"dt\":\"2014-09-28\",\"str\":\"abcd\",\"c\":\"world                                                                                                                                                                                                                                                          \",\"vc\":\"world\",\"m\":[{\"_key\":\"k3\",\"_value\":\"v3\"}],\"a\":[200,300],\"st\":{\"i\":20,\"s\":\"bar\"}}", lines[1]);
-  }
-  
-  // Test that if the fraction of rows that have distinct strings is greater than the configured
-  // threshold dictionary encoding is turned off.  If dictionary encoding is turned off the length
-  // of the dictionary stream for the column will be 0 in the ORC file dump.
-  @Test
-  public void testDictionaryThreshold() throws Exception {
-    ObjectInspector inspector;
-    synchronized (TestOrcFile.class) {
-      inspector = ObjectInspectorFactory.getReflectionObjectInspector
-          (MyRecord.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    }
-    Configuration conf = new Configuration();
-    conf.set(HiveConf.ConfVars.HIVE_ORC_ENCODING_STRATEGY.varname, "COMPRESSION");
-    conf.setFloat(HiveConf.ConfVars.HIVE_ORC_DICTIONARY_KEY_SIZE_THRESHOLD.varname, 0.49f);
-    Writer writer = OrcFile.createWriter(testFilePath,
-        OrcFile.writerOptions(conf)
-            .fileSystem(fs)
-            .batchSize(1000)
-            .inspector(inspector)
-            .stripeSize(100000)
-            .compress(CompressionKind.ZLIB)
-            .rowIndexStride(1000)
-            .bufferSize(10000));
-    Random r1 = new Random(1);
-    String[] words = new String[]{"It", "was", "the", "best", "of", "times,",
-        "it", "was", "the", "worst", "of", "times,", "it", "was", "the", "age",
-        "of", "wisdom,", "it", "was", "the", "age", "of", "foolishness,", "it",
-        "was", "the", "epoch", "of", "belief,", "it", "was", "the", "epoch",
-        "of", "incredulity,", "it", "was", "the", "season", "of", "Light,",
-        "it", "was", "the", "season", "of", "Darkness,", "it", "was", "the",
-        "spring", "of", "hope,", "it", "was", "the", "winter", "of", "despair,",
-        "we", "had", "everything", "before", "us,", "we", "had", "nothing",
-        "before", "us,", "we", "were", "all", "going", "direct", "to",
-        "Heaven,", "we", "were", "all", "going", "direct", "the", "other",
-        "way"};
-    int nextInt = 0;
-    for(int i=0; i < 21000; ++i) {
-      // Write out the same string twice, this guarantees the fraction of rows with
-      // distinct strings is 0.5
-      if (i % 2 == 0) {
-        nextInt = r1.nextInt(words.length);
-        // Append the value of i to the word, this guarantees when an index or word is repeated
-        // the actual string is unique.
-        words[nextInt] += "-" + i;
-      }
-      writer.addRow(new MyRecord(r1.nextInt(), r1.nextLong(),
-          words[nextInt]));
-    }
-    writer.close();
-    PrintStream origOut = System.out;
-    String outputFilename = "orc-file-dump-dictionary-threshold.out";
-    FileOutputStream myOut = new FileOutputStream(workDir + File.separator + outputFilename);
-
-    // replace stdout and run command
-    System.setOut(new PrintStream(myOut));
-    FileDump.main(new String[]{testFilePath.toString(), "--rowindex=1,2,3"});
-    System.out.flush();
-    System.setOut(origOut);
-
-    checkOutput(outputFilename, workDir + File.separator + outputFilename);
-  }
-
-  @Test
-  public void testBloomFilter() throws Exception {
-    ObjectInspector inspector;
-    synchronized (TestOrcFile.class) {
-      inspector = ObjectInspectorFactory.getReflectionObjectInspector
-          (MyRecord.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    }
-    conf.set(HiveConf.ConfVars.HIVE_ORC_ENCODING_STRATEGY.varname, "COMPRESSION");
-    OrcFile.WriterOptions options = OrcFile.writerOptions(conf)
-        .fileSystem(fs)
-        .inspector(inspector)
-        .stripeSize(100000)
-        .compress(CompressionKind.ZLIB)
-        .bufferSize(10000)
-        .rowIndexStride(1000)
-        .batchSize(1000)
-        .bloomFilterColumns("S");
-    Writer writer = OrcFile.createWriter(testFilePath, options);
-    Random r1 = new Random(1);
-    String[] words = new String[]{"It", "was", "the", "best", "of", "times,",
-        "it", "was", "the", "worst", "of", "times,", "it", "was", "the", "age",
-        "of", "wisdom,", "it", "was", "the", "age", "of", "foolishness,", "it",
-        "was", "the", "epoch", "of", "belief,", "it", "was", "the", "epoch",
-        "of", "incredulity,", "it", "was", "the", "season", "of", "Light,",
-        "it", "was", "the", "season", "of", "Darkness,", "it", "was", "the",
-        "spring", "of", "hope,", "it", "was", "the", "winter", "of", "despair,",
-        "we", "had", "everything", "before", "us,", "we", "had", "nothing",
-        "before", "us,", "we", "were", "all", "going", "direct", "to",
-        "Heaven,", "we", "were", "all", "going", "direct", "the", "other",
-        "way"};
-    for(int i=0; i < 21000; ++i) {
-      writer.addRow(new MyRecord(r1.nextInt(), r1.nextLong(),
-          words[r1.nextInt(words.length)]));
-    }
-    writer.close();
-    PrintStream origOut = System.out;
-    String outputFilename = "orc-file-dump-bloomfilter.out";
-    FileOutputStream myOut = new FileOutputStream(workDir + File.separator + outputFilename);
-
-    // replace stdout and run command
-    System.setOut(new PrintStream(myOut));
-    FileDump.main(new String[]{testFilePath.toString(), "--rowindex=3"});
-    System.out.flush();
-    System.setOut(origOut);
-
-
-    checkOutput(outputFilename, workDir + File.separator + outputFilename);
-  }
-
-  @Test
-  public void testBloomFilter2() throws Exception {
-    ObjectInspector inspector;
-    synchronized (TestOrcFile.class) {
-      inspector = ObjectInspectorFactory.getReflectionObjectInspector
-          (MyRecord.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    }
-    conf.set(HiveConf.ConfVars.HIVE_ORC_ENCODING_STRATEGY.varname, "COMPRESSION");
-    OrcFile.WriterOptions options = OrcFile.writerOptions(conf)
-        .fileSystem(fs)
-        .inspector(inspector)
-        .stripeSize(100000)
-        .compress(CompressionKind.ZLIB)
-        .bufferSize(10000)
-        .rowIndexStride(1000)
-        .bloomFilterColumns("l")
-        .bloomFilterFpp(0.01)
-        .batchSize(1000);
-    Writer writer = OrcFile.createWriter(testFilePath, options);
-    Random r1 = new Random(1);
-    String[] words = new String[]{"It", "was", "the", "best", "of", "times,",
-        "it", "was", "the", "worst", "of", "times,", "it", "was", "the", "age",
-        "of", "wisdom,", "it", "was", "the", "age", "of", "foolishness,", "it",
-        "was", "the", "epoch", "of", "belief,", "it", "was", "the", "epoch",
-        "of", "incredulity,", "it", "was", "the", "season", "of", "Light,",
-        "it", "was", "the", "season", "of", "Darkness,", "it", "was", "the",
-        "spring", "of", "hope,", "it", "was", "the", "winter", "of", "despair,",
-        "we", "had", "everything", "before", "us,", "we", "had", "nothing",
-        "before", "us,", "we", "were", "all", "going", "direct", "to",
-        "Heaven,", "we", "were", "all", "going", "direct", "the", "other",
-        "way"};
-    for(int i=0; i < 21000; ++i) {
-      writer.addRow(new MyRecord(r1.nextInt(), r1.nextLong(),
-          words[r1.nextInt(words.length)]));
-    }
-    writer.close();
-    PrintStream origOut = System.out;
-    String outputFilename = "orc-file-dump-bloomfilter2.out";
-    FileOutputStream myOut = new FileOutputStream(workDir + File.separator + outputFilename);
-
-    // replace stdout and run command
-    System.setOut(new PrintStream(myOut));
-    FileDump.main(new String[]{testFilePath.toString(), "--rowindex=2"});
-    System.out.flush();
-    System.setOut(origOut);
-
-
-    checkOutput(outputFilename, workDir + File.separator + outputFilename);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestJsonFileDump.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestJsonFileDump.java b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestJsonFileDump.java
deleted file mode 100644
index acf232d..0000000
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestJsonFileDump.java
+++ /dev/null
@@ -1,139 +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.hive.ql.io.orc;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
-
-import java.io.BufferedReader;
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.FileReader;
-import java.io.PrintStream;
-import java.util.Random;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
-import org.apache.hive.common.util.HiveTestUtils;
-import org.apache.orc.CompressionKind;
-import org.junit.Before;
-import org.junit.Test;
-
-public class TestJsonFileDump {
-
-  Path workDir = new Path(System.getProperty("test.tmp.dir"));
-  Configuration conf;
-  FileSystem fs;
-  Path testFilePath;
-
-  @Before
-  public void openFileSystem () throws Exception {
-    conf = new Configuration();
-    fs = FileSystem.getLocal(conf);
-    fs.setWorkingDirectory(workDir);
-    testFilePath = new Path("TestFileDump.testDump.orc");
-    fs.delete(testFilePath, false);
-  }
-
-  static class MyRecord {
-    int i;
-    long l;
-    String s;
-    MyRecord(int i, long l, String s) {
-      this.i = i;
-      this.l = l;
-      this.s = s;
-    }
-  }
-
-  static void checkOutput(String expected,
-                                  String actual) throws Exception {
-    BufferedReader eStream =
-        new BufferedReader(new FileReader(HiveTestUtils.getFileFromClasspath(expected)));
-    BufferedReader aStream =
-        new BufferedReader(new FileReader(actual));
-    String expectedLine = eStream.readLine();
-    while (expectedLine != null) {
-      String actualLine = aStream.readLine();
-      System.out.println("actual:   " + actualLine);
-      System.out.println("expected: " + expectedLine);
-      assertEquals(expectedLine, actualLine);
-      expectedLine = eStream.readLine();
-    }
-    assertNull(eStream.readLine());
-    assertNull(aStream.readLine());
-  }
-
-  @Test
-  public void testJsonDump() throws Exception {
-    ObjectInspector inspector;
-    synchronized (TestOrcFile.class) {
-      inspector = ObjectInspectorFactory.getReflectionObjectInspector
-          (MyRecord.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    }
-    conf.set(HiveConf.ConfVars.HIVE_ORC_ENCODING_STRATEGY.varname, "COMPRESSION");
-    OrcFile.WriterOptions options = OrcFile.writerOptions(conf)
-        .fileSystem(fs)
-        .inspector(inspector)
-        .stripeSize(100000)
-        .compress(CompressionKind.ZLIB)
-        .bufferSize(10000)
-        .rowIndexStride(1000)
-        .bloomFilterColumns("s");
-    Writer writer = OrcFile.createWriter(testFilePath, options);
-    Random r1 = new Random(1);
-    String[] words = new String[]{"It", "was", "the", "best", "of", "times,",
-        "it", "was", "the", "worst", "of", "times,", "it", "was", "the", "age",
-        "of", "wisdom,", "it", "was", "the", "age", "of", "foolishness,", "it",
-        "was", "the", "epoch", "of", "belief,", "it", "was", "the", "epoch",
-        "of", "incredulity,", "it", "was", "the", "season", "of", "Light,",
-        "it", "was", "the", "season", "of", "Darkness,", "it", "was", "the",
-        "spring", "of", "hope,", "it", "was", "the", "winter", "of", "despair,",
-        "we", "had", "everything", "before", "us,", "we", "had", "nothing",
-        "before", "us,", "we", "were", "all", "going", "direct", "to",
-        "Heaven,", "we", "were", "all", "going", "direct", "the", "other",
-        "way"};
-    for(int i=0; i < 21000; ++i) {
-      if (i % 100 == 0) {
-        writer.addRow(new MyRecord(r1.nextInt(), r1.nextLong(), null));
-      } else {
-        writer.addRow(new MyRecord(r1.nextInt(), r1.nextLong(),
-            words[r1.nextInt(words.length)]));
-      }
-    }
-
-    writer.close();
-    PrintStream origOut = System.out;
-    String outputFilename = "orc-file-dump.json";
-    FileOutputStream myOut = new FileOutputStream(workDir + File.separator + outputFilename);
-
-    // replace stdout and run command
-    System.setOut(new PrintStream(myOut));
-    FileDump.main(new String[]{testFilePath.toString(), "-j", "-p", "--rowindex=3"});
-    System.out.flush();
-    System.setOut(origOut);
-
-
-    checkOutput(outputFilename, workDir + File.separator + outputFilename);
-  }
-}


[20/27] hive git commit: HIVE-11417. Move the ReaderImpl and RowReaderImpl to the ORC module, by making shims for the row by row reader. (omalley reviewed by prasanth_j)

Posted by om...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/orc/src/test/org/apache/orc/TestStringDictionary.java
----------------------------------------------------------------------
diff --git a/orc/src/test/org/apache/orc/TestStringDictionary.java b/orc/src/test/org/apache/orc/TestStringDictionary.java
new file mode 100644
index 0000000..46209bb
--- /dev/null
+++ b/orc/src/test/org/apache/orc/TestStringDictionary.java
@@ -0,0 +1,290 @@
+/**
+ * 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.orc;
+
+import static org.junit.Assert.assertEquals;
+
+import java.io.File;
+import java.util.Random;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+
+import org.apache.orc.impl.RecordReaderImpl;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+
+public class TestStringDictionary {
+
+  Path workDir = new Path(System.getProperty("test.tmp.dir", "target" + File.separator + "test"
+      + File.separator + "tmp"));
+
+  Configuration conf;
+  FileSystem fs;
+  Path testFilePath;
+
+  @Rule
+  public TestName testCaseName = new TestName();
+
+  @Before
+  public void openFileSystem() throws Exception {
+    conf = new Configuration();
+    fs = FileSystem.getLocal(conf);
+    testFilePath = new Path(workDir, "TestOrcFile." + testCaseName.getMethodName() + ".orc");
+    fs.delete(testFilePath, false);
+  }
+
+  @Test
+  public void testTooManyDistinct() throws Exception {
+    TypeDescription schema = TypeDescription.createString();
+
+    Writer writer = OrcFile.createWriter(
+        testFilePath,
+        OrcFile.writerOptions(conf).setSchema(schema)
+                                   .compress(CompressionKind.NONE)
+                                   .bufferSize(10000));
+    VectorizedRowBatch batch = schema.createRowBatch();
+    BytesColumnVector col = (BytesColumnVector) batch.cols[0];
+    for (int i = 0; i < 20000; i++) {
+      if (batch.size == batch.getMaxSize()) {
+        writer.addRowBatch(batch);
+        batch.reset();
+      }
+      col.setVal(batch.size++, String.valueOf(i).getBytes());
+    }
+    writer.addRowBatch(batch);
+    writer.close();
+
+    Reader reader = OrcFile.createReader(testFilePath, OrcFile.readerOptions(conf).filesystem(fs));
+    RecordReader rows = reader.rows();
+    batch = reader.getSchema().createRowBatch();
+    col = (BytesColumnVector) batch.cols[0];
+    int idx = 0;
+    while (rows.nextBatch(batch)) {
+      for(int r=0; r < batch.size; ++r) {
+        assertEquals(String.valueOf(idx++), col.toString(r));
+      }
+    }
+
+    // make sure the encoding type is correct
+    for (StripeInformation stripe : reader.getStripes()) {
+      // hacky but does the job, this casting will work as long this test resides
+      // within the same package as ORC reader
+      OrcProto.StripeFooter footer = ((RecordReaderImpl) rows).readStripeFooter(stripe);
+      for (int i = 0; i < footer.getColumnsCount(); ++i) {
+        OrcProto.ColumnEncoding encoding = footer.getColumns(i);
+        assertEquals(OrcProto.ColumnEncoding.Kind.DIRECT_V2, encoding.getKind());
+      }
+    }
+  }
+
+  @Test
+  public void testHalfDistinct() throws Exception {
+    TypeDescription schema = TypeDescription.createString();
+
+    Writer writer = OrcFile.createWriter(
+        testFilePath,
+        OrcFile.writerOptions(conf).setSchema(schema).compress(CompressionKind.NONE)
+            .bufferSize(10000));
+    Random rand = new Random(123);
+    int[] input = new int[20000];
+    for (int i = 0; i < 20000; i++) {
+      input[i] = rand.nextInt(10000);
+    }
+
+    VectorizedRowBatch batch = schema.createRowBatch();
+    BytesColumnVector col = (BytesColumnVector) batch.cols[0];
+    for (int i = 0; i < 20000; i++) {
+      if (batch.size == batch.getMaxSize()) {
+        writer.addRowBatch(batch);
+        batch.reset();
+      }
+      col.setVal(batch.size++, String.valueOf(input[i]).getBytes());
+    }
+    writer.addRowBatch(batch);
+    writer.close();
+
+    Reader reader = OrcFile.createReader(testFilePath, OrcFile.readerOptions(conf).filesystem(fs));
+    RecordReader rows = reader.rows();
+    batch = reader.getSchema().createRowBatch();
+    col = (BytesColumnVector) batch.cols[0];
+    int idx = 0;
+    while (rows.nextBatch(batch)) {
+      for(int r=0; r < batch.size; ++r) {
+        assertEquals(String.valueOf(input[idx++]), col.toString(r));
+      }
+    }
+
+    // make sure the encoding type is correct
+    for (StripeInformation stripe : reader.getStripes()) {
+      // hacky but does the job, this casting will work as long this test resides
+      // within the same package as ORC reader
+      OrcProto.StripeFooter footer = ((RecordReaderImpl) rows).readStripeFooter(stripe);
+      for (int i = 0; i < footer.getColumnsCount(); ++i) {
+        OrcProto.ColumnEncoding encoding = footer.getColumns(i);
+        assertEquals(OrcProto.ColumnEncoding.Kind.DICTIONARY_V2, encoding.getKind());
+      }
+    }
+  }
+
+  @Test
+  public void testTooManyDistinctCheckDisabled() throws Exception {
+    TypeDescription schema = TypeDescription.createString();
+
+    conf.setBoolean(OrcConf.ROW_INDEX_STRIDE_DICTIONARY_CHECK.getAttribute(), false);
+    Writer writer = OrcFile.createWriter(
+        testFilePath,
+        OrcFile.writerOptions(conf).setSchema(schema).compress(CompressionKind.NONE)
+            .bufferSize(10000));
+    VectorizedRowBatch batch = schema.createRowBatch();
+    BytesColumnVector string = (BytesColumnVector) batch.cols[0];
+    for (int i = 0; i < 20000; i++) {
+      if (batch.size == batch.getMaxSize()) {
+        writer.addRowBatch(batch);
+        batch.reset();
+      }
+      string.setVal(batch.size++, String.valueOf(i).getBytes());
+    }
+    writer.addRowBatch(batch);
+    writer.close();
+
+    Reader reader = OrcFile.createReader(testFilePath, OrcFile.readerOptions(conf).filesystem(fs));
+    RecordReader rows = reader.rows();
+    batch = reader.getSchema().createRowBatch();
+    string = (BytesColumnVector) batch.cols[0];
+    int idx = 0;
+    while (rows.nextBatch(batch)) {
+      for(int r=0; r < batch.size; ++r) {
+        assertEquals(String.valueOf(idx++), string.toString(r));
+      }
+    }
+
+    // make sure the encoding type is correct
+    for (StripeInformation stripe : reader.getStripes()) {
+      // hacky but does the job, this casting will work as long this test resides
+      // within the same package as ORC reader
+      OrcProto.StripeFooter footer = ((RecordReaderImpl) rows).readStripeFooter(stripe);
+      for (int i = 0; i < footer.getColumnsCount(); ++i) {
+        OrcProto.ColumnEncoding encoding = footer.getColumns(i);
+        assertEquals(OrcProto.ColumnEncoding.Kind.DIRECT_V2, encoding.getKind());
+      }
+    }
+  }
+
+  @Test
+  public void testHalfDistinctCheckDisabled() throws Exception {
+    TypeDescription schema = TypeDescription.createString();
+
+    conf.setBoolean(OrcConf.ROW_INDEX_STRIDE_DICTIONARY_CHECK.getAttribute(),
+        false);
+    Writer writer = OrcFile.createWriter(
+        testFilePath,
+        OrcFile.writerOptions(conf).setSchema(schema)
+            .compress(CompressionKind.NONE)
+            .bufferSize(10000));
+    Random rand = new Random(123);
+    int[] input = new int[20000];
+    for (int i = 0; i < 20000; i++) {
+      input[i] = rand.nextInt(10000);
+    }
+    VectorizedRowBatch batch = schema.createRowBatch();
+    BytesColumnVector string = (BytesColumnVector) batch.cols[0];
+    for (int i = 0; i < 20000; i++) {
+      if (batch.size == batch.getMaxSize()) {
+        writer.addRowBatch(batch);
+        batch.reset();
+      }
+      string.setVal(batch.size++, String.valueOf(input[i]).getBytes());
+    }
+    writer.addRowBatch(batch);
+    writer.close();
+
+    Reader reader = OrcFile.createReader(testFilePath, OrcFile.readerOptions(conf).filesystem(fs));
+    RecordReader rows = reader.rows();
+    batch = reader.getSchema().createRowBatch();
+    string = (BytesColumnVector) batch.cols[0];
+    int idx = 0;
+    while (rows.nextBatch(batch)) {
+      for(int r=0; r < batch.size; ++r) {
+        assertEquals(String.valueOf(input[idx++]), string.toString(r));
+      }
+    }
+
+    // make sure the encoding type is correct
+    for (StripeInformation stripe : reader.getStripes()) {
+      // hacky but does the job, this casting will work as long this test resides
+      // within the same package as ORC reader
+      OrcProto.StripeFooter footer = ((RecordReaderImpl) rows).readStripeFooter(stripe);
+      for (int i = 0; i < footer.getColumnsCount(); ++i) {
+        OrcProto.ColumnEncoding encoding = footer.getColumns(i);
+        assertEquals(OrcProto.ColumnEncoding.Kind.DICTIONARY_V2, encoding.getKind());
+      }
+    }
+  }
+
+  @Test
+  public void testTooManyDistinctV11AlwaysDictionary() throws Exception {
+    TypeDescription schema = TypeDescription.createString();
+
+    Writer writer = OrcFile.createWriter(
+        testFilePath,
+        OrcFile.writerOptions(conf).setSchema(schema)
+            .compress(CompressionKind.NONE)
+            .version(OrcFile.Version.V_0_11).bufferSize(10000));
+    VectorizedRowBatch batch = schema.createRowBatch();
+    BytesColumnVector string = (BytesColumnVector) batch.cols[0];
+    for (int i = 0; i < 20000; i++) {
+      if (batch.size == batch.getMaxSize()) {
+        writer.addRowBatch(batch);
+        batch.reset();
+      }
+      string.setVal(batch.size++, String.valueOf(i).getBytes());
+    }
+    writer.addRowBatch(batch);
+    writer.close();
+
+    Reader reader = OrcFile.createReader(testFilePath, OrcFile.readerOptions(conf).filesystem(fs));
+    batch = reader.getSchema().createRowBatch();
+    string = (BytesColumnVector) batch.cols[0];
+    RecordReader rows = reader.rows();
+    int idx = 0;
+    while (rows.nextBatch(batch)) {
+      for(int r=0; r < batch.size; ++r) {
+        assertEquals(String.valueOf(idx++), string.toString(r));
+      }
+    }
+
+    // make sure the encoding type is correct
+    for (StripeInformation stripe : reader.getStripes()) {
+      // hacky but does the job, this casting will work as long this test resides
+      // within the same package as ORC reader
+      OrcProto.StripeFooter footer = ((RecordReaderImpl) rows).readStripeFooter(stripe);
+      for (int i = 0; i < footer.getColumnsCount(); ++i) {
+        OrcProto.ColumnEncoding encoding = footer.getColumns(i);
+        assertEquals(OrcProto.ColumnEncoding.Kind.DICTIONARY, encoding.getKind());
+      }
+    }
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/orc/src/test/org/apache/orc/TestTypeDescription.java
----------------------------------------------------------------------
diff --git a/orc/src/test/org/apache/orc/TestTypeDescription.java b/orc/src/test/org/apache/orc/TestTypeDescription.java
new file mode 100644
index 0000000..0ac1e64
--- /dev/null
+++ b/orc/src/test/org/apache/orc/TestTypeDescription.java
@@ -0,0 +1,68 @@
+/**
+ * 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.orc;
+
+import static org.junit.Assert.assertEquals;
+
+import org.apache.orc.TypeDescription;
+import org.junit.Test;
+
+public class TestTypeDescription {
+
+  @Test
+  public void testJson() {
+    TypeDescription bin = TypeDescription.createBinary();
+    assertEquals("{\"category\": \"binary\", \"id\": 0, \"max\": 0}",
+        bin.toJson());
+    assertEquals("binary", bin.toString());
+    TypeDescription struct = TypeDescription.createStruct()
+        .addField("f1", TypeDescription.createInt())
+        .addField("f2", TypeDescription.createString())
+        .addField("f3", TypeDescription.createDecimal());
+    assertEquals("struct<f1:int,f2:string,f3:decimal(38,10)>",
+        struct.toString());
+    assertEquals("{\"category\": \"struct\", \"id\": 0, \"max\": 3, \"fields\": [\n"
+            + "  \"f1\": {\"category\": \"int\", \"id\": 1, \"max\": 1},\n"
+            + "  \"f2\": {\"category\": \"string\", \"id\": 2, \"max\": 2},\n"
+            + "  \"f3\": {\"category\": \"decimal\", \"id\": 3, \"max\": 3, \"precision\": 38, \"scale\": 10}]}",
+        struct.toJson());
+    struct = TypeDescription.createStruct()
+        .addField("f1", TypeDescription.createUnion()
+            .addUnionChild(TypeDescription.createByte())
+            .addUnionChild(TypeDescription.createDecimal()
+                .withPrecision(20).withScale(10)))
+        .addField("f2", TypeDescription.createStruct()
+            .addField("f3", TypeDescription.createDate())
+            .addField("f4", TypeDescription.createDouble())
+            .addField("f5", TypeDescription.createBoolean()))
+        .addField("f6", TypeDescription.createChar().withMaxLength(100));
+    assertEquals("struct<f1:uniontype<tinyint,decimal(20,10)>,f2:struct<f3:date,f4:double,f5:boolean>,f6:char(100)>",
+        struct.toString());
+    assertEquals(
+        "{\"category\": \"struct\", \"id\": 0, \"max\": 8, \"fields\": [\n" +
+            "  \"f1\": {\"category\": \"uniontype\", \"id\": 1, \"max\": 3, \"children\": [\n" +
+            "    {\"category\": \"tinyint\", \"id\": 2, \"max\": 2},\n" +
+            "    {\"category\": \"decimal\", \"id\": 3, \"max\": 3, \"precision\": 20, \"scale\": 10}]},\n" +
+            "  \"f2\": {\"category\": \"struct\", \"id\": 4, \"max\": 7, \"fields\": [\n" +
+            "    \"f3\": {\"category\": \"date\", \"id\": 5, \"max\": 5},\n" +
+            "    \"f4\": {\"category\": \"double\", \"id\": 6, \"max\": 6},\n" +
+            "    \"f5\": {\"category\": \"boolean\", \"id\": 7, \"max\": 7}]},\n" +
+            "  \"f6\": {\"category\": \"char\", \"id\": 8, \"max\": 8, \"length\": 100}]}",
+        struct.toJson());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/orc/src/test/org/apache/orc/TestUnrolledBitPack.java
----------------------------------------------------------------------
diff --git a/orc/src/test/org/apache/orc/TestUnrolledBitPack.java b/orc/src/test/org/apache/orc/TestUnrolledBitPack.java
new file mode 100644
index 0000000..ef8fcd0
--- /dev/null
+++ b/orc/src/test/org/apache/orc/TestUnrolledBitPack.java
@@ -0,0 +1,114 @@
+/**
+ * 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.orc;
+
+import static org.junit.Assert.assertEquals;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+import com.google.common.collect.Lists;
+import com.google.common.primitives.Longs;
+
+@RunWith(value = Parameterized.class)
+public class TestUnrolledBitPack {
+
+  private long val;
+
+  public TestUnrolledBitPack(long val) {
+    this.val = val;
+  }
+
+  @Parameters
+  public static Collection<Object[]> data() {
+    Object[][] data = new Object[][] { { -1 }, { 1 }, { 7 }, { -128 }, { 32000 }, { 8300000 },
+        { Integer.MAX_VALUE }, { 540000000000L }, { 140000000000000L }, { 36000000000000000L },
+        { Long.MAX_VALUE } };
+    return Arrays.asList(data);
+  }
+
+  Path workDir = new Path(System.getProperty("test.tmp.dir", "target" + File.separator + "test"
+      + File.separator + "tmp"));
+
+  Configuration conf;
+  FileSystem fs;
+  Path testFilePath;
+
+  @Rule
+  public TestName testCaseName = new TestName();
+
+  @Before
+  public void openFileSystem() throws Exception {
+    conf = new Configuration();
+    fs = FileSystem.getLocal(conf);
+    testFilePath = new Path(workDir, "TestOrcFile." + testCaseName.getMethodName() + ".orc");
+    fs.delete(testFilePath, false);
+  }
+
+  @Test
+  public void testBitPacking() throws Exception {
+    TypeDescription schema = TypeDescription.createLong();
+
+    long[] inp = new long[] { val, 0, val, val, 0, val, 0, val, val, 0, val, 0, val, val, 0, 0,
+        val, val, 0, val, 0, 0, val, 0, val, 0, val, 0, 0, val, 0, val, 0, val, 0, 0, val, 0, val,
+        0, val, 0, 0, val, 0, val, 0, val, 0, 0, val, 0, val, 0, val, 0, 0, val, 0, val, 0, val, 0,
+        0, val, 0, val, 0, val, 0, 0, val, 0, val, 0, val, 0, 0, val, 0, val, 0, val, 0, 0, val, 0,
+        val, 0, val, 0, 0, val, 0, val, 0, 0, val, val };
+    List<Long> input = Lists.newArrayList(Longs.asList(inp));
+
+    Writer writer = OrcFile.createWriter(
+        testFilePath,
+        OrcFile.writerOptions(conf).setSchema(schema).stripeSize(100000)
+            .compress(CompressionKind.NONE).bufferSize(10000));
+    VectorizedRowBatch batch = schema.createRowBatch();
+    for (Long l : input) {
+      int row = batch.size++;
+      ((LongColumnVector) batch.cols[0]).vector[row] = l;
+    }
+    writer.addRowBatch(batch);
+    writer.close();
+
+    Reader reader = OrcFile.createReader(testFilePath, OrcFile.readerOptions(conf).filesystem(fs));
+    RecordReader rows = reader.rows();
+    batch = reader.getSchema().createRowBatch();
+    int idx = 0;
+    while (rows.nextBatch(batch)) {
+      for(int r=0; r < batch.size; ++r) {
+        assertEquals(input.get(idx++).longValue(),
+            ((LongColumnVector) batch.cols[0]).vector[r]);
+      }
+    }
+  }
+
+}


[17/27] hive git commit: HIVE-11417. Move the ReaderImpl and RowReaderImpl to the ORC module, by making shims for the row by row reader. (omalley reviewed by prasanth_j)

Posted by om...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/orc/src/test/org/apache/orc/impl/TestRecordReaderImpl.java
----------------------------------------------------------------------
diff --git a/orc/src/test/org/apache/orc/impl/TestRecordReaderImpl.java b/orc/src/test/org/apache/orc/impl/TestRecordReaderImpl.java
new file mode 100644
index 0000000..cdd62ac
--- /dev/null
+++ b/orc/src/test/org/apache/orc/impl/TestRecordReaderImpl.java
@@ -0,0 +1,1691 @@
+/**
+ * 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.orc.impl;
+
+import static junit.framework.Assert.assertEquals;
+import static org.hamcrest.core.Is.is;
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.atLeastOnce;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.sql.Timestamp;
+import java.util.ArrayList;
+import java.util.List;
+
+import junit.framework.Assert;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PositionedReadable;
+import org.apache.hadoop.fs.Seekable;
+import org.apache.hadoop.hive.common.io.DiskRangeList;
+import org.apache.hadoop.hive.common.type.HiveDecimal;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgumentImpl;
+import org.apache.orc.BloomFilterIO;
+import org.apache.orc.DataReader;
+import org.apache.orc.RecordReader;
+import org.apache.orc.TypeDescription;
+import org.apache.orc.Writer;
+import org.apache.orc.impl.RecordReaderImpl.Location;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument.TruthValue;
+import org.apache.hadoop.hive.serde2.io.DateWritable;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.orc.ColumnStatistics;
+import org.apache.orc.OrcFile;
+import org.apache.orc.Reader;
+import org.apache.orc.OrcProto;
+
+import org.junit.Test;
+import org.mockito.MockSettings;
+import org.mockito.Mockito;
+
+public class TestRecordReaderImpl {
+  /**
+   * Create a predicate leaf. This is used by another test.
+   */
+  public static PredicateLeaf createPredicateLeaf(PredicateLeaf.Operator operator,
+                                                  PredicateLeaf.Type type,
+                                                  String columnName,
+                                                  Object literal,
+                                                  List<Object> literalList) {
+    return new SearchArgumentImpl.PredicateLeafImpl(operator, type, columnName,
+        literal, literalList);
+  }
+
+  // can add .verboseLogging() to cause Mockito to log invocations
+  private final MockSettings settings = Mockito.withSettings().verboseLogging();
+
+  static class BufferInStream
+      extends InputStream implements PositionedReadable, Seekable {
+    private final byte[] buffer;
+    private final int length;
+    private int position = 0;
+
+    BufferInStream(byte[] bytes, int length) {
+      this.buffer = bytes;
+      this.length = length;
+    }
+
+    @Override
+    public int read() {
+      if (position < length) {
+        return buffer[position++];
+      }
+      return -1;
+    }
+
+    @Override
+    public int read(byte[] bytes, int offset, int length) {
+      int lengthToRead = Math.min(length, this.length - this.position);
+      if (lengthToRead >= 0) {
+        for(int i=0; i < lengthToRead; ++i) {
+          bytes[offset + i] = buffer[position++];
+        }
+        return lengthToRead;
+      } else {
+        return -1;
+      }
+    }
+
+    @Override
+    public int read(long position, byte[] bytes, int offset, int length) {
+      this.position = (int) position;
+      return read(bytes, offset, length);
+    }
+
+    @Override
+    public void readFully(long position, byte[] bytes, int offset,
+                          int length) throws IOException {
+      this.position = (int) position;
+      while (length > 0) {
+        int result = read(bytes, offset, length);
+        offset += result;
+        length -= result;
+        if (result < 0) {
+          throw new IOException("Read past end of buffer at " + offset);
+        }
+      }
+    }
+
+    @Override
+    public void readFully(long position, byte[] bytes) throws IOException {
+      readFully(position, bytes, 0, bytes.length);
+    }
+
+    @Override
+    public void seek(long position) {
+      this.position = (int) position;
+    }
+
+    @Override
+    public long getPos() {
+      return position;
+    }
+
+    @Override
+    public boolean seekToNewSource(long position) throws IOException {
+      this.position = (int) position;
+      return false;
+    }
+  }
+
+  @Test
+  public void testMaxLengthToReader() throws Exception {
+    Configuration conf = new Configuration();
+    OrcProto.Type rowType = OrcProto.Type.newBuilder()
+        .setKind(OrcProto.Type.Kind.STRUCT).build();
+    OrcProto.Footer footer = OrcProto.Footer.newBuilder()
+        .setHeaderLength(0).setContentLength(0).setNumberOfRows(0)
+        .setRowIndexStride(0).addTypes(rowType).build();
+    OrcProto.PostScript ps = OrcProto.PostScript.newBuilder()
+        .setCompression(OrcProto.CompressionKind.NONE)
+        .setFooterLength(footer.getSerializedSize())
+        .setMagic("ORC").addVersion(0).addVersion(11).build();
+    DataOutputBuffer buffer = new DataOutputBuffer();
+    footer.writeTo(buffer);
+    ps.writeTo(buffer);
+    buffer.write(ps.getSerializedSize());
+    FileSystem fs = mock(FileSystem.class, settings);
+    FSDataInputStream file =
+        new FSDataInputStream(new BufferInStream(buffer.getData(),
+            buffer.getLength()));
+    Path p = new Path("/dir/file.orc");
+    when(fs.open(p)).thenReturn(file);
+    OrcFile.ReaderOptions options = OrcFile.readerOptions(conf);
+    options.filesystem(fs);
+    options.maxLength(buffer.getLength());
+    when(fs.getFileStatus(p))
+        .thenReturn(new FileStatus(10, false, 3, 3000, 0, p));
+    Reader reader = OrcFile.createReader(p, options);
+  }
+
+  @Test
+  public void testCompareToRangeInt() throws Exception {
+    assertEquals(Location.BEFORE,
+      RecordReaderImpl.compareToRange(19L, 20L, 40L));
+    assertEquals(Location.AFTER,
+      RecordReaderImpl.compareToRange(41L, 20L, 40L));
+    assertEquals(Location.MIN,
+        RecordReaderImpl.compareToRange(20L, 20L, 40L));
+    assertEquals(Location.MIDDLE,
+        RecordReaderImpl.compareToRange(21L, 20L, 40L));
+    assertEquals(Location.MAX,
+      RecordReaderImpl.compareToRange(40L, 20L, 40L));
+    assertEquals(Location.BEFORE,
+      RecordReaderImpl.compareToRange(0L, 1L, 1L));
+    assertEquals(Location.MIN,
+      RecordReaderImpl.compareToRange(1L, 1L, 1L));
+    assertEquals(Location.AFTER,
+      RecordReaderImpl.compareToRange(2L, 1L, 1L));
+  }
+
+  @Test
+  public void testCompareToRangeString() throws Exception {
+    assertEquals(Location.BEFORE,
+        RecordReaderImpl.compareToRange("a", "b", "c"));
+    assertEquals(Location.AFTER,
+        RecordReaderImpl.compareToRange("d", "b", "c"));
+    assertEquals(Location.MIN,
+        RecordReaderImpl.compareToRange("b", "b", "c"));
+    assertEquals(Location.MIDDLE,
+        RecordReaderImpl.compareToRange("bb", "b", "c"));
+    assertEquals(Location.MAX,
+        RecordReaderImpl.compareToRange("c", "b", "c"));
+    assertEquals(Location.BEFORE,
+        RecordReaderImpl.compareToRange("a", "b", "b"));
+    assertEquals(Location.MIN,
+        RecordReaderImpl.compareToRange("b", "b", "b"));
+    assertEquals(Location.AFTER,
+        RecordReaderImpl.compareToRange("c", "b", "b"));
+  }
+
+  @Test
+  public void testCompareToCharNeedConvert() throws Exception {
+    assertEquals(Location.BEFORE,
+      RecordReaderImpl.compareToRange("apple", "hello", "world"));
+    assertEquals(Location.AFTER,
+      RecordReaderImpl.compareToRange("zombie", "hello", "world"));
+    assertEquals(Location.MIN,
+        RecordReaderImpl.compareToRange("hello", "hello", "world"));
+    assertEquals(Location.MIDDLE,
+        RecordReaderImpl.compareToRange("pilot", "hello", "world"));
+    assertEquals(Location.MAX,
+      RecordReaderImpl.compareToRange("world", "hello", "world"));
+    assertEquals(Location.BEFORE,
+      RecordReaderImpl.compareToRange("apple", "hello", "hello"));
+    assertEquals(Location.MIN,
+      RecordReaderImpl.compareToRange("hello", "hello", "hello"));
+    assertEquals(Location.AFTER,
+      RecordReaderImpl.compareToRange("zombie", "hello", "hello"));
+  }
+
+  @Test
+  public void testGetMin() throws Exception {
+    assertEquals(10L, RecordReaderImpl.getMin(
+      ColumnStatisticsImpl.deserialize(createIntStats(10L, 100L))));
+    assertEquals(10.0d, RecordReaderImpl.getMin(ColumnStatisticsImpl.deserialize(
+      OrcProto.ColumnStatistics.newBuilder()
+        .setDoubleStatistics(OrcProto.DoubleStatistics.newBuilder()
+          .setMinimum(10.0d).setMaximum(100.0d).build()).build())));
+    assertEquals(null, RecordReaderImpl.getMin(ColumnStatisticsImpl.deserialize(
+      OrcProto.ColumnStatistics.newBuilder()
+        .setStringStatistics(OrcProto.StringStatistics.newBuilder().build())
+        .build())));
+    assertEquals("a", RecordReaderImpl.getMin(ColumnStatisticsImpl.deserialize(
+      OrcProto.ColumnStatistics.newBuilder()
+        .setStringStatistics(OrcProto.StringStatistics.newBuilder()
+          .setMinimum("a").setMaximum("b").build()).build())));
+    assertEquals("hello", RecordReaderImpl.getMin(ColumnStatisticsImpl
+      .deserialize(createStringStats("hello", "world"))));
+    assertEquals(HiveDecimal.create("111.1"), RecordReaderImpl.getMin(ColumnStatisticsImpl
+      .deserialize(createDecimalStats("111.1", "112.1"))));
+  }
+
+  private static OrcProto.ColumnStatistics createIntStats(Long min,
+                                                          Long max) {
+    OrcProto.IntegerStatistics.Builder intStats =
+        OrcProto.IntegerStatistics.newBuilder();
+    if (min != null) {
+      intStats.setMinimum(min);
+    }
+    if (max != null) {
+      intStats.setMaximum(max);
+    }
+    return OrcProto.ColumnStatistics.newBuilder()
+        .setIntStatistics(intStats.build()).build();
+  }
+
+  private static OrcProto.ColumnStatistics createBooleanStats(int n, int trueCount) {
+    OrcProto.BucketStatistics.Builder boolStats = OrcProto.BucketStatistics.newBuilder();
+    boolStats.addCount(trueCount);
+    return OrcProto.ColumnStatistics.newBuilder().setNumberOfValues(n).setBucketStatistics(
+      boolStats.build()).build();
+  }
+
+  private static OrcProto.ColumnStatistics createIntStats(int min, int max) {
+    OrcProto.IntegerStatistics.Builder intStats = OrcProto.IntegerStatistics.newBuilder();
+    intStats.setMinimum(min);
+    intStats.setMaximum(max);
+    return OrcProto.ColumnStatistics.newBuilder().setIntStatistics(intStats.build()).build();
+  }
+
+  private static OrcProto.ColumnStatistics createDoubleStats(double min, double max) {
+    OrcProto.DoubleStatistics.Builder dblStats = OrcProto.DoubleStatistics.newBuilder();
+    dblStats.setMinimum(min);
+    dblStats.setMaximum(max);
+    return OrcProto.ColumnStatistics.newBuilder().setDoubleStatistics(dblStats.build()).build();
+  }
+
+  private static OrcProto.ColumnStatistics createStringStats(String min, String max,
+      boolean hasNull) {
+    OrcProto.StringStatistics.Builder strStats = OrcProto.StringStatistics.newBuilder();
+    strStats.setMinimum(min);
+    strStats.setMaximum(max);
+    return OrcProto.ColumnStatistics.newBuilder().setStringStatistics(strStats.build())
+        .setHasNull(hasNull).build();
+  }
+
+  private static OrcProto.ColumnStatistics createStringStats(String min, String max) {
+    OrcProto.StringStatistics.Builder strStats = OrcProto.StringStatistics.newBuilder();
+    strStats.setMinimum(min);
+    strStats.setMaximum(max);
+    return OrcProto.ColumnStatistics.newBuilder().setStringStatistics(strStats.build()).build();
+  }
+
+  private static OrcProto.ColumnStatistics createDateStats(int min, int max) {
+    OrcProto.DateStatistics.Builder dateStats = OrcProto.DateStatistics.newBuilder();
+    dateStats.setMinimum(min);
+    dateStats.setMaximum(max);
+    return OrcProto.ColumnStatistics.newBuilder().setDateStatistics(dateStats.build()).build();
+  }
+
+  private static OrcProto.ColumnStatistics createTimestampStats(long min, long max) {
+    OrcProto.TimestampStatistics.Builder tsStats = OrcProto.TimestampStatistics.newBuilder();
+    tsStats.setMinimum(min);
+    tsStats.setMaximum(max);
+    return OrcProto.ColumnStatistics.newBuilder().setTimestampStatistics(tsStats.build()).build();
+  }
+
+  private static OrcProto.ColumnStatistics createDecimalStats(String min, String max) {
+    OrcProto.DecimalStatistics.Builder decStats = OrcProto.DecimalStatistics.newBuilder();
+    decStats.setMinimum(min);
+    decStats.setMaximum(max);
+    return OrcProto.ColumnStatistics.newBuilder().setDecimalStatistics(decStats.build()).build();
+  }
+
+  private static OrcProto.ColumnStatistics createDecimalStats(String min, String max,
+      boolean hasNull) {
+    OrcProto.DecimalStatistics.Builder decStats = OrcProto.DecimalStatistics.newBuilder();
+    decStats.setMinimum(min);
+    decStats.setMaximum(max);
+    return OrcProto.ColumnStatistics.newBuilder().setDecimalStatistics(decStats.build())
+        .setHasNull(hasNull).build();
+  }
+
+  @Test
+  public void testGetMax() throws Exception {
+    assertEquals(100L, RecordReaderImpl.getMax(ColumnStatisticsImpl.deserialize(createIntStats(10L, 100L))));
+    assertEquals(100.0d, RecordReaderImpl.getMax(ColumnStatisticsImpl.deserialize(
+        OrcProto.ColumnStatistics.newBuilder()
+            .setDoubleStatistics(OrcProto.DoubleStatistics.newBuilder()
+                .setMinimum(10.0d).setMaximum(100.0d).build()).build())));
+    assertEquals(null, RecordReaderImpl.getMax(ColumnStatisticsImpl.deserialize(
+        OrcProto.ColumnStatistics.newBuilder()
+            .setStringStatistics(OrcProto.StringStatistics.newBuilder().build())
+            .build())));
+    assertEquals("b", RecordReaderImpl.getMax(ColumnStatisticsImpl.deserialize(
+        OrcProto.ColumnStatistics.newBuilder()
+            .setStringStatistics(OrcProto.StringStatistics.newBuilder()
+                .setMinimum("a").setMaximum("b").build()).build())));
+    assertEquals("world", RecordReaderImpl.getMax(ColumnStatisticsImpl
+      .deserialize(createStringStats("hello", "world"))));
+    assertEquals(HiveDecimal.create("112.1"), RecordReaderImpl.getMax(ColumnStatisticsImpl
+      .deserialize(createDecimalStats("111.1", "112.1"))));
+  }
+
+  @Test
+  public void testPredEvalWithBooleanStats() throws Exception {
+    PredicateLeaf pred = createPredicateLeaf(
+        PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.BOOLEAN, "x", true, null);
+    assertEquals(TruthValue.YES_NO,
+        RecordReaderImpl.evaluatePredicateProto(createBooleanStats(10, 10), pred, null));
+    assertEquals(TruthValue.NO,
+        RecordReaderImpl.evaluatePredicateProto(createBooleanStats(10, 0), pred, null));
+
+    pred = createPredicateLeaf(
+        PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.BOOLEAN, "x", true, null);
+    assertEquals(TruthValue.YES_NO,
+        RecordReaderImpl.evaluatePredicateProto(createBooleanStats(10, 10), pred, null));
+    assertEquals(TruthValue.NO,
+        RecordReaderImpl.evaluatePredicateProto(createBooleanStats(10, 0), pred, null));
+
+    pred = createPredicateLeaf(
+        PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.BOOLEAN, "x", false, null);
+    assertEquals(TruthValue.NO,
+      RecordReaderImpl.evaluatePredicateProto(createBooleanStats(10, 10), pred, null));
+    assertEquals(TruthValue.YES_NO,
+      RecordReaderImpl.evaluatePredicateProto(createBooleanStats(10, 0), pred, null));
+  }
+
+  @Test
+  public void testPredEvalWithIntStats() throws Exception {
+    PredicateLeaf pred = createPredicateLeaf(
+        PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.LONG, "x", 15L, null);
+    assertEquals(TruthValue.YES_NO,
+        RecordReaderImpl.evaluatePredicateProto(createIntStats(10, 100), pred, null));
+
+    pred = createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
+        PredicateLeaf.Type.FLOAT, "x", 15.0, null);
+    assertEquals(TruthValue.YES_NO,
+        RecordReaderImpl.evaluatePredicateProto(createIntStats(10, 100), pred, null));
+
+    // Stats gets converted to column type. "15" is outside of "10" and "100"
+    pred = createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
+        PredicateLeaf.Type.STRING, "x", "15", null);
+    assertEquals(TruthValue.NO,
+        RecordReaderImpl.evaluatePredicateProto(createIntStats(10, 100), pred, null));
+
+    // Integer stats will not be converted date because of days/seconds/millis ambiguity
+    pred = createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
+        PredicateLeaf.Type.DATE, "x", new DateWritable(15).get(), null);
+    assertEquals(TruthValue.YES_NO,
+        RecordReaderImpl.evaluatePredicateProto(createIntStats(10, 100), pred, null));
+
+    pred = createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
+        PredicateLeaf.Type.DECIMAL, "x", new HiveDecimalWritable("15"), null);
+    assertEquals(TruthValue.YES_NO,
+        RecordReaderImpl.evaluatePredicateProto(createIntStats(10, 100), pred, null));
+
+    pred = createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
+        PredicateLeaf.Type.TIMESTAMP, "x", new Timestamp(15), null);
+    assertEquals(TruthValue.YES_NO,
+      RecordReaderImpl.evaluatePredicateProto(createIntStats(10, 100), pred, null));
+  }
+
+  @Test
+  public void testPredEvalWithDoubleStats() throws Exception {
+    PredicateLeaf pred = createPredicateLeaf(
+        PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.LONG, "x", 15L, null);
+    assertEquals(TruthValue.YES_NO,
+        RecordReaderImpl.evaluatePredicateProto(createDoubleStats(10.0, 100.0), pred, null));
+
+    pred = createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
+        PredicateLeaf.Type.FLOAT, "x", 15.0, null);
+    assertEquals(TruthValue.YES_NO,
+        RecordReaderImpl.evaluatePredicateProto(createDoubleStats(10.0, 100.0), pred, null));
+
+    // Stats gets converted to column type. "15.0" is outside of "10.0" and "100.0"
+    pred = createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
+        PredicateLeaf.Type.STRING, "x", "15", null);
+    assertEquals(TruthValue.NO,
+        RecordReaderImpl.evaluatePredicateProto(createDoubleStats(10.0, 100.0), pred, null));
+
+    // Double is not converted to date type because of days/seconds/millis ambiguity
+    pred = createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
+        PredicateLeaf.Type.DATE, "x", new DateWritable(15).get(), null);
+    assertEquals(TruthValue.YES_NO,
+        RecordReaderImpl.evaluatePredicateProto(createDoubleStats(10.0, 100.0), pred, null));
+
+    pred = createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
+        PredicateLeaf.Type.DECIMAL, "x", new HiveDecimalWritable("15"), null);
+    assertEquals(TruthValue.YES_NO,
+        RecordReaderImpl.evaluatePredicateProto(createDoubleStats(10.0, 100.0), pred, null));
+
+    pred = createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
+        PredicateLeaf.Type.TIMESTAMP, "x", new Timestamp(15*1000L), null);
+    assertEquals(TruthValue.YES_NO,
+        RecordReaderImpl.evaluatePredicateProto(createDoubleStats(10.0, 100.0), pred, null));
+
+    pred = createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
+        PredicateLeaf.Type.TIMESTAMP, "x", new Timestamp(150*1000L), null);
+    assertEquals(TruthValue.NO,
+        RecordReaderImpl.evaluatePredicateProto(createDoubleStats(10.0, 100.0), pred, null));
+  }
+
+  @Test
+  public void testPredEvalWithStringStats() throws Exception {
+    PredicateLeaf pred = createPredicateLeaf(
+        PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.LONG, "x", 100L, null);
+    assertEquals(TruthValue.YES_NO,
+        RecordReaderImpl.evaluatePredicateProto(createStringStats("10", "1000"), pred, null));
+
+    pred = createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
+        PredicateLeaf.Type.FLOAT, "x", 100.0, null);
+    assertEquals(TruthValue.YES_NO,
+        RecordReaderImpl.evaluatePredicateProto(createStringStats("10", "1000"), pred, null));
+
+    pred = createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
+        PredicateLeaf.Type.STRING, "x", "100", null);
+    assertEquals(TruthValue.YES_NO,
+        RecordReaderImpl.evaluatePredicateProto(createStringStats("10", "1000"), pred, null));
+
+    // IllegalArgumentException is thrown when converting String to Date, hence YES_NO
+    pred = createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
+        PredicateLeaf.Type.DATE, "x", new DateWritable(100).get(), null);
+    assertEquals(TruthValue.YES_NO,
+        RecordReaderImpl.evaluatePredicateProto(createDateStats(10, 1000), pred, null));
+
+    pred = createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
+        PredicateLeaf.Type.DECIMAL, "x", new HiveDecimalWritable("100"), null);
+    assertEquals(TruthValue.YES_NO,
+        RecordReaderImpl.evaluatePredicateProto(createStringStats("10", "1000"), pred, null));
+
+    pred = createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
+        PredicateLeaf.Type.TIMESTAMP, "x", new Timestamp(100), null);
+    assertEquals(TruthValue.YES_NO,
+        RecordReaderImpl.evaluatePredicateProto(createStringStats("10", "1000"), pred, null));
+  }
+
+  @Test
+  public void testPredEvalWithDateStats() throws Exception {
+    PredicateLeaf pred = createPredicateLeaf(
+        PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.LONG, "x", 15L, null);
+    // Date to Integer conversion is not possible.
+    assertEquals(TruthValue.YES_NO,
+        RecordReaderImpl.evaluatePredicateProto(createDateStats(10, 100), pred, null));
+
+    // Date to Float conversion is also not possible.
+    pred = createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
+        PredicateLeaf.Type.FLOAT, "x", 15.0, null);
+    assertEquals(TruthValue.YES_NO,
+        RecordReaderImpl.evaluatePredicateProto(createDateStats(10, 100), pred, null));
+
+    pred = createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
+        PredicateLeaf.Type.STRING, "x", "15", null);
+    assertEquals(TruthValue.NO,
+        RecordReaderImpl.evaluatePredicateProto(createDateStats(10, 100), pred, null));
+
+    pred = createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
+        PredicateLeaf.Type.STRING, "x", "1970-01-11", null);
+    assertEquals(TruthValue.YES_NO,
+        RecordReaderImpl.evaluatePredicateProto(createDateStats(10, 100), pred, null));
+
+    pred = createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
+        PredicateLeaf.Type.STRING, "x", "15.1", null);
+    assertEquals(TruthValue.NO,
+        RecordReaderImpl.evaluatePredicateProto(createDateStats(10, 100), pred, null));
+
+    pred = createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
+        PredicateLeaf.Type.STRING, "x", "__a15__1", null);
+    assertEquals(TruthValue.NO,
+        RecordReaderImpl.evaluatePredicateProto(createDateStats(10, 100), pred, null));
+
+    pred = createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
+        PredicateLeaf.Type.STRING, "x", "2000-01-16", null);
+    assertEquals(TruthValue.NO,
+        RecordReaderImpl.evaluatePredicateProto(createDateStats(10, 100), pred, null));
+
+    pred = createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
+        PredicateLeaf.Type.STRING, "x", "1970-01-16", null);
+    assertEquals(TruthValue.YES_NO,
+        RecordReaderImpl.evaluatePredicateProto(createDateStats(10, 100), pred, null));
+
+    pred = createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
+        PredicateLeaf.Type.DATE, "x", new DateWritable(15).get(), null);
+    assertEquals(TruthValue.YES_NO,
+        RecordReaderImpl.evaluatePredicateProto(createDateStats(10, 100), pred, null));
+
+    pred = createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
+        PredicateLeaf.Type.DATE, "x", new DateWritable(150).get(), null);
+    assertEquals(TruthValue.NO,
+        RecordReaderImpl.evaluatePredicateProto(createDateStats(10, 100), pred, null));
+
+    // Date to Decimal conversion is also not possible.
+    pred = createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
+        PredicateLeaf.Type.DECIMAL, "x", new HiveDecimalWritable("15"), null);
+    assertEquals(TruthValue.YES_NO,
+        RecordReaderImpl.evaluatePredicateProto(createDateStats(10, 100), pred, null));
+
+    pred = createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
+        PredicateLeaf.Type.TIMESTAMP, "x", new Timestamp(15), null);
+    assertEquals(TruthValue.NO,
+        RecordReaderImpl.evaluatePredicateProto(createDateStats(10, 100), pred, null));
+
+    pred = createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
+        PredicateLeaf.Type.TIMESTAMP, "x", new Timestamp(15L * 24L * 60L * 60L * 1000L), null);
+    assertEquals(TruthValue.YES_NO,
+        RecordReaderImpl.evaluatePredicateProto(createDateStats(10, 100), pred, null));
+  }
+
+  @Test
+  public void testPredEvalWithDecimalStats() throws Exception {
+    PredicateLeaf pred = createPredicateLeaf(
+        PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.LONG, "x", 15L, null);
+    assertEquals(TruthValue.YES_NO,
+        RecordReaderImpl.evaluatePredicateProto(createDecimalStats("10.0", "100.0"), pred, null));
+
+    pred = createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
+        PredicateLeaf.Type.FLOAT, "x", 15.0, null);
+    assertEquals(TruthValue.YES_NO,
+        RecordReaderImpl.evaluatePredicateProto(createDecimalStats("10.0", "100.0"), pred, null));
+
+    // "15" out of range of "10.0" and "100.0"
+    pred = createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
+        PredicateLeaf.Type.STRING, "x", "15", null);
+    assertEquals(TruthValue.NO,
+        RecordReaderImpl.evaluatePredicateProto(createDecimalStats("10.0", "100.0"), pred, null));
+
+    // Decimal to Date not possible.
+    pred = createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
+        PredicateLeaf.Type.DATE, "x", new DateWritable(15).get(), null);
+    assertEquals(TruthValue.YES_NO,
+        RecordReaderImpl.evaluatePredicateProto(createDecimalStats("10.0", "100.0"), pred, null));
+
+    pred = createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
+        PredicateLeaf.Type.DECIMAL, "x", new HiveDecimalWritable("15"), null);
+    assertEquals(TruthValue.YES_NO,
+        RecordReaderImpl.evaluatePredicateProto(createDecimalStats("10.0", "100.0"), pred, null));
+
+    pred = createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
+        PredicateLeaf.Type.TIMESTAMP, "x", new Timestamp(15 * 1000L), null);
+    assertEquals(TruthValue.YES_NO,
+        RecordReaderImpl.evaluatePredicateProto(createDecimalStats("10.0", "100.0"), pred, null));
+
+    pred = createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
+        PredicateLeaf.Type.TIMESTAMP, "x", new Timestamp(150 * 1000L), null);
+    assertEquals(TruthValue.NO,
+        RecordReaderImpl.evaluatePredicateProto(createDecimalStats("10.0", "100.0"), pred, null));
+  }
+
+  @Test
+  public void testPredEvalWithTimestampStats() throws Exception {
+    PredicateLeaf pred = createPredicateLeaf(
+        PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.LONG, "x", 15L, null);
+    assertEquals(TruthValue.YES_NO,
+        RecordReaderImpl.evaluatePredicateProto(createTimestampStats(10, 100), pred, null));
+
+    pred = createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
+        PredicateLeaf.Type.FLOAT, "x", 15.0, null);
+    assertEquals(TruthValue.NO,
+        RecordReaderImpl.evaluatePredicateProto(createTimestampStats(10, 100), pred, null));
+    assertEquals(TruthValue.YES_NO,
+        RecordReaderImpl.evaluatePredicateProto(createTimestampStats(10000, 100000), pred, null));
+
+    pred = createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
+        PredicateLeaf.Type.STRING, "x", "15", null);
+    assertEquals(TruthValue.NO,
+        RecordReaderImpl.evaluatePredicateProto(createTimestampStats(10, 100), pred, null));
+
+    pred = createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
+        PredicateLeaf.Type.STRING, "x", new Timestamp(15).toString(), null);
+    assertEquals(TruthValue.YES_NO,
+        RecordReaderImpl.evaluatePredicateProto(createTimestampStats(10, 100), pred, null));
+
+    pred = createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
+        PredicateLeaf.Type.DATE, "x", new DateWritable(15).get(), null);
+    assertEquals(TruthValue.NO,
+        RecordReaderImpl.evaluatePredicateProto(createTimestampStats(10, 100), pred, null));
+    assertEquals(TruthValue.YES_NO,
+        RecordReaderImpl.evaluatePredicateProto(createTimestampStats(10 * 24L * 60L * 60L * 1000L,
+          100 * 24L * 60L * 60L * 1000L), pred, null));
+
+    pred = createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
+        PredicateLeaf.Type.DECIMAL, "x", new HiveDecimalWritable("15"), null);
+    assertEquals(TruthValue.NO,
+        RecordReaderImpl.evaluatePredicateProto(createTimestampStats(10, 100), pred, null));
+    assertEquals(TruthValue.YES_NO,
+        RecordReaderImpl.evaluatePredicateProto(createTimestampStats(10000, 100000), pred, null));
+
+    pred = createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
+        PredicateLeaf.Type.TIMESTAMP, "x", new Timestamp(15), null);
+    assertEquals(TruthValue.YES_NO,
+        RecordReaderImpl.evaluatePredicateProto(createTimestampStats(10, 100), pred, null));
+    assertEquals(TruthValue.NO,
+        RecordReaderImpl.evaluatePredicateProto(createTimestampStats(10000, 100000), pred, null));
+  }
+
+  @Test
+  public void testEquals() throws Exception {
+    PredicateLeaf pred = createPredicateLeaf
+        (PredicateLeaf.Operator.EQUALS, PredicateLeaf.Type.LONG,
+            "x", 15L, null);
+    assertEquals(TruthValue.NO_NULL,
+        RecordReaderImpl.evaluatePredicateProto(createIntStats(20L, 30L), pred, null));
+    assertEquals(TruthValue.YES_NO_NULL,
+        RecordReaderImpl.evaluatePredicateProto(createIntStats(15L, 30L), pred, null));
+    assertEquals(TruthValue.YES_NO_NULL,
+        RecordReaderImpl.evaluatePredicateProto(createIntStats(10L, 30L), pred, null));
+    assertEquals(TruthValue.YES_NO_NULL,
+        RecordReaderImpl.evaluatePredicateProto(createIntStats(10L, 15L), pred, null));
+    assertEquals(TruthValue.NO_NULL,
+        RecordReaderImpl.evaluatePredicateProto(createIntStats(0L, 10L), pred, null));
+    assertEquals(TruthValue.YES_NULL,
+        RecordReaderImpl.evaluatePredicateProto(createIntStats(15L, 15L), pred, null));
+  }
+
+  @Test
+  public void testNullSafeEquals() throws Exception {
+    PredicateLeaf pred = createPredicateLeaf
+        (PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.LONG,
+            "x", 15L, null);
+    assertEquals(TruthValue.NO,
+        RecordReaderImpl.evaluatePredicateProto(createIntStats(20L, 30L), pred, null));
+    assertEquals(TruthValue.YES_NO,
+        RecordReaderImpl.evaluatePredicateProto(createIntStats(15L, 30L), pred, null));
+    assertEquals(TruthValue.YES_NO,
+        RecordReaderImpl.evaluatePredicateProto(createIntStats(10L, 30L), pred, null));
+    assertEquals(TruthValue.YES_NO,
+        RecordReaderImpl.evaluatePredicateProto(createIntStats(10L, 15L), pred, null));
+    assertEquals(TruthValue.NO,
+        RecordReaderImpl.evaluatePredicateProto(createIntStats(0L, 10L), pred, null));
+    assertEquals(TruthValue.YES_NO,
+        RecordReaderImpl.evaluatePredicateProto(createIntStats(15L, 15L), pred, null));
+  }
+
+  @Test
+  public void testLessThan() throws Exception {
+    PredicateLeaf lessThan = createPredicateLeaf
+        (PredicateLeaf.Operator.LESS_THAN, PredicateLeaf.Type.LONG,
+            "x", 15L, null);
+    assertEquals(TruthValue.NO_NULL,
+        RecordReaderImpl.evaluatePredicateProto(createIntStats(20L, 30L), lessThan, null));
+    assertEquals(TruthValue.NO_NULL,
+        RecordReaderImpl.evaluatePredicateProto(createIntStats(15L, 30L), lessThan, null));
+    assertEquals(TruthValue.YES_NO_NULL,
+        RecordReaderImpl.evaluatePredicateProto(createIntStats(10L, 30L), lessThan, null));
+    assertEquals(TruthValue.YES_NO_NULL,
+        RecordReaderImpl.evaluatePredicateProto(createIntStats(10L, 15L), lessThan, null));
+    assertEquals(TruthValue.YES_NULL,
+        RecordReaderImpl.evaluatePredicateProto(createIntStats(0L, 10L), lessThan, null));
+  }
+
+  @Test
+  public void testLessThanEquals() throws Exception {
+    PredicateLeaf pred = createPredicateLeaf
+        (PredicateLeaf.Operator.LESS_THAN_EQUALS, PredicateLeaf.Type.LONG,
+            "x", 15L, null);
+    assertEquals(TruthValue.NO_NULL,
+        RecordReaderImpl.evaluatePredicateProto(createIntStats(20L, 30L), pred, null));
+    assertEquals(TruthValue.YES_NO_NULL,
+        RecordReaderImpl.evaluatePredicateProto(createIntStats(15L, 30L), pred, null));
+    assertEquals(TruthValue.YES_NO_NULL,
+        RecordReaderImpl.evaluatePredicateProto(createIntStats(10L, 30L), pred, null));
+    assertEquals(TruthValue.YES_NULL,
+        RecordReaderImpl.evaluatePredicateProto(createIntStats(10L, 15L), pred, null));
+    assertEquals(TruthValue.YES_NULL,
+        RecordReaderImpl.evaluatePredicateProto(createIntStats(0L, 10L), pred, null));
+  }
+
+  @Test
+  public void testIn() throws Exception {
+    List<Object> args = new ArrayList<Object>();
+    args.add(10L);
+    args.add(20L);
+    PredicateLeaf pred = createPredicateLeaf
+        (PredicateLeaf.Operator.IN, PredicateLeaf.Type.LONG,
+            "x", null, args);
+    assertEquals(TruthValue.YES_NULL,
+        RecordReaderImpl.evaluatePredicateProto(createIntStats(20L, 20L), pred, null));
+    assertEquals(TruthValue.NO_NULL,
+        RecordReaderImpl.evaluatePredicateProto(createIntStats(30L, 30L), pred, null));
+    assertEquals(TruthValue.YES_NO_NULL,
+        RecordReaderImpl.evaluatePredicateProto(createIntStats(10L, 30L), pred, null));
+    assertEquals(TruthValue.NO_NULL,
+        RecordReaderImpl.evaluatePredicateProto(createIntStats(12L, 18L), pred, null));
+  }
+
+  @Test
+  public void testBetween() throws Exception {
+    List<Object> args = new ArrayList<Object>();
+    args.add(10L);
+    args.add(20L);
+    PredicateLeaf pred = createPredicateLeaf
+        (PredicateLeaf.Operator.BETWEEN, PredicateLeaf.Type.LONG,
+            "x", null, args);
+    assertEquals(TruthValue.NO_NULL,
+        RecordReaderImpl.evaluatePredicateProto(createIntStats(0L, 5L), pred, null));
+    assertEquals(TruthValue.NO_NULL,
+      RecordReaderImpl.evaluatePredicateProto(createIntStats(30L, 40L), pred, null));
+    assertEquals(TruthValue.YES_NO_NULL,
+      RecordReaderImpl.evaluatePredicateProto(createIntStats(5L, 15L), pred, null));
+    assertEquals(TruthValue.YES_NO_NULL,
+        RecordReaderImpl.evaluatePredicateProto(createIntStats(15L, 25L), pred, null));
+    assertEquals(TruthValue.YES_NO_NULL,
+        RecordReaderImpl.evaluatePredicateProto(createIntStats(5L, 25L), pred, null));
+    assertEquals(TruthValue.YES_NULL,
+        RecordReaderImpl.evaluatePredicateProto(createIntStats(10L, 20L), pred, null));
+    assertEquals(TruthValue.YES_NULL,
+        RecordReaderImpl.evaluatePredicateProto(createIntStats(12L, 18L), pred, null));
+  }
+
+  @Test
+  public void testIsNull() throws Exception {
+    PredicateLeaf pred = createPredicateLeaf
+        (PredicateLeaf.Operator.IS_NULL, PredicateLeaf.Type.LONG,
+            "x", null, null);
+    assertEquals(TruthValue.YES_NO,
+        RecordReaderImpl.evaluatePredicateProto(createIntStats(20L, 30L), pred, null));
+  }
+
+
+  @Test
+  public void testEqualsWithNullInStats() throws Exception {
+    PredicateLeaf pred = createPredicateLeaf
+        (PredicateLeaf.Operator.EQUALS, PredicateLeaf.Type.STRING,
+            "x", "c", null);
+    assertEquals(TruthValue.NO_NULL,
+        RecordReaderImpl.evaluatePredicateProto(createStringStats("d", "e", true), pred, null)); // before
+    assertEquals(TruthValue.NO_NULL,
+        RecordReaderImpl.evaluatePredicateProto(createStringStats("a", "b", true), pred, null)); // after
+    assertEquals(TruthValue.YES_NO_NULL,
+        RecordReaderImpl.evaluatePredicateProto(createStringStats("b", "c", true), pred, null)); // max
+    assertEquals(TruthValue.YES_NO_NULL,
+        RecordReaderImpl.evaluatePredicateProto(createStringStats("c", "d", true), pred, null)); // min
+    assertEquals(TruthValue.YES_NO_NULL,
+        RecordReaderImpl.evaluatePredicateProto(createStringStats("b", "d", true), pred, null)); // middle
+    assertEquals(TruthValue.YES_NULL,
+        RecordReaderImpl.evaluatePredicateProto(createStringStats("c", "c", true), pred, null)); // same
+  }
+
+  @Test
+  public void testNullSafeEqualsWithNullInStats() throws Exception {
+    PredicateLeaf pred = createPredicateLeaf
+        (PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.STRING,
+            "x", "c", null);
+    assertEquals(TruthValue.NO,
+        RecordReaderImpl.evaluatePredicateProto(createStringStats("d", "e", true), pred, null)); // before
+    assertEquals(TruthValue.NO,
+        RecordReaderImpl.evaluatePredicateProto(createStringStats("a", "b", true), pred, null)); // after
+    assertEquals(TruthValue.YES_NO,
+        RecordReaderImpl.evaluatePredicateProto(createStringStats("b", "c", true), pred, null)); // max
+    assertEquals(TruthValue.YES_NO,
+        RecordReaderImpl.evaluatePredicateProto(createStringStats("c", "d", true), pred, null)); // min
+    assertEquals(TruthValue.YES_NO,
+        RecordReaderImpl.evaluatePredicateProto(createStringStats("b", "d", true), pred, null)); // middle
+    assertEquals(TruthValue.YES_NO,
+        RecordReaderImpl.evaluatePredicateProto(createStringStats("c", "c", true), pred, null)); // same
+  }
+
+  @Test
+  public void testLessThanWithNullInStats() throws Exception {
+    PredicateLeaf pred = createPredicateLeaf
+        (PredicateLeaf.Operator.LESS_THAN, PredicateLeaf.Type.STRING,
+            "x", "c", null);
+    assertEquals(TruthValue.NO_NULL,
+        RecordReaderImpl.evaluatePredicateProto(createStringStats("d", "e", true), pred, null)); // before
+    assertEquals(TruthValue.YES_NULL,
+        RecordReaderImpl.evaluatePredicateProto(createStringStats("a", "b", true), pred, null)); // after
+    assertEquals(TruthValue.YES_NO_NULL,
+        RecordReaderImpl.evaluatePredicateProto(createStringStats("b", "c", true), pred, null)); // max
+    assertEquals(TruthValue.NO_NULL,
+        RecordReaderImpl.evaluatePredicateProto(createStringStats("c", "d", true), pred, null)); // min
+    assertEquals(TruthValue.YES_NO_NULL,
+        RecordReaderImpl.evaluatePredicateProto(createStringStats("b", "d", true), pred, null)); // middle
+    assertEquals(TruthValue.NO_NULL, // min, same stats
+        RecordReaderImpl.evaluatePredicateProto(createStringStats("c", "c", true), pred, null));
+  }
+
+  @Test
+  public void testLessThanEqualsWithNullInStats() throws Exception {
+    PredicateLeaf pred = createPredicateLeaf
+        (PredicateLeaf.Operator.LESS_THAN_EQUALS, PredicateLeaf.Type.STRING,
+            "x", "c", null);
+    assertEquals(TruthValue.NO_NULL,
+        RecordReaderImpl.evaluatePredicateProto(createStringStats("d", "e", true), pred, null)); // before
+    assertEquals(TruthValue.YES_NULL,
+        RecordReaderImpl.evaluatePredicateProto(createStringStats("a", "b", true), pred, null)); // after
+    assertEquals(TruthValue.YES_NULL,
+        RecordReaderImpl.evaluatePredicateProto(createStringStats("b", "c", true), pred, null)); // max
+    assertEquals(TruthValue.YES_NO_NULL,
+        RecordReaderImpl.evaluatePredicateProto(createStringStats("c", "d", true), pred, null)); // min
+    assertEquals(TruthValue.YES_NO_NULL,
+        RecordReaderImpl.evaluatePredicateProto(createStringStats("b", "d", true), pred, null)); // middle
+    assertEquals(TruthValue.YES_NO_NULL,
+        RecordReaderImpl.evaluatePredicateProto(createStringStats("c", "c", true), pred, null)); // same
+  }
+
+  @Test
+  public void testInWithNullInStats() throws Exception {
+    List<Object> args = new ArrayList<Object>();
+    args.add("c");
+    args.add("f");
+    PredicateLeaf pred = createPredicateLeaf
+        (PredicateLeaf.Operator.IN, PredicateLeaf.Type.STRING,
+            "x", null, args);
+    assertEquals(TruthValue.NO_NULL, // before & after
+        RecordReaderImpl.evaluatePredicateProto(createStringStats("d", "e", true), pred, null));
+    assertEquals(TruthValue.NO_NULL,
+        RecordReaderImpl.evaluatePredicateProto(createStringStats("a", "b", true), pred, null)); // after
+    assertEquals(TruthValue.YES_NO_NULL,
+        RecordReaderImpl.evaluatePredicateProto(createStringStats("e", "f", true), pred, null)); // max
+    assertEquals(TruthValue.YES_NO_NULL,
+        RecordReaderImpl.evaluatePredicateProto(createStringStats("c", "d", true), pred, null)); // min
+    assertEquals(TruthValue.YES_NO_NULL,
+        RecordReaderImpl.evaluatePredicateProto(createStringStats("b", "d", true), pred, null)); // middle
+    assertEquals(TruthValue.YES_NULL,
+        RecordReaderImpl.evaluatePredicateProto(createStringStats("c", "c", true), pred, null)); // same
+  }
+
+  @Test
+  public void testBetweenWithNullInStats() throws Exception {
+    List<Object> args = new ArrayList<Object>();
+    args.add("c");
+    args.add("f");
+    PredicateLeaf pred = createPredicateLeaf
+        (PredicateLeaf.Operator.BETWEEN, PredicateLeaf.Type.STRING,
+            "x", null, args);
+    assertEquals(TruthValue.YES_NULL, // before & after
+        RecordReaderImpl.evaluatePredicateProto(createStringStats("d", "e", true), pred, null));
+    assertEquals(TruthValue.YES_NULL, // before & max
+        RecordReaderImpl.evaluatePredicateProto(createStringStats("e", "f", true), pred, null));
+    assertEquals(TruthValue.NO_NULL, // before & before
+        RecordReaderImpl.evaluatePredicateProto(createStringStats("h", "g", true), pred, null));
+    assertEquals(TruthValue.YES_NO_NULL, // before & min
+        RecordReaderImpl.evaluatePredicateProto(createStringStats("f", "g", true), pred, null));
+    assertEquals(TruthValue.YES_NO_NULL, // before & middle
+      RecordReaderImpl.evaluatePredicateProto(createStringStats("e", "g", true), pred, null));
+
+    assertEquals(TruthValue.YES_NULL, // min & after
+      RecordReaderImpl.evaluatePredicateProto(createStringStats("c", "e", true), pred, null));
+    assertEquals(TruthValue.YES_NULL, // min & max
+        RecordReaderImpl.evaluatePredicateProto(createStringStats("c", "f", true), pred, null));
+    assertEquals(TruthValue.YES_NO_NULL, // min & middle
+        RecordReaderImpl.evaluatePredicateProto(createStringStats("c", "g", true), pred, null));
+
+    assertEquals(TruthValue.NO_NULL,
+        RecordReaderImpl.evaluatePredicateProto(createStringStats("a", "b", true), pred, null)); // after
+    assertEquals(TruthValue.YES_NO_NULL,
+        RecordReaderImpl.evaluatePredicateProto(createStringStats("a", "c", true), pred, null)); // max
+    assertEquals(TruthValue.YES_NO_NULL,
+        RecordReaderImpl.evaluatePredicateProto(createStringStats("b", "d", true), pred, null)); // middle
+    assertEquals(TruthValue.YES_NULL, // min & after, same stats
+        RecordReaderImpl.evaluatePredicateProto(createStringStats("c", "c", true), pred, null));
+  }
+
+  @Test
+  public void testIsNullWithNullInStats() throws Exception {
+    PredicateLeaf pred = createPredicateLeaf
+        (PredicateLeaf.Operator.IS_NULL, PredicateLeaf.Type.STRING,
+            "x", null, null);
+    assertEquals(TruthValue.YES_NO,
+        RecordReaderImpl.evaluatePredicateProto(createStringStats("c", "d", true), pred, null));
+    assertEquals(TruthValue.NO,
+        RecordReaderImpl.evaluatePredicateProto(createStringStats("c", "d", false), pred, null));
+  }
+
+  @Test
+  public void testOverlap() throws Exception {
+    assertTrue(!RecordReaderUtils.overlap(0, 10, -10, -1));
+    assertTrue(RecordReaderUtils.overlap(0, 10, -1, 0));
+    assertTrue(RecordReaderUtils.overlap(0, 10, -1, 1));
+    assertTrue(RecordReaderUtils.overlap(0, 10, 2, 8));
+    assertTrue(RecordReaderUtils.overlap(0, 10, 5, 10));
+    assertTrue(RecordReaderUtils.overlap(0, 10, 10, 11));
+    assertTrue(RecordReaderUtils.overlap(0, 10, 0, 10));
+    assertTrue(RecordReaderUtils.overlap(0, 10, -1, 11));
+    assertTrue(!RecordReaderUtils.overlap(0, 10, 11, 12));
+  }
+
+  private static DiskRangeList diskRanges(Integer... points) {
+    DiskRangeList head = null, tail = null;
+    for(int i = 0; i < points.length; i += 2) {
+      DiskRangeList range = new DiskRangeList(points[i], points[i+1]);
+      if (tail == null) {
+        head = tail = range;
+      } else {
+        tail = tail.insertAfter(range);
+      }
+    }
+    return head;
+  }
+
+  @Test
+  public void testGetIndexPosition() throws Exception {
+    assertEquals(0, RecordReaderUtils.getIndexPosition
+        (OrcProto.ColumnEncoding.Kind.DIRECT, OrcProto.Type.Kind.INT,
+            OrcProto.Stream.Kind.PRESENT, true, true));
+    assertEquals(4, RecordReaderUtils.getIndexPosition
+        (OrcProto.ColumnEncoding.Kind.DIRECT, OrcProto.Type.Kind.INT,
+            OrcProto.Stream.Kind.DATA, true, true));
+    assertEquals(3, RecordReaderUtils.getIndexPosition
+        (OrcProto.ColumnEncoding.Kind.DIRECT, OrcProto.Type.Kind.INT,
+            OrcProto.Stream.Kind.DATA, false, true));
+    assertEquals(0, RecordReaderUtils.getIndexPosition
+        (OrcProto.ColumnEncoding.Kind.DIRECT, OrcProto.Type.Kind.INT,
+            OrcProto.Stream.Kind.DATA, true, false));
+    assertEquals(4, RecordReaderUtils.getIndexPosition
+        (OrcProto.ColumnEncoding.Kind.DICTIONARY, OrcProto.Type.Kind.STRING,
+            OrcProto.Stream.Kind.DATA, true, true));
+    assertEquals(4, RecordReaderUtils.getIndexPosition
+        (OrcProto.ColumnEncoding.Kind.DIRECT, OrcProto.Type.Kind.BINARY,
+            OrcProto.Stream.Kind.DATA, true, true));
+    assertEquals(3, RecordReaderUtils.getIndexPosition
+        (OrcProto.ColumnEncoding.Kind.DIRECT, OrcProto.Type.Kind.BINARY,
+            OrcProto.Stream.Kind.DATA, false, true));
+    assertEquals(6, RecordReaderUtils.getIndexPosition
+        (OrcProto.ColumnEncoding.Kind.DIRECT, OrcProto.Type.Kind.BINARY,
+            OrcProto.Stream.Kind.LENGTH, true, true));
+    assertEquals(4, RecordReaderUtils.getIndexPosition
+        (OrcProto.ColumnEncoding.Kind.DIRECT, OrcProto.Type.Kind.BINARY,
+            OrcProto.Stream.Kind.LENGTH, false, true));
+    assertEquals(4, RecordReaderUtils.getIndexPosition
+        (OrcProto.ColumnEncoding.Kind.DIRECT, OrcProto.Type.Kind.DECIMAL,
+            OrcProto.Stream.Kind.DATA, true, true));
+    assertEquals(3, RecordReaderUtils.getIndexPosition
+        (OrcProto.ColumnEncoding.Kind.DIRECT, OrcProto.Type.Kind.DECIMAL,
+            OrcProto.Stream.Kind.DATA, false, true));
+    assertEquals(6, RecordReaderUtils.getIndexPosition
+        (OrcProto.ColumnEncoding.Kind.DIRECT, OrcProto.Type.Kind.DECIMAL,
+            OrcProto.Stream.Kind.SECONDARY, true, true));
+    assertEquals(4, RecordReaderUtils.getIndexPosition
+        (OrcProto.ColumnEncoding.Kind.DIRECT, OrcProto.Type.Kind.DECIMAL,
+            OrcProto.Stream.Kind.SECONDARY, false, true));
+    assertEquals(4, RecordReaderUtils.getIndexPosition
+        (OrcProto.ColumnEncoding.Kind.DIRECT, OrcProto.Type.Kind.TIMESTAMP,
+            OrcProto.Stream.Kind.DATA, true, true));
+    assertEquals(3, RecordReaderUtils.getIndexPosition
+        (OrcProto.ColumnEncoding.Kind.DIRECT, OrcProto.Type.Kind.TIMESTAMP,
+            OrcProto.Stream.Kind.DATA, false, true));
+    assertEquals(7, RecordReaderUtils.getIndexPosition
+        (OrcProto.ColumnEncoding.Kind.DIRECT, OrcProto.Type.Kind.TIMESTAMP,
+            OrcProto.Stream.Kind.SECONDARY, true, true));
+    assertEquals(5, RecordReaderUtils.getIndexPosition
+        (OrcProto.ColumnEncoding.Kind.DIRECT, OrcProto.Type.Kind.TIMESTAMP,
+            OrcProto.Stream.Kind.SECONDARY, false, true));
+  }
+
+  @Test
+  public void testPartialPlan() throws Exception {
+    DiskRangeList result;
+
+    // set the streams
+    List<OrcProto.Stream> streams = new ArrayList<OrcProto.Stream>();
+    streams.add(OrcProto.Stream.newBuilder()
+        .setKind(OrcProto.Stream.Kind.PRESENT)
+        .setColumn(1).setLength(1000).build());
+    streams.add(OrcProto.Stream.newBuilder()
+        .setKind(OrcProto.Stream.Kind.DATA)
+        .setColumn(1).setLength(99000).build());
+    streams.add(OrcProto.Stream.newBuilder()
+        .setKind(OrcProto.Stream.Kind.PRESENT)
+        .setColumn(2).setLength(2000).build());
+    streams.add(OrcProto.Stream.newBuilder()
+        .setKind(OrcProto.Stream.Kind.DATA)
+        .setColumn(2).setLength(98000).build());
+
+    boolean[] columns = new boolean[]{true, true, false};
+    boolean[] rowGroups = new boolean[]{true, true, false, false, true, false};
+
+    // set the index
+    OrcProto.RowIndex[] indexes = new OrcProto.RowIndex[columns.length];
+    indexes[1] = OrcProto.RowIndex.newBuilder()
+        .addEntry(OrcProto.RowIndexEntry.newBuilder()
+            .addPositions(0).addPositions(-1).addPositions(-1)
+            .addPositions(0)
+            .build())
+        .addEntry(OrcProto.RowIndexEntry.newBuilder()
+            .addPositions(100).addPositions(-1).addPositions(-1)
+            .addPositions(10000)
+            .build())
+        .addEntry(OrcProto.RowIndexEntry.newBuilder()
+            .addPositions(200).addPositions(-1).addPositions(-1)
+            .addPositions(20000)
+            .build())
+        .addEntry(OrcProto.RowIndexEntry.newBuilder()
+            .addPositions(300).addPositions(-1).addPositions(-1)
+            .addPositions(30000)
+            .build())
+        .addEntry(OrcProto.RowIndexEntry.newBuilder()
+            .addPositions(400).addPositions(-1).addPositions(-1)
+            .addPositions(40000)
+            .build())
+        .addEntry(OrcProto.RowIndexEntry.newBuilder()
+            .addPositions(500).addPositions(-1).addPositions(-1)
+            .addPositions(50000)
+            .build())
+        .build();
+
+    // set encodings
+    List<OrcProto.ColumnEncoding> encodings =
+        new ArrayList<OrcProto.ColumnEncoding>();
+    encodings.add(OrcProto.ColumnEncoding.newBuilder()
+                    .setKind(OrcProto.ColumnEncoding.Kind.DIRECT).build());
+    encodings.add(OrcProto.ColumnEncoding.newBuilder()
+        .setKind(OrcProto.ColumnEncoding.Kind.DIRECT).build());
+    encodings.add(OrcProto.ColumnEncoding.newBuilder()
+        .setKind(OrcProto.ColumnEncoding.Kind.DIRECT).build());
+
+    // set types struct{x: int, y: int}
+    List<OrcProto.Type> types = new ArrayList<OrcProto.Type>();
+    types.add(OrcProto.Type.newBuilder().setKind(OrcProto.Type.Kind.STRUCT)
+                .addSubtypes(1).addSubtypes(2).addFieldNames("x")
+                .addFieldNames("y").build());
+    types.add(OrcProto.Type.newBuilder().setKind(OrcProto.Type.Kind.INT).build());
+    types.add(OrcProto.Type.newBuilder().setKind(OrcProto.Type.Kind.INT).build());
+
+    // filter by rows and groups
+    result = RecordReaderImpl.planReadPartialDataStreams(streams, indexes,
+        columns, rowGroups, false, encodings, types, 32768, false);
+    assertThat(result, is(diskRanges(0, 1000, 100, 1000, 400, 1000,
+        1000, 11000 + RecordReaderUtils.WORST_UNCOMPRESSED_SLOP,
+        11000, 21000 + RecordReaderUtils.WORST_UNCOMPRESSED_SLOP,
+        41000, 51000 + RecordReaderUtils.WORST_UNCOMPRESSED_SLOP)));
+    result = RecordReaderImpl.planReadPartialDataStreams(streams, indexes,
+        columns, rowGroups, false, encodings, types, 32768, true);
+    assertThat(result, is(diskRanges(0, 21000 + RecordReaderUtils.WORST_UNCOMPRESSED_SLOP,
+        41000, 51000 + RecordReaderUtils.WORST_UNCOMPRESSED_SLOP)));
+
+    // if we read no rows, don't read any bytes
+    rowGroups = new boolean[]{false, false, false, false, false, false};
+    result = RecordReaderImpl.planReadPartialDataStreams(streams, indexes,
+        columns, rowGroups, false, encodings, types, 32768, false);
+    assertNull(result);
+
+    // all rows, but only columns 0 and 2.
+    rowGroups = null;
+    columns = new boolean[]{true, false, true};
+    result = RecordReaderImpl.planReadPartialDataStreams(streams, indexes,
+        columns, null, false, encodings, types, 32768, false);
+    assertThat(result, is(diskRanges(100000, 102000, 102000, 200000)));
+    result = RecordReaderImpl.planReadPartialDataStreams(streams, indexes,
+        columns, null, false, encodings, types, 32768, true);
+    assertThat(result, is(diskRanges(100000, 200000)));
+
+    rowGroups = new boolean[]{false, true, false, false, false, false};
+    indexes[2] = indexes[1];
+    indexes[1] = null;
+    result = RecordReaderImpl.planReadPartialDataStreams(streams, indexes,
+        columns, rowGroups, false, encodings, types, 32768, false);
+    assertThat(result, is(diskRanges(100100, 102000,
+        112000, 122000 + RecordReaderUtils.WORST_UNCOMPRESSED_SLOP)));
+    result = RecordReaderImpl.planReadPartialDataStreams(streams, indexes,
+        columns, rowGroups, false, encodings, types, 32768, true);
+    assertThat(result, is(diskRanges(100100, 102000,
+        112000, 122000 + RecordReaderUtils.WORST_UNCOMPRESSED_SLOP)));
+
+    rowGroups = new boolean[]{false, false, false, false, false, true};
+    indexes[1] = indexes[2];
+    columns = new boolean[]{true, true, true};
+    result = RecordReaderImpl.planReadPartialDataStreams(streams, indexes,
+        columns, rowGroups, false, encodings, types, 32768, false);
+    assertThat(result, is(diskRanges(500, 1000, 51000, 100000, 100500, 102000,
+        152000, 200000)));
+    result = RecordReaderImpl.planReadPartialDataStreams(streams, indexes,
+        columns, rowGroups, false, encodings, types, 32768, true);
+    assertThat(result, is(diskRanges(500, 1000, 51000, 100000, 100500, 102000,
+        152000, 200000)));
+  }
+
+
+  @Test
+  public void testPartialPlanCompressed() throws Exception {
+    DiskRangeList result;
+
+    // set the streams
+    List<OrcProto.Stream> streams = new ArrayList<OrcProto.Stream>();
+    streams.add(OrcProto.Stream.newBuilder()
+        .setKind(OrcProto.Stream.Kind.PRESENT)
+        .setColumn(1).setLength(1000).build());
+    streams.add(OrcProto.Stream.newBuilder()
+        .setKind(OrcProto.Stream.Kind.DATA)
+        .setColumn(1).setLength(99000).build());
+    streams.add(OrcProto.Stream.newBuilder()
+        .setKind(OrcProto.Stream.Kind.PRESENT)
+        .setColumn(2).setLength(2000).build());
+    streams.add(OrcProto.Stream.newBuilder()
+        .setKind(OrcProto.Stream.Kind.DATA)
+        .setColumn(2).setLength(98000).build());
+
+    boolean[] columns = new boolean[]{true, true, false};
+    boolean[] rowGroups = new boolean[]{true, true, false, false, true, false};
+
+    // set the index
+    OrcProto.RowIndex[] indexes = new OrcProto.RowIndex[columns.length];
+    indexes[1] = OrcProto.RowIndex.newBuilder()
+        .addEntry(OrcProto.RowIndexEntry.newBuilder()
+            .addPositions(0).addPositions(-1).addPositions(-1).addPositions(-1)
+            .addPositions(0)
+            .build())
+        .addEntry(OrcProto.RowIndexEntry.newBuilder()
+            .addPositions(100).addPositions(-1).addPositions(-1).addPositions(-1)
+            .addPositions(10000)
+            .build())
+        .addEntry(OrcProto.RowIndexEntry.newBuilder()
+            .addPositions(200).addPositions(-1).addPositions(-1).addPositions(-1)
+            .addPositions(20000)
+            .build())
+        .addEntry(OrcProto.RowIndexEntry.newBuilder()
+            .addPositions(300).addPositions(-1).addPositions(-1).addPositions(-1)
+            .addPositions(30000)
+            .build())
+        .addEntry(OrcProto.RowIndexEntry.newBuilder()
+            .addPositions(400).addPositions(-1).addPositions(-1).addPositions(-1)
+            .addPositions(40000)
+            .build())
+        .addEntry(OrcProto.RowIndexEntry.newBuilder()
+            .addPositions(500).addPositions(-1).addPositions(-1).addPositions(-1)
+            .addPositions(50000)
+            .build())
+        .build();
+
+    // set encodings
+    List<OrcProto.ColumnEncoding> encodings =
+        new ArrayList<OrcProto.ColumnEncoding>();
+    encodings.add(OrcProto.ColumnEncoding.newBuilder()
+        .setKind(OrcProto.ColumnEncoding.Kind.DIRECT).build());
+    encodings.add(OrcProto.ColumnEncoding.newBuilder()
+        .setKind(OrcProto.ColumnEncoding.Kind.DIRECT).build());
+    encodings.add(OrcProto.ColumnEncoding.newBuilder()
+        .setKind(OrcProto.ColumnEncoding.Kind.DIRECT).build());
+
+    // set types struct{x: int, y: int}
+    List<OrcProto.Type> types = new ArrayList<OrcProto.Type>();
+    types.add(OrcProto.Type.newBuilder().setKind(OrcProto.Type.Kind.STRUCT)
+        .addSubtypes(1).addSubtypes(2).addFieldNames("x")
+        .addFieldNames("y").build());
+    types.add(OrcProto.Type.newBuilder().setKind(OrcProto.Type.Kind.INT).build());
+    types.add(OrcProto.Type.newBuilder().setKind(OrcProto.Type.Kind.INT).build());
+
+    // filter by rows and groups
+    result = RecordReaderImpl.planReadPartialDataStreams(streams, indexes,
+        columns, rowGroups, true, encodings, types, 32768, false);
+    assertThat(result, is(diskRanges(0, 1000, 100, 1000,
+        400, 1000, 1000, 11000+(2*32771),
+        11000, 21000+(2*32771), 41000, 100000)));
+
+    rowGroups = new boolean[]{false, false, false, false, false, true};
+    result = RecordReaderImpl.planReadPartialDataStreams(streams, indexes,
+        columns, rowGroups, true, encodings, types, 32768, false);
+    assertThat(result, is(diskRanges(500, 1000, 51000, 100000)));
+  }
+
+  @Test
+  public void testPartialPlanString() throws Exception {
+    DiskRangeList result;
+
+    // set the streams
+    List<OrcProto.Stream> streams = new ArrayList<OrcProto.Stream>();
+    streams.add(OrcProto.Stream.newBuilder()
+        .setKind(OrcProto.Stream.Kind.PRESENT)
+        .setColumn(1).setLength(1000).build());
+    streams.add(OrcProto.Stream.newBuilder()
+        .setKind(OrcProto.Stream.Kind.DATA)
+        .setColumn(1).setLength(94000).build());
+    streams.add(OrcProto.Stream.newBuilder()
+        .setKind(OrcProto.Stream.Kind.LENGTH)
+        .setColumn(1).setLength(2000).build());
+    streams.add(OrcProto.Stream.newBuilder()
+        .setKind(OrcProto.Stream.Kind.DICTIONARY_DATA)
+        .setColumn(1).setLength(3000).build());
+    streams.add(OrcProto.Stream.newBuilder()
+        .setKind(OrcProto.Stream.Kind.PRESENT)
+        .setColumn(2).setLength(2000).build());
+    streams.add(OrcProto.Stream.newBuilder()
+        .setKind(OrcProto.Stream.Kind.DATA)
+        .setColumn(2).setLength(98000).build());
+
+    boolean[] columns = new boolean[]{true, true, false};
+    boolean[] rowGroups = new boolean[]{false, true, false, false, true, true};
+
+    // set the index
+    OrcProto.RowIndex[] indexes = new OrcProto.RowIndex[columns.length];
+    indexes[1] = OrcProto.RowIndex.newBuilder()
+        .addEntry(OrcProto.RowIndexEntry.newBuilder()
+            .addPositions(0).addPositions(-1).addPositions(-1)
+            .addPositions(0)
+            .build())
+        .addEntry(OrcProto.RowIndexEntry.newBuilder()
+            .addPositions(100).addPositions(-1).addPositions(-1)
+            .addPositions(10000)
+            .build())
+        .addEntry(OrcProto.RowIndexEntry.newBuilder()
+            .addPositions(200).addPositions(-1).addPositions(-1)
+            .addPositions(20000)
+            .build())
+        .addEntry(OrcProto.RowIndexEntry.newBuilder()
+            .addPositions(300).addPositions(-1).addPositions(-1)
+            .addPositions(30000)
+            .build())
+        .addEntry(OrcProto.RowIndexEntry.newBuilder()
+            .addPositions(400).addPositions(-1).addPositions(-1)
+            .addPositions(40000)
+            .build())
+        .addEntry(OrcProto.RowIndexEntry.newBuilder()
+            .addPositions(500).addPositions(-1).addPositions(-1)
+            .addPositions(50000)
+            .build())
+        .build();
+
+    // set encodings
+    List<OrcProto.ColumnEncoding> encodings =
+        new ArrayList<OrcProto.ColumnEncoding>();
+    encodings.add(OrcProto.ColumnEncoding.newBuilder()
+        .setKind(OrcProto.ColumnEncoding.Kind.DIRECT).build());
+    encodings.add(OrcProto.ColumnEncoding.newBuilder()
+        .setKind(OrcProto.ColumnEncoding.Kind.DICTIONARY).build());
+    encodings.add(OrcProto.ColumnEncoding.newBuilder()
+        .setKind(OrcProto.ColumnEncoding.Kind.DIRECT).build());
+
+    // set types struct{x: string, y: int}
+    List<OrcProto.Type> types = new ArrayList<OrcProto.Type>();
+    types.add(OrcProto.Type.newBuilder().setKind(OrcProto.Type.Kind.STRUCT)
+        .addSubtypes(1).addSubtypes(2).addFieldNames("x")
+        .addFieldNames("y").build());
+    types.add(OrcProto.Type.newBuilder().setKind(OrcProto.Type.Kind.STRING).build());
+    types.add(OrcProto.Type.newBuilder().setKind(OrcProto.Type.Kind.INT).build());
+
+    // filter by rows and groups
+    result = RecordReaderImpl.planReadPartialDataStreams(streams, indexes,
+        columns, rowGroups, false, encodings, types, 32768, false);
+    assertThat(result, is(diskRanges(100, 1000, 400, 1000, 500, 1000,
+        11000, 21000 + RecordReaderUtils.WORST_UNCOMPRESSED_SLOP,
+        41000, 51000 + RecordReaderUtils.WORST_UNCOMPRESSED_SLOP,
+        51000, 95000, 95000, 97000, 97000, 100000)));
+  }
+
+  @Test
+  public void testIntNullSafeEqualsBloomFilter() throws Exception {
+    PredicateLeaf pred = createPredicateLeaf(
+        PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.LONG, "x", 15L, null);
+    BloomFilterIO bf = new BloomFilterIO(10000);
+    for (int i = 20; i < 1000; i++) {
+      bf.addLong(i);
+    }
+    ColumnStatistics cs = ColumnStatisticsImpl.deserialize(createIntStats(10, 100));
+    assertEquals(TruthValue.NO, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
+
+    bf.addLong(15);
+    assertEquals(TruthValue.YES_NO, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
+  }
+
+  @Test
+  public void testIntEqualsBloomFilter() throws Exception {
+    PredicateLeaf pred = createPredicateLeaf(
+        PredicateLeaf.Operator.EQUALS, PredicateLeaf.Type.LONG, "x", 15L, null);
+    BloomFilterIO bf = new BloomFilterIO(10000);
+    for (int i = 20; i < 1000; i++) {
+      bf.addLong(i);
+    }
+    ColumnStatistics cs = ColumnStatisticsImpl.deserialize(createIntStats(10, 100));
+    assertEquals(TruthValue.NO_NULL, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
+
+    bf.addLong(15);
+    assertEquals(TruthValue.YES_NO_NULL, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
+  }
+
+  @Test
+  public void testIntInBloomFilter() throws Exception {
+    List<Object> args = new ArrayList<Object>();
+    args.add(15L);
+    args.add(19L);
+    PredicateLeaf pred = createPredicateLeaf
+        (PredicateLeaf.Operator.IN, PredicateLeaf.Type.LONG,
+            "x", null, args);
+    BloomFilterIO bf = new BloomFilterIO(10000);
+    for (int i = 20; i < 1000; i++) {
+      bf.addLong(i);
+    }
+    ColumnStatistics cs = ColumnStatisticsImpl.deserialize(createIntStats(10, 100));
+    assertEquals(TruthValue.NO_NULL, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
+
+    bf.addLong(19);
+    assertEquals(TruthValue.YES_NO_NULL, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
+
+    bf.addLong(15);
+    assertEquals(TruthValue.YES_NO_NULL, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
+  }
+
+  @Test
+  public void testDoubleNullSafeEqualsBloomFilter() throws Exception {
+    PredicateLeaf pred = createPredicateLeaf(
+        PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.FLOAT, "x", 15.0, null);
+    BloomFilterIO bf = new BloomFilterIO(10000);
+    for (int i = 20; i < 1000; i++) {
+      bf.addDouble(i);
+    }
+    ColumnStatistics cs = ColumnStatisticsImpl.deserialize(createDoubleStats(10.0, 100.0));
+    assertEquals(TruthValue.NO, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
+
+    bf.addDouble(15.0);
+    assertEquals(TruthValue.YES_NO, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
+  }
+
+  @Test
+  public void testDoubleEqualsBloomFilter() throws Exception {
+    PredicateLeaf pred = createPredicateLeaf(
+        PredicateLeaf.Operator.EQUALS, PredicateLeaf.Type.FLOAT, "x", 15.0, null);
+    BloomFilterIO bf = new BloomFilterIO(10000);
+    for (int i = 20; i < 1000; i++) {
+      bf.addDouble(i);
+    }
+    ColumnStatistics cs = ColumnStatisticsImpl.deserialize(createDoubleStats(10.0, 100.0));
+    assertEquals(TruthValue.NO_NULL, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
+
+    bf.addDouble(15.0);
+    assertEquals(TruthValue.YES_NO_NULL, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
+  }
+
+  @Test
+  public void testDoubleInBloomFilter() throws Exception {
+    List<Object> args = new ArrayList<Object>();
+    args.add(15.0);
+    args.add(19.0);
+    PredicateLeaf pred = createPredicateLeaf
+        (PredicateLeaf.Operator.IN, PredicateLeaf.Type.FLOAT,
+            "x", null, args);
+    BloomFilterIO bf = new BloomFilterIO(10000);
+    for (int i = 20; i < 1000; i++) {
+      bf.addDouble(i);
+    }
+    ColumnStatistics cs = ColumnStatisticsImpl.deserialize(createDoubleStats(10.0, 100.0));
+    assertEquals(TruthValue.NO_NULL, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
+
+    bf.addDouble(19.0);
+    assertEquals(TruthValue.YES_NO_NULL, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
+
+    bf.addDouble(15.0);
+    assertEquals(TruthValue.YES_NO_NULL, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
+  }
+
+  @Test
+  public void testStringNullSafeEqualsBloomFilter() throws Exception {
+    PredicateLeaf pred = createPredicateLeaf(
+        PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.STRING, "x", "str_15", null);
+    BloomFilterIO bf = new BloomFilterIO(10000);
+    for (int i = 20; i < 1000; i++) {
+      bf.addString("str_" + i);
+    }
+    ColumnStatistics cs = ColumnStatisticsImpl.deserialize(createStringStats("str_10", "str_200"));
+    assertEquals(TruthValue.NO, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
+
+    bf.addString("str_15");
+    assertEquals(TruthValue.YES_NO, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
+  }
+
+  @Test
+  public void testStringEqualsBloomFilter() throws Exception {
+    PredicateLeaf pred = createPredicateLeaf(
+        PredicateLeaf.Operator.EQUALS, PredicateLeaf.Type.STRING, "x", "str_15", null);
+    BloomFilterIO bf = new BloomFilterIO(10000);
+    for (int i = 20; i < 1000; i++) {
+      bf.addString("str_" + i);
+    }
+    ColumnStatistics cs = ColumnStatisticsImpl.deserialize(createStringStats("str_10", "str_200"));
+    assertEquals(TruthValue.NO_NULL, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
+
+    bf.addString("str_15");
+    assertEquals(TruthValue.YES_NO_NULL, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
+  }
+
+  @Test
+  public void testStringInBloomFilter() throws Exception {
+    List<Object> args = new ArrayList<Object>();
+    args.add("str_15");
+    args.add("str_19");
+    PredicateLeaf pred = createPredicateLeaf
+        (PredicateLeaf.Operator.IN, PredicateLeaf.Type.STRING,
+            "x", null, args);
+    BloomFilterIO bf = new BloomFilterIO(10000);
+    for (int i = 20; i < 1000; i++) {
+      bf.addString("str_" + i);
+    }
+    ColumnStatistics cs = ColumnStatisticsImpl.deserialize(createStringStats("str_10", "str_200"));
+    assertEquals(TruthValue.NO_NULL, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
+
+    bf.addString("str_19");
+    assertEquals(TruthValue.YES_NO_NULL, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
+
+    bf.addString("str_15");
+    assertEquals(TruthValue.YES_NO_NULL, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
+  }
+
+  @Test
+  public void testDateWritableNullSafeEqualsBloomFilter() throws Exception {
+    PredicateLeaf pred = createPredicateLeaf(
+        PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.DATE, "x",
+        new DateWritable(15).get(), null);
+    BloomFilterIO bf = new BloomFilterIO(10000);
+    for (int i = 20; i < 1000; i++) {
+      bf.addLong((new DateWritable(i)).getDays());
+    }
+    ColumnStatistics cs = ColumnStatisticsImpl.deserialize(createDateStats(10, 100));
+    assertEquals(TruthValue.NO, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
+
+    bf.addLong((new DateWritable(15)).getDays());
+    assertEquals(TruthValue.YES_NO, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
+  }
+
+  @Test
+  public void testDateWritableEqualsBloomFilter() throws Exception {
+    PredicateLeaf pred = createPredicateLeaf(
+        PredicateLeaf.Operator.EQUALS, PredicateLeaf.Type.DATE, "x",
+        new DateWritable(15).get(), null);
+    BloomFilterIO bf = new BloomFilterIO(10000);
+    for (int i = 20; i < 1000; i++) {
+      bf.addLong((new DateWritable(i)).getDays());
+    }
+    ColumnStatistics cs = ColumnStatisticsImpl.deserialize(createDateStats(10, 100));
+    assertEquals(TruthValue.NO_NULL, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
+
+    bf.addLong((new DateWritable(15)).getDays());
+    assertEquals(TruthValue.YES_NO_NULL, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
+  }
+
+  @Test
+  public void testDateWritableInBloomFilter() throws Exception {
+    List<Object> args = new ArrayList<Object>();
+    args.add(new DateWritable(15).get());
+    args.add(new DateWritable(19).get());
+    PredicateLeaf pred = createPredicateLeaf
+        (PredicateLeaf.Operator.IN, PredicateLeaf.Type.DATE,
+            "x", null, args);
+    BloomFilterIO bf = new BloomFilterIO(10000);
+    for (int i = 20; i < 1000; i++) {
+      bf.addLong((new DateWritable(i)).getDays());
+    }
+    ColumnStatistics cs = ColumnStatisticsImpl.deserialize(createDateStats(10, 100));
+    assertEquals(TruthValue.NO_NULL, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
+
+    bf.addLong((new DateWritable(19)).getDays());
+    assertEquals(TruthValue.YES_NO_NULL, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
+
+    bf.addLong((new DateWritable(15)).getDays());
+    assertEquals(TruthValue.YES_NO_NULL, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
+  }
+
+  @Test
+  public void testTimestampNullSafeEqualsBloomFilter() throws Exception {
+    PredicateLeaf pred = createPredicateLeaf(
+        PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.TIMESTAMP, "x",
+        new Timestamp(15),
+        null);
+    BloomFilterIO bf = new BloomFilterIO(10000);
+    for (int i = 20; i < 1000; i++) {
+      bf.addLong((new Timestamp(i)).getTime());
+    }
+    ColumnStatistics cs = ColumnStatisticsImpl.deserialize(createTimestampStats(10, 100));
+    assertEquals(TruthValue.NO, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
+
+    bf.addLong((new Timestamp(15)).getTime());
+    assertEquals(TruthValue.YES_NO, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
+  }
+
+  @Test
+  public void testTimestampEqualsBloomFilter() throws Exception {
+    PredicateLeaf pred = createPredicateLeaf(
+        PredicateLeaf.Operator.EQUALS, PredicateLeaf.Type.TIMESTAMP, "x", new Timestamp(15), null);
+    BloomFilterIO bf = new BloomFilterIO(10000);
+    for (int i = 20; i < 1000; i++) {
+      bf.addLong((new Timestamp(i)).getTime());
+    }
+    ColumnStatistics cs = ColumnStatisticsImpl.deserialize(createTimestampStats(10, 100));
+    assertEquals(TruthValue.NO_NULL, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
+
+    bf.addLong((new Timestamp(15)).getTime());
+    assertEquals(TruthValue.YES_NO_NULL, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
+  }
+
+  @Test
+  public void testTimestampInBloomFilter() throws Exception {
+    List<Object> args = new ArrayList<Object>();
+    args.add(new Timestamp(15));
+    args.add(new Timestamp(19));
+    PredicateLeaf pred = createPredicateLeaf
+        (PredicateLeaf.Operator.IN, PredicateLeaf.Type.TIMESTAMP,
+            "x", null, args);
+    BloomFilterIO bf = new BloomFilterIO(10000);
+    for (int i = 20; i < 1000; i++) {
+      bf.addLong((new Timestamp(i)).getTime());
+    }
+    ColumnStatistics cs = ColumnStatisticsImpl.deserialize(createTimestampStats(10, 100));
+    assertEquals(TruthValue.NO_NULL, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
+
+    bf.addLong((new Timestamp(19)).getTime());
+    assertEquals(TruthValue.YES_NO_NULL, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
+
+    bf.addLong((new Timestamp(15)).getTime());
+    assertEquals(TruthValue.YES_NO_NULL, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
+  }
+
+  @Test
+  public void testDecimalNullSafeEqualsBloomFilter() throws Exception {
+    PredicateLeaf pred = createPredicateLeaf(
+        PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.DECIMAL, "x",
+        new HiveDecimalWritable("15"),
+        null);
+    BloomFilterIO bf = new BloomFilterIO(10000);
+    for (int i = 20; i < 1000; i++) {
+      bf.addString(HiveDecimal.create(i).toString());
+    }
+    ColumnStatistics cs = ColumnStatisticsImpl.deserialize(createDecimalStats("10", "200"));
+    assertEquals(TruthValue.NO, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
+
+    bf.addString(HiveDecimal.create(15).toString());
+    assertEquals(TruthValue.YES_NO, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
+  }
+
+  @Test
+  public void testDecimalEqualsBloomFilter() throws Exception {
+    PredicateLeaf pred = createPredicateLeaf(
+        PredicateLeaf.Operator.EQUALS, PredicateLeaf.Type.DECIMAL, "x",
+        new HiveDecimalWritable("15"),
+        null);
+    BloomFilterIO bf = new BloomFilterIO(10000);
+    for (int i = 20; i < 1000; i++) {
+      bf.addString(HiveDecimal.create(i).toString());
+    }
+    ColumnStatistics cs = ColumnStatisticsImpl.deserialize(createDecimalStats("10", "200"));
+    assertEquals(TruthValue.NO_NULL, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
+
+    bf.addString(HiveDecimal.create(15).toString());
+    assertEquals(TruthValue.YES_NO_NULL, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
+  }
+
+  @Test
+  public void testDecimalInBloomFilter() throws Exception {
+    List<Object> args = new ArrayList<Object>();
+    args.add(new HiveDecimalWritable("15"));
+    args.add(new HiveDecimalWritable("19"));
+    PredicateLeaf pred = createPredicateLeaf
+        (PredicateLeaf.Operator.IN, PredicateLeaf.Type.DECIMAL,
+            "x", null, args);
+    BloomFilterIO bf = new BloomFilterIO(10000);
+    for (int i = 20; i < 1000; i++) {
+      bf.addString(HiveDecimal.create(i).toString());
+    }
+    ColumnStatistics cs = ColumnStatisticsImpl.deserialize(createDecimalStats("10", "200"));
+    assertEquals(TruthValue.NO_NULL, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
+
+    bf.addString(HiveDecimal.create(19).toString());
+    assertEquals(TruthValue.YES_NO_NULL, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
+
+    bf.addString(HiveDecimal.create(15).toString());
+    assertEquals(TruthValue.YES_NO_NULL, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
+  }
+
+  @Test
+  public void testNullsInBloomFilter() throws Exception {
+    List<Object> args = new ArrayList<Object>();
+    args.add(new HiveDecimalWritable("15"));
+    args.add(null);
+    args.add(new HiveDecimalWritable("19"));
+    PredicateLeaf pred = createPredicateLeaf
+        (PredicateLeaf.Operator.IN, PredicateLeaf.Type.DECIMAL,
+            "x", null, args);
+    BloomFilterIO bf = new BloomFilterIO(10000);
+    for (int i = 20; i < 1000; i++) {
+      bf.addString(HiveDecimal.create(i).toString());
+    }
+    ColumnStatistics cs = ColumnStatisticsImpl.deserialize(createDecimalStats("10", "200", false));
+    // hasNull is false, so bloom filter should return NO
+    assertEquals(TruthValue.NO, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
+
+    cs = ColumnStatisticsImpl.deserialize(createDecimalStats("10", "200", true));
+    // hasNull is true, so bloom filter should return YES_NO_NULL
+    assertEquals(TruthValue.YES_NO_NULL, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
+
+    bf.addString(HiveDecimal.create(19).toString());
+    assertEquals(TruthValue.YES_NO_NULL, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
+
+    bf.addString(HiveDecimal.create(15).toString());
+    assertEquals(TruthValue.YES_NO_NULL, RecordReaderImpl.evaluatePredicate(cs, pred, bf));
+  }
+
+  @Test
+  public void testClose() throws Exception {
+    DataReader mockedDataReader = mock(DataReader.class);
+    closeMockedRecordReader(mockedDataReader);
+
+    verify(mockedDataReader, atLeastOnce()).close();
+  }
+
+  @Test
+  public void testCloseWithException() throws Exception {
+    DataReader mockedDataReader = mock(DataReader.class);
+    doThrow(IOException.class).when(mockedDataReader).close();
+
+    try {
+      closeMockedRecordReader(mockedDataReader);
+      fail("Exception should have been thrown when Record Reader was closed");
+    } catch (IOException expected) {
+
+    }
+
+    verify(mockedDataReader, atLeastOnce()).close();
+  }
+
+  Path workDir = new Path(System.getProperty("test.tmp.dir",
+      "target" + File.separator + "test" + File.separator + "tmp"));
+
+  private void closeMockedRecordReader(DataReader mockedDataReader) throws IOException {
+    Configuration conf = new Configuration();
+    Path path = new Path(workDir, "empty.orc");
+    FileSystem.get(conf).delete(path, true);
+    Writer writer = OrcFile.createWriter(path, OrcFile.writerOptions(conf)
+        .setSchema(TypeDescription.createLong()));
+    writer.close();
+    Reader reader = OrcFile.createReader(path, OrcFile.readerOptions(conf));
+
+    RecordReader recordReader = reader.rows(new Reader.Options()
+        .dataReader(mockedDataReader));
+
+    recordReader.close();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/orc/src/test/org/apache/orc/impl/TestStreamName.java
----------------------------------------------------------------------
diff --git a/orc/src/test/org/apache/orc/impl/TestStreamName.java b/orc/src/test/org/apache/orc/impl/TestStreamName.java
new file mode 100644
index 0000000..be58d4c
--- /dev/null
+++ b/orc/src/test/org/apache/orc/impl/TestStreamName.java
@@ -0,0 +1,49 @@
+/**
+ * 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.orc.impl;
+
+import org.apache.orc.OrcProto;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestStreamName {
+
+  @Test
+  public void test1() throws Exception {
+    StreamName s1 = new StreamName(3, OrcProto.Stream.Kind.DATA);
+    StreamName s2 = new StreamName(3,
+        OrcProto.Stream.Kind.DICTIONARY_DATA);
+    StreamName s3 = new StreamName(5, OrcProto.Stream.Kind.DATA);
+    StreamName s4 = new StreamName(5,
+        OrcProto.Stream.Kind.DICTIONARY_DATA);
+    StreamName s1p = new StreamName(3, OrcProto.Stream.Kind.DATA);
+    assertEquals(true, s1.equals(s1));
+    assertEquals(false, s1.equals(s2));
+    assertEquals(false, s1.equals(s3));
+    assertEquals(true, s1.equals(s1p));
+    assertEquals(true, s1.compareTo(null) < 0);
+    assertEquals(false, s1.equals(null));
+    assertEquals(true, s1.compareTo(s2) < 0);
+    assertEquals(true, s2.compareTo(s3) < 0);
+    assertEquals(true, s3.compareTo(s4) < 0);
+    assertEquals(true, s4.compareTo(s1p) > 0);
+    assertEquals(0, s1p.compareTo(s1));
+  }
+}


[04/27] hive git commit: HIVE-11417. Move the ReaderImpl and RowReaderImpl to the ORC module, by making shims for the row by row reader. (omalley reviewed by prasanth_j)

Posted by om...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestVectorOrcFile.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestVectorOrcFile.java b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestVectorOrcFile.java
deleted file mode 100644
index 6589692..0000000
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestVectorOrcFile.java
+++ /dev/null
@@ -1,2791 +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.hive.ql.io.orc;
-
-import com.google.common.collect.Lists;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hive.common.type.HiveDecimal;
-import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.ListColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.MapColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.StructColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.UnionColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
-import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
-import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
-import org.apache.hadoop.hive.ql.io.sarg.SearchArgumentFactory;
-import org.apache.hadoop.hive.serde2.io.DateWritable;
-import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
-import org.apache.hadoop.io.BytesWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hive.common.util.HiveTestUtils;
-import org.apache.orc.BinaryColumnStatistics;
-import org.apache.orc.BooleanColumnStatistics;
-import org.apache.orc.ColumnStatistics;
-import org.apache.orc.CompressionKind;
-import org.apache.orc.DataReader;
-import org.apache.orc.DecimalColumnStatistics;
-import org.apache.orc.DoubleColumnStatistics;
-import org.apache.orc.IntegerColumnStatistics;
-import org.apache.orc.impl.DataReaderProperties;
-import org.apache.orc.impl.MemoryManager;
-import org.apache.orc.impl.OrcIndex;
-import org.apache.orc.OrcProto;
-import org.apache.orc.OrcUtils;
-import org.apache.orc.StringColumnStatistics;
-import org.apache.orc.StripeInformation;
-import org.apache.orc.StripeStatistics;
-import org.apache.orc.TypeDescription;
-import org.apache.orc.Writer;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestName;
-
-import java.io.File;
-import java.io.IOException;
-import java.math.BigInteger;
-import java.nio.ByteBuffer;
-import java.sql.Date;
-import java.sql.Timestamp;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Random;
-
-import static junit.framework.Assert.assertEquals;
-import static junit.framework.Assert.assertNotNull;
-import static junit.framework.Assert.assertNull;
-import static junit.framework.Assert.assertTrue;
-
-/**
- * Tests for the vectorized reader and writer for ORC files.
- */
-public class TestVectorOrcFile {
-
-  public static class InnerStruct {
-    int int1;
-    Text string1 = new Text();
-    InnerStruct(int int1, Text string1) {
-      this.int1 = int1;
-      this.string1.set(string1);
-    }
-    InnerStruct(int int1, String string1) {
-      this.int1 = int1;
-      this.string1.set(string1);
-    }
-
-    public String toString() {
-      return "{" + int1 + ", " + string1 + "}";
-    }
-  }
-
-  public static class MiddleStruct {
-    List<InnerStruct> list = new ArrayList<InnerStruct>();
-
-    MiddleStruct(InnerStruct... items) {
-      list.clear();
-      list.addAll(Arrays.asList(items));
-    }
-  }
-
-  private static InnerStruct inner(int i, String s) {
-    return new InnerStruct(i, s);
-  }
-
-  private static Map<String, InnerStruct> map(InnerStruct... items)  {
-    Map<String, InnerStruct> result = new HashMap<String, InnerStruct>();
-    for(InnerStruct i: items) {
-      result.put(i.string1.toString(), i);
-    }
-    return result;
-  }
-
-  private static List<InnerStruct> list(InnerStruct... items) {
-    List<InnerStruct> result = new ArrayList<InnerStruct>();
-    result.addAll(Arrays.asList(items));
-    return result;
-  }
-
-  private static BytesWritable bytes(int... items) {
-    BytesWritable result = new BytesWritable();
-    result.setSize(items.length);
-    for(int i=0; i < items.length; ++i) {
-      result.getBytes()[i] = (byte) items[i];
-    }
-    return result;
-  }
-
-  private static byte[] bytesArray(int... items) {
-    byte[] result = new byte[items.length];
-    for(int i=0; i < items.length; ++i) {
-      result[i] = (byte) items[i];
-    }
-    return result;
-  }
-
-  private static ByteBuffer byteBuf(int... items) {
-    ByteBuffer result = ByteBuffer.allocate(items.length);
-    for(int item: items) {
-      result.put((byte) item);
-    }
-    result.flip();
-    return result;
-  }
-
-  Path workDir = new Path(System.getProperty("test.tmp.dir",
-      "target" + File.separator + "test" + File.separator + "tmp"));
-
-  Configuration conf;
-  FileSystem fs;
-  Path testFilePath;
-
-  @Rule
-  public TestName testCaseName = new TestName();
-
-  @Before
-  public void openFileSystem () throws Exception {
-    conf = new Configuration();
-    fs = FileSystem.getLocal(conf);
-    testFilePath = new Path(workDir, "TestVectorOrcFile." +
-        testCaseName.getMethodName() + ".orc");
-    fs.delete(testFilePath, false);
-  }
-
-  @Test
-  public void testReadFormat_0_11() throws Exception {
-    Path oldFilePath =
-        new Path(HiveTestUtils.getFileFromClasspath("orc-file-11-format.orc"));
-    Reader reader = OrcFile.createReader(oldFilePath,
-        OrcFile.readerOptions(conf).filesystem(fs));
-
-    int stripeCount = 0;
-    int rowCount = 0;
-    long currentOffset = -1;
-    for(StripeInformation stripe : reader.getStripes()) {
-      stripeCount += 1;
-      rowCount += stripe.getNumberOfRows();
-      if (currentOffset < 0) {
-        currentOffset = stripe.getOffset() + stripe.getIndexLength()
-            + stripe.getDataLength() + stripe.getFooterLength();
-      } else {
-        assertEquals(currentOffset, stripe.getOffset());
-        currentOffset += stripe.getIndexLength() + stripe.getDataLength()
-            + stripe.getFooterLength();
-      }
-    }
-    assertEquals(reader.getNumberOfRows(), rowCount);
-    assertEquals(2, stripeCount);
-
-    // check the stats
-    ColumnStatistics[] stats = reader.getStatistics();
-    assertEquals(7500, stats[1].getNumberOfValues());
-    assertEquals(3750, ((BooleanColumnStatistics) stats[1]).getFalseCount());
-    assertEquals(3750, ((BooleanColumnStatistics) stats[1]).getTrueCount());
-    assertEquals("count: 7500 hasNull: true true: 3750", stats[1].toString());
-
-    assertEquals(2048, ((IntegerColumnStatistics) stats[3]).getMaximum());
-    assertEquals(1024, ((IntegerColumnStatistics) stats[3]).getMinimum());
-    assertEquals(true, ((IntegerColumnStatistics) stats[3]).isSumDefined());
-    assertEquals(11520000, ((IntegerColumnStatistics) stats[3]).getSum());
-    assertEquals("count: 7500 hasNull: true min: 1024 max: 2048 sum: 11520000",
-        stats[3].toString());
-
-    assertEquals(Long.MAX_VALUE,
-        ((IntegerColumnStatistics) stats[5]).getMaximum());
-    assertEquals(Long.MAX_VALUE,
-        ((IntegerColumnStatistics) stats[5]).getMinimum());
-    assertEquals(false, ((IntegerColumnStatistics) stats[5]).isSumDefined());
-    assertEquals(
-        "count: 7500 hasNull: true min: 9223372036854775807 max: 9223372036854775807",
-        stats[5].toString());
-
-    assertEquals(-15.0, ((DoubleColumnStatistics) stats[7]).getMinimum());
-    assertEquals(-5.0, ((DoubleColumnStatistics) stats[7]).getMaximum());
-    assertEquals(-75000.0, ((DoubleColumnStatistics) stats[7]).getSum(),
-        0.00001);
-    assertEquals("count: 7500 hasNull: true min: -15.0 max: -5.0 sum: -75000.0",
-        stats[7].toString());
-
-    assertEquals("count: 7500 hasNull: true min: bye max: hi sum: 0", stats[9].toString());
-
-    // check the inspectors
-    TypeDescription schema = reader.getSchema();
-    assertEquals(TypeDescription.Category.STRUCT, schema.getCategory());
-    assertEquals("struct<boolean1:boolean,byte1:tinyint,short1:smallint,"
-        + "int1:int,long1:bigint,float1:float,double1:double,bytes1:"
-        + "binary,string1:string,middle:struct<list:array<struct<int1:int,"
-        + "string1:string>>>,list:array<struct<int1:int,string1:string>>,"
-        + "map:map<string,struct<int1:int,string1:string>>,ts:timestamp,"
-        + "decimal1:decimal(38,10)>", schema.toString());
-    VectorizedRowBatch batch = schema.createRowBatch();
-
-    RecordReader rows = reader.rows();
-    assertEquals(true, rows.nextBatch(batch));
-    assertEquals(1024, batch.size);
-
-    // check the contents of the first row
-    assertEquals(false, getBoolean(batch, 0));
-    assertEquals(1, getByte(batch, 0));
-    assertEquals(1024, getShort(batch, 0));
-    assertEquals(65536, getInt(batch, 0));
-    assertEquals(Long.MAX_VALUE, getLong(batch, 0));
-    assertEquals(1.0, getFloat(batch, 0), 0.00001);
-    assertEquals(-15.0, getDouble(batch, 0), 0.00001);
-    assertEquals(bytes(0, 1, 2, 3, 4), getBinary(batch, 0));
-    assertEquals("hi", getText(batch, 0).toString());
-
-    StructColumnVector middle = (StructColumnVector) batch.cols[9];
-    ListColumnVector midList = (ListColumnVector) middle.fields[0];
-    StructColumnVector midListStruct = (StructColumnVector) midList.child;
-    LongColumnVector midListInt = (LongColumnVector) midListStruct.fields[0];
-    BytesColumnVector midListStr = (BytesColumnVector) midListStruct.fields[1];
-    ListColumnVector list = (ListColumnVector) batch.cols[10];
-    StructColumnVector listStruct = (StructColumnVector) list.child;
-    LongColumnVector listInts = (LongColumnVector) listStruct.fields[0];
-    BytesColumnVector listStrs = (BytesColumnVector) listStruct.fields[1];
-    MapColumnVector map = (MapColumnVector) batch.cols[11];
-    BytesColumnVector mapKey = (BytesColumnVector) map.keys;
-    StructColumnVector mapValue = (StructColumnVector) map.values;
-    LongColumnVector mapValueInts = (LongColumnVector) mapValue.fields[0];
-    BytesColumnVector mapValueStrs = (BytesColumnVector) mapValue.fields[1];
-    TimestampColumnVector timestamp = (TimestampColumnVector) batch.cols[12];
-    DecimalColumnVector decs = (DecimalColumnVector) batch.cols[13];
-
-    assertEquals(false, middle.isNull[0]);
-    assertEquals(2, midList.lengths[0]);
-    int start = (int) midList.offsets[0];
-    assertEquals(1, midListInt.vector[start]);
-    assertEquals("bye", midListStr.toString(start));
-    assertEquals(2, midListInt.vector[start + 1]);
-    assertEquals("sigh", midListStr.toString(start + 1));
-
-    assertEquals(2, list.lengths[0]);
-    start = (int) list.offsets[0];
-    assertEquals(3, listInts.vector[start]);
-    assertEquals("good", listStrs.toString(start));
-    assertEquals(4, listInts.vector[start + 1]);
-    assertEquals("bad", listStrs.toString(start + 1));
-    assertEquals(0, map.lengths[0]);
-    assertEquals(Timestamp.valueOf("2000-03-12 15:00:00"),
-        timestamp.asScratchTimestamp(0));
-    assertEquals(new HiveDecimalWritable(HiveDecimal.create("12345678.6547456")),
-        decs.vector[0]);
-
-    // check the contents of row 7499
-    rows.seekToRow(7499);
-    assertEquals(true, rows.nextBatch(batch));
-    assertEquals(true, getBoolean(batch, 0));
-    assertEquals(100, getByte(batch, 0));
-    assertEquals(2048, getShort(batch, 0));
-    assertEquals(65536, getInt(batch, 0));
-    assertEquals(Long.MAX_VALUE, getLong(batch, 0));
-    assertEquals(2.0, getFloat(batch, 0), 0.00001);
-    assertEquals(-5.0, getDouble(batch, 0), 0.00001);
-    assertEquals(bytes(), getBinary(batch, 0));
-    assertEquals("bye", getText(batch, 0).toString());
-    assertEquals(false, middle.isNull[0]);
-    assertEquals(2, midList.lengths[0]);
-    start = (int) midList.offsets[0];
-    assertEquals(1, midListInt.vector[start]);
-    assertEquals("bye", midListStr.toString(start));
-    assertEquals(2, midListInt.vector[start + 1]);
-    assertEquals("sigh", midListStr.toString(start + 1));
-    assertEquals(3, list.lengths[0]);
-    start = (int) list.offsets[0];
-    assertEquals(100000000, listInts.vector[start]);
-    assertEquals("cat", listStrs.toString(start));
-    assertEquals(-100000, listInts.vector[start + 1]);
-    assertEquals("in", listStrs.toString(start + 1));
-    assertEquals(1234, listInts.vector[start + 2]);
-    assertEquals("hat", listStrs.toString(start + 2));
-    assertEquals(2, map.lengths[0]);
-    start = (int) map.offsets[0];
-    assertEquals("chani", mapKey.toString(start));
-    assertEquals(5, mapValueInts.vector[start]);
-    assertEquals("chani", mapValueStrs.toString(start));
-    assertEquals("mauddib", mapKey.toString(start + 1));
-    assertEquals(1, mapValueInts.vector[start + 1]);
-    assertEquals("mauddib", mapValueStrs.toString(start + 1));
-    assertEquals(Timestamp.valueOf("2000-03-12 15:00:01"),
-        timestamp.asScratchTimestamp(0));
-    assertEquals(new HiveDecimalWritable(HiveDecimal.create("12345678.6547457")),
-        decs.vector[0]);
-
-    // handle the close up
-    assertEquals(false, rows.nextBatch(batch));
-    rows.close();
-  }
-
-  @Test
-  public void testTimestamp() throws Exception {
-    TypeDescription schema = TypeDescription.createTimestamp();
-    Writer writer = OrcFile.createWriter(testFilePath,
-        OrcFile.writerOptions(conf).setSchema(schema).stripeSize(100000)
-            .bufferSize(10000).version(org.apache.orc.OrcFile.Version.V_0_11));
-    List<Timestamp> tslist = Lists.newArrayList();
-    tslist.add(Timestamp.valueOf("2037-01-01 00:00:00.000999"));
-    tslist.add(Timestamp.valueOf("2003-01-01 00:00:00.000000222"));
-    tslist.add(Timestamp.valueOf("1999-01-01 00:00:00.999999999"));
-    tslist.add(Timestamp.valueOf("1995-01-01 00:00:00.688888888"));
-    tslist.add(Timestamp.valueOf("2002-01-01 00:00:00.1"));
-    tslist.add(Timestamp.valueOf("2010-03-02 00:00:00.000009001"));
-    tslist.add(Timestamp.valueOf("2005-01-01 00:00:00.000002229"));
-    tslist.add(Timestamp.valueOf("2006-01-01 00:00:00.900203003"));
-    tslist.add(Timestamp.valueOf("2003-01-01 00:00:00.800000007"));
-    tslist.add(Timestamp.valueOf("1996-08-02 00:00:00.723100809"));
-    tslist.add(Timestamp.valueOf("1998-11-02 00:00:00.857340643"));
-    tslist.add(Timestamp.valueOf("2008-10-02 00:00:00"));
-
-    VectorizedRowBatch batch = new VectorizedRowBatch(1, 1024);
-    TimestampColumnVector vec = new TimestampColumnVector(1024);
-    batch.cols[0] = vec;
-    batch.reset();
-    batch.size = tslist.size();
-    for (int i=0; i < tslist.size(); ++i) {
-      Timestamp ts = tslist.get(i);
-      vec.set(i, ts);
-    }
-    writer.addRowBatch(batch);
-    writer.close();
-
-    Reader reader = OrcFile.createReader(testFilePath,
-        OrcFile.readerOptions(conf).filesystem(fs));
-    RecordReader rows = reader.rows();
-    batch = reader.getSchema().createRowBatch();
-    TimestampColumnVector timestamps = (TimestampColumnVector) batch.cols[0];
-    int idx = 0;
-    while (rows.nextBatch(batch)) {
-      for(int r=0; r < batch.size; ++r) {
-        assertEquals(tslist.get(idx++).getNanos(),
-            timestamps.asScratchTimestamp(r).getNanos());
-      }
-    }
-    assertEquals(tslist.size(), rows.getRowNumber());
-    assertEquals(0, writer.getSchema().getMaximumId());
-    boolean[] expected = new boolean[] {false};
-    boolean[] included = OrcUtils.includeColumns("", writer.getSchema());
-    assertEquals(true, Arrays.equals(expected, included));
-  }
-
-  @Test
-  public void testStringAndBinaryStatistics() throws Exception {
-
-    TypeDescription schema = TypeDescription.createStruct()
-        .addField("bytes1", TypeDescription.createBinary())
-        .addField("string1", TypeDescription.createString());
-    Writer writer = OrcFile.createWriter(testFilePath,
-                                         OrcFile.writerOptions(conf)
-                                         .setSchema(schema)
-                                         .stripeSize(100000)
-                                         .bufferSize(10000));
-    VectorizedRowBatch batch = schema.createRowBatch();
-    batch.size = 4;
-    BytesColumnVector field1 = (BytesColumnVector) batch.cols[0];
-    BytesColumnVector field2 = (BytesColumnVector) batch.cols[1];
-    field1.setVal(0, bytesArray(0, 1, 2, 3, 4));
-    field1.setVal(1, bytesArray(0, 1, 2, 3));
-    field1.setVal(2, bytesArray(0, 1, 2, 3, 4, 5));
-    field1.noNulls = false;
-    field1.isNull[3] = true;
-    field2.setVal(0, "foo".getBytes());
-    field2.setVal(1, "bar".getBytes());
-    field2.noNulls = false;
-    field2.isNull[2] = true;
-    field2.setVal(3, "hi".getBytes());
-    writer.addRowBatch(batch);
-    writer.close();
-    schema = writer.getSchema();
-    assertEquals(2, schema.getMaximumId());
-
-    Reader reader = OrcFile.createReader(testFilePath,
-        OrcFile.readerOptions(conf).filesystem(fs));
-
-    boolean[] expected = new boolean[] {false, false, true};
-    boolean[] included = OrcUtils.includeColumns("string1", schema);
-    assertEquals(true, Arrays.equals(expected, included));
-
-    expected = new boolean[] {false, false, false};
-    included = OrcUtils.includeColumns("", schema);
-    assertEquals(true, Arrays.equals(expected, included));
-
-    expected = new boolean[] {false, false, false};
-    included = OrcUtils.includeColumns(null, schema);
-    assertEquals(true, Arrays.equals(expected, included));
-
-    // check the stats
-    ColumnStatistics[] stats = reader.getStatistics();
-    assertEquals(4, stats[0].getNumberOfValues());
-    assertEquals("count: 4 hasNull: false", stats[0].toString());
-
-    assertEquals(3, stats[1].getNumberOfValues());
-    assertEquals(15, ((BinaryColumnStatistics) stats[1]).getSum());
-    assertEquals("count: 3 hasNull: true sum: 15", stats[1].toString());
-
-    assertEquals(3, stats[2].getNumberOfValues());
-    assertEquals("bar", ((StringColumnStatistics) stats[2]).getMinimum());
-    assertEquals("hi", ((StringColumnStatistics) stats[2]).getMaximum());
-    assertEquals(8, ((StringColumnStatistics) stats[2]).getSum());
-    assertEquals("count: 3 hasNull: true min: bar max: hi sum: 8",
-        stats[2].toString());
-
-    // check the inspectors
-    batch = reader.getSchema().createRowBatch();
-    BytesColumnVector bytes = (BytesColumnVector) batch.cols[0];
-    BytesColumnVector strs = (BytesColumnVector) batch.cols[1];
-    RecordReader rows = reader.rows();
-    assertEquals(true, rows.nextBatch(batch));
-    assertEquals(4, batch.size);
-
-    // check the contents of the first row
-    assertEquals(bytes(0,1,2,3,4), getBinary(bytes, 0));
-    assertEquals("foo", strs.toString(0));
-
-    // check the contents of second row
-    assertEquals(bytes(0,1,2,3), getBinary(bytes, 1));
-    assertEquals("bar", strs.toString(1));
-
-    // check the contents of third row
-    assertEquals(bytes(0,1,2,3,4,5), getBinary(bytes, 2));
-    assertNull(strs.toString(2));
-
-    // check the contents of fourth row
-    assertNull(getBinary(bytes, 3));
-    assertEquals("hi", strs.toString(3));
-
-    // handle the close up
-    assertEquals(false, rows.hasNext());
-    rows.close();
-  }
-
-
-  @Test
-  public void testStripeLevelStats() throws Exception {
-    TypeDescription schema = TypeDescription.createStruct()
-        .addField("int1", TypeDescription.createInt())
-        .addField("string1", TypeDescription.createString());
-    Writer writer = OrcFile.createWriter(testFilePath,
-        OrcFile.writerOptions(conf)
-            .setSchema(schema)
-            .stripeSize(100000)
-            .bufferSize(10000));
-    VectorizedRowBatch batch = schema.createRowBatch();
-    batch.size = 1000;
-    LongColumnVector field1 = (LongColumnVector) batch.cols[0];
-    BytesColumnVector field2 = (BytesColumnVector) batch.cols[1];
-    field1.isRepeating = true;
-    field2.isRepeating = true;
-    for (int b = 0; b < 11; b++) {
-      if (b >= 5) {
-        if (b >= 10) {
-          field1.vector[0] = 3;
-          field2.setVal(0, "three".getBytes());
-        } else {
-          field1.vector[0] = 2;
-          field2.setVal(0, "two".getBytes());
-        }
-      } else {
-        field1.vector[0] = 1;
-        field2.setVal(0, "one".getBytes());
-      }
-      writer.addRowBatch(batch);
-    }
-
-    writer.close();
-    Reader reader = OrcFile.createReader(testFilePath,
-        OrcFile.readerOptions(conf).filesystem(fs));
-
-    schema = writer.getSchema();
-    assertEquals(2, schema.getMaximumId());
-    boolean[] expected = new boolean[] {false, true, false};
-    boolean[] included = OrcUtils.includeColumns("int1", schema);
-    assertEquals(true, Arrays.equals(expected, included));
-
-    List<StripeStatistics> stats = reader.getStripeStatistics();
-    int numStripes = stats.size();
-    assertEquals(3, numStripes);
-    StripeStatistics ss1 = stats.get(0);
-    StripeStatistics ss2 = stats.get(1);
-    StripeStatistics ss3 = stats.get(2);
-
-    assertEquals(5000, ss1.getColumnStatistics()[0].getNumberOfValues());
-    assertEquals(5000, ss2.getColumnStatistics()[0].getNumberOfValues());
-    assertEquals(1000, ss3.getColumnStatistics()[0].getNumberOfValues());
-
-    assertEquals(5000, (ss1.getColumnStatistics()[1]).getNumberOfValues());
-    assertEquals(5000, (ss2.getColumnStatistics()[1]).getNumberOfValues());
-    assertEquals(1000, (ss3.getColumnStatistics()[1]).getNumberOfValues());
-    assertEquals(1, ((IntegerColumnStatistics)ss1.getColumnStatistics()[1]).getMinimum());
-    assertEquals(2, ((IntegerColumnStatistics)ss2.getColumnStatistics()[1]).getMinimum());
-    assertEquals(3, ((IntegerColumnStatistics)ss3.getColumnStatistics()[1]).getMinimum());
-    assertEquals(1, ((IntegerColumnStatistics)ss1.getColumnStatistics()[1]).getMaximum());
-    assertEquals(2, ((IntegerColumnStatistics)ss2.getColumnStatistics()[1]).getMaximum());
-    assertEquals(3, ((IntegerColumnStatistics)ss3.getColumnStatistics()[1]).getMaximum());
-    assertEquals(5000, ((IntegerColumnStatistics)ss1.getColumnStatistics()[1]).getSum());
-    assertEquals(10000, ((IntegerColumnStatistics)ss2.getColumnStatistics()[1]).getSum());
-    assertEquals(3000, ((IntegerColumnStatistics)ss3.getColumnStatistics()[1]).getSum());
-
-    assertEquals(5000, (ss1.getColumnStatistics()[2]).getNumberOfValues());
-    assertEquals(5000, (ss2.getColumnStatistics()[2]).getNumberOfValues());
-    assertEquals(1000, (ss3.getColumnStatistics()[2]).getNumberOfValues());
-    assertEquals("one", ((StringColumnStatistics)ss1.getColumnStatistics()[2]).getMinimum());
-    assertEquals("two", ((StringColumnStatistics)ss2.getColumnStatistics()[2]).getMinimum());
-    assertEquals("three", ((StringColumnStatistics)ss3.getColumnStatistics()[2]).getMinimum());
-    assertEquals("one", ((StringColumnStatistics)ss1.getColumnStatistics()[2]).getMaximum());
-    assertEquals("two", ((StringColumnStatistics) ss2.getColumnStatistics()[2]).getMaximum());
-    assertEquals("three", ((StringColumnStatistics)ss3.getColumnStatistics()[2]).getMaximum());
-    assertEquals(15000, ((StringColumnStatistics)ss1.getColumnStatistics()[2]).getSum());
-    assertEquals(15000, ((StringColumnStatistics)ss2.getColumnStatistics()[2]).getSum());
-    assertEquals(5000, ((StringColumnStatistics)ss3.getColumnStatistics()[2]).getSum());
-
-    RecordReaderImpl recordReader = (RecordReaderImpl) reader.rows();
-    OrcProto.RowIndex[] index = recordReader.readRowIndex(0, null, null).getRowGroupIndex();
-    assertEquals(3, index.length);
-    List<OrcProto.RowIndexEntry> items = index[1].getEntryList();
-    assertEquals(1, items.size());
-    assertEquals(3, items.get(0).getPositionsCount());
-    assertEquals(0, items.get(0).getPositions(0));
-    assertEquals(0, items.get(0).getPositions(1));
-    assertEquals(0, items.get(0).getPositions(2));
-    assertEquals(1,
-                 items.get(0).getStatistics().getIntStatistics().getMinimum());
-    index = recordReader.readRowIndex(1, null, null).getRowGroupIndex();
-    assertEquals(3, index.length);
-    items = index[1].getEntryList();
-    assertEquals(2,
-        items.get(0).getStatistics().getIntStatistics().getMaximum());
-  }
-
-  private static void setInner(StructColumnVector inner, int rowId,
-                               int i, String value) {
-    ((LongColumnVector) inner.fields[0]).vector[rowId] = i;
-    if (value != null) {
-      ((BytesColumnVector) inner.fields[1]).setVal(rowId, value.getBytes());
-    } else {
-      inner.fields[1].isNull[rowId] = true;
-      inner.fields[1].noNulls = false;
-    }
-  }
-
-  private static void checkInner(StructColumnVector inner, int rowId,
-                                 int rowInBatch, int i, String value) {
-    assertEquals("row " + rowId, i,
-        ((LongColumnVector) inner.fields[0]).vector[rowInBatch]);
-    if (value != null) {
-      assertEquals("row " + rowId, value,
-          ((BytesColumnVector) inner.fields[1]).toString(rowInBatch));
-    } else {
-      assertEquals("row " + rowId, true, inner.fields[1].isNull[rowInBatch]);
-      assertEquals("row " + rowId, false, inner.fields[1].noNulls);
-    }
-  }
-
-  private static void setInnerList(ListColumnVector list, int rowId,
-                                   List<InnerStruct> value) {
-    if (value != null) {
-      if (list.childCount + value.size() > list.child.isNull.length) {
-        list.child.ensureSize(list.childCount * 2, true);
-      }
-      list.lengths[rowId] = value.size();
-      list.offsets[rowId] = list.childCount;
-      for (int i = 0; i < list.lengths[rowId]; ++i) {
-        InnerStruct inner = value.get(i);
-        setInner((StructColumnVector) list.child, i + list.childCount,
-            inner.int1, inner.string1.toString());
-      }
-      list.childCount += value.size();
-    } else {
-      list.isNull[rowId] = true;
-      list.noNulls = false;
-    }
-  }
-
-  private static void checkInnerList(ListColumnVector list, int rowId,
-                                     int rowInBatch, List<InnerStruct> value) {
-    if (value != null) {
-      assertEquals("row " + rowId, value.size(), list.lengths[rowInBatch]);
-      int start = (int) list.offsets[rowInBatch];
-      for (int i = 0; i < list.lengths[rowInBatch]; ++i) {
-        InnerStruct inner = value.get(i);
-        checkInner((StructColumnVector) list.child, rowId, i + start,
-            inner.int1, inner.string1.toString());
-      }
-      list.childCount += value.size();
-    } else {
-      assertEquals("row " + rowId, true, list.isNull[rowInBatch]);
-      assertEquals("row " + rowId, false, list.noNulls);
-    }
-  }
-
-  private static void setInnerMap(MapColumnVector map, int rowId,
-                                  Map<String, InnerStruct> value) {
-    if (value != null) {
-      if (map.childCount >= map.keys.isNull.length) {
-        map.keys.ensureSize(map.childCount * 2, true);
-        map.values.ensureSize(map.childCount * 2, true);
-      }
-      map.lengths[rowId] = value.size();
-      int offset = map.childCount;
-      map.offsets[rowId] = offset;
-
-      for (Map.Entry<String, InnerStruct> entry : value.entrySet()) {
-        ((BytesColumnVector) map.keys).setVal(offset, entry.getKey().getBytes());
-        InnerStruct inner = entry.getValue();
-        setInner((StructColumnVector) map.values, offset, inner.int1,
-            inner.string1.toString());
-        offset += 1;
-      }
-      map.childCount = offset;
-    } else {
-      map.isNull[rowId] = true;
-      map.noNulls = false;
-    }
-  }
-
-  private static void checkInnerMap(MapColumnVector map, int rowId,
-                                    int rowInBatch,
-                                    Map<String, InnerStruct> value) {
-    if (value != null) {
-      assertEquals("row " + rowId, value.size(), map.lengths[rowInBatch]);
-      int offset = (int) map.offsets[rowInBatch];
-      for(int i=0; i < value.size(); ++i) {
-        String key = ((BytesColumnVector) map.keys).toString(offset + i);
-        InnerStruct expected = value.get(key);
-        checkInner((StructColumnVector) map.values, rowId, offset + i,
-            expected.int1, expected.string1.toString());
-      }
-    } else {
-      assertEquals("row " + rowId, true, map.isNull[rowId]);
-      assertEquals("row " + rowId, false, map.noNulls);
-    }
-  }
-
-  private static void setMiddleStruct(StructColumnVector middle, int rowId,
-                                      MiddleStruct value) {
-    if (value != null) {
-      setInnerList((ListColumnVector) middle.fields[0], rowId, value.list);
-    } else {
-      middle.isNull[rowId] = true;
-      middle.noNulls = false;
-    }
-  }
-
-  private static void checkMiddleStruct(StructColumnVector middle, int rowId,
-                                        int rowInBatch, MiddleStruct value) {
-    if (value != null) {
-      checkInnerList((ListColumnVector) middle.fields[0], rowId, rowInBatch,
-          value.list);
-    } else {
-      assertEquals("row " + rowId, true, middle.isNull[rowInBatch]);
-      assertEquals("row " + rowId, false, middle.noNulls);
-    }
-  }
-
-  private static void setBigRow(VectorizedRowBatch batch, int rowId,
-                                Boolean b1, Byte b2, Short s1,
-                                Integer i1, Long l1, Float f1,
-                                Double d1, BytesWritable b3, String s2,
-                                MiddleStruct m1, List<InnerStruct> l2,
-                                Map<String, InnerStruct> m2) {
-    ((LongColumnVector) batch.cols[0]).vector[rowId] = b1 ? 1 : 0;
-    ((LongColumnVector) batch.cols[1]).vector[rowId] = b2;
-    ((LongColumnVector) batch.cols[2]).vector[rowId] = s1;
-    ((LongColumnVector) batch.cols[3]).vector[rowId] = i1;
-    ((LongColumnVector) batch.cols[4]).vector[rowId] = l1;
-    ((DoubleColumnVector) batch.cols[5]).vector[rowId] = f1;
-    ((DoubleColumnVector) batch.cols[6]).vector[rowId] = d1;
-    if (b3 != null) {
-      ((BytesColumnVector) batch.cols[7]).setVal(rowId, b3.getBytes(), 0,
-          b3.getLength());
-    } else {
-      batch.cols[7].isNull[rowId] = true;
-      batch.cols[7].noNulls = false;
-    }
-    if (s2 != null) {
-      ((BytesColumnVector) batch.cols[8]).setVal(rowId, s2.getBytes());
-    } else {
-      batch.cols[8].isNull[rowId] = true;
-      batch.cols[8].noNulls = false;
-    }
-    setMiddleStruct((StructColumnVector) batch.cols[9], rowId, m1);
-    setInnerList((ListColumnVector) batch.cols[10], rowId, l2);
-    setInnerMap((MapColumnVector) batch.cols[11], rowId, m2);
-  }
-
-  private static void checkBigRow(VectorizedRowBatch batch,
-                                  int rowInBatch,
-                                  int rowId,
-                                  boolean b1, byte b2, short s1,
-                                  int i1, long l1, float f1,
-                                  double d1, BytesWritable b3, String s2,
-                                  MiddleStruct m1, List<InnerStruct> l2,
-                                  Map<String, InnerStruct> m2) {
-    assertEquals("row " + rowId, b1, getBoolean(batch, rowInBatch));
-    assertEquals("row " + rowId, b2, getByte(batch, rowInBatch));
-    assertEquals("row " + rowId, s1, getShort(batch, rowInBatch));
-    assertEquals("row " + rowId, i1, getInt(batch, rowInBatch));
-    assertEquals("row " + rowId, l1, getLong(batch, rowInBatch));
-    assertEquals("row " + rowId, f1, getFloat(batch, rowInBatch), 0.0001);
-    assertEquals("row " + rowId, d1, getDouble(batch, rowInBatch), 0.0001);
-    if (b3 != null) {
-      BytesColumnVector bytes = (BytesColumnVector) batch.cols[7];
-      assertEquals("row " + rowId, b3.getLength(), bytes.length[rowInBatch]);
-      for(int i=0; i < b3.getLength(); ++i) {
-        assertEquals("row " + rowId + " byte " + i, b3.getBytes()[i],
-            bytes.vector[rowInBatch][bytes.start[rowInBatch] + i]);
-      }
-    } else {
-      assertEquals("row " + rowId, true, batch.cols[7].isNull[rowInBatch]);
-      assertEquals("row " + rowId, false, batch.cols[7].noNulls);
-    }
-    if (s2 != null) {
-      assertEquals("row " + rowId, s2, getText(batch, rowInBatch).toString());
-    } else {
-      assertEquals("row " + rowId, true, batch.cols[8].isNull[rowInBatch]);
-      assertEquals("row " + rowId, false, batch.cols[8].noNulls);
-    }
-    checkMiddleStruct((StructColumnVector) batch.cols[9], rowId, rowInBatch,
-        m1);
-    checkInnerList((ListColumnVector) batch.cols[10], rowId, rowInBatch, l2);
-    checkInnerMap((MapColumnVector) batch.cols[11], rowId, rowInBatch, m2);
-  }
-
-  private static boolean getBoolean(VectorizedRowBatch batch, int rowId) {
-    return ((LongColumnVector) batch.cols[0]).vector[rowId] != 0;
-  }
-
-  private static byte getByte(VectorizedRowBatch batch, int rowId) {
-    return (byte) ((LongColumnVector) batch.cols[1]).vector[rowId];
-  }
-
-  private static short getShort(VectorizedRowBatch batch, int rowId) {
-    return (short) ((LongColumnVector) batch.cols[2]).vector[rowId];
-  }
-
-  private static int getInt(VectorizedRowBatch batch, int rowId) {
-    return (int) ((LongColumnVector) batch.cols[3]).vector[rowId];
-  }
-
-  private static long getLong(VectorizedRowBatch batch, int rowId) {
-    return ((LongColumnVector) batch.cols[4]).vector[rowId];
-  }
-
-  private static float getFloat(VectorizedRowBatch batch, int rowId) {
-    return (float) ((DoubleColumnVector) batch.cols[5]).vector[rowId];
-  }
-
-  private static double getDouble(VectorizedRowBatch batch, int rowId) {
-    return ((DoubleColumnVector) batch.cols[6]).vector[rowId];
-  }
-
-  private static BytesWritable getBinary(BytesColumnVector column, int rowId) {
-    if (column.isRepeating) {
-      rowId = 0;
-    }
-    if (column.noNulls || !column.isNull[rowId]) {
-      return new BytesWritable(Arrays.copyOfRange(column.vector[rowId],
-          column.start[rowId], column.start[rowId] + column.length[rowId]));
-    } else {
-      return null;
-    }
-  }
-
-  private static BytesWritable getBinary(VectorizedRowBatch batch, int rowId) {
-    return getBinary((BytesColumnVector) batch.cols[7], rowId);
-  }
-
-  private static Text getText(BytesColumnVector vector, int rowId) {
-    if (vector.isRepeating) {
-      rowId = 0;
-    }
-    if (vector.noNulls || !vector.isNull[rowId]) {
-      return new Text(Arrays.copyOfRange(vector.vector[rowId],
-          vector.start[rowId], vector.start[rowId] + vector.length[rowId]));
-    } else {
-      return null;
-    }
-  }
-
-  private static Text getText(VectorizedRowBatch batch, int rowId) {
-    return getText((BytesColumnVector) batch.cols[8], rowId);
-  }
-
-  private static InnerStruct getInner(StructColumnVector vector,
-                                      int rowId) {
-    return new InnerStruct(
-        (int) ((LongColumnVector) vector.fields[0]).vector[rowId],
-        getText((BytesColumnVector) vector.fields[1], rowId));
-  }
-
-  private static List<InnerStruct> getList(ListColumnVector cv,
-                                           int rowId) {
-    if (cv.isRepeating) {
-      rowId = 0;
-    }
-    if (cv.noNulls || !cv.isNull[rowId]) {
-      List<InnerStruct> result =
-          new ArrayList<InnerStruct>((int) cv.lengths[rowId]);
-      for(long i=cv.offsets[rowId];
-          i < cv.offsets[rowId] + cv.lengths[rowId]; ++i) {
-        result.add(getInner((StructColumnVector) cv.child, (int) i));
-      }
-      return result;
-    } else {
-      return null;
-    }
-  }
-
-  private static List<InnerStruct> getMidList(VectorizedRowBatch batch,
-                                              int rowId) {
-    return getList((ListColumnVector) ((StructColumnVector) batch.cols[9])
-        .fields[0], rowId);
-  }
-
-  private static List<InnerStruct> getList(VectorizedRowBatch batch,
-                                           int rowId) {
-    return getList((ListColumnVector) batch.cols[10], rowId);
-  }
-
-  private static Map<Text, InnerStruct> getMap(VectorizedRowBatch batch,
-                                               int rowId) {
-    MapColumnVector cv = (MapColumnVector) batch.cols[11];
-    if (cv.isRepeating) {
-      rowId = 0;
-    }
-    if (cv.noNulls || !cv.isNull[rowId]) {
-      Map<Text, InnerStruct> result =
-          new HashMap<Text, InnerStruct>((int) cv.lengths[rowId]);
-      for(long i=cv.offsets[rowId];
-          i < cv.offsets[rowId] + cv.lengths[rowId]; ++i) {
-        result.put(getText((BytesColumnVector) cv.keys, (int) i),
-            getInner((StructColumnVector) cv.values, (int) i));
-      }
-      return result;
-    } else {
-      return null;
-    }
-  }
-
-  private static TypeDescription createInnerSchema() {
-    return TypeDescription.createStruct()
-        .addField("int1", TypeDescription.createInt())
-        .addField("string1", TypeDescription.createString());
-  }
-
-  private static TypeDescription createBigRowSchema() {
-    return TypeDescription.createStruct()
-        .addField("boolean1", TypeDescription.createBoolean())
-        .addField("byte1", TypeDescription.createByte())
-        .addField("short1", TypeDescription.createShort())
-        .addField("int1", TypeDescription.createInt())
-        .addField("long1", TypeDescription.createLong())
-        .addField("float1", TypeDescription.createFloat())
-        .addField("double1", TypeDescription.createDouble())
-        .addField("bytes1", TypeDescription.createBinary())
-        .addField("string1", TypeDescription.createString())
-        .addField("middle", TypeDescription.createStruct()
-            .addField("list", TypeDescription.createList(createInnerSchema())))
-        .addField("list", TypeDescription.createList(createInnerSchema()))
-        .addField("map", TypeDescription.createMap(
-            TypeDescription.createString(),
-            createInnerSchema()));
-  }
-
-  static void assertArrayEquals(boolean[] expected, boolean[] actual) {
-    assertEquals(expected.length, actual.length);
-    boolean diff = false;
-    for(int i=0; i < expected.length; ++i) {
-      if (expected[i] != actual[i]) {
-        System.out.println("Difference at " + i + " expected: " + expected[i] +
-          " actual: " + actual[i]);
-        diff = true;
-      }
-    }
-    assertEquals(false, diff);
-  }
-
-  @Test
-  public void test1() throws Exception {
-    TypeDescription schema = createBigRowSchema();
-    Writer writer = OrcFile.createWriter(testFilePath,
-        OrcFile.writerOptions(conf)
-            .setSchema(schema)
-            .stripeSize(100000)
-            .bufferSize(10000));
-    VectorizedRowBatch batch = schema.createRowBatch();
-    batch.size = 2;
-    setBigRow(batch, 0, false, (byte) 1, (short) 1024, 65536,
-        Long.MAX_VALUE, (float) 1.0, -15.0, bytes(0, 1, 2, 3, 4), "hi",
-        new MiddleStruct(inner(1, "bye"), inner(2, "sigh")),
-        list(inner(3, "good"), inner(4, "bad")),
-        map());
-    setBigRow(batch, 1, true, (byte) 100, (short) 2048, 65536,
-        Long.MAX_VALUE, (float) 2.0, -5.0, bytes(), "bye",
-        new MiddleStruct(inner(1, "bye"), inner(2, "sigh")),
-        list(inner(100000000, "cat"), inner(-100000, "in"), inner(1234, "hat")),
-        map(inner(5, "chani"), inner(1, "mauddib")));
-    writer.addRowBatch(batch);
-    writer.close();
-    Reader reader = OrcFile.createReader(testFilePath,
-        OrcFile.readerOptions(conf).filesystem(fs));
-
-    schema = writer.getSchema();
-    assertEquals(23, schema.getMaximumId());
-    boolean[] expected = new boolean[] {false, false, false, false, false,
-        false, false, false, false, false,
-        false, false, false, false, false,
-        false, false, false, false, false,
-        false, false, false, false};
-    boolean[] included = OrcUtils.includeColumns("", schema);
-    assertEquals(true, Arrays.equals(expected, included));
-
-    expected = new boolean[] {false, true, false, false, false,
-        false, false, false, false, true,
-        true, true, true, true, true,
-        false, false, false, false, true,
-        true, true, true, true};
-    included = OrcUtils.includeColumns("boolean1,string1,middle,map", schema);
-
-    assertArrayEquals(expected, included);
-
-    expected = new boolean[] {false, true, false, false, false,
-        false, false, false, false, true,
-        true, true, true, true, true,
-        false, false, false, false, true,
-        true, true, true, true};
-    included = OrcUtils.includeColumns("boolean1,string1,middle,map", schema);
-    assertArrayEquals(expected, included);
-
-    expected = new boolean[] {false, true, true, true, true,
-        true, true, true, true, true,
-        true, true, true, true, true,
-        true, true, true, true, true,
-        true, true, true, true};
-    included = OrcUtils.includeColumns(
-        "boolean1,byte1,short1,int1,long1,float1,double1,bytes1,string1,middle,list,map",
-        schema);
-    assertEquals(true, Arrays.equals(expected, included));
-
-    // check the stats
-    ColumnStatistics[] stats = reader.getStatistics();
-    assertEquals(2, stats[1].getNumberOfValues());
-    assertEquals(1, ((BooleanColumnStatistics) stats[1]).getFalseCount());
-    assertEquals(1, ((BooleanColumnStatistics) stats[1]).getTrueCount());
-    assertEquals("count: 2 hasNull: false true: 1", stats[1].toString());
-
-    assertEquals(2048, ((IntegerColumnStatistics) stats[3]).getMaximum());
-    assertEquals(1024, ((IntegerColumnStatistics) stats[3]).getMinimum());
-    assertEquals(true, ((IntegerColumnStatistics) stats[3]).isSumDefined());
-    assertEquals(3072, ((IntegerColumnStatistics) stats[3]).getSum());
-    assertEquals("count: 2 hasNull: false min: 1024 max: 2048 sum: 3072",
-        stats[3].toString());
-
-    StripeStatistics ss = reader.getStripeStatistics().get(0);
-    assertEquals(2, ss.getColumnStatistics()[0].getNumberOfValues());
-    assertEquals(1, ((BooleanColumnStatistics) ss.getColumnStatistics()[1]).getTrueCount());
-    assertEquals(1024, ((IntegerColumnStatistics) ss.getColumnStatistics()[3]).getMinimum());
-    assertEquals(2048, ((IntegerColumnStatistics) ss.getColumnStatistics()[3]).getMaximum());
-    assertEquals(3072, ((IntegerColumnStatistics) ss.getColumnStatistics()[3]).getSum());
-    assertEquals(-15.0, ((DoubleColumnStatistics) stats[7]).getMinimum());
-    assertEquals(-5.0, ((DoubleColumnStatistics) stats[7]).getMaximum());
-    assertEquals(-20.0, ((DoubleColumnStatistics) stats[7]).getSum(), 0.00001);
-    assertEquals("count: 2 hasNull: false min: -15.0 max: -5.0 sum: -20.0",
-        stats[7].toString());
-
-    assertEquals("count: 2 hasNull: false min: bye max: hi sum: 5", stats[9].toString());
-
-    // check the schema
-    TypeDescription readerSchema = reader.getSchema();
-    assertEquals(TypeDescription.Category.STRUCT, readerSchema.getCategory());
-    assertEquals("struct<boolean1:boolean,byte1:tinyint,short1:smallint,"
-        + "int1:int,long1:bigint,float1:float,double1:double,bytes1:"
-        + "binary,string1:string,middle:struct<list:array<struct<int1:int,"
-        + "string1:string>>>,list:array<struct<int1:int,string1:string>>,"
-        + "map:map<string,struct<int1:int,string1:string>>>",
-        readerSchema.toString());
-    List<String> fieldNames = readerSchema.getFieldNames();
-    List<TypeDescription> fieldTypes = readerSchema.getChildren();
-    assertEquals("boolean1", fieldNames.get(0));
-    assertEquals(TypeDescription.Category.BOOLEAN, fieldTypes.get(0).getCategory());
-    assertEquals("byte1", fieldNames.get(1));
-    assertEquals(TypeDescription.Category.BYTE, fieldTypes.get(1).getCategory());
-    assertEquals("short1", fieldNames.get(2));
-    assertEquals(TypeDescription.Category.SHORT, fieldTypes.get(2).getCategory());
-    assertEquals("int1", fieldNames.get(3));
-    assertEquals(TypeDescription.Category.INT, fieldTypes.get(3).getCategory());
-    assertEquals("long1", fieldNames.get(4));
-    assertEquals(TypeDescription.Category.LONG, fieldTypes.get(4).getCategory());
-    assertEquals("float1", fieldNames.get(5));
-    assertEquals(TypeDescription.Category.FLOAT, fieldTypes.get(5).getCategory());
-    assertEquals("double1", fieldNames.get(6));
-    assertEquals(TypeDescription.Category.DOUBLE, fieldTypes.get(6).getCategory());
-    assertEquals("bytes1", fieldNames.get(7));
-    assertEquals(TypeDescription.Category.BINARY, fieldTypes.get(7).getCategory());
-    assertEquals("string1", fieldNames.get(8));
-    assertEquals(TypeDescription.Category.STRING, fieldTypes.get(8).getCategory());
-    assertEquals("middle", fieldNames.get(9));
-    TypeDescription middle = fieldTypes.get(9);
-    assertEquals(TypeDescription.Category.STRUCT, middle.getCategory());
-    TypeDescription midList = middle.getChildren().get(0);
-    assertEquals(TypeDescription.Category.LIST, midList.getCategory());
-    TypeDescription inner = midList.getChildren().get(0);
-    assertEquals(TypeDescription.Category.STRUCT, inner.getCategory());
-    assertEquals("int1", inner.getFieldNames().get(0));
-    assertEquals("string1", inner.getFieldNames().get(1));
-
-    RecordReader rows = reader.rows();
-    // create a new batch
-    batch = readerSchema.createRowBatch();
-    assertEquals(true, rows.nextBatch(batch));
-    assertEquals(2, batch.size);
-    assertEquals(false, rows.hasNext());
-
-    // check the contents of the first row
-    assertEquals(false, getBoolean(batch, 0));
-    assertEquals(1, getByte(batch, 0));
-    assertEquals(1024, getShort(batch, 0));
-    assertEquals(65536, getInt(batch, 0));
-    assertEquals(Long.MAX_VALUE, getLong(batch, 0));
-    assertEquals(1.0, getFloat(batch, 0), 0.00001);
-    assertEquals(-15.0, getDouble(batch, 0), 0.00001);
-    assertEquals(bytes(0,1,2,3,4), getBinary(batch, 0));
-    assertEquals("hi", getText(batch, 0).toString());
-    List<InnerStruct> midRow = getMidList(batch, 0);
-    assertNotNull(midRow);
-    assertEquals(2, midRow.size());
-    assertEquals(1, midRow.get(0).int1);
-    assertEquals("bye", midRow.get(0).string1.toString());
-    assertEquals(2, midRow.get(1).int1);
-    assertEquals("sigh", midRow.get(1).string1.toString());
-    List<InnerStruct> list = getList(batch, 0);
-    assertEquals(2, list.size());
-    assertEquals(3, list.get(0).int1);
-    assertEquals("good", list.get(0).string1.toString());
-    assertEquals(4, list.get(1).int1);
-    assertEquals("bad", list.get(1).string1.toString());
-    Map<Text, InnerStruct> map = getMap(batch, 0);
-    assertEquals(0, map.size());
-
-    // check the contents of second row
-    assertEquals(true, getBoolean(batch, 1));
-    assertEquals(100, getByte(batch, 1));
-    assertEquals(2048, getShort(batch, 1));
-    assertEquals(65536, getInt(batch, 1));
-    assertEquals(Long.MAX_VALUE, getLong(batch, 1));
-    assertEquals(2.0, getFloat(batch, 1), 0.00001);
-    assertEquals(-5.0, getDouble(batch, 1), 0.00001);
-    assertEquals(bytes(), getBinary(batch, 1));
-    assertEquals("bye", getText(batch, 1).toString());
-    midRow = getMidList(batch, 1);
-    assertNotNull(midRow);
-    assertEquals(2, midRow.size());
-    assertEquals(1, midRow.get(0).int1);
-    assertEquals("bye", midRow.get(0).string1.toString());
-    assertEquals(2, midRow.get(1).int1);
-    assertEquals("sigh", midRow.get(1).string1.toString());
-    list = getList(batch, 1);
-    assertEquals(3, list.size());
-    assertEquals(100000000, list.get(0).int1);
-    assertEquals("cat", list.get(0).string1.toString());
-    assertEquals(-100000, list.get(1).int1);
-    assertEquals("in", list.get(1).string1.toString());
-    assertEquals(1234, list.get(2).int1);
-    assertEquals("hat", list.get(2).string1.toString());
-    map = getMap(batch, 1);
-    assertEquals(2, map.size());
-    InnerStruct value = map.get(new Text("chani"));
-    assertEquals(5, value.int1);
-    assertEquals("chani", value.string1.toString());
-    value = map.get(new Text("mauddib"));
-    assertEquals(1, value.int1);
-    assertEquals("mauddib", value.string1.toString());
-
-    // handle the close up
-    assertEquals(false, rows.nextBatch(batch));
-    rows.close();
-  }
-
-  @Test
-  public void testColumnProjection() throws Exception {
-    TypeDescription schema = createInnerSchema();
-    Writer writer = OrcFile.createWriter(testFilePath,
-                                         OrcFile.writerOptions(conf)
-                                         .setSchema(schema)
-                                         .stripeSize(1000)
-                                         .compress(CompressionKind.NONE)
-                                         .bufferSize(100)
-                                         .rowIndexStride(1000));
-    VectorizedRowBatch batch = schema.createRowBatch();
-    Random r1 = new Random(1);
-    Random r2 = new Random(2);
-    int x;
-    int minInt=0, maxInt=0;
-    String y;
-    String minStr = null, maxStr = null;
-    batch.size = 1000;
-    boolean first = true;
-    for(int b=0; b < 21; ++b) {
-      for(int r=0; r < 1000; ++r) {
-        x = r1.nextInt();
-        y = Long.toHexString(r2.nextLong());
-        if (first || x < minInt) {
-          minInt = x;
-        }
-        if (first || x > maxInt) {
-          maxInt = x;
-        }
-        if (first || y.compareTo(minStr) < 0) {
-          minStr = y;
-        }
-        if (first || y.compareTo(maxStr) > 0) {
-          maxStr = y;
-        }
-        first = false;
-        ((LongColumnVector) batch.cols[0]).vector[r] = x;
-        ((BytesColumnVector) batch.cols[1]).setVal(r, y.getBytes());
-      }
-      writer.addRowBatch(batch);
-    }
-    writer.close();
-    Reader reader = OrcFile.createReader(testFilePath,
-        OrcFile.readerOptions(conf).filesystem(fs));
-
-    // check out the statistics
-    ColumnStatistics[] stats = reader.getStatistics();
-    assertEquals(3, stats.length);
-    for(ColumnStatistics s: stats) {
-      assertEquals(21000, s.getNumberOfValues());
-      if (s instanceof IntegerColumnStatistics) {
-        assertEquals(minInt, ((IntegerColumnStatistics) s).getMinimum());
-        assertEquals(maxInt, ((IntegerColumnStatistics) s).getMaximum());
-      } else if (s instanceof  StringColumnStatistics) {
-        assertEquals(maxStr, ((StringColumnStatistics) s).getMaximum());
-        assertEquals(minStr, ((StringColumnStatistics) s).getMinimum());
-      }
-    }
-
-    // check out the types
-    TypeDescription type = reader.getSchema();
-    assertEquals(TypeDescription.Category.STRUCT, type.getCategory());
-    assertEquals(2, type.getChildren().size());
-    TypeDescription type1 = type.getChildren().get(0);
-    TypeDescription type2 = type.getChildren().get(1);
-    assertEquals(TypeDescription.Category.INT, type1.getCategory());
-    assertEquals(TypeDescription.Category.STRING, type2.getCategory());
-    assertEquals("struct<int1:int,string1:string>", type.toString());
-
-    // read the contents and make sure they match
-    RecordReader rows1 = reader.rows(new boolean[]{true, true, false});
-    RecordReader rows2 = reader.rows(new boolean[]{true, false, true});
-    r1 = new Random(1);
-    r2 = new Random(2);
-    VectorizedRowBatch batch1 = reader.getSchema().createRowBatch(1000);
-    VectorizedRowBatch batch2 = reader.getSchema().createRowBatch(1000);
-    for(int i = 0; i < 21000; i += 1000) {
-      assertEquals(true, rows1.nextBatch(batch1));
-      assertEquals(true, rows2.nextBatch(batch2));
-      assertEquals(1000, batch1.size);
-      assertEquals(1000, batch2.size);
-      for(int j=0; j < 1000; ++j) {
-        assertEquals(r1.nextInt(),
-            ((LongColumnVector) batch1.cols[0]).vector[j]);
-        assertEquals(Long.toHexString(r2.nextLong()),
-            ((BytesColumnVector) batch2.cols[1]).toString(j));
-      }
-    }
-    assertEquals(false, rows1.nextBatch(batch1));
-    assertEquals(false, rows2.nextBatch(batch2));
-    rows1.close();
-    rows2.close();
-  }
-
-  @Test
-  public void testEmptyFile() throws Exception {
-    TypeDescription schema = createBigRowSchema();
-    Writer writer = OrcFile.createWriter(testFilePath,
-                                         OrcFile.writerOptions(conf)
-                                         .setSchema(schema)
-                                         .stripeSize(1000)
-                                         .compress(CompressionKind.NONE)
-                                         .bufferSize(100));
-    writer.close();
-    Reader reader = OrcFile.createReader(testFilePath,
-        OrcFile.readerOptions(conf).filesystem(fs));
-    assertEquals(false, reader.rows().hasNext());
-    assertEquals(CompressionKind.NONE, reader.getCompressionKind());
-    assertEquals(0, reader.getNumberOfRows());
-    assertEquals(0, reader.getCompressionSize());
-    assertEquals(false, reader.getMetadataKeys().iterator().hasNext());
-    assertEquals(3, reader.getContentLength());
-    assertEquals(false, reader.getStripes().iterator().hasNext());
-  }
-
-  @Test
-  public void metaData() throws Exception {
-    TypeDescription schema = createBigRowSchema();
-    Writer writer = OrcFile.createWriter(testFilePath,
-        OrcFile.writerOptions(conf)
-            .setSchema(schema)
-            .stripeSize(1000)
-            .compress(CompressionKind.NONE)
-            .bufferSize(100));
-    writer.addUserMetadata("my.meta", byteBuf(1, 2, 3, 4, 5, 6, 7, -1, -2, 127,
-                                              -128));
-    writer.addUserMetadata("clobber", byteBuf(1, 2, 3));
-    writer.addUserMetadata("clobber", byteBuf(4, 3, 2, 1));
-    ByteBuffer bigBuf = ByteBuffer.allocate(40000);
-    Random random = new Random(0);
-    random.nextBytes(bigBuf.array());
-    writer.addUserMetadata("big", bigBuf);
-    bigBuf.position(0);
-    VectorizedRowBatch batch = schema.createRowBatch();
-    batch.size = 1;
-    setBigRow(batch, 0, true, (byte) 127, (short) 1024, 42,
-        42L * 1024 * 1024 * 1024, (float) 3.1415, -2.713, null,
-        null, null, null, null);
-    writer.addRowBatch(batch);
-    writer.addUserMetadata("clobber", byteBuf(5,7,11,13,17,19));
-    writer.close();
-
-    Reader reader = OrcFile.createReader(testFilePath,
-        OrcFile.readerOptions(conf).filesystem(fs));
-    assertEquals(byteBuf(5,7,11,13,17,19), reader.getMetadataValue("clobber"));
-    assertEquals(byteBuf(1,2,3,4,5,6,7,-1,-2,127,-128),
-        reader.getMetadataValue("my.meta"));
-    assertEquals(bigBuf, reader.getMetadataValue("big"));
-    try {
-      reader.getMetadataValue("unknown");
-      assertTrue(false);
-    } catch (IllegalArgumentException iae) {
-      // PASS
-    }
-    int i = 0;
-    for(String key: reader.getMetadataKeys()) {
-      if ("my.meta".equals(key) ||
-          "clobber".equals(key) ||
-          "big".equals(key)) {
-        i += 1;
-      } else {
-        throw new IllegalArgumentException("unknown key " + key);
-      }
-    }
-    assertEquals(3, i);
-    int numStripes = reader.getStripeStatistics().size();
-    assertEquals(1, numStripes);
-  }
-
-  /**
-   * Generate an ORC file with a range of dates and times.
-   */
-  public void createOrcDateFile(Path file, int minYear, int maxYear
-                                ) throws IOException {
-    TypeDescription schema = TypeDescription.createStruct()
-        .addField("time", TypeDescription.createTimestamp())
-        .addField("date", TypeDescription.createDate());
-    Writer writer = OrcFile.createWriter(file,
-        OrcFile.writerOptions(conf)
-            .setSchema(schema)
-            .stripeSize(100000)
-            .bufferSize(10000)
-            .blockPadding(false));
-    VectorizedRowBatch batch = schema.createRowBatch();
-    batch.size = 1000;
-    for (int year = minYear; year < maxYear; ++year) {
-      for (int ms = 1000; ms < 2000; ++ms) {
-        TimestampColumnVector timestampColVector = (TimestampColumnVector) batch.cols[0];
-        timestampColVector.set(ms - 1000,
-            Timestamp.valueOf(year +
-                "-05-05 12:34:56." + ms));
-        ((LongColumnVector) batch.cols[1]).vector[ms - 1000] =
-            new DateWritable(new Date(year - 1900, 11, 25)).getDays();
-      }
-      writer.addRowBatch(batch);
-    }
-    writer.close();
-    Reader reader = OrcFile.createReader(file,
-        OrcFile.readerOptions(conf));
-    RecordReader rows = reader.rows();
-    batch = reader.getSchema().createRowBatch(1000);
-    TimestampColumnVector times = (TimestampColumnVector) batch.cols[0];
-    LongColumnVector dates = (LongColumnVector) batch.cols[1];
-    for (int year = minYear; year < maxYear; ++year) {
-      rows.nextBatch(batch);
-      assertEquals(1000, batch.size);
-      for(int ms = 1000; ms < 2000; ++ms) {
-        StringBuilder buffer = new StringBuilder();
-        times.stringifyValue(buffer, ms - 1000);
-        String expected = Integer.toString(year) + "-05-05 12:34:56.";
-        // suppress the final zeros on the string by dividing by the largest
-        // power of 10 that divides evenly.
-        int roundedMs = ms;
-        for(int round = 1000; round > 0; round /= 10) {
-          if (ms % round == 0) {
-            roundedMs = ms / round;
-            break;
-          }
-        }
-        expected += roundedMs;
-        assertEquals(expected, buffer.toString());
-        assertEquals(Integer.toString(year) + "-12-25",
-            new DateWritable((int) dates.vector[ms - 1000]).toString());
-      }
-    }
-    rows.nextBatch(batch);
-    assertEquals(0, batch.size);
-  }
-
-  @Test
-  public void testDate1900() throws Exception {
-    createOrcDateFile(testFilePath, 1900, 1970);
-  }
-
-  @Test
-  public void testDate2038() throws Exception {
-    createOrcDateFile(testFilePath, 2038, 2250);
-  }
-
-  private static void setUnion(VectorizedRowBatch batch, int rowId,
-                               Timestamp ts, Integer tag, Integer i, String s,
-                               HiveDecimalWritable dec) {
-    UnionColumnVector union = (UnionColumnVector) batch.cols[1];
-    if (ts != null) {
-      TimestampColumnVector timestampColVector = (TimestampColumnVector) batch.cols[0];
-      timestampColVector.set(rowId, ts);
-    } else {
-      batch.cols[0].isNull[rowId] = true;
-      batch.cols[0].noNulls = false;
-    }
-    if (tag != null) {
-      union.tags[rowId] = tag;
-      if (tag == 0) {
-        if (i != null) {
-          ((LongColumnVector) union.fields[tag]).vector[rowId] = i;
-        } else {
-          union.fields[tag].isNull[rowId] = true;
-          union.fields[tag].noNulls = false;
-        }
-      } else if (tag == 1) {
-        if (s != null) {
-          ((BytesColumnVector) union.fields[tag]).setVal(rowId, s.getBytes());
-        } else {
-          union.fields[tag].isNull[rowId] = true;
-          union.fields[tag].noNulls = false;
-        }
-      } else {
-        throw new IllegalArgumentException("Bad tag " + tag);
-      }
-    } else {
-      batch.cols[1].isNull[rowId] = true;
-      batch.cols[1].noNulls = false;
-    }
-    if (dec != null) {
-      ((DecimalColumnVector) batch.cols[2]).vector[rowId] = dec;
-    } else {
-      batch.cols[2].isNull[rowId] = true;
-      batch.cols[2].noNulls = false;
-    }
-  }
-
-  /**
-     * We test union, timestamp, and decimal separately since we need to make the
-     * object inspector manually. (The Hive reflection-based doesn't handle
-     * them properly.)
-     */
-  @Test
-  public void testUnionAndTimestamp() throws Exception {
-    TypeDescription schema = TypeDescription.createStruct()
-        .addField("time", TypeDescription.createTimestamp())
-        .addField("union", TypeDescription.createUnion()
-            .addUnionChild(TypeDescription.createInt())
-            .addUnionChild(TypeDescription.createString()))
-        .addField("decimal", TypeDescription.createDecimal()
-            .withPrecision(38)
-            .withScale(18));
-    HiveDecimal maxValue = HiveDecimal.create("10000000000000000000");
-    Writer writer = OrcFile.createWriter(testFilePath,
-                                         OrcFile.writerOptions(conf)
-                                         .setSchema(schema)
-                                         .stripeSize(1000)
-                                         .compress(CompressionKind.NONE)
-                                         .bufferSize(100)
-                                         .blockPadding(false));
-    VectorizedRowBatch batch = schema.createRowBatch();
-    batch.size = 6;
-    setUnion(batch, 0, Timestamp.valueOf("2000-03-12 15:00:00"), 0, 42, null,
-             new HiveDecimalWritable("12345678.6547456"));
-    setUnion(batch, 1, Timestamp.valueOf("2000-03-20 12:00:00.123456789"),
-        1, null, "hello", new HiveDecimalWritable("-5643.234"));
-
-    setUnion(batch, 2, null, null, null, null, null);
-    setUnion(batch, 3, null, 0, null, null, null);
-    setUnion(batch, 4, null, 1, null, null, null);
-
-    setUnion(batch, 5, Timestamp.valueOf("1970-01-01 00:00:00"), 0, 200000,
-        null, new HiveDecimalWritable("10000000000000000000"));
-    writer.addRowBatch(batch);
-
-    batch.reset();
-    Random rand = new Random(42);
-    for(int i=1970; i < 2038; ++i) {
-      Timestamp ts = Timestamp.valueOf(i + "-05-05 12:34:56." + i);
-      HiveDecimal dec =
-          HiveDecimal.create(new BigInteger(64, rand), rand.nextInt(18));
-      if ((i & 1) == 0) {
-        setUnion(batch, batch.size++, ts, 0, i*i, null,
-            new HiveDecimalWritable(dec));
-      } else {
-        setUnion(batch, batch.size++, ts, 1, null, Integer.toString(i*i),
-            new HiveDecimalWritable(dec));
-      }
-      if (maxValue.compareTo(dec) < 0) {
-        maxValue = dec;
-      }
-    }
-    writer.addRowBatch(batch);
-    batch.reset();
-
-    // let's add a lot of constant rows to test the rle
-    batch.size = 1000;
-    for(int c=0; c < batch.cols.length; ++c) {
-      batch.cols[c].setRepeating(true);
-    }
-    ((UnionColumnVector) batch.cols[1]).fields[0].isRepeating = true;
-    setUnion(batch, 0, null, 0, 1732050807, null, null);
-    for(int i=0; i < 5; ++i) {
-      writer.addRowBatch(batch);
-    }
-
-    batch.reset();
-    batch.size = 3;
-    setUnion(batch, 0, null, 0, 0, null, null);
-    setUnion(batch, 1, null, 0, 10, null, null);
-    setUnion(batch, 2, null, 0, 138, null, null);
-    writer.addRowBatch(batch);
-    writer.close();
-    Reader reader = OrcFile.createReader(testFilePath,
-        OrcFile.readerOptions(conf).filesystem(fs));
-
-    schema = writer.getSchema();
-    assertEquals(5, schema.getMaximumId());
-    boolean[] expected = new boolean[] {false, false, false, false, false, false};
-    boolean[] included = OrcUtils.includeColumns("", schema);
-    assertEquals(true, Arrays.equals(expected, included));
-
-    expected = new boolean[] {false, true, false, false, false, true};
-    included = OrcUtils.includeColumns("time,decimal", schema);
-    assertEquals(true, Arrays.equals(expected, included));
-
-    expected = new boolean[] {false, false, true, true, true, false};
-    included = OrcUtils.includeColumns("union", schema);
-    assertEquals(true, Arrays.equals(expected, included));
-
-    assertEquals(false, reader.getMetadataKeys().iterator().hasNext());
-    assertEquals(5077, reader.getNumberOfRows());
-    DecimalColumnStatistics stats =
-        (DecimalColumnStatistics) reader.getStatistics()[5];
-    assertEquals(71, stats.getNumberOfValues());
-    assertEquals(HiveDecimal.create("-5643.234"), stats.getMinimum());
-    assertEquals(maxValue, stats.getMaximum());
-    // TODO: fix this
-//    assertEquals(null,stats.getSum());
-    int stripeCount = 0;
-    int rowCount = 0;
-    long currentOffset = -1;
-    for(StripeInformation stripe: reader.getStripes()) {
-      stripeCount += 1;
-      rowCount += stripe.getNumberOfRows();
-      if (currentOffset < 0) {
-        currentOffset = stripe.getOffset() + stripe.getLength();
-      } else {
-        assertEquals(currentOffset, stripe.getOffset());
-        currentOffset += stripe.getLength();
-      }
-    }
-    assertEquals(reader.getNumberOfRows(), rowCount);
-    assertEquals(2, stripeCount);
-    assertEquals(reader.getContentLength(), currentOffset);
-    RecordReader rows = reader.rows();
-    assertEquals(0, rows.getRowNumber());
-    assertEquals(0.0, rows.getProgress(), 0.000001);
-
-    schema = reader.getSchema();
-    batch = schema.createRowBatch(74);
-    assertEquals(0, rows.getRowNumber());
-    rows.nextBatch(batch);
-    assertEquals(74, batch.size);
-    assertEquals(74, rows.getRowNumber());
-    TimestampColumnVector ts = (TimestampColumnVector) batch.cols[0];
-    UnionColumnVector union = (UnionColumnVector) batch.cols[1];
-    LongColumnVector longs = (LongColumnVector) union.fields[0];
-    BytesColumnVector strs = (BytesColumnVector) union.fields[1];
-    DecimalColumnVector decs = (DecimalColumnVector) batch.cols[2];
-
-    assertEquals("struct<time:timestamp,union:uniontype<int,string>,decimal:decimal(38,18)>",
-        schema.toString());
-    assertEquals("2000-03-12 15:00:00.0", ts.asScratchTimestamp(0).toString());
-    assertEquals(0, union.tags[0]);
-    assertEquals(42, longs.vector[0]);
-    assertEquals("12345678.6547456", decs.vector[0].toString());
-
-    assertEquals("2000-03-20 12:00:00.123456789", ts.asScratchTimestamp(1).toString());
-    assertEquals(1, union.tags[1]);
-    assertEquals("hello", strs.toString(1));
-    assertEquals("-5643.234", decs.vector[1].toString());
-
-    assertEquals(false, ts.noNulls);
-    assertEquals(false, union.noNulls);
-    assertEquals(false, decs.noNulls);
-    assertEquals(true, ts.isNull[2]);
-    assertEquals(true, union.isNull[2]);
-    assertEquals(true, decs.isNull[2]);
-
-    assertEquals(true, ts.isNull[3]);
-    assertEquals(false, union.isNull[3]);
-    assertEquals(0, union.tags[3]);
-    assertEquals(true, longs.isNull[3]);
-    assertEquals(true, decs.isNull[3]);
-
-    assertEquals(true, ts.isNull[4]);
-    assertEquals(false, union.isNull[4]);
-    assertEquals(1, union.tags[4]);
-    assertEquals(true, strs.isNull[4]);
-    assertEquals(true, decs.isNull[4]);
-
-    assertEquals(false, ts.isNull[5]);
-    assertEquals("1970-01-01 00:00:00.0", ts.asScratchTimestamp(5).toString());
-    assertEquals(false, union.isNull[5]);
-    assertEquals(0, union.tags[5]);
-    assertEquals(false, longs.isNull[5]);
-    assertEquals(200000, longs.vector[5]);
-    assertEquals(false, decs.isNull[5]);
-    assertEquals("10000000000000000000", decs.vector[5].toString());
-
-    rand = new Random(42);
-    for(int i=1970; i < 2038; ++i) {
-      int row = 6 + i - 1970;
-      assertEquals(Timestamp.valueOf(i + "-05-05 12:34:56." + i),
-          ts.asScratchTimestamp(row));
-      if ((i & 1) == 0) {
-        assertEquals(0, union.tags[row]);
-        assertEquals(i*i, longs.vector[row]);
-      } else {
-        assertEquals(1, union.tags[row]);
-        assertEquals(Integer.toString(i * i), strs.toString(row));
-      }
-      assertEquals(new HiveDecimalWritable(HiveDecimal.create(new BigInteger(64, rand),
-                                   rand.nextInt(18))), decs.vector[row]);
-    }
-
-    // rebuild the row batch, so that we can read by 1000 rows
-    batch = schema.createRowBatch(1000);
-    ts = (TimestampColumnVector) batch.cols[0];
-    union = (UnionColumnVector) batch.cols[1];
-    longs = (LongColumnVector) union.fields[0];
-    strs = (BytesColumnVector) union.fields[1];
-    decs = (DecimalColumnVector) batch.cols[2];
-
-    for(int i=0; i < 5; ++i) {
-      rows.nextBatch(batch);
-      assertEquals("batch " + i, 1000, batch.size);
-      assertEquals("batch " + i, false, union.isRepeating);
-      assertEquals("batch " + i, true, union.noNulls);
-      for(int r=0; r < batch.size; ++r) {
-        assertEquals("bad tag at " + i + "." +r, 0, union.tags[r]);
-      }
-      assertEquals("batch " + i, true, longs.isRepeating);
-      assertEquals("batch " + i, 1732050807, longs.vector[0]);
-    }
-
-    rows.nextBatch(batch);
-    assertEquals(3, batch.size);
-    assertEquals(0, union.tags[0]);
-    assertEquals(0, longs.vector[0]);
-    assertEquals(0, union.tags[1]);
-    assertEquals(10, longs.vector[1]);
-    assertEquals(0, union.tags[2]);
-    assertEquals(138, longs.vector[2]);
-
-    rows.nextBatch(batch);
-    assertEquals(0, batch.size);
-    assertEquals(1.0, rows.getProgress(), 0.00001);
-    assertEquals(reader.getNumberOfRows(), rows.getRowNumber());
-    rows.seekToRow(1);
-    rows.nextBatch(batch);
-    assertEquals(1000, batch.size);
-    assertEquals(Timestamp.valueOf("2000-03-20 12:00:00.123456789"), ts.asScratchTimestamp(0));
-    assertEquals(1, union.tags[0]);
-    assertEquals("hello", strs.toString(0));
-    assertEquals(new HiveDecimalWritable(HiveDecimal.create("-5643.234")), decs.vector[0]);
-    rows.close();
-  }
-
-  /**
-   * Read and write a randomly generated snappy file.
-   * @throws Exception
-   */
-  @Test
-  public void testSnappy() throws Exception {
-    TypeDescription schema = createInnerSchema();
-    Writer writer = OrcFile.createWriter(testFilePath,
-                                         OrcFile.writerOptions(conf)
-                                         .setSchema(schema)
-                                         .stripeSize(1000)
-                                         .compress(CompressionKind.SNAPPY)
-                                         .bufferSize(100));
-    VectorizedRowBatch batch = schema.createRowBatch();
-    Random rand = new Random(12);
-    batch.size = 1000;
-    for(int b=0; b < 10; ++b) {
-      for (int r=0; r < 1000; ++r) {
-        ((LongColumnVector) batch.cols[0]).vector[r] = rand.nextInt();
-        ((BytesColumnVector) batch.cols[1]).setVal(r,
-            Integer.toHexString(rand.nextInt()).getBytes());
-      }
-      writer.addRowBatch(batch);
-    }
-    writer.close();
-    Reader reader = OrcFile.createReader(testFilePath,
-        OrcFile.readerOptions(conf).filesystem(fs));
-    assertEquals(CompressionKind.SNAPPY, reader.getCompressionKind());
-    RecordReader rows = reader.rows();
-    batch = reader.getSchema().createRowBatch(1000);
-    rand = new Random(12);
-    LongColumnVector longs = (LongColumnVector) batch.cols[0];
-    BytesColumnVector strs = (BytesColumnVector) batch.cols[1];
-    for(int b=0; b < 10; ++b) {
-      rows.nextBatch(batch);
-      assertEquals(1000, batch.size);
-      for(int r=0; r < batch.size; ++r) {
-        assertEquals(rand.nextInt(), longs.vector[r]);
-        assertEquals(Integer.toHexString(rand.nextInt()), strs.toString(r));
-      }
-    }
-    rows.nextBatch(batch);
-    assertEquals(0, batch.size);
-    rows.close();
-  }
-
-  /**
-   * Read and write a randomly generated snappy file.
-   * @throws Exception
-   */
-  @Test
-  public void testWithoutIndex() throws Exception {
-    TypeDescription schema = createInnerSchema();
-    Writer writer = OrcFile.createWriter(testFilePath,
-                                         OrcFile.writerOptions(conf)
-                                         .setSchema(schema)
-                                         .stripeSize(5000)
-                                         .compress(CompressionKind.SNAPPY)
-                                         .bufferSize(1000)
-                                         .rowIndexStride(0));
-    VectorizedRowBatch batch = schema.createRowBatch();
-    Random rand = new Random(24);
-    batch.size = 5;
-    for(int c=0; c < batch.cols.length; ++c) {
-      batch.cols[c].setRepeating(true);
-    }
-    for(int i=0; i < 10000; ++i) {
-      ((LongColumnVector) batch.cols[0]).vector[0] = rand.nextInt();
-      ((BytesColumnVector) batch.cols[1])
-          .setVal(0, Integer.toBinaryString(rand.nextInt()).getBytes());
-      writer.addRowBatch(batch);
-    }
-    writer.close();
-    Reader reader = OrcFile.createReader(testFilePath,
-        OrcFile.readerOptions(conf).filesystem(fs));
-    assertEquals(50000, reader.getNumberOfRows());
-    assertEquals(0, reader.getRowIndexStride());
-    StripeInformation stripe = reader.getStripes().iterator().next();
-    assertEquals(true, stripe.getDataLength() != 0);
-    assertEquals(0, stripe.getIndexLength());
-    RecordReader rows = reader.rows();
-    rand = new Random(24);
-    batch = reader.getSchema().createRowBatch(1000);
-    LongColumnVector longs = (LongColumnVector) batch.cols[0];
-    BytesColumnVector strs = (BytesColumnVector) batch.cols[1];
-    for(int i=0; i < 50; ++i) {
-      rows.nextBatch(batch);
-      assertEquals("batch " + i, 1000, batch.size);
-      for(int j=0; j < 200; ++j) {
-        int intVal = rand.nextInt();
-        String strVal = Integer.toBinaryString(rand.nextInt());
-        for (int k = 0; k < 5; ++k) {
-          assertEquals(intVal, longs.vector[j * 5 + k]);
-          assertEquals(strVal, strs.toString(j * 5 + k));
-        }
-      }
-    }
-    rows.nextBatch(batch);
-    assertEquals(0, batch.size);
-    rows.close();
-  }
-
-  @Test
-  public void testSeek() throws Exception {
-    TypeDescription schema = createBigRowSchema();
-    Writer writer = OrcFile.createWriter(testFilePath,
-                                         OrcFile.writerOptions(conf)
-                                         .setSchema(schema)
-                                         .stripeSize(200000)
-                                         .bufferSize(65536)
-                                         .rowIndexStride(1000));
-    VectorizedRowBatch batch = schema.createRowBatch();
-    Random rand = new Random(42);
-    final int COUNT=32768;
-    long[] intValues= new long[COUNT];
-    double[] doubleValues = new double[COUNT];
-    String[] stringValues = new String[COUNT];
-    BytesWritable[] byteValues = new BytesWritable[COUNT];
-    String[] words = new String[128];
-    for(int i=0; i < words.length; ++i) {
-      words[i] = Integer.toHexString(rand.nextInt());
-    }
-    for(int i=0; i < COUNT/2; ++i) {
-      intValues[2*i] = rand.nextLong();
-      intValues[2*i+1] = intValues[2*i];
-      stringValues[2*i] = words[rand.nextInt(words.length)];
-      stringValues[2*i+1] = stringValues[2*i];
-    }
-    for(int i=0; i < COUNT; ++i) {
-      doubleValues[i] = rand.nextDouble();
-      byte[] buf = new byte[20];
-      rand.nextBytes(buf);
-      byteValues[i] = new BytesWritable(buf);
-    }
-    for(int i=0; i < COUNT; ++i) {
-      appendRandomRow(batch, intValues, doubleValues, stringValues,
-          byteValues, words, i);
-      if (batch.size == 1024) {
-        writer.addRowBatch(batch);
-        batch.reset();
-      }
-    }
-    if (batch.size != 0) {
-      writer.addRowBatch(batch);
-    }
-    writer.close();
-    Reader reader = OrcFile.createReader(testFilePath,
-        OrcFile.readerOptions(conf).filesystem(fs));
-    assertEquals(COUNT, reader.getNumberOfRows());
-    RecordReader rows = reader.rows();
-    // get the row index
-    DataReader meta = RecordReaderUtils.createDefaultDataReader(
-        DataReaderProperties.builder()
-            .withBufferSize(reader.getCompressionSize())
-            .withFileSystem(fs)
-            .withPath(testFilePath)
-            .withCompression(reader.getCompressionKind())
-            .withTypeCount(reader.getSchema().getMaximumId() + 1)
-            .withZeroCopy(false)
-            .build());
-    OrcIndex index =
-        meta.readRowIndex(reader.getStripes().get(0), null, null, null, null,
-            null);
-    // check the primitive columns to make sure they have the right number of
-    // items in the first row group
-    for(int c=1; c < 9; ++c) {
-      OrcProto.RowIndex colIndex = index.getRowGroupIndex()[c];
-      assertEquals(1000,
-          colIndex.getEntry(0).getStatistics().getNumberOfValues());
-    }
-    batch = reader.getSchema().createRowBatch();
-    int nextRowInBatch = -1;
-    for(int i=COUNT-1; i >= 0; --i, --nextRowInBatch) {
-      // if we have consumed the previous batch read a new one
-      if (nextRowInBatch < 0) {
-        long base = Math.max(i - 1023, 0);
-        rows.seekToRow(base);
-        assertEquals("row " + i, true, rows.nextBatch(batch));
-        nextRowInBatch = batch.size - 1;
-      }
-      checkRandomRow(batch, intValues, doubleValues,
-          stringValues, byteValues, words, i, nextRowInBatch);
-    }
-    rows.close();
-    Iterator<StripeInformation> stripeIterator =
-      reader.getStripes().iterator();
-    long offsetOfStripe2 = 0;
-    long offsetOfStripe4 = 0;
-    long lastRowOfStripe2 = 0;
-    for(int i = 0; i < 5; ++i) {
-      StripeInformation stripe = stripeIterator.next();
-      if (i < 2) {
-        lastRowOfStripe2 += stripe.getNumberOfRows();
-      } else if (i == 2) {
-        offsetOfStripe2 = stripe.getOffset();
-        lastRowOfStripe2 += stripe.getNumberOfRows() - 1;
-      } else if (i == 4) {
-        offsetOfStripe4 = stripe.getOffset();
-      }
-    }
-    boolean[] columns = new boolean[reader.getStatistics().length];
-    columns[5] = true; // long colulmn
-    columns[9] = true; // text column
-    rows = reader.rowsOptions(new Reader.Options()
-        .range(offsetOfStripe2, offsetOfStripe4 - offsetOfStripe2)
-        .include(columns));
-    rows.seekToRow(lastRowOfStripe2);
-    // we only want two rows
-    batch = reader.getSchema().createRowBatch(2);
-    assertEquals(true, rows.nextBatch(batch));
-    assertEquals(1, batch.size);
-    assertEquals(intValues[(int) lastRowOfStripe2], getLong(batch, 0));
-    assertEquals(stringValues[(int) lastRowOfStripe2],
-        getText(batch, 0).toString());
-    assertEquals(true, rows.nextBatch(batch));
-    assertEquals(intValues[(int) lastRowOfStripe2 + 1], getLong(batch, 0));
-    assertEquals(stringValues[(int) lastRowOfStripe2 + 1],
-        getText(batch, 0).toString());
-    rows.close();
-  }
-
-  private void appendRandomRow(VectorizedRowBatch batch,
-                               long[] intValues, double[] doubleValues,
-                               String[] stringValues,
-                               BytesWritable[] byteValues,
-                               String[] words, int i) {
-    InnerStruct inner = new InnerStruct((int) intValues[i], stringValues[i]);
-    InnerStruct inner2 = new InnerStruct((int) (intValues[i] >> 32),
-        words[i % words.length] + "-x");
-    setBigRow(batch, batch.size++, (intValues[i] & 1) == 0, (byte) intValues[i],
-        (short) intValues[i], (int) intValues[i], intValues[i],
-        (float) doubleValues[i], doubleValues[i], byteValues[i], stringValues[i],
-        new MiddleStruct(inner, inner2), list(), map(inner, inner2));
-  }
-
-  private void checkRandomRow(VectorizedRowBatch batch,
-                              long[] intValues, double[] doubleValues,
-                              String[] stringValues,
-                              BytesWritable[] byteValues,
-                              String[] words, int i, int rowInBatch) {
-    InnerStruct inner = new InnerStruct((int) intValues[i], stringValues[i]);
-    InnerStruct inner2 = new InnerStruct((int) (intValues[i] >> 32),
-        words[i % words.length] + "-x");
-    checkBigRow(batch, rowInBatch, i, (intValues[i] & 1) == 0, (byte) intValues[i],
-        (short) intValues[i], (int) intValues[i], intValues[i],
-        (float) doubleValues[i], doubleValues[i], byteValues[i], stringValues[i],
-        new MiddleStruct(inner, inner2), list(), map(inner, inner2));
-  }
-
-  private static class MyMemoryManager extends MemoryManager {
-    final long totalSpace;
-    double rate;
-    Path path = null;
-    long lastAllocation = 0;
-    int rows = 0;
-    Callback callback;
-
-    MyMemoryManager(Configuration conf, long totalSpace, double rate) {
-      super(conf);
-      this.totalSpace = totalSpace;
-      this.rate = rate;
-    }
-
-    @Override
-    public void addWriter(Path path, long requestedAllocation,
-                   Callback callback) {
-      this.path = path;
-      this.lastAllocation = requestedAllocation;
-      this.callback = callback;
-    }
-
-    @Override
-    public synchronized void removeWriter(Path path) {
-      this.path = null;
-      this.lastAllocation = 0;
-    }
-
-    @Override
-    public long getTotalMemoryPool() {
-      return totalSpace;
-    }
-
-    @Override
-    public double getAllocationScale() {
-      return rate;
-    }
-
-    @Override
-    public void addedRow(int count) throws IOException {
-      rows += count;
-      if (rows % 100 == 0) {
-        callback.checkMemory(rate);
-      }
-    }
-  }
-
-  @Test
-  public void testMemoryManagementV11() throws Exception {
-    TypeDescription schema = createInnerSchema();
-    MyMemoryManager memory = new MyMemoryManager(conf, 10000, 0.1);
-    Writer writer = OrcFile.createWriter(testFilePath,
-        OrcFile.writerOptions(conf)
-            .setSchema(schema)
-            .compress(CompressionKind.NONE)
-            .stripeSize(50000)
-            .bufferSize(100)
-            .rowIndexStride(0)
-            .memory(memory)
-            .version(OrcFile.Version.V_0_11));
-    assertEquals(testFilePath, memory.path);
-    VectorizedRowBatch batch = schema.createRowBatch();
-    batch.size = 1;
-    for(int i=0; i < 2500; ++i) {
-      ((LongColumnVector) batch.cols[0]).vector[0] = i * 300;
-      ((BytesColumnVector) batch.cols[1]).setVal(0,
-          Integer.toHexString(10*i).getBytes());
-      writer.addRowBatch(batch);
-    }
-    writer.close();
-    assertEquals(null, memory.path);
-    Reader reader = OrcFile.createReader(testFilePath,
-        OrcFile.readerOptions(conf).filesystem(fs));
-    int i = 0;
-    for(StripeInformation stripe: reader.getStripes()) {
-      i += 1;
-      assertTrue("stripe " + i + " is too long at " + stripe.getDataLength(),
-          stripe.getDataLength() < 5000);
-    }
-    assertEquals(25, i);
-    assertEquals(2500, reader.getNumberOfRows());
-  }
-
-  @Test
-  public void testMemoryManagementV12() throws Exception {
-    TypeDescription schema = createInnerSchema();
-    MyMemoryManager memory = new MyMemoryManager(conf, 10000, 0.1);
-    Writer writer = OrcFile.createWriter(testFilePath,
-                                         OrcFile.writerOptions(conf)
-                                         .setSchema(schema)
-                                         .compress(CompressionKind.NONE)
-                                         .stripeSize(50000)
-                                         .bufferSize(100)
-                                         .rowIndexStride(0)
-                                         .memory(memory)
-                                         .version(OrcFile.Version.V_0_12));
-    VectorizedRowBatch batch = schema.createRowBatch();
-    assertEquals(testFilePath, memory.path);
-    batch.size = 1;
-    for(int i=0; i < 2500; ++i) {
-      ((LongColumnVector) batch.cols[0]).vector[0] = i * 300;
-      ((BytesColumnVector) batch.cols[1]).setVal(0,
-          Integer.toHexString(10*i).getBytes());
-      writer.addRowBatch(batch);
-    }
-    writer.close();
-    assertEquals(null, memory.path);
-    Reader reader = OrcFile.createReader(testFilePath,
-        OrcFile.readerOptions(conf).filesystem(fs));
-    int i = 0;
-    for(StripeInformation stripe: reader.getStripes()) {
-      i += 1;
-      assertTrue("stripe " + i + " is too long at " + stripe.getDataLength(),
-          stripe.getDataLength() < 5000);
-    }
-    // with HIVE-7832, the dictionaries will be disabled after writing the first
-    // stripe as there are too many distinct values. Hence only 3 stripes as
-    // compared to 25 stripes in version 0.11 (above test case)
-    assertEquals(3, i);
-    assertEquals(2500, reader.getNumberOfRows());
-  }
-
-  @Test
-  public void testPredicatePushdown() throws Exception {
-    TypeDescription schema = createInnerSchema();
-    Writer writer = OrcFile.createWriter(testFilePath,
-        OrcFile.writerOptions(conf)
-            .setSchema(schema)
-            .stripeSize(400000L)
-            .compress(CompressionKind.NONE)
-            .bufferSize(500)
-            .rowIndexStride(1000));
-    VectorizedRowBatch batch = schema.createRowBatch();
-    batch.ensureSize(3500);
-    batch.size = 3500;
-    for(int i=0; i < 3500; ++i) {
-      ((LongColumnVector) batch.cols[0]).vector[i] = i * 300;
-      ((BytesColumnVector) batch.cols[1]).setVal(i,
-          Integer.toHexString(10*i).getBytes());
-    }
-    writer.addRowBatch(batch);
-    writer.close();
-    Reader reader = OrcFile.createReader(testFilePath,
-        OrcFile.readerOptions(conf).filesystem(fs));
-    assertEquals(3500, reader.getNumberOfRows());
-
-    SearchArgument sarg = SearchArgumentFactory.newBuilder()
-        .startAnd()
-          .startNot()
-             .lessThan("int1", PredicateLeaf.Type.LONG, 300000L)
-          .end()
-          .lessThan("int1", PredicateLeaf.Type.LONG, 600000L)
-        .end()
-        .build();
-    RecordReader rows = reader.rowsOptions(new Reader.Options()
-        .range(0L, Long.MAX_VALUE)
-        .include(new boolean[]{true, true, true})
-        .searchArgument(sarg, new String[]{null, "int1", "string1"}));
-    batch = reader.getSchema().createRowBatch(2000);
-    LongColumnVector ints = (LongColumnVector) batch.cols[0];
-    BytesColumnVector strs = (BytesColumnVector) batch.cols[1];
-
-    assertEquals(1000L, rows.getRowNumber());
-    assertEquals(true, rows.nextBatch(batch));
-    assertEquals(1000, batch.size);
-
-    for(int i=1000; i < 2000; ++i) {
-      assertEquals(300 * i, ints.vector[i - 1000]);
-      assertEquals(Integer.toHexString(10*i), strs.toString(i - 1000));
-    }
-    assertEquals(false, rows.nextBatch(batch));
-    assertEquals(3500, rows.getRowNumber());
-
-    // look through the file with no rows selected
-    sarg = SearchArgumentFactory.newBuilder()
-        .startAnd()
-          .lessThan("int1", PredicateLeaf.Type.LONG, 0L)
-        .end()
-        .build();
-    rows = reader.rowsOptions(new Reader.Options()
-        .range(0L, Long.MAX_VALUE)
-        .include(new boolean[]{true, true, true})
-        .searchArgument(sarg, new String[]{null, "int1", "string1"}));
-    assertEquals(3500L, rows.getRowNumber());
-    assertTrue(!rows.hasNext());
-
-    // select first 100 and last 100 rows
-    sarg = SearchArgumentFactory.newBuilder()
-        .startOr()
-          .lessThan("int1", PredicateLeaf.Type.LONG, 300L * 100)
-          .startNot()
-            .lessThan("int1", PredicateLeaf.Type.LONG, 300L * 3400)
-          .end()
-        .end()
-        .build();
-    rows = reader.rowsOptions(new Reader.Options()
-        .range(0L, Long.MAX_VALUE)
-        .include(new boolean[]{true, true, true})
-        .searchArgument(sarg, new String[]{null, "int1", "string1"}));
-    assertEquals(0, rows.getRowNumber());
-    assertEquals(true, rows.nextBatch(batch));
-    assertEquals(1000, batch.size);
-    assertEquals(3000, rows.getRowNumber());
-    for(int i=0; i < 1000; ++i) {
-      assertEquals(300 * i, ints.vector[i]);
-      assertEquals(Integer.toHexString(10*i), strs.toString(i));
-    }
-
-    assertEquals(true, rows.nextBatch(batch));
-    assertEquals(500, batch.size);
-    assertEquals(3500, rows.getRowNumber());
-    for(int i=3000; i < 3500; ++i) {
-      assertEquals(300 * i, ints.vector[i - 3000]);
-      assertEquals(Integer.toHexString(10*i), strs.toString(i - 3000));
-    }
-    assertEquals(false, rows.nextBatch(batch));
-    assertEquals(3500, rows.getRowNumber());
-  }
-
-  /**
-   * Test all of the types that have distinct ORC writers using the vectorized
-   * writer with different combinations of repeating and null values.
-   

<TRUNCATED>

[19/27] hive git commit: HIVE-11417. Move the ReaderImpl and RowReaderImpl to the ORC module, by making shims for the row by row reader. (omalley reviewed by prasanth_j)

Posted by om...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/orc/src/test/org/apache/orc/TestVectorOrcFile.java
----------------------------------------------------------------------
diff --git a/orc/src/test/org/apache/orc/TestVectorOrcFile.java b/orc/src/test/org/apache/orc/TestVectorOrcFile.java
new file mode 100644
index 0000000..112edb9
--- /dev/null
+++ b/orc/src/test/org/apache/orc/TestVectorOrcFile.java
@@ -0,0 +1,2782 @@
+/**
+ * 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.orc;
+
+import com.google.common.collect.Lists;
+
+import junit.framework.Assert;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.common.type.HiveDecimal;
+import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.ListColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.MapColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.StructColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.UnionColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgumentFactory;
+import org.apache.hadoop.hive.serde2.io.DateWritable;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.orc.impl.DataReaderProperties;
+import org.apache.orc.impl.MemoryManager;
+import org.apache.orc.impl.OrcIndex;
+import org.apache.orc.impl.RecordReaderImpl;
+import org.apache.orc.impl.RecordReaderUtils;
+import org.apache.orc.tools.TestJsonFileDump;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+
+import java.io.File;
+import java.io.IOException;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+
+import static junit.framework.TestCase.assertNotNull;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Tests for the vectorized reader and writer for ORC files.
+ */
+public class TestVectorOrcFile {
+
+  public static class InnerStruct {
+    int int1;
+    Text string1 = new Text();
+    InnerStruct(int int1, Text string1) {
+      this.int1 = int1;
+      this.string1.set(string1);
+    }
+    InnerStruct(int int1, String string1) {
+      this.int1 = int1;
+      this.string1.set(string1);
+    }
+
+    public String toString() {
+      return "{" + int1 + ", " + string1 + "}";
+    }
+  }
+
+  public static class MiddleStruct {
+    List<InnerStruct> list = new ArrayList<InnerStruct>();
+
+    MiddleStruct(InnerStruct... items) {
+      list.clear();
+      list.addAll(Arrays.asList(items));
+    }
+  }
+
+  private static InnerStruct inner(int i, String s) {
+    return new InnerStruct(i, s);
+  }
+
+  private static Map<String, InnerStruct> map(InnerStruct... items)  {
+    Map<String, InnerStruct> result = new HashMap<String, InnerStruct>();
+    for(InnerStruct i: items) {
+      result.put(i.string1.toString(), i);
+    }
+    return result;
+  }
+
+  private static List<InnerStruct> list(InnerStruct... items) {
+    List<InnerStruct> result = new ArrayList<InnerStruct>();
+    result.addAll(Arrays.asList(items));
+    return result;
+  }
+
+  private static BytesWritable bytes(int... items) {
+    BytesWritable result = new BytesWritable();
+    result.setSize(items.length);
+    for(int i=0; i < items.length; ++i) {
+      result.getBytes()[i] = (byte) items[i];
+    }
+    return result;
+  }
+
+  private static byte[] bytesArray(int... items) {
+    byte[] result = new byte[items.length];
+    for(int i=0; i < items.length; ++i) {
+      result[i] = (byte) items[i];
+    }
+    return result;
+  }
+
+  private static ByteBuffer byteBuf(int... items) {
+    ByteBuffer result = ByteBuffer.allocate(items.length);
+    for(int item: items) {
+      result.put((byte) item);
+    }
+    result.flip();
+    return result;
+  }
+
+  Path workDir = new Path(System.getProperty("test.tmp.dir",
+      "target" + File.separator + "test" + File.separator + "tmp"));
+
+  Configuration conf;
+  FileSystem fs;
+  Path testFilePath;
+
+  @Rule
+  public TestName testCaseName = new TestName();
+
+  @Before
+  public void openFileSystem () throws Exception {
+    conf = new Configuration();
+    fs = FileSystem.getLocal(conf);
+    testFilePath = new Path(workDir, "TestVectorOrcFile." +
+        testCaseName.getMethodName() + ".orc");
+    fs.delete(testFilePath, false);
+  }
+
+  @Test
+  public void testReadFormat_0_11() throws Exception {
+    Path oldFilePath =
+        new Path(TestJsonFileDump.getFileFromClasspath("orc-file-11-format.orc"));
+    Reader reader = OrcFile.createReader(oldFilePath,
+        OrcFile.readerOptions(conf).filesystem(fs));
+
+    int stripeCount = 0;
+    int rowCount = 0;
+    long currentOffset = -1;
+    for(StripeInformation stripe : reader.getStripes()) {
+      stripeCount += 1;
+      rowCount += stripe.getNumberOfRows();
+      if (currentOffset < 0) {
+        currentOffset = stripe.getOffset() + stripe.getIndexLength()
+            + stripe.getDataLength() + stripe.getFooterLength();
+      } else {
+        assertEquals(currentOffset, stripe.getOffset());
+        currentOffset += stripe.getIndexLength() + stripe.getDataLength()
+            + stripe.getFooterLength();
+      }
+    }
+    Assert.assertEquals(reader.getNumberOfRows(), rowCount);
+    assertEquals(2, stripeCount);
+
+    // check the stats
+    ColumnStatistics[] stats = reader.getStatistics();
+    assertEquals(7500, stats[1].getNumberOfValues());
+    assertEquals(3750, ((BooleanColumnStatistics) stats[1]).getFalseCount());
+    assertEquals(3750, ((BooleanColumnStatistics) stats[1]).getTrueCount());
+    assertEquals("count: 7500 hasNull: true true: 3750", stats[1].toString());
+
+    assertEquals(2048, ((IntegerColumnStatistics) stats[3]).getMaximum());
+    assertEquals(1024, ((IntegerColumnStatistics) stats[3]).getMinimum());
+    assertEquals(true, ((IntegerColumnStatistics) stats[3]).isSumDefined());
+    assertEquals(11520000, ((IntegerColumnStatistics) stats[3]).getSum());
+    assertEquals("count: 7500 hasNull: true min: 1024 max: 2048 sum: 11520000",
+        stats[3].toString());
+
+    assertEquals(Long.MAX_VALUE,
+        ((IntegerColumnStatistics) stats[5]).getMaximum());
+    assertEquals(Long.MAX_VALUE,
+        ((IntegerColumnStatistics) stats[5]).getMinimum());
+    assertEquals(false, ((IntegerColumnStatistics) stats[5]).isSumDefined());
+    assertEquals(
+        "count: 7500 hasNull: true min: 9223372036854775807 max: 9223372036854775807",
+        stats[5].toString());
+
+    assertEquals(-15.0, ((DoubleColumnStatistics) stats[7]).getMinimum(), 0.0001);
+    assertEquals(-5.0, ((DoubleColumnStatistics) stats[7]).getMaximum(), 0.0001);
+    assertEquals(-75000.0, ((DoubleColumnStatistics) stats[7]).getSum(),
+        0.00001);
+    assertEquals("count: 7500 hasNull: true min: -15.0 max: -5.0 sum: -75000.0",
+        stats[7].toString());
+
+    assertEquals("count: 7500 hasNull: true min: bye max: hi sum: 0", stats[9].toString());
+
+    // check the inspectors
+    TypeDescription schema = reader.getSchema();
+    assertEquals(TypeDescription.Category.STRUCT, schema.getCategory());
+    assertEquals("struct<boolean1:boolean,byte1:tinyint,short1:smallint,"
+        + "int1:int,long1:bigint,float1:float,double1:double,bytes1:"
+        + "binary,string1:string,middle:struct<list:array<struct<int1:int,"
+        + "string1:string>>>,list:array<struct<int1:int,string1:string>>,"
+        + "map:map<string,struct<int1:int,string1:string>>,ts:timestamp,"
+        + "decimal1:decimal(38,10)>", schema.toString());
+    VectorizedRowBatch batch = schema.createRowBatch();
+
+    RecordReader rows = reader.rows();
+    Assert.assertEquals(true, rows.nextBatch(batch));
+    assertEquals(1024, batch.size);
+
+    // check the contents of the first row
+    assertEquals(false, getBoolean(batch, 0));
+    assertEquals(1, getByte(batch, 0));
+    assertEquals(1024, getShort(batch, 0));
+    assertEquals(65536, getInt(batch, 0));
+    assertEquals(Long.MAX_VALUE, getLong(batch, 0));
+    assertEquals(1.0, getFloat(batch, 0), 0.00001);
+    assertEquals(-15.0, getDouble(batch, 0), 0.00001);
+    assertEquals(bytes(0, 1, 2, 3, 4), getBinary(batch, 0));
+    assertEquals("hi", getText(batch, 0).toString());
+
+    StructColumnVector middle = (StructColumnVector) batch.cols[9];
+    ListColumnVector midList = (ListColumnVector) middle.fields[0];
+    StructColumnVector midListStruct = (StructColumnVector) midList.child;
+    LongColumnVector midListInt = (LongColumnVector) midListStruct.fields[0];
+    BytesColumnVector midListStr = (BytesColumnVector) midListStruct.fields[1];
+    ListColumnVector list = (ListColumnVector) batch.cols[10];
+    StructColumnVector listStruct = (StructColumnVector) list.child;
+    LongColumnVector listInts = (LongColumnVector) listStruct.fields[0];
+    BytesColumnVector listStrs = (BytesColumnVector) listStruct.fields[1];
+    MapColumnVector map = (MapColumnVector) batch.cols[11];
+    BytesColumnVector mapKey = (BytesColumnVector) map.keys;
+    StructColumnVector mapValue = (StructColumnVector) map.values;
+    LongColumnVector mapValueInts = (LongColumnVector) mapValue.fields[0];
+    BytesColumnVector mapValueStrs = (BytesColumnVector) mapValue.fields[1];
+    TimestampColumnVector timestamp = (TimestampColumnVector) batch.cols[12];
+    DecimalColumnVector decs = (DecimalColumnVector) batch.cols[13];
+
+    assertEquals(false, middle.isNull[0]);
+    assertEquals(2, midList.lengths[0]);
+    int start = (int) midList.offsets[0];
+    assertEquals(1, midListInt.vector[start]);
+    assertEquals("bye", midListStr.toString(start));
+    assertEquals(2, midListInt.vector[start + 1]);
+    assertEquals("sigh", midListStr.toString(start + 1));
+
+    assertEquals(2, list.lengths[0]);
+    start = (int) list.offsets[0];
+    assertEquals(3, listInts.vector[start]);
+    assertEquals("good", listStrs.toString(start));
+    assertEquals(4, listInts.vector[start + 1]);
+    assertEquals("bad", listStrs.toString(start + 1));
+    assertEquals(0, map.lengths[0]);
+    assertEquals(Timestamp.valueOf("2000-03-12 15:00:00"),
+        timestamp.asScratchTimestamp(0));
+    assertEquals(new HiveDecimalWritable(HiveDecimal.create("12345678.6547456")),
+        decs.vector[0]);
+
+    // check the contents of row 7499
+    rows.seekToRow(7499);
+    Assert.assertEquals(true, rows.nextBatch(batch));
+    assertEquals(true, getBoolean(batch, 0));
+    assertEquals(100, getByte(batch, 0));
+    assertEquals(2048, getShort(batch, 0));
+    assertEquals(65536, getInt(batch, 0));
+    assertEquals(Long.MAX_VALUE, getLong(batch, 0));
+    assertEquals(2.0, getFloat(batch, 0), 0.00001);
+    assertEquals(-5.0, getDouble(batch, 0), 0.00001);
+    assertEquals(bytes(), getBinary(batch, 0));
+    assertEquals("bye", getText(batch, 0).toString());
+    assertEquals(false, middle.isNull[0]);
+    assertEquals(2, midList.lengths[0]);
+    start = (int) midList.offsets[0];
+    assertEquals(1, midListInt.vector[start]);
+    assertEquals("bye", midListStr.toString(start));
+    assertEquals(2, midListInt.vector[start + 1]);
+    assertEquals("sigh", midListStr.toString(start + 1));
+    assertEquals(3, list.lengths[0]);
+    start = (int) list.offsets[0];
+    assertEquals(100000000, listInts.vector[start]);
+    assertEquals("cat", listStrs.toString(start));
+    assertEquals(-100000, listInts.vector[start + 1]);
+    assertEquals("in", listStrs.toString(start + 1));
+    assertEquals(1234, listInts.vector[start + 2]);
+    assertEquals("hat", listStrs.toString(start + 2));
+    assertEquals(2, map.lengths[0]);
+    start = (int) map.offsets[0];
+    assertEquals("chani", mapKey.toString(start));
+    assertEquals(5, mapValueInts.vector[start]);
+    assertEquals("chani", mapValueStrs.toString(start));
+    assertEquals("mauddib", mapKey.toString(start + 1));
+    assertEquals(1, mapValueInts.vector[start + 1]);
+    assertEquals("mauddib", mapValueStrs.toString(start + 1));
+    assertEquals(Timestamp.valueOf("2000-03-12 15:00:01"),
+        timestamp.asScratchTimestamp(0));
+    assertEquals(new HiveDecimalWritable(HiveDecimal.create("12345678.6547457")),
+        decs.vector[0]);
+
+    // handle the close up
+    Assert.assertEquals(false, rows.nextBatch(batch));
+    rows.close();
+  }
+
+  @Test
+  public void testTimestamp() throws Exception {
+    TypeDescription schema = TypeDescription.createTimestamp();
+    Writer writer = OrcFile.createWriter(testFilePath,
+        OrcFile.writerOptions(conf).setSchema(schema).stripeSize(100000)
+            .bufferSize(10000).version(org.apache.orc.OrcFile.Version.V_0_11));
+    List<Timestamp> tslist = Lists.newArrayList();
+    tslist.add(Timestamp.valueOf("2037-01-01 00:00:00.000999"));
+    tslist.add(Timestamp.valueOf("2003-01-01 00:00:00.000000222"));
+    tslist.add(Timestamp.valueOf("1999-01-01 00:00:00.999999999"));
+    tslist.add(Timestamp.valueOf("1995-01-01 00:00:00.688888888"));
+    tslist.add(Timestamp.valueOf("2002-01-01 00:00:00.1"));
+    tslist.add(Timestamp.valueOf("2010-03-02 00:00:00.000009001"));
+    tslist.add(Timestamp.valueOf("2005-01-01 00:00:00.000002229"));
+    tslist.add(Timestamp.valueOf("2006-01-01 00:00:00.900203003"));
+    tslist.add(Timestamp.valueOf("2003-01-01 00:00:00.800000007"));
+    tslist.add(Timestamp.valueOf("1996-08-02 00:00:00.723100809"));
+    tslist.add(Timestamp.valueOf("1998-11-02 00:00:00.857340643"));
+    tslist.add(Timestamp.valueOf("2008-10-02 00:00:00"));
+
+    VectorizedRowBatch batch = new VectorizedRowBatch(1, 1024);
+    TimestampColumnVector vec = new TimestampColumnVector(1024);
+    batch.cols[0] = vec;
+    batch.reset();
+    batch.size = tslist.size();
+    for (int i=0; i < tslist.size(); ++i) {
+      Timestamp ts = tslist.get(i);
+      vec.set(i, ts);
+    }
+    writer.addRowBatch(batch);
+    writer.close();
+
+    Reader reader = OrcFile.createReader(testFilePath,
+        OrcFile.readerOptions(conf).filesystem(fs));
+    RecordReader rows = reader.rows();
+    batch = reader.getSchema().createRowBatch();
+    TimestampColumnVector timestamps = (TimestampColumnVector) batch.cols[0];
+    int idx = 0;
+    while (rows.nextBatch(batch)) {
+      for(int r=0; r < batch.size; ++r) {
+        assertEquals(tslist.get(idx++).getNanos(),
+            timestamps.asScratchTimestamp(r).getNanos());
+      }
+    }
+    Assert.assertEquals(tslist.size(), rows.getRowNumber());
+    assertEquals(0, writer.getSchema().getMaximumId());
+    boolean[] expected = new boolean[] {false};
+    boolean[] included = OrcUtils.includeColumns("", writer.getSchema());
+    assertEquals(true, Arrays.equals(expected, included));
+  }
+
+  @Test
+  public void testStringAndBinaryStatistics() throws Exception {
+
+    TypeDescription schema = TypeDescription.createStruct()
+        .addField("bytes1", TypeDescription.createBinary())
+        .addField("string1", TypeDescription.createString());
+    Writer writer = OrcFile.createWriter(testFilePath,
+                                         OrcFile.writerOptions(conf)
+                                         .setSchema(schema)
+                                         .stripeSize(100000)
+                                         .bufferSize(10000));
+    VectorizedRowBatch batch = schema.createRowBatch();
+    batch.size = 4;
+    BytesColumnVector field1 = (BytesColumnVector) batch.cols[0];
+    BytesColumnVector field2 = (BytesColumnVector) batch.cols[1];
+    field1.setVal(0, bytesArray(0, 1, 2, 3, 4));
+    field1.setVal(1, bytesArray(0, 1, 2, 3));
+    field1.setVal(2, bytesArray(0, 1, 2, 3, 4, 5));
+    field1.noNulls = false;
+    field1.isNull[3] = true;
+    field2.setVal(0, "foo".getBytes());
+    field2.setVal(1, "bar".getBytes());
+    field2.noNulls = false;
+    field2.isNull[2] = true;
+    field2.setVal(3, "hi".getBytes());
+    writer.addRowBatch(batch);
+    writer.close();
+    schema = writer.getSchema();
+    assertEquals(2, schema.getMaximumId());
+
+    Reader reader = OrcFile.createReader(testFilePath,
+        OrcFile.readerOptions(conf).filesystem(fs));
+
+    boolean[] expected = new boolean[] {false, false, true};
+    boolean[] included = OrcUtils.includeColumns("string1", schema);
+    assertEquals(true, Arrays.equals(expected, included));
+
+    expected = new boolean[] {false, false, false};
+    included = OrcUtils.includeColumns("", schema);
+    assertEquals(true, Arrays.equals(expected, included));
+
+    expected = new boolean[] {false, false, false};
+    included = OrcUtils.includeColumns(null, schema);
+    assertEquals(true, Arrays.equals(expected, included));
+
+    // check the stats
+    ColumnStatistics[] stats = reader.getStatistics();
+    assertEquals(4, stats[0].getNumberOfValues());
+    assertEquals("count: 4 hasNull: false", stats[0].toString());
+
+    assertEquals(3, stats[1].getNumberOfValues());
+    assertEquals(15, ((BinaryColumnStatistics) stats[1]).getSum());
+    assertEquals("count: 3 hasNull: true sum: 15", stats[1].toString());
+
+    assertEquals(3, stats[2].getNumberOfValues());
+    assertEquals("bar", ((StringColumnStatistics) stats[2]).getMinimum());
+    assertEquals("hi", ((StringColumnStatistics) stats[2]).getMaximum());
+    assertEquals(8, ((StringColumnStatistics) stats[2]).getSum());
+    assertEquals("count: 3 hasNull: true min: bar max: hi sum: 8",
+        stats[2].toString());
+
+    // check the inspectors
+    batch = reader.getSchema().createRowBatch();
+    BytesColumnVector bytes = (BytesColumnVector) batch.cols[0];
+    BytesColumnVector strs = (BytesColumnVector) batch.cols[1];
+    RecordReader rows = reader.rows();
+    Assert.assertEquals(true, rows.nextBatch(batch));
+    assertEquals(4, batch.size);
+
+    // check the contents of the first row
+    assertEquals(bytes(0,1,2,3,4), getBinary(bytes, 0));
+    assertEquals("foo", strs.toString(0));
+
+    // check the contents of second row
+    assertEquals(bytes(0,1,2,3), getBinary(bytes, 1));
+    assertEquals("bar", strs.toString(1));
+
+    // check the contents of third row
+    assertEquals(bytes(0,1,2,3,4,5), getBinary(bytes, 2));
+    assertNull(strs.toString(2));
+
+    // check the contents of fourth row
+    assertNull(getBinary(bytes, 3));
+    assertEquals("hi", strs.toString(3));
+
+    // handle the close up
+    Assert.assertEquals(false, rows.nextBatch(batch));
+    rows.close();
+  }
+
+
+  @Test
+  public void testStripeLevelStats() throws Exception {
+    TypeDescription schema = TypeDescription.createStruct()
+        .addField("int1", TypeDescription.createInt())
+        .addField("string1", TypeDescription.createString());
+    Writer writer = OrcFile.createWriter(testFilePath,
+        OrcFile.writerOptions(conf)
+            .setSchema(schema)
+            .stripeSize(100000)
+            .bufferSize(10000));
+    VectorizedRowBatch batch = schema.createRowBatch();
+    batch.size = 1000;
+    LongColumnVector field1 = (LongColumnVector) batch.cols[0];
+    BytesColumnVector field2 = (BytesColumnVector) batch.cols[1];
+    field1.isRepeating = true;
+    field2.isRepeating = true;
+    for (int b = 0; b < 11; b++) {
+      if (b >= 5) {
+        if (b >= 10) {
+          field1.vector[0] = 3;
+          field2.setVal(0, "three".getBytes());
+        } else {
+          field1.vector[0] = 2;
+          field2.setVal(0, "two".getBytes());
+        }
+      } else {
+        field1.vector[0] = 1;
+        field2.setVal(0, "one".getBytes());
+      }
+      writer.addRowBatch(batch);
+    }
+
+    writer.close();
+    Reader reader = OrcFile.createReader(testFilePath,
+        OrcFile.readerOptions(conf).filesystem(fs));
+
+    schema = writer.getSchema();
+    assertEquals(2, schema.getMaximumId());
+    boolean[] expected = new boolean[] {false, true, false};
+    boolean[] included = OrcUtils.includeColumns("int1", schema);
+    assertEquals(true, Arrays.equals(expected, included));
+
+    List<StripeStatistics> stats = reader.getStripeStatistics();
+    int numStripes = stats.size();
+    assertEquals(3, numStripes);
+    StripeStatistics ss1 = stats.get(0);
+    StripeStatistics ss2 = stats.get(1);
+    StripeStatistics ss3 = stats.get(2);
+
+    assertEquals(5000, ss1.getColumnStatistics()[0].getNumberOfValues());
+    assertEquals(5000, ss2.getColumnStatistics()[0].getNumberOfValues());
+    assertEquals(1000, ss3.getColumnStatistics()[0].getNumberOfValues());
+
+    assertEquals(5000, (ss1.getColumnStatistics()[1]).getNumberOfValues());
+    assertEquals(5000, (ss2.getColumnStatistics()[1]).getNumberOfValues());
+    assertEquals(1000, (ss3.getColumnStatistics()[1]).getNumberOfValues());
+    assertEquals(1, ((IntegerColumnStatistics)ss1.getColumnStatistics()[1]).getMinimum());
+    assertEquals(2, ((IntegerColumnStatistics)ss2.getColumnStatistics()[1]).getMinimum());
+    assertEquals(3, ((IntegerColumnStatistics)ss3.getColumnStatistics()[1]).getMinimum());
+    assertEquals(1, ((IntegerColumnStatistics)ss1.getColumnStatistics()[1]).getMaximum());
+    assertEquals(2, ((IntegerColumnStatistics)ss2.getColumnStatistics()[1]).getMaximum());
+    assertEquals(3, ((IntegerColumnStatistics)ss3.getColumnStatistics()[1]).getMaximum());
+    assertEquals(5000, ((IntegerColumnStatistics)ss1.getColumnStatistics()[1]).getSum());
+    assertEquals(10000, ((IntegerColumnStatistics)ss2.getColumnStatistics()[1]).getSum());
+    assertEquals(3000, ((IntegerColumnStatistics)ss3.getColumnStatistics()[1]).getSum());
+
+    assertEquals(5000, (ss1.getColumnStatistics()[2]).getNumberOfValues());
+    assertEquals(5000, (ss2.getColumnStatistics()[2]).getNumberOfValues());
+    assertEquals(1000, (ss3.getColumnStatistics()[2]).getNumberOfValues());
+    assertEquals("one", ((StringColumnStatistics)ss1.getColumnStatistics()[2]).getMinimum());
+    assertEquals("two", ((StringColumnStatistics)ss2.getColumnStatistics()[2]).getMinimum());
+    assertEquals("three", ((StringColumnStatistics)ss3.getColumnStatistics()[2]).getMinimum());
+    assertEquals("one", ((StringColumnStatistics)ss1.getColumnStatistics()[2]).getMaximum());
+    assertEquals("two", ((StringColumnStatistics) ss2.getColumnStatistics()[2]).getMaximum());
+    assertEquals("three", ((StringColumnStatistics)ss3.getColumnStatistics()[2]).getMaximum());
+    assertEquals(15000, ((StringColumnStatistics)ss1.getColumnStatistics()[2]).getSum());
+    assertEquals(15000, ((StringColumnStatistics)ss2.getColumnStatistics()[2]).getSum());
+    assertEquals(5000, ((StringColumnStatistics)ss3.getColumnStatistics()[2]).getSum());
+
+    RecordReaderImpl recordReader = (RecordReaderImpl) reader.rows();
+    OrcProto.RowIndex[] index = recordReader.readRowIndex(0, null, null).getRowGroupIndex();
+    assertEquals(3, index.length);
+    List<OrcProto.RowIndexEntry> items = index[1].getEntryList();
+    assertEquals(1, items.size());
+    assertEquals(3, items.get(0).getPositionsCount());
+    assertEquals(0, items.get(0).getPositions(0));
+    assertEquals(0, items.get(0).getPositions(1));
+    assertEquals(0, items.get(0).getPositions(2));
+    assertEquals(1,
+                 items.get(0).getStatistics().getIntStatistics().getMinimum());
+    index = recordReader.readRowIndex(1, null, null).getRowGroupIndex();
+    assertEquals(3, index.length);
+    items = index[1].getEntryList();
+    assertEquals(2,
+        items.get(0).getStatistics().getIntStatistics().getMaximum());
+  }
+
+  private static void setInner(StructColumnVector inner, int rowId,
+                               int i, String value) {
+    ((LongColumnVector) inner.fields[0]).vector[rowId] = i;
+    if (value != null) {
+      ((BytesColumnVector) inner.fields[1]).setVal(rowId, value.getBytes());
+    } else {
+      inner.fields[1].isNull[rowId] = true;
+      inner.fields[1].noNulls = false;
+    }
+  }
+
+  private static void checkInner(StructColumnVector inner, int rowId,
+                                 int rowInBatch, int i, String value) {
+    assertEquals("row " + rowId, i,
+        ((LongColumnVector) inner.fields[0]).vector[rowInBatch]);
+    if (value != null) {
+      assertEquals("row " + rowId, value,
+          ((BytesColumnVector) inner.fields[1]).toString(rowInBatch));
+    } else {
+      assertEquals("row " + rowId, true, inner.fields[1].isNull[rowInBatch]);
+      assertEquals("row " + rowId, false, inner.fields[1].noNulls);
+    }
+  }
+
+  private static void setInnerList(ListColumnVector list, int rowId,
+                                   List<InnerStruct> value) {
+    if (value != null) {
+      if (list.childCount + value.size() > list.child.isNull.length) {
+        list.child.ensureSize(list.childCount * 2, true);
+      }
+      list.lengths[rowId] = value.size();
+      list.offsets[rowId] = list.childCount;
+      for (int i = 0; i < list.lengths[rowId]; ++i) {
+        InnerStruct inner = value.get(i);
+        setInner((StructColumnVector) list.child, i + list.childCount,
+            inner.int1, inner.string1.toString());
+      }
+      list.childCount += value.size();
+    } else {
+      list.isNull[rowId] = true;
+      list.noNulls = false;
+    }
+  }
+
+  private static void checkInnerList(ListColumnVector list, int rowId,
+                                     int rowInBatch, List<InnerStruct> value) {
+    if (value != null) {
+      assertEquals("row " + rowId, value.size(), list.lengths[rowInBatch]);
+      int start = (int) list.offsets[rowInBatch];
+      for (int i = 0; i < list.lengths[rowInBatch]; ++i) {
+        InnerStruct inner = value.get(i);
+        checkInner((StructColumnVector) list.child, rowId, i + start,
+            inner.int1, inner.string1.toString());
+      }
+      list.childCount += value.size();
+    } else {
+      assertEquals("row " + rowId, true, list.isNull[rowInBatch]);
+      assertEquals("row " + rowId, false, list.noNulls);
+    }
+  }
+
+  private static void setInnerMap(MapColumnVector map, int rowId,
+                                  Map<String, InnerStruct> value) {
+    if (value != null) {
+      if (map.childCount >= map.keys.isNull.length) {
+        map.keys.ensureSize(map.childCount * 2, true);
+        map.values.ensureSize(map.childCount * 2, true);
+      }
+      map.lengths[rowId] = value.size();
+      int offset = map.childCount;
+      map.offsets[rowId] = offset;
+
+      for (Map.Entry<String, InnerStruct> entry : value.entrySet()) {
+        ((BytesColumnVector) map.keys).setVal(offset, entry.getKey().getBytes());
+        InnerStruct inner = entry.getValue();
+        setInner((StructColumnVector) map.values, offset, inner.int1,
+            inner.string1.toString());
+        offset += 1;
+      }
+      map.childCount = offset;
+    } else {
+      map.isNull[rowId] = true;
+      map.noNulls = false;
+    }
+  }
+
+  private static void checkInnerMap(MapColumnVector map, int rowId,
+                                    int rowInBatch,
+                                    Map<String, InnerStruct> value) {
+    if (value != null) {
+      assertEquals("row " + rowId, value.size(), map.lengths[rowInBatch]);
+      int offset = (int) map.offsets[rowInBatch];
+      for(int i=0; i < value.size(); ++i) {
+        String key = ((BytesColumnVector) map.keys).toString(offset + i);
+        InnerStruct expected = value.get(key);
+        checkInner((StructColumnVector) map.values, rowId, offset + i,
+            expected.int1, expected.string1.toString());
+      }
+    } else {
+      assertEquals("row " + rowId, true, map.isNull[rowId]);
+      assertEquals("row " + rowId, false, map.noNulls);
+    }
+  }
+
+  private static void setMiddleStruct(StructColumnVector middle, int rowId,
+                                      MiddleStruct value) {
+    if (value != null) {
+      setInnerList((ListColumnVector) middle.fields[0], rowId, value.list);
+    } else {
+      middle.isNull[rowId] = true;
+      middle.noNulls = false;
+    }
+  }
+
+  private static void checkMiddleStruct(StructColumnVector middle, int rowId,
+                                        int rowInBatch, MiddleStruct value) {
+    if (value != null) {
+      checkInnerList((ListColumnVector) middle.fields[0], rowId, rowInBatch,
+          value.list);
+    } else {
+      assertEquals("row " + rowId, true, middle.isNull[rowInBatch]);
+      assertEquals("row " + rowId, false, middle.noNulls);
+    }
+  }
+
+  private static void setBigRow(VectorizedRowBatch batch, int rowId,
+                                Boolean b1, Byte b2, Short s1,
+                                Integer i1, Long l1, Float f1,
+                                Double d1, BytesWritable b3, String s2,
+                                MiddleStruct m1, List<InnerStruct> l2,
+                                Map<String, InnerStruct> m2) {
+    ((LongColumnVector) batch.cols[0]).vector[rowId] = b1 ? 1 : 0;
+    ((LongColumnVector) batch.cols[1]).vector[rowId] = b2;
+    ((LongColumnVector) batch.cols[2]).vector[rowId] = s1;
+    ((LongColumnVector) batch.cols[3]).vector[rowId] = i1;
+    ((LongColumnVector) batch.cols[4]).vector[rowId] = l1;
+    ((DoubleColumnVector) batch.cols[5]).vector[rowId] = f1;
+    ((DoubleColumnVector) batch.cols[6]).vector[rowId] = d1;
+    if (b3 != null) {
+      ((BytesColumnVector) batch.cols[7]).setVal(rowId, b3.getBytes(), 0,
+          b3.getLength());
+    } else {
+      batch.cols[7].isNull[rowId] = true;
+      batch.cols[7].noNulls = false;
+    }
+    if (s2 != null) {
+      ((BytesColumnVector) batch.cols[8]).setVal(rowId, s2.getBytes());
+    } else {
+      batch.cols[8].isNull[rowId] = true;
+      batch.cols[8].noNulls = false;
+    }
+    setMiddleStruct((StructColumnVector) batch.cols[9], rowId, m1);
+    setInnerList((ListColumnVector) batch.cols[10], rowId, l2);
+    setInnerMap((MapColumnVector) batch.cols[11], rowId, m2);
+  }
+
+  private static void checkBigRow(VectorizedRowBatch batch,
+                                  int rowInBatch,
+                                  int rowId,
+                                  boolean b1, byte b2, short s1,
+                                  int i1, long l1, float f1,
+                                  double d1, BytesWritable b3, String s2,
+                                  MiddleStruct m1, List<InnerStruct> l2,
+                                  Map<String, InnerStruct> m2) {
+    assertEquals("row " + rowId, b1, getBoolean(batch, rowInBatch));
+    assertEquals("row " + rowId, b2, getByte(batch, rowInBatch));
+    assertEquals("row " + rowId, s1, getShort(batch, rowInBatch));
+    assertEquals("row " + rowId, i1, getInt(batch, rowInBatch));
+    assertEquals("row " + rowId, l1, getLong(batch, rowInBatch));
+    assertEquals("row " + rowId, f1, getFloat(batch, rowInBatch), 0.0001);
+    assertEquals("row " + rowId, d1, getDouble(batch, rowInBatch), 0.0001);
+    if (b3 != null) {
+      BytesColumnVector bytes = (BytesColumnVector) batch.cols[7];
+      assertEquals("row " + rowId, b3.getLength(), bytes.length[rowInBatch]);
+      for(int i=0; i < b3.getLength(); ++i) {
+        assertEquals("row " + rowId + " byte " + i, b3.getBytes()[i],
+            bytes.vector[rowInBatch][bytes.start[rowInBatch] + i]);
+      }
+    } else {
+      assertEquals("row " + rowId, true, batch.cols[7].isNull[rowInBatch]);
+      assertEquals("row " + rowId, false, batch.cols[7].noNulls);
+    }
+    if (s2 != null) {
+      assertEquals("row " + rowId, s2, getText(batch, rowInBatch).toString());
+    } else {
+      assertEquals("row " + rowId, true, batch.cols[8].isNull[rowInBatch]);
+      assertEquals("row " + rowId, false, batch.cols[8].noNulls);
+    }
+    checkMiddleStruct((StructColumnVector) batch.cols[9], rowId, rowInBatch,
+        m1);
+    checkInnerList((ListColumnVector) batch.cols[10], rowId, rowInBatch, l2);
+    checkInnerMap((MapColumnVector) batch.cols[11], rowId, rowInBatch, m2);
+  }
+
+  private static boolean getBoolean(VectorizedRowBatch batch, int rowId) {
+    return ((LongColumnVector) batch.cols[0]).vector[rowId] != 0;
+  }
+
+  private static byte getByte(VectorizedRowBatch batch, int rowId) {
+    return (byte) ((LongColumnVector) batch.cols[1]).vector[rowId];
+  }
+
+  private static short getShort(VectorizedRowBatch batch, int rowId) {
+    return (short) ((LongColumnVector) batch.cols[2]).vector[rowId];
+  }
+
+  private static int getInt(VectorizedRowBatch batch, int rowId) {
+    return (int) ((LongColumnVector) batch.cols[3]).vector[rowId];
+  }
+
+  private static long getLong(VectorizedRowBatch batch, int rowId) {
+    return ((LongColumnVector) batch.cols[4]).vector[rowId];
+  }
+
+  private static float getFloat(VectorizedRowBatch batch, int rowId) {
+    return (float) ((DoubleColumnVector) batch.cols[5]).vector[rowId];
+  }
+
+  private static double getDouble(VectorizedRowBatch batch, int rowId) {
+    return ((DoubleColumnVector) batch.cols[6]).vector[rowId];
+  }
+
+  private static BytesWritable getBinary(BytesColumnVector column, int rowId) {
+    if (column.isRepeating) {
+      rowId = 0;
+    }
+    if (column.noNulls || !column.isNull[rowId]) {
+      return new BytesWritable(Arrays.copyOfRange(column.vector[rowId],
+          column.start[rowId], column.start[rowId] + column.length[rowId]));
+    } else {
+      return null;
+    }
+  }
+
+  private static BytesWritable getBinary(VectorizedRowBatch batch, int rowId) {
+    return getBinary((BytesColumnVector) batch.cols[7], rowId);
+  }
+
+  private static Text getText(BytesColumnVector vector, int rowId) {
+    if (vector.isRepeating) {
+      rowId = 0;
+    }
+    if (vector.noNulls || !vector.isNull[rowId]) {
+      return new Text(Arrays.copyOfRange(vector.vector[rowId],
+          vector.start[rowId], vector.start[rowId] + vector.length[rowId]));
+    } else {
+      return null;
+    }
+  }
+
+  private static Text getText(VectorizedRowBatch batch, int rowId) {
+    return getText((BytesColumnVector) batch.cols[8], rowId);
+  }
+
+  private static InnerStruct getInner(StructColumnVector vector,
+                                      int rowId) {
+    return new InnerStruct(
+        (int) ((LongColumnVector) vector.fields[0]).vector[rowId],
+        getText((BytesColumnVector) vector.fields[1], rowId));
+  }
+
+  private static List<InnerStruct> getList(ListColumnVector cv,
+                                           int rowId) {
+    if (cv.isRepeating) {
+      rowId = 0;
+    }
+    if (cv.noNulls || !cv.isNull[rowId]) {
+      List<InnerStruct> result =
+          new ArrayList<InnerStruct>((int) cv.lengths[rowId]);
+      for(long i=cv.offsets[rowId];
+          i < cv.offsets[rowId] + cv.lengths[rowId]; ++i) {
+        result.add(getInner((StructColumnVector) cv.child, (int) i));
+      }
+      return result;
+    } else {
+      return null;
+    }
+  }
+
+  private static List<InnerStruct> getMidList(VectorizedRowBatch batch,
+                                              int rowId) {
+    return getList((ListColumnVector) ((StructColumnVector) batch.cols[9])
+        .fields[0], rowId);
+  }
+
+  private static List<InnerStruct> getList(VectorizedRowBatch batch,
+                                           int rowId) {
+    return getList((ListColumnVector) batch.cols[10], rowId);
+  }
+
+  private static Map<Text, InnerStruct> getMap(VectorizedRowBatch batch,
+                                               int rowId) {
+    MapColumnVector cv = (MapColumnVector) batch.cols[11];
+    if (cv.isRepeating) {
+      rowId = 0;
+    }
+    if (cv.noNulls || !cv.isNull[rowId]) {
+      Map<Text, InnerStruct> result =
+          new HashMap<Text, InnerStruct>((int) cv.lengths[rowId]);
+      for(long i=cv.offsets[rowId];
+          i < cv.offsets[rowId] + cv.lengths[rowId]; ++i) {
+        result.put(getText((BytesColumnVector) cv.keys, (int) i),
+            getInner((StructColumnVector) cv.values, (int) i));
+      }
+      return result;
+    } else {
+      return null;
+    }
+  }
+
+  private static TypeDescription createInnerSchema() {
+    return TypeDescription.createStruct()
+        .addField("int1", TypeDescription.createInt())
+        .addField("string1", TypeDescription.createString());
+  }
+
+  private static TypeDescription createBigRowSchema() {
+    return TypeDescription.createStruct()
+        .addField("boolean1", TypeDescription.createBoolean())
+        .addField("byte1", TypeDescription.createByte())
+        .addField("short1", TypeDescription.createShort())
+        .addField("int1", TypeDescription.createInt())
+        .addField("long1", TypeDescription.createLong())
+        .addField("float1", TypeDescription.createFloat())
+        .addField("double1", TypeDescription.createDouble())
+        .addField("bytes1", TypeDescription.createBinary())
+        .addField("string1", TypeDescription.createString())
+        .addField("middle", TypeDescription.createStruct()
+            .addField("list", TypeDescription.createList(createInnerSchema())))
+        .addField("list", TypeDescription.createList(createInnerSchema()))
+        .addField("map", TypeDescription.createMap(
+            TypeDescription.createString(),
+            createInnerSchema()));
+  }
+
+  static void assertArrayEquals(boolean[] expected, boolean[] actual) {
+    assertEquals(expected.length, actual.length);
+    boolean diff = false;
+    for(int i=0; i < expected.length; ++i) {
+      if (expected[i] != actual[i]) {
+        System.out.println("Difference at " + i + " expected: " + expected[i] +
+          " actual: " + actual[i]);
+        diff = true;
+      }
+    }
+    assertEquals(false, diff);
+  }
+
+  @Test
+  public void test1() throws Exception {
+    TypeDescription schema = createBigRowSchema();
+    Writer writer = OrcFile.createWriter(testFilePath,
+        OrcFile.writerOptions(conf)
+            .setSchema(schema)
+            .stripeSize(100000)
+            .bufferSize(10000));
+    VectorizedRowBatch batch = schema.createRowBatch();
+    batch.size = 2;
+    setBigRow(batch, 0, false, (byte) 1, (short) 1024, 65536,
+        Long.MAX_VALUE, (float) 1.0, -15.0, bytes(0, 1, 2, 3, 4), "hi",
+        new MiddleStruct(inner(1, "bye"), inner(2, "sigh")),
+        list(inner(3, "good"), inner(4, "bad")),
+        map());
+    setBigRow(batch, 1, true, (byte) 100, (short) 2048, 65536,
+        Long.MAX_VALUE, (float) 2.0, -5.0, bytes(), "bye",
+        new MiddleStruct(inner(1, "bye"), inner(2, "sigh")),
+        list(inner(100000000, "cat"), inner(-100000, "in"), inner(1234, "hat")),
+        map(inner(5, "chani"), inner(1, "mauddib")));
+    writer.addRowBatch(batch);
+    writer.close();
+    Reader reader = OrcFile.createReader(testFilePath,
+        OrcFile.readerOptions(conf).filesystem(fs));
+
+    schema = writer.getSchema();
+    assertEquals(23, schema.getMaximumId());
+    boolean[] expected = new boolean[] {false, false, false, false, false,
+        false, false, false, false, false,
+        false, false, false, false, false,
+        false, false, false, false, false,
+        false, false, false, false};
+    boolean[] included = OrcUtils.includeColumns("", schema);
+    assertEquals(true, Arrays.equals(expected, included));
+
+    expected = new boolean[] {false, true, false, false, false,
+        false, false, false, false, true,
+        true, true, true, true, true,
+        false, false, false, false, true,
+        true, true, true, true};
+    included = OrcUtils.includeColumns("boolean1,string1,middle,map", schema);
+
+    assertArrayEquals(expected, included);
+
+    expected = new boolean[] {false, true, false, false, false,
+        false, false, false, false, true,
+        true, true, true, true, true,
+        false, false, false, false, true,
+        true, true, true, true};
+    included = OrcUtils.includeColumns("boolean1,string1,middle,map", schema);
+    assertArrayEquals(expected, included);
+
+    expected = new boolean[] {false, true, true, true, true,
+        true, true, true, true, true,
+        true, true, true, true, true,
+        true, true, true, true, true,
+        true, true, true, true};
+    included = OrcUtils.includeColumns(
+        "boolean1,byte1,short1,int1,long1,float1,double1,bytes1,string1,middle,list,map",
+        schema);
+    assertEquals(true, Arrays.equals(expected, included));
+
+    // check the stats
+    ColumnStatistics[] stats = reader.getStatistics();
+    assertEquals(2, stats[1].getNumberOfValues());
+    assertEquals(1, ((BooleanColumnStatistics) stats[1]).getFalseCount());
+    assertEquals(1, ((BooleanColumnStatistics) stats[1]).getTrueCount());
+    assertEquals("count: 2 hasNull: false true: 1", stats[1].toString());
+
+    assertEquals(2048, ((IntegerColumnStatistics) stats[3]).getMaximum());
+    assertEquals(1024, ((IntegerColumnStatistics) stats[3]).getMinimum());
+    assertEquals(true, ((IntegerColumnStatistics) stats[3]).isSumDefined());
+    assertEquals(3072, ((IntegerColumnStatistics) stats[3]).getSum());
+    assertEquals("count: 2 hasNull: false min: 1024 max: 2048 sum: 3072",
+        stats[3].toString());
+
+    StripeStatistics ss = reader.getStripeStatistics().get(0);
+    assertEquals(2, ss.getColumnStatistics()[0].getNumberOfValues());
+    assertEquals(1, ((BooleanColumnStatistics) ss.getColumnStatistics()[1]).getTrueCount());
+    assertEquals(1024, ((IntegerColumnStatistics) ss.getColumnStatistics()[3]).getMinimum());
+    assertEquals(2048, ((IntegerColumnStatistics) ss.getColumnStatistics()[3]).getMaximum());
+    assertEquals(3072, ((IntegerColumnStatistics) ss.getColumnStatistics()[3]).getSum());
+    assertEquals(-15.0, ((DoubleColumnStatistics) stats[7]).getMinimum(), 0.0001);
+    assertEquals(-5.0, ((DoubleColumnStatistics) stats[7]).getMaximum(), 0.0001);
+    assertEquals(-20.0, ((DoubleColumnStatistics) stats[7]).getSum(), 0.00001);
+    assertEquals("count: 2 hasNull: false min: -15.0 max: -5.0 sum: -20.0",
+        stats[7].toString());
+
+    assertEquals("count: 2 hasNull: false min: bye max: hi sum: 5", stats[9].toString());
+
+    // check the schema
+    TypeDescription readerSchema = reader.getSchema();
+    assertEquals(TypeDescription.Category.STRUCT, readerSchema.getCategory());
+    assertEquals("struct<boolean1:boolean,byte1:tinyint,short1:smallint,"
+        + "int1:int,long1:bigint,float1:float,double1:double,bytes1:"
+        + "binary,string1:string,middle:struct<list:array<struct<int1:int,"
+        + "string1:string>>>,list:array<struct<int1:int,string1:string>>,"
+        + "map:map<string,struct<int1:int,string1:string>>>",
+        readerSchema.toString());
+    List<String> fieldNames = readerSchema.getFieldNames();
+    List<TypeDescription> fieldTypes = readerSchema.getChildren();
+    assertEquals("boolean1", fieldNames.get(0));
+    assertEquals(TypeDescription.Category.BOOLEAN, fieldTypes.get(0).getCategory());
+    assertEquals("byte1", fieldNames.get(1));
+    assertEquals(TypeDescription.Category.BYTE, fieldTypes.get(1).getCategory());
+    assertEquals("short1", fieldNames.get(2));
+    assertEquals(TypeDescription.Category.SHORT, fieldTypes.get(2).getCategory());
+    assertEquals("int1", fieldNames.get(3));
+    assertEquals(TypeDescription.Category.INT, fieldTypes.get(3).getCategory());
+    assertEquals("long1", fieldNames.get(4));
+    assertEquals(TypeDescription.Category.LONG, fieldTypes.get(4).getCategory());
+    assertEquals("float1", fieldNames.get(5));
+    assertEquals(TypeDescription.Category.FLOAT, fieldTypes.get(5).getCategory());
+    assertEquals("double1", fieldNames.get(6));
+    assertEquals(TypeDescription.Category.DOUBLE, fieldTypes.get(6).getCategory());
+    assertEquals("bytes1", fieldNames.get(7));
+    assertEquals(TypeDescription.Category.BINARY, fieldTypes.get(7).getCategory());
+    assertEquals("string1", fieldNames.get(8));
+    assertEquals(TypeDescription.Category.STRING, fieldTypes.get(8).getCategory());
+    assertEquals("middle", fieldNames.get(9));
+    TypeDescription middle = fieldTypes.get(9);
+    assertEquals(TypeDescription.Category.STRUCT, middle.getCategory());
+    TypeDescription midList = middle.getChildren().get(0);
+    assertEquals(TypeDescription.Category.LIST, midList.getCategory());
+    TypeDescription inner = midList.getChildren().get(0);
+    assertEquals(TypeDescription.Category.STRUCT, inner.getCategory());
+    assertEquals("int1", inner.getFieldNames().get(0));
+    assertEquals("string1", inner.getFieldNames().get(1));
+
+    RecordReader rows = reader.rows();
+    // create a new batch
+    batch = readerSchema.createRowBatch();
+    Assert.assertEquals(true, rows.nextBatch(batch));
+    assertEquals(2, batch.size);
+    Assert.assertEquals(false, rows.nextBatch(batch));
+
+    // check the contents of the first row
+    assertEquals(false, getBoolean(batch, 0));
+    assertEquals(1, getByte(batch, 0));
+    assertEquals(1024, getShort(batch, 0));
+    assertEquals(65536, getInt(batch, 0));
+    assertEquals(Long.MAX_VALUE, getLong(batch, 0));
+    assertEquals(1.0, getFloat(batch, 0), 0.00001);
+    assertEquals(-15.0, getDouble(batch, 0), 0.00001);
+    assertEquals(bytes(0,1,2,3,4), getBinary(batch, 0));
+    assertEquals("hi", getText(batch, 0).toString());
+    List<InnerStruct> midRow = getMidList(batch, 0);
+    assertNotNull(midRow);
+    assertEquals(2, midRow.size());
+    assertEquals(1, midRow.get(0).int1);
+    assertEquals("bye", midRow.get(0).string1.toString());
+    assertEquals(2, midRow.get(1).int1);
+    assertEquals("sigh", midRow.get(1).string1.toString());
+    List<InnerStruct> list = getList(batch, 0);
+    assertEquals(2, list.size());
+    assertEquals(3, list.get(0).int1);
+    assertEquals("good", list.get(0).string1.toString());
+    assertEquals(4, list.get(1).int1);
+    assertEquals("bad", list.get(1).string1.toString());
+    Map<Text, InnerStruct> map = getMap(batch, 0);
+    assertEquals(0, map.size());
+
+    // check the contents of second row
+    assertEquals(true, getBoolean(batch, 1));
+    assertEquals(100, getByte(batch, 1));
+    assertEquals(2048, getShort(batch, 1));
+    assertEquals(65536, getInt(batch, 1));
+    assertEquals(Long.MAX_VALUE, getLong(batch, 1));
+    assertEquals(2.0, getFloat(batch, 1), 0.00001);
+    assertEquals(-5.0, getDouble(batch, 1), 0.00001);
+    assertEquals(bytes(), getBinary(batch, 1));
+    assertEquals("bye", getText(batch, 1).toString());
+    midRow = getMidList(batch, 1);
+    assertNotNull(midRow);
+    assertEquals(2, midRow.size());
+    assertEquals(1, midRow.get(0).int1);
+    assertEquals("bye", midRow.get(0).string1.toString());
+    assertEquals(2, midRow.get(1).int1);
+    assertEquals("sigh", midRow.get(1).string1.toString());
+    list = getList(batch, 1);
+    assertEquals(3, list.size());
+    assertEquals(100000000, list.get(0).int1);
+    assertEquals("cat", list.get(0).string1.toString());
+    assertEquals(-100000, list.get(1).int1);
+    assertEquals("in", list.get(1).string1.toString());
+    assertEquals(1234, list.get(2).int1);
+    assertEquals("hat", list.get(2).string1.toString());
+    map = getMap(batch, 1);
+    assertEquals(2, map.size());
+    InnerStruct value = map.get(new Text("chani"));
+    assertEquals(5, value.int1);
+    assertEquals("chani", value.string1.toString());
+    value = map.get(new Text("mauddib"));
+    assertEquals(1, value.int1);
+    assertEquals("mauddib", value.string1.toString());
+
+    // handle the close up
+    Assert.assertEquals(false, rows.nextBatch(batch));
+    rows.close();
+  }
+
+  @Test
+  public void testColumnProjection() throws Exception {
+    TypeDescription schema = createInnerSchema();
+    Writer writer = OrcFile.createWriter(testFilePath,
+                                         OrcFile.writerOptions(conf)
+                                         .setSchema(schema)
+                                         .stripeSize(1000)
+                                         .compress(CompressionKind.NONE)
+                                         .bufferSize(100)
+                                         .rowIndexStride(1000));
+    VectorizedRowBatch batch = schema.createRowBatch();
+    Random r1 = new Random(1);
+    Random r2 = new Random(2);
+    int x;
+    int minInt=0, maxInt=0;
+    String y;
+    String minStr = null, maxStr = null;
+    batch.size = 1000;
+    boolean first = true;
+    for(int b=0; b < 21; ++b) {
+      for(int r=0; r < 1000; ++r) {
+        x = r1.nextInt();
+        y = Long.toHexString(r2.nextLong());
+        if (first || x < minInt) {
+          minInt = x;
+        }
+        if (first || x > maxInt) {
+          maxInt = x;
+        }
+        if (first || y.compareTo(minStr) < 0) {
+          minStr = y;
+        }
+        if (first || y.compareTo(maxStr) > 0) {
+          maxStr = y;
+        }
+        first = false;
+        ((LongColumnVector) batch.cols[0]).vector[r] = x;
+        ((BytesColumnVector) batch.cols[1]).setVal(r, y.getBytes());
+      }
+      writer.addRowBatch(batch);
+    }
+    writer.close();
+    Reader reader = OrcFile.createReader(testFilePath,
+        OrcFile.readerOptions(conf).filesystem(fs));
+
+    // check out the statistics
+    ColumnStatistics[] stats = reader.getStatistics();
+    assertEquals(3, stats.length);
+    for(ColumnStatistics s: stats) {
+      assertEquals(21000, s.getNumberOfValues());
+      if (s instanceof IntegerColumnStatistics) {
+        assertEquals(minInt, ((IntegerColumnStatistics) s).getMinimum());
+        assertEquals(maxInt, ((IntegerColumnStatistics) s).getMaximum());
+      } else if (s instanceof  StringColumnStatistics) {
+        assertEquals(maxStr, ((StringColumnStatistics) s).getMaximum());
+        assertEquals(minStr, ((StringColumnStatistics) s).getMinimum());
+      }
+    }
+
+    // check out the types
+    TypeDescription type = reader.getSchema();
+    assertEquals(TypeDescription.Category.STRUCT, type.getCategory());
+    assertEquals(2, type.getChildren().size());
+    TypeDescription type1 = type.getChildren().get(0);
+    TypeDescription type2 = type.getChildren().get(1);
+    assertEquals(TypeDescription.Category.INT, type1.getCategory());
+    assertEquals(TypeDescription.Category.STRING, type2.getCategory());
+    assertEquals("struct<int1:int,string1:string>", type.toString());
+
+    // read the contents and make sure they match
+    RecordReader rows1 = reader.rows(
+        new Reader.Options().include(new boolean[]{true, true, false}));
+    RecordReader rows2 = reader.rows(
+        new Reader.Options().include(new boolean[]{true, false, true}));
+    r1 = new Random(1);
+    r2 = new Random(2);
+    VectorizedRowBatch batch1 = reader.getSchema().createRowBatch(1000);
+    VectorizedRowBatch batch2 = reader.getSchema().createRowBatch(1000);
+    for(int i = 0; i < 21000; i += 1000) {
+      Assert.assertEquals(true, rows1.nextBatch(batch1));
+      Assert.assertEquals(true, rows2.nextBatch(batch2));
+      assertEquals(1000, batch1.size);
+      assertEquals(1000, batch2.size);
+      for(int j=0; j < 1000; ++j) {
+        assertEquals(r1.nextInt(),
+            ((LongColumnVector) batch1.cols[0]).vector[j]);
+        assertEquals(Long.toHexString(r2.nextLong()),
+            ((BytesColumnVector) batch2.cols[1]).toString(j));
+      }
+    }
+    Assert.assertEquals(false, rows1.nextBatch(batch1));
+    Assert.assertEquals(false, rows2.nextBatch(batch2));
+    rows1.close();
+    rows2.close();
+  }
+
+  @Test
+  public void testEmptyFile() throws Exception {
+    TypeDescription schema = createBigRowSchema();
+    Writer writer = OrcFile.createWriter(testFilePath,
+                                         OrcFile.writerOptions(conf)
+                                         .setSchema(schema)
+                                         .stripeSize(1000)
+                                         .compress(CompressionKind.NONE)
+                                         .bufferSize(100));
+    writer.close();
+    Reader reader = OrcFile.createReader(testFilePath,
+        OrcFile.readerOptions(conf).filesystem(fs));
+    VectorizedRowBatch batch = reader.getSchema().createRowBatch();
+    Assert.assertEquals(false, reader.rows().nextBatch(batch));
+    Assert.assertEquals(CompressionKind.NONE, reader.getCompressionKind());
+    Assert.assertEquals(0, reader.getNumberOfRows());
+    Assert.assertEquals(0, reader.getCompressionSize());
+    Assert.assertEquals(false, reader.getMetadataKeys().iterator().hasNext());
+    Assert.assertEquals(3, reader.getContentLength());
+    Assert.assertEquals(false, reader.getStripes().iterator().hasNext());
+  }
+
+  @Test
+  public void metaData() throws Exception {
+    TypeDescription schema = createBigRowSchema();
+    Writer writer = OrcFile.createWriter(testFilePath,
+        OrcFile.writerOptions(conf)
+            .setSchema(schema)
+            .stripeSize(1000)
+            .compress(CompressionKind.NONE)
+            .bufferSize(100));
+    writer.addUserMetadata("my.meta", byteBuf(1, 2, 3, 4, 5, 6, 7, -1, -2, 127,
+                                              -128));
+    writer.addUserMetadata("clobber", byteBuf(1, 2, 3));
+    writer.addUserMetadata("clobber", byteBuf(4, 3, 2, 1));
+    ByteBuffer bigBuf = ByteBuffer.allocate(40000);
+    Random random = new Random(0);
+    random.nextBytes(bigBuf.array());
+    writer.addUserMetadata("big", bigBuf);
+    bigBuf.position(0);
+    VectorizedRowBatch batch = schema.createRowBatch();
+    batch.size = 1;
+    setBigRow(batch, 0, true, (byte) 127, (short) 1024, 42,
+        42L * 1024 * 1024 * 1024, (float) 3.1415, -2.713, null,
+        null, null, null, null);
+    writer.addRowBatch(batch);
+    writer.addUserMetadata("clobber", byteBuf(5,7,11,13,17,19));
+    writer.close();
+
+    Reader reader = OrcFile.createReader(testFilePath,
+        OrcFile.readerOptions(conf).filesystem(fs));
+    Assert.assertEquals(byteBuf(5, 7, 11, 13, 17, 19), reader.getMetadataValue("clobber"));
+    Assert.assertEquals(byteBuf(1, 2, 3, 4, 5, 6, 7, -1, -2, 127, -128),
+        reader.getMetadataValue("my.meta"));
+    Assert.assertEquals(bigBuf, reader.getMetadataValue("big"));
+    try {
+      reader.getMetadataValue("unknown");
+      assertTrue(false);
+    } catch (IllegalArgumentException iae) {
+      // PASS
+    }
+    int i = 0;
+    for(String key: reader.getMetadataKeys()) {
+      if ("my.meta".equals(key) ||
+          "clobber".equals(key) ||
+          "big".equals(key)) {
+        i += 1;
+      } else {
+        throw new IllegalArgumentException("unknown key " + key);
+      }
+    }
+    assertEquals(3, i);
+    int numStripes = reader.getStripeStatistics().size();
+    assertEquals(1, numStripes);
+  }
+
+  /**
+   * Generate an ORC file with a range of dates and times.
+   */
+  public void createOrcDateFile(Path file, int minYear, int maxYear
+                                ) throws IOException {
+    TypeDescription schema = TypeDescription.createStruct()
+        .addField("time", TypeDescription.createTimestamp())
+        .addField("date", TypeDescription.createDate());
+    Writer writer = OrcFile.createWriter(file,
+        OrcFile.writerOptions(conf)
+            .setSchema(schema)
+            .stripeSize(100000)
+            .bufferSize(10000)
+            .blockPadding(false));
+    VectorizedRowBatch batch = schema.createRowBatch();
+    batch.size = 1000;
+    for (int year = minYear; year < maxYear; ++year) {
+      for (int ms = 1000; ms < 2000; ++ms) {
+        TimestampColumnVector timestampColVector = (TimestampColumnVector) batch.cols[0];
+        timestampColVector.set(ms - 1000,
+            Timestamp.valueOf(year +
+                "-05-05 12:34:56." + ms));
+        ((LongColumnVector) batch.cols[1]).vector[ms - 1000] =
+            new DateWritable(new Date(year - 1900, 11, 25)).getDays();
+      }
+      writer.addRowBatch(batch);
+    }
+    writer.close();
+    Reader reader = OrcFile.createReader(file,
+        OrcFile.readerOptions(conf));
+    RecordReader rows = reader.rows();
+    batch = reader.getSchema().createRowBatch(1000);
+    TimestampColumnVector times = (TimestampColumnVector) batch.cols[0];
+    LongColumnVector dates = (LongColumnVector) batch.cols[1];
+    for (int year = minYear; year < maxYear; ++year) {
+      rows.nextBatch(batch);
+      assertEquals(1000, batch.size);
+      for(int ms = 1000; ms < 2000; ++ms) {
+        StringBuilder buffer = new StringBuilder();
+        times.stringifyValue(buffer, ms - 1000);
+        String expected = Integer.toString(year) + "-05-05 12:34:56.";
+        // suppress the final zeros on the string by dividing by the largest
+        // power of 10 that divides evenly.
+        int roundedMs = ms;
+        for(int round = 1000; round > 0; round /= 10) {
+          if (ms % round == 0) {
+            roundedMs = ms / round;
+            break;
+          }
+        }
+        expected += roundedMs;
+        assertEquals(expected, buffer.toString());
+        assertEquals(Integer.toString(year) + "-12-25",
+            new DateWritable((int) dates.vector[ms - 1000]).toString());
+      }
+    }
+    rows.nextBatch(batch);
+    assertEquals(0, batch.size);
+  }
+
+  @Test
+  public void testDate1900() throws Exception {
+    createOrcDateFile(testFilePath, 1900, 1970);
+  }
+
+  @Test
+  public void testDate2038() throws Exception {
+    createOrcDateFile(testFilePath, 2038, 2250);
+  }
+
+  private static void setUnion(VectorizedRowBatch batch, int rowId,
+                               Timestamp ts, Integer tag, Integer i, String s,
+                               HiveDecimalWritable dec) {
+    UnionColumnVector union = (UnionColumnVector) batch.cols[1];
+    if (ts != null) {
+      TimestampColumnVector timestampColVector = (TimestampColumnVector) batch.cols[0];
+      timestampColVector.set(rowId, ts);
+    } else {
+      batch.cols[0].isNull[rowId] = true;
+      batch.cols[0].noNulls = false;
+    }
+    if (tag != null) {
+      union.tags[rowId] = tag;
+      if (tag == 0) {
+        if (i != null) {
+          ((LongColumnVector) union.fields[tag]).vector[rowId] = i;
+        } else {
+          union.fields[tag].isNull[rowId] = true;
+          union.fields[tag].noNulls = false;
+        }
+      } else if (tag == 1) {
+        if (s != null) {
+          ((BytesColumnVector) union.fields[tag]).setVal(rowId, s.getBytes());
+        } else {
+          union.fields[tag].isNull[rowId] = true;
+          union.fields[tag].noNulls = false;
+        }
+      } else {
+        throw new IllegalArgumentException("Bad tag " + tag);
+      }
+    } else {
+      batch.cols[1].isNull[rowId] = true;
+      batch.cols[1].noNulls = false;
+    }
+    if (dec != null) {
+      ((DecimalColumnVector) batch.cols[2]).vector[rowId] = dec;
+    } else {
+      batch.cols[2].isNull[rowId] = true;
+      batch.cols[2].noNulls = false;
+    }
+  }
+
+  /**
+     * We test union, timestamp, and decimal separately since we need to make the
+     * object inspector manually. (The Hive reflection-based doesn't handle
+     * them properly.)
+     */
+  @Test
+  public void testUnionAndTimestamp() throws Exception {
+    TypeDescription schema = TypeDescription.createStruct()
+        .addField("time", TypeDescription.createTimestamp())
+        .addField("union", TypeDescription.createUnion()
+            .addUnionChild(TypeDescription.createInt())
+            .addUnionChild(TypeDescription.createString()))
+        .addField("decimal", TypeDescription.createDecimal()
+            .withPrecision(38)
+            .withScale(18));
+    HiveDecimal maxValue = HiveDecimal.create("10000000000000000000");
+    Writer writer = OrcFile.createWriter(testFilePath,
+                                         OrcFile.writerOptions(conf)
+                                         .setSchema(schema)
+                                         .stripeSize(1000)
+                                         .compress(CompressionKind.NONE)
+                                         .bufferSize(100)
+                                         .blockPadding(false));
+    VectorizedRowBatch batch = schema.createRowBatch();
+    batch.size = 6;
+    setUnion(batch, 0, Timestamp.valueOf("2000-03-12 15:00:00"), 0, 42, null,
+             new HiveDecimalWritable("12345678.6547456"));
+    setUnion(batch, 1, Timestamp.valueOf("2000-03-20 12:00:00.123456789"),
+        1, null, "hello", new HiveDecimalWritable("-5643.234"));
+
+    setUnion(batch, 2, null, null, null, null, null);
+    setUnion(batch, 3, null, 0, null, null, null);
+    setUnion(batch, 4, null, 1, null, null, null);
+
+    setUnion(batch, 5, Timestamp.valueOf("1970-01-01 00:00:00"), 0, 200000,
+        null, new HiveDecimalWritable("10000000000000000000"));
+    writer.addRowBatch(batch);
+
+    batch.reset();
+    Random rand = new Random(42);
+    for(int i=1970; i < 2038; ++i) {
+      Timestamp ts = Timestamp.valueOf(i + "-05-05 12:34:56." + i);
+      HiveDecimal dec =
+          HiveDecimal.create(new BigInteger(64, rand), rand.nextInt(18));
+      if ((i & 1) == 0) {
+        setUnion(batch, batch.size++, ts, 0, i*i, null,
+            new HiveDecimalWritable(dec));
+      } else {
+        setUnion(batch, batch.size++, ts, 1, null, Integer.toString(i*i),
+            new HiveDecimalWritable(dec));
+      }
+      if (maxValue.compareTo(dec) < 0) {
+        maxValue = dec;
+      }
+    }
+    writer.addRowBatch(batch);
+    batch.reset();
+
+    // let's add a lot of constant rows to test the rle
+    batch.size = 1000;
+    for(int c=0; c < batch.cols.length; ++c) {
+      batch.cols[c].setRepeating(true);
+    }
+    ((UnionColumnVector) batch.cols[1]).fields[0].isRepeating = true;
+    setUnion(batch, 0, null, 0, 1732050807, null, null);
+    for(int i=0; i < 5; ++i) {
+      writer.addRowBatch(batch);
+    }
+
+    batch.reset();
+    batch.size = 3;
+    setUnion(batch, 0, null, 0, 0, null, null);
+    setUnion(batch, 1, null, 0, 10, null, null);
+    setUnion(batch, 2, null, 0, 138, null, null);
+    writer.addRowBatch(batch);
+    writer.close();
+    Reader reader = OrcFile.createReader(testFilePath,
+        OrcFile.readerOptions(conf).filesystem(fs));
+
+    schema = writer.getSchema();
+    assertEquals(5, schema.getMaximumId());
+    boolean[] expected = new boolean[] {false, false, false, false, false, false};
+    boolean[] included = OrcUtils.includeColumns("", schema);
+    assertEquals(true, Arrays.equals(expected, included));
+
+    expected = new boolean[] {false, true, false, false, false, true};
+    included = OrcUtils.includeColumns("time,decimal", schema);
+    assertEquals(true, Arrays.equals(expected, included));
+
+    expected = new boolean[] {false, false, true, true, true, false};
+    included = OrcUtils.includeColumns("union", schema);
+    assertEquals(true, Arrays.equals(expected, included));
+
+    Assert.assertEquals(false, reader.getMetadataKeys().iterator().hasNext());
+    Assert.assertEquals(5077, reader.getNumberOfRows());
+    DecimalColumnStatistics stats =
+        (DecimalColumnStatistics) reader.getStatistics()[5];
+    assertEquals(71, stats.getNumberOfValues());
+    assertEquals(HiveDecimal.create("-5643.234"), stats.getMinimum());
+    assertEquals(maxValue, stats.getMaximum());
+    // TODO: fix this
+//    assertEquals(null,stats.getSum());
+    int stripeCount = 0;
+    int rowCount = 0;
+    long currentOffset = -1;
+    for(StripeInformation stripe: reader.getStripes()) {
+      stripeCount += 1;
+      rowCount += stripe.getNumberOfRows();
+      if (currentOffset < 0) {
+        currentOffset = stripe.getOffset() + stripe.getLength();
+      } else {
+        assertEquals(currentOffset, stripe.getOffset());
+        currentOffset += stripe.getLength();
+      }
+    }
+    Assert.assertEquals(reader.getNumberOfRows(), rowCount);
+    assertEquals(2, stripeCount);
+    Assert.assertEquals(reader.getContentLength(), currentOffset);
+    RecordReader rows = reader.rows();
+    Assert.assertEquals(0, rows.getRowNumber());
+    Assert.assertEquals(0.0, rows.getProgress(), 0.000001);
+
+    schema = reader.getSchema();
+    batch = schema.createRowBatch(74);
+    Assert.assertEquals(0, rows.getRowNumber());
+    rows.nextBatch(batch);
+    assertEquals(74, batch.size);
+    Assert.assertEquals(74, rows.getRowNumber());
+    TimestampColumnVector ts = (TimestampColumnVector) batch.cols[0];
+    UnionColumnVector union = (UnionColumnVector) batch.cols[1];
+    LongColumnVector longs = (LongColumnVector) union.fields[0];
+    BytesColumnVector strs = (BytesColumnVector) union.fields[1];
+    DecimalColumnVector decs = (DecimalColumnVector) batch.cols[2];
+
+    assertEquals("struct<time:timestamp,union:uniontype<int,string>,decimal:decimal(38,18)>",
+        schema.toString());
+    assertEquals("2000-03-12 15:00:00.0", ts.asScratchTimestamp(0).toString());
+    assertEquals(0, union.tags[0]);
+    assertEquals(42, longs.vector[0]);
+    assertEquals("12345678.6547456", decs.vector[0].toString());
+
+    assertEquals("2000-03-20 12:00:00.123456789", ts.asScratchTimestamp(1).toString());
+    assertEquals(1, union.tags[1]);
+    assertEquals("hello", strs.toString(1));
+    assertEquals("-5643.234", decs.vector[1].toString());
+
+    assertEquals(false, ts.noNulls);
+    assertEquals(false, union.noNulls);
+    assertEquals(false, decs.noNulls);
+    assertEquals(true, ts.isNull[2]);
+    assertEquals(true, union.isNull[2]);
+    assertEquals(true, decs.isNull[2]);
+
+    assertEquals(true, ts.isNull[3]);
+    assertEquals(false, union.isNull[3]);
+    assertEquals(0, union.tags[3]);
+    assertEquals(true, longs.isNull[3]);
+    assertEquals(true, decs.isNull[3]);
+
+    assertEquals(true, ts.isNull[4]);
+    assertEquals(false, union.isNull[4]);
+    assertEquals(1, union.tags[4]);
+    assertEquals(true, strs.isNull[4]);
+    assertEquals(true, decs.isNull[4]);
+
+    assertEquals(false, ts.isNull[5]);
+    assertEquals("1970-01-01 00:00:00.0", ts.asScratchTimestamp(5).toString());
+    assertEquals(false, union.isNull[5]);
+    assertEquals(0, union.tags[5]);
+    assertEquals(false, longs.isNull[5]);
+    assertEquals(200000, longs.vector[5]);
+    assertEquals(false, decs.isNull[5]);
+    assertEquals("10000000000000000000", decs.vector[5].toString());
+
+    rand = new Random(42);
+    for(int i=1970; i < 2038; ++i) {
+      int row = 6 + i - 1970;
+      assertEquals(Timestamp.valueOf(i + "-05-05 12:34:56." + i),
+          ts.asScratchTimestamp(row));
+      if ((i & 1) == 0) {
+        assertEquals(0, union.tags[row]);
+        assertEquals(i*i, longs.vector[row]);
+      } else {
+        assertEquals(1, union.tags[row]);
+        assertEquals(Integer.toString(i * i), strs.toString(row));
+      }
+      assertEquals(new HiveDecimalWritable(HiveDecimal.create(new BigInteger(64, rand),
+                                   rand.nextInt(18))), decs.vector[row]);
+    }
+
+    // rebuild the row batch, so that we can read by 1000 rows
+    batch = schema.createRowBatch(1000);
+    ts = (TimestampColumnVector) batch.cols[0];
+    union = (UnionColumnVector) batch.cols[1];
+    longs = (LongColumnVector) union.fields[0];
+    strs = (BytesColumnVector) union.fields[1];
+    decs = (DecimalColumnVector) batch.cols[2];
+
+    for(int i=0; i < 5; ++i) {
+      rows.nextBatch(batch);
+      assertEquals("batch " + i, 1000, batch.size);
+      assertEquals("batch " + i, false, union.isRepeating);
+      assertEquals("batch " + i, true, union.noNulls);
+      for(int r=0; r < batch.size; ++r) {
+        assertEquals("bad tag at " + i + "." +r, 0, union.tags[r]);
+      }
+      assertEquals("batch " + i, true, longs.isRepeating);
+      assertEquals("batch " + i, 1732050807, longs.vector[0]);
+    }
+
+    rows.nextBatch(batch);
+    assertEquals(3, batch.size);
+    assertEquals(0, union.tags[0]);
+    assertEquals(0, longs.vector[0]);
+    assertEquals(0, union.tags[1]);
+    assertEquals(10, longs.vector[1]);
+    assertEquals(0, union.tags[2]);
+    assertEquals(138, longs.vector[2]);
+
+    rows.nextBatch(batch);
+    assertEquals(0, batch.size);
+    Assert.assertEquals(1.0, rows.getProgress(), 0.00001);
+    Assert.assertEquals(reader.getNumberOfRows(), rows.getRowNumber());
+    rows.seekToRow(1);
+    rows.nextBatch(batch);
+    assertEquals(1000, batch.size);
+    assertEquals(Timestamp.valueOf("2000-03-20 12:00:00.123456789"), ts.asScratchTimestamp(0));
+    assertEquals(1, union.tags[0]);
+    assertEquals("hello", strs.toString(0));
+    assertEquals(new HiveDecimalWritable(HiveDecimal.create("-5643.234")), decs.vector[0]);
+    rows.close();
+  }
+
+  /**
+   * Read and write a randomly generated snappy file.
+   * @throws Exception
+   */
+  @Test
+  public void testSnappy() throws Exception {
+    TypeDescription schema = createInnerSchema();
+    Writer writer = OrcFile.createWriter(testFilePath,
+                                         OrcFile.writerOptions(conf)
+                                         .setSchema(schema)
+                                         .stripeSize(1000)
+                                         .compress(CompressionKind.SNAPPY)
+                                         .bufferSize(100));
+    VectorizedRowBatch batch = schema.createRowBatch();
+    Random rand = new Random(12);
+    batch.size = 1000;
+    for(int b=0; b < 10; ++b) {
+      for (int r=0; r < 1000; ++r) {
+        ((LongColumnVector) batch.cols[0]).vector[r] = rand.nextInt();
+        ((BytesColumnVector) batch.cols[1]).setVal(r,
+            Integer.toHexString(rand.nextInt()).getBytes());
+      }
+      writer.addRowBatch(batch);
+    }
+    writer.close();
+    Reader reader = OrcFile.createReader(testFilePath,
+        OrcFile.readerOptions(conf).filesystem(fs));
+    Assert.assertEquals(CompressionKind.SNAPPY, reader.getCompressionKind());
+    RecordReader rows = reader.rows();
+    batch = reader.getSchema().createRowBatch(1000);
+    rand = new Random(12);
+    LongColumnVector longs = (LongColumnVector) batch.cols[0];
+    BytesColumnVector strs = (BytesColumnVector) batch.cols[1];
+    for(int b=0; b < 10; ++b) {
+      rows.nextBatch(batch);
+      assertEquals(1000, batch.size);
+      for(int r=0; r < batch.size; ++r) {
+        assertEquals(rand.nextInt(), longs.vector[r]);
+        assertEquals(Integer.toHexString(rand.nextInt()), strs.toString(r));
+      }
+    }
+    rows.nextBatch(batch);
+    assertEquals(0, batch.size);
+    rows.close();
+  }
+
+  /**
+   * Read and write a randomly generated snappy file.
+   * @throws Exception
+   */
+  @Test
+  public void testWithoutIndex() throws Exception {
+    TypeDescription schema = createInnerSchema();
+    Writer writer = OrcFile.createWriter(testFilePath,
+                                         OrcFile.writerOptions(conf)
+                                         .setSchema(schema)
+                                         .stripeSize(5000)
+                                         .compress(CompressionKind.SNAPPY)
+                                         .bufferSize(1000)
+                                         .rowIndexStride(0));
+    VectorizedRowBatch batch = schema.createRowBatch();
+    Random rand = new Random(24);
+    batch.size = 5;
+    for(int c=0; c < batch.cols.length; ++c) {
+      batch.cols[c].setRepeating(true);
+    }
+    for(int i=0; i < 10000; ++i) {
+      ((LongColumnVector) batch.cols[0]).vector[0] = rand.nextInt();
+      ((BytesColumnVector) batch.cols[1])
+          .setVal(0, Integer.toBinaryString(rand.nextInt()).getBytes());
+      writer.addRowBatch(batch);
+    }
+    writer.close();
+    Reader reader = OrcFile.createReader(testFilePath,
+        OrcFile.readerOptions(conf).filesystem(fs));
+    Assert.assertEquals(50000, reader.getNumberOfRows());
+    Assert.assertEquals(0, reader.getRowIndexStride());
+    StripeInformation stripe = reader.getStripes().iterator().next();
+    assertEquals(true, stripe.getDataLength() != 0);
+    assertEquals(0, stripe.getIndexLength());
+    RecordReader rows = reader.rows();
+    rand = new Random(24);
+    batch = reader.getSchema().createRowBatch(1000);
+    LongColumnVector longs = (LongColumnVector) batch.cols[0];
+    BytesColumnVector strs = (BytesColumnVector) batch.cols[1];
+    for(int i=0; i < 50; ++i) {
+      rows.nextBatch(batch);
+      assertEquals("batch " + i, 1000, batch.size);
+      for(int j=0; j < 200; ++j) {
+        int intVal = rand.nextInt();
+        String strVal = Integer.toBinaryString(rand.nextInt());
+        for (int k = 0; k < 5; ++k) {
+          assertEquals(intVal, longs.vector[j * 5 + k]);
+          assertEquals(strVal, strs.toString(j * 5 + k));
+        }
+      }
+    }
+    rows.nextBatch(batch);
+    assertEquals(0, batch.size);
+    rows.close();
+  }
+
+  @Test
+  public void testSeek() throws Exception {
+    TypeDescription schema = createBigRowSchema();
+    Writer writer = OrcFile.createWriter(testFilePath,
+                                         OrcFile.writerOptions(conf)
+                                         .setSchema(schema)
+                                         .stripeSize(200000)
+                                         .bufferSize(65536)
+                                         .rowIndexStride(1000));
+    VectorizedRowBatch batch = schema.createRowBatch();
+    Random rand = new Random(42);
+    final int COUNT=32768;
+    long[] intValues= new long[COUNT];
+    double[] doubleValues = new double[COUNT];
+    String[] stringValues = new String[COUNT];
+    BytesWritable[] byteValues = new BytesWritable[COUNT];
+    String[] words = new String[128];
+    for(int i=0; i < words.length; ++i) {
+      words[i] = Integer.toHexString(rand.nextInt());
+    }
+    for(int i=0; i < COUNT/2; ++i) {
+      intValues[2*i] = rand.nextLong();
+      intValues[2*i+1] = intValues[2*i];
+      stringValues[2*i] = words[rand.nextInt(words.length)];
+      stringValues[2*i+1] = stringValues[2*i];
+    }
+    for(int i=0; i < COUNT; ++i) {
+      doubleValues[i] = rand.nextDouble();
+      byte[] buf = new byte[20];
+      rand.nextBytes(buf);
+      byteValues[i] = new BytesWritable(buf);
+    }
+    for(int i=0; i < COUNT; ++i) {
+      appendRandomRow(batch, intValues, doubleValues, stringValues,
+          byteValues, words, i);
+      if (batch.size == 1024) {
+        writer.addRowBatch(batch);
+        batch.reset();
+      }
+    }
+    if (batch.size != 0) {
+      writer.addRowBatch(batch);
+    }
+    writer.close();
+    Reader reader = OrcFile.createReader(testFilePath,
+        OrcFile.readerOptions(conf).filesystem(fs));
+    Assert.assertEquals(COUNT, reader.getNumberOfRows());
+    RecordReader rows = reader.rows();
+    // get the row index
+    DataReader meta = RecordReaderUtils.createDefaultDataReader(
+        DataReaderProperties.builder()
+            .withBufferSize(reader.getCompressionSize())
+            .withFileSystem(fs)
+            .withPath(testFilePath)
+            .withCompression(reader.getCompressionKind())
+            .withTypeCount(reader.getSchema().getMaximumId() + 1)
+            .withZeroCopy(false)
+            .build());
+    OrcIndex index =
+        meta.readRowIndex(reader.getStripes().get(0), null, null, null, null,
+            null);
+    // check the primitive columns to make sure they have the right number of
+    // items in the first row group
+    for(int c=1; c < 9; ++c) {
+      OrcProto.RowIndex colIndex = index.getRowGroupIndex()[c];
+      assertEquals(1000,
+          colIndex.getEntry(0).getStatistics().getNumberOfValues());
+    }
+    batch = reader.getSchema().createRowBatch();
+    int nextRowInBatch = -1;
+    for(int i=COUNT-1; i >= 0; --i, --nextRowInBatch) {
+      // if we have consumed the previous batch read a new one
+      if (nextRowInBatch < 0) {
+        long base = Math.max(i - 1023, 0);
+        rows.seekToRow(base);
+        Assert.assertEquals("row " + i, true, rows.nextBatch(batch));
+        nextRowInBatch = batch.size - 1;
+      }
+      checkRandomRow(batch, intValues, doubleValues,
+          stringValues, byteValues, words, i, nextRowInBatch);
+    }
+    rows.close();
+    Iterator<StripeInformation> stripeIterator =
+      reader.getStripes().iterator();
+    long offsetOfStripe2 = 0;
+    long offsetOfStripe4 = 0;
+    long lastRowOfStripe2 = 0;
+    for(int i = 0; i < 5; ++i) {
+      StripeInformation stripe = stripeIterator.next();
+      if (i < 2) {
+        lastRowOfStripe2 += stripe.getNumberOfRows();
+      } else if (i == 2) {
+        offsetOfStripe2 = stripe.getOffset();
+        lastRowOfStripe2 += stripe.getNumberOfRows() - 1;
+      } else if (i == 4) {
+        offsetOfStripe4 = stripe.getOffset();
+      }
+    }
+    boolean[] columns = new boolean[reader.getStatistics().length];
+    columns[5] = true; // long colulmn
+    columns[9] = true; // text column
+    rows = reader.rows(new Reader.Options()
+        .range(offsetOfStripe2, offsetOfStripe4 - offsetOfStripe2)
+        .include(columns));
+    rows.seekToRow(lastRowOfStripe2);
+    // we only want two rows
+    batch = reader.getSchema().createRowBatch(2);
+    Assert.assertEquals(true, rows.nextBatch(batch));
+    assertEquals(1, batch.size);
+    assertEquals(intValues[(int) lastRowOfStripe2], getLong(batch, 0));
+    assertEquals(stringValues[(int) lastRowOfStripe2],
+        getText(batch, 0).toString());
+    Assert.assertEquals(true, rows.nextBatch(batch));
+    assertEquals(intValues[(int) lastRowOfStripe2 + 1], getLong(batch, 0));
+    assertEquals(stringValues[(int) lastRowOfStripe2 + 1],
+        getText(batch, 0).toString());
+    rows.close();
+  }
+
+  private void appendRandomRow(VectorizedRowBatch batch,
+                               long[] intValues, double[] doubleValues,
+                               String[] stringValues,
+                               BytesWritable[] byteValues,
+                               String[] words, int i) {
+    InnerStruct inner = new InnerStruct((int) intValues[i], stringValues[i]);
+    InnerStruct inner2 = new InnerStruct((int) (intValues[i] >> 32),
+        words[i % words.length] + "-x");
+    setBigRow(batch, batch.size++, (intValues[i] & 1) == 0, (byte) intValues[i],
+        (short) intValues[i], (int) intValues[i], intValues[i],
+        (float) doubleValues[i], doubleValues[i], byteValues[i], stringValues[i],
+        new MiddleStruct(inner, inner2), list(), map(inner, inner2));
+  }
+
+  private void checkRandomRow(VectorizedRowBatch batch,
+                              long[] intValues, double[] doubleValues,
+                              String[] stringValues,
+                              BytesWritable[] byteValues,
+                              String[] words, int i, int rowInBatch) {
+    InnerStruct inner = new InnerStruct((int) intValues[i], stringValues[i]);
+    InnerStruct inner2 = new InnerStruct((int) (intValues[i] >> 32),
+        words[i % words.length] + "-x");
+    checkBigRow(batch, rowInBatch, i, (intValues[i] & 1) == 0, (byte) intValues[i],
+        (short) intValues[i], (int) intValues[i], intValues[i],
+        (float) doubleValues[i], doubleValues[i], byteValues[i], stringValues[i],
+        new MiddleStruct(inner, inner2), list(), map(inner, inner2));
+  }
+
+  private static class MyMemoryManager extends MemoryManager {
+    final long totalSpace;
+    double rate;
+    Path path = null;
+    long lastAllocation = 0;
+    int rows = 0;
+    Callback callback;
+
+    MyMemoryManager(Configuration conf, long totalSpace, double rate) {
+      super(conf);
+      this.totalSpace = totalSpace;
+      this.rate = rate;
+    }
+
+    @Override
+    public void addWriter(Path path, long requestedAllocation,
+                   Callback callback) {
+      this.path = path;
+      this.lastAllocation = requestedAllocation;
+      this.callback = callback;
+    }
+
+    @Override
+    public synchronized void removeWriter(Path path) {
+      this.path = null;
+      this.lastAllocation = 0;
+    }
+
+    @Override
+    public long getTotalMemoryPool() {
+      return totalSpace;
+    }
+
+    @Override
+    public double getAllocationScale() {
+      return rate;
+    }
+
+    @Override
+    public void addedRow(int count) throws IOException {
+      rows += count;
+      if (rows % 100 == 0) {
+        callback.checkMemory(rate);
+      }
+    }
+  }
+
+  @Test
+  public void testMemoryManagementV11() throws Exception {
+    TypeDescription schema = createInnerSchema();
+    MyMemoryManager memory = new MyMemoryManager(conf, 10000, 0.1);
+    Writer writer = OrcFile.createWriter(testFilePath,
+        OrcFile.writerOptions(conf)
+            .setSchema(schema)
+            .compress(CompressionKind.NONE)
+            .stripeSize(50000)
+            .bufferSize(100)
+            .rowIndexStride(0)
+            .memory(memory)
+            .version(OrcFile.Version.V_0_11));
+    assertEquals(testFilePath, memory.path);
+    VectorizedRowBatch batch = schema.createRowBatch();
+    batch.size = 1;
+    for(int i=0; i < 2500; ++i) {
+      ((LongColumnVector) batch.cols[0]).vector[0] = i * 300;
+      ((BytesColumnVector) batch.cols[1]).setVal(0,
+          Integer.toHexString(10*i).getBytes());
+      writer.addRowBatch(batch);
+    }
+    writer.close();
+    assertEquals(null, memory.path);
+    Reader reader = OrcFile.createReader(testFilePath,
+        OrcFile.readerOptions(conf).filesystem(fs));
+    int i = 0;
+    for(StripeInformation stripe: reader.getStripes()) {
+      i += 1;
+      assertTrue("stripe " + i + " is too long at " + stripe.getDataLength(),
+          stripe.getDataLength() < 5000);
+    }
+    assertEquals(25, i);
+    assertEquals(2500, reader.getNumberOfRows());
+  }
+
+  @Test
+  public void testMemoryManagementV12() throws Exception {
+    TypeDescription schema = createInnerSchema();
+    MyMemoryManager memory = new MyMemoryManager(conf, 10000, 0.1);
+    Writer writer = OrcFile.createWriter(testFilePath,
+                                         OrcFile.writerOptions(conf)
+                                         .setSchema(schema)
+                                         .compress(CompressionKind.NONE)
+                                         .stripeSize(50000)
+                                         .bufferSize(100)
+                                         .rowIndexStride(0)
+                                         .memory(memory)
+                                         .version(OrcFile.Version.V_0_12));
+    VectorizedRowBatch batch = schema.createRowBatch();
+    assertEquals(testFilePath, memory.path);
+    batch.size = 1;
+    for(int i=0; i < 2500; ++i) {
+      ((LongColumnVector) batch.cols[0]).vector[0] = i * 300;
+      ((BytesColumnVector) batch.cols[1]).setVal(0,
+          Integer.toHexString(10*i).getBytes());
+      writer.addRowBatch(batch);
+    }
+    writer.close();
+    assertEquals(null, memory.path);
+    Reader reader = OrcFile.createReader(testFilePath,
+        OrcFile.readerOptions(conf).filesystem(fs));
+    int i = 0;
+    for(StripeInformation stripe: reader.getStripes()) {
+      i += 1;
+      assertTrue("stripe " + i + " is too long at " + stripe.getDataLength(),
+          stripe.getDataLength() < 5000);
+    }
+    // with HIVE-7832, the dictionaries will be disabled after writing the first
+    // stripe as there are too many distinct values. Hence only 3 stripes as
+    // compared to 25 stripes in version 0.11 (above test case)
+    assertEquals(3, i);
+    assertEquals(2500, reader.getNumberOfRows());
+  }
+
+  @Test
+  public void testPredicatePushdown() throws Exception {
+    TypeDescription schema = createInnerSchema();
+    Writer writer = OrcFile.createWriter(testFilePath,
+        OrcFile.writerOptions(conf)
+            .setSchema(schema)
+            .stripeSize(400000L)
+            .compress(CompressionKind.NONE)
+            .bufferSize(500)
+            .rowIndexStride(1000));
+    VectorizedRowBatch batch = schema.createRowBatch();
+    batch.ensureSize(3500);
+    batch.size = 3500;
+    for(int i=0; i < 3500; ++i) {
+      ((LongColumnVector) batch.cols[0]).vector[i] = i * 300;
+      ((BytesColumnVector) batch.cols[1]).setVal(i,
+          Integer.toHexString(10*i).getBytes());
+    }
+    writer.addRowBatch(batch);
+    writer.close();
+    Reader reader = OrcFile.createReader(testFilePath,
+        OrcFile.readerOptions(conf).filesystem(fs));
+    assertEquals(3500, reader.getNumberOfRows());
+
+    SearchArgument sarg = SearchArgumentFactory.newBuilder()
+        .startAnd()
+          .startNot()
+             .lessThan("int1", PredicateLeaf.Type.LONG, 300000L)
+          .end()
+          .lessThan("int1", PredicateLeaf.Type.LONG, 600000L)
+        .end()
+        .build();
+    RecordReader rows = reader.rows(new Reader.Options()
+        .range(0L, Long.MAX_VALUE)
+        .include(new boolean[]{true, true, true})
+        .searchArgument(sarg, new String[]{null, "int1", "string1"}));
+    batch = reader.getSchema().createRowBatch(2000);
+    LongColumnVector ints = (LongColumnVector) batch.cols[0];
+    BytesColumnVector strs = (BytesColumnVector) batch.cols[1];
+
+    Assert.assertEquals(1000L, rows.getRowNumber());
+    Assert.assertEquals(true, rows.nextBatch(batch));
+    assertEquals(1000, batch.size);
+
+    for(int i=1000; i < 2000; ++i) {
+      assertEquals(300 * i, ints.vector[i - 1000]);
+      assertEquals(Integer.toHexString(10*i), strs.toString(i - 1000));
+    }
+    Assert.assertEquals(false, rows.nextBatch(batch));
+    Assert.assertEquals(3500, rows.getRowNumber());
+
+    // look through the file with no rows selected
+    sarg = SearchArgumentFactory.newBuilder()
+        .startAnd()
+          .lessThan("int1", PredicateLeaf.Type.LONG, 0L)
+        .end()
+        .build();
+    rows = reader.rows(new Reader.Options()
+        .range(0L, Long.MAX_VALUE)
+        .include(new boolean[]{true, true, true})
+        .searchArgument(sarg, new String[]{null, "int1", "string1"}));
+    Assert.assertEquals(3500L, rows.getRowNumber());
+    assertTrue(!rows.nextBatch(batch));
+
+    // select first 100 and last 100 rows
+    sarg = SearchArgumentFactory.newBuilder()
+        .startOr()
+          .lessThan("int1", PredicateLeaf.Type.LONG, 300L * 100)
+          .startNot()
+            .lessThan("int1", PredicateLeaf.Type.LONG, 300L * 3400)
+          .end()
+        .end()
+        .build();
+    rows = reader.rows(new Reader.Options()
+        .range(0L, Long.MAX_VALUE)
+        .include(new boolean[]{true, true, true})
+        .searchArgument(sarg, new String[]{null, "int1", "string1"}));
+    Assert.assertEquals(0, rows.getRowNumber());
+    Assert.assertEquals(true, rows.nextBatch(batch));
+    assertEquals(1000, batch.size);
+    Assert.assertEquals(3000, rows.getRowNumber());
+    for(int i=0; i < 1000; ++i) {
+      assertEquals(300 * i, ints.vector[i]);
+      assertEquals(Integer.toHexString(10*i), strs.toString(i));
+    }
+
+    Assert.assertEquals(true, rows.nextBatch(batch));
+    assertEquals(500, batch.size);
+    Assert.assertEquals(3500, rows.getRowNumber());
+    for(int i=3000; i < 3500; ++i) {
+      assertEquals(300 * i, ints.vector[i - 3000]);
+      assertEquals(Integer.toHexString(10*i), strs.toString(i - 3000));
+    }
+    Assert.assertEquals(false, rows.nextBatch(batch));
+    Assert.assertEquals(3500, rows.getRowNumber());
+  }
+
+  /**
+   * Test all of the types that have distinct ORC writers using the vectorized
+   * writer with different combinations of repeating and null values.
+   * @throws Exception
+   */
+  @Test


<TRUNCATED>

[15/27] hive git commit: HIVE-11417. Move the ReaderImpl and RowReaderImpl to the ORC module, by making shims for the row by row reader. (omalley reviewed by prasanth_j)

Posted by om...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/orc/src/test/resources/orc-file-dump.json
----------------------------------------------------------------------
diff --git a/orc/src/test/resources/orc-file-dump.json b/orc/src/test/resources/orc-file-dump.json
new file mode 100644
index 0000000..bf654a1
--- /dev/null
+++ b/orc/src/test/resources/orc-file-dump.json
@@ -0,0 +1,1355 @@
+{
+  "fileName": "TestFileDump.testDump.orc",
+  "fileVersion": "0.12",
+  "writerVersion": "HIVE_13083",
+  "numberOfRows": 21000,
+  "compression": "ZLIB",
+  "compressionBufferSize": 4096,
+  "schemaString": "struct<i:int,l:bigint,s:string>",
+  "schema": [
+    {
+      "columnId": 0,
+      "columnType": "STRUCT",
+      "childColumnNames": [
+        "i",
+        "l",
+        "s"
+      ],
+      "childColumnIds": [
+        1,
+        2,
+        3
+      ]
+    },
+    {
+      "columnId": 1,
+      "columnType": "INT"
+    },
+    {
+      "columnId": 2,
+      "columnType": "LONG"
+    },
+    {
+      "columnId": 3,
+      "columnType": "STRING"
+    }
+  ],
+  "stripeStatistics": [
+    {
+      "stripeNumber": 1,
+      "columnStatistics": [
+        {
+          "columnId": 0,
+          "count": 5000,
+          "hasNull": false
+        },
+        {
+          "columnId": 1,
+          "count": 5000,
+          "hasNull": false,
+          "min": -2147115959,
+          "max": 2145210552,
+          "sum": 50111854553,
+          "type": "LONG"
+        },
+        {
+          "columnId": 2,
+          "count": 5000,
+          "hasNull": false,
+          "min": -9223180583305557329,
+          "max": 9221614132680747961,
+          "type": "LONG"
+        },
+        {
+          "columnId": 3,
+          "count": 4950,
+          "hasNull": true,
+          "min": "Darkness,",
+          "max": "worst",
+          "totalLength": 19283,
+          "type": "STRING"
+        }
+      ]
+    },
+    {
+      "stripeNumber": 2,
+      "columnStatistics": [
+        {
+          "columnId": 0,
+          "count": 5000,
+          "hasNull": false
+        },
+        {
+          "columnId": 1,
+          "count": 5000,
+          "hasNull": false,
+          "min": -2147390285,
+          "max": 2147224606,
+          "sum": -22290798217,
+          "type": "LONG"
+        },
+        {
+          "columnId": 2,
+          "count": 5000,
+          "hasNull": false,
+          "min": -9219295160509160427,
+          "max": 9217571024994660020,
+          "type": "LONG"
+        },
+        {
+          "columnId": 3,
+          "count": 4950,
+          "hasNull": true,
+          "min": "Darkness,",
+          "max": "worst",
+          "totalLength": 19397,
+          "type": "STRING"
+        }
+      ]
+    },
+    {
+      "stripeNumber": 3,
+      "columnStatistics": [
+        {
+          "columnId": 0,
+          "count": 5000,
+          "hasNull": false
+        },
+        {
+          "columnId": 1,
+          "count": 5000,
+          "hasNull": false,
+          "min": -2146954065,
+          "max": 2146722468,
+          "sum": 20639652136,
+          "type": "LONG"
+        },
+        {
+          "columnId": 2,
+          "count": 5000,
+          "hasNull": false,
+          "min": -9214076359988107846,
+          "max": 9222919052987871506,
+          "type": "LONG"
+        },
+        {
+          "columnId": 3,
+          "count": 4950,
+          "hasNull": true,
+          "min": "Darkness,",
+          "max": "worst",
+          "totalLength": 19031,
+          "type": "STRING"
+        }
+      ]
+    },
+    {
+      "stripeNumber": 4,
+      "columnStatistics": [
+        {
+          "columnId": 0,
+          "count": 5000,
+          "hasNull": false
+        },
+        {
+          "columnId": 1,
+          "count": 5000,
+          "hasNull": false,
+          "min": -2146969085,
+          "max": 2146025044,
+          "sum": -5156814387,
+          "type": "LONG"
+        },
+        {
+          "columnId": 2,
+          "count": 5000,
+          "hasNull": false,
+          "min": -9222731174895935707,
+          "max": 9220625004936875965,
+          "type": "LONG"
+        },
+        {
+          "columnId": 3,
+          "count": 4950,
+          "hasNull": true,
+          "min": "Darkness,",
+          "max": "worst",
+          "totalLength": 19459,
+          "type": "STRING"
+        }
+      ]
+    },
+    {
+      "stripeNumber": 5,
+      "columnStatistics": [
+        {
+          "columnId": 0,
+          "count": 1000,
+          "hasNull": false
+        },
+        {
+          "columnId": 1,
+          "count": 1000,
+          "hasNull": false,
+          "min": -2144303438,
+          "max": 2127599049,
+          "sum": 62841564778,
+          "type": "LONG"
+        },
+        {
+          "columnId": 2,
+          "count": 1000,
+          "hasNull": false,
+          "min": -9195133638801798919,
+          "max": 9218626063131504414,
+          "type": "LONG"
+        },
+        {
+          "columnId": 3,
+          "count": 990,
+          "hasNull": true,
+          "min": "Darkness,",
+          "max": "worst",
+          "totalLength": 3963,
+          "type": "STRING"
+        }
+      ]
+    }
+  ],
+  "fileStatistics": [
+    {
+      "columnId": 0,
+      "count": 21000,
+      "hasNull": false
+    },
+    {
+      "columnId": 1,
+      "count": 21000,
+      "hasNull": false,
+      "min": -2147390285,
+      "max": 2147224606,
+      "sum": 106145458863,
+      "type": "LONG"
+    },
+    {
+      "columnId": 2,
+      "count": 21000,
+      "hasNull": false,
+      "min": -9223180583305557329,
+      "max": 9222919052987871506,
+      "type": "LONG"
+    },
+    {
+      "columnId": 3,
+      "count": 20790,
+      "hasNull": true,
+      "min": "Darkness,",
+      "max": "worst",
+      "totalLength": 81133,
+      "type": "STRING"
+    }
+  ],
+  "stripes": [
+    {
+      "stripeNumber": 1,
+      "stripeInformation": {
+        "offset": 3,
+        "indexLength": 970,
+        "dataLength": 63770,
+        "footerLength": 90,
+        "rowCount": 5000
+      },
+      "streams": [
+        {
+          "columnId": 0,
+          "section": "ROW_INDEX",
+          "startOffset": 3,
+          "length": 17
+        },
+        {
+          "columnId": 1,
+          "section": "ROW_INDEX",
+          "startOffset": 20,
+          "length": 167
+        },
+        {
+          "columnId": 2,
+          "section": "ROW_INDEX",
+          "startOffset": 187,
+          "length": 171
+        },
+        {
+          "columnId": 3,
+          "section": "ROW_INDEX",
+          "startOffset": 358,
+          "length": 103
+        },
+        {
+          "columnId": 3,
+          "section": "BLOOM_FILTER",
+          "startOffset": 461,
+          "length": 512
+        },
+        {
+          "columnId": 1,
+          "section": "DATA",
+          "startOffset": 973,
+          "length": 20035
+        },
+        {
+          "columnId": 2,
+          "section": "DATA",
+          "startOffset": 21008,
+          "length": 40050
+        },
+        {
+          "columnId": 3,
+          "section": "PRESENT",
+          "startOffset": 61058,
+          "length": 17
+        },
+        {
+          "columnId": 3,
+          "section": "DATA",
+          "startOffset": 61075,
+          "length": 3510
+        },
+        {
+          "columnId": 3,
+          "section": "LENGTH",
+          "startOffset": 64585,
+          "length": 25
+        },
+        {
+          "columnId": 3,
+          "section": "DICTIONARY_DATA",
+          "startOffset": 64610,
+          "length": 133
+        }
+      ],
+      "encodings": [
+        {
+          "columnId": 0,
+          "kind": "DIRECT"
+        },
+        {
+          "columnId": 1,
+          "kind": "DIRECT_V2"
+        },
+        {
+          "columnId": 2,
+          "kind": "DIRECT_V2"
+        },
+        {
+          "columnId": 3,
+          "kind": "DICTIONARY_V2",
+          "dictionarySize": 35
+        }
+      ],
+      "indexes": [{
+        "columnId": 3,
+        "rowGroupIndexes": [
+          {
+            "entryId": 0,
+            "count": 990,
+            "hasNull": true,
+            "min": "Darkness,",
+            "max": "worst",
+            "totalLength": 3873,
+            "type": "STRING",
+            "positions": [
+              0,
+              0,
+              0,
+              0,
+              0,
+              0,
+              0
+            ]
+          },
+          {
+            "entryId": 1,
+            "count": 990,
+            "hasNull": true,
+            "min": "Darkness,",
+            "max": "worst",
+            "totalLength": 3861,
+            "type": "STRING",
+            "positions": [
+              0,
+              38,
+              12,
+              0,
+              0,
+              736,
+              23
+            ]
+          },
+          {
+            "entryId": 2,
+            "count": 990,
+            "hasNull": true,
+            "min": "Darkness,",
+            "max": "worst",
+            "totalLength": 3946,
+            "type": "STRING",
+            "positions": [
+              0,
+              78,
+              12,
+              0,
+              0,
+              1473,
+              43
+            ]
+          },
+          {
+            "entryId": 3,
+            "count": 990,
+            "hasNull": true,
+            "min": "Darkness,",
+            "max": "worst",
+            "totalLength": 3774,
+            "type": "STRING",
+            "positions": [
+              0,
+              118,
+              12,
+              0,
+              0,
+              2067,
+              261
+            ]
+          },
+          {
+            "entryId": 4,
+            "count": 990,
+            "hasNull": true,
+            "min": "Darkness,",
+            "max": "worst",
+            "totalLength": 3829,
+            "type": "STRING",
+            "positions": [
+              0,
+              158,
+              12,
+              0,
+              0,
+              2992,
+              35
+            ]
+          }
+        ],
+        "bloomFilterIndexes": [
+          {
+            "entryId": 0,
+            "numHashFunctions": 4,
+            "bitCount": 6272,
+            "popCount": 138,
+            "loadFactor": 0.022002551704645157,
+            "expectedFpp": 2.3436470542037569E-7
+          },
+          {
+            "entryId": 1,
+            "numHashFunctions": 4,
+            "bitCount": 6272,
+            "popCount": 138,
+            "loadFactor": 0.022002551704645157,
+            "expectedFpp": 2.3436470542037569E-7
+          },
+          {
+            "entryId": 2,
+            "numHashFunctions": 4,
+            "bitCount": 6272,
+            "popCount": 138,
+            "loadFactor": 0.022002551704645157,
+            "expectedFpp": 2.3436470542037569E-7
+          },
+          {
+            "entryId": 3,
+            "numHashFunctions": 4,
+            "bitCount": 6272,
+            "popCount": 138,
+            "loadFactor": 0.022002551704645157,
+            "expectedFpp": 2.3436470542037569E-7
+          },
+          {
+            "entryId": 4,
+            "numHashFunctions": 4,
+            "bitCount": 6272,
+            "popCount": 138,
+            "loadFactor": 0.022002551704645157,
+            "expectedFpp": 2.3436470542037569E-7
+          }
+        ],
+        "stripeLevelBloomFilter": {
+          "numHashFunctions": 4,
+          "bitCount": 6272,
+          "popCount": 138,
+          "loadFactor": 0.022002551704645157,
+          "expectedFpp": 2.3436470542037569E-7
+        }
+      }]
+    },
+    {
+      "stripeNumber": 2,
+      "stripeInformation": {
+        "offset": 64833,
+        "indexLength": 961,
+        "dataLength": 63763,
+        "footerLength": 88,
+        "rowCount": 5000
+      },
+      "streams": [
+        {
+          "columnId": 0,
+          "section": "ROW_INDEX",
+          "startOffset": 64833,
+          "length": 17
+        },
+        {
+          "columnId": 1,
+          "section": "ROW_INDEX",
+          "startOffset": 64850,
+          "length": 166
+        },
+        {
+          "columnId": 2,
+          "section": "ROW_INDEX",
+          "startOffset": 65016,
+          "length": 166
+        },
+        {
+          "columnId": 3,
+          "section": "ROW_INDEX",
+          "startOffset": 65182,
+          "length": 100
+        },
+        {
+          "columnId": 3,
+          "section": "BLOOM_FILTER",
+          "startOffset": 65282,
+          "length": 512
+        },
+        {
+          "columnId": 1,
+          "section": "DATA",
+          "startOffset": 65794,
+          "length": 20035
+        },
+        {
+          "columnId": 2,
+          "section": "DATA",
+          "startOffset": 85829,
+          "length": 40050
+        },
+        {
+          "columnId": 3,
+          "section": "PRESENT",
+          "startOffset": 125879,
+          "length": 17
+        },
+        {
+          "columnId": 3,
+          "section": "DATA",
+          "startOffset": 125896,
+          "length": 3503
+        },
+        {
+          "columnId": 3,
+          "section": "LENGTH",
+          "startOffset": 129399,
+          "length": 25
+        },
+        {
+          "columnId": 3,
+          "section": "DICTIONARY_DATA",
+          "startOffset": 129424,
+          "length": 133
+        }
+      ],
+      "encodings": [
+        {
+          "columnId": 0,
+          "kind": "DIRECT"
+        },
+        {
+          "columnId": 1,
+          "kind": "DIRECT_V2"
+        },
+        {
+          "columnId": 2,
+          "kind": "DIRECT_V2"
+        },
+        {
+          "columnId": 3,
+          "kind": "DICTIONARY_V2",
+          "dictionarySize": 35
+        }
+      ],
+      "indexes": [{
+        "columnId": 3,
+        "rowGroupIndexes": [
+          {
+            "entryId": 0,
+            "count": 990,
+            "hasNull": true,
+            "min": "Darkness,",
+            "max": "worst",
+            "totalLength": 3946,
+            "type": "STRING",
+            "positions": [
+              0,
+              0,
+              0,
+              0,
+              0,
+              0,
+              0
+            ]
+          },
+          {
+            "entryId": 1,
+            "count": 990,
+            "hasNull": true,
+            "min": "Darkness,",
+            "max": "worst",
+            "totalLength": 3836,
+            "type": "STRING",
+            "positions": [
+              0,
+              38,
+              12,
+              0,
+              0,
+              746,
+              11
+            ]
+          },
+          {
+            "entryId": 2,
+            "count": 990,
+            "hasNull": true,
+            "min": "Darkness,",
+            "max": "worst",
+            "totalLength": 3791,
+            "type": "STRING",
+            "positions": [
+              0,
+              78,
+              12,
+              0,
+              0,
+              1430,
+              95
+            ]
+          },
+          {
+            "entryId": 3,
+            "count": 990,
+            "hasNull": true,
+            "min": "Darkness,",
+            "max": "worst",
+            "totalLength": 3904,
+            "type": "STRING",
+            "positions": [
+              0,
+              118,
+              12,
+              0,
+              0,
+              2239,
+              23
+            ]
+          },
+          {
+            "entryId": 4,
+            "count": 990,
+            "hasNull": true,
+            "min": "Darkness,",
+            "max": "worst",
+            "totalLength": 3920,
+            "type": "STRING",
+            "positions": [
+              0,
+              158,
+              12,
+              0,
+              0,
+              2994,
+              17
+            ]
+          }
+        ],
+        "bloomFilterIndexes": [
+          {
+            "entryId": 0,
+            "numHashFunctions": 4,
+            "bitCount": 6272,
+            "popCount": 138,
+            "loadFactor": 0.022002551704645157,
+            "expectedFpp": 2.3436470542037569E-7
+          },
+          {
+            "entryId": 1,
+            "numHashFunctions": 4,
+            "bitCount": 6272,
+            "popCount": 138,
+            "loadFactor": 0.022002551704645157,
+            "expectedFpp": 2.3436470542037569E-7
+          },
+          {
+            "entryId": 2,
+            "numHashFunctions": 4,
+            "bitCount": 6272,
+            "popCount": 138,
+            "loadFactor": 0.022002551704645157,
+            "expectedFpp": 2.3436470542037569E-7
+          },
+          {
+            "entryId": 3,
+            "numHashFunctions": 4,
+            "bitCount": 6272,
+            "popCount": 138,
+            "loadFactor": 0.022002551704645157,
+            "expectedFpp": 2.3436470542037569E-7
+          },
+          {
+            "entryId": 4,
+            "numHashFunctions": 4,
+            "bitCount": 6272,
+            "popCount": 138,
+            "loadFactor": 0.022002551704645157,
+            "expectedFpp": 2.3436470542037569E-7
+          }
+        ],
+        "stripeLevelBloomFilter": {
+          "numHashFunctions": 4,
+          "bitCount": 6272,
+          "popCount": 138,
+          "loadFactor": 0.022002551704645157,
+          "expectedFpp": 2.3436470542037569E-7
+        }
+      }]
+    },
+    {
+      "stripeNumber": 3,
+      "stripeInformation": {
+        "offset": 129645,
+        "indexLength": 962,
+        "dataLength": 63770,
+        "footerLength": 91,
+        "rowCount": 5000
+      },
+      "streams": [
+        {
+          "columnId": 0,
+          "section": "ROW_INDEX",
+          "startOffset": 129645,
+          "length": 17
+        },
+        {
+          "columnId": 1,
+          "section": "ROW_INDEX",
+          "startOffset": 129662,
+          "length": 164
+        },
+        {
+          "columnId": 2,
+          "section": "ROW_INDEX",
+          "startOffset": 129826,
+          "length": 167
+        },
+        {
+          "columnId": 3,
+          "section": "ROW_INDEX",
+          "startOffset": 129993,
+          "length": 102
+        },
+        {
+          "columnId": 3,
+          "section": "BLOOM_FILTER",
+          "startOffset": 130095,
+          "length": 512
+        },
+        {
+          "columnId": 1,
+          "section": "DATA",
+          "startOffset": 130607,
+          "length": 20035
+        },
+        {
+          "columnId": 2,
+          "section": "DATA",
+          "startOffset": 150642,
+          "length": 40050
+        },
+        {
+          "columnId": 3,
+          "section": "PRESENT",
+          "startOffset": 190692,
+          "length": 17
+        },
+        {
+          "columnId": 3,
+          "section": "DATA",
+          "startOffset": 190709,
+          "length": 3510
+        },
+        {
+          "columnId": 3,
+          "section": "LENGTH",
+          "startOffset": 194219,
+          "length": 25
+        },
+        {
+          "columnId": 3,
+          "section": "DICTIONARY_DATA",
+          "startOffset": 194244,
+          "length": 133
+        }
+      ],
+      "encodings": [
+        {
+          "columnId": 0,
+          "kind": "DIRECT"
+        },
+        {
+          "columnId": 1,
+          "kind": "DIRECT_V2"
+        },
+        {
+          "columnId": 2,
+          "kind": "DIRECT_V2"
+        },
+        {
+          "columnId": 3,
+          "kind": "DICTIONARY_V2",
+          "dictionarySize": 35
+        }
+      ],
+      "indexes": [{
+        "columnId": 3,
+        "rowGroupIndexes": [
+          {
+            "entryId": 0,
+            "count": 990,
+            "hasNull": true,
+            "min": "Darkness,",
+            "max": "worst",
+            "totalLength": 3829,
+            "type": "STRING",
+            "positions": [
+              0,
+              0,
+              0,
+              0,
+              0,
+              0,
+              0
+            ]
+          },
+          {
+            "entryId": 1,
+            "count": 990,
+            "hasNull": true,
+            "min": "Darkness,",
+            "max": "worst",
+            "totalLength": 3853,
+            "type": "STRING",
+            "positions": [
+              0,
+              38,
+              12,
+              0,
+              0,
+              698,
+              74
+            ]
+          },
+          {
+            "entryId": 2,
+            "count": 990,
+            "hasNull": true,
+            "min": "Darkness,",
+            "max": "worst",
+            "totalLength": 3796,
+            "type": "STRING",
+            "positions": [
+              0,
+              78,
+              12,
+              0,
+              0,
+              1483,
+              39
+            ]
+          },
+          {
+            "entryId": 3,
+            "count": 990,
+            "hasNull": true,
+            "min": "Darkness,",
+            "max": "worst",
+            "totalLength": 3736,
+            "type": "STRING",
+            "positions": [
+              0,
+              118,
+              12,
+              0,
+              0,
+              2148,
+              155
+            ]
+          },
+          {
+            "entryId": 4,
+            "count": 990,
+            "hasNull": true,
+            "min": "Darkness,",
+            "max": "worst",
+            "totalLength": 3817,
+            "type": "STRING",
+            "positions": [
+              0,
+              158,
+              12,
+              0,
+              0,
+              3018,
+              8
+            ]
+          }
+        ],
+        "bloomFilterIndexes": [
+          {
+            "entryId": 0,
+            "numHashFunctions": 4,
+            "bitCount": 6272,
+            "popCount": 138,
+            "loadFactor": 0.022002551704645157,
+            "expectedFpp": 2.3436470542037569E-7
+          },
+          {
+            "entryId": 1,
+            "numHashFunctions": 4,
+            "bitCount": 6272,
+            "popCount": 138,
+            "loadFactor": 0.022002551704645157,
+            "expectedFpp": 2.3436470542037569E-7
+          },
+          {
+            "entryId": 2,
+            "numHashFunctions": 4,
+            "bitCount": 6272,
+            "popCount": 138,
+            "loadFactor": 0.022002551704645157,
+            "expectedFpp": 2.3436470542037569E-7
+          },
+          {
+            "entryId": 3,
+            "numHashFunctions": 4,
+            "bitCount": 6272,
+            "popCount": 138,
+            "loadFactor": 0.022002551704645157,
+            "expectedFpp": 2.3436470542037569E-7
+          },
+          {
+            "entryId": 4,
+            "numHashFunctions": 4,
+            "bitCount": 6272,
+            "popCount": 138,
+            "loadFactor": 0.022002551704645157,
+            "expectedFpp": 2.3436470542037569E-7
+          }
+        ],
+        "stripeLevelBloomFilter": {
+          "numHashFunctions": 4,
+          "bitCount": 6272,
+          "popCount": 138,
+          "loadFactor": 0.022002551704645157,
+          "expectedFpp": 2.3436470542037569E-7
+        }
+      }]
+    },
+    {
+      "stripeNumber": 4,
+      "stripeInformation": {
+        "offset": 194468,
+        "indexLength": 973,
+        "dataLength": 63756,
+        "footerLength": 91,
+        "rowCount": 5000
+      },
+      "streams": [
+        {
+          "columnId": 0,
+          "section": "ROW_INDEX",
+          "startOffset": 194468,
+          "length": 17
+        },
+        {
+          "columnId": 1,
+          "section": "ROW_INDEX",
+          "startOffset": 194485,
+          "length": 166
+        },
+        {
+          "columnId": 2,
+          "section": "ROW_INDEX",
+          "startOffset": 194651,
+          "length": 171
+        },
+        {
+          "columnId": 3,
+          "section": "ROW_INDEX",
+          "startOffset": 194822,
+          "length": 107
+        },
+        {
+          "columnId": 3,
+          "section": "BLOOM_FILTER",
+          "startOffset": 194929,
+          "length": 512
+        },
+        {
+          "columnId": 1,
+          "section": "DATA",
+          "startOffset": 195441,
+          "length": 20035
+        },
+        {
+          "columnId": 2,
+          "section": "DATA",
+          "startOffset": 215476,
+          "length": 40050
+        },
+        {
+          "columnId": 3,
+          "section": "PRESENT",
+          "startOffset": 255526,
+          "length": 17
+        },
+        {
+          "columnId": 3,
+          "section": "DATA",
+          "startOffset": 255543,
+          "length": 3496
+        },
+        {
+          "columnId": 3,
+          "section": "LENGTH",
+          "startOffset": 259039,
+          "length": 25
+        },
+        {
+          "columnId": 3,
+          "section": "DICTIONARY_DATA",
+          "startOffset": 259064,
+          "length": 133
+        }
+      ],
+      "encodings": [
+        {
+          "columnId": 0,
+          "kind": "DIRECT"
+        },
+        {
+          "columnId": 1,
+          "kind": "DIRECT_V2"
+        },
+        {
+          "columnId": 2,
+          "kind": "DIRECT_V2"
+        },
+        {
+          "columnId": 3,
+          "kind": "DICTIONARY_V2",
+          "dictionarySize": 35
+        }
+      ],
+      "indexes": [{
+        "columnId": 3,
+        "rowGroupIndexes": [
+          {
+            "entryId": 0,
+            "count": 990,
+            "hasNull": true,
+            "min": "Darkness,",
+            "max": "worst",
+            "totalLength": 3959,
+            "type": "STRING",
+            "positions": [
+              0,
+              0,
+              0,
+              0,
+              0,
+              0,
+              0
+            ]
+          },
+          {
+            "entryId": 1,
+            "count": 990,
+            "hasNull": true,
+            "min": "Darkness,",
+            "max": "worst",
+            "totalLength": 3816,
+            "type": "STRING",
+            "positions": [
+              0,
+              38,
+              12,
+              0,
+              0,
+              495,
+              338
+            ]
+          },
+          {
+            "entryId": 2,
+            "count": 990,
+            "hasNull": true,
+            "min": "Darkness,",
+            "max": "worst",
+            "totalLength": 3883,
+            "type": "STRING",
+            "positions": [
+              0,
+              78,
+              12,
+              0,
+              0,
+              1449,
+              71
+            ]
+          },
+          {
+            "entryId": 3,
+            "count": 990,
+            "hasNull": true,
+            "min": "Darkness,",
+            "max": "worst",
+            "totalLength": 3938,
+            "type": "STRING",
+            "positions": [
+              0,
+              118,
+              12,
+              0,
+              0,
+              2207,
+              59
+            ]
+          },
+          {
+            "entryId": 4,
+            "count": 990,
+            "hasNull": true,
+            "min": "Darkness,",
+            "max": "worst",
+            "totalLength": 3863,
+            "type": "STRING",
+            "positions": [
+              0,
+              158,
+              12,
+              0,
+              0,
+              2838,
+              223
+            ]
+          }
+        ],
+        "bloomFilterIndexes": [
+          {
+            "entryId": 0,
+            "numHashFunctions": 4,
+            "bitCount": 6272,
+            "popCount": 138,
+            "loadFactor": 0.022002551704645157,
+            "expectedFpp": 2.3436470542037569E-7
+          },
+          {
+            "entryId": 1,
+            "numHashFunctions": 4,
+            "bitCount": 6272,
+            "popCount": 138,
+            "loadFactor": 0.022002551704645157,
+            "expectedFpp": 2.3436470542037569E-7
+          },
+          {
+            "entryId": 2,
+            "numHashFunctions": 4,
+            "bitCount": 6272,
+            "popCount": 138,
+            "loadFactor": 0.022002551704645157,
+            "expectedFpp": 2.3436470542037569E-7
+          },
+          {
+            "entryId": 3,
+            "numHashFunctions": 4,
+            "bitCount": 6272,
+            "popCount": 138,
+            "loadFactor": 0.022002551704645157,
+            "expectedFpp": 2.3436470542037569E-7
+          },
+          {
+            "entryId": 4,
+            "numHashFunctions": 4,
+            "bitCount": 6272,
+            "popCount": 138,
+            "loadFactor": 0.022002551704645157,
+            "expectedFpp": 2.3436470542037569E-7
+          }
+        ],
+        "stripeLevelBloomFilter": {
+          "numHashFunctions": 4,
+          "bitCount": 6272,
+          "popCount": 138,
+          "loadFactor": 0.022002551704645157,
+          "expectedFpp": 2.3436470542037569E-7
+        }
+      }]
+    },
+    {
+      "stripeNumber": 5,
+      "stripeInformation": {
+        "offset": 259288,
+        "indexLength": 433,
+        "dataLength": 12943,
+        "footerLength": 83,
+        "rowCount": 1000
+      },
+      "streams": [
+        {
+          "columnId": 0,
+          "section": "ROW_INDEX",
+          "startOffset": 259288,
+          "length": 12
+        },
+        {
+          "columnId": 1,
+          "section": "ROW_INDEX",
+          "startOffset": 259300,
+          "length": 38
+        },
+        {
+          "columnId": 2,
+          "section": "ROW_INDEX",
+          "startOffset": 259338,
+          "length": 41
+        },
+        {
+          "columnId": 3,
+          "section": "ROW_INDEX",
+          "startOffset": 259379,
+          "length": 41
+        },
+        {
+          "columnId": 3,
+          "section": "BLOOM_FILTER",
+          "startOffset": 259420,
+          "length": 301
+        },
+        {
+          "columnId": 1,
+          "section": "DATA",
+          "startOffset": 259721,
+          "length": 4007
+        },
+        {
+          "columnId": 2,
+          "section": "DATA",
+          "startOffset": 263728,
+          "length": 8010
+        },
+        {
+          "columnId": 3,
+          "section": "PRESENT",
+          "startOffset": 271738,
+          "length": 16
+        },
+        {
+          "columnId": 3,
+          "section": "DATA",
+          "startOffset": 271754,
+          "length": 752
+        },
+        {
+          "columnId": 3,
+          "section": "LENGTH",
+          "startOffset": 272506,
+          "length": 25
+        },
+        {
+          "columnId": 3,
+          "section": "DICTIONARY_DATA",
+          "startOffset": 272531,
+          "length": 133
+        }
+      ],
+      "encodings": [
+        {
+          "columnId": 0,
+          "kind": "DIRECT"
+        },
+        {
+          "columnId": 1,
+          "kind": "DIRECT_V2"
+        },
+        {
+          "columnId": 2,
+          "kind": "DIRECT_V2"
+        },
+        {
+          "columnId": 3,
+          "kind": "DICTIONARY_V2",
+          "dictionarySize": 35
+        }
+      ],
+      "indexes": [{
+        "columnId": 3,
+        "rowGroupIndexes": [{
+          "entryId": 0,
+          "count": 990,
+          "hasNull": true,
+          "min": "Darkness,",
+          "max": "worst",
+          "totalLength": 3963,
+          "type": "STRING",
+          "positions": [
+            0,
+            0,
+            0,
+            0,
+            0,
+            0,
+            0
+          ]
+        }],
+        "bloomFilterIndexes": [{
+          "entryId": 0,
+          "numHashFunctions": 4,
+          "bitCount": 6272,
+          "popCount": 138,
+          "loadFactor": 0.022002551704645157,
+          "expectedFpp": 2.3436470542037569E-7
+        }],
+        "stripeLevelBloomFilter": {
+          "numHashFunctions": 4,
+          "bitCount": 6272,
+          "popCount": 138,
+          "loadFactor": 0.022002551704645157,
+          "expectedFpp": 2.3436470542037569E-7
+        }
+      }]
+    }
+  ],
+  "fileLength": 273300,
+  "paddingLength": 0,
+  "paddingRatio": 0,
+  "status": "OK"
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/orc/src/test/resources/orc-file-dump.out
----------------------------------------------------------------------
diff --git a/orc/src/test/resources/orc-file-dump.out b/orc/src/test/resources/orc-file-dump.out
new file mode 100644
index 0000000..70f7fbd
--- /dev/null
+++ b/orc/src/test/resources/orc-file-dump.out
@@ -0,0 +1,195 @@
+Structure for TestFileDump.testDump.orc
+File Version: 0.12 with HIVE_13083
+Rows: 21000
+Compression: ZLIB
+Compression size: 4096
+Type: struct<i:int,l:bigint,s:string>
+
+Stripe Statistics:
+  Stripe 1:
+    Column 0: count: 5000 hasNull: false
+    Column 1: count: 5000 hasNull: false min: -2146021688 max: 2147223299 sum: 515792826
+    Column 2: count: 5000 hasNull: false min: -9218592812243954469 max: 9221614132680747961
+    Column 3: count: 5000 hasNull: false min: Darkness, max: worst sum: 19280
+  Stripe 2:
+    Column 0: count: 5000 hasNull: false
+    Column 1: count: 5000 hasNull: false min: -2146733128 max: 2147001622 sum: 7673427
+    Column 2: count: 5000 hasNull: false min: -9220818777591257749 max: 9222259462014003839
+    Column 3: count: 5000 hasNull: false min: Darkness, max: worst sum: 19504
+  Stripe 3:
+    Column 0: count: 5000 hasNull: false
+    Column 1: count: 5000 hasNull: false min: -2146993718 max: 2147378179 sum: 132660742551
+    Column 2: count: 5000 hasNull: false min: -9218342074710552826 max: 9222303228623055266
+    Column 3: count: 5000 hasNull: false min: Darkness, max: worst sum: 19641
+  Stripe 4:
+    Column 0: count: 5000 hasNull: false
+    Column 1: count: 5000 hasNull: false min: -2146658006 max: 2145520931 sum: 8533549236
+    Column 2: count: 5000 hasNull: false min: -9222758097219661129 max: 9221043130193737406
+    Column 3: count: 5000 hasNull: false min: Darkness, max: worst sum: 19470
+  Stripe 5:
+    Column 0: count: 1000 hasNull: false
+    Column 1: count: 1000 hasNull: false min: -2146245500 max: 2146378640 sum: 51299706363
+    Column 2: count: 1000 hasNull: false min: -9208193203370316142 max: 9218567213558056476
+    Column 3: count: 1000 hasNull: false min: Darkness, max: worst sum: 3866
+
+File Statistics:
+  Column 0: count: 21000 hasNull: false
+  Column 1: count: 21000 hasNull: false min: -2146993718 max: 2147378179 sum: 193017464403
+  Column 2: count: 21000 hasNull: false min: -9222758097219661129 max: 9222303228623055266
+  Column 3: count: 21000 hasNull: false min: Darkness, max: worst sum: 81761
+
+Stripes:
+  Stripe: offset: 3 data: 63786 rows: 5000 tail: 79 index: 439
+    Stream: column 0 section ROW_INDEX start: 3 length 17
+    Stream: column 1 section ROW_INDEX start: 20 length 166
+    Stream: column 2 section ROW_INDEX start: 186 length 169
+    Stream: column 3 section ROW_INDEX start: 355 length 87
+    Stream: column 1 section DATA start: 442 length 20035
+    Stream: column 2 section DATA start: 20477 length 40050
+    Stream: column 3 section DATA start: 60527 length 3543
+    Stream: column 3 section LENGTH start: 64070 length 25
+    Stream: column 3 section DICTIONARY_DATA start: 64095 length 133
+    Encoding column 0: DIRECT
+    Encoding column 1: DIRECT_V2
+    Encoding column 2: DIRECT_V2
+    Encoding column 3: DICTIONARY_V2[35]
+    Row group indices for column 1:
+      Entry 0: count: 1000 hasNull: false min: -2145365268 max: 2135491313 sum: 7521792925 positions: 0,0,0
+      Entry 1: count: 1000 hasNull: false min: -2139452528 max: 2147223299 sum: -12923774313 positions: 0,2050,488
+      Entry 2: count: 1000 hasNull: false min: -2142420586 max: 2143898386 sum: -25521983511 positions: 4099,2054,464
+      Entry 3: count: 1000 hasNull: false min: -2137233441 max: 2144267163 sum: 40993386199 positions: 8198,2058,440
+      Entry 4: count: 1000 hasNull: false min: -2146021688 max: 2146838901 sum: -9553628474 positions: 12297,2062,416
+    Row group indices for column 2:
+      Entry 0: count: 1000 hasNull: false min: -9200577545527640566 max: 9175500305011173751 positions: 0,0,0
+      Entry 1: count: 1000 hasNull: false min: -9203618157670445774 max: 9208123824411178101 positions: 4099,2,488
+      Entry 2: count: 1000 hasNull: false min: -9218592812243954469 max: 9221351515892923972 positions: 12297,6,464
+      Entry 3: count: 1000 hasNull: false min: -9206585617947511272 max: 9167703224425685487 positions: 20495,10,440
+      Entry 4: count: 1000 hasNull: false min: -9206645795733282496 max: 9221614132680747961 positions: 28693,14,416
+    Row group indices for column 3:
+      Entry 0: count: 1000 hasNull: false min: Darkness, max: worst sum: 3862 positions: 0,0,0
+      Entry 1: count: 1000 hasNull: false min: Darkness, max: worst sum: 3884 positions: 0,659,149
+      Entry 2: count: 1000 hasNull: false min: Darkness, max: worst sum: 3893 positions: 0,1531,3
+      Entry 3: count: 1000 hasNull: false min: Darkness, max: worst sum: 3798 positions: 0,2281,32
+      Entry 4: count: 1000 hasNull: false min: Darkness, max: worst sum: 3843 positions: 0,3033,45
+  Stripe: offset: 64307 data: 63775 rows: 5000 tail: 79 index: 432
+    Stream: column 0 section ROW_INDEX start: 64307 length 17
+    Stream: column 1 section ROW_INDEX start: 64324 length 164
+    Stream: column 2 section ROW_INDEX start: 64488 length 168
+    Stream: column 3 section ROW_INDEX start: 64656 length 83
+    Stream: column 1 section DATA start: 64739 length 20035
+    Stream: column 2 section DATA start: 84774 length 40050
+    Stream: column 3 section DATA start: 124824 length 3532
+    Stream: column 3 section LENGTH start: 128356 length 25
+    Stream: column 3 section DICTIONARY_DATA start: 128381 length 133
+    Encoding column 0: DIRECT
+    Encoding column 1: DIRECT_V2
+    Encoding column 2: DIRECT_V2
+    Encoding column 3: DICTIONARY_V2[35]
+    Row group indices for column 1:
+      Entry 0: count: 1000 hasNull: false min: -2143799121 max: 2145249879 sum: -6966266181 positions: 0,0,0
+      Entry 1: count: 1000 hasNull: false min: -2146733128 max: 2147001622 sum: -35930106333 positions: 0,2050,488
+      Entry 2: count: 1000 hasNull: false min: -2144302712 max: 2146299933 sum: 6944230435 positions: 4099,2054,464
+      Entry 3: count: 1000 hasNull: false min: -2145172948 max: 2144335014 sum: -29624404959 positions: 8198,2058,440
+      Entry 4: count: 1000 hasNull: false min: -2146428427 max: 2144067253 sum: 65584220465 positions: 12297,2062,416
+    Row group indices for column 2:
+      Entry 0: count: 1000 hasNull: false min: -9218450653857701562 max: 9189819526332228512 positions: 0,0,0
+      Entry 1: count: 1000 hasNull: false min: -9220818777591257749 max: 9178821722829648113 positions: 4099,2,488
+      Entry 2: count: 1000 hasNull: false min: -9220031433030423388 max: 9210838931786956852 positions: 12297,6,464
+      Entry 3: count: 1000 hasNull: false min: -9208195729739635607 max: 9222259462014003839 positions: 20495,10,440
+      Entry 4: count: 1000 hasNull: false min: -9174271499932339698 max: 9212277876771676916 positions: 28693,14,416
+    Row group indices for column 3:
+      Entry 0: count: 1000 hasNull: false min: Darkness, max: worst sum: 3923 positions: 0,0,0
+      Entry 1: count: 1000 hasNull: false min: Darkness, max: worst sum: 3869 positions: 0,761,12
+      Entry 2: count: 1000 hasNull: false min: Darkness, max: worst sum: 3817 positions: 0,1472,70
+      Entry 3: count: 1000 hasNull: false min: Darkness, max: worst sum: 3931 positions: 0,2250,43
+      Entry 4: count: 1000 hasNull: false min: Darkness, max: worst sum: 3964 positions: 0,2978,88
+  Stripe: offset: 128593 data: 63787 rows: 5000 tail: 79 index: 438
+    Stream: column 0 section ROW_INDEX start: 128593 length 17
+    Stream: column 1 section ROW_INDEX start: 128610 length 163
+    Stream: column 2 section ROW_INDEX start: 128773 length 168
+    Stream: column 3 section ROW_INDEX start: 128941 length 90
+    Stream: column 1 section DATA start: 129031 length 20035
+    Stream: column 2 section DATA start: 149066 length 40050
+    Stream: column 3 section DATA start: 189116 length 3544
+    Stream: column 3 section LENGTH start: 192660 length 25
+    Stream: column 3 section DICTIONARY_DATA start: 192685 length 133
+    Encoding column 0: DIRECT
+    Encoding column 1: DIRECT_V2
+    Encoding column 2: DIRECT_V2
+    Encoding column 3: DICTIONARY_V2[35]
+    Row group indices for column 1:
+      Entry 0: count: 1000 hasNull: false min: -2146993718 max: 2144179881 sum: -7829543271 positions: 0,0,0
+      Entry 1: count: 1000 hasNull: false min: -2144095505 max: 2144883384 sum: 51623839692 positions: 0,2050,488
+      Entry 2: count: 1000 hasNull: false min: -2144113995 max: 2143773575 sum: 56574412741 positions: 4099,2054,464
+      Entry 3: count: 1000 hasNull: false min: -2146954065 max: 2146794873 sum: 4336083432 positions: 8198,2058,440
+      Entry 4: count: 1000 hasNull: false min: -2135511523 max: 2147378179 sum: 27955949957 positions: 12297,2062,416
+    Row group indices for column 2:
+      Entry 0: count: 1000 hasNull: false min: -9211978436552246208 max: 9179058898902097152 positions: 0,0,0
+      Entry 1: count: 1000 hasNull: false min: -9195645160817780503 max: 9189147759444307708 positions: 4099,2,488
+      Entry 2: count: 1000 hasNull: false min: -9202888157616520823 max: 9193561362676960747 positions: 12297,6,464
+      Entry 3: count: 1000 hasNull: false min: -9216318198067839390 max: 9221286760675829363 positions: 20495,10,440
+      Entry 4: count: 1000 hasNull: false min: -9218342074710552826 max: 9222303228623055266 positions: 28693,14,416
+    Row group indices for column 3:
+      Entry 0: count: 1000 hasNull: false min: Darkness, max: worst sum: 3817 positions: 0,0,0
+      Entry 1: count: 1000 hasNull: false min: Darkness, max: worst sum: 4008 positions: 0,634,174
+      Entry 2: count: 1000 hasNull: false min: Darkness, max: worst sum: 3999 positions: 0,1469,69
+      Entry 3: count: 1000 hasNull: false min: Darkness, max: worst sum: 3817 positions: 0,2133,194
+      Entry 4: count: 1000 hasNull: false min: Darkness, max: worst sum: 4000 positions: 0,3005,43
+  Stripe: offset: 192897 data: 63817 rows: 5000 tail: 79 index: 440
+    Stream: column 0 section ROW_INDEX start: 192897 length 17
+    Stream: column 1 section ROW_INDEX start: 192914 length 165
+    Stream: column 2 section ROW_INDEX start: 193079 length 167
+    Stream: column 3 section ROW_INDEX start: 193246 length 91
+    Stream: column 1 section DATA start: 193337 length 20035
+    Stream: column 2 section DATA start: 213372 length 40050
+    Stream: column 3 section DATA start: 253422 length 3574
+    Stream: column 3 section LENGTH start: 256996 length 25
+    Stream: column 3 section DICTIONARY_DATA start: 257021 length 133
+    Encoding column 0: DIRECT
+    Encoding column 1: DIRECT_V2
+    Encoding column 2: DIRECT_V2
+    Encoding column 3: DICTIONARY_V2[35]
+    Row group indices for column 1:
+      Entry 0: count: 1000 hasNull: false min: -2141355639 max: 2145520931 sum: 2726719912 positions: 0,0,0
+      Entry 1: count: 1000 hasNull: false min: -2138324170 max: 2140167376 sum: -23606674002 positions: 0,2050,488
+      Entry 2: count: 1000 hasNull: false min: -2146658006 max: 2144329742 sum: -41530109703 positions: 4099,2054,464
+      Entry 3: count: 1000 hasNull: false min: -2144207593 max: 2139456355 sum: 13559842458 positions: 8198,2058,440
+      Entry 4: count: 1000 hasNull: false min: -2145744719 max: 2145417153 sum: 57383770571 positions: 12297,2062,416
+    Row group indices for column 2:
+      Entry 0: count: 1000 hasNull: false min: -9222731174895935707 max: 9214167447015056056 positions: 0,0,0
+      Entry 1: count: 1000 hasNull: false min: -9222758097219661129 max: 9221043130193737406 positions: 4099,2,488
+      Entry 2: count: 1000 hasNull: false min: -9174483776261243438 max: 9208134757538374043 positions: 12297,6,464
+      Entry 3: count: 1000 hasNull: false min: -9174329712613510612 max: 9197412874152820822 positions: 20495,10,440
+      Entry 4: count: 1000 hasNull: false min: -9221162005892422758 max: 9220625004936875965 positions: 28693,14,416
+    Row group indices for column 3:
+      Entry 0: count: 1000 hasNull: false min: Darkness, max: worst sum: 3901 positions: 0,0,0
+      Entry 1: count: 1000 hasNull: false min: Darkness, max: worst sum: 3900 positions: 0,431,431
+      Entry 2: count: 1000 hasNull: false min: Darkness, max: worst sum: 3909 positions: 0,1485,52
+      Entry 3: count: 1000 hasNull: false min: Darkness, max: worst sum: 3947 positions: 0,2196,104
+      Entry 4: count: 1000 hasNull: false min: Darkness, max: worst sum: 3813 positions: 0,2934,131
+  Stripe: offset: 257233 data: 12943 rows: 1000 tail: 71 index: 131
+    Stream: column 0 section ROW_INDEX start: 257233 length 12
+    Stream: column 1 section ROW_INDEX start: 257245 length 38
+    Stream: column 2 section ROW_INDEX start: 257283 length 41
+    Stream: column 3 section ROW_INDEX start: 257324 length 40
+    Stream: column 1 section DATA start: 257364 length 4007
+    Stream: column 2 section DATA start: 261371 length 8010
+    Stream: column 3 section DATA start: 269381 length 768
+    Stream: column 3 section LENGTH start: 270149 length 25
+    Stream: column 3 section DICTIONARY_DATA start: 270174 length 133
+    Encoding column 0: DIRECT
+    Encoding column 1: DIRECT_V2
+    Encoding column 2: DIRECT_V2
+    Encoding column 3: DICTIONARY_V2[35]
+    Row group indices for column 1:
+      Entry 0: count: 1000 hasNull: false min: -2146245500 max: 2146378640 sum: 51299706363 positions: 0,0,0
+    Row group indices for column 2:
+      Entry 0: count: 1000 hasNull: false min: -9208193203370316142 max: 9218567213558056476 positions: 0,0,0
+    Row group indices for column 3:
+      Entry 0: count: 1000 hasNull: false min: Darkness, max: worst sum: 3866 positions: 0,0,0
+
+File length: 270923 bytes
+Padding length: 0 bytes
+Padding ratio: 0%
+________________________________________________________________________________________________________________________
+

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/orc/src/test/resources/orc-file-has-null.out
----------------------------------------------------------------------
diff --git a/orc/src/test/resources/orc-file-has-null.out b/orc/src/test/resources/orc-file-has-null.out
new file mode 100644
index 0000000..e98a73f
--- /dev/null
+++ b/orc/src/test/resources/orc-file-has-null.out
@@ -0,0 +1,112 @@
+Structure for TestOrcFile.testHasNull.orc
+File Version: 0.12 with HIVE_13083
+Rows: 20000
+Compression: ZLIB
+Compression size: 4096
+Type: struct<bytes1:binary,string1:string>
+
+Stripe Statistics:
+  Stripe 1:
+    Column 0: count: 5000 hasNull: false
+    Column 1: count: 5000 hasNull: false sum: 15000
+    Column 2: count: 2000 hasNull: true min: RG1 max: RG3 sum: 6000
+  Stripe 2:
+    Column 0: count: 5000 hasNull: false
+    Column 1: count: 5000 hasNull: false sum: 15000
+    Column 2: count: 0 hasNull: true
+  Stripe 3:
+    Column 0: count: 5000 hasNull: false
+    Column 1: count: 5000 hasNull: false sum: 15000
+    Column 2: count: 5000 hasNull: false min: STRIPE-3 max: STRIPE-3 sum: 40000
+  Stripe 4:
+    Column 0: count: 5000 hasNull: false
+    Column 1: count: 5000 hasNull: false sum: 15000
+    Column 2: count: 0 hasNull: true
+
+File Statistics:
+  Column 0: count: 20000 hasNull: false
+  Column 1: count: 20000 hasNull: false sum: 60000
+  Column 2: count: 7000 hasNull: true min: RG1 max: STRIPE-3 sum: 46000
+
+Stripes:
+  Stripe: offset: 3 data: 220 rows: 5000 tail: 65 index: 154
+    Stream: column 0 section ROW_INDEX start: 3 length 17
+    Stream: column 1 section ROW_INDEX start: 20 length 60
+    Stream: column 2 section ROW_INDEX start: 80 length 77
+    Stream: column 1 section DATA start: 157 length 159
+    Stream: column 1 section LENGTH start: 316 length 15
+    Stream: column 2 section PRESENT start: 331 length 13
+    Stream: column 2 section DATA start: 344 length 18
+    Stream: column 2 section LENGTH start: 362 length 6
+    Stream: column 2 section DICTIONARY_DATA start: 368 length 9
+    Encoding column 0: DIRECT
+    Encoding column 1: DIRECT_V2
+    Encoding column 2: DICTIONARY_V2[2]
+    Row group indices for column 2:
+      Entry 0: count: 1000 hasNull: false min: RG1 max: RG1 sum: 3000 positions: 0,0,0,0,0,0,0
+      Entry 1: count: 0 hasNull: true positions: 0,0,125,0,0,4,488
+      Entry 2: count: 1000 hasNull: false min: RG3 max: RG3 sum: 3000 positions: 0,2,125,0,0,4,488
+      Entry 3: count: 0 hasNull: true positions: 0,4,125,0,0,12,488
+      Entry 4: count: 0 hasNull: true positions: 0,6,125,0,0,12,488
+  Stripe: offset: 442 data: 185 rows: 5000 tail: 64 index: 116
+    Stream: column 0 section ROW_INDEX start: 442 length 17
+    Stream: column 1 section ROW_INDEX start: 459 length 60
+    Stream: column 2 section ROW_INDEX start: 519 length 39
+    Stream: column 1 section DATA start: 558 length 159
+    Stream: column 1 section LENGTH start: 717 length 15
+    Stream: column 2 section PRESENT start: 732 length 11
+    Stream: column 2 section DATA start: 743 length 0
+    Stream: column 2 section LENGTH start: 743 length 0
+    Stream: column 2 section DICTIONARY_DATA start: 743 length 0
+    Encoding column 0: DIRECT
+    Encoding column 1: DIRECT_V2
+    Encoding column 2: DICTIONARY_V2[0]
+    Row group indices for column 2:
+      Entry 0: count: 0 hasNull: true positions: 0,0,0,0,0,0,0
+      Entry 1: count: 0 hasNull: true positions: 0,0,125,0,0,0,0
+      Entry 2: count: 0 hasNull: true positions: 0,2,120,0,0,0,0
+      Entry 3: count: 0 hasNull: true positions: 0,4,115,0,0,0,0
+      Entry 4: count: 0 hasNull: true positions: 0,6,110,0,0,0,0
+  Stripe: offset: 807 data: 206 rows: 5000 tail: 60 index: 137
+    Stream: column 0 section ROW_INDEX start: 807 length 17
+    Stream: column 1 section ROW_INDEX start: 824 length 60
+    Stream: column 2 section ROW_INDEX start: 884 length 60
+    Stream: column 1 section DATA start: 944 length 159
+    Stream: column 1 section LENGTH start: 1103 length 15
+    Stream: column 2 section DATA start: 1118 length 15
+    Stream: column 2 section LENGTH start: 1133 length 6
+    Stream: column 2 section DICTIONARY_DATA start: 1139 length 11
+    Encoding column 0: DIRECT
+    Encoding column 1: DIRECT_V2
+    Encoding column 2: DICTIONARY_V2[1]
+    Row group indices for column 2:
+      Entry 0: count: 1000 hasNull: false min: STRIPE-3 max: STRIPE-3 sum: 8000 positions: 0,0,0
+      Entry 1: count: 1000 hasNull: false min: STRIPE-3 max: STRIPE-3 sum: 8000 positions: 0,4,488
+      Entry 2: count: 1000 hasNull: false min: STRIPE-3 max: STRIPE-3 sum: 8000 positions: 0,12,464
+      Entry 3: count: 1000 hasNull: false min: STRIPE-3 max: STRIPE-3 sum: 8000 positions: 0,20,440
+      Entry 4: count: 1000 hasNull: false min: STRIPE-3 max: STRIPE-3 sum: 8000 positions: 0,28,416
+  Stripe: offset: 1210 data: 185 rows: 5000 tail: 64 index: 116
+    Stream: column 0 section ROW_INDEX start: 1210 length 17
+    Stream: column 1 section ROW_INDEX start: 1227 length 60
+    Stream: column 2 section ROW_INDEX start: 1287 length 39
+    Stream: column 1 section DATA start: 1326 length 159
+    Stream: column 1 section LENGTH start: 1485 length 15
+    Stream: column 2 section PRESENT start: 1500 length 11
+    Stream: column 2 section DATA start: 1511 length 0
+    Stream: column 2 section LENGTH start: 1511 length 0
+    Stream: column 2 section DICTIONARY_DATA start: 1511 length 0
+    Encoding column 0: DIRECT
+    Encoding column 1: DIRECT_V2
+    Encoding column 2: DICTIONARY_V2[0]
+    Row group indices for column 2:
+      Entry 0: count: 0 hasNull: true positions: 0,0,0,0,0,0,0
+      Entry 1: count: 0 hasNull: true positions: 0,0,125,0,0,0,0
+      Entry 2: count: 0 hasNull: true positions: 0,2,120,0,0,0,0
+      Entry 3: count: 0 hasNull: true positions: 0,4,115,0,0,0,0
+      Entry 4: count: 0 hasNull: true positions: 0,6,110,0,0,0,0
+
+File length: 1823 bytes
+Padding length: 0 bytes
+Padding ratio: 0%
+________________________________________________________________________________________________________________________
+

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToTimestamp.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToTimestamp.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToTimestamp.java
index 6225ade..8963449 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToTimestamp.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToTimestamp.java
@@ -20,12 +20,9 @@ package org.apache.hadoop.hive.ql.exec.vector.expressions;
 
 import java.sql.Timestamp;
 
-import org.apache.hadoop.hive.common.type.HiveDecimal;
 import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.TimestampUtils;
-import org.apache.hadoop.hive.serde2.io.TimestampWritable;
+import org.apache.hadoop.hive.ql.util.TimestampUtils;
 
 /**
  * Type cast decimal to timestamp. The decimal value is interpreted
@@ -44,6 +41,7 @@ public class CastDecimalToTimestamp extends FuncDecimalToTimestamp {
 
   @Override
   protected void func(TimestampColumnVector outV, DecimalColumnVector inV,  int i) {
-    outV.set(i, TimestampWritable.decimalToTimestamp(inV.vector[i].getHiveDecimal()));
+    Timestamp timestamp = TimestampUtils.decimalToTimestamp(inV.vector[i].getHiveDecimal());
+    outV.set(i, timestamp);
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDoubleToTimestamp.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDoubleToTimestamp.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDoubleToTimestamp.java
index 31d2f78..07f94f5 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDoubleToTimestamp.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDoubleToTimestamp.java
@@ -18,9 +18,11 @@
 
 package org.apache.hadoop.hive.ql.exec.vector.expressions;
 
-import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
-import org.apache.hadoop.hive.ql.exec.vector.*;
-import org.apache.hadoop.hive.serde2.io.TimestampWritable;
+import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.util.TimestampUtils;
 
 public class CastDoubleToTimestamp extends VectorExpression {
   private static final long serialVersionUID = 1L;
@@ -40,9 +42,8 @@ public class CastDoubleToTimestamp extends VectorExpression {
 
   private void setDouble(TimestampColumnVector timestampColVector,
       double[] vector, int elementNum) {
-    TimestampWritable.setTimestampFromDouble(
-        timestampColVector.getScratchTimestamp(), vector[elementNum]);
-    timestampColVector.setFromScratchTimestamp(elementNum);
+    timestampColVector.set(elementNum,
+        TimestampUtils.doubleToTimestamp(vector[elementNum]));
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastLongToTimestamp.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastLongToTimestamp.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastLongToTimestamp.java
index a2ee52d..4de95a5 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastLongToTimestamp.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastLongToTimestamp.java
@@ -39,9 +39,7 @@ public class CastLongToTimestamp extends VectorExpression {
   }
 
   private void setSeconds(TimestampColumnVector timestampColVector, long[] vector, int elementNum) {
-    TimestampWritable.setTimestampFromLong(
-        timestampColVector.getScratchTimestamp(), vector[elementNum],
-        /* intToTimestampInSeconds */ true);
+    timestampColVector.getScratchTimestamp().setTime(vector[elementNum] * 1000);
     timestampColVector.setFromScratchTimestamp(elementNum);
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastMillisecondsLongToTimestamp.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastMillisecondsLongToTimestamp.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastMillisecondsLongToTimestamp.java
index 01c8810..b1c6b2d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastMillisecondsLongToTimestamp.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastMillisecondsLongToTimestamp.java
@@ -38,10 +38,9 @@ public class CastMillisecondsLongToTimestamp extends VectorExpression {
     super();
   }
 
-  private void setMilliseconds(TimestampColumnVector timestampColVector, long[] vector, int elementNum) {
-    TimestampWritable.setTimestampFromLong(
-        timestampColVector.getScratchTimestamp(), vector[elementNum],
-        /* intToTimestampInSeconds */ false);
+  private void setMilliseconds(TimestampColumnVector timestampColVector,
+                               long[] vector, int elementNum) {
+    timestampColVector.getScratchTimestamp().setTime(vector[elementNum]);
     timestampColVector.setFromScratchTimestamp(elementNum);
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringExpr.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringExpr.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringExpr.java
deleted file mode 100644
index 90817a5..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringExpr.java
+++ /dev/null
@@ -1,354 +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.hive.ql.exec.vector.expressions;
-
-import java.util.Arrays;
-
-import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
-
-/**
- * String expression evaluation helper functions.
- */
-public class StringExpr {
-
-  /* Compare two strings from two byte arrays each
-   * with their own start position and length.
-   * Use lexicographic unsigned byte value order.
-   * This is what's used for UTF-8 sort order.
-   * Return negative value if arg1 < arg2, 0 if arg1 = arg2,
-   * positive if arg1 > arg2.
-   */
-  public static int compare(byte[] arg1, int start1, int len1, byte[] arg2, int start2, int len2) {
-    for (int i = 0; i < len1 && i < len2; i++) {
-      // Note the "& 0xff" is just a way to convert unsigned bytes to signed integer.
-      int b1 = arg1[i + start1] & 0xff;
-      int b2 = arg2[i + start2] & 0xff;
-      if (b1 != b2) {
-        return b1 - b2;
-      }
-    }
-    return len1 - len2;
-  }
-
-  /* Determine if two strings are equal from two byte arrays each
-   * with their own start position and length.
-   * Use lexicographic unsigned byte value order.
-   * This is what's used for UTF-8 sort order.
-   */
-  public static boolean equal(byte[] arg1, final int start1, final int len1,
-      byte[] arg2, final int start2, final int len2) {
-    if (len1 != len2) {
-      return false;
-    }
-    if (len1 == 0) {
-      return true;
-    }
-
-    // do bounds check for OOB exception
-    if (arg1[start1] != arg2[start2]
-        || arg1[start1 + len1 - 1] != arg2[start2 + len2 - 1]) {
-      return false;
-    }
-
-    if (len1 == len2) {
-      // prove invariant to the compiler: len1 = len2
-      // all array access between (start1, start1+len1) 
-      // and (start2, start2+len2) are valid
-      // no more OOB exceptions are possible
-      final int step = 8;
-      final int remainder = len1 % step;
-      final int wlen = len1 - remainder;
-      // suffix first
-      for (int i = wlen; i < len1; i++) {
-        if (arg1[start1 + i] != arg2[start2 + i]) {
-          return false;
-        }
-      }
-      // SIMD loop
-      for (int i = 0; i < wlen; i += step) {
-        final int s1 = start1 + i;
-        final int s2 = start2 + i;
-        boolean neq = false;
-        for (int j = 0; j < step; j++) {
-          neq = (arg1[s1 + j] != arg2[s2 + j]) || neq;
-        }
-        if (neq) {
-          return false;
-        }
-      }
-    }
-
-    return true;
-  }
-
-  public static int characterCount(byte[] bytes) {
-    int end = bytes.length;
-
-    // count characters
-    int j = 0;
-    int charCount = 0;
-    while(j < end) {
-      // UTF-8 continuation bytes have 2 high bits equal to 0x80.
-      if ((bytes[j] & 0xc0) != 0x80) {
-        ++charCount;
-      }
-      j++;
-    }
-    return charCount;
-  }
-
-  public static int characterCount(byte[] bytes, int start, int length) {
-    int end = start + length;
-
-    // count characters
-    int j = start;
-    int charCount = 0;
-    while(j < end) {
-      // UTF-8 continuation bytes have 2 high bits equal to 0x80.
-      if ((bytes[j] & 0xc0) != 0x80) {
-        ++charCount;
-      }
-      j++;
-    }
-    return charCount;
-  }
-
-  // A setVal with the same function signature as rightTrim, leftTrim, truncate, etc, below.
-  // Useful for class generation via templates.
-  public static void assign(BytesColumnVector outV, int i, byte[] bytes, int start, int length) {
-    // set output vector
-    outV.setVal(i, bytes, start, length);
-  }
-
-  /*
-   * Right trim a slice of a byte array and return the new byte length.
-   */
-  public static int rightTrim(byte[] bytes, int start, int length) {
-    // skip trailing blank characters
-    int j = start + length - 1;
-    while(j >= start && bytes[j] == 0x20) {
-      j--;
-    }
-
-    return (j - start) + 1;
-  }
-
-  /*
-   * Right trim a slice of a byte array and place the result into element i of a vector.
-   */
-  public static void rightTrim(BytesColumnVector outV, int i, byte[] bytes, int start, int length) {
-    // skip trailing blank characters
-    int j = start + length - 1;
-    while(j >= start && bytes[j] == 0x20) {
-      j--;
-    }
-
-    // set output vector
-    outV.setVal(i, bytes, start, (j - start) + 1);
-  }
-
-  /*
-   * Truncate a slice of a byte array to a maximum number of characters and
-   * return the new byte length.
-   */
-  public static int truncate(byte[] bytes, int start, int length, int maxLength) {
-    int end = start + length;
-
-    // count characters forward
-    int j = start;
-    int charCount = 0;
-    while(j < end) {
-      // UTF-8 continuation bytes have 2 high bits equal to 0x80.
-      if ((bytes[j] & 0xc0) != 0x80) {
-        if (charCount == maxLength) {
-          break;
-        }
-        ++charCount;
-      }
-      j++;
-    }
-    return (j - start);
-  }
-
-  /*
-   * Truncate a slice of a byte array to a maximum number of characters and
-   * place the result into element i of a vector.
-   */
-  public static void truncate(BytesColumnVector outV, int i, byte[] bytes, int start, int length, int maxLength) {
-    int end = start + length;
-
-    // count characters forward
-    int j = start;
-    int charCount = 0;
-    while(j < end) {
-      // UTF-8 continuation bytes have 2 high bits equal to 0x80.
-      if ((bytes[j] & 0xc0) != 0x80) {
-        if (charCount == maxLength) {
-          break;
-        }
-        ++charCount;
-      }
-      j++;
-    }
-
-    // set output vector
-    outV.setVal(i, bytes, start, (j - start));
-  }
-
-  /*
-   * Truncate a byte array to a maximum number of characters and
-   * return a byte array with only truncated bytes.
-   */
-  public static byte[] truncateScalar(byte[] bytes, int maxLength) {
-    int end = bytes.length;
-
-    // count characters forward
-    int j = 0;
-    int charCount = 0;
-    while(j < end) {
-      // UTF-8 continuation bytes have 2 high bits equal to 0x80.
-      if ((bytes[j] & 0xc0) != 0x80) {
-        if (charCount == maxLength) {
-          break;
-        }
-        ++charCount;
-      }
-      j++;
-    }
-    if (j == end) {
-      return bytes;
-    } else {
-      return Arrays.copyOf(bytes, j);
-    }
-  }
-
-  /*
-   * Right trim and truncate a slice of a byte array to a maximum number of characters and
-   * return the new byte length.
-   */
-  public static int rightTrimAndTruncate(byte[] bytes, int start, int length, int maxLength) {
-    int end = start + length;
-
-    // count characters forward and watch for final run of pads
-    int j = start;
-    int charCount = 0;
-    int padRunStart = -1;
-    while(j < end) {
-      // UTF-8 continuation bytes have 2 high bits equal to 0x80.
-      if ((bytes[j] & 0xc0) != 0x80) {
-        if (charCount == maxLength) {
-          break;
-        }
-        if (bytes[j] == 0x20) {
-          if (padRunStart == -1) {
-            padRunStart = j;
-          }
-        } else {
-          padRunStart = -1;
-        }
-        ++charCount;
-      } else {
-        padRunStart = -1;
-      }
-      j++;
-    }
-    if (padRunStart != -1) {
-      return (padRunStart - start);
-    } else {
-      return (j - start);
-    }
-  }
-
-  /*
-   * Right trim and truncate a slice of a byte array to a maximum number of characters and
-   * place the result into element i of a vector.
-   */
-  public static void rightTrimAndTruncate(BytesColumnVector outV, int i, byte[] bytes, int start, int length, int maxLength) {
-    int end = start + length;
-
-    // count characters forward and watch for final run of pads
-    int j = start;
-    int charCount = 0;
-    int padRunStart = -1;
-    while(j < end) {
-      // UTF-8 continuation bytes have 2 high bits equal to 0x80.
-      if ((bytes[j] & 0xc0) != 0x80) {
-        if (charCount == maxLength) {
-          break;
-        }
-        if (bytes[j] == 0x20) {
-          if (padRunStart == -1) {
-            padRunStart = j;
-          }
-        } else {
-          padRunStart = -1;
-        }
-        ++charCount;
-      } else {
-        padRunStart = -1;
-      }
-      j++;
-    }
-    // set output vector
-    if (padRunStart != -1) {
-      outV.setVal(i, bytes, start, (padRunStart - start));
-    } else {
-      outV.setVal(i, bytes, start, (j - start) );
-    }
-  }
-
-  /*
-   * Right trim and truncate a byte array to a maximum number of characters and
-   * return a byte array with only the trimmed and truncated bytes.
-   */
-  public static byte[] rightTrimAndTruncateScalar(byte[] bytes, int maxLength) {
-    int end = bytes.length;
-
-    // count characters forward and watch for final run of pads
-    int j = 0;
-    int charCount = 0;
-    int padRunStart = -1;
-    while(j < end) {
-      // UTF-8 continuation bytes have 2 high bits equal to 0x80.
-      if ((bytes[j] & 0xc0) != 0x80) {
-        if (charCount == maxLength) {
-          break;
-        }
-        if (bytes[j] == 0x20) {
-          if (padRunStart == -1) {
-            padRunStart = j;
-          }
-        } else {
-          padRunStart = -1;
-        }
-        ++charCount;
-      } else {
-        padRunStart = -1;
-      }
-      j++;
-    }
-    if (padRunStart != -1) {
-      return Arrays.copyOf(bytes, padRunStart);
-    } else if (j == end) {
-      return bytes;
-    } else {
-      return Arrays.copyOf(bytes, j);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/ql/src/java/org/apache/hadoop/hive/ql/hooks/PostExecOrcFileDump.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/hooks/PostExecOrcFileDump.java b/ql/src/java/org/apache/hadoop/hive/ql/hooks/PostExecOrcFileDump.java
index d5d1370..e184fcb 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/hooks/PostExecOrcFileDump.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/hooks/PostExecOrcFileDump.java
@@ -30,8 +30,8 @@ import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.QueryPlan;
 import org.apache.hadoop.hive.ql.exec.FetchTask;
-import org.apache.hadoop.hive.ql.io.FileFormatException;
-import org.apache.hadoop.hive.ql.io.orc.FileDump;
+import org.apache.orc.FileFormatException;
+import org.apache.orc.tools.FileDump;
 import org.apache.hadoop.hive.ql.io.orc.OrcFile;
 import org.apache.hadoop.hive.ql.plan.FetchWork;
 import org.apache.hadoop.hive.ql.session.SessionState;

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/ql/src/java/org/apache/hadoop/hive/ql/io/FileFormatException.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/FileFormatException.java b/ql/src/java/org/apache/hadoop/hive/ql/io/FileFormatException.java
deleted file mode 100644
index 12417aa..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/FileFormatException.java
+++ /dev/null
@@ -1,30 +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
- * <p/>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p/>
- * 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.hive.ql.io;
-
-import java.io.IOException;
-
-/**
- * Thrown when an invalid file format is encountered.
- */
-public class FileFormatException extends IOException {
-
-  public FileFormatException(String errMsg) {
-    super(errMsg);
-  }
-}


[18/27] hive git commit: HIVE-11417. Move the ReaderImpl and RowReaderImpl to the ORC module, by making shims for the row by row reader. (omalley reviewed by prasanth_j)

Posted by om...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/orc/src/test/org/apache/orc/impl/TestOrcWideTable.java
----------------------------------------------------------------------
diff --git a/orc/src/test/org/apache/orc/impl/TestOrcWideTable.java b/orc/src/test/org/apache/orc/impl/TestOrcWideTable.java
new file mode 100644
index 0000000..289a86e
--- /dev/null
+++ b/orc/src/test/org/apache/orc/impl/TestOrcWideTable.java
@@ -0,0 +1,64 @@
+/**
+ * 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.orc.impl;
+
+import static org.junit.Assert.assertEquals;
+
+import java.io.IOException;
+
+import org.junit.Test;
+
+public class TestOrcWideTable {
+
+  @Test
+  public void testBufferSizeFor1Col() throws IOException {
+    assertEquals(128 * 1024, WriterImpl.getEstimatedBufferSize(512 * 1024 * 1024,
+        1, 128*1024));
+  }
+
+  @Test
+  public void testBufferSizeFor50Col() throws IOException {
+    assertEquals(256 * 1024, WriterImpl.getEstimatedBufferSize(256 * 1024 * 1024,
+        50, 256*1024));
+  }
+
+  @Test
+  public void testBufferSizeFor1000Col() throws IOException {
+    assertEquals(32 * 1024, WriterImpl.getEstimatedBufferSize(512 * 1024 * 1024,
+        1000, 128*1024));
+  }
+
+  @Test
+  public void testBufferSizeFor2000Col() throws IOException {
+    assertEquals(16 * 1024, WriterImpl.getEstimatedBufferSize(512 * 1024 * 1024,
+        2000, 256*1024));
+  }
+
+  @Test
+  public void testBufferSizeFor4000Col() throws IOException {
+    assertEquals(8 * 1024, WriterImpl.getEstimatedBufferSize(512 * 1024 * 1024,
+        4000, 256*1024));
+  }
+
+  @Test
+  public void testBufferSizeFor25000Col() throws IOException {
+    assertEquals(4 * 1024, WriterImpl.getEstimatedBufferSize(512 * 1024 * 1024,
+        25000, 256*1024));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/orc/src/test/org/apache/orc/impl/TestRLEv2.java
----------------------------------------------------------------------
diff --git a/orc/src/test/org/apache/orc/impl/TestRLEv2.java b/orc/src/test/org/apache/orc/impl/TestRLEv2.java
new file mode 100644
index 0000000..e139619
--- /dev/null
+++ b/orc/src/test/org/apache/orc/impl/TestRLEv2.java
@@ -0,0 +1,307 @@
+/**
+ * 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.orc.impl;
+
+import static org.junit.Assert.assertEquals;
+
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.PrintStream;
+import java.util.Random;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.orc.CompressionKind;
+import org.apache.orc.OrcFile;
+import org.apache.orc.TypeDescription;
+import org.apache.orc.Writer;
+import org.apache.orc.tools.FileDump;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+
+public class TestRLEv2 {
+  Path workDir = new Path(System.getProperty("test.tmp.dir",
+      "target" + File.separator + "test" + File.separator + "tmp"));
+  Path testFilePath;
+  Configuration conf;
+  FileSystem fs;
+
+  @Rule
+  public TestName testCaseName = new TestName();
+
+  @Before
+  public void openFileSystem () throws Exception {
+    conf = new Configuration();
+    fs = FileSystem.getLocal(conf);
+    testFilePath = new Path(workDir, "TestRLEv2." +
+        testCaseName.getMethodName() + ".orc");
+    fs.delete(testFilePath, false);
+  }
+
+  void appendInt(VectorizedRowBatch batch, int i) {
+    ((LongColumnVector) batch.cols[0]).vector[batch.size++] = i;
+  }
+
+  @Test
+  public void testFixedDeltaZero() throws Exception {
+    TypeDescription schema = TypeDescription.createInt();
+    Writer w = OrcFile.createWriter(testFilePath,
+        OrcFile.writerOptions(conf)
+            .compress(CompressionKind.NONE)
+            .setSchema(schema)
+            .rowIndexStride(0)
+            .encodingStrategy(OrcFile.EncodingStrategy.COMPRESSION)
+            .version(OrcFile.Version.V_0_12)
+    );
+    VectorizedRowBatch batch = schema.createRowBatch(5120);
+    for (int i = 0; i < 5120; ++i) {
+      appendInt(batch, 123);
+    }
+    w.addRowBatch(batch);
+    w.close();
+
+    PrintStream origOut = System.out;
+    ByteArrayOutputStream myOut = new ByteArrayOutputStream();
+    System.setOut(new PrintStream(myOut));
+    FileDump.main(new String[]{testFilePath.toUri().toString()});
+    System.out.flush();
+    String outDump = new String(myOut.toByteArray());
+    // 10 runs of 512 elements. Each run has 2 bytes header, 2 bytes base (base = 123,
+    // zigzag encoded varint) and 1 byte delta (delta = 0). In total, 5 bytes per run.
+    assertEquals(true, outDump.contains("Stream: column 0 section DATA start: 3 length 50"));
+    System.setOut(origOut);
+  }
+
+  @Test
+  public void testFixedDeltaOne() throws Exception {
+    TypeDescription schema = TypeDescription.createInt();
+    Writer w = OrcFile.createWriter(testFilePath,
+        OrcFile.writerOptions(conf)
+            .compress(CompressionKind.NONE)
+            .setSchema(schema)
+            .rowIndexStride(0)
+            .encodingStrategy(OrcFile.EncodingStrategy.COMPRESSION)
+            .version(OrcFile.Version.V_0_12)
+    );
+    VectorizedRowBatch batch = schema.createRowBatch(5120);
+    for (int i = 0; i < 5120; ++i) {
+      appendInt(batch, i % 512);
+    }
+    w.addRowBatch(batch);
+    w.close();
+
+    PrintStream origOut = System.out;
+    ByteArrayOutputStream myOut = new ByteArrayOutputStream();
+    System.setOut(new PrintStream(myOut));
+    FileDump.main(new String[]{testFilePath.toUri().toString()});
+    System.out.flush();
+    String outDump = new String(myOut.toByteArray());
+    // 10 runs of 512 elements. Each run has 2 bytes header, 1 byte base (base = 0)
+    // and 1 byte delta (delta = 1). In total, 4 bytes per run.
+    assertEquals(true, outDump.contains("Stream: column 0 section DATA start: 3 length 40"));
+    System.setOut(origOut);
+  }
+
+  @Test
+  public void testFixedDeltaOneDescending() throws Exception {
+    TypeDescription schema = TypeDescription.createInt();
+    Writer w = OrcFile.createWriter(testFilePath,
+        OrcFile.writerOptions(conf)
+            .compress(CompressionKind.NONE)
+            .setSchema(schema)
+            .rowIndexStride(0)
+            .encodingStrategy(OrcFile.EncodingStrategy.COMPRESSION)
+            .version(OrcFile.Version.V_0_12)
+    );
+    VectorizedRowBatch batch = schema.createRowBatch(5120);
+    for (int i = 0; i < 5120; ++i) {
+      appendInt(batch, 512 - (i % 512));
+    }
+    w.addRowBatch(batch);
+    w.close();
+
+    PrintStream origOut = System.out;
+    ByteArrayOutputStream myOut = new ByteArrayOutputStream();
+    System.setOut(new PrintStream(myOut));
+    FileDump.main(new String[]{testFilePath.toUri().toString()});
+    System.out.flush();
+    String outDump = new String(myOut.toByteArray());
+    // 10 runs of 512 elements. Each run has 2 bytes header, 2 byte base (base = 512, zigzag + varint)
+    // and 1 byte delta (delta = 1). In total, 5 bytes per run.
+    assertEquals(true, outDump.contains("Stream: column 0 section DATA start: 3 length 50"));
+    System.setOut(origOut);
+  }
+
+  @Test
+  public void testFixedDeltaLarge() throws Exception {
+    TypeDescription schema = TypeDescription.createInt();
+    Writer w = OrcFile.createWriter(testFilePath,
+        OrcFile.writerOptions(conf)
+            .compress(CompressionKind.NONE)
+            .setSchema(schema)
+            .rowIndexStride(0)
+            .encodingStrategy(OrcFile.EncodingStrategy.COMPRESSION)
+            .version(OrcFile.Version.V_0_12)
+    );
+    VectorizedRowBatch batch = schema.createRowBatch(5120);
+    for (int i = 0; i < 5120; ++i) {
+      appendInt(batch, i % 512 + ((i % 512) * 100));
+    }
+    w.addRowBatch(batch);
+    w.close();
+
+    PrintStream origOut = System.out;
+    ByteArrayOutputStream myOut = new ByteArrayOutputStream();
+    System.setOut(new PrintStream(myOut));
+    FileDump.main(new String[]{testFilePath.toUri().toString()});
+    System.out.flush();
+    String outDump = new String(myOut.toByteArray());
+    // 10 runs of 512 elements. Each run has 2 bytes header, 1 byte base (base = 0)
+    // and 2 bytes delta (delta = 100, zigzag encoded varint). In total, 5 bytes per run.
+    assertEquals(true, outDump.contains("Stream: column 0 section DATA start: 3 length 50"));
+    System.setOut(origOut);
+  }
+
+  @Test
+  public void testFixedDeltaLargeDescending() throws Exception {
+    TypeDescription schema = TypeDescription.createInt();
+    Writer w = OrcFile.createWriter(testFilePath,
+        OrcFile.writerOptions(conf)
+            .compress(CompressionKind.NONE)
+            .setSchema(schema)
+            .rowIndexStride(0)
+            .encodingStrategy(OrcFile.EncodingStrategy.COMPRESSION)
+            .version(OrcFile.Version.V_0_12)
+    );
+    VectorizedRowBatch batch = schema.createRowBatch(5120);
+    for (int i = 0; i < 5120; ++i) {
+      appendInt(batch, (512 - i % 512) + ((i % 512) * 100));
+    }
+    w.addRowBatch(batch);
+    w.close();
+
+    PrintStream origOut = System.out;
+    ByteArrayOutputStream myOut = new ByteArrayOutputStream();
+    System.setOut(new PrintStream(myOut));
+    FileDump.main(new String[]{testFilePath.toUri().toString()});
+    System.out.flush();
+    String outDump = new String(myOut.toByteArray());
+    // 10 runs of 512 elements. Each run has 2 bytes header, 2 byte base (base = 512, zigzag + varint)
+    // and 2 bytes delta (delta = 100, zigzag encoded varint). In total, 6 bytes per run.
+    assertEquals(true, outDump.contains("Stream: column 0 section DATA start: 3 length 60"));
+    System.setOut(origOut);
+  }
+
+  @Test
+  public void testShortRepeat() throws Exception {
+    TypeDescription schema = TypeDescription.createInt();
+    Writer w = OrcFile.createWriter(testFilePath,
+        OrcFile.writerOptions(conf)
+            .compress(CompressionKind.NONE)
+            .setSchema(schema)
+            .rowIndexStride(0)
+            .encodingStrategy(OrcFile.EncodingStrategy.COMPRESSION)
+            .version(OrcFile.Version.V_0_12)
+    );
+    VectorizedRowBatch batch = schema.createRowBatch(5120);
+    for (int i = 0; i < 5; ++i) {
+      appendInt(batch, 10);
+    }
+    w.addRowBatch(batch);
+    w.close();
+
+    PrintStream origOut = System.out;
+    ByteArrayOutputStream myOut = new ByteArrayOutputStream();
+    System.setOut(new PrintStream(myOut));
+    FileDump.main(new String[]{testFilePath.toUri().toString()});
+    System.out.flush();
+    String outDump = new String(myOut.toByteArray());
+    // 1 byte header + 1 byte value
+    assertEquals(true, outDump.contains("Stream: column 0 section DATA start: 3 length 2"));
+    System.setOut(origOut);
+  }
+
+  @Test
+  public void testDeltaUnknownSign() throws Exception {
+    TypeDescription schema = TypeDescription.createInt();
+    Writer w = OrcFile.createWriter(testFilePath,
+        OrcFile.writerOptions(conf)
+            .compress(CompressionKind.NONE)
+            .setSchema(schema)
+            .rowIndexStride(0)
+            .encodingStrategy(OrcFile.EncodingStrategy.COMPRESSION)
+            .version(OrcFile.Version.V_0_12)
+    );
+    VectorizedRowBatch batch = schema.createRowBatch(5120);
+    appendInt(batch, 0);
+    for (int i = 0; i < 511; ++i) {
+      appendInt(batch, i);
+    }
+    w.addRowBatch(batch);
+    w.close();
+
+    PrintStream origOut = System.out;
+    ByteArrayOutputStream myOut = new ByteArrayOutputStream();
+    System.setOut(new PrintStream(myOut));
+    FileDump.main(new String[]{testFilePath.toUri().toString()});
+    System.out.flush();
+    String outDump = new String(myOut.toByteArray());
+    // monotonicity will be undetermined for this sequence 0,0,1,2,3,...510. Hence DIRECT encoding
+    // will be used. 2 bytes for header and 640 bytes for data (512 values with fixed bit of 10 bits
+    // each, 5120/8 = 640). Total bytes 642
+    assertEquals(true, outDump.contains("Stream: column 0 section DATA start: 3 length 642"));
+    System.setOut(origOut);
+  }
+
+  @Test
+  public void testPatchedBase() throws Exception {
+    TypeDescription schema = TypeDescription.createInt();
+    Writer w = OrcFile.createWriter(testFilePath,
+        OrcFile.writerOptions(conf)
+            .compress(CompressionKind.NONE)
+            .setSchema(schema)
+            .rowIndexStride(0)
+            .encodingStrategy(OrcFile.EncodingStrategy.COMPRESSION)
+            .version(OrcFile.Version.V_0_12)
+    );
+
+    Random rand = new Random(123);
+    VectorizedRowBatch batch = schema.createRowBatch(5120);
+    appendInt(batch, 10000000);
+    for (int i = 0; i < 511; ++i) {
+      appendInt(batch, rand.nextInt(i+1));
+    }
+    w.addRowBatch(batch);
+    w.close();
+
+    PrintStream origOut = System.out;
+    ByteArrayOutputStream myOut = new ByteArrayOutputStream();
+    System.setOut(new PrintStream(myOut));
+    FileDump.main(new String[]{testFilePath.toUri().toString()});
+    System.out.flush();
+    String outDump = new String(myOut.toByteArray());
+    // use PATCHED_BASE encoding
+    assertEquals(true, outDump.contains("Stream: column 0 section DATA start: 3 length 583"));
+    System.setOut(origOut);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/orc/src/test/org/apache/orc/impl/TestReaderImpl.java
----------------------------------------------------------------------
diff --git a/orc/src/test/org/apache/orc/impl/TestReaderImpl.java b/orc/src/test/org/apache/orc/impl/TestReaderImpl.java
new file mode 100644
index 0000000..23d0dab
--- /dev/null
+++ b/orc/src/test/org/apache/orc/impl/TestReaderImpl.java
@@ -0,0 +1,152 @@
+/*
+ * Copyright 2016 The Apache Software Foundation.
+ *
+ * Licensed 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.orc.impl;
+
+import java.io.ByteArrayInputStream;
+import java.io.EOFException;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.CharacterCodingException;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PositionedReadable;
+import org.apache.hadoop.fs.Seekable;
+import org.apache.orc.FileFormatException;
+import org.apache.hadoop.io.Text;
+import org.apache.orc.OrcFile;
+import org.junit.Test;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.rules.ExpectedException;
+
+public class TestReaderImpl {
+
+  @Rule
+  public ExpectedException thrown = ExpectedException.none();
+
+  private final Path path = new Path("test-file.orc");
+  private FSDataInputStream in;
+  private int psLen;
+  private ByteBuffer buffer;
+
+  @Before
+  public void setup() {
+    in = null;
+  }
+
+  @Test
+  public void testEnsureOrcFooterSmallTextFile() throws IOException {
+    prepareTestCase("1".getBytes());
+    thrown.expect(FileFormatException.class);
+    ReaderImpl.ensureOrcFooter(in, path, psLen, buffer);
+  }
+
+  @Test
+  public void testEnsureOrcFooterLargeTextFile() throws IOException {
+    prepareTestCase("This is Some Text File".getBytes());
+    thrown.expect(FileFormatException.class);
+    ReaderImpl.ensureOrcFooter(in, path, psLen, buffer);
+  }
+
+  @Test
+  public void testEnsureOrcFooter011ORCFile() throws IOException {
+    prepareTestCase(composeContent(OrcFile.MAGIC, "FOOTER"));
+    ReaderImpl.ensureOrcFooter(in, path, psLen, buffer);
+  }
+
+  @Test
+  public void testEnsureOrcFooterCorrectORCFooter() throws IOException {
+    prepareTestCase(composeContent("", OrcFile.MAGIC));
+    ReaderImpl.ensureOrcFooter(in, path, psLen, buffer);
+  }
+
+  private void prepareTestCase(byte[] bytes) {
+    buffer = ByteBuffer.wrap(bytes);
+    psLen = buffer.get(bytes.length - 1) & 0xff;
+    in = new FSDataInputStream(new SeekableByteArrayInputStream(bytes));
+  }
+
+  private byte[] composeContent(String headerStr, String footerStr) throws CharacterCodingException {
+    ByteBuffer header = Text.encode(headerStr);
+    ByteBuffer footer = Text.encode(footerStr);
+    int headerLen = header.remaining();
+    int footerLen = footer.remaining() + 1;
+
+    ByteBuffer buf = ByteBuffer.allocate(headerLen + footerLen);
+
+    buf.put(header);
+    buf.put(footer);
+    buf.put((byte) footerLen);
+    return buf.array();
+  }
+
+  private static final class SeekableByteArrayInputStream extends ByteArrayInputStream
+          implements Seekable, PositionedReadable {
+
+    public SeekableByteArrayInputStream(byte[] buf) {
+      super(buf);
+    }
+
+    @Override
+    public void seek(long pos) throws IOException {
+      this.reset();
+      this.skip(pos);
+    }
+
+    @Override
+    public long getPos() throws IOException {
+      return pos;
+    }
+
+    @Override
+    public boolean seekToNewSource(long targetPos) throws IOException {
+      return false;
+    }
+
+    @Override
+    public int read(long position, byte[] buffer, int offset, int length)
+            throws IOException {
+      long oldPos = getPos();
+      int nread = -1;
+      try {
+        seek(position);
+        nread = read(buffer, offset, length);
+      } finally {
+        seek(oldPos);
+      }
+      return nread;
+    }
+
+    @Override
+    public void readFully(long position, byte[] buffer, int offset, int length)
+            throws IOException {
+      int nread = 0;
+      while (nread < length) {
+        int nbytes = read(position + nread, buffer, offset + nread, length - nread);
+        if (nbytes < 0) {
+          throw new EOFException("End of file reached before reading fully.");
+        }
+        nread += nbytes;
+      }
+    }
+
+    @Override
+    public void readFully(long position, byte[] buffer)
+            throws IOException {
+      readFully(position, buffer, 0, buffer.length);
+    }
+  }
+}


[02/27] hive git commit: HIVE-11417. Move the ReaderImpl and RowReaderImpl to the ORC module, by making shims for the row by row reader. (omalley reviewed by prasanth_j)

Posted by om...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/ql/src/test/resources/orc-file-dump.json
----------------------------------------------------------------------
diff --git a/ql/src/test/resources/orc-file-dump.json b/ql/src/test/resources/orc-file-dump.json
deleted file mode 100644
index bf654a1..0000000
--- a/ql/src/test/resources/orc-file-dump.json
+++ /dev/null
@@ -1,1355 +0,0 @@
-{
-  "fileName": "TestFileDump.testDump.orc",
-  "fileVersion": "0.12",
-  "writerVersion": "HIVE_13083",
-  "numberOfRows": 21000,
-  "compression": "ZLIB",
-  "compressionBufferSize": 4096,
-  "schemaString": "struct<i:int,l:bigint,s:string>",
-  "schema": [
-    {
-      "columnId": 0,
-      "columnType": "STRUCT",
-      "childColumnNames": [
-        "i",
-        "l",
-        "s"
-      ],
-      "childColumnIds": [
-        1,
-        2,
-        3
-      ]
-    },
-    {
-      "columnId": 1,
-      "columnType": "INT"
-    },
-    {
-      "columnId": 2,
-      "columnType": "LONG"
-    },
-    {
-      "columnId": 3,
-      "columnType": "STRING"
-    }
-  ],
-  "stripeStatistics": [
-    {
-      "stripeNumber": 1,
-      "columnStatistics": [
-        {
-          "columnId": 0,
-          "count": 5000,
-          "hasNull": false
-        },
-        {
-          "columnId": 1,
-          "count": 5000,
-          "hasNull": false,
-          "min": -2147115959,
-          "max": 2145210552,
-          "sum": 50111854553,
-          "type": "LONG"
-        },
-        {
-          "columnId": 2,
-          "count": 5000,
-          "hasNull": false,
-          "min": -9223180583305557329,
-          "max": 9221614132680747961,
-          "type": "LONG"
-        },
-        {
-          "columnId": 3,
-          "count": 4950,
-          "hasNull": true,
-          "min": "Darkness,",
-          "max": "worst",
-          "totalLength": 19283,
-          "type": "STRING"
-        }
-      ]
-    },
-    {
-      "stripeNumber": 2,
-      "columnStatistics": [
-        {
-          "columnId": 0,
-          "count": 5000,
-          "hasNull": false
-        },
-        {
-          "columnId": 1,
-          "count": 5000,
-          "hasNull": false,
-          "min": -2147390285,
-          "max": 2147224606,
-          "sum": -22290798217,
-          "type": "LONG"
-        },
-        {
-          "columnId": 2,
-          "count": 5000,
-          "hasNull": false,
-          "min": -9219295160509160427,
-          "max": 9217571024994660020,
-          "type": "LONG"
-        },
-        {
-          "columnId": 3,
-          "count": 4950,
-          "hasNull": true,
-          "min": "Darkness,",
-          "max": "worst",
-          "totalLength": 19397,
-          "type": "STRING"
-        }
-      ]
-    },
-    {
-      "stripeNumber": 3,
-      "columnStatistics": [
-        {
-          "columnId": 0,
-          "count": 5000,
-          "hasNull": false
-        },
-        {
-          "columnId": 1,
-          "count": 5000,
-          "hasNull": false,
-          "min": -2146954065,
-          "max": 2146722468,
-          "sum": 20639652136,
-          "type": "LONG"
-        },
-        {
-          "columnId": 2,
-          "count": 5000,
-          "hasNull": false,
-          "min": -9214076359988107846,
-          "max": 9222919052987871506,
-          "type": "LONG"
-        },
-        {
-          "columnId": 3,
-          "count": 4950,
-          "hasNull": true,
-          "min": "Darkness,",
-          "max": "worst",
-          "totalLength": 19031,
-          "type": "STRING"
-        }
-      ]
-    },
-    {
-      "stripeNumber": 4,
-      "columnStatistics": [
-        {
-          "columnId": 0,
-          "count": 5000,
-          "hasNull": false
-        },
-        {
-          "columnId": 1,
-          "count": 5000,
-          "hasNull": false,
-          "min": -2146969085,
-          "max": 2146025044,
-          "sum": -5156814387,
-          "type": "LONG"
-        },
-        {
-          "columnId": 2,
-          "count": 5000,
-          "hasNull": false,
-          "min": -9222731174895935707,
-          "max": 9220625004936875965,
-          "type": "LONG"
-        },
-        {
-          "columnId": 3,
-          "count": 4950,
-          "hasNull": true,
-          "min": "Darkness,",
-          "max": "worst",
-          "totalLength": 19459,
-          "type": "STRING"
-        }
-      ]
-    },
-    {
-      "stripeNumber": 5,
-      "columnStatistics": [
-        {
-          "columnId": 0,
-          "count": 1000,
-          "hasNull": false
-        },
-        {
-          "columnId": 1,
-          "count": 1000,
-          "hasNull": false,
-          "min": -2144303438,
-          "max": 2127599049,
-          "sum": 62841564778,
-          "type": "LONG"
-        },
-        {
-          "columnId": 2,
-          "count": 1000,
-          "hasNull": false,
-          "min": -9195133638801798919,
-          "max": 9218626063131504414,
-          "type": "LONG"
-        },
-        {
-          "columnId": 3,
-          "count": 990,
-          "hasNull": true,
-          "min": "Darkness,",
-          "max": "worst",
-          "totalLength": 3963,
-          "type": "STRING"
-        }
-      ]
-    }
-  ],
-  "fileStatistics": [
-    {
-      "columnId": 0,
-      "count": 21000,
-      "hasNull": false
-    },
-    {
-      "columnId": 1,
-      "count": 21000,
-      "hasNull": false,
-      "min": -2147390285,
-      "max": 2147224606,
-      "sum": 106145458863,
-      "type": "LONG"
-    },
-    {
-      "columnId": 2,
-      "count": 21000,
-      "hasNull": false,
-      "min": -9223180583305557329,
-      "max": 9222919052987871506,
-      "type": "LONG"
-    },
-    {
-      "columnId": 3,
-      "count": 20790,
-      "hasNull": true,
-      "min": "Darkness,",
-      "max": "worst",
-      "totalLength": 81133,
-      "type": "STRING"
-    }
-  ],
-  "stripes": [
-    {
-      "stripeNumber": 1,
-      "stripeInformation": {
-        "offset": 3,
-        "indexLength": 970,
-        "dataLength": 63770,
-        "footerLength": 90,
-        "rowCount": 5000
-      },
-      "streams": [
-        {
-          "columnId": 0,
-          "section": "ROW_INDEX",
-          "startOffset": 3,
-          "length": 17
-        },
-        {
-          "columnId": 1,
-          "section": "ROW_INDEX",
-          "startOffset": 20,
-          "length": 167
-        },
-        {
-          "columnId": 2,
-          "section": "ROW_INDEX",
-          "startOffset": 187,
-          "length": 171
-        },
-        {
-          "columnId": 3,
-          "section": "ROW_INDEX",
-          "startOffset": 358,
-          "length": 103
-        },
-        {
-          "columnId": 3,
-          "section": "BLOOM_FILTER",
-          "startOffset": 461,
-          "length": 512
-        },
-        {
-          "columnId": 1,
-          "section": "DATA",
-          "startOffset": 973,
-          "length": 20035
-        },
-        {
-          "columnId": 2,
-          "section": "DATA",
-          "startOffset": 21008,
-          "length": 40050
-        },
-        {
-          "columnId": 3,
-          "section": "PRESENT",
-          "startOffset": 61058,
-          "length": 17
-        },
-        {
-          "columnId": 3,
-          "section": "DATA",
-          "startOffset": 61075,
-          "length": 3510
-        },
-        {
-          "columnId": 3,
-          "section": "LENGTH",
-          "startOffset": 64585,
-          "length": 25
-        },
-        {
-          "columnId": 3,
-          "section": "DICTIONARY_DATA",
-          "startOffset": 64610,
-          "length": 133
-        }
-      ],
-      "encodings": [
-        {
-          "columnId": 0,
-          "kind": "DIRECT"
-        },
-        {
-          "columnId": 1,
-          "kind": "DIRECT_V2"
-        },
-        {
-          "columnId": 2,
-          "kind": "DIRECT_V2"
-        },
-        {
-          "columnId": 3,
-          "kind": "DICTIONARY_V2",
-          "dictionarySize": 35
-        }
-      ],
-      "indexes": [{
-        "columnId": 3,
-        "rowGroupIndexes": [
-          {
-            "entryId": 0,
-            "count": 990,
-            "hasNull": true,
-            "min": "Darkness,",
-            "max": "worst",
-            "totalLength": 3873,
-            "type": "STRING",
-            "positions": [
-              0,
-              0,
-              0,
-              0,
-              0,
-              0,
-              0
-            ]
-          },
-          {
-            "entryId": 1,
-            "count": 990,
-            "hasNull": true,
-            "min": "Darkness,",
-            "max": "worst",
-            "totalLength": 3861,
-            "type": "STRING",
-            "positions": [
-              0,
-              38,
-              12,
-              0,
-              0,
-              736,
-              23
-            ]
-          },
-          {
-            "entryId": 2,
-            "count": 990,
-            "hasNull": true,
-            "min": "Darkness,",
-            "max": "worst",
-            "totalLength": 3946,
-            "type": "STRING",
-            "positions": [
-              0,
-              78,
-              12,
-              0,
-              0,
-              1473,
-              43
-            ]
-          },
-          {
-            "entryId": 3,
-            "count": 990,
-            "hasNull": true,
-            "min": "Darkness,",
-            "max": "worst",
-            "totalLength": 3774,
-            "type": "STRING",
-            "positions": [
-              0,
-              118,
-              12,
-              0,
-              0,
-              2067,
-              261
-            ]
-          },
-          {
-            "entryId": 4,
-            "count": 990,
-            "hasNull": true,
-            "min": "Darkness,",
-            "max": "worst",
-            "totalLength": 3829,
-            "type": "STRING",
-            "positions": [
-              0,
-              158,
-              12,
-              0,
-              0,
-              2992,
-              35
-            ]
-          }
-        ],
-        "bloomFilterIndexes": [
-          {
-            "entryId": 0,
-            "numHashFunctions": 4,
-            "bitCount": 6272,
-            "popCount": 138,
-            "loadFactor": 0.022002551704645157,
-            "expectedFpp": 2.3436470542037569E-7
-          },
-          {
-            "entryId": 1,
-            "numHashFunctions": 4,
-            "bitCount": 6272,
-            "popCount": 138,
-            "loadFactor": 0.022002551704645157,
-            "expectedFpp": 2.3436470542037569E-7
-          },
-          {
-            "entryId": 2,
-            "numHashFunctions": 4,
-            "bitCount": 6272,
-            "popCount": 138,
-            "loadFactor": 0.022002551704645157,
-            "expectedFpp": 2.3436470542037569E-7
-          },
-          {
-            "entryId": 3,
-            "numHashFunctions": 4,
-            "bitCount": 6272,
-            "popCount": 138,
-            "loadFactor": 0.022002551704645157,
-            "expectedFpp": 2.3436470542037569E-7
-          },
-          {
-            "entryId": 4,
-            "numHashFunctions": 4,
-            "bitCount": 6272,
-            "popCount": 138,
-            "loadFactor": 0.022002551704645157,
-            "expectedFpp": 2.3436470542037569E-7
-          }
-        ],
-        "stripeLevelBloomFilter": {
-          "numHashFunctions": 4,
-          "bitCount": 6272,
-          "popCount": 138,
-          "loadFactor": 0.022002551704645157,
-          "expectedFpp": 2.3436470542037569E-7
-        }
-      }]
-    },
-    {
-      "stripeNumber": 2,
-      "stripeInformation": {
-        "offset": 64833,
-        "indexLength": 961,
-        "dataLength": 63763,
-        "footerLength": 88,
-        "rowCount": 5000
-      },
-      "streams": [
-        {
-          "columnId": 0,
-          "section": "ROW_INDEX",
-          "startOffset": 64833,
-          "length": 17
-        },
-        {
-          "columnId": 1,
-          "section": "ROW_INDEX",
-          "startOffset": 64850,
-          "length": 166
-        },
-        {
-          "columnId": 2,
-          "section": "ROW_INDEX",
-          "startOffset": 65016,
-          "length": 166
-        },
-        {
-          "columnId": 3,
-          "section": "ROW_INDEX",
-          "startOffset": 65182,
-          "length": 100
-        },
-        {
-          "columnId": 3,
-          "section": "BLOOM_FILTER",
-          "startOffset": 65282,
-          "length": 512
-        },
-        {
-          "columnId": 1,
-          "section": "DATA",
-          "startOffset": 65794,
-          "length": 20035
-        },
-        {
-          "columnId": 2,
-          "section": "DATA",
-          "startOffset": 85829,
-          "length": 40050
-        },
-        {
-          "columnId": 3,
-          "section": "PRESENT",
-          "startOffset": 125879,
-          "length": 17
-        },
-        {
-          "columnId": 3,
-          "section": "DATA",
-          "startOffset": 125896,
-          "length": 3503
-        },
-        {
-          "columnId": 3,
-          "section": "LENGTH",
-          "startOffset": 129399,
-          "length": 25
-        },
-        {
-          "columnId": 3,
-          "section": "DICTIONARY_DATA",
-          "startOffset": 129424,
-          "length": 133
-        }
-      ],
-      "encodings": [
-        {
-          "columnId": 0,
-          "kind": "DIRECT"
-        },
-        {
-          "columnId": 1,
-          "kind": "DIRECT_V2"
-        },
-        {
-          "columnId": 2,
-          "kind": "DIRECT_V2"
-        },
-        {
-          "columnId": 3,
-          "kind": "DICTIONARY_V2",
-          "dictionarySize": 35
-        }
-      ],
-      "indexes": [{
-        "columnId": 3,
-        "rowGroupIndexes": [
-          {
-            "entryId": 0,
-            "count": 990,
-            "hasNull": true,
-            "min": "Darkness,",
-            "max": "worst",
-            "totalLength": 3946,
-            "type": "STRING",
-            "positions": [
-              0,
-              0,
-              0,
-              0,
-              0,
-              0,
-              0
-            ]
-          },
-          {
-            "entryId": 1,
-            "count": 990,
-            "hasNull": true,
-            "min": "Darkness,",
-            "max": "worst",
-            "totalLength": 3836,
-            "type": "STRING",
-            "positions": [
-              0,
-              38,
-              12,
-              0,
-              0,
-              746,
-              11
-            ]
-          },
-          {
-            "entryId": 2,
-            "count": 990,
-            "hasNull": true,
-            "min": "Darkness,",
-            "max": "worst",
-            "totalLength": 3791,
-            "type": "STRING",
-            "positions": [
-              0,
-              78,
-              12,
-              0,
-              0,
-              1430,
-              95
-            ]
-          },
-          {
-            "entryId": 3,
-            "count": 990,
-            "hasNull": true,
-            "min": "Darkness,",
-            "max": "worst",
-            "totalLength": 3904,
-            "type": "STRING",
-            "positions": [
-              0,
-              118,
-              12,
-              0,
-              0,
-              2239,
-              23
-            ]
-          },
-          {
-            "entryId": 4,
-            "count": 990,
-            "hasNull": true,
-            "min": "Darkness,",
-            "max": "worst",
-            "totalLength": 3920,
-            "type": "STRING",
-            "positions": [
-              0,
-              158,
-              12,
-              0,
-              0,
-              2994,
-              17
-            ]
-          }
-        ],
-        "bloomFilterIndexes": [
-          {
-            "entryId": 0,
-            "numHashFunctions": 4,
-            "bitCount": 6272,
-            "popCount": 138,
-            "loadFactor": 0.022002551704645157,
-            "expectedFpp": 2.3436470542037569E-7
-          },
-          {
-            "entryId": 1,
-            "numHashFunctions": 4,
-            "bitCount": 6272,
-            "popCount": 138,
-            "loadFactor": 0.022002551704645157,
-            "expectedFpp": 2.3436470542037569E-7
-          },
-          {
-            "entryId": 2,
-            "numHashFunctions": 4,
-            "bitCount": 6272,
-            "popCount": 138,
-            "loadFactor": 0.022002551704645157,
-            "expectedFpp": 2.3436470542037569E-7
-          },
-          {
-            "entryId": 3,
-            "numHashFunctions": 4,
-            "bitCount": 6272,
-            "popCount": 138,
-            "loadFactor": 0.022002551704645157,
-            "expectedFpp": 2.3436470542037569E-7
-          },
-          {
-            "entryId": 4,
-            "numHashFunctions": 4,
-            "bitCount": 6272,
-            "popCount": 138,
-            "loadFactor": 0.022002551704645157,
-            "expectedFpp": 2.3436470542037569E-7
-          }
-        ],
-        "stripeLevelBloomFilter": {
-          "numHashFunctions": 4,
-          "bitCount": 6272,
-          "popCount": 138,
-          "loadFactor": 0.022002551704645157,
-          "expectedFpp": 2.3436470542037569E-7
-        }
-      }]
-    },
-    {
-      "stripeNumber": 3,
-      "stripeInformation": {
-        "offset": 129645,
-        "indexLength": 962,
-        "dataLength": 63770,
-        "footerLength": 91,
-        "rowCount": 5000
-      },
-      "streams": [
-        {
-          "columnId": 0,
-          "section": "ROW_INDEX",
-          "startOffset": 129645,
-          "length": 17
-        },
-        {
-          "columnId": 1,
-          "section": "ROW_INDEX",
-          "startOffset": 129662,
-          "length": 164
-        },
-        {
-          "columnId": 2,
-          "section": "ROW_INDEX",
-          "startOffset": 129826,
-          "length": 167
-        },
-        {
-          "columnId": 3,
-          "section": "ROW_INDEX",
-          "startOffset": 129993,
-          "length": 102
-        },
-        {
-          "columnId": 3,
-          "section": "BLOOM_FILTER",
-          "startOffset": 130095,
-          "length": 512
-        },
-        {
-          "columnId": 1,
-          "section": "DATA",
-          "startOffset": 130607,
-          "length": 20035
-        },
-        {
-          "columnId": 2,
-          "section": "DATA",
-          "startOffset": 150642,
-          "length": 40050
-        },
-        {
-          "columnId": 3,
-          "section": "PRESENT",
-          "startOffset": 190692,
-          "length": 17
-        },
-        {
-          "columnId": 3,
-          "section": "DATA",
-          "startOffset": 190709,
-          "length": 3510
-        },
-        {
-          "columnId": 3,
-          "section": "LENGTH",
-          "startOffset": 194219,
-          "length": 25
-        },
-        {
-          "columnId": 3,
-          "section": "DICTIONARY_DATA",
-          "startOffset": 194244,
-          "length": 133
-        }
-      ],
-      "encodings": [
-        {
-          "columnId": 0,
-          "kind": "DIRECT"
-        },
-        {
-          "columnId": 1,
-          "kind": "DIRECT_V2"
-        },
-        {
-          "columnId": 2,
-          "kind": "DIRECT_V2"
-        },
-        {
-          "columnId": 3,
-          "kind": "DICTIONARY_V2",
-          "dictionarySize": 35
-        }
-      ],
-      "indexes": [{
-        "columnId": 3,
-        "rowGroupIndexes": [
-          {
-            "entryId": 0,
-            "count": 990,
-            "hasNull": true,
-            "min": "Darkness,",
-            "max": "worst",
-            "totalLength": 3829,
-            "type": "STRING",
-            "positions": [
-              0,
-              0,
-              0,
-              0,
-              0,
-              0,
-              0
-            ]
-          },
-          {
-            "entryId": 1,
-            "count": 990,
-            "hasNull": true,
-            "min": "Darkness,",
-            "max": "worst",
-            "totalLength": 3853,
-            "type": "STRING",
-            "positions": [
-              0,
-              38,
-              12,
-              0,
-              0,
-              698,
-              74
-            ]
-          },
-          {
-            "entryId": 2,
-            "count": 990,
-            "hasNull": true,
-            "min": "Darkness,",
-            "max": "worst",
-            "totalLength": 3796,
-            "type": "STRING",
-            "positions": [
-              0,
-              78,
-              12,
-              0,
-              0,
-              1483,
-              39
-            ]
-          },
-          {
-            "entryId": 3,
-            "count": 990,
-            "hasNull": true,
-            "min": "Darkness,",
-            "max": "worst",
-            "totalLength": 3736,
-            "type": "STRING",
-            "positions": [
-              0,
-              118,
-              12,
-              0,
-              0,
-              2148,
-              155
-            ]
-          },
-          {
-            "entryId": 4,
-            "count": 990,
-            "hasNull": true,
-            "min": "Darkness,",
-            "max": "worst",
-            "totalLength": 3817,
-            "type": "STRING",
-            "positions": [
-              0,
-              158,
-              12,
-              0,
-              0,
-              3018,
-              8
-            ]
-          }
-        ],
-        "bloomFilterIndexes": [
-          {
-            "entryId": 0,
-            "numHashFunctions": 4,
-            "bitCount": 6272,
-            "popCount": 138,
-            "loadFactor": 0.022002551704645157,
-            "expectedFpp": 2.3436470542037569E-7
-          },
-          {
-            "entryId": 1,
-            "numHashFunctions": 4,
-            "bitCount": 6272,
-            "popCount": 138,
-            "loadFactor": 0.022002551704645157,
-            "expectedFpp": 2.3436470542037569E-7
-          },
-          {
-            "entryId": 2,
-            "numHashFunctions": 4,
-            "bitCount": 6272,
-            "popCount": 138,
-            "loadFactor": 0.022002551704645157,
-            "expectedFpp": 2.3436470542037569E-7
-          },
-          {
-            "entryId": 3,
-            "numHashFunctions": 4,
-            "bitCount": 6272,
-            "popCount": 138,
-            "loadFactor": 0.022002551704645157,
-            "expectedFpp": 2.3436470542037569E-7
-          },
-          {
-            "entryId": 4,
-            "numHashFunctions": 4,
-            "bitCount": 6272,
-            "popCount": 138,
-            "loadFactor": 0.022002551704645157,
-            "expectedFpp": 2.3436470542037569E-7
-          }
-        ],
-        "stripeLevelBloomFilter": {
-          "numHashFunctions": 4,
-          "bitCount": 6272,
-          "popCount": 138,
-          "loadFactor": 0.022002551704645157,
-          "expectedFpp": 2.3436470542037569E-7
-        }
-      }]
-    },
-    {
-      "stripeNumber": 4,
-      "stripeInformation": {
-        "offset": 194468,
-        "indexLength": 973,
-        "dataLength": 63756,
-        "footerLength": 91,
-        "rowCount": 5000
-      },
-      "streams": [
-        {
-          "columnId": 0,
-          "section": "ROW_INDEX",
-          "startOffset": 194468,
-          "length": 17
-        },
-        {
-          "columnId": 1,
-          "section": "ROW_INDEX",
-          "startOffset": 194485,
-          "length": 166
-        },
-        {
-          "columnId": 2,
-          "section": "ROW_INDEX",
-          "startOffset": 194651,
-          "length": 171
-        },
-        {
-          "columnId": 3,
-          "section": "ROW_INDEX",
-          "startOffset": 194822,
-          "length": 107
-        },
-        {
-          "columnId": 3,
-          "section": "BLOOM_FILTER",
-          "startOffset": 194929,
-          "length": 512
-        },
-        {
-          "columnId": 1,
-          "section": "DATA",
-          "startOffset": 195441,
-          "length": 20035
-        },
-        {
-          "columnId": 2,
-          "section": "DATA",
-          "startOffset": 215476,
-          "length": 40050
-        },
-        {
-          "columnId": 3,
-          "section": "PRESENT",
-          "startOffset": 255526,
-          "length": 17
-        },
-        {
-          "columnId": 3,
-          "section": "DATA",
-          "startOffset": 255543,
-          "length": 3496
-        },
-        {
-          "columnId": 3,
-          "section": "LENGTH",
-          "startOffset": 259039,
-          "length": 25
-        },
-        {
-          "columnId": 3,
-          "section": "DICTIONARY_DATA",
-          "startOffset": 259064,
-          "length": 133
-        }
-      ],
-      "encodings": [
-        {
-          "columnId": 0,
-          "kind": "DIRECT"
-        },
-        {
-          "columnId": 1,
-          "kind": "DIRECT_V2"
-        },
-        {
-          "columnId": 2,
-          "kind": "DIRECT_V2"
-        },
-        {
-          "columnId": 3,
-          "kind": "DICTIONARY_V2",
-          "dictionarySize": 35
-        }
-      ],
-      "indexes": [{
-        "columnId": 3,
-        "rowGroupIndexes": [
-          {
-            "entryId": 0,
-            "count": 990,
-            "hasNull": true,
-            "min": "Darkness,",
-            "max": "worst",
-            "totalLength": 3959,
-            "type": "STRING",
-            "positions": [
-              0,
-              0,
-              0,
-              0,
-              0,
-              0,
-              0
-            ]
-          },
-          {
-            "entryId": 1,
-            "count": 990,
-            "hasNull": true,
-            "min": "Darkness,",
-            "max": "worst",
-            "totalLength": 3816,
-            "type": "STRING",
-            "positions": [
-              0,
-              38,
-              12,
-              0,
-              0,
-              495,
-              338
-            ]
-          },
-          {
-            "entryId": 2,
-            "count": 990,
-            "hasNull": true,
-            "min": "Darkness,",
-            "max": "worst",
-            "totalLength": 3883,
-            "type": "STRING",
-            "positions": [
-              0,
-              78,
-              12,
-              0,
-              0,
-              1449,
-              71
-            ]
-          },
-          {
-            "entryId": 3,
-            "count": 990,
-            "hasNull": true,
-            "min": "Darkness,",
-            "max": "worst",
-            "totalLength": 3938,
-            "type": "STRING",
-            "positions": [
-              0,
-              118,
-              12,
-              0,
-              0,
-              2207,
-              59
-            ]
-          },
-          {
-            "entryId": 4,
-            "count": 990,
-            "hasNull": true,
-            "min": "Darkness,",
-            "max": "worst",
-            "totalLength": 3863,
-            "type": "STRING",
-            "positions": [
-              0,
-              158,
-              12,
-              0,
-              0,
-              2838,
-              223
-            ]
-          }
-        ],
-        "bloomFilterIndexes": [
-          {
-            "entryId": 0,
-            "numHashFunctions": 4,
-            "bitCount": 6272,
-            "popCount": 138,
-            "loadFactor": 0.022002551704645157,
-            "expectedFpp": 2.3436470542037569E-7
-          },
-          {
-            "entryId": 1,
-            "numHashFunctions": 4,
-            "bitCount": 6272,
-            "popCount": 138,
-            "loadFactor": 0.022002551704645157,
-            "expectedFpp": 2.3436470542037569E-7
-          },
-          {
-            "entryId": 2,
-            "numHashFunctions": 4,
-            "bitCount": 6272,
-            "popCount": 138,
-            "loadFactor": 0.022002551704645157,
-            "expectedFpp": 2.3436470542037569E-7
-          },
-          {
-            "entryId": 3,
-            "numHashFunctions": 4,
-            "bitCount": 6272,
-            "popCount": 138,
-            "loadFactor": 0.022002551704645157,
-            "expectedFpp": 2.3436470542037569E-7
-          },
-          {
-            "entryId": 4,
-            "numHashFunctions": 4,
-            "bitCount": 6272,
-            "popCount": 138,
-            "loadFactor": 0.022002551704645157,
-            "expectedFpp": 2.3436470542037569E-7
-          }
-        ],
-        "stripeLevelBloomFilter": {
-          "numHashFunctions": 4,
-          "bitCount": 6272,
-          "popCount": 138,
-          "loadFactor": 0.022002551704645157,
-          "expectedFpp": 2.3436470542037569E-7
-        }
-      }]
-    },
-    {
-      "stripeNumber": 5,
-      "stripeInformation": {
-        "offset": 259288,
-        "indexLength": 433,
-        "dataLength": 12943,
-        "footerLength": 83,
-        "rowCount": 1000
-      },
-      "streams": [
-        {
-          "columnId": 0,
-          "section": "ROW_INDEX",
-          "startOffset": 259288,
-          "length": 12
-        },
-        {
-          "columnId": 1,
-          "section": "ROW_INDEX",
-          "startOffset": 259300,
-          "length": 38
-        },
-        {
-          "columnId": 2,
-          "section": "ROW_INDEX",
-          "startOffset": 259338,
-          "length": 41
-        },
-        {
-          "columnId": 3,
-          "section": "ROW_INDEX",
-          "startOffset": 259379,
-          "length": 41
-        },
-        {
-          "columnId": 3,
-          "section": "BLOOM_FILTER",
-          "startOffset": 259420,
-          "length": 301
-        },
-        {
-          "columnId": 1,
-          "section": "DATA",
-          "startOffset": 259721,
-          "length": 4007
-        },
-        {
-          "columnId": 2,
-          "section": "DATA",
-          "startOffset": 263728,
-          "length": 8010
-        },
-        {
-          "columnId": 3,
-          "section": "PRESENT",
-          "startOffset": 271738,
-          "length": 16
-        },
-        {
-          "columnId": 3,
-          "section": "DATA",
-          "startOffset": 271754,
-          "length": 752
-        },
-        {
-          "columnId": 3,
-          "section": "LENGTH",
-          "startOffset": 272506,
-          "length": 25
-        },
-        {
-          "columnId": 3,
-          "section": "DICTIONARY_DATA",
-          "startOffset": 272531,
-          "length": 133
-        }
-      ],
-      "encodings": [
-        {
-          "columnId": 0,
-          "kind": "DIRECT"
-        },
-        {
-          "columnId": 1,
-          "kind": "DIRECT_V2"
-        },
-        {
-          "columnId": 2,
-          "kind": "DIRECT_V2"
-        },
-        {
-          "columnId": 3,
-          "kind": "DICTIONARY_V2",
-          "dictionarySize": 35
-        }
-      ],
-      "indexes": [{
-        "columnId": 3,
-        "rowGroupIndexes": [{
-          "entryId": 0,
-          "count": 990,
-          "hasNull": true,
-          "min": "Darkness,",
-          "max": "worst",
-          "totalLength": 3963,
-          "type": "STRING",
-          "positions": [
-            0,
-            0,
-            0,
-            0,
-            0,
-            0,
-            0
-          ]
-        }],
-        "bloomFilterIndexes": [{
-          "entryId": 0,
-          "numHashFunctions": 4,
-          "bitCount": 6272,
-          "popCount": 138,
-          "loadFactor": 0.022002551704645157,
-          "expectedFpp": 2.3436470542037569E-7
-        }],
-        "stripeLevelBloomFilter": {
-          "numHashFunctions": 4,
-          "bitCount": 6272,
-          "popCount": 138,
-          "loadFactor": 0.022002551704645157,
-          "expectedFpp": 2.3436470542037569E-7
-        }
-      }]
-    }
-  ],
-  "fileLength": 273300,
-  "paddingLength": 0,
-  "paddingRatio": 0,
-  "status": "OK"
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/ql/src/test/resources/orc-file-dump.out
----------------------------------------------------------------------
diff --git a/ql/src/test/resources/orc-file-dump.out b/ql/src/test/resources/orc-file-dump.out
deleted file mode 100644
index 70f7fbd..0000000
--- a/ql/src/test/resources/orc-file-dump.out
+++ /dev/null
@@ -1,195 +0,0 @@
-Structure for TestFileDump.testDump.orc
-File Version: 0.12 with HIVE_13083
-Rows: 21000
-Compression: ZLIB
-Compression size: 4096
-Type: struct<i:int,l:bigint,s:string>
-
-Stripe Statistics:
-  Stripe 1:
-    Column 0: count: 5000 hasNull: false
-    Column 1: count: 5000 hasNull: false min: -2146021688 max: 2147223299 sum: 515792826
-    Column 2: count: 5000 hasNull: false min: -9218592812243954469 max: 9221614132680747961
-    Column 3: count: 5000 hasNull: false min: Darkness, max: worst sum: 19280
-  Stripe 2:
-    Column 0: count: 5000 hasNull: false
-    Column 1: count: 5000 hasNull: false min: -2146733128 max: 2147001622 sum: 7673427
-    Column 2: count: 5000 hasNull: false min: -9220818777591257749 max: 9222259462014003839
-    Column 3: count: 5000 hasNull: false min: Darkness, max: worst sum: 19504
-  Stripe 3:
-    Column 0: count: 5000 hasNull: false
-    Column 1: count: 5000 hasNull: false min: -2146993718 max: 2147378179 sum: 132660742551
-    Column 2: count: 5000 hasNull: false min: -9218342074710552826 max: 9222303228623055266
-    Column 3: count: 5000 hasNull: false min: Darkness, max: worst sum: 19641
-  Stripe 4:
-    Column 0: count: 5000 hasNull: false
-    Column 1: count: 5000 hasNull: false min: -2146658006 max: 2145520931 sum: 8533549236
-    Column 2: count: 5000 hasNull: false min: -9222758097219661129 max: 9221043130193737406
-    Column 3: count: 5000 hasNull: false min: Darkness, max: worst sum: 19470
-  Stripe 5:
-    Column 0: count: 1000 hasNull: false
-    Column 1: count: 1000 hasNull: false min: -2146245500 max: 2146378640 sum: 51299706363
-    Column 2: count: 1000 hasNull: false min: -9208193203370316142 max: 9218567213558056476
-    Column 3: count: 1000 hasNull: false min: Darkness, max: worst sum: 3866
-
-File Statistics:
-  Column 0: count: 21000 hasNull: false
-  Column 1: count: 21000 hasNull: false min: -2146993718 max: 2147378179 sum: 193017464403
-  Column 2: count: 21000 hasNull: false min: -9222758097219661129 max: 9222303228623055266
-  Column 3: count: 21000 hasNull: false min: Darkness, max: worst sum: 81761
-
-Stripes:
-  Stripe: offset: 3 data: 63786 rows: 5000 tail: 79 index: 439
-    Stream: column 0 section ROW_INDEX start: 3 length 17
-    Stream: column 1 section ROW_INDEX start: 20 length 166
-    Stream: column 2 section ROW_INDEX start: 186 length 169
-    Stream: column 3 section ROW_INDEX start: 355 length 87
-    Stream: column 1 section DATA start: 442 length 20035
-    Stream: column 2 section DATA start: 20477 length 40050
-    Stream: column 3 section DATA start: 60527 length 3543
-    Stream: column 3 section LENGTH start: 64070 length 25
-    Stream: column 3 section DICTIONARY_DATA start: 64095 length 133
-    Encoding column 0: DIRECT
-    Encoding column 1: DIRECT_V2
-    Encoding column 2: DIRECT_V2
-    Encoding column 3: DICTIONARY_V2[35]
-    Row group indices for column 1:
-      Entry 0: count: 1000 hasNull: false min: -2145365268 max: 2135491313 sum: 7521792925 positions: 0,0,0
-      Entry 1: count: 1000 hasNull: false min: -2139452528 max: 2147223299 sum: -12923774313 positions: 0,2050,488
-      Entry 2: count: 1000 hasNull: false min: -2142420586 max: 2143898386 sum: -25521983511 positions: 4099,2054,464
-      Entry 3: count: 1000 hasNull: false min: -2137233441 max: 2144267163 sum: 40993386199 positions: 8198,2058,440
-      Entry 4: count: 1000 hasNull: false min: -2146021688 max: 2146838901 sum: -9553628474 positions: 12297,2062,416
-    Row group indices for column 2:
-      Entry 0: count: 1000 hasNull: false min: -9200577545527640566 max: 9175500305011173751 positions: 0,0,0
-      Entry 1: count: 1000 hasNull: false min: -9203618157670445774 max: 9208123824411178101 positions: 4099,2,488
-      Entry 2: count: 1000 hasNull: false min: -9218592812243954469 max: 9221351515892923972 positions: 12297,6,464
-      Entry 3: count: 1000 hasNull: false min: -9206585617947511272 max: 9167703224425685487 positions: 20495,10,440
-      Entry 4: count: 1000 hasNull: false min: -9206645795733282496 max: 9221614132680747961 positions: 28693,14,416
-    Row group indices for column 3:
-      Entry 0: count: 1000 hasNull: false min: Darkness, max: worst sum: 3862 positions: 0,0,0
-      Entry 1: count: 1000 hasNull: false min: Darkness, max: worst sum: 3884 positions: 0,659,149
-      Entry 2: count: 1000 hasNull: false min: Darkness, max: worst sum: 3893 positions: 0,1531,3
-      Entry 3: count: 1000 hasNull: false min: Darkness, max: worst sum: 3798 positions: 0,2281,32
-      Entry 4: count: 1000 hasNull: false min: Darkness, max: worst sum: 3843 positions: 0,3033,45
-  Stripe: offset: 64307 data: 63775 rows: 5000 tail: 79 index: 432
-    Stream: column 0 section ROW_INDEX start: 64307 length 17
-    Stream: column 1 section ROW_INDEX start: 64324 length 164
-    Stream: column 2 section ROW_INDEX start: 64488 length 168
-    Stream: column 3 section ROW_INDEX start: 64656 length 83
-    Stream: column 1 section DATA start: 64739 length 20035
-    Stream: column 2 section DATA start: 84774 length 40050
-    Stream: column 3 section DATA start: 124824 length 3532
-    Stream: column 3 section LENGTH start: 128356 length 25
-    Stream: column 3 section DICTIONARY_DATA start: 128381 length 133
-    Encoding column 0: DIRECT
-    Encoding column 1: DIRECT_V2
-    Encoding column 2: DIRECT_V2
-    Encoding column 3: DICTIONARY_V2[35]
-    Row group indices for column 1:
-      Entry 0: count: 1000 hasNull: false min: -2143799121 max: 2145249879 sum: -6966266181 positions: 0,0,0
-      Entry 1: count: 1000 hasNull: false min: -2146733128 max: 2147001622 sum: -35930106333 positions: 0,2050,488
-      Entry 2: count: 1000 hasNull: false min: -2144302712 max: 2146299933 sum: 6944230435 positions: 4099,2054,464
-      Entry 3: count: 1000 hasNull: false min: -2145172948 max: 2144335014 sum: -29624404959 positions: 8198,2058,440
-      Entry 4: count: 1000 hasNull: false min: -2146428427 max: 2144067253 sum: 65584220465 positions: 12297,2062,416
-    Row group indices for column 2:
-      Entry 0: count: 1000 hasNull: false min: -9218450653857701562 max: 9189819526332228512 positions: 0,0,0
-      Entry 1: count: 1000 hasNull: false min: -9220818777591257749 max: 9178821722829648113 positions: 4099,2,488
-      Entry 2: count: 1000 hasNull: false min: -9220031433030423388 max: 9210838931786956852 positions: 12297,6,464
-      Entry 3: count: 1000 hasNull: false min: -9208195729739635607 max: 9222259462014003839 positions: 20495,10,440
-      Entry 4: count: 1000 hasNull: false min: -9174271499932339698 max: 9212277876771676916 positions: 28693,14,416
-    Row group indices for column 3:
-      Entry 0: count: 1000 hasNull: false min: Darkness, max: worst sum: 3923 positions: 0,0,0
-      Entry 1: count: 1000 hasNull: false min: Darkness, max: worst sum: 3869 positions: 0,761,12
-      Entry 2: count: 1000 hasNull: false min: Darkness, max: worst sum: 3817 positions: 0,1472,70
-      Entry 3: count: 1000 hasNull: false min: Darkness, max: worst sum: 3931 positions: 0,2250,43
-      Entry 4: count: 1000 hasNull: false min: Darkness, max: worst sum: 3964 positions: 0,2978,88
-  Stripe: offset: 128593 data: 63787 rows: 5000 tail: 79 index: 438
-    Stream: column 0 section ROW_INDEX start: 128593 length 17
-    Stream: column 1 section ROW_INDEX start: 128610 length 163
-    Stream: column 2 section ROW_INDEX start: 128773 length 168
-    Stream: column 3 section ROW_INDEX start: 128941 length 90
-    Stream: column 1 section DATA start: 129031 length 20035
-    Stream: column 2 section DATA start: 149066 length 40050
-    Stream: column 3 section DATA start: 189116 length 3544
-    Stream: column 3 section LENGTH start: 192660 length 25
-    Stream: column 3 section DICTIONARY_DATA start: 192685 length 133
-    Encoding column 0: DIRECT
-    Encoding column 1: DIRECT_V2
-    Encoding column 2: DIRECT_V2
-    Encoding column 3: DICTIONARY_V2[35]
-    Row group indices for column 1:
-      Entry 0: count: 1000 hasNull: false min: -2146993718 max: 2144179881 sum: -7829543271 positions: 0,0,0
-      Entry 1: count: 1000 hasNull: false min: -2144095505 max: 2144883384 sum: 51623839692 positions: 0,2050,488
-      Entry 2: count: 1000 hasNull: false min: -2144113995 max: 2143773575 sum: 56574412741 positions: 4099,2054,464
-      Entry 3: count: 1000 hasNull: false min: -2146954065 max: 2146794873 sum: 4336083432 positions: 8198,2058,440
-      Entry 4: count: 1000 hasNull: false min: -2135511523 max: 2147378179 sum: 27955949957 positions: 12297,2062,416
-    Row group indices for column 2:
-      Entry 0: count: 1000 hasNull: false min: -9211978436552246208 max: 9179058898902097152 positions: 0,0,0
-      Entry 1: count: 1000 hasNull: false min: -9195645160817780503 max: 9189147759444307708 positions: 4099,2,488
-      Entry 2: count: 1000 hasNull: false min: -9202888157616520823 max: 9193561362676960747 positions: 12297,6,464
-      Entry 3: count: 1000 hasNull: false min: -9216318198067839390 max: 9221286760675829363 positions: 20495,10,440
-      Entry 4: count: 1000 hasNull: false min: -9218342074710552826 max: 9222303228623055266 positions: 28693,14,416
-    Row group indices for column 3:
-      Entry 0: count: 1000 hasNull: false min: Darkness, max: worst sum: 3817 positions: 0,0,0
-      Entry 1: count: 1000 hasNull: false min: Darkness, max: worst sum: 4008 positions: 0,634,174
-      Entry 2: count: 1000 hasNull: false min: Darkness, max: worst sum: 3999 positions: 0,1469,69
-      Entry 3: count: 1000 hasNull: false min: Darkness, max: worst sum: 3817 positions: 0,2133,194
-      Entry 4: count: 1000 hasNull: false min: Darkness, max: worst sum: 4000 positions: 0,3005,43
-  Stripe: offset: 192897 data: 63817 rows: 5000 tail: 79 index: 440
-    Stream: column 0 section ROW_INDEX start: 192897 length 17
-    Stream: column 1 section ROW_INDEX start: 192914 length 165
-    Stream: column 2 section ROW_INDEX start: 193079 length 167
-    Stream: column 3 section ROW_INDEX start: 193246 length 91
-    Stream: column 1 section DATA start: 193337 length 20035
-    Stream: column 2 section DATA start: 213372 length 40050
-    Stream: column 3 section DATA start: 253422 length 3574
-    Stream: column 3 section LENGTH start: 256996 length 25
-    Stream: column 3 section DICTIONARY_DATA start: 257021 length 133
-    Encoding column 0: DIRECT
-    Encoding column 1: DIRECT_V2
-    Encoding column 2: DIRECT_V2
-    Encoding column 3: DICTIONARY_V2[35]
-    Row group indices for column 1:
-      Entry 0: count: 1000 hasNull: false min: -2141355639 max: 2145520931 sum: 2726719912 positions: 0,0,0
-      Entry 1: count: 1000 hasNull: false min: -2138324170 max: 2140167376 sum: -23606674002 positions: 0,2050,488
-      Entry 2: count: 1000 hasNull: false min: -2146658006 max: 2144329742 sum: -41530109703 positions: 4099,2054,464
-      Entry 3: count: 1000 hasNull: false min: -2144207593 max: 2139456355 sum: 13559842458 positions: 8198,2058,440
-      Entry 4: count: 1000 hasNull: false min: -2145744719 max: 2145417153 sum: 57383770571 positions: 12297,2062,416
-    Row group indices for column 2:
-      Entry 0: count: 1000 hasNull: false min: -9222731174895935707 max: 9214167447015056056 positions: 0,0,0
-      Entry 1: count: 1000 hasNull: false min: -9222758097219661129 max: 9221043130193737406 positions: 4099,2,488
-      Entry 2: count: 1000 hasNull: false min: -9174483776261243438 max: 9208134757538374043 positions: 12297,6,464
-      Entry 3: count: 1000 hasNull: false min: -9174329712613510612 max: 9197412874152820822 positions: 20495,10,440
-      Entry 4: count: 1000 hasNull: false min: -9221162005892422758 max: 9220625004936875965 positions: 28693,14,416
-    Row group indices for column 3:
-      Entry 0: count: 1000 hasNull: false min: Darkness, max: worst sum: 3901 positions: 0,0,0
-      Entry 1: count: 1000 hasNull: false min: Darkness, max: worst sum: 3900 positions: 0,431,431
-      Entry 2: count: 1000 hasNull: false min: Darkness, max: worst sum: 3909 positions: 0,1485,52
-      Entry 3: count: 1000 hasNull: false min: Darkness, max: worst sum: 3947 positions: 0,2196,104
-      Entry 4: count: 1000 hasNull: false min: Darkness, max: worst sum: 3813 positions: 0,2934,131
-  Stripe: offset: 257233 data: 12943 rows: 1000 tail: 71 index: 131
-    Stream: column 0 section ROW_INDEX start: 257233 length 12
-    Stream: column 1 section ROW_INDEX start: 257245 length 38
-    Stream: column 2 section ROW_INDEX start: 257283 length 41
-    Stream: column 3 section ROW_INDEX start: 257324 length 40
-    Stream: column 1 section DATA start: 257364 length 4007
-    Stream: column 2 section DATA start: 261371 length 8010
-    Stream: column 3 section DATA start: 269381 length 768
-    Stream: column 3 section LENGTH start: 270149 length 25
-    Stream: column 3 section DICTIONARY_DATA start: 270174 length 133
-    Encoding column 0: DIRECT
-    Encoding column 1: DIRECT_V2
-    Encoding column 2: DIRECT_V2
-    Encoding column 3: DICTIONARY_V2[35]
-    Row group indices for column 1:
-      Entry 0: count: 1000 hasNull: false min: -2146245500 max: 2146378640 sum: 51299706363 positions: 0,0,0
-    Row group indices for column 2:
-      Entry 0: count: 1000 hasNull: false min: -9208193203370316142 max: 9218567213558056476 positions: 0,0,0
-    Row group indices for column 3:
-      Entry 0: count: 1000 hasNull: false min: Darkness, max: worst sum: 3866 positions: 0,0,0
-
-File length: 270923 bytes
-Padding length: 0 bytes
-Padding ratio: 0%
-________________________________________________________________________________________________________________________
-

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/ql/src/test/resources/orc-file-has-null.out
----------------------------------------------------------------------
diff --git a/ql/src/test/resources/orc-file-has-null.out b/ql/src/test/resources/orc-file-has-null.out
deleted file mode 100644
index e98a73f..0000000
--- a/ql/src/test/resources/orc-file-has-null.out
+++ /dev/null
@@ -1,112 +0,0 @@
-Structure for TestOrcFile.testHasNull.orc
-File Version: 0.12 with HIVE_13083
-Rows: 20000
-Compression: ZLIB
-Compression size: 4096
-Type: struct<bytes1:binary,string1:string>
-
-Stripe Statistics:
-  Stripe 1:
-    Column 0: count: 5000 hasNull: false
-    Column 1: count: 5000 hasNull: false sum: 15000
-    Column 2: count: 2000 hasNull: true min: RG1 max: RG3 sum: 6000
-  Stripe 2:
-    Column 0: count: 5000 hasNull: false
-    Column 1: count: 5000 hasNull: false sum: 15000
-    Column 2: count: 0 hasNull: true
-  Stripe 3:
-    Column 0: count: 5000 hasNull: false
-    Column 1: count: 5000 hasNull: false sum: 15000
-    Column 2: count: 5000 hasNull: false min: STRIPE-3 max: STRIPE-3 sum: 40000
-  Stripe 4:
-    Column 0: count: 5000 hasNull: false
-    Column 1: count: 5000 hasNull: false sum: 15000
-    Column 2: count: 0 hasNull: true
-
-File Statistics:
-  Column 0: count: 20000 hasNull: false
-  Column 1: count: 20000 hasNull: false sum: 60000
-  Column 2: count: 7000 hasNull: true min: RG1 max: STRIPE-3 sum: 46000
-
-Stripes:
-  Stripe: offset: 3 data: 220 rows: 5000 tail: 65 index: 154
-    Stream: column 0 section ROW_INDEX start: 3 length 17
-    Stream: column 1 section ROW_INDEX start: 20 length 60
-    Stream: column 2 section ROW_INDEX start: 80 length 77
-    Stream: column 1 section DATA start: 157 length 159
-    Stream: column 1 section LENGTH start: 316 length 15
-    Stream: column 2 section PRESENT start: 331 length 13
-    Stream: column 2 section DATA start: 344 length 18
-    Stream: column 2 section LENGTH start: 362 length 6
-    Stream: column 2 section DICTIONARY_DATA start: 368 length 9
-    Encoding column 0: DIRECT
-    Encoding column 1: DIRECT_V2
-    Encoding column 2: DICTIONARY_V2[2]
-    Row group indices for column 2:
-      Entry 0: count: 1000 hasNull: false min: RG1 max: RG1 sum: 3000 positions: 0,0,0,0,0,0,0
-      Entry 1: count: 0 hasNull: true positions: 0,0,125,0,0,4,488
-      Entry 2: count: 1000 hasNull: false min: RG3 max: RG3 sum: 3000 positions: 0,2,125,0,0,4,488
-      Entry 3: count: 0 hasNull: true positions: 0,4,125,0,0,12,488
-      Entry 4: count: 0 hasNull: true positions: 0,6,125,0,0,12,488
-  Stripe: offset: 442 data: 185 rows: 5000 tail: 64 index: 116
-    Stream: column 0 section ROW_INDEX start: 442 length 17
-    Stream: column 1 section ROW_INDEX start: 459 length 60
-    Stream: column 2 section ROW_INDEX start: 519 length 39
-    Stream: column 1 section DATA start: 558 length 159
-    Stream: column 1 section LENGTH start: 717 length 15
-    Stream: column 2 section PRESENT start: 732 length 11
-    Stream: column 2 section DATA start: 743 length 0
-    Stream: column 2 section LENGTH start: 743 length 0
-    Stream: column 2 section DICTIONARY_DATA start: 743 length 0
-    Encoding column 0: DIRECT
-    Encoding column 1: DIRECT_V2
-    Encoding column 2: DICTIONARY_V2[0]
-    Row group indices for column 2:
-      Entry 0: count: 0 hasNull: true positions: 0,0,0,0,0,0,0
-      Entry 1: count: 0 hasNull: true positions: 0,0,125,0,0,0,0
-      Entry 2: count: 0 hasNull: true positions: 0,2,120,0,0,0,0
-      Entry 3: count: 0 hasNull: true positions: 0,4,115,0,0,0,0
-      Entry 4: count: 0 hasNull: true positions: 0,6,110,0,0,0,0
-  Stripe: offset: 807 data: 206 rows: 5000 tail: 60 index: 137
-    Stream: column 0 section ROW_INDEX start: 807 length 17
-    Stream: column 1 section ROW_INDEX start: 824 length 60
-    Stream: column 2 section ROW_INDEX start: 884 length 60
-    Stream: column 1 section DATA start: 944 length 159
-    Stream: column 1 section LENGTH start: 1103 length 15
-    Stream: column 2 section DATA start: 1118 length 15
-    Stream: column 2 section LENGTH start: 1133 length 6
-    Stream: column 2 section DICTIONARY_DATA start: 1139 length 11
-    Encoding column 0: DIRECT
-    Encoding column 1: DIRECT_V2
-    Encoding column 2: DICTIONARY_V2[1]
-    Row group indices for column 2:
-      Entry 0: count: 1000 hasNull: false min: STRIPE-3 max: STRIPE-3 sum: 8000 positions: 0,0,0
-      Entry 1: count: 1000 hasNull: false min: STRIPE-3 max: STRIPE-3 sum: 8000 positions: 0,4,488
-      Entry 2: count: 1000 hasNull: false min: STRIPE-3 max: STRIPE-3 sum: 8000 positions: 0,12,464
-      Entry 3: count: 1000 hasNull: false min: STRIPE-3 max: STRIPE-3 sum: 8000 positions: 0,20,440
-      Entry 4: count: 1000 hasNull: false min: STRIPE-3 max: STRIPE-3 sum: 8000 positions: 0,28,416
-  Stripe: offset: 1210 data: 185 rows: 5000 tail: 64 index: 116
-    Stream: column 0 section ROW_INDEX start: 1210 length 17
-    Stream: column 1 section ROW_INDEX start: 1227 length 60
-    Stream: column 2 section ROW_INDEX start: 1287 length 39
-    Stream: column 1 section DATA start: 1326 length 159
-    Stream: column 1 section LENGTH start: 1485 length 15
-    Stream: column 2 section PRESENT start: 1500 length 11
-    Stream: column 2 section DATA start: 1511 length 0
-    Stream: column 2 section LENGTH start: 1511 length 0
-    Stream: column 2 section DICTIONARY_DATA start: 1511 length 0
-    Encoding column 0: DIRECT
-    Encoding column 1: DIRECT_V2
-    Encoding column 2: DICTIONARY_V2[0]
-    Row group indices for column 2:
-      Entry 0: count: 0 hasNull: true positions: 0,0,0,0,0,0,0
-      Entry 1: count: 0 hasNull: true positions: 0,0,125,0,0,0,0
-      Entry 2: count: 0 hasNull: true positions: 0,2,120,0,0,0,0
-      Entry 3: count: 0 hasNull: true positions: 0,4,115,0,0,0,0
-      Entry 4: count: 0 hasNull: true positions: 0,6,110,0,0,0,0
-
-File length: 1823 bytes
-Padding length: 0 bytes
-Padding ratio: 0%
-________________________________________________________________________________________________________________________
-

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/ql/src/test/results/clientpositive/orc_create.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/orc_create.q.out b/ql/src/test/results/clientpositive/orc_create.q.out
index 20c3fce..34ab00d 100644
--- a/ql/src/test/results/clientpositive/orc_create.q.out
+++ b/ql/src/test/results/clientpositive/orc_create.q.out
@@ -380,9 +380,9 @@ POSTHOOK: query: SELECT * from orc_create_complex
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@orc_create_complex
 #### A masked pattern was here ####
-line1	{"key11":"value11","key12":"value12","key13":"value13"}	["a","b","c"]	{"A":"one","B":"two"}
-line2	{"key21":"value21","key22":"value22","key23":"value23"}	["d","e","f"]	{"A":"three","B":"four"}
-line3	{"key31":"value31","key32":"value32","key33":"value33"}	["g","h","i"]	{"A":"five","B":"six"}
+line1	{"key13":"value13","key12":"value12","key11":"value11"}	["a","b","c"]	{"A":"one","B":"two"}
+line2	{"key21":"value21","key23":"value23","key22":"value22"}	["d","e","f"]	{"A":"three","B":"four"}
+line3	{"key33":"value33","key31":"value31","key32":"value32"}	["g","h","i"]	{"A":"five","B":"six"}
 PREHOOK: query: SELECT str from orc_create_complex
 PREHOOK: type: QUERY
 PREHOOK: Input: default@orc_create_complex
@@ -402,9 +402,9 @@ POSTHOOK: query: SELECT mp from orc_create_complex
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@orc_create_complex
 #### A masked pattern was here ####
-{"key11":"value11","key12":"value12","key13":"value13"}
-{"key21":"value21","key22":"value22","key23":"value23"}
-{"key31":"value31","key32":"value32","key33":"value33"}
+{"key13":"value13","key12":"value12","key11":"value11"}
+{"key21":"value21","key23":"value23","key22":"value22"}
+{"key33":"value33","key31":"value31","key32":"value32"}
 PREHOOK: query: SELECT lst from orc_create_complex
 PREHOOK: type: QUERY
 PREHOOK: Input: default@orc_create_complex

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/ql/src/test/results/clientpositive/orc_int_type_promotion.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/orc_int_type_promotion.q.out b/ql/src/test/results/clientpositive/orc_int_type_promotion.q.out
index 4b7b0b0..3b2e962 100644
--- a/ql/src/test/results/clientpositive/orc_int_type_promotion.q.out
+++ b/ql/src/test/results/clientpositive/orc_int_type_promotion.q.out
@@ -126,8 +126,8 @@ POSTHOOK: query: select * from alltypes_orc
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@alltypes_orc
 #### A masked pattern was here ####
-true	10	100	1000	10000	4.0	20.0	4.222	1969-12-31 15:59:58.174	1970-01-01	string	hello	hello	{"k1":"v1","k2":"v2"}	[100,200]	{"c1":null,"c2":" \"foo\"}"}
-false	20	200	2000	20000	8.0	40.0	2.222	1970-12-31 15:59:58.174	1971-01-01	abcd	world	world	{"k3":"v3","k4":"v4"}	[200,300]	{"c1":null,"c2":" \"bar\"}"}
+true	10	100	1000	10000	4.0	20.0	4.222	1969-12-31 15:59:58.174	1970-01-01	string	hello	hello	{"k2":"v2","k1":"v1"}	[100,200]	{"c1":null,"c2":" \"foo\"}"}
+false	20	200	2000	20000	8.0	40.0	2.222	1970-12-31 15:59:58.174	1971-01-01	abcd	world	world	{"k4":"v4","k3":"v3"}	[200,300]	{"c1":null,"c2":" \"bar\"}"}
 PREHOOK: query: alter table alltypes_orc change si si int
 PREHOOK: type: ALTERTABLE_RENAMECOL
 PREHOOK: Input: default@alltypes_orc
@@ -144,8 +144,8 @@ POSTHOOK: query: select * from alltypes_orc
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@alltypes_orc
 #### A masked pattern was here ####
-true	10	100	1000	10000	4.0	20.0	4.222	1969-12-31 15:59:58.174	1970-01-01	string	hello	hello	{"k1":"v1","k2":"v2"}	[100,200]	{"c1":null,"c2":" \"foo\"}"}
-false	20	200	2000	20000	8.0	40.0	2.222	1970-12-31 15:59:58.174	1971-01-01	abcd	world	world	{"k3":"v3","k4":"v4"}	[200,300]	{"c1":null,"c2":" \"bar\"}"}
+true	10	100	1000	10000	4.0	20.0	4.222	1969-12-31 15:59:58.174	1970-01-01	string	hello	hello	{"k2":"v2","k1":"v1"}	[100,200]	{"c1":null,"c2":" \"foo\"}"}
+false	20	200	2000	20000	8.0	40.0	2.222	1970-12-31 15:59:58.174	1971-01-01	abcd	world	world	{"k4":"v4","k3":"v3"}	[200,300]	{"c1":null,"c2":" \"bar\"}"}
 PREHOOK: query: alter table alltypes_orc change si si bigint
 PREHOOK: type: ALTERTABLE_RENAMECOL
 PREHOOK: Input: default@alltypes_orc
@@ -170,8 +170,8 @@ POSTHOOK: query: select * from alltypes_orc
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@alltypes_orc
 #### A masked pattern was here ####
-true	10	100	1000	10000	4.0	20.0	4.222	1969-12-31 15:59:58.174	1970-01-01	string	hello	hello	{"k1":"v1","k2":"v2"}	[100,200]	{"c1":null,"c2":" \"foo\"}"}
-false	20	200	2000	20000	8.0	40.0	2.222	1970-12-31 15:59:58.174	1971-01-01	abcd	world	world	{"k3":"v3","k4":"v4"}	[200,300]	{"c1":null,"c2":" \"bar\"}"}
+true	10	100	1000	10000	4.0	20.0	4.222	1969-12-31 15:59:58.174	1970-01-01	string	hello	hello	{"k2":"v2","k1":"v1"}	[100,200]	{"c1":null,"c2":" \"foo\"}"}
+false	20	200	2000	20000	8.0	40.0	2.222	1970-12-31 15:59:58.174	1971-01-01	abcd	world	world	{"k4":"v4","k3":"v3"}	[200,300]	{"c1":null,"c2":" \"bar\"}"}
 PREHOOK: query: explain select ti, si, i, bi from alltypes_orc
 PREHOOK: type: QUERY
 POSTHOOK: query: explain select ti, si, i, bi from alltypes_orc

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/ql/src/test/results/clientpositive/schema_evol_orc_vec_mapwork_part_all_primitive.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/schema_evol_orc_vec_mapwork_part_all_primitive.q.out b/ql/src/test/results/clientpositive/schema_evol_orc_vec_mapwork_part_all_primitive.q.out
index bd309e6..e29b357 100644
--- a/ql/src/test/results/clientpositive/schema_evol_orc_vec_mapwork_part_all_primitive.q.out
+++ b/ql/src/test/results/clientpositive/schema_evol_orc_vec_mapwork_part_all_primitive.q.out
@@ -1719,10 +1719,10 @@ POSTHOOK: Input: default@part_change_various_various_string@part=1
 POSTHOOK: Input: default@part_change_various_various_string@part=2
 #### A masked pattern was here ####
 insert_num	part	c1	c2	c3	c4	c5	c6	c7	c8	c9	c10	c11	c12	c13	b
-1	1	TRUE	NULL	NULL	3244222	-99999999999	-29.0764	4.70614135E8	470614135	dynamic reptile	dynamic reptile  	0004-09-22 18:26:29.519542222	2007-02-09	binary	original
-2	1	TRUE	100	NULL	14	-23866739993	-3651.672	46114.284799488	46114.284799488	  baffling	  baffling    	2007-02-09 05:17:29.368756876	0004-09-22	binary	original
-3	1	FALSE	72	NULL	-93222	30	-66475.56	-66475.561431	0.561431	1	1	6229-06-28 02:54:28.970117179	5966-07-09	binary	original
-4	1	TRUE	-90	NULL	3289094	46114	9250341.0	9250340.75	9250340.75	junkyard	junkyard	2002-05-10 05:29:48.990818073	1815-05-06	binary	original
+1	1	TRUE	NULL	NULL	3244222	-99999999999	-29.0764	4.70614135E8	470614135	dynamic reptile	dynamic reptile  	0004-09-22 18:26:29.519542222	2007-02-09	62 69 6e 61 72 79	original
+2	1	TRUE	100	NULL	14	-23866739993	-3651.672	46114.284799488	46114.284799488	  baffling	  baffling    	2007-02-09 05:17:29.368756876	0004-09-22	62 69 6e 61 72 79	original
+3	1	FALSE	72	NULL	-93222	30	-66475.56	-66475.561431	0.561431	1	1	6229-06-28 02:54:28.970117179	5966-07-09	62 69 6e 61 72 79	original
+4	1	TRUE	-90	NULL	3289094	46114	9250341.0	9250340.75	9250340.75	junkyard	junkyard	2002-05-10 05:29:48.990818073	1815-05-06	62 69 6e 61 72 79	original
 5	2	true	400	44388	-100	953967041.	62.079153	718.78	1	verdict	verdict	timestamp	date	binary	new
 6	1	-false	-67	833	63993	 1255178165.77663	905070.974	-4314.7918	-1240033819	trial	trial	2016-03-07 03:02:22.0	2016-03-07	binary	new
 PREHOOK: query: drop table part_change_various_various_string
@@ -1916,10 +1916,10 @@ POSTHOOK: Input: default@part_change_various_various_char@part=1
 POSTHOOK: Input: default@part_change_various_various_char@part=2
 #### A masked pattern was here ####
 insert_num	part	c1	c2	c3	c4	c5	c6	c7	c8	c9	c10	c11	c12	c13	b
-1	1	TRUE                     	NULL	NULL	3244222                  	-99999999999             	-29.0764                 	4.70614135E8             	470614135                	dynamic reptile          	dynamic reptile          	0004-09-22 18:26:29.51954	2007-02-09               	binary                   	original
-2	1	TRUE                     	100                      	NULL	14                       	-23866739993             	-3651.672                	46114.284799488          	46114.284799488          	  baffling               	  baffling               	2007-02-09 05:17:29.36875	0004-09-22               	binary                   	original
-3	1	FALSE                    	72                       	NULL	-93222                   	30                       	-66475.56                	-66475.561431            	0.561431                 	1                        	1                        	6229-06-28 02:54:28.97011	5966-07-09               	binary                   	original
-4	1	TRUE                     	-90                      	NULL	3289094                  	46114                    	9250341.0                	9250340.75               	9250340.75               	junkyard                 	junkyard                 	2002-05-10 05:29:48.99081	1815-05-06               	binary                   	original
+1	1	TRUE                     	NULL	NULL	3244222                  	-99999999999             	-29.0764                 	4.70614135E8             	470614135                	dynamic reptile          	dynamic reptile          	0004-09-22 18:26:29.51954	2007-02-09               	62 69 6e 61 72 79        	original
+2	1	TRUE                     	100                      	NULL	14                       	-23866739993             	-3651.672                	46114.284799488          	46114.284799488          	  baffling               	  baffling               	2007-02-09 05:17:29.36875	0004-09-22               	62 69 6e 61 72 79        	original
+3	1	FALSE                    	72                       	NULL	-93222                   	30                       	-66475.56                	-66475.561431            	0.561431                 	1                        	1                        	6229-06-28 02:54:28.97011	5966-07-09               	62 69 6e 61 72 79        	original
+4	1	TRUE                     	-90                      	NULL	3289094                  	46114                    	9250341.0                	9250340.75               	9250340.75               	junkyard                 	junkyard                 	2002-05-10 05:29:48.99081	1815-05-06               	62 69 6e 61 72 79        	original
 5	2	true                     	400                      	44388                    	-100                     	953967041.               	62.079153                	718.78                   	1                        	verdict                  	verdict                  	timestamp                	date                     	binary                   	new
 6	1	-false                   	-67                      	833                      	63993                    	 1255178165.77663        	905070.974               	-4314.7918               	-1240033819              	trial                    	trial                    	2016-03-07 03:02:22.0    	2016-03-07               	binary                   	new
 PREHOOK: query: drop table part_change_various_various_char
@@ -2113,10 +2113,10 @@ POSTHOOK: Input: default@part_change_various_various_char_trunc@part=1
 POSTHOOK: Input: default@part_change_various_various_char_trunc@part=2
 #### A masked pattern was here ####
 insert_num	part	c1	c2	c3	c4	c5	c6	c7	c8	c9	c10	c11	c12	c13	b
-1	1	TRUE    	NULL	NULL	3244222 	-9999999	-29.0764	4.706141	47061413	dynamic 	dynamic 	0004-09-	2007-02-	binary  	original
-2	1	TRUE    	100     	NULL	14      	-2386673	-3651.67	46114.28	46114.28	  baffli	  baffli	2007-02-	0004-09-	binary  	original
-3	1	FALSE   	72      	NULL	-93222  	30      	-66475.5	-66475.5	0.561431	1       	1       	6229-06-	5966-07-	binary  	original
-4	1	TRUE    	-90     	NULL	3289094 	46114   	9250341.	9250340.	9250340.	junkyard	junkyard	2002-05-	1815-05-	binary  	original
+1	1	TRUE    	NULL	NULL	3244222 	-9999999	-29.0764	4.706141	47061413	dynamic 	dynamic 	0004-09-	2007-02-	62 69 6e	original
+2	1	TRUE    	100     	NULL	14      	-2386673	-3651.67	46114.28	46114.28	  baffli	  baffli	2007-02-	0004-09-	62 69 6e	original
+3	1	FALSE   	72      	NULL	-93222  	30      	-66475.5	-66475.5	0.561431	1       	1       	6229-06-	5966-07-	62 69 6e	original
+4	1	TRUE    	-90     	NULL	3289094 	46114   	9250341.	9250340.	9250340.	junkyard	junkyard	2002-05-	1815-05-	62 69 6e	original
 5	2	true    	400     	44388   	-100    	95396704	62.07915	718.78  	1       	verdict 	verdict 	timestam	date    	binary  	new
 6	1	-false  	-67     	833     	63993   	 1255178	905070.9	-4314.79	-1240033	trial   	trial   	2016-03-	2016-03-	binary  	new
 PREHOOK: query: drop table part_change_various_various_char_trunc
@@ -2310,10 +2310,10 @@ POSTHOOK: Input: default@part_change_various_various_varchar@part=1
 POSTHOOK: Input: default@part_change_various_various_varchar@part=2
 #### A masked pattern was here ####
 insert_num	part	c1	c2	c3	c4	c5	c6	c7	c8	c9	c10	c11	c12	c13	b
-1	1	TRUE	NULL	NULL	3244222	-99999999999	-29.0764	4.70614135E8	470614135	dynamic reptile  	dynamic reptile	0004-09-22 18:26:29.51954	2007-02-09	binary	original
-2	1	TRUE	100	NULL	14	-23866739993	-3651.672	46114.284799488	46114.284799488	  baffling    	  baffling	2007-02-09 05:17:29.36875	0004-09-22	binary	original
-3	1	FALSE	72	NULL	-93222	30	-66475.56	-66475.561431	0.561431	1	1	6229-06-28 02:54:28.97011	5966-07-09	binary	original
-4	1	TRUE	-90	NULL	3289094	46114	9250341.0	9250340.75	9250340.75	junkyard	junkyard	2002-05-10 05:29:48.99081	1815-05-06	binary	original
+1	1	TRUE	NULL	NULL	3244222	-99999999999	-29.0764	4.70614135E8	470614135	dynamic reptile  	dynamic reptile	0004-09-22 18:26:29.51954	2007-02-09	62 69 6e 61 72 79	original
+2	1	TRUE	100	NULL	14	-23866739993	-3651.672	46114.284799488	46114.284799488	  baffling    	  baffling	2007-02-09 05:17:29.36875	0004-09-22	62 69 6e 61 72 79	original
+3	1	FALSE	72	NULL	-93222	30	-66475.56	-66475.561431	0.561431	1	1	6229-06-28 02:54:28.97011	5966-07-09	62 69 6e 61 72 79	original
+4	1	TRUE	-90	NULL	3289094	46114	9250341.0	9250340.75	9250340.75	junkyard	junkyard	2002-05-10 05:29:48.99081	1815-05-06	62 69 6e 61 72 79	original
 5	2	true	400	44388	-100	953967041.	62.079153	718.78	1	verdict	verdict	timestamp	date	binary	new
 6	1	-false	-67	833	63993	 1255178165.77663	905070.974	-4314.7918	-1240033819	trial	trial	2016-03-07 03:02:22.0	2016-03-07	binary	new
 PREHOOK: query: drop table part_change_various_various_varchar
@@ -2507,10 +2507,10 @@ POSTHOOK: Input: default@part_change_various_various_varchar_trunc@part=1
 POSTHOOK: Input: default@part_change_various_various_varchar_trunc@part=2
 #### A masked pattern was here ####
 insert_num	part	c1	c2	c3	c4	c5	c6	c7	c8	c9	c10	c11	c12	c13	b
-1	1	TRUE	NULL	NULL	3244222	-9999999	-29.0764	4.706141	47061413	dynamic 	dynamic 	0004-09-	2007-02-	binary	original
-2	1	TRUE	100	NULL	14	-2386673	-3651.67	46114.28	46114.28	  baffli	  baffli	2007-02-	0004-09-	binary	original
-3	1	FALSE	72	NULL	-93222	30	-66475.5	-66475.5	0.561431	1	1	6229-06-	5966-07-	binary	original
-4	1	TRUE	-90	NULL	3289094	46114	9250341.	9250340.	9250340.	junkyard	junkyard	2002-05-	1815-05-	binary	original
+1	1	TRUE	NULL	NULL	3244222	-9999999	-29.0764	4.706141	47061413	dynamic 	dynamic 	0004-09-	2007-02-	62 69 6e	original
+2	1	TRUE	100	NULL	14	-2386673	-3651.67	46114.28	46114.28	  baffli	  baffli	2007-02-	0004-09-	62 69 6e	original
+3	1	FALSE	72	NULL	-93222	30	-66475.5	-66475.5	0.561431	1	1	6229-06-	5966-07-	62 69 6e	original
+4	1	TRUE	-90	NULL	3289094	46114	9250341.	9250340.	9250340.	junkyard	junkyard	2002-05-	1815-05-	62 69 6e	original
 5	2	true	400	44388	-100	95396704	62.07915	718.78	1	verdict	verdict	timestam	date	binary	new
 6	1	-false	-67	833	63993	 1255178	905070.9	-4314.79	-1240033	trial	trial	2016-03-	2016-03-	binary	new
 PREHOOK: query: drop table part_change_various_various_varchar_trunc

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/ql/src/test/results/clientpositive/tez/schema_evol_orc_vec_mapwork_part_all_primitive.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/schema_evol_orc_vec_mapwork_part_all_primitive.q.out b/ql/src/test/results/clientpositive/tez/schema_evol_orc_vec_mapwork_part_all_primitive.q.out
index f9f4d0b..3721f5b 100644
--- a/ql/src/test/results/clientpositive/tez/schema_evol_orc_vec_mapwork_part_all_primitive.q.out
+++ b/ql/src/test/results/clientpositive/tez/schema_evol_orc_vec_mapwork_part_all_primitive.q.out
@@ -1539,10 +1539,10 @@ POSTHOOK: Input: default@part_change_various_various_string@part=1
 POSTHOOK: Input: default@part_change_various_various_string@part=2
 #### A masked pattern was here ####
 insert_num	part	c1	c2	c3	c4	c5	c6	c7	c8	c9	c10	c11	c12	c13	b
-1	1	TRUE	NULL	NULL	3244222	-99999999999	-29.0764	4.70614135E8	470614135	dynamic reptile	dynamic reptile  	0004-09-22 18:26:29.519542222	2007-02-09	binary	original
-2	1	TRUE	100	NULL	14	-23866739993	-3651.672	46114.284799488	46114.284799488	  baffling	  baffling    	2007-02-09 05:17:29.368756876	0004-09-22	binary	original
-3	1	FALSE	72	NULL	-93222	30	-66475.56	-66475.561431	0.561431	1	1	6229-06-28 02:54:28.970117179	5966-07-09	binary	original
-4	1	TRUE	-90	NULL	3289094	46114	9250341.0	9250340.75	9250340.75	junkyard	junkyard	2002-05-10 05:29:48.990818073	1815-05-06	binary	original
+1	1	TRUE	NULL	NULL	3244222	-99999999999	-29.0764	4.70614135E8	470614135	dynamic reptile	dynamic reptile  	0004-09-22 18:26:29.519542222	2007-02-09	62 69 6e 61 72 79	original
+2	1	TRUE	100	NULL	14	-23866739993	-3651.672	46114.284799488	46114.284799488	  baffling	  baffling    	2007-02-09 05:17:29.368756876	0004-09-22	62 69 6e 61 72 79	original
+3	1	FALSE	72	NULL	-93222	30	-66475.56	-66475.561431	0.561431	1	1	6229-06-28 02:54:28.970117179	5966-07-09	62 69 6e 61 72 79	original
+4	1	TRUE	-90	NULL	3289094	46114	9250341.0	9250340.75	9250340.75	junkyard	junkyard	2002-05-10 05:29:48.990818073	1815-05-06	62 69 6e 61 72 79	original
 5	2	true	400	44388	-100	953967041.	62.079153	718.78	1	verdict	verdict	timestamp	date	binary	new
 6	1	-false	-67	833	63993	 1255178165.77663	905070.974	-4314.7918	-1240033819	trial	trial	2016-03-07 03:02:22.0	2016-03-07	binary	new
 PREHOOK: query: drop table part_change_various_various_string
@@ -1716,10 +1716,10 @@ POSTHOOK: Input: default@part_change_various_various_char@part=1
 POSTHOOK: Input: default@part_change_various_various_char@part=2
 #### A masked pattern was here ####
 insert_num	part	c1	c2	c3	c4	c5	c6	c7	c8	c9	c10	c11	c12	c13	b
-1	1	TRUE                     	NULL	NULL	3244222                  	-99999999999             	-29.0764                 	4.70614135E8             	470614135                	dynamic reptile          	dynamic reptile          	0004-09-22 18:26:29.51954	2007-02-09               	binary                   	original
-2	1	TRUE                     	100                      	NULL	14                       	-23866739993             	-3651.672                	46114.284799488          	46114.284799488          	  baffling               	  baffling               	2007-02-09 05:17:29.36875	0004-09-22               	binary                   	original
-3	1	FALSE                    	72                       	NULL	-93222                   	30                       	-66475.56                	-66475.561431            	0.561431                 	1                        	1                        	6229-06-28 02:54:28.97011	5966-07-09               	binary                   	original
-4	1	TRUE                     	-90                      	NULL	3289094                  	46114                    	9250341.0                	9250340.75               	9250340.75               	junkyard                 	junkyard                 	2002-05-10 05:29:48.99081	1815-05-06               	binary                   	original
+1	1	TRUE                     	NULL	NULL	3244222                  	-99999999999             	-29.0764                 	4.70614135E8             	470614135                	dynamic reptile          	dynamic reptile          	0004-09-22 18:26:29.51954	2007-02-09               	62 69 6e 61 72 79        	original
+2	1	TRUE                     	100                      	NULL	14                       	-23866739993             	-3651.672                	46114.284799488          	46114.284799488          	  baffling               	  baffling               	2007-02-09 05:17:29.36875	0004-09-22               	62 69 6e 61 72 79        	original
+3	1	FALSE                    	72                       	NULL	-93222                   	30                       	-66475.56                	-66475.561431            	0.561431                 	1                        	1                        	6229-06-28 02:54:28.97011	5966-07-09               	62 69 6e 61 72 79        	original
+4	1	TRUE                     	-90                      	NULL	3289094                  	46114                    	9250341.0                	9250340.75               	9250340.75               	junkyard                 	junkyard                 	2002-05-10 05:29:48.99081	1815-05-06               	62 69 6e 61 72 79        	original
 5	2	true                     	400                      	44388                    	-100                     	953967041.               	62.079153                	718.78                   	1                        	verdict                  	verdict                  	timestamp                	date                     	binary                   	new
 6	1	-false                   	-67                      	833                      	63993                    	 1255178165.77663        	905070.974               	-4314.7918               	-1240033819              	trial                    	trial                    	2016-03-07 03:02:22.0    	2016-03-07               	binary                   	new
 PREHOOK: query: drop table part_change_various_various_char
@@ -1893,10 +1893,10 @@ POSTHOOK: Input: default@part_change_various_various_char_trunc@part=1
 POSTHOOK: Input: default@part_change_various_various_char_trunc@part=2
 #### A masked pattern was here ####
 insert_num	part	c1	c2	c3	c4	c5	c6	c7	c8	c9	c10	c11	c12	c13	b
-1	1	TRUE    	NULL	NULL	3244222 	-9999999	-29.0764	4.706141	47061413	dynamic 	dynamic 	0004-09-	2007-02-	binary  	original
-2	1	TRUE    	100     	NULL	14      	-2386673	-3651.67	46114.28	46114.28	  baffli	  baffli	2007-02-	0004-09-	binary  	original
-3	1	FALSE   	72      	NULL	-93222  	30      	-66475.5	-66475.5	0.561431	1       	1       	6229-06-	5966-07-	binary  	original
-4	1	TRUE    	-90     	NULL	3289094 	46114   	9250341.	9250340.	9250340.	junkyard	junkyard	2002-05-	1815-05-	binary  	original
+1	1	TRUE    	NULL	NULL	3244222 	-9999999	-29.0764	4.706141	47061413	dynamic 	dynamic 	0004-09-	2007-02-	62 69 6e	original
+2	1	TRUE    	100     	NULL	14      	-2386673	-3651.67	46114.28	46114.28	  baffli	  baffli	2007-02-	0004-09-	62 69 6e	original
+3	1	FALSE   	72      	NULL	-93222  	30      	-66475.5	-66475.5	0.561431	1       	1       	6229-06-	5966-07-	62 69 6e	original
+4	1	TRUE    	-90     	NULL	3289094 	46114   	9250341.	9250340.	9250340.	junkyard	junkyard	2002-05-	1815-05-	62 69 6e	original
 5	2	true    	400     	44388   	-100    	95396704	62.07915	718.78  	1       	verdict 	verdict 	timestam	date    	binary  	new
 6	1	-false  	-67     	833     	63993   	 1255178	905070.9	-4314.79	-1240033	trial   	trial   	2016-03-	2016-03-	binary  	new
 PREHOOK: query: drop table part_change_various_various_char_trunc
@@ -2070,10 +2070,10 @@ POSTHOOK: Input: default@part_change_various_various_varchar@part=1
 POSTHOOK: Input: default@part_change_various_various_varchar@part=2
 #### A masked pattern was here ####
 insert_num	part	c1	c2	c3	c4	c5	c6	c7	c8	c9	c10	c11	c12	c13	b
-1	1	TRUE	NULL	NULL	3244222	-99999999999	-29.0764	4.70614135E8	470614135	dynamic reptile  	dynamic reptile	0004-09-22 18:26:29.51954	2007-02-09	binary	original
-2	1	TRUE	100	NULL	14	-23866739993	-3651.672	46114.284799488	46114.284799488	  baffling    	  baffling	2007-02-09 05:17:29.36875	0004-09-22	binary	original
-3	1	FALSE	72	NULL	-93222	30	-66475.56	-66475.561431	0.561431	1	1	6229-06-28 02:54:28.97011	5966-07-09	binary	original
-4	1	TRUE	-90	NULL	3289094	46114	9250341.0	9250340.75	9250340.75	junkyard	junkyard	2002-05-10 05:29:48.99081	1815-05-06	binary	original
+1	1	TRUE	NULL	NULL	3244222	-99999999999	-29.0764	4.70614135E8	470614135	dynamic reptile  	dynamic reptile	0004-09-22 18:26:29.51954	2007-02-09	62 69 6e 61 72 79	original
+2	1	TRUE	100	NULL	14	-23866739993	-3651.672	46114.284799488	46114.284799488	  baffling    	  baffling	2007-02-09 05:17:29.36875	0004-09-22	62 69 6e 61 72 79	original
+3	1	FALSE	72	NULL	-93222	30	-66475.56	-66475.561431	0.561431	1	1	6229-06-28 02:54:28.97011	5966-07-09	62 69 6e 61 72 79	original
+4	1	TRUE	-90	NULL	3289094	46114	9250341.0	9250340.75	9250340.75	junkyard	junkyard	2002-05-10 05:29:48.99081	1815-05-06	62 69 6e 61 72 79	original
 5	2	true	400	44388	-100	953967041.	62.079153	718.78	1	verdict	verdict	timestamp	date	binary	new
 6	1	-false	-67	833	63993	 1255178165.77663	905070.974	-4314.7918	-1240033819	trial	trial	2016-03-07 03:02:22.0	2016-03-07	binary	new
 PREHOOK: query: drop table part_change_various_various_varchar
@@ -2247,10 +2247,10 @@ POSTHOOK: Input: default@part_change_various_various_varchar_trunc@part=1
 POSTHOOK: Input: default@part_change_various_various_varchar_trunc@part=2
 #### A masked pattern was here ####
 insert_num	part	c1	c2	c3	c4	c5	c6	c7	c8	c9	c10	c11	c12	c13	b
-1	1	TRUE	NULL	NULL	3244222	-9999999	-29.0764	4.706141	47061413	dynamic 	dynamic 	0004-09-	2007-02-	binary	original
-2	1	TRUE	100	NULL	14	-2386673	-3651.67	46114.28	46114.28	  baffli	  baffli	2007-02-	0004-09-	binary	original
-3	1	FALSE	72	NULL	-93222	30	-66475.5	-66475.5	0.561431	1	1	6229-06-	5966-07-	binary	original
-4	1	TRUE	-90	NULL	3289094	46114	9250341.	9250340.	9250340.	junkyard	junkyard	2002-05-	1815-05-	binary	original
+1	1	TRUE	NULL	NULL	3244222	-9999999	-29.0764	4.706141	47061413	dynamic 	dynamic 	0004-09-	2007-02-	62 69 6e	original
+2	1	TRUE	100	NULL	14	-2386673	-3651.67	46114.28	46114.28	  baffli	  baffli	2007-02-	0004-09-	62 69 6e	original
+3	1	FALSE	72	NULL	-93222	30	-66475.5	-66475.5	0.561431	1	1	6229-06-	5966-07-	62 69 6e	original
+4	1	TRUE	-90	NULL	3289094	46114	9250341.	9250340.	9250340.	junkyard	junkyard	2002-05-	1815-05-	62 69 6e	original
 5	2	true	400	44388	-100	95396704	62.07915	718.78	1	verdict	verdict	timestam	date	binary	new
 6	1	-false	-67	833	63993	 1255178	905070.9	-4314.79	-1240033	trial	trial	2016-03-	2016-03-	binary	new
 PREHOOK: query: drop table part_change_various_various_varchar_trunc

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/ql/src/test/results/clientpositive/vector_complex_all.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vector_complex_all.q.out b/ql/src/test/results/clientpositive/vector_complex_all.q.out
index 1af37c3..2ae7c1b 100644
--- a/ql/src/test/results/clientpositive/vector_complex_all.q.out
+++ b/ql/src/test/results/clientpositive/vector_complex_all.q.out
@@ -108,9 +108,9 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: default@orc_create_complex
 #### A masked pattern was here ####
 orc_create_complex.str	orc_create_complex.mp	orc_create_complex.lst	orc_create_complex.strct
-line1	{"key11":"value11","key12":"value12","key13":"value13"}	["a","b","c"]	{"a":"one","b":"two"}
-line2	{"key21":"value21","key22":"value22","key23":"value23"}	["d","e","f"]	{"a":"three","b":"four"}
-line3	{"key31":"value31","key32":"value32","key33":"value33"}	["g","h","i"]	{"a":"five","b":"six"}
+line1	{"key13":"value13","key12":"value12","key11":"value11"}	["a","b","c"]	{"a":"one","b":"two"}
+line2	{"key21":"value21","key23":"value23","key22":"value22"}	["d","e","f"]	{"a":"three","b":"four"}
+line3	{"key33":"value33","key31":"value31","key32":"value32"}	["g","h","i"]	{"a":"five","b":"six"}
 PREHOOK: query: -- However, since this query is not referencing the complex fields, it should vectorize.
 EXPLAIN
 SELECT COUNT(*) FROM orc_create_complex

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/serde/src/java/org/apache/hadoop/hive/serde2/io/TimestampWritable.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/io/TimestampWritable.java b/serde/src/java/org/apache/hadoop/hive/serde2/io/TimestampWritable.java
index 305fdbe..7d136b4 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/io/TimestampWritable.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/io/TimestampWritable.java
@@ -21,13 +21,13 @@ import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
 import java.io.OutputStream;
-import java.math.BigDecimal;
 import java.sql.Timestamp;
 import java.text.DateFormat;
 import java.text.SimpleDateFormat;
 import java.util.Date;
 
 import org.apache.hadoop.hive.common.type.HiveDecimal;
+import org.apache.hadoop.hive.ql.util.TimestampUtils;
 import org.apache.hadoop.hive.serde2.ByteStream.RandomAccessOutput;
 import org.apache.hadoop.hive.serde2.lazybinary.LazyBinaryUtils;
 import org.apache.hadoop.hive.serde2.lazybinary.LazyBinaryUtils.VInt;
@@ -61,7 +61,6 @@ public class TimestampWritable implements WritableComparable<TimestampWritable>
 
   private static final long SEVEN_BYTE_LONG_SIGN_FLIP = 0xff80L << 48;
 
-  private static final BigDecimal BILLION_BIG_DECIMAL = BigDecimal.valueOf(1000000000);
 
   /** The maximum number of bytes required for a TimestampWritable */
   public static final int MAX_BYTES = 13;
@@ -181,7 +180,7 @@ public class TimestampWritable implements WritableComparable<TimestampWritable>
    */
   public long getSeconds() {
     if (!timestampEmpty) {
-      return millisToSeconds(timestamp.getTime());
+      return TimestampUtils.millisToSeconds(timestamp.getTime());
     } else if (!bytesEmpty) {
       return TimestampWritable.getSeconds(currentBytes, offset);
     } else {
@@ -313,7 +312,7 @@ public class TimestampWritable implements WritableComparable<TimestampWritable>
   public double getDouble() {
     double seconds, nanos;
     if (bytesEmpty) {
-      seconds = millisToSeconds(timestamp.getTime());
+      seconds = TimestampUtils.millisToSeconds(timestamp.getTime());
       nanos = timestamp.getNanos();
     } else {
       seconds = getSeconds();
@@ -326,17 +325,6 @@ public class TimestampWritable implements WritableComparable<TimestampWritable>
     return timestamp.getTime() / 1000;
   }
 
-  /**
-  *
-  * @return double representation of the timestamp, accurate to nanoseconds
-  */
- public static double getDouble(Timestamp timestamp) {
-   double seconds, nanos;
-   seconds = millisToSeconds(timestamp.getTime());
-   nanos = timestamp.getNanos();
-   return seconds + nanos / 1000000000;
- }
-
   public void readFields(DataInput in) throws IOException {
     in.readFully(internalBytes, 0, 4);
     if (TimestampWritable.hasDecimalOrSecondVInt(internalBytes[0])) {
@@ -493,7 +481,7 @@ public class TimestampWritable implements WritableComparable<TimestampWritable>
     long millis = t.getTime();
     int nanos = t.getNanos();
 
-    long seconds = millisToSeconds(millis);
+    long seconds = TimestampUtils.millisToSeconds(millis);
     boolean hasSecondVInt = seconds < 0 || seconds > Integer.MAX_VALUE;
     boolean hasDecimal = setNanosBytes(nanos, b, offset+4, hasSecondVInt);
 
@@ -541,20 +529,6 @@ public class TimestampWritable implements WritableComparable<TimestampWritable>
     return decimal != 0;
   }
 
-  public static Timestamp decimalToTimestamp(HiveDecimal d) {
-    BigDecimal nanoInstant = d.bigDecimalValue().multiply(BILLION_BIG_DECIMAL);
-    int nanos = nanoInstant.remainder(BILLION_BIG_DECIMAL).intValue();
-    if (nanos < 0) {
-      nanos += 1000000000;
-    }
-    long seconds =
-      nanoInstant.subtract(new BigDecimal(nanos)).divide(BILLION_BIG_DECIMAL).longValue();
-    Timestamp t = new Timestamp(seconds * 1000);
-    t.setNanos(nanos);
-
-    return t;
-  }
-
   public HiveDecimal getHiveDecimal() {
     if (timestampEmpty) {
       populateTimestamp();
@@ -565,11 +539,12 @@ public class TimestampWritable implements WritableComparable<TimestampWritable>
   public static HiveDecimal getHiveDecimal(Timestamp timestamp) {
     // The BigDecimal class recommends not converting directly from double to BigDecimal,
     // so we convert through a string...
-    Double timestampDouble = TimestampWritable.getDouble(timestamp);
+    Double timestampDouble = TimestampUtils.getDouble(timestamp);
     HiveDecimal result = HiveDecimal.create(timestampDouble.toString());
     return result;
   }
 
+
   /**
    * Converts the time in seconds or milliseconds to a timestamp.
    * @param time time in seconds or in milliseconds
@@ -580,71 +555,6 @@ public class TimestampWritable implements WritableComparable<TimestampWritable>
       return new Timestamp(intToTimestampInSeconds ?  time * 1000 : time);
   }
 
-  /**
-   * Converts the time in seconds or milliseconds to a timestamp.
-   * @param time time in seconds or in milliseconds
-   * @return the timestamp
-   */
-  public static void setTimestampFromLong(Timestamp timestamp, long time,
-      boolean intToTimestampInSeconds) {
-      // If the time is in seconds, converts it to milliseconds first.
-    timestamp.setTime(intToTimestampInSeconds ?  time * 1000 : time);
-  }
-
-  public static Timestamp doubleToTimestamp(double f) {
-    long seconds = (long) f;
-
-    // We must ensure the exactness of the double's fractional portion.
-    // 0.6 as the fraction part will be converted to 0.59999... and
-    // significantly reduce the savings from binary serialization
-    BigDecimal bd = new BigDecimal(String.valueOf(f));
-    bd = bd.subtract(new BigDecimal(seconds)).multiply(new BigDecimal(1000000000));
-    int nanos = bd.intValue();
-
-    // Convert to millis
-    long millis = seconds * 1000;
-    if (nanos < 0) {
-      millis -= 1000;
-      nanos += 1000000000;
-    }
-    Timestamp t = new Timestamp(millis);
-
-    // Set remaining fractional portion to nanos
-    t.setNanos(nanos);
-    return t;
-  }
-
-  public static void setTimestampFromDouble(Timestamp timestamp, double f) {
-    // Otherwise, BigDecimal throws an exception.  (Support vector operations that sometimes
-    // do work on double Not-a-Number NaN values).
-    if (Double.isNaN(f)) {
-      timestamp.setTime(0);
-      return;
-    }
-    // Algorithm used by TimestampWritable.doubleToTimestamp method.
-    // Allocates a BigDecimal object!
-
-    long seconds = (long) f;
-
-    // We must ensure the exactness of the double's fractional portion.
-    // 0.6 as the fraction part will be converted to 0.59999... and
-    // significantly reduce the savings from binary serialization
-    BigDecimal bd = new BigDecimal(String.valueOf(f));
-    bd = bd.subtract(new BigDecimal(seconds)).multiply(new BigDecimal(1000000000));
-    int nanos = bd.intValue();
-
-    // Convert to millis
-    long millis = seconds * 1000;
-    if (nanos < 0) {
-      millis -= 1000;
-      nanos += 1000000000;
-    }
-    timestamp.setTime(millis);
-
-    // Set remaining fractional portion to nanos
-    timestamp.setNanos(nanos);
-  }
-
   public static void setTimestamp(Timestamp t, byte[] bytes, int offset) {
     boolean hasDecimalOrSecondVInt = hasDecimalOrSecondVInt(bytes[offset]);
     long seconds = (long) TimestampWritable.getSeconds(bytes, offset);
@@ -737,16 +647,4 @@ public class TimestampWritable implements WritableComparable<TimestampWritable>
         | ((0xFFL & bytes[offset+5]) << 16)
         | ((0xFFL & bytes[offset+6]) << 8)) >> 8;
   }
-
-  /**
-   * Rounds the number of milliseconds relative to the epoch down to the nearest whole number of
-   * seconds. 500 would round to 0, -500 would round to -1.
-   */
-  public static long millisToSeconds(long millis) {
-    if (millis >= 0) {
-      return millis / 1000;
-    } else {
-      return (millis - 999) / 1000;
-    }
-  }
 }