You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tajo.apache.org by ji...@apache.org on 2016/03/23 02:41:46 UTC

[1/7] tajo git commit: TAJO-2102: Migrate to Apache Orc from Presto's one.

Repository: tajo
Updated Branches:
  refs/heads/master 9fcc9fd3a -> 682635852


http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/WriterImpl.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/WriterImpl.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/WriterImpl.java
index 833d102..e0ad3d7 100644
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/WriterImpl.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/WriterImpl.java
@@ -19,7 +19,6 @@
 package org.apache.tajo.storage.thirdparty.orc;
 
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Joiner;
 import com.google.common.collect.Lists;
 import com.google.common.primitives.Longs;
 import com.google.protobuf.ByteString;
@@ -30,21 +29,20 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hive.ql.io.IOConstants;
-import org.apache.hadoop.hive.shims.ShimLoader;
-import org.apache.tajo.datum.*;
-import org.apache.tajo.storage.Tuple;
-import org.apache.tajo.storage.thirdparty.orc.CompressionCodec.Modifier;
-import org.apache.tajo.storage.thirdparty.orc.OrcProto.RowIndexEntry;
-import org.apache.tajo.storage.thirdparty.orc.OrcProto.StripeStatistics;
-import org.apache.tajo.storage.thirdparty.orc.OrcProto.Type;
-import org.apache.tajo.storage.thirdparty.orc.OrcProto.UserMetadataItem;
 import org.apache.hadoop.hive.ql.util.JavaDataModel;
-import org.apache.hadoop.hive.serde2.objectinspector.*;
-import org.apache.hadoop.hive.serde2.typeinfo.CharTypeInfo;
-import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo;
-import org.apache.hadoop.hive.serde2.typeinfo.VarcharTypeInfo;
 import org.apache.hadoop.io.Text;
+import org.apache.orc.*;
+import org.apache.orc.CompressionCodec.Modifier;
+import org.apache.orc.OrcProto.RowIndexEntry;
+import org.apache.orc.OrcUtils;
+import org.apache.orc.impl.*;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.Inet4Datum;
+import org.apache.tajo.datum.Int4Datum;
+import org.apache.tajo.datum.Int8Datum;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.storage.thirdparty.orc.OrcFile.*;
+import org.apache.tajo.util.datetime.DateTimeConstants;
 import org.apache.tajo.util.datetime.DateTimeUtil;
 
 import java.io.IOException;
@@ -94,10 +92,11 @@ public class WriterImpl implements Writer, MemoryManager.Callback {
   private final boolean addBlockPadding;
   private final int bufferSize;
   private final long blockSize;
-  private final float paddingTolerance;
+  private final double paddingTolerance;
+  private final TypeDescription schema;
+
   // the streams that make up the current stripe
-  private final Map<StreamName, BufferedStream> streams =
-          new TreeMap<>();
+  private final Map<StreamName, BufferedStream> streams = new TreeMap<>();
 
   private FSDataOutputStream rawWriter = null;
   // the compressed metadata information outStream
@@ -111,47 +110,32 @@ public class WriterImpl implements Writer, MemoryManager.Callback {
   private long rawDataSize = 0;
   private int rowsInIndex = 0;
   private int stripesAtLastFlush = -1;
-  private final List<OrcProto.StripeInformation> stripes =
-          new ArrayList<>();
-  private final Map<String, ByteString> userMetadata =
-          new TreeMap<>();
+  private final List<OrcProto.StripeInformation> stripes = new ArrayList<>();
+  private final Map<String, ByteString> userMetadata = new TreeMap<>();
+  private final StreamFactory streamFactory = new StreamFactory();
   private final TreeWriter treeWriter;
   private final boolean buildIndex;
   private final MemoryManager memoryManager;
-  private final OrcFile.Version version;
+  private final Version version;
   private final Configuration conf;
-  private final OrcFile.WriterCallback callback;
-  private final OrcFile.WriterContext callbackContext;
-  private final OrcFile.EncodingStrategy encodingStrategy;
-  private final OrcFile.CompressionStrategy compressionStrategy;
+  private final WriterCallback callback;
+  private final WriterContext callbackContext;
+  private final EncodingStrategy encodingStrategy;
+  private final CompressionStrategy compressionStrategy;
   private final boolean[] bloomFilterColumns;
   private final double bloomFilterFpp;
   private boolean writeTimeZone;
   private TimeZone timeZone;
 
-  WriterImpl(FileSystem fs,
-      Path path,
-      Configuration conf,
-      ObjectInspector inspector,
-      long stripeSize,
-      CompressionKind compress,
-      int bufferSize,
-      int rowIndexStride,
-      MemoryManager memoryManager,
-      boolean addBlockPadding,
-      OrcFile.Version version,
-      OrcFile.WriterCallback callback,
-      OrcFile.EncodingStrategy encodingStrategy,
-      OrcFile.CompressionStrategy compressionStrategy,
-      float paddingTolerance,
-      long blockSizeValue,
-      String bloomFilterColumnNames,
-      double bloomFilterFpp,
-      TimeZone timeZone) throws IOException {
+  public WriterImpl(FileSystem fs,
+                    Path path,
+                    OrcFile.WriterOptions opts,
+                    TimeZone timeZone) throws IOException {
     this.fs = fs;
     this.path = path;
-    this.conf = conf;
-    this.callback = callback;
+    this.conf = opts.getConfiguration();
+    this.callback = opts.getCallback();
+    this.schema = opts.getSchema();
     if (callback != null) {
       callbackContext = new OrcFile.WriterContext(){
 
@@ -163,100 +147,60 @@ public class WriterImpl implements Writer, MemoryManager.Callback {
     } else {
       callbackContext = null;
     }
-    this.adjustedStripeSize = stripeSize;
-    this.defaultStripeSize = stripeSize;
-    this.version = version;
-    this.encodingStrategy = encodingStrategy;
-    this.compressionStrategy = compressionStrategy;
-    this.addBlockPadding = addBlockPadding;
-    this.blockSize = blockSizeValue;
-    this.paddingTolerance = paddingTolerance;
-    this.compress = compress;
-    this.rowIndexStride = rowIndexStride;
-    this.memoryManager = memoryManager;
-    this.timeZone = timeZone;
+    this.adjustedStripeSize = opts.getStripeSize();
+    this.defaultStripeSize = opts.getStripeSize();
+    this.version = opts.getVersion();
+    this.encodingStrategy = opts.getEncodingStrategy();
+    this.compressionStrategy = opts.getCompressionStrategy();
+    this.addBlockPadding = opts.getBlockPadding();
+    this.blockSize = opts.getBlockSize();
+    this.paddingTolerance = opts.getPaddingTolerance();
+    this.compress = opts.getCompress();
+    this.rowIndexStride = opts.getRowIndexStride();
+    this.memoryManager = opts.getMemoryManager();
     buildIndex = rowIndexStride > 0;
     codec = createCodec(compress);
-    String allColumns = conf.get(IOConstants.COLUMNS);
-    if (allColumns == null) {
-      allColumns = getColumnNamesFromInspector(inspector);
-    }
-    this.bufferSize = getEstimatedBufferSize(allColumns, bufferSize);
+    int numColumns = schema.getMaximumId() + 1;
+    this.bufferSize = getEstimatedBufferSize(defaultStripeSize,
+        numColumns, opts.getBufferSize());
     if (version == OrcFile.Version.V_0_11) {
       /* do not write bloom filters for ORC v11 */
-      this.bloomFilterColumns =
-          OrcUtils.includeColumns(null, allColumns, inspector);
+      this.bloomFilterColumns = new boolean[schema.getMaximumId() + 1];
     } else {
       this.bloomFilterColumns =
-          OrcUtils.includeColumns(bloomFilterColumnNames, allColumns, inspector);
+          OrcUtils.includeColumns(opts.getBloomFilterColumns(), schema);
     }
-    this.bloomFilterFpp = bloomFilterFpp;
-    treeWriter = createTreeWriter(inspector, new StreamFactory(), false);
+    this.bloomFilterFpp = opts.getBloomFilterFpp();
+    this.timeZone = timeZone;
+    treeWriter = createTreeWriter(schema, streamFactory, false);
     if (buildIndex && rowIndexStride < MIN_ROW_INDEX_STRIDE) {
       throw new IllegalArgumentException("Row stride must be at least " +
           MIN_ROW_INDEX_STRIDE);
     }
 
     // ensure that we are able to handle callbacks before we register ourselves
-    memoryManager.addWriter(path, stripeSize, this);
-  }
-
-  private String getColumnNamesFromInspector(ObjectInspector inspector) {
-    List<String> fieldNames = Lists.newArrayList();
-    Joiner joiner = Joiner.on(",");
-    if (inspector instanceof StructObjectInspector) {
-      StructObjectInspector soi = (StructObjectInspector) inspector;
-      List<? extends StructField> fields = soi.getAllStructFieldRefs();
-      for(StructField sf : fields) {
-        fieldNames.add(sf.getFieldName());
-      }
-    }
-    return joiner.join(fieldNames);
+    memoryManager.addWriter(path, opts.getStripeSize(), this);
   }
 
   @VisibleForTesting
-  int getEstimatedBufferSize(int bs) {
-      return getEstimatedBufferSize(conf.get(IOConstants.COLUMNS), bs);
-  }
-
-  int getEstimatedBufferSize(String colNames, int bs) {
-    long availableMem = getMemoryAvailableForORC();
-    if (colNames != null) {
-      final int numCols = colNames.split(",").length;
-      if (numCols > COLUMN_COUNT_THRESHOLD) {
-        // In BufferedStream, there are 3 outstream buffers (compressed,
-        // uncompressed and overflow) and list of previously compressed buffers.
-        // Since overflow buffer is rarely used, lets consider only 2 allocation.
-        // Also, initially, the list of compression buffers will be empty.
-        final int outStreamBuffers = codec == null ? 1 : 2;
-
-        // max possible streams per column is 5. For string columns, there is
-        // ROW_INDEX, PRESENT, DATA, LENGTH, DICTIONARY_DATA streams.
-        final int maxStreams = 5;
-
-        // Lets assume 10% memory for holding dictionary in memory and other
-        // object allocations
-        final long miscAllocation = (long) (0.1f * availableMem);
-
-        // compute the available memory
-        final long remainingMem = availableMem - miscAllocation;
-
-        int estBufferSize = (int) (remainingMem /
-            (maxStreams * outStreamBuffers * numCols));
-        estBufferSize = getClosestBufferSize(estBufferSize, bs);
-        if (estBufferSize > bs) {
-          estBufferSize = bs;
-        }
-
-        LOG.info("WIDE TABLE - Number of columns: " + numCols +
-            " Chosen compression buffer size: " + estBufferSize);
-        return estBufferSize;
-      }
+  public static int getEstimatedBufferSize(long stripeSize, int numColumns,
+                                           int bs) {
+    // The worst case is that there are 2 big streams per a column and
+    // we want to guarantee that each stream gets ~10 buffers.
+    // This keeps buffers small enough that we don't get really small stripe
+    // sizes.
+    int estBufferSize = (int) (stripeSize / (20 * numColumns));
+    estBufferSize = getClosestBufferSize(estBufferSize);
+    if (estBufferSize > bs) {
+      estBufferSize = bs;
+    } else {
+      LOG.info("WIDE TABLE - Number of columns: " + numColumns +
+          " Chosen compression buffer size: " + estBufferSize);
     }
-    return bs;
+    return estBufferSize;
   }
 
-  private int getClosestBufferSize(int estBufferSize, int bs) {
+  private static int getClosestBufferSize(int estBufferSize) {
     final int kb4 = 4 * 1024;
     final int kb8 = 8 * 1024;
     final int kb16 = 16 * 1024;
@@ -616,8 +560,7 @@ public class WriterImpl implements Writer, MemoryManager.Callback {
    */
   private abstract static class TreeWriter {
     protected final int id;
-    protected final ObjectInspector inspector;
-    private final BitFieldWriter isPresent;
+    protected final BitFieldWriter isPresent;
     private final boolean isCompressed;
     protected final ColumnStatisticsImpl indexStatistics;
     protected final ColumnStatisticsImpl stripeColStatistics;
@@ -634,24 +577,24 @@ public class WriterImpl implements Writer, MemoryManager.Callback {
     private final OrcProto.BloomFilter.Builder bloomFilterEntry;
     private boolean foundNulls;
     private OutStream isPresentOutStream;
-    private final List<StripeStatistics.Builder> stripeStatsBuilders;
+    private final List<OrcProto.StripeStatistics.Builder> stripeStatsBuilders;
     private final StreamFactory streamFactory;
 
     /**
      * Create a tree writer.
      * @param columnId the column id of the column to write
-     * @param inspector the object inspector to use
+     * @param schema the row schema
      * @param streamFactory limited access to the Writer's data.
      * @param nullable can the value be null?
      * @throws IOException
      */
-    TreeWriter(int columnId, ObjectInspector inspector,
+    TreeWriter(int columnId,
+               TypeDescription schema,
                StreamFactory streamFactory,
                boolean nullable) throws IOException {
       this.streamFactory = streamFactory;
       this.isCompressed = streamFactory.isCompressed();
       this.id = columnId;
-      this.inspector = inspector;
       if (nullable) {
         isPresentOutStream = streamFactory.createStream(id,
             OrcProto.Stream.Kind.PRESENT);
@@ -661,9 +604,9 @@ public class WriterImpl implements Writer, MemoryManager.Callback {
       }
       this.foundNulls = false;
       createBloomFilter = streamFactory.getBloomFilterColumns()[columnId];
-      indexStatistics = ColumnStatisticsImpl.create(inspector);
-      stripeColStatistics = ColumnStatisticsImpl.create(inspector);
-      fileStatistics = ColumnStatisticsImpl.create(inspector);
+      indexStatistics = ColumnStatisticsImpl.create(schema);
+      stripeColStatistics = ColumnStatisticsImpl.create(schema);
+      fileStatistics = ColumnStatisticsImpl.create(schema);
       childrenWriters = new TreeWriter[0];
       rowIndex = OrcProto.RowIndex.newBuilder();
       rowIndexEntry = OrcProto.RowIndexEntry.newBuilder();
@@ -912,10 +855,10 @@ public class WriterImpl implements Writer, MemoryManager.Callback {
     private final BitFieldWriter writer;
 
     BooleanTreeWriter(int columnId,
-                      ObjectInspector inspector,
+                      TypeDescription schema,
                       StreamFactory writer,
                       boolean nullable) throws IOException {
-      super(columnId, inspector, writer, nullable);
+      super(columnId, schema, writer, nullable);
       PositionedOutputStream out = writer.createStream(id,
           OrcProto.Stream.Kind.DATA);
       this.writer = new BitFieldWriter(out, 1);
@@ -927,7 +870,7 @@ public class WriterImpl implements Writer, MemoryManager.Callback {
       super.write(datum);
       if (datum != null && datum.isNotNull()) {
         boolean val = datum.asBool();
-        indexStatistics.updateBoolean(val);
+        indexStatistics.updateBoolean(val, 1);
         writer.write(val ? 1 : 0);
       }
     }
@@ -951,10 +894,10 @@ public class WriterImpl implements Writer, MemoryManager.Callback {
     private final RunLengthByteWriter writer;
 
     ByteTreeWriter(int columnId,
-                      ObjectInspector inspector,
-                      StreamFactory writer,
-                      boolean nullable) throws IOException {
-      super(columnId, inspector, writer, nullable);
+                   TypeDescription schema,
+                   StreamFactory writer,
+                   boolean nullable) throws IOException {
+      super(columnId, schema, writer, nullable);
       this.writer = new RunLengthByteWriter(writer.createStream(id,
           OrcProto.Stream.Kind.DATA));
       recordPosition(rowIndexPosition);
@@ -965,7 +908,7 @@ public class WriterImpl implements Writer, MemoryManager.Callback {
       super.write(datum);
       if (datum != null && datum.isNotNull()) {
         byte val = datum.asByte();
-        indexStatistics.updateInteger(val);
+        indexStatistics.updateInteger(val, 1);
         if (createBloomFilter) {
           bloomFilter.addLong(val);
         }
@@ -993,10 +936,10 @@ public class WriterImpl implements Writer, MemoryManager.Callback {
     private boolean isDirectV2 = true;
 
     IntegerTreeWriter(int columnId,
-                      ObjectInspector inspector,
+                      TypeDescription schema,
                       StreamFactory writer,
                       boolean nullable) throws IOException {
-      super(columnId, inspector, writer, nullable);
+      super(columnId, schema, writer, nullable);
       OutStream out = writer.createStream(id,
           OrcProto.Stream.Kind.DATA);
       this.isDirectV2 = isNewWriteFormat(writer);
@@ -1026,7 +969,7 @@ public class WriterImpl implements Writer, MemoryManager.Callback {
         } else {
           val = datum.asInt2();
         }
-        indexStatistics.updateInteger(val);
+        indexStatistics.updateInteger(val, 1);
         if (createBloomFilter) {
           // integers are converted to longs in column statistics and during SARG evaluation
           bloomFilter.addLong(val);
@@ -1055,10 +998,10 @@ public class WriterImpl implements Writer, MemoryManager.Callback {
     private final SerializationUtils utils;
 
     FloatTreeWriter(int columnId,
-                      ObjectInspector inspector,
-                      StreamFactory writer,
-                      boolean nullable) throws IOException {
-      super(columnId, inspector, writer, nullable);
+                    TypeDescription schema,
+                    StreamFactory writer,
+                    boolean nullable) throws IOException {
+      super(columnId, schema, writer, nullable);
       this.stream = writer.createStream(id,
           OrcProto.Stream.Kind.DATA);
       this.utils = new SerializationUtils();
@@ -1099,10 +1042,10 @@ public class WriterImpl implements Writer, MemoryManager.Callback {
     private final SerializationUtils utils;
 
     DoubleTreeWriter(int columnId,
-                    ObjectInspector inspector,
-                    StreamFactory writer,
-                    boolean nullable) throws IOException {
-      super(columnId, inspector, writer, nullable);
+                     TypeDescription schema,
+                     StreamFactory writer,
+                     boolean nullable) throws IOException {
+      super(columnId, schema, writer, nullable);
       this.stream = writer.createStream(id,
           OrcProto.Stream.Kind.DATA);
       this.utils = new SerializationUtils();
@@ -1137,33 +1080,33 @@ public class WriterImpl implements Writer, MemoryManager.Callback {
     }
   }
 
-  private static class StringTreeWriter extends TreeWriter {
+  private static abstract class StringBaseTreeWriter extends TreeWriter {
     private static final int INITIAL_DICTIONARY_SIZE = 4096;
     private final OutStream stringOutput;
     private final IntegerWriter lengthOutput;
     private final IntegerWriter rowOutput;
-    private final StringRedBlackTree dictionary =
+    protected final StringRedBlackTree dictionary =
         new StringRedBlackTree(INITIAL_DICTIONARY_SIZE);
-    private final DynamicIntArray rows = new DynamicIntArray();
-    private final PositionedOutputStream directStreamOutput;
-    private final IntegerWriter directLengthOutput;
-    private final List<RowIndexEntry> savedRowIndex =
-            new ArrayList<>();
+    protected final DynamicIntArray rows = new DynamicIntArray();
+    protected final PositionedOutputStream directStreamOutput;
+    protected final IntegerWriter directLengthOutput;
+    private final List<OrcProto.RowIndexEntry> savedRowIndex =
+        new ArrayList<OrcProto.RowIndexEntry>();
     private final boolean buildIndex;
-    private final List<Long> rowIndexValueCount = new ArrayList<>();
+    private final List<Long> rowIndexValueCount = new ArrayList<Long>();
     // If the number of keys in a dictionary is greater than this fraction of
     //the total number of non-null rows, turn off dictionary encoding
-    private final float dictionaryKeySizeThreshold;
-    private boolean useDictionaryEncoding = true;
+    private final double dictionaryKeySizeThreshold;
+    protected boolean useDictionaryEncoding = true;
     private boolean isDirectV2 = true;
     private boolean doneDictionaryCheck;
-    private final boolean strideDictionaryCheck;
+    protected final boolean strideDictionaryCheck;
 
-    StringTreeWriter(int columnId,
-                     ObjectInspector inspector,
-                     StreamFactory writer,
-                     boolean nullable) throws IOException {
-      super(columnId, inspector, writer, nullable);
+    StringBaseTreeWriter(int columnId,
+                         TypeDescription schema,
+                         StreamFactory writer,
+                         boolean nullable) throws IOException {
+      super(columnId, schema, writer, nullable);
       this.isDirectV2 = isNewWriteFormat(writer);
       stringOutput = writer.createStream(id,
           OrcProto.Stream.Kind.DICTIONARY_DATA);
@@ -1177,33 +1120,14 @@ public class WriterImpl implements Writer, MemoryManager.Callback {
       directStreamOutput = writer.createStream(id, OrcProto.Stream.Kind.DATA);
       directLengthOutput = createIntegerWriter(writer.createStream(id,
           OrcProto.Stream.Kind.LENGTH), false, isDirectV2, writer);
-      dictionaryKeySizeThreshold = writer.getConfiguration().getFloat(
-          OrcConf.ConfVars.HIVE_ORC_DICTIONARY_KEY_SIZE_THRESHOLD.varname,
-          OrcConf.ConfVars.HIVE_ORC_DICTIONARY_KEY_SIZE_THRESHOLD.defaultFloatVal);
-      strideDictionaryCheck = writer.getConfiguration().getBoolean(
-          OrcConf.ConfVars.HIVE_ORC_ROW_INDEX_STRIDE_DICTIONARY_CHECK.varname,
-          OrcConf.ConfVars.HIVE_ORC_ROW_INDEX_STRIDE_DICTIONARY_CHECK.defaultBoolVal);
+      Configuration conf = writer.getConfiguration();
+      dictionaryKeySizeThreshold =
+          org.apache.orc.OrcConf.DICTIONARY_KEY_SIZE_THRESHOLD.getDouble(conf);
+      strideDictionaryCheck =
+          org.apache.orc.OrcConf.ROW_INDEX_STRIDE_DICTIONARY_CHECK.getBoolean(conf);
       doneDictionaryCheck = false;
     }
 
-    @Override
-    void write(Datum datum) throws IOException {
-      super.write(datum);
-      if (datum != null && datum.isNotNull()) {
-        if (useDictionaryEncoding || !strideDictionaryCheck) {
-          rows.add(dictionary.add(datum.toString()));
-        } else {
-          // write data and length
-          directStreamOutput.write(datum.asByteArray(), 0, datum.size());
-          directLengthOutput.write(datum.size());
-        }
-        indexStatistics.updateString(datum.toString());
-        if (createBloomFilter) {
-          bloomFilter.addBytes(datum.asByteArray(), datum.size());
-        }
-      }
-    }
-
     private boolean checkDictionaryEncoding() {
       if (!doneDictionaryCheck) {
         // Set the flag indicating whether or not to use dictionary encoding
@@ -1269,7 +1193,7 @@ public class WriterImpl implements Writer, MemoryManager.Callback {
           private int currentId = 0;
           @Override
           public void visit(StringRedBlackTree.VisitorContext context
-                           ) throws IOException {
+          ) throws IOException {
             context.writeBytes(stringOutput);
             lengthOutput.write(context.getLength());
             dumpOrder[context.getOriginalPosition()] = currentId++;
@@ -1383,29 +1307,76 @@ public class WriterImpl implements Writer, MemoryManager.Callback {
     }
   }
 
+  private static class StringTreeWriter extends StringBaseTreeWriter {
+    StringTreeWriter(int columnId,
+                     TypeDescription schema,
+                     StreamFactory writer,
+                     boolean nullable) throws IOException {
+      super(columnId, schema, writer, nullable);
+    }
+
+    @Override
+    void write(Datum datum) throws IOException {
+      super.write(datum);
+      if (datum != null && datum.isNotNull()) {
+        if (useDictionaryEncoding || !strideDictionaryCheck) {
+          rows.add(dictionary.add(datum.toString()));
+        } else {
+          // write data and length
+          directStreamOutput.write(datum.asByteArray(), 0, datum.size());
+          directLengthOutput.write(datum.size());
+        }
+        byte[] buf = datum.asByteArray();
+        indexStatistics.updateString(buf, 0, buf.length, 1);
+        if (createBloomFilter) {
+          bloomFilter.addBytes(buf, 0, buf.length);
+        }
+      }
+    }
+  }
+
   /**
    * Under the covers, char is written to ORC the same way as string.
    */
   private static class CharTreeWriter extends StringTreeWriter {
+    private final int itemLength;
+    private final byte[] padding;
 
     CharTreeWriter(int columnId,
-        ObjectInspector inspector,
-        StreamFactory writer,
-        boolean nullable) throws IOException {
-      super(columnId, inspector, writer, nullable);
+                   TypeDescription schema,
+                   StreamFactory writer,
+                   boolean nullable) throws IOException {
+      super(columnId, schema, writer, nullable);
+      itemLength = schema.getMaxLength();
+      padding = new byte[itemLength];
     }
-  }
 
-  /**
-   * Under the covers, varchar is written to ORC the same way as string.
-   */
-  private static class VarcharTreeWriter extends StringTreeWriter {
+    @Override
+    void write(Datum datum) throws IOException {
+      super.write(datum);
+      if (datum != null && datum.isNotNull()) {
+        byte[] ptr;
+        byte[] buf = datum.asByteArray();
+        if (buf.length >= itemLength) {
+          ptr = buf;
+        } else {
+          ptr = padding;
+          System.arraycopy(buf, 0, ptr, 0, buf.length);
+          Arrays.fill(ptr, buf.length, itemLength, (byte) ' ');
+        }
+        if (useDictionaryEncoding || !strideDictionaryCheck) {
+          rows.add(dictionary.add(ptr, 0, itemLength));
+        } else {
+          // write data and length
+          directStreamOutput.write(ptr, 0, itemLength);
+          directLengthOutput.write(itemLength);
+        }
 
-    VarcharTreeWriter(int columnId,
-        ObjectInspector inspector,
-        StreamFactory writer,
-        boolean nullable) throws IOException {
-      super(columnId, inspector, writer, nullable);
+        indexStatistics.updateString(ptr, 0, ptr.length, 1);
+        if (createBloomFilter) {
+          bloomFilter.addBytes(ptr, 0, ptr.length);
+        }
+      }
     }
   }
 
@@ -1415,10 +1386,10 @@ public class WriterImpl implements Writer, MemoryManager.Callback {
     private boolean isDirectV2 = true;
 
     BinaryTreeWriter(int columnId,
-                     ObjectInspector inspector,
+                     TypeDescription schema,
                      StreamFactory writer,
                      boolean nullable) throws IOException {
-      super(columnId, inspector, writer, nullable);
+      super(columnId, schema, writer, nullable);
       this.stream = writer.createStream(id,
           OrcProto.Stream.Kind.DATA);
       this.isDirectV2 = isNewWriteFormat(writer);
@@ -1441,11 +1412,12 @@ public class WriterImpl implements Writer, MemoryManager.Callback {
     void write(Datum datum) throws IOException {
       super.write(datum);
       if (datum != null && datum.isNotNull()) {
-        stream.write(datum.asByteArray(), 0, datum.size());
+        byte[] buf = datum.asByteArray();
+        stream.write(buf, 0, buf.length);
         length.write(datum.size());
-        indexStatistics.updateBinary(datum);
+        indexStatistics.updateBinary(buf, 0, buf.length, 1);
         if (createBloomFilter) {
-          bloomFilter.addBytes(datum.asByteArray(), datum.size());
+          bloomFilter.addBytes(buf, 0, buf.length);
         }
       }
     }
@@ -1467,7 +1439,6 @@ public class WriterImpl implements Writer, MemoryManager.Callback {
     }
   }
 
-  static final int MILLIS_PER_SECOND = 1000;
   static final String BASE_TIMESTAMP_STRING = "2015-01-01 00:00:00";
 
   private static class TimestampTreeWriter extends TreeWriter {
@@ -1478,10 +1449,10 @@ public class WriterImpl implements Writer, MemoryManager.Callback {
     private TimeZone timeZone;
 
     TimestampTreeWriter(int columnId,
-                     ObjectInspector inspector,
-                     StreamFactory writer,
-                     boolean nullable) throws IOException {
-      super(columnId, inspector, writer, nullable);
+                        TypeDescription schema,
+                        StreamFactory writer,
+                        boolean nullable) throws IOException {
+      super(columnId, schema, writer, nullable);
       this.isDirectV2 = isNewWriteFormat(writer);
       this.seconds = createIntegerWriter(writer.createStream(id,
           OrcProto.Stream.Kind.DATA), true, isDirectV2, writer);
@@ -1489,7 +1460,7 @@ public class WriterImpl implements Writer, MemoryManager.Callback {
           OrcProto.Stream.Kind.SECONDARY), false, isDirectV2, writer);
       recordPosition(rowIndexPosition);
       // for unit tests to set different time zones
-      this.base_timestamp = Timestamp.valueOf(BASE_TIMESTAMP_STRING).getTime() / MILLIS_PER_SECOND;
+      this.base_timestamp = Timestamp.valueOf(BASE_TIMESTAMP_STRING).getTime() / DateTimeConstants.MSECS_PER_SEC;
       writer.useWriterTimeZone(true);
       timeZone = writer.getTimeZone();
     }
@@ -1515,7 +1486,7 @@ public class WriterImpl implements Writer, MemoryManager.Callback {
 
         Timestamp val = new Timestamp(javaTimestamp);
         indexStatistics.updateTimestamp(val);
-        seconds.write((val.getTime() / MILLIS_PER_SECOND) - base_timestamp);
+        seconds.write((val.getTime() / DateTimeConstants.MSECS_PER_SEC) - base_timestamp);
         nanos.write(formatNanos(val.getNanos()));
         if (createBloomFilter) {
           bloomFilter.addLong(val.getTime());
@@ -1561,12 +1532,12 @@ public class WriterImpl implements Writer, MemoryManager.Callback {
     private final boolean isDirectV2;
 
     DateTreeWriter(int columnId,
-                   ObjectInspector inspector,
+                   TypeDescription schema,
                    StreamFactory writer,
                    boolean nullable) throws IOException {
-      super(columnId, inspector, writer, nullable);
+      super(columnId, schema, writer, nullable);
       OutStream out = writer.createStream(id,
-        OrcProto.Stream.Kind.DATA);
+          OrcProto.Stream.Kind.DATA);
       this.isDirectV2 = isNewWriteFormat(writer);
       this.writer = createIntegerWriter(out, true, isDirectV2, writer);
       recordPosition(rowIndexPosition);
@@ -1612,19 +1583,17 @@ public class WriterImpl implements Writer, MemoryManager.Callback {
   }
 
   private static class StructTreeWriter extends TreeWriter {
-    private final List<? extends StructField> fields;
     StructTreeWriter(int columnId,
-                     ObjectInspector inspector,
+                     TypeDescription schema,
                      StreamFactory writer,
                      boolean nullable) throws IOException {
-      super(columnId, inspector, writer, nullable);
-      StructObjectInspector structObjectInspector =
-        (StructObjectInspector) inspector;
-      fields = structObjectInspector.getAllStructFieldRefs();
-      childrenWriters = new TreeWriter[fields.size()];
+      super(columnId, schema, writer, nullable);
+      List<TypeDescription> children = schema.getChildren();
+      childrenWriters = new TreeWriter[children.size()];
       for(int i=0; i < childrenWriters.length; ++i) {
         childrenWriters[i] = createTreeWriter(
-          fields.get(i).getFieldObjectInspector(), writer, true);
+            children.get(i), writer,
+            true);
       }
       recordPosition(rowIndexPosition);
     }
@@ -1636,9 +1605,8 @@ public class WriterImpl implements Writer, MemoryManager.Callback {
     void writeTuple(Tuple tuple) throws IOException {
       super.write(tuple);
       if (tuple != null) {
-        for(int i = 0; i < fields.size(); ++i) {
-          TreeWriter writer = childrenWriters[i];
-          writer.write(tuple.asDatum(i));
+        for(int i = 0; i < childrenWriters.length; ++i) {
+          childrenWriters[i].write(tuple.asDatum(i));
         }
       }
     }
@@ -1654,159 +1622,136 @@ public class WriterImpl implements Writer, MemoryManager.Callback {
     }
   }
 
-  private static TreeWriter createTreeWriter(ObjectInspector inspector,
+  private static TreeWriter createTreeWriter(TypeDescription schema,
                                              StreamFactory streamFactory,
                                              boolean nullable) throws IOException {
-    switch (inspector.getCategory()) {
-      case PRIMITIVE:
-        switch (((PrimitiveObjectInspector) inspector).getPrimitiveCategory()) {
-          case BOOLEAN:
-          case VOID:
-            return new BooleanTreeWriter(streamFactory.getNextColumnId(),
-                inspector, streamFactory, nullable);
-          case BYTE:
-            return new ByteTreeWriter(streamFactory.getNextColumnId(),
-                inspector, streamFactory, nullable);
-          case SHORT:
-          case INT:
-          case LONG:
-            return new IntegerTreeWriter(streamFactory.getNextColumnId(),
-                inspector, streamFactory, nullable);
-          case FLOAT:
-            return new FloatTreeWriter(streamFactory.getNextColumnId(),
-                inspector, streamFactory, nullable);
-          case DOUBLE:
-            return new DoubleTreeWriter(streamFactory.getNextColumnId(),
-                inspector, streamFactory, nullable);
-          case STRING:
-            return new StringTreeWriter(streamFactory.getNextColumnId(),
-                inspector, streamFactory, nullable);
-          case CHAR:
-            return new CharTreeWriter(streamFactory.getNextColumnId(),
-                inspector, streamFactory, nullable);
-          case VARCHAR:
-            return new VarcharTreeWriter(streamFactory.getNextColumnId(),
-                inspector, streamFactory, nullable);
-          case BINARY:
-            return new BinaryTreeWriter(streamFactory.getNextColumnId(),
-                inspector, streamFactory, nullable);
-          case TIMESTAMP:
-            return new TimestampTreeWriter(streamFactory.getNextColumnId(),
-                inspector, streamFactory, nullable);
-          case DATE:
-            return new DateTreeWriter(streamFactory.getNextColumnId(),
-              inspector, streamFactory, nullable);
-          default:
-            throw new IllegalArgumentException("Bad primitive category " +
-              ((PrimitiveObjectInspector) inspector).getPrimitiveCategory());
-        }
+    switch (schema.getCategory()) {
+      case BOOLEAN:
+        return new BooleanTreeWriter(streamFactory.getNextColumnId(),
+            schema, streamFactory, nullable);
+      case BYTE:
+        return new ByteTreeWriter(streamFactory.getNextColumnId(),
+            schema, streamFactory, nullable);
+      case SHORT:
+      case INT:
+      case LONG:
+        return new IntegerTreeWriter(streamFactory.getNextColumnId(),
+            schema, streamFactory, nullable);
+      case FLOAT:
+        return new FloatTreeWriter(streamFactory.getNextColumnId(),
+            schema, streamFactory, nullable);
+      case DOUBLE:
+        return new DoubleTreeWriter(streamFactory.getNextColumnId(),
+            schema, streamFactory, nullable);
+      case STRING:
+        return new StringTreeWriter(streamFactory.getNextColumnId(),
+            schema, streamFactory, nullable);
+      case CHAR:
+        return new CharTreeWriter(streamFactory.getNextColumnId(),
+            schema, streamFactory, nullable);
+      case BINARY:
+        return new BinaryTreeWriter(streamFactory.getNextColumnId(),
+            schema, streamFactory, nullable);
+      case TIMESTAMP:
+        return new TimestampTreeWriter(streamFactory.getNextColumnId(),
+            schema, streamFactory, nullable);
+      case DATE:
+        return new DateTreeWriter(streamFactory.getNextColumnId(),
+            schema, streamFactory, nullable);
       case STRUCT:
-        return new StructTreeWriter(streamFactory.getNextColumnId(), inspector,
-            streamFactory, nullable);
+        return new StructTreeWriter(streamFactory.getNextColumnId(),
+            schema, streamFactory, nullable);
       default:
         throw new IllegalArgumentException("Bad category: " +
-          inspector.getCategory());
+            schema.getCategory());
     }
   }
 
   private static void writeTypes(OrcProto.Footer.Builder builder,
-                                 TreeWriter treeWriter) {
+                                 TypeDescription schema) {
     OrcProto.Type.Builder type = OrcProto.Type.newBuilder();
-    switch (treeWriter.inspector.getCategory()) {
-      case PRIMITIVE:
-        switch (((PrimitiveObjectInspector) treeWriter.inspector).
-                 getPrimitiveCategory()) {
-          case VOID:
-          case BOOLEAN:
-            type.setKind(OrcProto.Type.Kind.BOOLEAN);
-            break;
-          case BYTE:
-            type.setKind(OrcProto.Type.Kind.BYTE);
-            break;
-          case SHORT:
-            type.setKind(OrcProto.Type.Kind.SHORT);
-            break;
-          case INT:
-            type.setKind(OrcProto.Type.Kind.INT);
-            break;
-          case LONG:
-            type.setKind(OrcProto.Type.Kind.LONG);
-            break;
-          case FLOAT:
-            type.setKind(OrcProto.Type.Kind.FLOAT);
-            break;
-          case DOUBLE:
-            type.setKind(OrcProto.Type.Kind.DOUBLE);
-            break;
-          case STRING:
-            type.setKind(OrcProto.Type.Kind.STRING);
-            break;
-          case CHAR:
-            // The char length needs to be written to file and should be available
-            // from the object inspector
-            CharTypeInfo charTypeInfo = (CharTypeInfo) ((PrimitiveObjectInspector) treeWriter.inspector).getTypeInfo();
-            type.setKind(Type.Kind.CHAR);
-            type.setMaximumLength(charTypeInfo.getLength());
-            break;
-          case VARCHAR:
-            // The varchar length needs to be written to file and should be available
-            // from the object inspector
-            VarcharTypeInfo typeInfo = (VarcharTypeInfo) ((PrimitiveObjectInspector) treeWriter.inspector).getTypeInfo();
-            type.setKind(Type.Kind.VARCHAR);
-            type.setMaximumLength(typeInfo.getLength());
-            break;
-          case BINARY:
-            type.setKind(OrcProto.Type.Kind.BINARY);
-            break;
-          case TIMESTAMP:
-            type.setKind(OrcProto.Type.Kind.TIMESTAMP);
-            break;
-          case DATE:
-            type.setKind(OrcProto.Type.Kind.DATE);
-            break;
-          case DECIMAL:
-            DecimalTypeInfo decTypeInfo = (DecimalTypeInfo)((PrimitiveObjectInspector)treeWriter.inspector).getTypeInfo();
-            type.setKind(OrcProto.Type.Kind.DECIMAL);
-            type.setPrecision(decTypeInfo.precision());
-            type.setScale(decTypeInfo.scale());
-            break;
-          default:
-            throw new IllegalArgumentException("Unknown primitive category: " +
-              ((PrimitiveObjectInspector) treeWriter.inspector).
-                getPrimitiveCategory());
-        }
+    List<TypeDescription> children = schema.getChildren();
+    switch (schema.getCategory()) {
+      case BOOLEAN:
+        type.setKind(OrcProto.Type.Kind.BOOLEAN);
+        break;
+      case BYTE:
+        type.setKind(OrcProto.Type.Kind.BYTE);
+        break;
+      case SHORT:
+        type.setKind(OrcProto.Type.Kind.SHORT);
+        break;
+      case INT:
+        type.setKind(OrcProto.Type.Kind.INT);
+        break;
+      case LONG:
+        type.setKind(OrcProto.Type.Kind.LONG);
+        break;
+      case FLOAT:
+        type.setKind(OrcProto.Type.Kind.FLOAT);
+        break;
+      case DOUBLE:
+        type.setKind(OrcProto.Type.Kind.DOUBLE);
+        break;
+      case STRING:
+        type.setKind(OrcProto.Type.Kind.STRING);
+        break;
+      case CHAR:
+        type.setKind(OrcProto.Type.Kind.CHAR);
+        type.setMaximumLength(schema.getMaxLength());
+        break;
+      case VARCHAR:
+        type.setKind(OrcProto.Type.Kind.VARCHAR);
+        type.setMaximumLength(schema.getMaxLength());
+        break;
+      case BINARY:
+        type.setKind(OrcProto.Type.Kind.BINARY);
+        break;
+      case TIMESTAMP:
+        type.setKind(OrcProto.Type.Kind.TIMESTAMP);
+        break;
+      case DATE:
+        type.setKind(OrcProto.Type.Kind.DATE);
+        break;
+      case DECIMAL:
+        type.setKind(OrcProto.Type.Kind.DECIMAL);
+        type.setPrecision(schema.getPrecision());
+        type.setScale(schema.getScale());
         break;
       case LIST:
         type.setKind(OrcProto.Type.Kind.LIST);
-        type.addSubtypes(treeWriter.childrenWriters[0].id);
+        type.addSubtypes(children.get(0).getId());
         break;
       case MAP:
         type.setKind(OrcProto.Type.Kind.MAP);
-        type.addSubtypes(treeWriter.childrenWriters[0].id);
-        type.addSubtypes(treeWriter.childrenWriters[1].id);
+        for(TypeDescription t: children) {
+          type.addSubtypes(t.getId());
+        }
         break;
       case STRUCT:
         type.setKind(OrcProto.Type.Kind.STRUCT);
-        for(TreeWriter child: treeWriter.childrenWriters) {
-          type.addSubtypes(child.id);
+        for(TypeDescription t: children) {
+          type.addSubtypes(t.getId());
         }
-        for(StructField field: ((StructTreeWriter) treeWriter).fields) {
-          type.addFieldNames(field.getFieldName());
+        for(String field: schema.getFieldNames()) {
+          type.addFieldNames(field);
         }
         break;
       case UNION:
         type.setKind(OrcProto.Type.Kind.UNION);
-        for(TreeWriter child: treeWriter.childrenWriters) {
-          type.addSubtypes(child.id);
+        for(TypeDescription t: children) {
+          type.addSubtypes(t.getId());
         }
         break;
       default:
         throw new IllegalArgumentException("Unknown category: " +
-          treeWriter.inspector.getCategory());
+            schema.getCategory());
     }
     builder.addTypes(type);
-    for(TreeWriter child: treeWriter.childrenWriters) {
-      writeTypes(builder, child);
+    if (children != null) {
+      for(TypeDescription child: children) {
+        writeTypes(builder, child);
+      }
     }
   }
 
@@ -1853,9 +1798,9 @@ public class WriterImpl implements Writer, MemoryManager.Callback {
           StreamName name = pair.getKey();
           long streamSize = pair.getValue().getOutputSize();
           builder.addStreams(OrcProto.Stream.newBuilder()
-                             .setColumn(name.getColumn())
-                             .setKind(name.getKind())
-                             .setLength(streamSize));
+              .setColumn(name.getColumn())
+              .setKind(name.getKind())
+              .setLength(streamSize));
           if (StreamName.Area.INDEX == name.getArea()) {
             indexSize += streamSize;
           } else {
@@ -1880,8 +1825,8 @@ public class WriterImpl implements Writer, MemoryManager.Callback {
         // and user specified padding tolerance. Since stripe size can overflow
         // the default stripe size we should apply this correction to avoid
         // writing portion of last stripe to next hdfs block.
-        float correction = overflow > 0 ? (float) overflow
-            / (float) adjustedStripeSize : 0.0f;
+        double correction = overflow > 0 ? (double) overflow
+            / (double) adjustedStripeSize : 0.0;
 
         // correction should not be greater than user specified padding
         // tolerance
@@ -1939,75 +1884,60 @@ public class WriterImpl implements Writer, MemoryManager.Callback {
   }
 
   private long computeRawDataSize() {
-    long result = 0;
-    for (TreeWriter child : treeWriter.getChildrenWriters()) {
-      result += getRawDataSizeFromInspectors(child, child.inspector);
-    }
-    return result;
+    return getRawDataSize(treeWriter, schema);
   }
 
-  private long getRawDataSizeFromInspectors(TreeWriter child, ObjectInspector oi) {
+  private long getRawDataSize(TreeWriter child,
+                              TypeDescription schema) {
     long total = 0;
-    switch (oi.getCategory()) {
-    case PRIMITIVE:
-      total += getRawDataSizeFromPrimitives(child, oi);
-      break;
-    case LIST:
-    case MAP:
-    case UNION:
-    case STRUCT:
-      for (TreeWriter tw : child.childrenWriters) {
-        total += getRawDataSizeFromInspectors(tw, tw.inspector);
+    long numVals = child.fileStatistics.getNumberOfValues();
+    switch (schema.getCategory()) {
+      case BOOLEAN:
+      case BYTE:
+      case SHORT:
+      case INT:
+      case FLOAT:
+        return numVals * JavaDataModel.get().primitive1();
+      case LONG:
+      case DOUBLE:
+        return numVals * JavaDataModel.get().primitive2();
+      case STRING:
+      case VARCHAR:
+      case CHAR:
+        // ORC strings are converted to java Strings. so use JavaDataModel to
+        // compute the overall size of strings
+        StringColumnStatistics scs = (StringColumnStatistics) child.fileStatistics;
+        numVals = numVals == 0 ? 1 : numVals;
+        int avgStringLen = (int) (scs.getSum() / numVals);
+        return numVals * JavaDataModel.get().lengthForStringOfLength(avgStringLen);
+      case DECIMAL:
+        return numVals * JavaDataModel.get().lengthOfDecimal();
+      case DATE:
+        return numVals * JavaDataModel.get().lengthOfDate();
+      case BINARY:
+        // get total length of binary blob
+        BinaryColumnStatistics bcs = (BinaryColumnStatistics) child.fileStatistics;
+        return bcs.getSum();
+      case TIMESTAMP:
+        return numVals * JavaDataModel.get().lengthOfTimestamp();
+      case LIST:
+      case MAP:
+      case UNION:
+      case STRUCT: {
+        TreeWriter[] childWriters = child.getChildrenWriters();
+        List<TypeDescription> childTypes = schema.getChildren();
+        for (int i=0; i < childWriters.length; ++i) {
+          total += getRawDataSize(childWriters[i], childTypes.get(i));
+        }
+        break;
       }
-      break;
-    default:
-      LOG.debug("Unknown object inspector category.");
-      break;
+      default:
+        LOG.debug("Unknown object inspector category.");
+        break;
     }
     return total;
   }
 
-  private long getRawDataSizeFromPrimitives(TreeWriter child, ObjectInspector oi) {
-    long result = 0;
-    long numVals = child.fileStatistics.getNumberOfValues();
-    switch (((PrimitiveObjectInspector) oi).getPrimitiveCategory()) {
-    case BOOLEAN:
-    case BYTE:
-    case SHORT:
-    case INT:
-    case FLOAT:
-      return numVals * JavaDataModel.get().primitive1();
-    case LONG:
-    case DOUBLE:
-      return numVals * JavaDataModel.get().primitive2();
-    case STRING:
-    case VARCHAR:
-    case CHAR:
-      // ORC strings are converted to java Strings. so use JavaDataModel to
-      // compute the overall size of strings
-      child = (StringTreeWriter) child;
-      StringColumnStatistics scs = (StringColumnStatistics) child.fileStatistics;
-      numVals = numVals == 0 ? 1 : numVals;
-      int avgStringLen = (int) (scs.getSum() / numVals);
-      return numVals * JavaDataModel.get().lengthForStringOfLength(avgStringLen);
-    case DECIMAL:
-      return numVals * JavaDataModel.get().lengthOfDecimal();
-    case DATE:
-      return numVals * JavaDataModel.get().lengthOfDate();
-    case BINARY:
-      // get total length of binary blob
-      BinaryColumnStatistics bcs = (BinaryColumnStatistics) child.fileStatistics;
-      return bcs.getSum();
-    case TIMESTAMP:
-      return numVals * JavaDataModel.get().lengthOfTimestamp();
-    default:
-      LOG.debug("Unknown primitive category.");
-      break;
-    }
-
-    return result;
-  }
-
   private OrcProto.CompressionKind writeCompressionKind(CompressionKind kind) {
     switch (kind) {
       case NONE: return OrcProto.CompressionKind.NONE;
@@ -2027,7 +1957,7 @@ public class WriterImpl implements Writer, MemoryManager.Callback {
     }
   }
 
-  private int writeMetadata(long bodyLength) throws IOException {
+  private int writeMetadata() throws IOException {
     getStream();
     OrcProto.Metadata.Builder builder = OrcProto.Metadata.newBuilder();
     for(OrcProto.StripeStatistics.Builder ssb : treeWriter.stripeStatsBuilders) {
@@ -2052,7 +1982,7 @@ public class WriterImpl implements Writer, MemoryManager.Callback {
     // populate raw data size
     rawDataSize = computeRawDataSize();
     // serialize the types
-    writeTypes(builder, treeWriter);
+    writeTypes(builder, schema);
     // add the stripe information
     for(OrcProto.StripeInformation stripe: stripes) {
       builder.addStripes(stripe);
@@ -2062,7 +1992,7 @@ public class WriterImpl implements Writer, MemoryManager.Callback {
     // add all of the user metadata
     for(Map.Entry<String, ByteString> entry: userMetadata.entrySet()) {
       builder.addMetadata(OrcProto.UserMetadataItem.newBuilder()
-        .setName(entry.getKey()).setValue(entry.getValue()));
+          .setName(entry.getKey()).setValue(entry.getValue()));
     }
     long startPosn = rawWriter.getPos();
     OrcProto.Footer footer = builder.build();
@@ -2074,14 +2004,14 @@ public class WriterImpl implements Writer, MemoryManager.Callback {
 
   private int writePostScript(int footerLength, int metadataLength) throws IOException {
     OrcProto.PostScript.Builder builder =
-      OrcProto.PostScript.newBuilder()
-        .setCompression(writeCompressionKind(compress))
-        .setFooterLength(footerLength)
-        .setMetadataLength(metadataLength)
-        .setMagic(OrcFile.MAGIC)
-        .addVersion(version.getMajor())
-        .addVersion(version.getMinor())
-        .setWriterVersion(OrcFile.WriterVersion.HIVE_8732.getId());
+        OrcProto.PostScript.newBuilder()
+            .setCompression(writeCompressionKind(compress))
+            .setFooterLength(footerLength)
+            .setMetadataLength(metadataLength)
+            .setMagic(OrcFile.MAGIC)
+            .addVersion(version.getMajor())
+            .addVersion(version.getMinor())
+            .setWriterVersion(OrcFile.CURRENT_WRITER.getId());
     if (compress != CompressionKind.NONE) {
       builder.setCompressionBlockSize(bufferSize);
     }
@@ -2120,7 +2050,7 @@ public class WriterImpl implements Writer, MemoryManager.Callback {
         createRowIndexEntry();
       }
     }
-    memoryManager.addedRow();
+    memoryManager.addedRow(1);
   }
 
   @Override
@@ -2132,7 +2062,7 @@ public class WriterImpl implements Writer, MemoryManager.Callback {
     memoryManager.removeWriter(path);
     // actually close the file
     flushStripe();
-    int metadataLength = writeMetadata(rawWriter.getPos());
+    int metadataLength = writeMetadata();
     int footerLength = writeFooter(rawWriter.getPos() - metadataLength);
     rawWriter.writeByte(writePostScript(footerLength, metadataLength));
     rawWriter.close();
@@ -2165,19 +2095,19 @@ public class WriterImpl implements Writer, MemoryManager.Callback {
       if (callback != null) {
         callback.preFooterWrite(callbackContext);
       }
-      int metaLength = writeMetadata(rawWriter.getPos());
+      int metaLength = writeMetadata();
       int footLength = writeFooter(rawWriter.getPos() - metaLength);
       rawWriter.writeByte(writePostScript(footLength, metaLength));
       stripesAtLastFlush = stripes.size();
-      ShimLoader.getHadoopShims().hflush(rawWriter);
+      rawWriter.hflush();
     }
     return rawWriter.getPos();
   }
 
   @Override
   public void appendStripe(byte[] stripe, int offset, int length,
-      StripeInformation stripeInfo,
-      OrcProto.StripeStatistics stripeStatistics) throws IOException {
+                           StripeInformation stripeInfo,
+                           OrcProto.StripeStatistics stripeStatistics) throws IOException {
     checkArgument(stripe != null, "Stripe must not be null");
     checkArgument(length <= stripe.length,
         "Specified length must not be greater specified array length");
@@ -2187,12 +2117,11 @@ public class WriterImpl implements Writer, MemoryManager.Callback {
 
     getStream();
     long start = rawWriter.getPos();
-    long stripeLen = length;
     long availBlockSpace = blockSize - (start % blockSize);
 
     // see if stripe can fit in the current hdfs block, else pad the remaining
     // space in the block
-    if (stripeLen < blockSize && stripeLen > availBlockSpace &&
+    if (length < blockSize && length > availBlockSpace &&
         addBlockPadding) {
       byte[] pad = new byte[(int) Math.min(HDFS_BUFFER_SIZE, availBlockSpace)];
       LOG.info(String.format("Padding ORC by %d bytes while merging..",
@@ -2245,7 +2174,7 @@ public class WriterImpl implements Writer, MemoryManager.Callback {
   }
 
   private void getAllColumnTreeWritersImpl(TreeWriter tw,
-      List<TreeWriter> result) {
+                                           List<TreeWriter> result) {
     result.add(tw);
     for (TreeWriter child : tw.childrenWriters) {
       getAllColumnTreeWritersImpl(child, result);
@@ -2253,9 +2182,9 @@ public class WriterImpl implements Writer, MemoryManager.Callback {
   }
 
   @Override
-  public void appendUserMetadata(List<UserMetadataItem> userMetadata) {
+  public void appendUserMetadata(List<OrcProto.UserMetadataItem> userMetadata) {
     if (userMetadata != null) {
-      for (UserMetadataItem item : userMetadata) {
+      for (OrcProto.UserMetadataItem item : userMetadata) {
         this.userMetadata.put(item.getName(), item.getValue());
       }
     }

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/ZeroCopyAdapter.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/ZeroCopyAdapter.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/ZeroCopyAdapter.java
new file mode 100644
index 0000000..2886fe7
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/ZeroCopyAdapter.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage.thirdparty.orc;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.ReadOption;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.EnumSet;
+
+public class ZeroCopyAdapter {
+  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, ByteBufferAllocatorPool 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);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/ZlibCodec.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/ZlibCodec.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/ZlibCodec.java
deleted file mode 100644
index d0a8fa7..0000000
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/ZlibCodec.java
+++ /dev/null
@@ -1,169 +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.tajo.storage.thirdparty.orc;
-
-import org.apache.hadoop.hive.shims.HadoopShims.DirectCompressionType;
-import org.apache.hadoop.hive.shims.HadoopShims.DirectDecompressorShim;
-import org.apache.hadoop.hive.shims.ShimLoader;
-
-import javax.annotation.Nullable;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.EnumSet;
-import java.util.zip.DataFormatException;
-import java.util.zip.Deflater;
-import java.util.zip.Inflater;
-
-class ZlibCodec implements CompressionCodec, DirectDecompressionCodec {
-
-  private Boolean direct = null;
-
-  private final int level;
-  private final int strategy;
-
-  public ZlibCodec() {
-    level = Deflater.DEFAULT_COMPRESSION;
-    strategy = Deflater.DEFAULT_STRATEGY;
-  }
-
-  private ZlibCodec(int level, int strategy) {
-    this.level = level;
-    this.strategy = strategy;
-  }
-
-  @Override
-  public boolean compress(ByteBuffer in, ByteBuffer out,
-                          ByteBuffer overflow) throws IOException {
-    Deflater deflater = new Deflater(level, true);
-    deflater.setStrategy(strategy);
-    int length = in.remaining();
-    deflater.setInput(in.array(), in.arrayOffset() + in.position(), length);
-    deflater.finish();
-    int outSize = 0;
-    int offset = out.arrayOffset() + out.position();
-    while (!deflater.finished() && (length > outSize)) {
-      int size = deflater.deflate(out.array(), offset, out.remaining());
-      out.position(size + out.position());
-      outSize += size;
-      offset += size;
-      // if we run out of space in the out buffer, use the overflow
-      if (out.remaining() == 0) {
-        if (overflow == null) {
-          deflater.end();
-          return false;
-        }
-        out = overflow;
-        offset = out.arrayOffset() + out.position();
-      }
-    }
-    deflater.end();
-    return length > outSize;
-  }
-
-  @Override
-  public void decompress(ByteBuffer in, ByteBuffer out) throws IOException {
-
-    if(in.isDirect() && out.isDirect()) {
-      directDecompress(in, out);
-      return;
-    }
-
-    Inflater inflater = new Inflater(true);
-    inflater.setInput(in.array(), in.arrayOffset() + in.position(),
-                      in.remaining());
-    while (!(inflater.finished() || inflater.needsDictionary() ||
-             inflater.needsInput())) {
-      try {
-        int count = inflater.inflate(out.array(),
-                                     out.arrayOffset() + out.position(),
-                                     out.remaining());
-        out.position(count + out.position());
-      } catch (DataFormatException dfe) {
-        throw new IOException("Bad compression data", dfe);
-      }
-    }
-    out.flip();
-    inflater.end();
-    in.position(in.limit());
-  }
-
-  @Override
-  public boolean isAvailable() {
-    if (direct == null) {
-      // see nowrap option in new Inflater(boolean) which disables zlib headers
-      try {
-        if (ShimLoader.getHadoopShims().getDirectDecompressor(
-            DirectCompressionType.ZLIB_NOHEADER) != null) {
-          direct = Boolean.valueOf(true);
-        } else {
-          direct = Boolean.valueOf(false);
-        }
-      } catch (UnsatisfiedLinkError ule) {
-        direct = Boolean.valueOf(false);
-      }
-    }
-    return direct.booleanValue();
-  }
-
-  @Override
-  public void directDecompress(ByteBuffer in, ByteBuffer out)
-      throws IOException {
-    DirectDecompressorShim decompressShim = ShimLoader.getHadoopShims()
-        .getDirectDecompressor(DirectCompressionType.ZLIB_NOHEADER);
-    decompressShim.decompress(in, out);
-    out.flip(); // flip for read
-  }
-
-  @Override
-  public CompressionCodec modify(@Nullable EnumSet<Modifier> modifiers) {
-
-    if (modifiers == null) {
-      return this;
-    }
-
-    int l = this.level;
-    int s = this.strategy;
-
-    for (Modifier m : modifiers) {
-      switch (m) {
-      case BINARY:
-        /* filtered == less LZ77, more huffman */
-        s = Deflater.FILTERED;
-        break;
-      case TEXT:
-        s = Deflater.DEFAULT_STRATEGY;
-        break;
-      case FASTEST:
-        // deflate_fast looking for 8 byte patterns
-        l = Deflater.BEST_SPEED;
-        break;
-      case FAST:
-        // deflate_fast looking for 16 byte patterns
-        l = Deflater.BEST_SPEED + 1;
-        break;
-      case DEFAULT:
-        // deflate_slow looking for 128 byte patterns
-        l = Deflater.DEFAULT_COMPRESSION;
-        break;
-      default:
-        break;
-      }
-    }
-    return new ZlibCodec(l, s);
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-storage/tajo-storage-hdfs/src/main/proto/orc_proto.proto
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/proto/orc_proto.proto b/tajo-storage/tajo-storage-hdfs/src/main/proto/orc_proto.proto
deleted file mode 100644
index c80cf6c..0000000
--- a/tajo-storage/tajo-storage-hdfs/src/main/proto/orc_proto.proto
+++ /dev/null
@@ -1,217 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.storage.thirdparty.orc;
-
-message IntegerStatistics  {
-  optional sint64 minimum = 1;
-  optional sint64 maximum = 2;
-  optional sint64 sum = 3;
-}
-
-message DoubleStatistics {
-  optional double minimum = 1;
-  optional double maximum = 2;
-  optional double sum = 3;
-}
-
-message StringStatistics {
-  optional string minimum = 1;
-  optional string maximum = 2;
-  // sum will store the total length of all strings in a stripe
-  optional sint64 sum = 3;
-}
-
-message BucketStatistics {
-  repeated uint64 count = 1 [packed=true];
-}
-
-message DecimalStatistics {
-  optional string minimum = 1;
-  optional string maximum = 2;
-  optional string sum = 3;
-}
-
-message DateStatistics {
-  // min,max values saved as days since epoch
-  optional sint32 minimum = 1;
-  optional sint32 maximum = 2;
-}
-
-message TimestampStatistics {
-  // min,max values saved as milliseconds since epoch
-  optional sint64 minimum = 1;
-  optional sint64 maximum = 2;
-}
-
-message BinaryStatistics {
-  // sum will store the total binary blob length in a stripe
-  optional sint64 sum = 1;
-}
-
-message ColumnStatistics {
-  optional uint64 numberOfValues = 1;
-  optional IntegerStatistics intStatistics = 2;
-  optional DoubleStatistics doubleStatistics = 3;
-  optional StringStatistics stringStatistics = 4;
-  optional BucketStatistics bucketStatistics = 5;
-  optional DecimalStatistics decimalStatistics = 6;
-  optional DateStatistics dateStatistics = 7;
-  optional BinaryStatistics binaryStatistics = 8;
-  optional TimestampStatistics timestampStatistics = 9;
-  optional bool hasNull = 10;
-}
-
-message RowIndexEntry {
-  repeated uint64 positions = 1 [packed=true];
-  optional ColumnStatistics statistics = 2;
-}
-
-message RowIndex {
-  repeated RowIndexEntry entry = 1;
-}
-
-message BloomFilter {
-  optional uint32 numHashFunctions = 1;
-  repeated fixed64 bitset = 2;
-}
-
-message BloomFilterIndex {
-  repeated BloomFilter bloomFilter = 1;
-}
-
-message Stream {
-  // if you add new index stream kinds, you need to make sure to update
-  // StreamName to ensure it is added to the stripe in the right area
-  enum Kind {
-    PRESENT = 0;
-    DATA = 1;
-    LENGTH = 2;
-    DICTIONARY_DATA = 3;
-    DICTIONARY_COUNT = 4;
-    SECONDARY = 5;
-    ROW_INDEX = 6;
-    BLOOM_FILTER = 7;
-  }
-  optional Kind kind = 1;
-  optional uint32 column = 2;
-  optional uint64 length = 3;
-}
-
-message ColumnEncoding {
-  enum Kind {
-    DIRECT = 0;
-    DICTIONARY = 1;
-    DIRECT_V2 = 2;
-    DICTIONARY_V2 = 3;
-  }
-  optional Kind kind = 1;
-  optional uint32 dictionarySize = 2;
-}
-
-message StripeFooter {
-  repeated Stream streams = 1;
-  repeated ColumnEncoding columns = 2;
-  optional string writerTimezone = 3;
-}
-
-message Type {
-  enum Kind {
-    BOOLEAN = 0;
-    BYTE = 1;
-    SHORT = 2;
-    INT = 3;
-    LONG = 4;
-    FLOAT = 5;
-    DOUBLE = 6;
-    STRING = 7;
-    BINARY = 8;
-    TIMESTAMP = 9;
-    LIST = 10;
-    MAP = 11;
-    STRUCT = 12;
-    UNION = 13;
-    DECIMAL = 14;
-    DATE = 15;
-    VARCHAR = 16;
-    CHAR = 17;
-  }
-  optional Kind kind = 1;
-  repeated uint32 subtypes = 2 [packed=true];
-  repeated string fieldNames = 3;
-  optional uint32 maximumLength = 4;
-  optional uint32 precision = 5;
-  optional uint32 scale = 6;
-}
-
-message StripeInformation {
-  optional uint64 offset = 1;
-  optional uint64 indexLength = 2;
-  optional uint64 dataLength = 3;
-  optional uint64 footerLength = 4;
-  optional uint64 numberOfRows = 5;
-}
-
-message UserMetadataItem {
-  optional string name = 1;
-  optional bytes value = 2;
-}
-
-message StripeStatistics {
-  repeated ColumnStatistics colStats = 1;
-}
-
-message Metadata {
-  repeated StripeStatistics stripeStats = 1;
-}
-
-message Footer {
-  optional uint64 headerLength = 1;
-  optional uint64 contentLength = 2;
-  repeated StripeInformation stripes = 3;
-  repeated Type types = 4;
-  repeated UserMetadataItem metadata = 5;
-  optional uint64 numberOfRows = 6;
-  repeated ColumnStatistics statistics = 7;
-  optional uint32 rowIndexStride = 8;
-}
-
-enum CompressionKind {
-  NONE = 0;
-  ZLIB = 1;
-  SNAPPY = 2;
-  LZO = 3;
-}
-
-// Serialized length must be less that 255 bytes
-message PostScript {
-  optional uint64 footerLength = 1;
-  optional CompressionKind compression = 2;
-  optional uint64 compressionBlockSize = 3;
-  // the version of the file format
-  //   [0, 11] = Hive 0.11
-  //   [0, 12] = Hive 0.12
-  repeated uint32 version = 4 [packed = true];
-  optional uint64 metadataLength = 5;
-  // Version of the writer:
-  //   0 (or missing) = original
-  //   1 = HIVE-8732 fixed
-  optional uint32 writerVersion = 6;
-  // Leave this last in the record
-  optional string magic = 8000;
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestCompressionStorages.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestCompressionStorages.java b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestCompressionStorages.java
index b63b497..608d066 100644
--- a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestCompressionStorages.java
+++ b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestCompressionStorages.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.compress.*;
 import org.apache.hadoop.io.compress.zlib.ZlibFactory;
 import org.apache.hadoop.util.NativeCodeLoader;
+import org.apache.orc.OrcConf;
 import org.apache.tajo.BuiltinStorages;
 import org.apache.tajo.catalog.CatalogUtil;
 import org.apache.tajo.catalog.Schema;
@@ -61,6 +62,7 @@ public class TestCompressionStorages {
   public TestCompressionStorages(String type) throws IOException {
     this.dataFormat = type;
     conf = new TajoConf();
+    conf.setBoolean("hive.exec.orc.zerocopy", true);
 
     testDir = CommonTestingUtil.getTestDir(TEST_PATH);
     fs = testDir.getFileSystem(conf);
@@ -71,7 +73,8 @@ public class TestCompressionStorages {
     return Arrays.asList(new Object[][]{
         {BuiltinStorages.TEXT},
         {BuiltinStorages.RCFILE},
-        {BuiltinStorages.SEQUENCE_FILE}
+        {BuiltinStorages.SEQUENCE_FILE},
+        {BuiltinStorages.ORC}
     });
   }
 
@@ -120,6 +123,14 @@ public class TestCompressionStorages {
     meta.putProperty("rcfile.serde", TextSerializerDeserializer.class.getName());
     meta.putProperty("sequencefile.serde", TextSerializerDeserializer.class.getName());
 
+    if (codec.equals(SnappyCodec.class)) {
+      meta.putProperty(OrcConf.COMPRESS.getAttribute(), "SNAPPY");
+    } else if (codec.equals(Lz4Codec.class)) {
+      meta.putProperty(OrcConf.COMPRESS.getAttribute(), "ZLIB");
+    } else {
+      meta.putProperty(OrcConf.COMPRESS.getAttribute(), "NONE");
+    }
+
     String fileName = "Compression_" + codec.getSimpleName();
     Path tablePath = new Path(testDir, fileName);
     Appender appender = ((FileTablespace) TablespaceManager.getLocalFs()).getAppender(meta, schema, tablePath);

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestStorages.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestStorages.java b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestStorages.java
index 552dc2e..a9d61d5 100644
--- a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestStorages.java
+++ b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestStorages.java
@@ -167,6 +167,21 @@ public class TestStorages {
    fs.delete(testDir, true);
   }
 
+  private boolean protoTypeSupport() {
+    return internalType;
+  }
+
+  private boolean timeTypeSupport() {
+    return internalType
+        || dataFormat.equalsIgnoreCase(BuiltinStorages.TEXT);
+  }
+
+  private boolean dateTypeSupport() {
+    return internalType
+        || dataFormat.equalsIgnoreCase(BuiltinStorages.TEXT)
+        || dataFormat.equalsIgnoreCase(BuiltinStorages.ORC);
+  }
+
   @Test
   public void testSplitable() throws IOException {
     if (splitable) {
@@ -385,8 +400,6 @@ public class TestStorages {
 
   @Test
   public void testVariousTypes() throws IOException {
-    boolean handleProtobuf = !dataFormat.equalsIgnoreCase(BuiltinStorages.JSON);
-
     Schema schema = new Schema();
     schema.addColumn("col1", Type.BOOLEAN);
     schema.addColumn("col2", Type.CHAR, 7);
@@ -398,7 +411,7 @@ public class TestStorages {
     schema.addColumn("col8", Type.TEXT);
     schema.addColumn("col9", Type.BLOB);
     schema.addColumn("col10", Type.INET4);
-    if (handleProtobuf) {
+    if (protoTypeSupport()) {
       schema.addColumn("col11", CatalogUtil.newDataType(Type.PROTOBUF, TajoIdProtos.QueryIdProto.class.getName()));
     }
 
@@ -418,7 +431,7 @@ public class TestStorages {
     QueryId queryid = new QueryId("12345", 5);
     ProtobufDatumFactory factory = ProtobufDatumFactory.get(TajoIdProtos.QueryIdProto.class.getName());
 
-    VTuple tuple = new VTuple(10 + (handleProtobuf ? 1 : 0));
+    VTuple tuple = new VTuple(10 + (protoTypeSupport() ? 1 : 0));
     tuple.put(new Datum[] {
         DatumFactory.createBool(true),
         DatumFactory.createChar("hyunsik"),
@@ -432,7 +445,7 @@ public class TestStorages {
         DatumFactory.createInet4("192.168.0.1"),
     });
 
-    if (handleProtobuf) {
+    if (protoTypeSupport()) {
       tuple.put(10, factory.createDatum(queryid.getProto()));
     }
 
@@ -456,8 +469,6 @@ public class TestStorages {
 
   @Test
   public void testNullHandlingTypes() throws IOException {
-    boolean handleProtobuf = !dataFormat.equalsIgnoreCase(BuiltinStorages.JSON);
-
     Schema schema = new Schema();
     schema.addColumn("col1", Type.BOOLEAN);
     schema.addColumn("col2", Type.CHAR, 7);
@@ -470,7 +481,7 @@ public class TestStorages {
     schema.addColumn("col9", Type.BLOB);
     schema.addColumn("col10", Type.INET4);
 
-    if (handleProtobuf) {
+    if (protoTypeSupport()) {
       schema.addColumn("col11", CatalogUtil.newDataType(Type.PROTOBUF, TajoIdProtos.QueryIdProto.class.getName()));
     }
 
@@ -492,7 +503,7 @@ public class TestStorages {
 
     QueryId queryid = new QueryId("12345", 5);
     ProtobufDatumFactory factory = ProtobufDatumFactory.get(TajoIdProtos.QueryIdProto.class.getName());
-    int columnNum = 10 + (handleProtobuf ? 1 : 0);
+    int columnNum = 10 + (protoTypeSupport() ? 1 : 0);
     VTuple seedTuple = new VTuple(columnNum);
     seedTuple.put(new Datum[]{
         DatumFactory.createBool(true),                // 0
@@ -507,7 +518,7 @@ public class TestStorages {
         DatumFactory.createInet4("192.168.0.1")       // 10
     });
 
-    if (handleProtobuf) {
+    if (protoTypeSupport()) {
       seedTuple.put(10, factory.createDatum(queryid.getProto()));       // 11
     }
 
@@ -553,8 +564,6 @@ public class TestStorages {
   public void testNullHandlingTypesWithProjection() throws IOException {
     if (internalType) return;
 
-    boolean handleProtobuf = !dataFormat.equalsIgnoreCase(BuiltinStorages.JSON);
-
     Schema schema = new Schema();
     schema.addColumn("col1", Type.BOOLEAN);
     schema.addColumn("col2", Type.CHAR, 7);
@@ -567,7 +576,7 @@ public class TestStorages {
     schema.addColumn("col9", Type.BLOB);
     schema.addColumn("col10", Type.INET4);
 
-    if (handleProtobuf) {
+    if (protoTypeSupport()) {
       schema.addColumn("col11", CatalogUtil.newDataType(Type.PROTOBUF, TajoIdProtos.QueryIdProto.class.getName()));
     }
 
@@ -589,7 +598,7 @@ public class TestStorages {
 
     QueryId queryid = new QueryId("12345", 5);
     ProtobufDatumFactory factory = ProtobufDatumFactory.get(TajoIdProtos.QueryIdProto.class.getName());
-    int columnNum = 10 + (handleProtobuf ? 1 : 0);
+    int columnNum = 10 + (protoTypeSupport() ? 1 : 0);
     VTuple seedTuple = new VTuple(columnNum);
     seedTuple.put(new Datum[]{
         DatumFactory.createBool(true),                // 0
@@ -604,7 +613,7 @@ public class TestStorages {
         DatumFactory.createInet4("192.168.0.1")       // 10
     });
 
-    if (handleProtobuf) {
+    if (protoTypeSupport()) {
       seedTuple.put(10, factory.createDatum(queryid.getProto()));       // 11
     }
 
@@ -933,11 +942,17 @@ public class TestStorages {
 
   @Test
   public void testTime() throws IOException {
-    if (dataFormat.equalsIgnoreCase(BuiltinStorages.TEXT) || internalType) {
+    if (dateTypeSupport() || timeTypeSupport()) {
+
+      int index = 2;
       Schema schema = new Schema();
-      schema.addColumn("col1", Type.DATE);
-      schema.addColumn("col2", Type.TIME);
-      schema.addColumn("col3", Type.TIMESTAMP);
+      schema.addColumn("col1", Type.TIMESTAMP);
+      if (dateTypeSupport()) {
+        schema.addColumn("col" + index++, Type.DATE);
+      }
+      if (timeTypeSupport()) {
+        schema.addColumn("col" + index++, Type.TIME);
+      }
 
       KeyValueSet options = new KeyValueSet();
       TableMeta meta = CatalogUtil.newTableMeta(dataFormat, options);
@@ -947,11 +962,15 @@ public class TestStorages {
       Appender appender = sm.getAppender(meta, schema, tablePath);
       appender.init();
 
-      VTuple tuple = new VTuple(new Datum[]{
-          DatumFactory.createDate("1980-04-01"),
-          DatumFactory.createTime("12:34:56"),
-          DatumFactory.createTimestmpDatumWithUnixTime((int)(System.currentTimeMillis() / 1000))
-      });
+      VTuple tuple = new VTuple(index - 1);
+      index = 0;
+      tuple.put(index++, DatumFactory.createTimestmpDatumWithUnixTime((int)(System.currentTimeMillis() / 1000)));
+      if (dateTypeSupport()) {
+        tuple.put(index++, DatumFactory.createDate("1980-04-01"));
+      }
+      if (timeTypeSupport()) {
+        tuple.put(index, DatumFactory.createTime("12:34:56"));
+      }
       appender.addTuple(tuple);
       appender.flush();
       appender.close();
@@ -964,7 +983,7 @@ public class TestStorages {
       Tuple retrieved;
       while ((retrieved = scanner.next()) != null) {
         for (int i = 0; i < tuple.size(); i++) {
-          assertEquals(tuple.get(i), retrieved.asDatum(i));
+          assertEquals("failed at " + i + " th column", tuple.get(i), retrieved.asDatum(i));
         }
       }
       scanner.close();

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-storage/tajo-storage-hdfs/src/test/resources/dataset/testVariousTypes.avsc
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/test/resources/dataset/testVariousTypes.avsc b/tajo-storage/tajo-storage-hdfs/src/test/resources/dataset/testVariousTypes.avsc
index f71f052..f1d1368 100644
--- a/tajo-storage/tajo-storage-hdfs/src/test/resources/dataset/testVariousTypes.avsc
+++ b/tajo-storage/tajo-storage-hdfs/src/test/resources/dataset/testVariousTypes.avsc
@@ -12,8 +12,7 @@
     { "name": "col7", "type": "double" },
     { "name": "col8", "type": "string" },
     { "name": "col9", "type": "bytes" },
-    { "name": "col10", "type": "bytes" },
-    { "name": "col11", "type": "bytes" }
+    { "name": "col10", "type": "bytes" }
   ]
 }
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-storage/tajo-storage-hdfs/src/test/resources/storage-default.xml
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/test/resources/storage-default.xml b/tajo-storage/tajo-storage-hdfs/src/test/resources/storage-default.xml
index 6f7e53b..3283f9f 100644
--- a/tajo-storage/tajo-storage-hdfs/src/test/resources/storage-default.xml
+++ b/tajo-storage/tajo-storage-hdfs/src/test/resources/storage-default.xml
@@ -117,7 +117,7 @@
 
   <property>
     <name>tajo.storage.scanner-handler.orc.class</name>
-    <value>org.apache.tajo.storage.orc.ORCScanner</value>
+    <value>org.apache.tajo.storage.orc.OrcScanner</value>
   </property>
 
   <property>


[2/7] tajo git commit: TAJO-2102: Migrate to Apache Orc from Presto's one.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/TreeReaderFactory.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/TreeReaderFactory.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/TreeReaderFactory.java
new file mode 100644
index 0000000..6ab630a
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/TreeReaderFactory.java
@@ -0,0 +1,1557 @@
+/*
+ * 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.tajo.storage.thirdparty.orc;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
+import org.apache.hadoop.io.Text;
+import org.apache.orc.OrcProto;
+import org.apache.orc.impl.*;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.TypeDesc;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.datum.NullDatum;
+import org.apache.tajo.exception.TajoRuntimeException;
+import org.apache.tajo.exception.UnsupportedException;
+import org.apache.tajo.util.datetime.DateTimeConstants;
+import org.apache.tajo.util.datetime.DateTimeUtil;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.sql.Timestamp;
+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 static org.apache.tajo.storage.thirdparty.orc.WriterImpl.BASE_TIMESTAMP_STRING;
+
+public class TreeReaderFactory {
+
+  private final static Log LOG = LogFactory.getLog(TreeReaderFactory.class);
+
+  public static class TreeReaderSchema {
+
+    /**
+     * The types in the ORC file.
+     */
+    List<OrcProto.Type> fileTypes;
+
+    /**
+     * The treeReaderSchema that the reader should read as.
+     */
+    List<OrcProto.Type> schemaTypes;
+
+    /**
+     * The subtype of the row STRUCT.  Different than 0 for ACID.
+     */
+    int innerStructSubtype;
+
+    public TreeReaderSchema() {
+      fileTypes = null;
+      schemaTypes = null;
+      innerStructSubtype = -1;
+    }
+
+    public TreeReaderSchema fileTypes(List<OrcProto.Type> fileTypes) {
+      this.fileTypes = fileTypes;
+      return this;
+    }
+
+    public TreeReaderSchema schemaTypes(List<OrcProto.Type> schemaTypes) {
+      this.schemaTypes = schemaTypes;
+      return this;
+    }
+
+    public TreeReaderSchema innerStructSubtype(int innerStructSubtype) {
+      this.innerStructSubtype = innerStructSubtype;
+      return this;
+    }
+
+    public List<OrcProto.Type> getFileTypes() {
+      return fileTypes;
+    }
+
+    public List<OrcProto.Type> getSchemaTypes() {
+      return schemaTypes;
+    }
+
+    public int getInnerStructSubtype() {
+      return innerStructSubtype;
+    }
+  }
+
+  public abstract static class TreeReader {
+    protected final int columnId;
+    protected BitFieldReader present = null;
+    protected boolean valuePresent = false;
+
+    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);
+      }
+    }
+
+    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<org.apache.orc.impl.StreamName, InStream> streams,
+                     OrcProto.StripeFooter stripeFooter
+    ) throws IOException {
+      checkEncoding(stripeFooter.getColumnsList().get(columnId));
+      InStream in = streams.get(new org.apache.orc.impl.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;
+
+    public BitFieldReader getPresent() {
+      return present;
+    }
+  }
+
+  public abstract static class DatumTreeReader extends TreeReader {
+
+    DatumTreeReader(int columnId) throws IOException {
+      super(columnId);
+    }
+
+    protected DatumTreeReader(int columnId, InStream in) throws IOException {
+      super(columnId, in);
+    }
+
+    Datum next() throws IOException {
+      if (present != null) {
+        valuePresent = present.next() == 1;
+      }
+      return NullDatum.get();
+    }
+  }
+
+  public abstract static class RawStringTreeReader extends TreeReader {
+    RawStringTreeReader(int columnId) throws IOException {
+      super(columnId);
+    }
+
+    protected RawStringTreeReader(int columnId, InStream in) throws IOException {
+      super(columnId, in);
+    }
+
+    byte[] next() throws IOException {
+      if (present != null) {
+        valuePresent = present.next() == 1;
+      }
+      return null;
+    }
+  }
+
+  public static class BooleanTreeReader extends DatumTreeReader {
+    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<org.apache.orc.impl.StreamName, InStream> streams,
+                     OrcProto.StripeFooter stripeFooter
+    ) throws IOException {
+      super.startStripe(streams, stripeFooter);
+      reader = new BitFieldReader(streams.get(new org.apache.orc.impl.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
+    Datum next() throws IOException {
+      super.next();
+      return valuePresent ? DatumFactory.createBool(reader.next() == 1) : NullDatum.get();
+    }
+  }
+
+  public static class ByteTreeReader extends DatumTreeReader {
+    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<org.apache.orc.impl.StreamName, InStream> streams,
+                     OrcProto.StripeFooter stripeFooter
+    ) throws IOException {
+      super.startStripe(streams, stripeFooter);
+      reader = new RunLengthByteReader(streams.get(new org.apache.orc.impl.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
+    Datum next() throws IOException {
+      super.next();
+      return valuePresent ? DatumFactory.createBit(reader.next()) : NullDatum.get();
+    }
+
+    @Override
+    void skipRows(long items) throws IOException {
+      reader.skip(countNonNulls(items));
+    }
+  }
+
+  public static class ShortTreeReader extends DatumTreeReader {
+    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<org.apache.orc.impl.StreamName, InStream> streams,
+                     OrcProto.StripeFooter stripeFooter
+    ) throws IOException {
+      super.startStripe(streams, stripeFooter);
+      org.apache.orc.impl.StreamName name = new org.apache.orc.impl.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
+    Datum next() throws IOException {
+      super.next();
+      return valuePresent ? DatumFactory.createInt2((short) reader.next()) : NullDatum.get();
+    }
+
+    @Override
+    void skipRows(long items) throws IOException {
+      reader.skip(countNonNulls(items));
+    }
+  }
+
+  public static class InetTreeReader extends DatumTreeReader {
+    protected IntegerReader reader = null;
+
+    InetTreeReader(int columnId) throws IOException {
+      this(columnId, null, null, null);
+    }
+
+    protected InetTreeReader(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<org.apache.orc.impl.StreamName, InStream> streams,
+                     OrcProto.StripeFooter stripeFooter
+    ) throws IOException {
+      super.startStripe(streams, stripeFooter);
+      org.apache.orc.impl.StreamName name = new org.apache.orc.impl.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
+    Datum next() throws IOException {
+      super.next();
+      return valuePresent ? DatumFactory.createInet4((int) reader.next()) : NullDatum.get();
+    }
+
+    @Override
+    void skipRows(long items) throws IOException {
+      reader.skip(countNonNulls(items));
+    }
+  }
+
+  public static class IntTreeReader extends DatumTreeReader {
+    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<org.apache.orc.impl.StreamName, InStream> streams,
+                     OrcProto.StripeFooter stripeFooter
+    ) throws IOException {
+      super.startStripe(streams, stripeFooter);
+      org.apache.orc.impl.StreamName name = new org.apache.orc.impl.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
+    Datum next() throws IOException {
+      super.next();
+      return valuePresent ? DatumFactory.createInt4((int) reader.next()) : NullDatum.get();
+    }
+
+    @Override
+    void skipRows(long items) throws IOException {
+      reader.skip(countNonNulls(items));
+    }
+  }
+
+  public static class LongTreeReader extends DatumTreeReader {
+    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<org.apache.orc.impl.StreamName, InStream> streams,
+                     OrcProto.StripeFooter stripeFooter
+    ) throws IOException {
+      super.startStripe(streams, stripeFooter);
+      org.apache.orc.impl.StreamName name = new org.apache.orc.impl.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
+    Datum next() throws IOException {
+      super.next();
+      return valuePresent ? DatumFactory.createInt8(reader.next()) : NullDatum.get();
+    }
+
+    @Override
+    void skipRows(long items) throws IOException {
+      reader.skip(countNonNulls(items));
+    }
+  }
+
+  public static class FloatTreeReader extends DatumTreeReader {
+    protected InStream stream;
+    private final org.apache.orc.impl.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 org.apache.orc.impl.SerializationUtils();
+      this.stream = data;
+    }
+
+    @Override
+    void startStripe(Map<org.apache.orc.impl.StreamName, InStream> streams,
+                     OrcProto.StripeFooter stripeFooter
+    ) throws IOException {
+      super.startStripe(streams, stripeFooter);
+      org.apache.orc.impl.StreamName name = new org.apache.orc.impl.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
+    Datum next() throws IOException {
+      super.next();
+      return valuePresent ? DatumFactory.createFloat4(utils.readFloat(stream)) : NullDatum.get();
+    }
+
+    @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 DatumTreeReader {
+    protected InStream stream;
+    private final org.apache.orc.impl.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<org.apache.orc.impl.StreamName, InStream> streams,
+                     OrcProto.StripeFooter stripeFooter
+    ) throws IOException {
+      super.startStripe(streams, stripeFooter);
+      org.apache.orc.impl.StreamName name =
+          new org.apache.orc.impl.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
+    Datum next() throws IOException {
+      super.next();
+      return valuePresent ? DatumFactory.createFloat8(utils.readDouble(stream)) : NullDatum.get();
+    }
+
+    @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 DatumTreeReader {
+    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<org.apache.orc.impl.StreamName, InStream> streams,
+                     OrcProto.StripeFooter stripeFooter
+    ) throws IOException {
+      super.startStripe(streams, stripeFooter);
+      org.apache.orc.impl.StreamName name = new org.apache.orc.impl.StreamName(columnId,
+          OrcProto.Stream.Kind.DATA);
+      stream = streams.get(name);
+      lengths = createIntegerReader(stripeFooter.getColumnsList().get(columnId).getKind(),
+          streams.get(new org.apache.orc.impl.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
+    Datum next() throws IOException {
+      super.next();
+
+      if (valuePresent) {
+        int len = (int) lengths.next();
+        byte[] buf = new byte[len];
+        int offset = 0;
+        while (len > 0) {
+          int written = stream.read(buf, offset, len);
+          if (written < 0) {
+            throw new EOFException("Can't finish byte read from " + stream);
+          }
+          len -= written;
+          offset += written;
+        }
+        return DatumFactory.createBlob(buf);
+      } else {
+        return NullDatum.get();
+      }
+    }
+
+    @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 DatumTreeReader {
+    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;
+    private final TimeZone timeZone;
+
+    TimestampTreeReader(TimeZone timeZone, int columnId, boolean skipCorrupt) throws IOException {
+      this(timeZone, columnId, null, null, null, null, skipCorrupt);
+    }
+
+    protected TimestampTreeReader(TimeZone timeZone, 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);
+        }
+      }
+      this.timeZone = timeZone;
+    }
+
+    @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<org.apache.orc.impl.StreamName, InStream> streams,
+                     OrcProto.StripeFooter stripeFooter
+    ) throws IOException {
+      super.startStripe(streams, stripeFooter);
+      data = createIntegerReader(stripeFooter.getColumnsList().get(columnId).getKind(),
+          streams.get(new org.apache.orc.impl.StreamName(columnId,
+              OrcProto.Stream.Kind.DATA)), true, skipCorrupt);
+      nanos = createIntegerReader(stripeFooter.getColumnsList().get(columnId).getKind(),
+          streams.get(new org.apache.orc.impl.StreamName(columnId,
+              OrcProto.Stream.Kind.SECONDARY)), false, skipCorrupt);
+      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(BASE_TIMESTAMP_STRING).getTime() / DateTimeConstants.MSECS_PER_SEC;
+          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
+    Datum next() throws IOException {
+      super.next();
+
+      if (valuePresent) {
+        long millis = decodeTimestamp(data.next(), nanos.next(), base_timestamp);
+        long adjustedMillis = millis - writerTimeZone.getRawOffset();
+        return DatumFactory.createTimestamp(DateTimeUtil.javaTimeToJulianTime(adjustedMillis));
+      } else {
+        return NullDatum.get();
+      }
+    }
+
+    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;
+    }
+
+    // borrowed from Facebook's TimestampStreamReader
+    private static long decodeTimestamp(long seconds, long serializedNanos, long baseTimestampInSeconds) {
+      long millis = (seconds + baseTimestampInSeconds) * DateTimeConstants.MSECS_PER_SEC;
+      long nanos = parseNanos(serializedNanos);
+
+      // the rounding error exists because java always rounds up when dividing integers
+      // -42001/1000 = -42; and -42001 % 1000 = -1 (+ 1000)
+      // to get the correct value we need
+      // (-42 - 1)*1000 + 999 = -42001
+      // (42)*1000 + 1 = 42001
+      if (millis < 0 && nanos != 0) {
+        millis -= 1000;
+      }
+      // Truncate nanos to millis and add to mills
+      return millis + (nanos / 1_000_000);
+    }
+
+    @Override
+    void skipRows(long items) throws IOException {
+      items = countNonNulls(items);
+      data.skip(items);
+      nanos.skip(items);
+    }
+  }
+
+  public static class DateTreeReader extends DatumTreeReader {
+    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<org.apache.orc.impl.StreamName, InStream> streams,
+                     OrcProto.StripeFooter stripeFooter
+    ) throws IOException {
+      super.startStripe(streams, stripeFooter);
+      org.apache.orc.impl.StreamName name = new org.apache.orc.impl.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
+    Datum next() throws IOException {
+      super.next();
+      return valuePresent ?
+          DatumFactory.createDate((int) reader.next() + DateTimeUtil.DAYS_FROM_JULIAN_TO_EPOCH) : NullDatum.get();
+    }
+
+    @Override
+    void skipRows(long items) throws IOException {
+      reader.skip(countNonNulls(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 DatumTreeReader {
+    protected RawStringTreeReader 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<org.apache.orc.impl.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
+    Datum next() throws IOException {
+      byte[] bytes = reader.next();
+      return bytes == null ? NullDatum.get() : DatumFactory.createText(bytes);
+    }
+
+    @Override
+    void skipRows(long items) throws IOException {
+      reader.skipRows(items);
+    }
+  }
+
+  private final static class BasicTextReaderShim {
+    private final InputStream in;
+
+    public BasicTextReaderShim(InputStream in) {
+      this.in = in;
+    }
+
+    public byte[] read(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;
+      }
+      return bytes;
+    }
+  }
+
+  /**
+   * A reader for string columns that are direct encoded in the current
+   * stripe.
+   */
+  public static class StringDirectTreeReader extends RawStringTreeReader {
+    protected InStream stream;
+    protected BasicTextReaderShim 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);
+      }
+    }
+
+    @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<org.apache.orc.impl.StreamName, InStream> streams,
+                     OrcProto.StripeFooter stripeFooter
+    ) throws IOException {
+      super.startStripe(streams, stripeFooter);
+      org.apache.orc.impl.StreamName name = new org.apache.orc.impl.StreamName(columnId,
+          OrcProto.Stream.Kind.DATA);
+      stream = streams.get(name);
+      data = new BasicTextReaderShim(stream);
+
+      lengths = createIntegerReader(stripeFooter.getColumnsList().get(columnId).getKind(),
+          streams.get(new org.apache.orc.impl.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
+    byte[] next() throws IOException {
+      super.next();
+      int len = (int) lengths.next();
+      return valuePresent ? data.read(len) : null;
+    }
+
+    @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 RawStringTreeReader {
+    private org.apache.orc.impl.DynamicByteArray dictionaryBuffer;
+    private int[] dictionaryOffsets;
+    protected IntegerReader reader;
+
+    private byte[] dictionaryBufferInBytesCache = null;
+    private final LongColumnVector scratchlcv;
+    private final Text result = new Text();
+
+    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<org.apache.orc.impl.StreamName, InStream> streams,
+                     OrcProto.StripeFooter stripeFooter
+    ) throws IOException {
+      super.startStripe(streams, stripeFooter);
+
+      // read the dictionary blob
+      org.apache.orc.impl.StreamName name = new org.apache.orc.impl.StreamName(columnId,
+          OrcProto.Stream.Kind.DICTIONARY_DATA);
+      InStream in = streams.get(name);
+      readDictionaryStream(in);
+
+      // read the lengths
+      name = new org.apache.orc.impl.StreamName(columnId, OrcProto.Stream.Kind.LENGTH);
+      in = streams.get(name);
+      readDictionaryLengthStream(in, stripeFooter.getColumnsList().get(columnId));
+
+      // set up the row reader
+      name = new org.apache.orc.impl.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
+    byte[] next() throws IOException {
+      super.next();
+      if (valuePresent) {
+        int entry = (int) reader.next();
+        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.getBytes();
+      } else {
+        return null;
+      }
+    }
+
+    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;
+    }
+  }
+
+  /**
+   * 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 CharTreeReader extends DatumTreeReader {
+    protected RawStringTreeReader reader;
+    private final int maxLength;
+
+    CharTreeReader(int columnId, int maxLength) throws IOException {
+      this(columnId, null, null, null, null, null, maxLength);
+    }
+
+    protected CharTreeReader(int columnId, InStream present, InStream data, InStream length,
+                             InStream dictionary, OrcProto.ColumnEncoding encoding, int maxLength) throws IOException {
+      super(columnId, present);
+      this.maxLength = maxLength;
+      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
+    Datum next() throws IOException {
+      byte[] bytes = reader.next();
+
+      if (bytes == null) {
+        return NullDatum.get();
+      }
+      // TODO: enforce char length
+      return DatumFactory.createChar(bytes);
+    }
+
+    @Override
+    void skipRows(long items) throws IOException {
+      reader.skipRows(items);
+    }
+  }
+
+  // TODO: enable this to support record type
+//  protected static class StructTreeReader extends TreeReader {
+//    private final int fileColumnCount;
+//    private final int resultColumnCount;
+//    protected final TreeReader[] fields;
+//    private final String[] fieldNames;
+//
+//    protected StructTreeReader(
+//        int columnId,
+//        TreeReaderSchema treeReaderSchema,
+//        boolean[] included,
+//        boolean skipCorrupt) throws IOException {
+//      super(columnId);
+//
+//      OrcProto.Type fileStructType = treeReaderSchema.getFileTypes().get(columnId);
+//      fileColumnCount = fileStructType.getFieldNamesCount();
+//
+//      OrcProto.Type schemaStructType = treeReaderSchema.getSchemaTypes().get(columnId);
+//
+//      if (columnId == treeReaderSchema.getInnerStructSubtype()) {
+//        // If there are more result columns than reader columns, we will default those additional
+//        // columns to NULL.
+//        resultColumnCount = schemaStructType.getFieldNamesCount();
+//      } else {
+//        resultColumnCount = fileColumnCount;
+//      }
+//
+//      this.fields = new TreeReader[fileColumnCount];
+//      this.fieldNames = new String[fileColumnCount];
+//
+//      if (included == null) {
+//        for (int i = 0; i < fileColumnCount; ++i) {
+//          int subtype = schemaStructType.getSubtypes(i);
+//          this.fields[i] = createTreeReader(subtype, treeReaderSchema, included, skipCorrupt);
+//          // Use the treeReaderSchema evolution name since file/reader types may not have the real column name.
+//          this.fieldNames[i] = schemaStructType.getFieldNames(i);
+//        }
+//      } else {
+//        for (int i = 0; i < fileColumnCount; ++i) {
+//          int subtype = schemaStructType.getSubtypes(i);
+//          if (subtype >= included.length) {
+//            throw new IOException("subtype " + subtype + " exceeds the included array size " +
+//                included.length + " fileTypes " + treeReaderSchema.getFileTypes().toString() +
+//                " schemaTypes " + treeReaderSchema.getSchemaTypes().toString() +
+//                " innerStructSubtype " + treeReaderSchema.getInnerStructSubtype());
+//          }
+//          if (included[subtype]) {
+//            this.fields[i] = createTreeReader(subtype, treeReaderSchema, included, skipCorrupt);
+//          }
+//          // Use the treeReaderSchema evolution name since file/reader types may not have the real column name.
+//          this.fieldNames[i] = schemaStructType.getFieldNames(i);
+//        }
+//      }
+//    }
+//
+//    @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(resultColumnCount);
+//        } 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
+//          if (result.getNumFields() != resultColumnCount) {
+//            result.setNumFields(resultColumnCount);
+//          }
+//        }
+//        for (int i = 0; i < fileColumnCount; ++i) {
+//          if (fields[i] != null) {
+//            result.setFieldValue(i, fields[i].next(result.getFieldValue(i)));
+//          }
+//        }
+//        if (resultColumnCount > fileColumnCount) {
+//          for (int i = fileColumnCount; i < resultColumnCount; ++i) {
+//            // Default new treeReaderSchema evolution fields to NULL.
+//            result.setFieldValue(i, null);
+//          }
+//        }
+//      }
+//      return result;
+//    }
+//
+//    @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 DatumTreeReader createTreeReader(TimeZone timeZone,
+                                                 int columnId,
+                                                 Column column,
+                                                 boolean skipCorrupt
+  ) throws IOException {
+    TypeDesc typeDesc = column.getTypeDesc();
+    int orcColumnId = columnId + 1; // root record column is considered
+    switch (typeDesc.getDataType().getType()) {
+      case BOOLEAN:
+        return new BooleanTreeReader(orcColumnId);
+      case BIT:
+        return new ByteTreeReader(orcColumnId);
+      case FLOAT8:
+        return new DoubleTreeReader(orcColumnId);
+      case FLOAT4:
+        return new FloatTreeReader(orcColumnId);
+      case INT2:
+        return new ShortTreeReader(orcColumnId);
+      case INT4:
+        return new IntTreeReader(orcColumnId);
+      case INT8:
+        return new LongTreeReader(orcColumnId, skipCorrupt);
+      case TEXT:
+        return new StringTreeReader(orcColumnId);
+      case CHAR:
+        return new CharTreeReader(orcColumnId, typeDesc.getDataType().getLength());
+      case BLOB:
+        return new BinaryTreeReader(orcColumnId);
+      case TIMESTAMP:
+        return new TimestampTreeReader(timeZone, orcColumnId, skipCorrupt);
+      case DATE:
+        return new DateTreeReader(orcColumnId);
+      case INET4:
+        return new InetTreeReader(orcColumnId);
+//      case STRUCT:
+//        return new StructTreeReader(columnId, treeReaderSchema, included, skipCorrupt);
+      default:
+        throw new TajoRuntimeException(new UnsupportedException("Unsupported type " +
+            typeDesc.getDataType().getType().name()));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/Writer.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/Writer.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/Writer.java
index 669b44f..2c85aa6 100644
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/Writer.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/Writer.java
@@ -18,6 +18,8 @@
 
 package org.apache.tajo.storage.thirdparty.orc;
 
+import org.apache.orc.OrcProto;
+import org.apache.orc.StripeInformation;
 import org.apache.tajo.storage.Tuple;
 
 import java.io.IOException;


[6/7] tajo git commit: TAJO-2102: Migrate to Apache Orc from Presto's one.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/objectinspector/TajoBlobObjectInspector.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/objectinspector/TajoBlobObjectInspector.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/objectinspector/TajoBlobObjectInspector.java
deleted file mode 100644
index d241f84..0000000
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/objectinspector/TajoBlobObjectInspector.java
+++ /dev/null
@@ -1,82 +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.tajo.storage.orc.objectinspector;
-
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.BinaryObjectInspector;
-import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
-import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
-import org.apache.hadoop.io.BytesWritable;
-import org.apache.tajo.datum.Datum;
-
-public class TajoBlobObjectInspector extends TajoPrimitiveObjectInspector implements BinaryObjectInspector {
-  @Override
-  public PrimitiveTypeInfo getTypeInfo() {
-    return TypeInfoFactory.binaryTypeInfo;
-  }
-
-  @Override
-  public PrimitiveCategory getPrimitiveCategory() {
-    return PrimitiveCategory.BINARY;
-  }
-
-  @Override
-  public Class<?> getPrimitiveWritableClass() {
-    return null;
-  }
-
-  @Override
-  public BytesWritable getPrimitiveWritableObject(Object o) {
-    return null;
-  }
-
-  @Override
-  public Class<?> getJavaPrimitiveClass() {
-    return byte [].class;
-  }
-
-  @Override
-  public byte[] getPrimitiveJavaObject(Object o) {
-    return ((Datum)o).asByteArray();
-  }
-
-  @Override
-  public Object copyObject(Object o) {
-    return null;
-  }
-
-  @Override
-  public boolean preferWritable() {
-    return false;
-  }
-
-  @Override
-  public int precision() {
-    return 0;
-  }
-
-  @Override
-  public int scale() {
-    return 0;
-  }
-
-  @Override
-  public String getTypeName() {
-    return "BINARY";
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/objectinspector/TajoBooleanObjectInspector.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/objectinspector/TajoBooleanObjectInspector.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/objectinspector/TajoBooleanObjectInspector.java
deleted file mode 100644
index 273505f..0000000
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/objectinspector/TajoBooleanObjectInspector.java
+++ /dev/null
@@ -1,76 +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.tajo.storage.orc.objectinspector;
-
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.BooleanObjectInspector;
-import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
-import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
-import org.apache.tajo.datum.Datum;
-
-public class TajoBooleanObjectInspector extends TajoPrimitiveObjectInspector implements BooleanObjectInspector {
-  @Override
-  public boolean get(Object o) {
-    return ((Datum)o).asBool();
-  }
-
-  @Override
-  public PrimitiveTypeInfo getTypeInfo() {
-    return TypeInfoFactory.booleanTypeInfo;
-  }
-
-  @Override
-  public PrimitiveCategory getPrimitiveCategory() {
-    return PrimitiveCategory.BOOLEAN;
-  }
-
-  @Override
-  public Class<?> getPrimitiveWritableClass() {
-    return null;
-  }
-
-  @Override
-  public Object getPrimitiveWritableObject(Object o) {
-    return null;
-  }
-
-  @Override
-  public Class<?> getJavaPrimitiveClass() {
-    return Boolean.class;
-  }
-
-  @Override
-  public Object getPrimitiveJavaObject(Object o) {
-    return null;
-  }
-
-  @Override
-  public Object copyObject(Object o) {
-    return null;
-  }
-
-  @Override
-  public boolean preferWritable() {
-    return false;
-  }
-
-  @Override
-  public String getTypeName() {
-    return "BOOLEAN";
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/objectinspector/TajoDateObjectInspector.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/objectinspector/TajoDateObjectInspector.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/objectinspector/TajoDateObjectInspector.java
deleted file mode 100644
index f12706b..0000000
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/objectinspector/TajoDateObjectInspector.java
+++ /dev/null
@@ -1,73 +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.tajo.storage.orc.objectinspector;
-
-import org.apache.hadoop.hive.serde2.io.DateWritable;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.DateObjectInspector;
-import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
-import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
-
-import java.sql.Date;
-
-public class TajoDateObjectInspector extends TajoPrimitiveObjectInspector implements DateObjectInspector {
-  @Override
-  public PrimitiveTypeInfo getTypeInfo() {
-    return TypeInfoFactory.dateTypeInfo;
-  }
-
-  @Override
-  public PrimitiveCategory getPrimitiveCategory() {
-    return PrimitiveCategory.DATE;
-  }
-
-  @Override
-  public Class<?> getPrimitiveWritableClass() {
-    return null;
-  }
-
-  @Override
-  public DateWritable getPrimitiveWritableObject(Object o) {
-    return null;
-  }
-
-  @Override
-  public Class<?> getJavaPrimitiveClass() {
-    return null;
-  }
-
-  @Override
-  public Date getPrimitiveJavaObject(Object o) {
-    return null;
-  }
-
-  @Override
-  public Object copyObject(Object o) {
-    return null;
-  }
-
-  @Override
-  public boolean preferWritable() {
-    return false;
-  }
-
-  @Override
-  public String getTypeName() {
-    return "DATE";
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/objectinspector/TajoDoubleObjectInspector.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/objectinspector/TajoDoubleObjectInspector.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/objectinspector/TajoDoubleObjectInspector.java
deleted file mode 100644
index 6dc1f8c..0000000
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/objectinspector/TajoDoubleObjectInspector.java
+++ /dev/null
@@ -1,76 +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.tajo.storage.orc.objectinspector;
-
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.DoubleObjectInspector;
-import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
-import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
-import org.apache.tajo.datum.Float8Datum;
-
-public class TajoDoubleObjectInspector extends TajoPrimitiveObjectInspector implements DoubleObjectInspector {
-  @Override
-  public double get(Object o) {
-    return ((Float8Datum)o).asFloat8();
-  }
-
-  @Override
-  public PrimitiveTypeInfo getTypeInfo() {
-    return TypeInfoFactory.doubleTypeInfo;
-  }
-
-  @Override
-  public PrimitiveCategory getPrimitiveCategory() {
-    return PrimitiveCategory.DOUBLE;
-  }
-
-  @Override
-  public Class<?> getPrimitiveWritableClass() {
-    return null;
-  }
-
-  @Override
-  public Object getPrimitiveWritableObject(Object o) {
-    return null;
-  }
-
-  @Override
-  public Class<?> getJavaPrimitiveClass() {
-    return Double.class;
-  }
-
-  @Override
-  public Object getPrimitiveJavaObject(Object o) {
-    return null;
-  }
-
-  @Override
-  public Object copyObject(Object o) {
-    return null;
-  }
-
-  @Override
-  public boolean preferWritable() {
-    return false;
-  }
-
-  @Override
-  public String getTypeName() {
-    return "DOUBLE";
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/objectinspector/TajoFloatObjectInspector.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/objectinspector/TajoFloatObjectInspector.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/objectinspector/TajoFloatObjectInspector.java
deleted file mode 100644
index bed8784..0000000
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/objectinspector/TajoFloatObjectInspector.java
+++ /dev/null
@@ -1,76 +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.tajo.storage.orc.objectinspector;
-
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.DoubleObjectInspector;
-import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
-import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
-import org.apache.tajo.datum.Float4Datum;
-
-public class TajoFloatObjectInspector extends TajoPrimitiveObjectInspector implements DoubleObjectInspector {
-  @Override
-  public double get(Object o) {
-    return ((Float4Datum)o).asFloat4();
-  }
-
-  @Override
-  public PrimitiveTypeInfo getTypeInfo() {
-    return TypeInfoFactory.floatTypeInfo;
-  }
-
-  @Override
-  public PrimitiveCategory getPrimitiveCategory() {
-    return PrimitiveCategory.FLOAT;
-  }
-
-  @Override
-  public Class<?> getPrimitiveWritableClass() {
-    return null;
-  }
-
-  @Override
-  public Object getPrimitiveWritableObject(Object o) {
-    return null;
-  }
-
-  @Override
-  public Class<?> getJavaPrimitiveClass() {
-    return Float.class;
-  }
-
-  @Override
-  public Object getPrimitiveJavaObject(Object o) {
-    return null;
-  }
-
-  @Override
-  public Object copyObject(Object o) {
-    return null;
-  }
-
-  @Override
-  public boolean preferWritable() {
-    return false;
-  }
-
-  @Override
-  public String getTypeName() {
-    return "FLOAT";
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/objectinspector/TajoIntObjectInspector.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/objectinspector/TajoIntObjectInspector.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/objectinspector/TajoIntObjectInspector.java
deleted file mode 100644
index a0c2209..0000000
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/objectinspector/TajoIntObjectInspector.java
+++ /dev/null
@@ -1,76 +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.tajo.storage.orc.objectinspector;
-
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.IntObjectInspector;
-import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
-import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
-import org.apache.tajo.datum.Int4Datum;
-
-public class TajoIntObjectInspector extends TajoPrimitiveObjectInspector implements IntObjectInspector {
-  @Override
-  public int get(Object o) {
-    return ((Int4Datum)o).asInt4();
-  }
-
-  @Override
-  public PrimitiveTypeInfo getTypeInfo() {
-    return TypeInfoFactory.intTypeInfo;
-  }
-
-  @Override
-  public PrimitiveCategory getPrimitiveCategory() {
-    return PrimitiveCategory.INT;
-  }
-
-  @Override
-  public Class<?> getPrimitiveWritableClass() {
-    return null;
-  }
-
-  @Override
-  public Object getPrimitiveWritableObject(Object o) {
-    return null;
-  }
-
-  @Override
-  public Class<?> getJavaPrimitiveClass() {
-    return Integer.class;
-  }
-
-  @Override
-  public Object getPrimitiveJavaObject(Object o) {
-    return null;
-  }
-
-  @Override
-  public Object copyObject(Object o) {
-    return null;
-  }
-
-  @Override
-  public boolean preferWritable() {
-    return false;
-  }
-
-  @Override
-  public String getTypeName() {
-    return "INT";
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/objectinspector/TajoLongObjectInspector.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/objectinspector/TajoLongObjectInspector.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/objectinspector/TajoLongObjectInspector.java
deleted file mode 100644
index b30b333..0000000
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/objectinspector/TajoLongObjectInspector.java
+++ /dev/null
@@ -1,76 +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.tajo.storage.orc.objectinspector;
-
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.LongObjectInspector;
-import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
-import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
-import org.apache.tajo.datum.Int8Datum;
-
-public class TajoLongObjectInspector extends TajoPrimitiveObjectInspector implements LongObjectInspector {
-  @Override
-  public long get(Object o) {
-    return ((Int8Datum)o).asInt8();
-  }
-
-  @Override
-  public PrimitiveTypeInfo getTypeInfo() {
-    return TypeInfoFactory.shortTypeInfo;
-  }
-
-  @Override
-  public PrimitiveCategory getPrimitiveCategory() {
-    return PrimitiveCategory.LONG;
-  }
-
-  @Override
-  public Class<?> getPrimitiveWritableClass() {
-    return null;
-  }
-
-  @Override
-  public Object getPrimitiveWritableObject(Object o) {
-    return null;
-  }
-
-  @Override
-  public Class<?> getJavaPrimitiveClass() {
-    return Long.class;
-  }
-
-  @Override
-  public Object getPrimitiveJavaObject(Object o) {
-    return null;
-  }
-
-  @Override
-  public Object copyObject(Object o) {
-    return null;
-  }
-
-  @Override
-  public boolean preferWritable() {
-    return false;
-  }
-
-  @Override
-  public String getTypeName() {
-    return "LONG";
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/objectinspector/TajoNullObjectInspector.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/objectinspector/TajoNullObjectInspector.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/objectinspector/TajoNullObjectInspector.java
deleted file mode 100644
index 49998ce..0000000
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/objectinspector/TajoNullObjectInspector.java
+++ /dev/null
@@ -1,69 +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.tajo.storage.orc.objectinspector;
-
-import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
-import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
-
-public class TajoNullObjectInspector extends TajoPrimitiveObjectInspector {
-  @Override
-  public PrimitiveTypeInfo getTypeInfo() {
-    return TypeInfoFactory.voidTypeInfo;
-  }
-
-  @Override
-  public PrimitiveCategory getPrimitiveCategory() {
-    return PrimitiveCategory.VOID;
-  }
-
-  @Override
-  public Class<?> getPrimitiveWritableClass() {
-    return null;
-  }
-
-  @Override
-  public Object getPrimitiveWritableObject(Object o) {
-    return null;
-  }
-
-  @Override
-  public Class<?> getJavaPrimitiveClass() {
-    return Void.class;
-  }
-
-  @Override
-  public Object getPrimitiveJavaObject(Object o) {
-    return null;
-  }
-
-  @Override
-  public Object copyObject(Object o) {
-    return null;
-  }
-
-  @Override
-  public boolean preferWritable() {
-    return false;
-  }
-
-  @Override
-  public String getTypeName() {
-    return "NULL";
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/objectinspector/TajoPrimitiveObjectInspector.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/objectinspector/TajoPrimitiveObjectInspector.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/objectinspector/TajoPrimitiveObjectInspector.java
deleted file mode 100644
index 90ac178..0000000
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/objectinspector/TajoPrimitiveObjectInspector.java
+++ /dev/null
@@ -1,38 +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.tajo.storage.orc.objectinspector;
-
-import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
-
-public abstract class TajoPrimitiveObjectInspector implements PrimitiveObjectInspector {
-  @Override
-  public Category getCategory() {
-    return Category.PRIMITIVE;
-  }
-
-  @Override
-  public int precision() {
-    return 0;
-  }
-
-  @Override
-  public int scale() {
-    return 0;
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/objectinspector/TajoShortObjectInspector.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/objectinspector/TajoShortObjectInspector.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/objectinspector/TajoShortObjectInspector.java
deleted file mode 100644
index d32bee1..0000000
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/objectinspector/TajoShortObjectInspector.java
+++ /dev/null
@@ -1,76 +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.tajo.storage.orc.objectinspector;
-
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.ShortObjectInspector;
-import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
-import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
-import org.apache.tajo.datum.Int2Datum;
-
-public class TajoShortObjectInspector extends TajoPrimitiveObjectInspector implements ShortObjectInspector {
-  @Override
-  public short get(Object o) {
-    return ((Int2Datum)o).asInt2();
-  }
-
-  @Override
-  public PrimitiveTypeInfo getTypeInfo() {
-    return TypeInfoFactory.shortTypeInfo;
-  }
-
-  @Override
-  public PrimitiveCategory getPrimitiveCategory() {
-    return PrimitiveCategory.SHORT;
-  }
-
-  @Override
-  public Class<?> getPrimitiveWritableClass() {
-    return null;
-  }
-
-  @Override
-  public Object getPrimitiveWritableObject(Object o) {
-    return null;
-  }
-
-  @Override
-  public Class<?> getJavaPrimitiveClass() {
-    return Short.class;
-  }
-
-  @Override
-  public Object getPrimitiveJavaObject(Object o) {
-    return null;
-  }
-
-  @Override
-  public Object copyObject(Object o) {
-    return null;
-  }
-
-  @Override
-  public boolean preferWritable() {
-    return false;
-  }
-
-  @Override
-  public String getTypeName() {
-    return "SHORT";
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/objectinspector/TajoStringObjectInspector.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/objectinspector/TajoStringObjectInspector.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/objectinspector/TajoStringObjectInspector.java
deleted file mode 100644
index b9331da..0000000
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/objectinspector/TajoStringObjectInspector.java
+++ /dev/null
@@ -1,71 +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.tajo.storage.orc.objectinspector;
-
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.StringObjectInspector;
-import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
-import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
-import org.apache.hadoop.io.Text;
-
-public class TajoStringObjectInspector extends TajoPrimitiveObjectInspector implements StringObjectInspector {
-  @Override
-  public PrimitiveTypeInfo getTypeInfo() {
-    return TypeInfoFactory.stringTypeInfo;
-  }
-
-  @Override
-  public PrimitiveCategory getPrimitiveCategory() {
-    return PrimitiveCategory.STRING;
-  }
-
-  @Override
-  public Class<?> getPrimitiveWritableClass() {
-    return null;
-  }
-
-  @Override
-  public Text getPrimitiveWritableObject(Object o) {
-    return null;
-  }
-
-  @Override
-  public Class<?> getJavaPrimitiveClass() {
-    return null;
-  }
-
-  @Override
-  public String getPrimitiveJavaObject(Object o) {
-    return null;
-  }
-
-  @Override
-  public Object copyObject(Object o) {
-    return null;
-  }
-
-  @Override
-  public boolean preferWritable() {
-    return false;
-  }
-
-  @Override
-  public String getTypeName() {
-    return "STRING";
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/objectinspector/TajoStructObjectInspector.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/objectinspector/TajoStructObjectInspector.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/objectinspector/TajoStructObjectInspector.java
deleted file mode 100644
index 7521fa3..0000000
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/objectinspector/TajoStructObjectInspector.java
+++ /dev/null
@@ -1,122 +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.tajo.storage.orc.objectinspector;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.StructField;
-import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
-import org.apache.tajo.catalog.Column;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.exception.UnsupportedException;
-
-import java.util.ArrayList;
-import java.util.List;
-
-public class TajoStructObjectInspector extends StructObjectInspector {
-  private final static Log LOG = LogFactory.getLog(TajoStructObjectInspector.class);
-  private List<TajoStructField> structFields;
-
-  static class TajoStructField implements StructField {
-    private String name;
-    private ObjectInspector oi;
-    private String comment;
-
-    TajoStructField(String name, ObjectInspector oi) {
-      this(name, oi, null);
-    }
-
-    TajoStructField(String name, ObjectInspector oi, String comment) {
-      this.name = name;
-      this.oi = oi;
-      this.comment = comment;
-    }
-
-    @Override
-    public String getFieldName() {
-      return name;
-    }
-
-    @Override
-    public ObjectInspector getFieldObjectInspector() {
-      return oi;
-    }
-
-    @Override
-    public int getFieldID() {
-      return 0;
-    }
-
-    @Override
-    public String getFieldComment() {
-      return comment;
-    }
-  }
-
-  TajoStructObjectInspector(Schema schema) {
-    structFields = new ArrayList<>(schema.size());
-
-    for (Column c: schema.getRootColumns()) {
-      try {
-        TajoStructField field = new TajoStructField(c.getSimpleName(),
-          ObjectInspectorFactory.buildObjectInspectorByType(c.getDataType().getType()));
-        structFields.add(field);
-      } catch (UnsupportedException e) {
-        LOG.error(e.getMessage());
-      }
-    }
-  }
-
-  @Override
-  public List<? extends StructField> getAllStructFieldRefs() {
-    return structFields;
-  }
-
-  @Override
-  public StructField getStructFieldRef(String s) {
-    for (TajoStructField field:structFields) {
-      if (field.getFieldName().equals(s)) {
-        return field;
-      }
-    }
-
-    return null;
-  }
-
-  @Override
-  public Object getStructFieldData(Object o, StructField structField) {
-    return null;
-  }
-
-  @Override
-  public List<Object> getStructFieldsDataAsList(Object o) {
-    return null;
-  }
-
-  @Override
-  public String getTypeName() {
-    return "STRUCT";
-  }
-
-  @Override
-  public Category getCategory() {
-    return Category.STRUCT;
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/objectinspector/TajoTimestampObjectInspector.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/objectinspector/TajoTimestampObjectInspector.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/objectinspector/TajoTimestampObjectInspector.java
deleted file mode 100644
index bb887e7..0000000
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/objectinspector/TajoTimestampObjectInspector.java
+++ /dev/null
@@ -1,73 +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.tajo.storage.orc.objectinspector;
-
-import org.apache.hadoop.hive.serde2.io.TimestampWritable;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.TimestampObjectInspector;
-import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
-import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
-
-import java.sql.Timestamp;
-
-public class TajoTimestampObjectInspector extends TajoPrimitiveObjectInspector implements TimestampObjectInspector {
-  @Override
-  public PrimitiveTypeInfo getTypeInfo() {
-    return TypeInfoFactory.timestampTypeInfo;
-  }
-
-  @Override
-  public PrimitiveCategory getPrimitiveCategory() {
-    return PrimitiveCategory.TIMESTAMP;
-  }
-
-  @Override
-  public Class<?> getPrimitiveWritableClass() {
-    return null;
-  }
-
-  @Override
-  public TimestampWritable getPrimitiveWritableObject(Object o) {
-    return null;
-  }
-
-  @Override
-  public Class<?> getJavaPrimitiveClass() {
-    return null;
-  }
-
-  @Override
-  public Timestamp getPrimitiveJavaObject(Object o) {
-    return null;
-  }
-
-  @Override
-  public Object copyObject(Object o) {
-    return null;
-  }
-
-  @Override
-  public boolean preferWritable() {
-    return false;
-  }
-
-  @Override
-  public String getTypeName() {
-    return "TIMESTAMP";
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/BinaryColumnStatistics.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/BinaryColumnStatistics.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/BinaryColumnStatistics.java
deleted file mode 100644
index bee29fb..0000000
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/BinaryColumnStatistics.java
+++ /dev/null
@@ -1,25 +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.tajo.storage.thirdparty.orc;
-
-/**
- * Statistics for binary columns.
- */
-public interface BinaryColumnStatistics extends ColumnStatistics {
-  long getSum();
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/BitFieldWriter.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/BitFieldWriter.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/BitFieldWriter.java
deleted file mode 100644
index 23719bd..0000000
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/BitFieldWriter.java
+++ /dev/null
@@ -1,69 +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.tajo.storage.thirdparty.orc;
-
-import java.io.IOException;
-
-class BitFieldWriter {
-  private RunLengthByteWriter output;
-  private final int bitSize;
-  private byte current = 0;
-  private int bitsLeft = 8;
-
-  BitFieldWriter(PositionedOutputStream output,
-                 int bitSize) throws IOException {
-    this.output = new RunLengthByteWriter(output);
-    this.bitSize = bitSize;
-  }
-
-  private void writeByte() throws IOException {
-    output.write(current);
-    current = 0;
-    bitsLeft = 8;
-  }
-
-  void flush() throws IOException {
-    if (bitsLeft != 8) {
-      writeByte();
-    }
-    output.flush();
-  }
-
-  void write(int value) throws IOException {
-    int bitsToWrite = bitSize;
-    while (bitsToWrite > bitsLeft) {
-      // add the bits to the bottom of the current word
-      current |= value >>> (bitsToWrite - bitsLeft);
-      // subtract out the bits we just added
-      bitsToWrite -= bitsLeft;
-      // zero out the bits above bitsToWrite
-      value &= (1 << bitsToWrite) - 1;
-      writeByte();
-    }
-    bitsLeft -= bitsToWrite;
-    current |= value << bitsLeft;
-    if (bitsLeft == 0) {
-      writeByte();
-    }
-  }
-
-  void getPosition(PositionRecorder recorder) throws IOException {
-    output.getPosition(recorder);
-    recorder.addPosition(8 - bitsLeft);
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/BloomFilterIO.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/BloomFilterIO.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/BloomFilterIO.java
deleted file mode 100644
index 9d7c09c..0000000
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/BloomFilterIO.java
+++ /dev/null
@@ -1,42 +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.tajo.storage.thirdparty.orc;
-
-import com.google.common.primitives.Longs;
-import org.apache.tajo.storage.thirdparty.orc.util.BloomFilter;
-
-public class BloomFilterIO extends BloomFilter {
-
-  public BloomFilterIO(long expectedEntries) {
-    super(expectedEntries, DEFAULT_FPP);
-  }
-
-  public BloomFilterIO(long expectedEntries, double fpp) {
-    super(expectedEntries, fpp);
-  }
-
-/**
- * Initializes the BloomFilter from the given Orc BloomFilter
- */
-  public BloomFilterIO(OrcProto.BloomFilter bloomFilter) {
-    this.bitSet = new BitSet(Longs.toArray(bloomFilter.getBitsetList()));
-    this.numHashFunctions = bloomFilter.getNumHashFunctions();
-    this.numBits = (int) this.bitSet.bitSize();
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/BooleanColumnStatistics.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/BooleanColumnStatistics.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/BooleanColumnStatistics.java
deleted file mode 100644
index 0f55697..0000000
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/BooleanColumnStatistics.java
+++ /dev/null
@@ -1,27 +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.tajo.storage.thirdparty.orc;
-
-/**
- * Statistics for boolean columns.
- */
-public interface BooleanColumnStatistics extends ColumnStatistics {
-  long getFalseCount();
-
-  long getTrueCount();
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/ByteBufferAllocatorPool.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/ByteBufferAllocatorPool.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/ByteBufferAllocatorPool.java
new file mode 100644
index 0000000..de60bb2
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/ByteBufferAllocatorPool.java
@@ -0,0 +1,102 @@
+/*
+ * 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.tajo.storage.thirdparty.orc;
+
+import com.google.common.collect.ComparisonChain;
+import org.apache.commons.lang.builder.HashCodeBuilder;
+
+import java.nio.ByteBuffer;
+import java.util.Map;
+import java.util.TreeMap;
+
+public class ByteBufferAllocatorPool {
+  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();
+  }
+
+  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();
+  }
+
+  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/tajo/blob/68263585/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/ByteBufferPoolAdapter.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/ByteBufferPoolAdapter.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/ByteBufferPoolAdapter.java
new file mode 100644
index 0000000..2e9aec1
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/ByteBufferPoolAdapter.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.storage.thirdparty.orc;
+
+import org.apache.hadoop.io.ByteBufferPool;
+
+import java.nio.ByteBuffer;
+
+public class ByteBufferPoolAdapter implements ByteBufferPool {
+  private ByteBufferAllocatorPool pool;
+
+  public ByteBufferPoolAdapter(ByteBufferAllocatorPool 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);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/ColumnStatistics.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/ColumnStatistics.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/ColumnStatistics.java
deleted file mode 100644
index b317e41..0000000
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/ColumnStatistics.java
+++ /dev/null
@@ -1,36 +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.tajo.storage.thirdparty.orc;
-
-/**
- * Statistics that are available for all types of columns.
- */
-public interface ColumnStatistics {
-  /**
-   * Get the number of values in this column. It will differ from the number
-   * of rows because of NULL values and repeated values.
-   * @return the number of values
-   */
-  long getNumberOfValues();
-
-  /**
-   * Returns true if there are nulls in the scope of column statistics.
-   * @return true if null present else false
-   */
-  boolean hasNull();
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/ColumnStatisticsImpl.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/ColumnStatisticsImpl.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/ColumnStatisticsImpl.java
deleted file mode 100644
index d74f989..0000000
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/ColumnStatisticsImpl.java
+++ /dev/null
@@ -1,1017 +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.tajo.storage.thirdparty.orc;
-
-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.PrimitiveObjectInspector;
-import org.apache.tajo.datum.Datum;
-
-import java.sql.Date;
-import java.sql.Timestamp;
-
-class ColumnStatisticsImpl implements ColumnStatistics {
-
-  private static final class BooleanStatisticsImpl extends ColumnStatisticsImpl
-      implements BooleanColumnStatistics {
-    private long trueCount = 0;
-
-    BooleanStatisticsImpl(OrcProto.ColumnStatistics stats) {
-      super(stats);
-      OrcProto.BucketStatistics bkt = stats.getBucketStatistics();
-      trueCount = bkt.getCount(0);
-    }
-
-    BooleanStatisticsImpl() {
-    }
-
-    @Override
-    void reset() {
-      super.reset();
-      trueCount = 0;
-    }
-
-    @Override
-    void updateBoolean(boolean value) {
-      if (value) {
-        trueCount += 1;
-      }
-    }
-
-    @Override
-    void merge(ColumnStatisticsImpl other) {
-      if (other instanceof BooleanStatisticsImpl) {
-        BooleanStatisticsImpl bkt = (BooleanStatisticsImpl) other;
-        trueCount += bkt.trueCount;
-      } else {
-        if (isStatsExists() && trueCount != 0) {
-          throw new IllegalArgumentException("Incompatible merging of boolean column statistics");
-        }
-      }
-      super.merge(other);
-    }
-
-    @Override
-    OrcProto.ColumnStatistics.Builder serialize() {
-      OrcProto.ColumnStatistics.Builder builder = super.serialize();
-      OrcProto.BucketStatistics.Builder bucket =
-        OrcProto.BucketStatistics.newBuilder();
-      bucket.addCount(trueCount);
-      builder.setBucketStatistics(bucket);
-      return builder;
-    }
-
-    @Override
-    public long getFalseCount() {
-      return getNumberOfValues() - trueCount;
-    }
-
-    @Override
-    public long getTrueCount() {
-      return trueCount;
-    }
-
-    @Override
-    public String toString() {
-      return super.toString() + " true: " + trueCount;
-    }
-  }
-
-  private static final class IntegerStatisticsImpl extends ColumnStatisticsImpl
-      implements IntegerColumnStatistics {
-
-    private long minimum = Long.MAX_VALUE;
-    private long maximum = Long.MIN_VALUE;
-    private long sum = 0;
-    private boolean hasMinimum = false;
-    private boolean overflow = false;
-
-    IntegerStatisticsImpl() {
-    }
-
-    IntegerStatisticsImpl(OrcProto.ColumnStatistics stats) {
-      super(stats);
-      OrcProto.IntegerStatistics intStat = stats.getIntStatistics();
-      if (intStat.hasMinimum()) {
-        hasMinimum = true;
-        minimum = intStat.getMinimum();
-      }
-      if (intStat.hasMaximum()) {
-        maximum = intStat.getMaximum();
-      }
-      if (intStat.hasSum()) {
-        sum = intStat.getSum();
-      } else {
-        overflow = true;
-      }
-    }
-
-    @Override
-    void reset() {
-      super.reset();
-      hasMinimum = false;
-      minimum = Long.MAX_VALUE;
-      maximum = Long.MIN_VALUE;
-      sum = 0;
-      overflow = false;
-    }
-
-    @Override
-    void updateInteger(long value) {
-      if (!hasMinimum) {
-        hasMinimum = true;
-        minimum = value;
-        maximum = value;
-      } else if (value < minimum) {
-        minimum = value;
-      } else if (value > maximum) {
-        maximum = value;
-      }
-      if (!overflow) {
-        boolean wasPositive = sum >= 0;
-        sum += value;
-        if ((value >= 0) == wasPositive) {
-          overflow = (sum >= 0) != wasPositive;
-        }
-      }
-    }
-
-    @Override
-    void merge(ColumnStatisticsImpl other) {
-      if (other instanceof IntegerStatisticsImpl) {
-        IntegerStatisticsImpl otherInt = (IntegerStatisticsImpl) other;
-        if (!hasMinimum) {
-          hasMinimum = otherInt.hasMinimum;
-          minimum = otherInt.minimum;
-          maximum = otherInt.maximum;
-        } else if (otherInt.hasMinimum) {
-          if (otherInt.minimum < minimum) {
-            minimum = otherInt.minimum;
-          }
-          if (otherInt.maximum > maximum) {
-            maximum = otherInt.maximum;
-          }
-        }
-
-        overflow |= otherInt.overflow;
-        if (!overflow) {
-          boolean wasPositive = sum >= 0;
-          sum += otherInt.sum;
-          if ((otherInt.sum >= 0) == wasPositive) {
-            overflow = (sum >= 0) != wasPositive;
-          }
-        }
-      } else {
-        if (isStatsExists() && hasMinimum) {
-          throw new IllegalArgumentException("Incompatible merging of integer column statistics");
-        }
-      }
-      super.merge(other);
-    }
-
-    @Override
-    OrcProto.ColumnStatistics.Builder serialize() {
-      OrcProto.ColumnStatistics.Builder builder = super.serialize();
-      OrcProto.IntegerStatistics.Builder intb =
-        OrcProto.IntegerStatistics.newBuilder();
-      if (hasMinimum) {
-        intb.setMinimum(minimum);
-        intb.setMaximum(maximum);
-      }
-      if (!overflow) {
-        intb.setSum(sum);
-      }
-      builder.setIntStatistics(intb);
-      return builder;
-    }
-
-    @Override
-    public long getMinimum() {
-      return minimum;
-    }
-
-    @Override
-    public long getMaximum() {
-      return maximum;
-    }
-
-    @Override
-    public boolean isSumDefined() {
-      return !overflow;
-    }
-
-    @Override
-    public long getSum() {
-      return sum;
-    }
-
-    @Override
-    public String toString() {
-      StringBuilder buf = new StringBuilder(super.toString());
-      if (hasMinimum) {
-        buf.append(" min: ");
-        buf.append(minimum);
-        buf.append(" max: ");
-        buf.append(maximum);
-      }
-      if (!overflow) {
-        buf.append(" sum: ");
-        buf.append(sum);
-      }
-      return buf.toString();
-    }
-  }
-
-  private static final class DoubleStatisticsImpl extends ColumnStatisticsImpl
-       implements DoubleColumnStatistics {
-    private boolean hasMinimum = false;
-    private double minimum = Double.MAX_VALUE;
-    private double maximum = Double.MIN_VALUE;
-    private double sum = 0;
-
-    DoubleStatisticsImpl() {
-    }
-
-    DoubleStatisticsImpl(OrcProto.ColumnStatistics stats) {
-      super(stats);
-      OrcProto.DoubleStatistics dbl = stats.getDoubleStatistics();
-      if (dbl.hasMinimum()) {
-        hasMinimum = true;
-        minimum = dbl.getMinimum();
-      }
-      if (dbl.hasMaximum()) {
-        maximum = dbl.getMaximum();
-      }
-      if (dbl.hasSum()) {
-        sum = dbl.getSum();
-      }
-    }
-
-    @Override
-    void reset() {
-      super.reset();
-      hasMinimum = false;
-      minimum = Double.MAX_VALUE;
-      maximum = Double.MIN_VALUE;
-      sum = 0;
-    }
-
-    @Override
-    void updateDouble(double value) {
-      if (!hasMinimum) {
-        hasMinimum = true;
-        minimum = value;
-        maximum = value;
-      } else if (value < minimum) {
-        minimum = value;
-      } else if (value > maximum) {
-        maximum = value;
-      }
-      sum += value;
-    }
-
-    @Override
-    void merge(ColumnStatisticsImpl other) {
-      if (other instanceof DoubleStatisticsImpl) {
-        DoubleStatisticsImpl dbl = (DoubleStatisticsImpl) other;
-        if (!hasMinimum) {
-          hasMinimum = dbl.hasMinimum;
-          minimum = dbl.minimum;
-          maximum = dbl.maximum;
-        } else if (dbl.hasMinimum) {
-          if (dbl.minimum < minimum) {
-            minimum = dbl.minimum;
-          }
-          if (dbl.maximum > maximum) {
-            maximum = dbl.maximum;
-          }
-        }
-        sum += dbl.sum;
-      } else {
-        if (isStatsExists() && hasMinimum) {
-          throw new IllegalArgumentException("Incompatible merging of double column statistics");
-        }
-      }
-      super.merge(other);
-    }
-
-    @Override
-    OrcProto.ColumnStatistics.Builder serialize() {
-      OrcProto.ColumnStatistics.Builder builder = super.serialize();
-      OrcProto.DoubleStatistics.Builder dbl =
-        OrcProto.DoubleStatistics.newBuilder();
-      if (hasMinimum) {
-        dbl.setMinimum(minimum);
-        dbl.setMaximum(maximum);
-      }
-      dbl.setSum(sum);
-      builder.setDoubleStatistics(dbl);
-      return builder;
-    }
-
-    @Override
-    public double getMinimum() {
-      return minimum;
-    }
-
-    @Override
-    public double getMaximum() {
-      return maximum;
-    }
-
-    @Override
-    public double getSum() {
-      return sum;
-    }
-
-    @Override
-    public String toString() {
-      StringBuilder buf = new StringBuilder(super.toString());
-      if (hasMinimum) {
-        buf.append(" min: ");
-        buf.append(minimum);
-        buf.append(" max: ");
-        buf.append(maximum);
-      }
-      buf.append(" sum: ");
-      buf.append(sum);
-      return buf.toString();
-    }
-  }
-
-  protected static final class StringStatisticsImpl extends ColumnStatisticsImpl
-      implements StringColumnStatistics {
-    private String minimum = null;
-    private String maximum = null;
-    private long sum = 0;
-
-    StringStatisticsImpl() {
-    }
-
-    StringStatisticsImpl(OrcProto.ColumnStatistics stats) {
-      super(stats);
-      OrcProto.StringStatistics str = stats.getStringStatistics();
-      if (str.hasMaximum()) {
-        maximum = str.getMaximum();
-      }
-      if (str.hasMinimum()) {
-        minimum = str.getMinimum();
-      }
-      if(str.hasSum()) {
-        sum = str.getSum();
-      }
-    }
-
-    @Override
-    void reset() {
-      super.reset();
-      minimum = null;
-      maximum = null;
-      sum = 0;
-    }
-
-    @Override
-    void updateString(String value) {
-      if (minimum == null) {
-        maximum = minimum = value;
-      } else if (minimum.compareTo(value) > 0) {
-        minimum = value;
-      } else if (maximum.compareTo(value) < 0) {
-        maximum = value;
-      }
-      sum += value.length();
-    }
-
-    @Override
-    void merge(ColumnStatisticsImpl other) {
-      if (other instanceof StringStatisticsImpl) {
-        StringStatisticsImpl str = (StringStatisticsImpl) other;
-        if (minimum == null) {
-          if (str.minimum != null) {
-            maximum = str.getMaximum();
-            minimum = str.getMinimum();
-          } else {
-          /* both are empty */
-            maximum = minimum = null;
-          }
-        } else if (str.minimum != null) {
-          if (minimum.compareTo(str.minimum) > 0) {
-            minimum = str.getMinimum();
-          }
-          if (maximum.compareTo(str.maximum) < 0) {
-            maximum = str.getMaximum();
-          }
-        }
-        sum += str.sum;
-      } else {
-        if (isStatsExists() && minimum != null) {
-          throw new IllegalArgumentException("Incompatible merging of string column statistics");
-        }
-      }
-      super.merge(other);
-    }
-
-    @Override
-    OrcProto.ColumnStatistics.Builder serialize() {
-      OrcProto.ColumnStatistics.Builder result = super.serialize();
-      OrcProto.StringStatistics.Builder str =
-        OrcProto.StringStatistics.newBuilder();
-      if (getNumberOfValues() != 0) {
-        str.setMinimum(getMinimum());
-        str.setMaximum(getMaximum());
-        str.setSum(sum);
-      }
-      result.setStringStatistics(str);
-      return result;
-    }
-
-    @Override
-    public String getMinimum() {
-      return minimum;
-    }
-
-    @Override
-    public String getMaximum() {
-      return maximum;
-    }
-
-    @Override
-    public long getSum() {
-      return sum;
-    }
-
-    @Override
-    public String toString() {
-      StringBuilder buf = new StringBuilder(super.toString());
-      if (getNumberOfValues() != 0) {
-        buf.append(" min: ");
-        buf.append(getMinimum());
-        buf.append(" max: ");
-        buf.append(getMaximum());
-        buf.append(" sum: ");
-        buf.append(sum);
-      }
-      return buf.toString();
-    }
-  }
-
-  protected static final class BinaryStatisticsImpl extends ColumnStatisticsImpl implements
-    BinaryColumnStatistics {
-
-    private long sum = 0;
-
-    BinaryStatisticsImpl() {
-    }
-
-    BinaryStatisticsImpl(OrcProto.ColumnStatistics stats) {
-      super(stats);
-      OrcProto.BinaryStatistics binStats = stats.getBinaryStatistics();
-      if (binStats.hasSum()) {
-        sum = binStats.getSum();
-      }
-    }
-
-    @Override
-    void reset() {
-      super.reset();
-      sum = 0;
-    }
-
-    @Override
-    void updateBinary(Datum value) {
-      sum += value.size();
-    }
-
-    @Override
-    void merge(ColumnStatisticsImpl other) {
-      if (other instanceof BinaryColumnStatistics) {
-        BinaryStatisticsImpl bin = (BinaryStatisticsImpl) other;
-        sum += bin.sum;
-      } else {
-        if (isStatsExists() && sum != 0) {
-          throw new IllegalArgumentException("Incompatible merging of binary column statistics");
-        }
-      }
-      super.merge(other);
-    }
-
-    @Override
-    public long getSum() {
-      return sum;
-    }
-
-    @Override
-    OrcProto.ColumnStatistics.Builder serialize() {
-      OrcProto.ColumnStatistics.Builder result = super.serialize();
-      OrcProto.BinaryStatistics.Builder bin = OrcProto.BinaryStatistics.newBuilder();
-      bin.setSum(sum);
-      result.setBinaryStatistics(bin);
-      return result;
-    }
-
-    @Override
-    public String toString() {
-      StringBuilder buf = new StringBuilder(super.toString());
-      if (getNumberOfValues() != 0) {
-        buf.append(" sum: ");
-        buf.append(sum);
-      }
-      return buf.toString();
-    }
-  }
-
-  private static final class DecimalStatisticsImpl extends ColumnStatisticsImpl
-      implements DecimalColumnStatistics {
-    private HiveDecimal minimum = null;
-    private HiveDecimal maximum = null;
-    private HiveDecimal sum = HiveDecimal.ZERO;
-
-    DecimalStatisticsImpl() {
-    }
-
-    DecimalStatisticsImpl(OrcProto.ColumnStatistics stats) {
-      super(stats);
-      OrcProto.DecimalStatistics dec = stats.getDecimalStatistics();
-      if (dec.hasMaximum()) {
-        maximum = HiveDecimal.create(dec.getMaximum());
-      }
-      if (dec.hasMinimum()) {
-        minimum = HiveDecimal.create(dec.getMinimum());
-      }
-      if (dec.hasSum()) {
-        sum = HiveDecimal.create(dec.getSum());
-      } else {
-        sum = null;
-      }
-    }
-
-    @Override
-    void reset() {
-      super.reset();
-      minimum = null;
-      maximum = null;
-      sum = HiveDecimal.ZERO;
-    }
-
-    @Override
-    void updateDecimal(HiveDecimal value) {
-      if (minimum == null) {
-        minimum = value;
-        maximum = value;
-      } else if (minimum.compareTo(value) > 0) {
-        minimum = value;
-      } else if (maximum.compareTo(value) < 0) {
-        maximum = value;
-      }
-      if (sum != null) {
-        sum = sum.add(value);
-      }
-    }
-
-    @Override
-    void merge(ColumnStatisticsImpl other) {
-      if (other instanceof DecimalStatisticsImpl) {
-        DecimalStatisticsImpl dec = (DecimalStatisticsImpl) other;
-        if (minimum == null) {
-          minimum = dec.minimum;
-          maximum = dec.maximum;
-          sum = dec.sum;
-        } else if (dec.minimum != null) {
-          if (minimum.compareTo(dec.minimum) > 0) {
-            minimum = dec.minimum;
-          }
-          if (maximum.compareTo(dec.maximum) < 0) {
-            maximum = dec.maximum;
-          }
-          if (sum == null || dec.sum == null) {
-            sum = null;
-          } else {
-            sum = sum.add(dec.sum);
-          }
-        }
-      } else {
-        if (isStatsExists() && minimum != null) {
-          throw new IllegalArgumentException("Incompatible merging of decimal column statistics");
-        }
-      }
-      super.merge(other);
-    }
-
-    @Override
-    OrcProto.ColumnStatistics.Builder serialize() {
-      OrcProto.ColumnStatistics.Builder result = super.serialize();
-      OrcProto.DecimalStatistics.Builder dec =
-          OrcProto.DecimalStatistics.newBuilder();
-      if (getNumberOfValues() != 0 && minimum != null) {
-        dec.setMinimum(minimum.toString());
-        dec.setMaximum(maximum.toString());
-      }
-      if (sum != null) {
-        dec.setSum(sum.toString());
-      }
-      result.setDecimalStatistics(dec);
-      return result;
-    }
-
-    @Override
-    public HiveDecimal getMinimum() {
-      return minimum;
-    }
-
-    @Override
-    public HiveDecimal getMaximum() {
-      return maximum;
-    }
-
-    @Override
-    public HiveDecimal getSum() {
-      return sum;
-    }
-
-    @Override
-    public String toString() {
-      StringBuilder buf = new StringBuilder(super.toString());
-      if (getNumberOfValues() != 0) {
-        buf.append(" min: ");
-        buf.append(minimum);
-        buf.append(" max: ");
-        buf.append(maximum);
-        if (sum != null) {
-          buf.append(" sum: ");
-          buf.append(sum);
-        }
-      }
-      return buf.toString();
-    }
-  }
-
-  private static final class DateStatisticsImpl extends ColumnStatisticsImpl
-      implements DateColumnStatistics {
-    private Integer minimum = null;
-    private Integer maximum = null;
-
-    DateStatisticsImpl() {
-    }
-
-    DateStatisticsImpl(OrcProto.ColumnStatistics stats) {
-      super(stats);
-      OrcProto.DateStatistics dateStats = stats.getDateStatistics();
-      // min,max values serialized/deserialized as int (days since epoch)
-      if (dateStats.hasMaximum()) {
-        maximum = dateStats.getMaximum();
-      }
-      if (dateStats.hasMinimum()) {
-        minimum = dateStats.getMinimum();
-      }
-    }
-
-    @Override
-    void reset() {
-      super.reset();
-      minimum = null;
-      maximum = null;
-    }
-
-    @Override
-    void updateDate(int daysSinceEpoch) {
-      if (minimum == null) {
-        minimum = daysSinceEpoch;
-        maximum = daysSinceEpoch;
-      } else if (minimum > daysSinceEpoch) {
-        minimum = daysSinceEpoch;
-      } else if (maximum < daysSinceEpoch) {
-        maximum = daysSinceEpoch;
-      }
-    }
-
-    @Override
-    void merge(ColumnStatisticsImpl other) {
-      if (other instanceof DateStatisticsImpl) {
-        DateStatisticsImpl dateStats = (DateStatisticsImpl) other;
-        if (minimum == null) {
-          minimum = dateStats.minimum;
-          maximum = dateStats.maximum;
-        } else if (dateStats.minimum != null) {
-          if (minimum > dateStats.minimum) {
-            minimum = dateStats.minimum;
-          }
-          if (maximum < dateStats.maximum) {
-            maximum = dateStats.maximum;
-          }
-        }
-      } else {
-        if (isStatsExists() && minimum != null) {
-          throw new IllegalArgumentException("Incompatible merging of date column statistics");
-        }
-      }
-      super.merge(other);
-    }
-
-    @Override
-    OrcProto.ColumnStatistics.Builder serialize() {
-      OrcProto.ColumnStatistics.Builder result = super.serialize();
-      OrcProto.DateStatistics.Builder dateStats =
-          OrcProto.DateStatistics.newBuilder();
-      if (getNumberOfValues() != 0 && minimum != null) {
-        dateStats.setMinimum(minimum);
-        dateStats.setMaximum(maximum);
-      }
-      result.setDateStatistics(dateStats);
-      return result;
-    }
-
-    private transient final DateWritable minDate = new DateWritable();
-    private transient final DateWritable maxDate = new DateWritable();
-
-    @Override
-    public Date getMinimum() {
-      if (minimum == null) {
-        return null;
-      }
-      minDate.set(minimum);
-      return minDate.get();
-    }
-
-    @Override
-    public Date getMaximum() {
-      if (maximum == null) {
-        return null;
-      }
-      maxDate.set(maximum);
-      return maxDate.get();
-    }
-
-    @Override
-    public String toString() {
-      StringBuilder buf = new StringBuilder(super.toString());
-      if (getNumberOfValues() != 0) {
-        buf.append(" min: ");
-        buf.append(getMinimum());
-        buf.append(" max: ");
-        buf.append(getMaximum());
-      }
-      return buf.toString();
-    }
-  }
-
-  private static final class TimestampStatisticsImpl extends ColumnStatisticsImpl
-      implements TimestampColumnStatistics {
-    private Long minimum = null;
-    private Long maximum = null;
-
-    TimestampStatisticsImpl() {
-    }
-
-    TimestampStatisticsImpl(OrcProto.ColumnStatistics stats) {
-      super(stats);
-      OrcProto.TimestampStatistics timestampStats = stats.getTimestampStatistics();
-      // min,max values serialized/deserialized as int (milliseconds since epoch)
-      if (timestampStats.hasMaximum()) {
-        maximum = timestampStats.getMaximum();
-      }
-      if (timestampStats.hasMinimum()) {
-        minimum = timestampStats.getMinimum();
-      }
-    }
-
-    @Override
-    void reset() {
-      super.reset();
-      minimum = null;
-      maximum = null;
-    }
-
-    @Override
-    void updateTimestamp(Timestamp value) {
-      if (minimum == null) {
-        minimum = value.getTime();
-        maximum = value.getTime();
-      } else if (minimum > value.getTime()) {
-        minimum = value.getTime();
-      } else if (maximum < value.getTime()) {
-        maximum = value.getTime();
-      }
-    }
-
-    @Override
-    void merge(ColumnStatisticsImpl other) {
-      if (other instanceof TimestampStatisticsImpl) {
-        TimestampStatisticsImpl timestampStats = (TimestampStatisticsImpl) other;
-        if (minimum == null) {
-          minimum = timestampStats.minimum;
-          maximum = timestampStats.maximum;
-        } else if (timestampStats.minimum != null) {
-          if (minimum > timestampStats.minimum) {
-            minimum = timestampStats.minimum;
-          }
-          if (maximum < timestampStats.maximum) {
-            maximum = timestampStats.maximum;
-          }
-        }
-      } else {
-        if (isStatsExists() && minimum != null) {
-          throw new IllegalArgumentException("Incompatible merging of timestamp column statistics");
-        }
-      }
-      super.merge(other);
-    }
-
-    @Override
-    OrcProto.ColumnStatistics.Builder serialize() {
-      OrcProto.ColumnStatistics.Builder result = super.serialize();
-      OrcProto.TimestampStatistics.Builder timestampStats = OrcProto.TimestampStatistics
-          .newBuilder();
-      if (getNumberOfValues() != 0 && minimum != null) {
-        timestampStats.setMinimum(minimum);
-        timestampStats.setMaximum(maximum);
-      }
-      result.setTimestampStatistics(timestampStats);
-      return result;
-    }
-
-    @Override
-    public Timestamp getMinimum() {
-      return minimum == null ? null : new Timestamp(minimum);
-    }
-
-    @Override
-    public Timestamp getMaximum() {
-      return maximum == null ? null : new Timestamp(maximum);
-    }
-
-    @Override
-    public String toString() {
-      StringBuilder buf = new StringBuilder(super.toString());
-      if (getNumberOfValues() != 0) {
-        buf.append(" min: ");
-        buf.append(getMinimum());
-        buf.append(" max: ");
-        buf.append(getMaximum());
-      }
-      return buf.toString();
-    }
-  }
-
-  private long count = 0;
-  private boolean hasNull = false;
-
-  ColumnStatisticsImpl(OrcProto.ColumnStatistics stats) {
-    if (stats.hasNumberOfValues()) {
-      count = stats.getNumberOfValues();
-    }
-
-    hasNull = !stats.hasHasNull() || stats.getHasNull();
-  }
-
-  ColumnStatisticsImpl() {
-  }
-
-  void increment() {
-    count += 1;
-  }
-
-  void setNull() {
-    hasNull = true;
-  }
-
-  void updateBoolean(boolean value) {
-    throw new UnsupportedOperationException("Can't update boolean");
-  }
-
-  void updateInteger(long value) {
-    throw new UnsupportedOperationException("Can't update integer");
-  }
-
-  void updateDouble(double value) {
-    throw new UnsupportedOperationException("Can't update double");
-  }
-
-  void updateString(String value) {
-    throw new UnsupportedOperationException("Can't update string");
-  }
-
-  void updateBinary(Datum value) {
-    throw new UnsupportedOperationException("Can't update binary");
-  }
-
-  void updateDecimal(HiveDecimal value) {
-    throw new UnsupportedOperationException("Can't update decimal");
-  }
-
-  void updateDate(int days) {
-    throw new UnsupportedOperationException("Can't update date");
-  }
-
-  void updateTimestamp(Timestamp value) {
-    throw new UnsupportedOperationException("Can't update timestamp");
-  }
-
-  boolean isStatsExists() {
-    return (count > 0 || hasNull);
-  }
-
-  void merge(ColumnStatisticsImpl stats) {
-    count += stats.count;
-    hasNull |= stats.hasNull;
-  }
-
-  void reset() {
-    count = 0;
-    hasNull = false;
-  }
-
-  @Override
-  public long getNumberOfValues() {
-    return count;
-  }
-
-  @Override
-  public boolean hasNull() {
-    return hasNull;
-  }
-
-  @Override
-  public String toString() {
-    return "count: " + count + " hasNull: " + hasNull;
-  }
-
-  OrcProto.ColumnStatistics.Builder serialize() {
-    OrcProto.ColumnStatistics.Builder builder =
-      OrcProto.ColumnStatistics.newBuilder();
-    builder.setNumberOfValues(count);
-    builder.setHasNull(hasNull);
-    return builder;
-  }
-
-  static ColumnStatisticsImpl create(ObjectInspector inspector) {
-    switch (inspector.getCategory()) {
-      case PRIMITIVE:
-        switch (((PrimitiveObjectInspector) inspector).getPrimitiveCategory()) {
-          case BOOLEAN:
-            return new BooleanStatisticsImpl();
-          case BYTE:
-          case SHORT:
-          case INT:
-          case LONG:
-            return new IntegerStatisticsImpl();
-          case FLOAT:
-          case DOUBLE:
-            return new DoubleStatisticsImpl();
-          case STRING:
-          case CHAR:
-          case VARCHAR:
-            return new StringStatisticsImpl();
-          case DECIMAL:
-            return new DecimalStatisticsImpl();
-          case DATE:
-            return new DateStatisticsImpl();
-          case TIMESTAMP:
-            return new TimestampStatisticsImpl();
-          case BINARY:
-            return new BinaryStatisticsImpl();
-          default:
-            return new ColumnStatisticsImpl();
-        }
-      default:
-        return new ColumnStatisticsImpl();
-    }
-  }
-
-  static ColumnStatisticsImpl deserialize(OrcProto.ColumnStatistics stats) {
-    if (stats.hasBucketStatistics()) {
-      return new BooleanStatisticsImpl(stats);
-    } else if (stats.hasIntStatistics()) {
-      return new IntegerStatisticsImpl(stats);
-    } else if (stats.hasDoubleStatistics()) {
-      return new DoubleStatisticsImpl(stats);
-    } else if (stats.hasStringStatistics()) {
-      return new StringStatisticsImpl(stats);
-    } else if (stats.hasDecimalStatistics()) {
-      return new DecimalStatisticsImpl(stats);
-    } else if (stats.hasDateStatistics()) {
-      return new DateStatisticsImpl(stats);
-    } else if (stats.hasTimestampStatistics()) {
-      return new TimestampStatisticsImpl(stats);
-    } else if(stats.hasBinaryStatistics()) {
-      return new BinaryStatisticsImpl(stats);
-    } else {
-      return new ColumnStatisticsImpl(stats);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/CompressionCodec.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/CompressionCodec.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/CompressionCodec.java
deleted file mode 100644
index 769ca50..0000000
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/CompressionCodec.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.tajo.storage.thirdparty.orc;
-
-import javax.annotation.Nullable;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.EnumSet;
-
-public interface CompressionCodec {
-
-  public enum Modifier {
-    /* speed/compression tradeoffs */
-    FASTEST,
-    FAST,
-    DEFAULT,
-    /* data sensitivity modifiers */
-    TEXT,
-    BINARY
-  };
-
-  /**
-   * Compress the in buffer to the out buffer.
-   * @param in the bytes to compress
-   * @param out the uncompressed bytes
-   * @param overflow put any additional bytes here
-   * @return true if the output is smaller than input
-   * @throws IOException
-   */
-  boolean compress(ByteBuffer in, ByteBuffer out, ByteBuffer overflow
-  ) throws IOException;
-
-  /**
-   * Decompress the in buffer to the out buffer.
-   * @param in the bytes to decompress
-   * @param out the decompressed bytes
-   * @throws IOException
-   */
-  void decompress(ByteBuffer in, ByteBuffer out) throws IOException;
-
-  /**
-   * Produce a modified compression codec if the underlying algorithm allows
-   * modification.
-   *
-   * This does not modify the current object, but returns a new object if
-   * modifications are possible. Returns the same object if no modifications
-   * are possible.
-   * @param modifiers compression modifiers
-   * @return codec for use after optional modification
-   */
-  CompressionCodec modify(@Nullable EnumSet<Modifier> modifiers);
-
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/CompressionKind.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/CompressionKind.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/CompressionKind.java
deleted file mode 100644
index 8b16c67..0000000
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/CompressionKind.java
+++ /dev/null
@@ -1,27 +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.tajo.storage.thirdparty.orc;
-
-/**
- * An enumeration that lists the generic compression algorithms that
- * can be applied to ORC files.
- */
-public enum CompressionKind {
-  NONE, ZLIB, SNAPPY, LZO
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/DateColumnStatistics.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/DateColumnStatistics.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/DateColumnStatistics.java
deleted file mode 100644
index cb3405e..0000000
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/DateColumnStatistics.java
+++ /dev/null
@@ -1,37 +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.tajo.storage.thirdparty.orc;
-
-import java.util.Date;
-
-/**
- * Statistics for DATE columns.
- */
-public interface DateColumnStatistics extends ColumnStatistics {
-  /**
-   * Get the minimum value for the column.
-   * @return minimum value
-   */
-  Date getMinimum();
-
-  /**
-   * Get the maximum value for the column.
-   * @return maximum value
-   */
-  Date getMaximum();
-}


[4/7] tajo git commit: TAJO-2102: Migrate to Apache Orc from Presto's one.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/OutStream.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/OutStream.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/OutStream.java
deleted file mode 100644
index f6cfd57..0000000
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/OutStream.java
+++ /dev/null
@@ -1,286 +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.tajo.storage.thirdparty.orc;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-class OutStream extends PositionedOutputStream {
-
-  interface OutputReceiver {
-    /**
-     * Output the given buffer to the final destination
-     * @param buffer the buffer to output
-     * @throws IOException
-     */
-    void output(ByteBuffer buffer) throws IOException;
-  }
-
-  static final int HEADER_SIZE = 3;
-  private final String name;
-  private final OutputReceiver receiver;
-  // if enabled the stream will be suppressed when writing stripe
-  private boolean suppress;
-
-  /**
-   * Stores the uncompressed bytes that have been serialized, but not
-   * compressed yet. When this fills, we compress the entire buffer.
-   */
-  private ByteBuffer current = null;
-
-  /**
-   * Stores the compressed bytes until we have a full buffer and then outputs
-   * them to the receiver. If no compression is being done, this (and overflow)
-   * will always be null and the current buffer will be sent directly to the
-   * receiver.
-   */
-  private ByteBuffer compressed = null;
-
-  /**
-   * Since the compressed buffer may start with contents from previous
-   * compression blocks, we allocate an overflow buffer so that the
-   * output of the codec can be split between the two buffers. After the
-   * compressed buffer is sent to the receiver, the overflow buffer becomes
-   * the new compressed buffer.
-   */
-  private ByteBuffer overflow = null;
-  private final int bufferSize;
-  private final CompressionCodec codec;
-  private long compressedBytes = 0;
-  private long uncompressedBytes = 0;
-
-  OutStream(String name,
-            int bufferSize,
-            CompressionCodec codec,
-            OutputReceiver receiver) throws IOException {
-    this.name = name;
-    this.bufferSize = bufferSize;
-    this.codec = codec;
-    this.receiver = receiver;
-    this.suppress = false;
-  }
-
-  public void clear() throws IOException {
-    flush();
-    suppress = false;
-  }
-
-  /**
-   * Write the length of the compressed bytes. Life is much easier if the
-   * header is constant length, so just use 3 bytes. Considering most of the
-   * codecs want between 32k (snappy) and 256k (lzo, zlib), 3 bytes should
-   * be plenty. We also use the low bit for whether it is the original or
-   * compressed bytes.
-   * @param buffer the buffer to write the header to
-   * @param position the position in the buffer to write at
-   * @param val the size in the file
-   * @param original is it uncompressed
-   */
-  private static void writeHeader(ByteBuffer buffer,
-                                  int position,
-                                  int val,
-                                  boolean original) {
-    buffer.put(position, (byte) ((val << 1) + (original ? 1 : 0)));
-    buffer.put(position + 1, (byte) (val >> 7));
-    buffer.put(position + 2, (byte) (val >> 15));
-  }
-
-  private void getNewInputBuffer() throws IOException {
-    if (codec == null) {
-      current = ByteBuffer.allocate(bufferSize);
-    } else {
-      current = ByteBuffer.allocate(bufferSize + HEADER_SIZE);
-      writeHeader(current, 0, bufferSize, true);
-      current.position(HEADER_SIZE);
-    }
-  }
-
-  /**
-   * Allocate a new output buffer if we are compressing.
-   */
-  private ByteBuffer getNewOutputBuffer() throws IOException {
-    return ByteBuffer.allocate(bufferSize + HEADER_SIZE);
-  }
-
-  private void flip() throws IOException {
-    current.limit(current.position());
-    current.position(codec == null ? 0 : HEADER_SIZE);
-  }
-
-  @Override
-  public void write(int i) throws IOException {
-    if (current == null) {
-      getNewInputBuffer();
-    }
-    if (current.remaining() < 1) {
-      spill();
-    }
-    uncompressedBytes += 1;
-    current.put((byte) i);
-  }
-
-  @Override
-  public void write(byte[] bytes, int offset, int length) throws IOException {
-    if (current == null) {
-      getNewInputBuffer();
-    }
-    int remaining = Math.min(current.remaining(), length);
-    current.put(bytes, offset, remaining);
-    uncompressedBytes += remaining;
-    length -= remaining;
-    while (length != 0) {
-      spill();
-      offset += remaining;
-      remaining = Math.min(current.remaining(), length);
-      current.put(bytes, offset, remaining);
-      uncompressedBytes += remaining;
-      length -= remaining;
-    }
-  }
-
-  private void spill() throws IOException {
-    // if there isn't anything in the current buffer, don't spill
-    if (current == null ||
-        current.position() == (codec == null ? 0 : HEADER_SIZE)) {
-      return;
-    }
-    flip();
-    if (codec == null) {
-      receiver.output(current);
-      getNewInputBuffer();
-    } else {
-      if (compressed == null) {
-        compressed = getNewOutputBuffer();
-      } else if (overflow == null) {
-        overflow = getNewOutputBuffer();
-      }
-      int sizePosn = compressed.position();
-      compressed.position(compressed.position() + HEADER_SIZE);
-      if (codec.compress(current, compressed, overflow)) {
-        uncompressedBytes = 0;
-        // move position back to after the header
-        current.position(HEADER_SIZE);
-        current.limit(current.capacity());
-        // find the total bytes in the chunk
-        int totalBytes = compressed.position() - sizePosn - HEADER_SIZE;
-        if (overflow != null) {
-          totalBytes += overflow.position();
-        }
-        compressedBytes += totalBytes + HEADER_SIZE;
-        writeHeader(compressed, sizePosn, totalBytes, false);
-        // if we have less than the next header left, spill it.
-        if (compressed.remaining() < HEADER_SIZE) {
-          compressed.flip();
-          receiver.output(compressed);
-          compressed = overflow;
-          overflow = null;
-        }
-      } else {
-        compressedBytes += uncompressedBytes + HEADER_SIZE;
-        uncompressedBytes = 0;
-        // we are using the original, but need to spill the current
-        // compressed buffer first. So back up to where we started,
-        // flip it and add it to done.
-        if (sizePosn != 0) {
-          compressed.position(sizePosn);
-          compressed.flip();
-          receiver.output(compressed);
-          compressed = null;
-          // if we have an overflow, clear it and make it the new compress
-          // buffer
-          if (overflow != null) {
-            overflow.clear();
-            compressed = overflow;
-            overflow = null;
-          }
-        } else {
-          compressed.clear();
-          if (overflow != null) {
-            overflow.clear();
-          }
-        }
-
-        // now add the current buffer into the done list and get a new one.
-        current.position(0);
-        // update the header with the current length
-        writeHeader(current, 0, current.limit() - HEADER_SIZE, true);
-        receiver.output(current);
-        getNewInputBuffer();
-      }
-    }
-  }
-
-  void getPosition(PositionRecorder recorder) throws IOException {
-    if (codec == null) {
-      recorder.addPosition(uncompressedBytes);
-    } else {
-      recorder.addPosition(compressedBytes);
-      recorder.addPosition(uncompressedBytes);
-    }
-  }
-
-  @Override
-  public void flush() throws IOException {
-    spill();
-    if (compressed != null && compressed.position() != 0) {
-      compressed.flip();
-      receiver.output(compressed);
-      compressed = null;
-    }
-    uncompressedBytes = 0;
-    compressedBytes = 0;
-    overflow = null;
-    current = null;
-  }
-
-  @Override
-  public String toString() {
-    return name;
-  }
-
-  @Override
-  public long getBufferSize() {
-    long result = 0;
-    if (current != null) {
-      result += current.capacity();
-    }
-    if (compressed != null) {
-      result += compressed.capacity();
-    }
-    if (overflow != null) {
-      result += overflow.capacity();
-    }
-    return result;
-  }
-
-  /**
-   * Set suppress flag
-   */
-  public void suppress() {
-    suppress = true;
-  }
-
-  /**
-   * Returns the state of suppress flag
-   * @return value of suppress flag
-   */
-  public boolean isSuppressed() {
-    return suppress;
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/PositionRecorder.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/PositionRecorder.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/PositionRecorder.java
deleted file mode 100644
index a39926e..0000000
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/PositionRecorder.java
+++ /dev/null
@@ -1,25 +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.tajo.storage.thirdparty.orc;
-
-/**
- * An interface for recording positions in a stream.
- */
-interface PositionRecorder {
-  void addPosition(long offset);
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/PositionedOutputStream.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/PositionedOutputStream.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/PositionedOutputStream.java
deleted file mode 100644
index 748c98c..0000000
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/PositionedOutputStream.java
+++ /dev/null
@@ -1,38 +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.tajo.storage.thirdparty.orc;
-
-import java.io.IOException;
-import java.io.OutputStream;
-
-abstract class PositionedOutputStream extends OutputStream {
-
-  /**
-   * Record the current position to the recorder.
-   * @param recorder the object that receives the position
-   * @throws IOException
-   */
-  abstract void getPosition(PositionRecorder recorder) throws IOException;
-
-  /**
-   * Get the memory size currently allocated as buffer associated with this
-   * stream.
-   * @return the number of bytes used by buffers.
-   */
-  abstract long getBufferSize();
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/RecordReaderUtils.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/RecordReaderUtils.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/RecordReaderUtils.java
new file mode 100644
index 0000000..bc882e0
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/RecordReaderUtils.java
@@ -0,0 +1,393 @@
+/*
+ * 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.tajo.storage.thirdparty.orc;
+
+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.orc.CompressionCodec;
+import org.apache.orc.DataReader;
+import org.apache.orc.OrcProto;
+import org.apache.orc.impl.BufferChunk;
+import org.apache.orc.impl.DirectDecompressionCodec;
+import org.apache.orc.impl.OutStream;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+
+public class RecordReaderUtils {
+
+  public static class DefaultDataReader implements DataReader {
+    private FSDataInputStream file;
+    private ByteBufferAllocatorPool pool;
+    private ZeroCopyAdapter zcr;
+    private FileSystem fs;
+    private Path path;
+    private boolean useZeroCopy;
+    private CompressionCodec codec;
+    private long readBytes = 0;
+
+    public DefaultDataReader(
+        FileSystem fs, Path path, boolean useZeroCopy, CompressionCodec codec) {
+      this.fs = fs;
+      this.path = path;
+      this.useZeroCopy = useZeroCopy;
+      this.codec = codec;
+    }
+
+    @Override
+    public void open() throws IOException {
+      this.file = fs.open(path);
+      if (useZeroCopy) {
+        pool = new ByteBufferAllocatorPool();
+        zcr = RecordReaderUtils.createZeroCopyShim(file, codec, pool);
+      } else {
+        pool = null;
+        zcr = null;
+      }
+    }
+
+    @Override
+    public DiskRangeList readFileData(
+        DiskRangeList range, long baseOffset, boolean doForceDirect) throws IOException {
+      return 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);
+    }
+
+    public long getReadBytes() {
+      return readBytes;
+    }
+
+    /**
+     * 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
+     */
+    private DiskRangeList readDiskRanges(FSDataInputStream file,
+                                         ZeroCopyAdapter zcr,
+                                        long base,
+                                        DiskRangeList range,
+                                        boolean doForceDirect) throws IOException {
+      if (range == null) return null;
+      DiskRangeList prev = range.prev;
+      if (prev == null) {
+        prev = new DiskRangeList.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);
+            readBytes += partial.remaining();
+            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);
+          readBytes += 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;
+    }
+  }
+
+  public static DataReader createDefaultDataReader(
+      FileSystem fs, Path path, boolean useZeroCopy, CompressionCodec codec) {
+    return new DefaultDataReader(fs, path, useZeroCopy, codec);
+  }
+
+  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, DiskRangeList.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, DiskRangeList.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();
+  }
+
+  public 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 ZeroCopyAdapter 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 new ZeroCopyAdapter(file, pool);
+    }
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/RedBlackTree.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/RedBlackTree.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/RedBlackTree.java
deleted file mode 100644
index 2482f93..0000000
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/RedBlackTree.java
+++ /dev/null
@@ -1,309 +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.tajo.storage.thirdparty.orc;
-
-/**
- * A memory efficient red-black tree that does not allocate any objects per
- * an element. This class is abstract and assumes that the child class
- * handles the key and comparisons with the key.
- */
-abstract class RedBlackTree {
-  public static final int NULL = -1;
-
-  // Various values controlling the offset of the data within the array.
-  private static final int LEFT_OFFSET = 0;
-  private static final int RIGHT_OFFSET = 1;
-  private static final int ELEMENT_SIZE = 2;
-
-  protected int size = 0;
-  private final DynamicIntArray data;
-  protected int root = NULL;
-  protected int lastAdd = 0;
-  private boolean wasAdd = false;
-
-  /**
-   * Create a set with the given initial capacity.
-   */
-  public RedBlackTree(int initialCapacity) {
-    data = new DynamicIntArray(initialCapacity * ELEMENT_SIZE);
-  }
-
-  /**
-   * Insert a new node into the data array, growing the array as necessary.
-   *
-   * @return Returns the position of the new node.
-   */
-  private int insert(int left, int right, boolean isRed) {
-    int position = size;
-    size += 1;
-    setLeft(position, left, isRed);
-    setRight(position, right);
-    return position;
-  }
-
-  /**
-   * Compare the value at the given position to the new value.
-   * @return 0 if the values are the same, -1 if the new value is smaller and
-   *         1 if the new value is larger.
-   */
-  protected abstract int compareValue(int position);
-
-  /**
-   * Is the given node red as opposed to black? To prevent having an extra word
-   * in the data array, we just the low bit on the left child index.
-   */
-  protected boolean isRed(int position) {
-    return position != NULL &&
-        (data.get(position * ELEMENT_SIZE + LEFT_OFFSET) & 1) == 1;
-  }
-
-  /**
-   * Set the red bit true or false.
-   */
-  private void setRed(int position, boolean isRed) {
-    int offset = position * ELEMENT_SIZE + LEFT_OFFSET;
-    if (isRed) {
-      data.set(offset, data.get(offset) | 1);
-    } else {
-      data.set(offset, data.get(offset) & ~1);
-    }
-  }
-
-  /**
-   * Get the left field of the given position.
-   */
-  protected int getLeft(int position) {
-    return data.get(position * ELEMENT_SIZE + LEFT_OFFSET) >> 1;
-  }
-
-  /**
-   * Get the right field of the given position.
-   */
-  protected int getRight(int position) {
-    return data.get(position * ELEMENT_SIZE + RIGHT_OFFSET);
-  }
-
-  /**
-   * Set the left field of the given position.
-   * Note that we are storing the node color in the low bit of the left pointer.
-   */
-  private void setLeft(int position, int left) {
-    int offset = position * ELEMENT_SIZE + LEFT_OFFSET;
-    data.set(offset, (left << 1) | (data.get(offset) & 1));
-  }
-
-  /**
-   * Set the left field of the given position.
-   * Note that we are storing the node color in the low bit of the left pointer.
-   */
-  private void setLeft(int position, int left, boolean isRed) {
-    int offset = position * ELEMENT_SIZE + LEFT_OFFSET;
-    data.set(offset, (left << 1) | (isRed ? 1 : 0));
-  }
-
-  /**
-   * Set the right field of the given position.
-   */
-  private void setRight(int position, int right) {
-    data.set(position * ELEMENT_SIZE + RIGHT_OFFSET, right);
-  }
-
-  /**
-   * Insert or find a given key in the tree and rebalance the tree correctly.
-   * Rebalancing restores the red-black aspect of the tree to maintain the
-   * invariants:
-   *   1. If a node is red, both of its children are black.
-   *   2. Each child of a node has the same black height (the number of black
-   *      nodes between it and the leaves of the tree).
-   *
-   * Inserted nodes are at the leaves and are red, therefore there is at most a
-   * violation of rule 1 at the node we just put in. Instead of always keeping
-   * the parents, this routine passing down the context.
-   *
-   * The fix is broken down into 6 cases (1.{1,2,3} and 2.{1,2,3} that are
-   * left-right mirror images of each other). See Algorighms by Cormen,
-   * Leiserson, and Rivest for the explaination of the subcases.
-   *
-   * @param node The node that we are fixing right now.
-   * @param fromLeft Did we come down from the left?
-   * @param parent Nodes' parent
-   * @param grandparent Parent's parent
-   * @param greatGrandparent Grandparent's parent
-   * @return Does parent also need to be checked and/or fixed?
-   */
-  private boolean add(int node, boolean fromLeft, int parent,
-                      int grandparent, int greatGrandparent) {
-    if (node == NULL) {
-      if (root == NULL) {
-        lastAdd = insert(NULL, NULL, false);
-        root = lastAdd;
-        wasAdd = true;
-        return false;
-      } else {
-        lastAdd = insert(NULL, NULL, true);
-        node = lastAdd;
-        wasAdd = true;
-        // connect the new node into the tree
-        if (fromLeft) {
-          setLeft(parent, node);
-        } else {
-          setRight(parent, node);
-        }
-      }
-    } else {
-      int compare = compareValue(node);
-      boolean keepGoing;
-
-      // Recurse down to find where the node needs to be added
-      if (compare < 0) {
-        keepGoing = add(getLeft(node), true, node, parent, grandparent);
-      } else if (compare > 0) {
-        keepGoing = add(getRight(node), false, node, parent, grandparent);
-      } else {
-        lastAdd = node;
-        wasAdd = false;
-        return false;
-      }
-
-      // we don't need to fix the root (because it is always set to black)
-      if (node == root || !keepGoing) {
-        return false;
-      }
-    }
-
-
-    // Do we need to fix this node? Only if there are two reds right under each
-    // other.
-    if (isRed(node) && isRed(parent)) {
-      if (parent == getLeft(grandparent)) {
-        int uncle = getRight(grandparent);
-        if (isRed(uncle)) {
-          // case 1.1
-          setRed(parent, false);
-          setRed(uncle, false);
-          setRed(grandparent, true);
-          return true;
-        } else {
-          if (node == getRight(parent)) {
-            // case 1.2
-            // swap node and parent
-            int tmp = node;
-            node = parent;
-            parent = tmp;
-            // left-rotate on node
-            setLeft(grandparent, parent);
-            setRight(node, getLeft(parent));
-            setLeft(parent, node);
-          }
-
-          // case 1.2 and 1.3
-          setRed(parent, false);
-          setRed(grandparent, true);
-
-          // right-rotate on grandparent
-          if (greatGrandparent == NULL) {
-            root = parent;
-          } else if (getLeft(greatGrandparent) == grandparent) {
-            setLeft(greatGrandparent, parent);
-          } else {
-            setRight(greatGrandparent, parent);
-          }
-          setLeft(grandparent, getRight(parent));
-          setRight(parent, grandparent);
-          return false;
-        }
-      } else {
-        int uncle = getLeft(grandparent);
-        if (isRed(uncle)) {
-          // case 2.1
-          setRed(parent, false);
-          setRed(uncle, false);
-          setRed(grandparent, true);
-          return true;
-        } else {
-          if (node == getLeft(parent)) {
-            // case 2.2
-            // swap node and parent
-            int tmp = node;
-            node = parent;
-            parent = tmp;
-            // right-rotate on node
-            setRight(grandparent, parent);
-            setLeft(node, getRight(parent));
-            setRight(parent, node);
-          }
-          // case 2.2 and 2.3
-          setRed(parent, false);
-          setRed(grandparent, true);
-          // left-rotate on grandparent
-          if (greatGrandparent == NULL) {
-            root = parent;
-          } else if (getRight(greatGrandparent) == grandparent) {
-            setRight(greatGrandparent, parent);
-          } else {
-            setLeft(greatGrandparent, parent);
-          }
-          setRight(grandparent, getLeft(parent));
-          setLeft(parent, grandparent);
-          return false;
-        }
-      }
-    } else {
-      return true;
-    }
-  }
-
-  /**
-   * Add the new key to the tree.
-   * @return true if the element is a new one.
-   */
-  protected boolean add() {
-    add(root, false, NULL, NULL, NULL);
-    if (wasAdd) {
-      setRed(root, false);
-      return true;
-    } else {
-      return false;
-    }
-  }
-
-  /**
-   * Get the number of elements in the set.
-   */
-  public int size() {
-    return size;
-  }
-
-  /**
-   * Reset the table to empty.
-   */
-  public void clear() {
-    root = NULL;
-    size = 0;
-    data.clear();
-  }
-
-  /**
-   * Get the buffer size in bytes.
-   */
-  public long getSizeInBytes() {
-    return data.getSizeInBytes();
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/RunLengthByteWriter.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/RunLengthByteWriter.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/RunLengthByteWriter.java
deleted file mode 100644
index 0953cdd..0000000
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/RunLengthByteWriter.java
+++ /dev/null
@@ -1,106 +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.tajo.storage.thirdparty.orc;
-
-import java.io.IOException;
-
-/**
- * A streamFactory that writes a sequence of bytes. A control byte is written before
- * each run with positive values 0 to 127 meaning 2 to 129 repetitions. If the
- * bytes is -1 to -128, 1 to 128 literal byte values follow.
- */
-class RunLengthByteWriter {
-  static final int MIN_REPEAT_SIZE = 3;
-  static final int MAX_LITERAL_SIZE = 128;
-  static final int MAX_REPEAT_SIZE= 127 + MIN_REPEAT_SIZE;
-  private final PositionedOutputStream output;
-  private final byte[] literals = new byte[MAX_LITERAL_SIZE];
-  private int numLiterals = 0;
-  private boolean repeat = false;
-  private int tailRunLength = 0;
-
-  RunLengthByteWriter(PositionedOutputStream output) {
-    this.output = output;
-  }
-
-  private void writeValues() throws IOException {
-    if (numLiterals != 0) {
-      if (repeat) {
-        output.write(numLiterals - MIN_REPEAT_SIZE);
-        output.write(literals, 0, 1);
-     } else {
-        output.write(-numLiterals);
-        output.write(literals, 0, numLiterals);
-      }
-      repeat = false;
-      tailRunLength = 0;
-      numLiterals = 0;
-    }
-  }
-
-  void flush() throws IOException {
-    writeValues();
-    output.flush();
-  }
-
-  void write(byte value) throws IOException {
-    if (numLiterals == 0) {
-      literals[numLiterals++] = value;
-      tailRunLength = 1;
-    } else if (repeat) {
-      if (value == literals[0]) {
-        numLiterals += 1;
-        if (numLiterals == MAX_REPEAT_SIZE) {
-          writeValues();
-        }
-      } else {
-        writeValues();
-        literals[numLiterals++] = value;
-        tailRunLength = 1;
-      }
-    } else {
-      if (value == literals[numLiterals - 1]) {
-        tailRunLength += 1;
-      } else {
-        tailRunLength = 1;
-      }
-      if (tailRunLength == MIN_REPEAT_SIZE) {
-        if (numLiterals + 1 == MIN_REPEAT_SIZE) {
-          repeat = true;
-          numLiterals += 1;
-        } else {
-          numLiterals -= MIN_REPEAT_SIZE - 1;
-          writeValues();
-          literals[0] = value;
-          repeat = true;
-          numLiterals = MIN_REPEAT_SIZE;
-        }
-      } else {
-        literals[numLiterals++] = value;
-        if (numLiterals == MAX_LITERAL_SIZE) {
-          writeValues();
-        }
-      }
-    }
-  }
-
-  void getPosition(PositionRecorder recorder) throws IOException {
-    output.getPosition(recorder);
-    recorder.addPosition(numLiterals);
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/RunLengthIntegerWriter.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/RunLengthIntegerWriter.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/RunLengthIntegerWriter.java
deleted file mode 100644
index 867f041..0000000
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/RunLengthIntegerWriter.java
+++ /dev/null
@@ -1,143 +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.tajo.storage.thirdparty.orc;
-
-import java.io.IOException;
-
-/**
- * A streamFactory that writes a sequence of integers. A control byte is written before
- * each run with positive values 0 to 127 meaning 3 to 130 repetitions, each
- * repetition is offset by a delta. If the control byte is -1 to -128, 1 to 128
- * literal vint values follow.
- */
-class RunLengthIntegerWriter implements IntegerWriter {
-  static final int MIN_REPEAT_SIZE = 3;
-  static final int MAX_DELTA = 127;
-  static final int MIN_DELTA = -128;
-  static final int MAX_LITERAL_SIZE = 128;
-  private static final int MAX_REPEAT_SIZE = 127 + MIN_REPEAT_SIZE;
-  private final PositionedOutputStream output;
-  private final boolean signed;
-  private final long[] literals = new long[MAX_LITERAL_SIZE];
-  private int numLiterals = 0;
-  private long delta = 0;
-  private boolean repeat = false;
-  private int tailRunLength = 0;
-  private SerializationUtils utils;
-
-  RunLengthIntegerWriter(PositionedOutputStream output,
-                         boolean signed) {
-    this.output = output;
-    this.signed = signed;
-    this.utils = new SerializationUtils();
-  }
-
-  private void writeValues() throws IOException {
-    if (numLiterals != 0) {
-      if (repeat) {
-        output.write(numLiterals - MIN_REPEAT_SIZE);
-        output.write((byte) delta);
-        if (signed) {
-          utils.writeVslong(output, literals[0]);
-        } else {
-          utils.writeVulong(output, literals[0]);
-        }
-      } else {
-        output.write(-numLiterals);
-        for(int i=0; i < numLiterals; ++i) {
-          if (signed) {
-            utils.writeVslong(output, literals[i]);
-          } else {
-            utils.writeVulong(output, literals[i]);
-          }
-        }
-      }
-      repeat = false;
-      numLiterals = 0;
-      tailRunLength = 0;
-    }
-  }
-
-  @Override
-  public void flush() throws IOException {
-    writeValues();
-    output.flush();
-  }
-
-  @Override
-  public void write(long value) throws IOException {
-    if (numLiterals == 0) {
-      literals[numLiterals++] = value;
-      tailRunLength = 1;
-    } else if (repeat) {
-      if (value == literals[0] + delta * numLiterals) {
-        numLiterals += 1;
-        if (numLiterals == MAX_REPEAT_SIZE) {
-          writeValues();
-        }
-      } else {
-        writeValues();
-        literals[numLiterals++] = value;
-        tailRunLength = 1;
-      }
-    } else {
-      if (tailRunLength == 1) {
-        delta = value - literals[numLiterals - 1];
-        if (delta < MIN_DELTA || delta > MAX_DELTA) {
-          tailRunLength = 1;
-        } else {
-          tailRunLength = 2;
-        }
-      } else if (value == literals[numLiterals - 1] + delta) {
-        tailRunLength += 1;
-      } else {
-        delta = value - literals[numLiterals - 1];
-        if (delta < MIN_DELTA || delta > MAX_DELTA) {
-          tailRunLength = 1;
-        } else {
-          tailRunLength = 2;
-        }
-      }
-      if (tailRunLength == MIN_REPEAT_SIZE) {
-        if (numLiterals + 1 == MIN_REPEAT_SIZE) {
-          repeat = true;
-          numLiterals += 1;
-        } else {
-          numLiterals -= MIN_REPEAT_SIZE - 1;
-          long base = literals[numLiterals];
-          writeValues();
-          literals[0] = base;
-          repeat = true;
-          numLiterals = MIN_REPEAT_SIZE;
-        }
-      } else {
-        literals[numLiterals++] = value;
-        if (numLiterals == MAX_LITERAL_SIZE) {
-          writeValues();
-        }
-      }
-    }
-  }
-
-  @Override
-  public void getPosition(PositionRecorder recorder) throws IOException {
-    output.getPosition(recorder);
-    recorder.addPosition(numLiterals);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/RunLengthIntegerWriterV2.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/RunLengthIntegerWriterV2.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/RunLengthIntegerWriterV2.java
deleted file mode 100644
index 7237b2e..0000000
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/RunLengthIntegerWriterV2.java
+++ /dev/null
@@ -1,832 +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.tajo.storage.thirdparty.orc;
-
-import java.io.IOException;
-
-/**
- * A writer that performs light weight compression over sequence of integers.
- * <p>
- * There are four types of lightweight integer compression
- * <ul>
- * <li>SHORT_REPEAT</li>
- * <li>DIRECT</li>
- * <li>PATCHED_BASE</li>
- * <li>DELTA</li>
- * </ul>
- * </p>
- * The description and format for these types are as below:
- * <p>
- * <b>SHORT_REPEAT:</b> Used for short repeated integer sequences.
- * <ul>
- * <li>1 byte header
- * <ul>
- * <li>2 bits for encoding type</li>
- * <li>3 bits for bytes required for repeating value</li>
- * <li>3 bits for repeat count (MIN_REPEAT + run length)</li>
- * </ul>
- * </li>
- * <li>Blob - repeat value (fixed bytes)</li>
- * </ul>
- * </p>
- * <p>
- * <b>DIRECT:</b> Used for random integer sequences whose number of bit
- * requirement doesn't vary a lot.
- * <ul>
- * <li>2 bytes header
- * <ul>
- * 1st byte
- * <li>2 bits for encoding type</li>
- * <li>5 bits for fixed bit width of values in blob</li>
- * <li>1 bit for storing MSB of run length</li>
- * </ul>
- * <ul>
- * 2nd byte
- * <li>8 bits for lower run length bits</li>
- * </ul>
- * </li>
- * <li>Blob - stores the direct values using fixed bit width. The length of the
- * data blob is (fixed width * run length) bits long</li>
- * </ul>
- * </p>
- * <p>
- * <b>PATCHED_BASE:</b> Used for random integer sequences whose number of bit
- * requirement varies beyond a threshold.
- * <ul>
- * <li>4 bytes header
- * <ul>
- * 1st byte
- * <li>2 bits for encoding type</li>
- * <li>5 bits for fixed bit width of values in blob</li>
- * <li>1 bit for storing MSB of run length</li>
- * </ul>
- * <ul>
- * 2nd byte
- * <li>8 bits for lower run length bits</li>
- * </ul>
- * <ul>
- * 3rd byte
- * <li>3 bits for bytes required to encode base value</li>
- * <li>5 bits for patch width</li>
- * </ul>
- * <ul>
- * 4th byte
- * <li>3 bits for patch gap width</li>
- * <li>5 bits for patch length</li>
- * </ul>
- * </li>
- * <li>Base value - Stored using fixed number of bytes. If MSB is set, base
- * value is negative else positive. Length of base value is (base width * 8)
- * bits.</li>
- * <li>Data blob - Base reduced values as stored using fixed bit width. Length
- * of data blob is (fixed width * run length) bits.</li>
- * <li>Patch blob - Patch blob is a list of gap and patch value. Each entry in
- * the patch list is (patch width + patch gap width) bits long. Gap between the
- * subsequent elements to be patched are stored in upper part of entry whereas
- * patch values are stored in lower part of entry. Length of patch blob is
- * ((patch width + patch gap width) * patch length) bits.</li>
- * </ul>
- * </p>
- * <p>
- * <b>DELTA</b> Used for monotonically increasing or decreasing sequences,
- * sequences with fixed delta values or long repeated sequences.
- * <ul>
- * <li>2 bytes header
- * <ul>
- * 1st byte
- * <li>2 bits for encoding type</li>
- * <li>5 bits for fixed bit width of values in blob</li>
- * <li>1 bit for storing MSB of run length</li>
- * </ul>
- * <ul>
- * 2nd byte
- * <li>8 bits for lower run length bits</li>
- * </ul>
- * </li>
- * <li>Base value - encoded as varint</li>
- * <li>Delta base - encoded as varint</li>
- * <li>Delta blob - only positive values. monotonicity and orderness are decided
- * based on the sign of the base value and delta base</li>
- * </ul>
- * </p>
- */
-class RunLengthIntegerWriterV2 implements IntegerWriter {
-
-  public enum EncodingType {
-    SHORT_REPEAT, DIRECT, PATCHED_BASE, DELTA
-  }
-
-  static final int MAX_SCOPE = 512;
-  static final int MIN_REPEAT = 3;
-  private static final int MAX_SHORT_REPEAT_LENGTH = 10;
-  private long prevDelta = 0;
-  private int fixedRunLength = 0;
-  private int variableRunLength = 0;
-  private final long[] literals = new long[MAX_SCOPE];
-  private final PositionedOutputStream output;
-  private final boolean signed;
-  private EncodingType encoding;
-  private int numLiterals;
-  private final long[] zigzagLiterals = new long[MAX_SCOPE];
-  private final long[] baseRedLiterals = new long[MAX_SCOPE];
-  private final long[] adjDeltas = new long[MAX_SCOPE];
-  private long fixedDelta;
-  private int zzBits90p;
-  private int zzBits100p;
-  private int brBits95p;
-  private int brBits100p;
-  private int bitsDeltaMax;
-  private int patchWidth;
-  private int patchGapWidth;
-  private int patchLength;
-  private long[] gapVsPatchList;
-  private long min;
-  private boolean isFixedDelta;
-  private SerializationUtils utils;
-  private boolean alignedBitpacking;
-
-  RunLengthIntegerWriterV2(PositionedOutputStream output, boolean signed) {
-    this(output, signed, true);
-  }
-
-  RunLengthIntegerWriterV2(PositionedOutputStream output, boolean signed,
-      boolean alignedBitpacking) {
-    this.output = output;
-    this.signed = signed;
-    this.alignedBitpacking = alignedBitpacking;
-    this.utils = new SerializationUtils();
-    clear();
-  }
-
-  private void writeValues() throws IOException {
-    if (numLiterals != 0) {
-
-      if (encoding.equals(EncodingType.SHORT_REPEAT)) {
-        writeShortRepeatValues();
-      } else if (encoding.equals(EncodingType.DIRECT)) {
-        writeDirectValues();
-      } else if (encoding.equals(EncodingType.PATCHED_BASE)) {
-        writePatchedBaseValues();
-      } else {
-        writeDeltaValues();
-      }
-
-      // clear all the variables
-      clear();
-    }
-  }
-
-  private void writeDeltaValues() throws IOException {
-    int len = 0;
-    int fb = bitsDeltaMax;
-    int efb = 0;
-
-    if (alignedBitpacking) {
-      fb = utils.getClosestAlignedFixedBits(fb);
-    }
-
-    if (isFixedDelta) {
-      // if fixed run length is greater than threshold then it will be fixed
-      // delta sequence with delta value 0 else fixed delta sequence with
-      // non-zero delta value
-      if (fixedRunLength > MIN_REPEAT) {
-        // ex. sequence: 2 2 2 2 2 2 2 2
-        len = fixedRunLength - 1;
-        fixedRunLength = 0;
-      } else {
-        // ex. sequence: 4 6 8 10 12 14 16
-        len = variableRunLength - 1;
-        variableRunLength = 0;
-      }
-    } else {
-      // fixed width 0 is used for long repeating values.
-      // sequences that require only 1 bit to encode will have an additional bit
-      if (fb == 1) {
-        fb = 2;
-      }
-      efb = utils.encodeBitWidth(fb);
-      efb = efb << 1;
-      len = variableRunLength - 1;
-      variableRunLength = 0;
-    }
-
-    // extract the 9th bit of run length
-    final int tailBits = (len & 0x100) >>> 8;
-
-    // create first byte of the header
-    final int headerFirstByte = getOpcode() | efb | tailBits;
-
-    // second byte of the header stores the remaining 8 bits of runlength
-    final int headerSecondByte = len & 0xff;
-
-    // write header
-    output.write(headerFirstByte);
-    output.write(headerSecondByte);
-
-    // store the first value from zigzag literal array
-    if (signed) {
-      utils.writeVslong(output, literals[0]);
-    } else {
-      utils.writeVulong(output, literals[0]);
-    }
-
-    if (isFixedDelta) {
-      // if delta is fixed then we don't need to store delta blob
-      utils.writeVslong(output, fixedDelta);
-    } else {
-      // store the first value as delta value using zigzag encoding
-      utils.writeVslong(output, adjDeltas[0]);
-
-      // adjacent delta values are bit packed. The length of adjDeltas array is
-      // always one less than the number of literals (delta difference for n
-      // elements is n-1). We have already written one element, write the
-      // remaining numLiterals - 2 elements here
-      utils.writeInts(adjDeltas, 1, numLiterals - 2, fb, output);
-    }
-  }
-
-  private void writePatchedBaseValues() throws IOException {
-
-    // NOTE: Aligned bit packing cannot be applied for PATCHED_BASE encoding
-    // because patch is applied to MSB bits. For example: If fixed bit width of
-    // base value is 7 bits and if patch is 3 bits, the actual value is
-    // constructed by shifting the patch to left by 7 positions.
-    // actual_value = patch << 7 | base_value
-    // So, if we align base_value then actual_value can not be reconstructed.
-
-    // write the number of fixed bits required in next 5 bits
-    final int fb = brBits95p;
-    final int efb = utils.encodeBitWidth(fb) << 1;
-
-    // adjust variable run length, they are one off
-    variableRunLength -= 1;
-
-    // extract the 9th bit of run length
-    final int tailBits = (variableRunLength & 0x100) >>> 8;
-
-    // create first byte of the header
-    final int headerFirstByte = getOpcode() | efb | tailBits;
-
-    // second byte of the header stores the remaining 8 bits of runlength
-    final int headerSecondByte = variableRunLength & 0xff;
-
-    // if the min value is negative toggle the sign
-    final boolean isNegative = min < 0 ? true : false;
-    if (isNegative) {
-      min = -min;
-    }
-
-    // find the number of bytes required for base and shift it by 5 bits
-    // to accommodate patch width. The additional bit is used to store the sign
-    // of the base value.
-    final int baseWidth = utils.findClosestNumBits(min) + 1;
-    final int baseBytes = baseWidth % 8 == 0 ? baseWidth / 8 : (baseWidth / 8) + 1;
-    final int bb = (baseBytes - 1) << 5;
-
-    // if the base value is negative then set MSB to 1
-    if (isNegative) {
-      min |= (1L << ((baseBytes * 8) - 1));
-    }
-
-    // third byte contains 3 bits for number of bytes occupied by base
-    // and 5 bits for patchWidth
-    final int headerThirdByte = bb | utils.encodeBitWidth(patchWidth);
-
-    // fourth byte contains 3 bits for page gap width and 5 bits for
-    // patch length
-    final int headerFourthByte = (patchGapWidth - 1) << 5 | patchLength;
-
-    // write header
-    output.write(headerFirstByte);
-    output.write(headerSecondByte);
-    output.write(headerThirdByte);
-    output.write(headerFourthByte);
-
-    // write the base value using fixed bytes in big endian order
-    for(int i = baseBytes - 1; i >= 0; i--) {
-      byte b = (byte) ((min >>> (i * 8)) & 0xff);
-      output.write(b);
-    }
-
-    // base reduced literals are bit packed
-    int closestFixedBits = utils.getClosestFixedBits(fb);
-
-    utils.writeInts(baseRedLiterals, 0, numLiterals, closestFixedBits,
-        output);
-
-    // write patch list
-    closestFixedBits = utils.getClosestFixedBits(patchGapWidth + patchWidth);
-
-    utils.writeInts(gapVsPatchList, 0, gapVsPatchList.length, closestFixedBits,
-        output);
-
-    // reset run length
-    variableRunLength = 0;
-  }
-
-  /**
-   * Store the opcode in 2 MSB bits
-   * @return opcode
-   */
-  private int getOpcode() {
-    return encoding.ordinal() << 6;
-  }
-
-  private void writeDirectValues() throws IOException {
-
-    // write the number of fixed bits required in next 5 bits
-    int fb = zzBits100p;
-
-    if (alignedBitpacking) {
-      fb = utils.getClosestAlignedFixedBits(fb);
-    }
-
-    final int efb = utils.encodeBitWidth(fb) << 1;
-
-    // adjust variable run length
-    variableRunLength -= 1;
-
-    // extract the 9th bit of run length
-    final int tailBits = (variableRunLength & 0x100) >>> 8;
-
-    // create first byte of the header
-    final int headerFirstByte = getOpcode() | efb | tailBits;
-
-    // second byte of the header stores the remaining 8 bits of runlength
-    final int headerSecondByte = variableRunLength & 0xff;
-
-    // write header
-    output.write(headerFirstByte);
-    output.write(headerSecondByte);
-
-    // bit packing the zigzag encoded literals
-    utils.writeInts(zigzagLiterals, 0, numLiterals, fb, output);
-
-    // reset run length
-    variableRunLength = 0;
-  }
-
-  private void writeShortRepeatValues() throws IOException {
-    // get the value that is repeating, compute the bits and bytes required
-    long repeatVal = 0;
-    if (signed) {
-      repeatVal = utils.zigzagEncode(literals[0]);
-    } else {
-      repeatVal = literals[0];
-    }
-
-    final int numBitsRepeatVal = utils.findClosestNumBits(repeatVal);
-    final int numBytesRepeatVal = numBitsRepeatVal % 8 == 0 ? numBitsRepeatVal >>> 3
-        : (numBitsRepeatVal >>> 3) + 1;
-
-    // write encoding type in top 2 bits
-    int header = getOpcode();
-
-    // write the number of bytes required for the value
-    header |= ((numBytesRepeatVal - 1) << 3);
-
-    // write the run length
-    fixedRunLength -= MIN_REPEAT;
-    header |= fixedRunLength;
-
-    // write the header
-    output.write(header);
-
-    // write the repeating value in big endian byte order
-    for(int i = numBytesRepeatVal - 1; i >= 0; i--) {
-      int b = (int) ((repeatVal >>> (i * 8)) & 0xff);
-      output.write(b);
-    }
-
-    fixedRunLength = 0;
-  }
-
-  private void determineEncoding() {
-
-    // we need to compute zigzag values for DIRECT encoding if we decide to
-    // break early for delta overflows or for shorter runs
-    computeZigZagLiterals();
-
-    zzBits100p = utils.percentileBits(zigzagLiterals, 0, numLiterals, 1.0);
-
-    // not a big win for shorter runs to determine encoding
-    if (numLiterals <= MIN_REPEAT) {
-      encoding = EncodingType.DIRECT;
-      return;
-    }
-
-    // DELTA encoding check
-
-    // for identifying monotonic sequences
-    boolean isIncreasing = true;
-    boolean isDecreasing = true;
-    this.isFixedDelta = true;
-
-    this.min = literals[0];
-    long max = literals[0];
-    final long initialDelta = literals[1] - literals[0];
-    long currDelta = initialDelta;
-    long deltaMax = initialDelta;
-    this.adjDeltas[0] = initialDelta;
-
-    for (int i = 1; i < numLiterals; i++) {
-      final long l1 = literals[i];
-      final long l0 = literals[i - 1];
-      currDelta = l1 - l0;
-      min = Math.min(min, l1);
-      max = Math.max(max, l1);
-
-      isIncreasing &= (l0 <= l1);
-      isDecreasing &= (l0 >= l1);
-
-      isFixedDelta &= (currDelta == initialDelta);
-      if (i > 1) {
-        adjDeltas[i - 1] = Math.abs(currDelta);
-        deltaMax = Math.max(deltaMax, adjDeltas[i - 1]);
-      }
-    }
-
-    // its faster to exit under delta overflow condition without checking for
-    // PATCHED_BASE condition as encoding using DIRECT is faster and has less
-    // overhead than PATCHED_BASE
-    if (!utils.isSafeSubtract(max, min)) {
-      encoding = EncodingType.DIRECT;
-      return;
-    }
-
-    // invariant - subtracting any number from any other in the literals after
-    // this point won't overflow
-
-    // if initialDelta is 0 then we cannot delta encode as we cannot identify
-    // the sign of deltas (increasing or decreasing)
-    if (initialDelta != 0) {
-
-      // if min is equal to max then the delta is 0, this condition happens for
-      // fixed values run >10 which cannot be encoded with SHORT_REPEAT
-      if (min == max) {
-        assert isFixedDelta : min + "==" + max +
-            ", isFixedDelta cannot be false";
-        assert currDelta == 0 : min + "==" + max + ", currDelta should be zero";
-        fixedDelta = 0;
-        encoding = EncodingType.DELTA;
-        return;
-      }
-
-      if (isFixedDelta) {
-        assert currDelta == initialDelta
-            : "currDelta should be equal to initialDelta for fixed delta encoding";
-        encoding = EncodingType.DELTA;
-        fixedDelta = currDelta;
-        return;
-      }
-
-      // stores the number of bits required for packing delta blob in
-      // delta encoding
-      bitsDeltaMax = utils.findClosestNumBits(deltaMax);
-
-      // monotonic condition
-      if (isIncreasing || isDecreasing) {
-        encoding = EncodingType.DELTA;
-        return;
-      }
-    }
-
-    // PATCHED_BASE encoding check
-
-    // percentile values are computed for the zigzag encoded values. if the
-    // number of bit requirement between 90th and 100th percentile varies
-    // beyond a threshold then we need to patch the values. if the variation
-    // is not significant then we can use direct encoding
-
-    zzBits90p = utils.percentileBits(zigzagLiterals, 0, numLiterals, 0.9);
-    int diffBitsLH = zzBits100p - zzBits90p;
-
-    // if the difference between 90th percentile and 100th percentile fixed
-    // bits is > 1 then we need patch the values
-    if (diffBitsLH > 1) {
-
-      // patching is done only on base reduced values.
-      // remove base from literals
-      for (int i = 0; i < numLiterals; i++) {
-        baseRedLiterals[i] = literals[i] - min;
-      }
-
-      // 95th percentile width is used to determine max allowed value
-      // after which patching will be done
-      brBits95p = utils.percentileBits(baseRedLiterals, 0, numLiterals, 0.95);
-
-      // 100th percentile is used to compute the max patch width
-      brBits100p = utils.percentileBits(baseRedLiterals, 0, numLiterals, 1.0);
-
-      // after base reducing the values, if the difference in bits between
-      // 95th percentile and 100th percentile value is zero then there
-      // is no point in patching the values, in which case we will
-      // fallback to DIRECT encoding.
-      // The decision to use patched base was based on zigzag values, but the
-      // actual patching is done on base reduced literals.
-      if ((brBits100p - brBits95p) != 0) {
-        encoding = EncodingType.PATCHED_BASE;
-        preparePatchedBlob();
-        return;
-      } else {
-        encoding = EncodingType.DIRECT;
-        return;
-      }
-    } else {
-      // if difference in bits between 95th percentile and 100th percentile is
-      // 0, then patch length will become 0. Hence we will fallback to direct
-      encoding = EncodingType.DIRECT;
-      return;
-    }
-  }
-
-  private void computeZigZagLiterals() {
-    // populate zigzag encoded literals
-    long zzEncVal = 0;
-    for (int i = 0; i < numLiterals; i++) {
-      if (signed) {
-        zzEncVal = utils.zigzagEncode(literals[i]);
-      } else {
-        zzEncVal = literals[i];
-      }
-      zigzagLiterals[i] = zzEncVal;
-    }
-  }
-
-  private void preparePatchedBlob() {
-    // mask will be max value beyond which patch will be generated
-    long mask = (1L << brBits95p) - 1;
-
-    // since we are considering only 95 percentile, the size of gap and
-    // patch array can contain only be 5% values
-    patchLength = (int) Math.ceil((numLiterals * 0.05));
-
-    int[] gapList = new int[patchLength];
-    long[] patchList = new long[patchLength];
-
-    // #bit for patch
-    patchWidth = brBits100p - brBits95p;
-    patchWidth = utils.getClosestFixedBits(patchWidth);
-
-    // if patch bit requirement is 64 then it will not possible to pack
-    // gap and patch together in a long. To make sure gap and patch can be
-    // packed together adjust the patch width
-    if (patchWidth == 64) {
-      patchWidth = 56;
-      brBits95p = 8;
-      mask = (1L << brBits95p) - 1;
-    }
-
-    int gapIdx = 0;
-    int patchIdx = 0;
-    int prev = 0;
-    int gap = 0;
-    int maxGap = 0;
-
-    for(int i = 0; i < numLiterals; i++) {
-      // if value is above mask then create the patch and record the gap
-      if (baseRedLiterals[i] > mask) {
-        gap = i - prev;
-        if (gap > maxGap) {
-          maxGap = gap;
-        }
-
-        // gaps are relative, so store the previous patched value index
-        prev = i;
-        gapList[gapIdx++] = gap;
-
-        // extract the most significant bits that are over mask bits
-        long patch = baseRedLiterals[i] >>> brBits95p;
-        patchList[patchIdx++] = patch;
-
-        // strip off the MSB to enable safe bit packing
-        baseRedLiterals[i] &= mask;
-      }
-    }
-
-    // adjust the patch length to number of entries in gap list
-    patchLength = gapIdx;
-
-    // if the element to be patched is the first and only element then
-    // max gap will be 0, but to store the gap as 0 we need atleast 1 bit
-    if (maxGap == 0 && patchLength != 0) {
-      patchGapWidth = 1;
-    } else {
-      patchGapWidth = utils.findClosestNumBits(maxGap);
-    }
-
-    // special case: if the patch gap width is greater than 256, then
-    // we need 9 bits to encode the gap width. But we only have 3 bits in
-    // header to record the gap width. To deal with this case, we will save
-    // two entries in patch list in the following way
-    // 256 gap width => 0 for patch value
-    // actual gap - 256 => actual patch value
-    // We will do the same for gap width = 511. If the element to be patched is
-    // the last element in the scope then gap width will be 511. In this case we
-    // will have 3 entries in the patch list in the following way
-    // 255 gap width => 0 for patch value
-    // 255 gap width => 0 for patch value
-    // 1 gap width => actual patch value
-    if (patchGapWidth > 8) {
-      patchGapWidth = 8;
-      // for gap = 511, we need two additional entries in patch list
-      if (maxGap == 511) {
-        patchLength += 2;
-      } else {
-        patchLength += 1;
-      }
-    }
-
-    // create gap vs patch list
-    gapIdx = 0;
-    patchIdx = 0;
-    gapVsPatchList = new long[patchLength];
-    for(int i = 0; i < patchLength; i++) {
-      long g = gapList[gapIdx++];
-      long p = patchList[patchIdx++];
-      while (g > 255) {
-        gapVsPatchList[i++] = (255L << patchWidth);
-        g -= 255;
-      }
-
-      // store patch value in LSBs and gap in MSBs
-      gapVsPatchList[i] = (g << patchWidth) | p;
-    }
-  }
-
-  /**
-   * clears all the variables
-   */
-  private void clear() {
-    numLiterals = 0;
-    encoding = null;
-    prevDelta = 0;
-    fixedDelta = 0;
-    zzBits90p = 0;
-    zzBits100p = 0;
-    brBits95p = 0;
-    brBits100p = 0;
-    bitsDeltaMax = 0;
-    patchGapWidth = 0;
-    patchLength = 0;
-    patchWidth = 0;
-    gapVsPatchList = null;
-    min = 0;
-    isFixedDelta = true;
-  }
-
-  @Override
-  public void flush() throws IOException {
-    if (numLiterals != 0) {
-      if (variableRunLength != 0) {
-        determineEncoding();
-        writeValues();
-      } else if (fixedRunLength != 0) {
-        if (fixedRunLength < MIN_REPEAT) {
-          variableRunLength = fixedRunLength;
-          fixedRunLength = 0;
-          determineEncoding();
-          writeValues();
-        } else if (fixedRunLength >= MIN_REPEAT
-            && fixedRunLength <= MAX_SHORT_REPEAT_LENGTH) {
-          encoding = EncodingType.SHORT_REPEAT;
-          writeValues();
-        } else {
-          encoding = EncodingType.DELTA;
-          isFixedDelta = true;
-          writeValues();
-        }
-      }
-    }
-    output.flush();
-  }
-
-  @Override
-  public void write(long val) throws IOException {
-    if (numLiterals == 0) {
-      initializeLiterals(val);
-    } else {
-      if (numLiterals == 1) {
-        prevDelta = val - literals[0];
-        literals[numLiterals++] = val;
-        // if both values are same count as fixed run else variable run
-        if (val == literals[0]) {
-          fixedRunLength = 2;
-          variableRunLength = 0;
-        } else {
-          fixedRunLength = 0;
-          variableRunLength = 2;
-        }
-      } else {
-        long currentDelta = val - literals[numLiterals - 1];
-        if (prevDelta == 0 && currentDelta == 0) {
-          // fixed delta run
-
-          literals[numLiterals++] = val;
-
-          // if variable run is non-zero then we are seeing repeating
-          // values at the end of variable run in which case keep
-          // updating variable and fixed runs
-          if (variableRunLength > 0) {
-            fixedRunLength = 2;
-          }
-          fixedRunLength += 1;
-
-          // if fixed run met the minimum condition and if variable
-          // run is non-zero then flush the variable run and shift the
-          // tail fixed runs to start of the buffer
-          if (fixedRunLength >= MIN_REPEAT && variableRunLength > 0) {
-            numLiterals -= MIN_REPEAT;
-            variableRunLength -= MIN_REPEAT - 1;
-            // copy the tail fixed runs
-            long[] tailVals = new long[MIN_REPEAT];
-            System.arraycopy(literals, numLiterals, tailVals, 0, MIN_REPEAT);
-
-            // determine variable encoding and flush values
-            determineEncoding();
-            writeValues();
-
-            // shift tail fixed runs to beginning of the buffer
-            for(long l : tailVals) {
-              literals[numLiterals++] = l;
-            }
-          }
-
-          // if fixed runs reached max repeat length then write values
-          if (fixedRunLength == MAX_SCOPE) {
-            determineEncoding();
-            writeValues();
-          }
-        } else {
-          // variable delta run
-
-          // if fixed run length is non-zero and if it satisfies the
-          // short repeat conditions then write the values as short repeats
-          // else use delta encoding
-          if (fixedRunLength >= MIN_REPEAT) {
-            if (fixedRunLength <= MAX_SHORT_REPEAT_LENGTH) {
-              encoding = EncodingType.SHORT_REPEAT;
-              writeValues();
-            } else {
-              encoding = EncodingType.DELTA;
-              isFixedDelta = true;
-              writeValues();
-            }
-          }
-
-          // if fixed run length is <MIN_REPEAT and current value is
-          // different from previous then treat it as variable run
-          if (fixedRunLength > 0 && fixedRunLength < MIN_REPEAT) {
-            if (val != literals[numLiterals - 1]) {
-              variableRunLength = fixedRunLength;
-              fixedRunLength = 0;
-            }
-          }
-
-          // after writing values re-initialize the variables
-          if (numLiterals == 0) {
-            initializeLiterals(val);
-          } else {
-            // keep updating variable run lengths
-            prevDelta = val - literals[numLiterals - 1];
-            literals[numLiterals++] = val;
-            variableRunLength += 1;
-
-            // if variable run length reach the max scope, write it
-            if (variableRunLength == MAX_SCOPE) {
-              determineEncoding();
-              writeValues();
-            }
-          }
-        }
-      }
-    }
-  }
-
-  private void initializeLiterals(long val) {
-    literals[numLiterals++] = val;
-    fixedRunLength = 1;
-    variableRunLength = 1;
-  }
-
-  @Override
-  public void getPosition(PositionRecorder recorder) throws IOException {
-    output.getPosition(recorder);
-    recorder.addPosition(numLiterals);
-  }
-}


[7/7] tajo git commit: TAJO-2102: Migrate to Apache Orc from Presto's one.

Posted by ji...@apache.org.
TAJO-2102: Migrate to Apache Orc from Presto's one.

Closes #985


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

Branch: refs/heads/master
Commit: 68263585296e30f93e541c36908a652df7398b9e
Parents: 9fcc9fd
Author: Jihoon Son <ji...@apache.org>
Authored: Wed Mar 23 10:39:31 2016 +0900
Committer: Jihoon Son <ji...@apache.org>
Committed: Wed Mar 23 10:39:59 2016 +0900

----------------------------------------------------------------------
 .../java/org/apache/tajo/catalog/TypeDesc.java  |    4 +
 .../tajo-catalog-drivers/tajo-hive/pom.xml      |  198 ++-
 .../tajo/catalog/store/HiveCatalogStore.java    |   15 +-
 .../tajo/catalog/store/HiveCatalogUtil.java     |    3 +
 .../catalog/store/TestHiveCatalogStore.java     |    1 +
 .../org/apache/tajo/cli/tools/TajoDump.java     |    2 +-
 .../org/apache/tajo/datum/TimestampDatum.java   |    2 +-
 .../apache/tajo/storage/StorageConstants.java   |    6 +-
 .../tajo/engine/query/TestSelectQuery.java      |   19 -
 .../apache/tajo/storage/TestQueryOnOrcFile.java |   79 +
 .../TestQueryOnOrcFile/timezoned/timezoned1.tbl |    3 +
 .../TestSelectQuery/timezoned/table1.tbl        |    3 -
 .../TestSelectQuery/timezoned/timezoned1.tbl    |    3 +
 .../datetime_table_timezoned_ddl.sql            |    5 +
 .../datetime_table_timezoned_orc_ddl.sql        |    4 +
 .../TestQueryOnOrcFile/testTimezone1.sql        |    1 +
 .../datetime_table_timezoned_orc_ddl.sql        |    4 -
 .../TestSelectQuery/testTimezonedORCTable.sql   |    2 -
 .../TestQueryOnOrcFile/testTimezone1.result     |    5 +
 .../TestQueryOnOrcFile/testTimezone2.result     |    5 +
 .../TestQueryOnOrcFile/testTimezone3.result     |    5 +
 .../TestQueryOnOrcFile/testTimezone4.result     |    5 +
 .../testTimezonedORCTable.result                |    5 -
 tajo-dist/pom.xml                               |   14 +-
 tajo-dist/src/main/bin/tajo                     |   10 +-
 tajo-project/pom.xml                            |    3 +-
 .../src/main/resources/storage-default.xml      |    2 +-
 .../src/test/resources/storage-default.xml      |    2 +-
 tajo-storage/tajo-storage-hdfs/pom.xml          |   34 +-
 .../apache/tajo/storage/orc/ORCAppender.java    |   93 +-
 .../org/apache/tajo/storage/orc/ORCScanner.java |  332 ----
 .../org/apache/tajo/storage/orc/OrcScanner.java |  460 ++++++
 .../objectinspector/ObjectInspectorFactory.java |   91 -
 .../TajoBlobObjectInspector.java                |   82 -
 .../TajoBooleanObjectInspector.java             |   76 -
 .../TajoDateObjectInspector.java                |   73 -
 .../TajoDoubleObjectInspector.java              |   76 -
 .../TajoFloatObjectInspector.java               |   76 -
 .../objectinspector/TajoIntObjectInspector.java |   76 -
 .../TajoLongObjectInspector.java                |   76 -
 .../TajoNullObjectInspector.java                |   69 -
 .../TajoPrimitiveObjectInspector.java           |   38 -
 .../TajoShortObjectInspector.java               |   76 -
 .../TajoStringObjectInspector.java              |   71 -
 .../TajoStructObjectInspector.java              |  122 --
 .../TajoTimestampObjectInspector.java           |   73 -
 .../thirdparty/orc/BinaryColumnStatistics.java  |   25 -
 .../storage/thirdparty/orc/BitFieldWriter.java  |   69 -
 .../storage/thirdparty/orc/BloomFilterIO.java   |   42 -
 .../thirdparty/orc/BooleanColumnStatistics.java |   27 -
 .../thirdparty/orc/ByteBufferAllocatorPool.java |  102 ++
 .../thirdparty/orc/ByteBufferPoolAdapter.java   |   41 +
 .../thirdparty/orc/ColumnStatistics.java        |   36 -
 .../thirdparty/orc/ColumnStatisticsImpl.java    | 1017 ------------
 .../thirdparty/orc/CompressionCodec.java        |   68 -
 .../storage/thirdparty/orc/CompressionKind.java |   27 -
 .../thirdparty/orc/DateColumnStatistics.java    |   37 -
 .../thirdparty/orc/DecimalColumnStatistics.java |   45 -
 .../orc/DirectDecompressionCodec.java           |   26 -
 .../thirdparty/orc/DoubleColumnStatistics.java  |   44 -
 .../thirdparty/orc/DynamicByteArray.java        |  303 ----
 .../storage/thirdparty/orc/DynamicIntArray.java |  142 --
 .../thirdparty/orc/HdfsOrcDataSource.java       |  133 --
 .../thirdparty/orc/IntegerColumnStatistics.java |   50 -
 .../storage/thirdparty/orc/IntegerWriter.java   |   47 -
 .../storage/thirdparty/orc/MemoryManager.java   |  212 ---
 .../tajo/storage/thirdparty/orc/Metadata.java   |   45 -
 .../tajo/storage/thirdparty/orc/OrcFile.java    |  389 +++--
 .../storage/thirdparty/orc/OrcRecordReader.java |  454 +++++
 .../tajo/storage/thirdparty/orc/OrcUtils.java   |  242 +--
 .../tajo/storage/thirdparty/orc/OutStream.java  |  286 ----
 .../thirdparty/orc/PositionRecorder.java        |   25 -
 .../thirdparty/orc/PositionedOutputStream.java  |   38 -
 .../thirdparty/orc/RecordReaderUtils.java       |  393 +++++
 .../storage/thirdparty/orc/RedBlackTree.java    |  309 ----
 .../thirdparty/orc/RunLengthByteWriter.java     |  106 --
 .../thirdparty/orc/RunLengthIntegerWriter.java  |  143 --
 .../orc/RunLengthIntegerWriterV2.java           |  832 ----------
 .../thirdparty/orc/SerializationUtils.java      |  844 ----------
 .../storage/thirdparty/orc/SnappyCodec.java     |  109 --
 .../tajo/storage/thirdparty/orc/StreamName.java |   95 --
 .../thirdparty/orc/StringColumnStatistics.java  |   41 -
 .../thirdparty/orc/StringRedBlackTree.java      |  202 ---
 .../thirdparty/orc/StripeInformation.java       |   59 -
 .../thirdparty/orc/StripeStatistics.java        |   42 -
 .../orc/TimestampColumnStatistics.java          |   38 -
 .../thirdparty/orc/TreeReaderFactory.java       | 1557 ++++++++++++++++++
 .../tajo/storage/thirdparty/orc/Writer.java     |    2 +
 .../tajo/storage/thirdparty/orc/WriterImpl.java |  813 +++++----
 .../storage/thirdparty/orc/ZeroCopyAdapter.java |   57 +
 .../tajo/storage/thirdparty/orc/ZlibCodec.java  |  169 --
 .../src/main/proto/orc_proto.proto              |  217 ---
 .../tajo/storage/TestCompressionStorages.java   |   13 +-
 .../org/apache/tajo/storage/TestStorages.java   |   69 +-
 .../resources/dataset/testVariousTypes.avsc     |    3 +-
 .../src/test/resources/storage-default.xml      |    2 +-
 96 files changed, 4214 insertions(+), 8277 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/TypeDesc.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/TypeDesc.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/TypeDesc.java
index 3bd0f00..3ca83f9 100644
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/TypeDesc.java
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/TypeDesc.java
@@ -55,6 +55,10 @@ public class TypeDesc {
     }
   }
 
+  public Schema getNestedSchema() {
+    return nestedRecordSchema;
+  }
+
   public int hashCode() {
     return Objects.hashCode(dataType.hashCode(), nestedRecordSchema);
   }

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-catalog/tajo-catalog-drivers/tajo-hive/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-drivers/tajo-hive/pom.xml b/tajo-catalog/tajo-catalog-drivers/tajo-hive/pom.xml
index 1a8a188..d848461 100644
--- a/tajo-catalog/tajo-catalog-drivers/tajo-hive/pom.xml
+++ b/tajo-catalog/tajo-catalog-drivers/tajo-hive/pom.xml
@@ -33,8 +33,6 @@
   <properties>
     <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
     <project.reporting.outputEncoding>UTF-8</project.reporting.outputEncoding>
-    <parquet.version>1.5.0</parquet.version>
-    <parquet.format.version>2.1.0</parquet.format.version>
   </properties>
 
   <build>
@@ -136,19 +134,35 @@
     </dependency>
     <dependency>
       <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-mapreduce-client-core</artifactId>
+      <artifactId>hadoop-common</artifactId>
       <version>${hadoop.version}</version>
       <scope>provided</scope>
+      <exclusions>
+        <exclusion>
+          <artifactId>zookeeper</artifactId>
+          <groupId>org.apache.zookeeper</groupId>
+        </exclusion>
+      </exclusions>
     </dependency>
     <dependency>
       <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-common</artifactId>
+      <artifactId>hadoop-mapreduce-client-core</artifactId>
       <version>${hadoop.version}</version>
       <scope>provided</scope>
+      <exclusions>
+        <exclusion>
+          <artifactId>hadoop-yarn-common</artifactId>
+          <groupId>org.apache.hadoop</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>netty</artifactId>
+          <groupId>io.netty</groupId>
+        </exclusion>
+      </exclusions>
     </dependency>
     <dependency>
       <groupId>org.apache.hive</groupId>
-      <artifactId>hive-exec</artifactId>
+      <artifactId>hive-metastore</artifactId>
       <version>${hive.version}</version>
       <scope>provided</scope>
       <exclusions>
@@ -158,129 +172,201 @@
         </exclusion>
         <exclusion>
           <groupId>org.apache.hive</groupId>
-          <artifactId>hive-contrib</artifactId>
+          <artifactId>hive-serde</artifactId>
         </exclusion>
         <exclusion>
           <groupId>org.apache.hive</groupId>
-          <artifactId>hive-hbase-handler</artifactId>
+          <artifactId>hive-shimss</artifactId>
         </exclusion>
         <exclusion>
-          <groupId>org.apache.hive</groupId>
-          <artifactId>hive-metastore</artifactId>
+          <groupId>org.apache.thrift</groupId>
+          <artifactId>libfb303</artifactId>
         </exclusion>
         <exclusion>
-          <groupId>org.apache.hive</groupId>
-          <artifactId>hive-serde</artifactId>
+          <groupId>org.apache.thrift</groupId>
+          <artifactId>libthrift</artifactId>
         </exclusion>
         <exclusion>
-          <groupId>org.apache.hive</groupId>
-          <artifactId>hive-shims</artifactId>
+          <groupId>com.jolbox</groupId>
+          <artifactId>bonecp</artifactId>
         </exclusion>
         <exclusion>
-          <groupId>org.apache.hive</groupId>
-          <artifactId>hive-testutils</artifactId>
+          <artifactId>tephra-hbase-compat-1.0</artifactId>
+          <groupId>co.cask.tephra</groupId>
         </exclusion>
         <exclusion>
-          <groupId>org.apache.thrift</groupId>
-          <artifactId>libfb303</artifactId>
+          <artifactId>tephra-core</artifactId>
+          <groupId>co.cask.tephra</groupId>
         </exclusion>
         <exclusion>
-          <groupId>org.apache.thrift</groupId>
-          <artifactId>libthrift</artifactId>
+          <artifactId>tephra-api</artifactId>
+          <groupId>co.cask.tephra</groupId>
         </exclusion>
         <exclusion>
-          <groupId>com.jolbox</groupId>
-          <artifactId>bonecp</artifactId>
+          <artifactId>hbase-client</artifactId>
+          <groupId>org.apache.hbase</groupId>
         </exclusion>
         <exclusion>
-          <groupId>com.google.protobuf</groupId>
-          <artifactId>protobuf-java</artifactId>
+          <artifactId>hadoop-yarn-server-resourcemanager</artifactId>
+          <groupId>org.apache.hadoop</groupId>
         </exclusion>
         <exclusion>
-          <groupId>org.apache.calcite</groupId>
-          <artifactId>calcite-core</artifactId>
+          <artifactId>antlr-runtime</artifactId>
+          <groupId>org.antlr</groupId>
         </exclusion>
         <exclusion>
-          <groupId>org.apache.calcite</groupId>
-          <artifactId>calcite-avatica</artifactId>
+          <artifactId>log4j-slf4j-impl</artifactId>
+          <groupId>org.apache.logging.log4j</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>zookeeper</artifactId>
+          <groupId>org.apache.zookeeper</groupId>
         </exclusion>
       </exclusions>
     </dependency>
     <dependency>
       <groupId>org.apache.hive</groupId>
-      <artifactId>hive-metastore</artifactId>
+      <artifactId>hive-common</artifactId>
       <version>${hive.version}</version>
       <scope>provided</scope>
       <exclusions>
         <exclusion>
-          <groupId>org.apache.hive</groupId>
-          <artifactId>hive-common</artifactId>
+          <artifactId>jetty-all</artifactId>
+          <groupId>org.eclipse.jetty.aggregate</groupId>
         </exclusion>
         <exclusion>
-          <groupId>org.apache.hive</groupId>
-          <artifactId>hive-serde</artifactId>
+          <artifactId>javax.servlet</artifactId>
+          <groupId>org.eclipse.jetty.orbit</groupId>
         </exclusion>
         <exclusion>
-          <groupId>org.apache.hive</groupId>
-          <artifactId>hive-shimss</artifactId>
+          <artifactId>joda-time</artifactId>
+          <groupId>joda-time</groupId>
         </exclusion>
         <exclusion>
-          <groupId>org.apache.thrift</groupId>
-          <artifactId>libfb303</artifactId>
+          <artifactId>jackson-databind</artifactId>
+          <groupId>com.fasterxml.jackson.core</groupId>
         </exclusion>
         <exclusion>
-          <groupId>org.apache.thrift</groupId>
-          <artifactId>libthrift</artifactId>
+          <artifactId>metrics-json</artifactId>
+          <groupId>io.dropwizard.metrics</groupId>
         </exclusion>
         <exclusion>
-          <groupId>com.jolbox</groupId>
-          <artifactId>bonecp</artifactId>
+          <artifactId>metrics-jvm</artifactId>
+          <groupId>io.dropwizard.metrics</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>metrics-core</artifactId>
+          <groupId>io.dropwizard.metrics</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>ant</artifactId>
+          <groupId>org.apache.ant</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>json</artifactId>
+          <groupId>org.json</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>log4j-slf4j-impl</artifactId>
+          <groupId>org.apache.logging.log4j</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>log4j-web</artifactId>
+          <groupId>org.apache.logging.log4j</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>log4j-1.2-api</artifactId>
+          <groupId>org.apache.logging.log4j</groupId>
         </exclusion>
       </exclusions>
     </dependency>
     <dependency>
       <groupId>org.apache.hive</groupId>
-      <artifactId>hive-cli</artifactId>
+      <artifactId>hive-exec</artifactId>
       <version>${hive.version}</version>
       <scope>provided</scope>
       <exclusions>
         <exclusion>
+          <artifactId>hive-ant</artifactId>
           <groupId>org.apache.hive</groupId>
-          <artifactId>hive-common</artifactId>
         </exclusion>
         <exclusion>
+          <artifactId>hive-llap-tez</artifactId>
           <groupId>org.apache.hive</groupId>
-          <artifactId>hive-exec</artifactId>
         </exclusion>
         <exclusion>
-          <groupId>org.apache.hive</groupId>
-          <artifactId>hive-metastore</artifactId>
+          <artifactId>ST4</artifactId>
+          <groupId>org.antlr</groupId>
         </exclusion>
         <exclusion>
-          <groupId>org.apache.hive</groupId>
-          <artifactId>hive-serde</artifactId>
+          <artifactId>ivy</artifactId>
+          <groupId>org.apache.ivy</groupId>
         </exclusion>
         <exclusion>
-          <groupId>org.apache.hive</groupId>
-          <artifactId>hive-service</artifactId>
+          <artifactId>curator-framework</artifactId>
+          <groupId>org.apache.curator</groupId>
         </exclusion>
         <exclusion>
-          <groupId>org.apache.hive</groupId>
-          <artifactId>hive-shims</artifactId>
+          <artifactId>apache-curator</artifactId>
+          <groupId>org.apache.curator</groupId>
         </exclusion>
         <exclusion>
-          <groupId>com.jolbox</groupId>
-          <artifactId>bonecp</artifactId>
+          <artifactId>groovy-all</artifactId>
+          <groupId>org.codehaus.groovy</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>calcite-core</artifactId>
+          <groupId>org.apache.calcite</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>calcite-avatica</artifactId>
+          <groupId>org.apache.calcite</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>stax-api</artifactId>
+          <groupId>stax</groupId>
         </exclusion>
         <exclusion>
-          <groupId>jline</groupId>
           <artifactId>jline</artifactId>
+          <groupId>jline</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>log4j-1.2-api</artifactId>
+          <groupId>org.apache.logging.log4j</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>log4j-slf4j-impl</artifactId>
+          <groupId>org.apache.logging.log4j</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>ant</artifactId>
+          <groupId>org.apache.ant</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>zookeeper</artifactId>
+          <groupId>org.apache.zookeeper</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>antlr-runtime</artifactId>
+          <groupId>org.antlr</groupId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hive</groupId>
+      <artifactId>hive-serde</artifactId>
+      <version>${hive.version}</version>
+      <scope>provided</scope>
+      <exclusions>
+        <exclusion>
+          <artifactId>opencsv</artifactId>
+          <groupId>net.sf.opencsv</groupId>
         </exclusion>
       </exclusions>
     </dependency>
     <dependency>
-      <groupId>com.twitter</groupId>
-      <artifactId>parquet-hive-bundle</artifactId>
+      <groupId>org.apache.parquet</groupId>
+      <artifactId>parquet-hadoop-bundle</artifactId>
       <version>${parquet.version}</version>
     </dependency>
   </dependencies>

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-catalog/tajo-catalog-drivers/tajo-hive/src/main/java/org/apache/tajo/catalog/store/HiveCatalogStore.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-drivers/tajo-hive/src/main/java/org/apache/tajo/catalog/store/HiveCatalogStore.java b/tajo-catalog/tajo-catalog-drivers/tajo-hive/src/main/java/org/apache/tajo/catalog/store/HiveCatalogStore.java
index 63f18b6..95cbf18 100644
--- a/tajo-catalog/tajo-catalog-drivers/tajo-hive/src/main/java/org/apache/tajo/catalog/store/HiveCatalogStore.java
+++ b/tajo-catalog/tajo-catalog-drivers/tajo-hive/src/main/java/org/apache/tajo/catalog/store/HiveCatalogStore.java
@@ -38,12 +38,15 @@ import org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe;
 import org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe;
 import org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe;
 import org.apache.hadoop.mapred.TextInputFormat;
+import org.apache.orc.OrcConf;
+import org.apache.parquet.hadoop.ParquetOutputFormat;
 import org.apache.tajo.BuiltinStorages;
 import org.apache.tajo.TajoConstants;
 import org.apache.tajo.algebra.Expr;
 import org.apache.tajo.algebra.IsNullPredicate;
 import org.apache.tajo.algebra.JsonHelper;
 import org.apache.tajo.catalog.*;
+import org.apache.tajo.catalog.TableMeta;
 import org.apache.tajo.catalog.partition.PartitionMethodDesc;
 import org.apache.tajo.catalog.proto.CatalogProtos;
 import org.apache.tajo.catalog.proto.CatalogProtos.*;
@@ -56,10 +59,8 @@ import org.apache.tajo.plan.util.PartitionFilterAlgebraVisitor;
 import org.apache.tajo.storage.StorageConstants;
 import org.apache.tajo.util.KeyValueSet;
 import org.apache.thrift.TException;
-import parquet.hadoop.ParquetOutputFormat;
 
 import java.io.File;
-import java.io.IOException;
 import java.util.*;
 
 public class HiveCatalogStore extends CatalogConstants implements CatalogStore {
@@ -564,6 +565,16 @@ public class HiveCatalogStore extends CatalogConstants implements CatalogStore {
           table.putToParameters(ParquetOutputFormat.COMPRESSION,
               tableDesc.getMeta().getProperty(ParquetOutputFormat.COMPRESSION));
         }
+      } else if (tableDesc.getMeta().getDataFormat().equalsIgnoreCase(BuiltinStorages.ORC)) {
+        StorageFormatDescriptor descriptor = storageFormatFactory.get(IOConstants.ORC);
+        sd.setInputFormat(descriptor.getInputFormat());
+        sd.setOutputFormat(descriptor.getOutputFormat());
+        sd.getSerdeInfo().setSerializationLib(descriptor.getSerde());
+
+        if (tableDesc.getMeta().containsProperty(OrcConf.COMPRESS.getAttribute())) {
+          table.putToParameters(OrcConf.COMPRESS.getAttribute(),
+              tableDesc.getMeta().getProperty(OrcConf.COMPRESS.getAttribute()));
+        }
       } else {
         throw new UnsupportedException(tableDesc.getMeta().getDataFormat() + " in HivecatalogStore");
       }

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-catalog/tajo-catalog-drivers/tajo-hive/src/main/java/org/apache/tajo/catalog/store/HiveCatalogUtil.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-drivers/tajo-hive/src/main/java/org/apache/tajo/catalog/store/HiveCatalogUtil.java b/tajo-catalog/tajo-catalog-drivers/tajo-hive/src/main/java/org/apache/tajo/catalog/store/HiveCatalogUtil.java
index bbb7ade..87b391e 100644
--- a/tajo-catalog/tajo-catalog-drivers/tajo-hive/src/main/java/org/apache/tajo/catalog/store/HiveCatalogUtil.java
+++ b/tajo-catalog/tajo-catalog-drivers/tajo-hive/src/main/java/org/apache/tajo/catalog/store/HiveCatalogUtil.java
@@ -22,6 +22,7 @@ import org.apache.hadoop.hive.metastore.IMetaStoreClient;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
 import org.apache.hadoop.hive.ql.io.RCFileInputFormat;
+import org.apache.hadoop.hive.ql.io.orc.OrcSerde;
 import org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe;
 import org.apache.hadoop.hive.ql.metadata.Table;
 import org.apache.hadoop.hive.serde.serdeConstants;
@@ -137,6 +138,8 @@ public class HiveCatalogUtil {
       return BuiltinStorages.PARQUET;
     } else if (AvroSerDe.class.getName().equals(serde)) {
       return BuiltinStorages.AVRO;
+    } else if (OrcSerde.class.getName().equals(serde)) {
+      return BuiltinStorages.ORC;
     } else {
       throw new TajoRuntimeException(new UnknownDataFormatException(inputFormat));
     }

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-catalog/tajo-catalog-drivers/tajo-hive/src/test/java/org/apache/tajo/catalog/store/TestHiveCatalogStore.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-drivers/tajo-hive/src/test/java/org/apache/tajo/catalog/store/TestHiveCatalogStore.java b/tajo-catalog/tajo-catalog-drivers/tajo-hive/src/test/java/org/apache/tajo/catalog/store/TestHiveCatalogStore.java
index 7e1a3a4..46935fc 100644
--- a/tajo-catalog/tajo-catalog-drivers/tajo-hive/src/test/java/org/apache/tajo/catalog/store/TestHiveCatalogStore.java
+++ b/tajo-catalog/tajo-catalog-drivers/tajo-hive/src/test/java/org/apache/tajo/catalog/store/TestHiveCatalogStore.java
@@ -78,6 +78,7 @@ public class TestHiveCatalogStore {
     conf.set(HiveConf.ConfVars.METASTOREWAREHOUSE.varname, warehousePath.toUri().toString());
     conf.set(HiveConf.ConfVars.METASTORECONNECTURLKEY.varname, jdbcUri);
     conf.set(TajoConf.ConfVars.WAREHOUSE_DIR.varname, warehousePath.toUri().toString());
+    conf.setBoolean("datanucleus.schema.autoCreateAll", true);
 
     // create local HiveCatalogStore.
     TajoConf tajoConf = new TajoConf(conf);

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-cli/src/main/java/org/apache/tajo/cli/tools/TajoDump.java
----------------------------------------------------------------------
diff --git a/tajo-cli/src/main/java/org/apache/tajo/cli/tools/TajoDump.java b/tajo-cli/src/main/java/org/apache/tajo/cli/tools/TajoDump.java
index 4df418f..c9fa2b4 100644
--- a/tajo-cli/src/main/java/org/apache/tajo/cli/tools/TajoDump.java
+++ b/tajo-cli/src/main/java/org/apache/tajo/cli/tools/TajoDump.java
@@ -208,7 +208,7 @@ public class TajoDump {
           }
         }
         writer.write("\n\n");
-      } catch (Exception e) {
+      } catch (Throwable e) {
         // dump for each table can throw any exception. We need to skip the exception case.
         // here, the error message prints out via stderr.
         System.err.println("ERROR:" + tableName + "," + e.getMessage());

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-common/src/main/java/org/apache/tajo/datum/TimestampDatum.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/datum/TimestampDatum.java b/tajo-common/src/main/java/org/apache/tajo/datum/TimestampDatum.java
index 5b4c152..f69e7da 100644
--- a/tajo-common/src/main/java/org/apache/tajo/datum/TimestampDatum.java
+++ b/tajo-common/src/main/java/org/apache/tajo/datum/TimestampDatum.java
@@ -125,7 +125,7 @@ public class TimestampDatum extends Datum {
 
   /**
    *
-   * @param tm TimeMEta
+   * @param tm TimeMeta
    * @param timeZone Timezone
    * @param includeTimeZone Add timezone if it is true. It is usually used for TIMEZONEZ
    * @return A timestamp string

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-common/src/main/java/org/apache/tajo/storage/StorageConstants.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/storage/StorageConstants.java b/tajo-common/src/main/java/org/apache/tajo/storage/StorageConstants.java
index 097963c..4612323 100644
--- a/tajo-common/src/main/java/org/apache/tajo/storage/StorageConstants.java
+++ b/tajo-common/src/main/java/org/apache/tajo/storage/StorageConstants.java
@@ -89,11 +89,7 @@ public class StorageConstants {
   public static final String DEFAULT_ORC_STRIPE_SIZE = "67108864"; // 64MB
 
   public static final String ORC_COMPRESSION = "orc.compress";
-  public static final String ORC_COMPRESSION_KIND_NONE = "none";
-  public static final String ORC_COMPRESSION_KIND_SNAPPY = "snappy";
-  public static final String ORC_COMPRESSION_KIND_LZO = "lzo";
-  public static final String ORC_COMPRESSION_KIND_ZIP = "zlip";
-  public static final String DEFAULT_ORC_COMPRESSION_KIND = ORC_COMPRESSION_KIND_NONE;
+  public static final String DEFAULT_ORC_COMPRESSION_KIND = "none";
 
   public static final String ORC_BUFFER_SIZE = "orc.buffer.size";
   public static final String DEFAULT_ORC_BUFFER_SIZE = "262144"; // 256KB

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-core-tests/src/test/java/org/apache/tajo/engine/query/TestSelectQuery.java
----------------------------------------------------------------------
diff --git a/tajo-core-tests/src/test/java/org/apache/tajo/engine/query/TestSelectQuery.java b/tajo-core-tests/src/test/java/org/apache/tajo/engine/query/TestSelectQuery.java
index e55acf1..a2dec50 100644
--- a/tajo-core-tests/src/test/java/org/apache/tajo/engine/query/TestSelectQuery.java
+++ b/tajo-core-tests/src/test/java/org/apache/tajo/engine/query/TestSelectQuery.java
@@ -682,25 +682,6 @@ public class TestSelectQuery extends QueryTestCaseBase {
       executeString("DROP TABLE IF EXISTS timezoned_load2 PURGE");
     }
   }
-
-  @Test
-  public void testTimezonedORCTable() throws Exception {
-    try {
-
-      executeDDL("datetime_table_timezoned_ddl.sql", "timezoned", "timezoned");
-      executeDDL("datetime_table_timezoned_orc_ddl.sql", null, "timezoned_orc");
-
-      executeString("INSERT OVERWRITE INTO timezoned_orc SELECT t_timestamp, t_date FROM timezoned");
-
-      ResultSet res = executeQuery();
-      assertResultSet(res, "testTimezonedORCTable.result");
-      executeString("SET TIME ZONE 'GMT'");
-      cleanupQuery(res);
-    } finally {
-      executeString("DROP TABLE IF EXISTS timezoned");
-      executeString("DROP TABLE IF EXISTS timezoned_orc PURGE");
-    }
-  }
   
   @Test
   public void testMultiBytesDelimiter1() throws Exception {

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-core-tests/src/test/java/org/apache/tajo/storage/TestQueryOnOrcFile.java
----------------------------------------------------------------------
diff --git a/tajo-core-tests/src/test/java/org/apache/tajo/storage/TestQueryOnOrcFile.java b/tajo-core-tests/src/test/java/org/apache/tajo/storage/TestQueryOnOrcFile.java
new file mode 100644
index 0000000..29d132e
--- /dev/null
+++ b/tajo-core-tests/src/test/java/org/apache/tajo/storage/TestQueryOnOrcFile.java
@@ -0,0 +1,79 @@
+/*
+ * 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.tajo.storage;
+
+import org.apache.tajo.IntegrationTest;
+import org.apache.tajo.QueryTestCaseBase;
+import org.junit.*;
+import org.junit.experimental.categories.Category;
+
+import java.sql.ResultSet;
+
+@Category(IntegrationTest.class)
+public class TestQueryOnOrcFile extends QueryTestCaseBase {
+
+  @Before
+  public void setup() throws Exception {
+    executeDDL("datetime_table_timezoned_ddl.sql", "timezoned", "timezoned");
+    executeDDL("datetime_table_timezoned_orc_ddl.sql", null, "timezoned_orc");
+
+    executeString("INSERT OVERWRITE INTO timezoned_orc SELECT t_timestamp, t_date FROM timezoned");
+  }
+
+  @After
+  public void teardown() throws Exception {
+    executeString("DROP TABLE IF EXISTS timezoned");
+    executeString("DROP TABLE IF EXISTS timezoned_orc PURGE");
+  }
+
+  @Test
+  public void testTimezone1() throws Exception {
+    executeString("SET TIME ZONE 'GMT+9'");
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    executeString("SET TIME ZONE 'GMT'");
+    cleanupQuery(res);
+  }
+
+  @Test
+  public void testTimezone2() throws Exception {
+    executeString("SET TIME ZONE 'GMT+1'");
+    ResultSet res = executeString("select * from timezoned_orc");
+    assertResultSet(res);
+    executeString("SET TIME ZONE 'GMT'");
+    cleanupQuery(res);
+  }
+
+  @Test
+  public void testTimezone3() throws Exception {
+    executeString("SET TIME ZONE 'GMT'");
+    ResultSet res = executeString("select * from timezoned_orc");
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public void testTimezone4() throws Exception {
+    executeString("\\set TIMEZONE 'GMT-5'");
+    ResultSet res = executeString("select * from timezoned_orc");
+    assertResultSet(res);
+    executeString("SET TIME ZONE 'GMT'");
+    cleanupQuery(res);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-core-tests/src/test/resources/dataset/TestQueryOnOrcFile/timezoned/timezoned1.tbl
----------------------------------------------------------------------
diff --git a/tajo-core-tests/src/test/resources/dataset/TestQueryOnOrcFile/timezoned/timezoned1.tbl b/tajo-core-tests/src/test/resources/dataset/TestQueryOnOrcFile/timezoned/timezoned1.tbl
new file mode 100644
index 0000000..74b2e1b
--- /dev/null
+++ b/tajo-core-tests/src/test/resources/dataset/TestQueryOnOrcFile/timezoned/timezoned1.tbl
@@ -0,0 +1,3 @@
+1980-4-1 01:50:30.010|01:50:30.010|1980-04-01
+80/4/1 1:50:30 AM|1:50:30 AM|80/4/1
+1980 April 1 1:50:30|1:50:30|1980-04-01
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-core-tests/src/test/resources/dataset/TestSelectQuery/timezoned/table1.tbl
----------------------------------------------------------------------
diff --git a/tajo-core-tests/src/test/resources/dataset/TestSelectQuery/timezoned/table1.tbl b/tajo-core-tests/src/test/resources/dataset/TestSelectQuery/timezoned/table1.tbl
deleted file mode 100644
index 74b2e1b..0000000
--- a/tajo-core-tests/src/test/resources/dataset/TestSelectQuery/timezoned/table1.tbl
+++ /dev/null
@@ -1,3 +0,0 @@
-1980-4-1 01:50:30.010|01:50:30.010|1980-04-01
-80/4/1 1:50:30 AM|1:50:30 AM|80/4/1
-1980 April 1 1:50:30|1:50:30|1980-04-01
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-core-tests/src/test/resources/dataset/TestSelectQuery/timezoned/timezoned1.tbl
----------------------------------------------------------------------
diff --git a/tajo-core-tests/src/test/resources/dataset/TestSelectQuery/timezoned/timezoned1.tbl b/tajo-core-tests/src/test/resources/dataset/TestSelectQuery/timezoned/timezoned1.tbl
new file mode 100644
index 0000000..74b2e1b
--- /dev/null
+++ b/tajo-core-tests/src/test/resources/dataset/TestSelectQuery/timezoned/timezoned1.tbl
@@ -0,0 +1,3 @@
+1980-4-1 01:50:30.010|01:50:30.010|1980-04-01
+80/4/1 1:50:30 AM|1:50:30 AM|80/4/1
+1980 April 1 1:50:30|1:50:30|1980-04-01
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-core-tests/src/test/resources/queries/TestQueryOnOrcFile/datetime_table_timezoned_ddl.sql
----------------------------------------------------------------------
diff --git a/tajo-core-tests/src/test/resources/queries/TestQueryOnOrcFile/datetime_table_timezoned_ddl.sql b/tajo-core-tests/src/test/resources/queries/TestQueryOnOrcFile/datetime_table_timezoned_ddl.sql
new file mode 100644
index 0000000..9c5d30d
--- /dev/null
+++ b/tajo-core-tests/src/test/resources/queries/TestQueryOnOrcFile/datetime_table_timezoned_ddl.sql
@@ -0,0 +1,5 @@
+CREATE EXTERNAL TABLE ${0} (
+  t_timestamp  TIMESTAMP,
+  t_time       TIME,
+  t_date       DATE
+) USING TEXT WITH ('timezone' = 'GMT+9') LOCATION ${table.path}

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-core-tests/src/test/resources/queries/TestQueryOnOrcFile/datetime_table_timezoned_orc_ddl.sql
----------------------------------------------------------------------
diff --git a/tajo-core-tests/src/test/resources/queries/TestQueryOnOrcFile/datetime_table_timezoned_orc_ddl.sql b/tajo-core-tests/src/test/resources/queries/TestQueryOnOrcFile/datetime_table_timezoned_orc_ddl.sql
new file mode 100644
index 0000000..49e1f7e
--- /dev/null
+++ b/tajo-core-tests/src/test/resources/queries/TestQueryOnOrcFile/datetime_table_timezoned_orc_ddl.sql
@@ -0,0 +1,4 @@
+CREATE TABLE ${0} (
+  t_timestamp  TIMESTAMP,
+  t_date    DATE
+) USING ORC WITH ('timezone' = 'GMT+9')

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-core-tests/src/test/resources/queries/TestQueryOnOrcFile/testTimezone1.sql
----------------------------------------------------------------------
diff --git a/tajo-core-tests/src/test/resources/queries/TestQueryOnOrcFile/testTimezone1.sql b/tajo-core-tests/src/test/resources/queries/TestQueryOnOrcFile/testTimezone1.sql
new file mode 100644
index 0000000..2464c97
--- /dev/null
+++ b/tajo-core-tests/src/test/resources/queries/TestQueryOnOrcFile/testTimezone1.sql
@@ -0,0 +1 @@
+SELECT * FROM timezoned_orc;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-core-tests/src/test/resources/queries/TestSelectQuery/datetime_table_timezoned_orc_ddl.sql
----------------------------------------------------------------------
diff --git a/tajo-core-tests/src/test/resources/queries/TestSelectQuery/datetime_table_timezoned_orc_ddl.sql b/tajo-core-tests/src/test/resources/queries/TestSelectQuery/datetime_table_timezoned_orc_ddl.sql
deleted file mode 100644
index 49e1f7e..0000000
--- a/tajo-core-tests/src/test/resources/queries/TestSelectQuery/datetime_table_timezoned_orc_ddl.sql
+++ /dev/null
@@ -1,4 +0,0 @@
-CREATE TABLE ${0} (
-  t_timestamp  TIMESTAMP,
-  t_date    DATE
-) USING ORC WITH ('timezone' = 'GMT+9')

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-core-tests/src/test/resources/queries/TestSelectQuery/testTimezonedORCTable.sql
----------------------------------------------------------------------
diff --git a/tajo-core-tests/src/test/resources/queries/TestSelectQuery/testTimezonedORCTable.sql b/tajo-core-tests/src/test/resources/queries/TestSelectQuery/testTimezonedORCTable.sql
deleted file mode 100644
index 1d898bd..0000000
--- a/tajo-core-tests/src/test/resources/queries/TestSelectQuery/testTimezonedORCTable.sql
+++ /dev/null
@@ -1,2 +0,0 @@
-SET SESSION TIMEZONE = 'GMT+9';
-SELECT * FROM timezoned_orc;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-core-tests/src/test/resources/results/TestQueryOnOrcFile/testTimezone1.result
----------------------------------------------------------------------
diff --git a/tajo-core-tests/src/test/resources/results/TestQueryOnOrcFile/testTimezone1.result b/tajo-core-tests/src/test/resources/results/TestQueryOnOrcFile/testTimezone1.result
new file mode 100644
index 0000000..39f593b
--- /dev/null
+++ b/tajo-core-tests/src/test/resources/results/TestQueryOnOrcFile/testTimezone1.result
@@ -0,0 +1,5 @@
+t_timestamp,t_date
+-------------------------------
+1980-04-01 01:50:30.01,1980-04-01
+1980-04-01 01:50:30,1980-04-01
+1980-04-01 01:50:30,1980-04-01
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-core-tests/src/test/resources/results/TestQueryOnOrcFile/testTimezone2.result
----------------------------------------------------------------------
diff --git a/tajo-core-tests/src/test/resources/results/TestQueryOnOrcFile/testTimezone2.result b/tajo-core-tests/src/test/resources/results/TestQueryOnOrcFile/testTimezone2.result
new file mode 100644
index 0000000..c0e5cef
--- /dev/null
+++ b/tajo-core-tests/src/test/resources/results/TestQueryOnOrcFile/testTimezone2.result
@@ -0,0 +1,5 @@
+t_timestamp,t_date
+-------------------------------
+1980-03-31 17:50:30.01,1980-04-01
+1980-03-31 17:50:30,1980-04-01
+1980-03-31 17:50:30,1980-04-01
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-core-tests/src/test/resources/results/TestQueryOnOrcFile/testTimezone3.result
----------------------------------------------------------------------
diff --git a/tajo-core-tests/src/test/resources/results/TestQueryOnOrcFile/testTimezone3.result b/tajo-core-tests/src/test/resources/results/TestQueryOnOrcFile/testTimezone3.result
new file mode 100644
index 0000000..916f4be
--- /dev/null
+++ b/tajo-core-tests/src/test/resources/results/TestQueryOnOrcFile/testTimezone3.result
@@ -0,0 +1,5 @@
+t_timestamp,t_date
+-------------------------------
+1980-03-31 16:50:30.01,1980-04-01
+1980-03-31 16:50:30,1980-04-01
+1980-03-31 16:50:30,1980-04-01
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-core-tests/src/test/resources/results/TestQueryOnOrcFile/testTimezone4.result
----------------------------------------------------------------------
diff --git a/tajo-core-tests/src/test/resources/results/TestQueryOnOrcFile/testTimezone4.result b/tajo-core-tests/src/test/resources/results/TestQueryOnOrcFile/testTimezone4.result
new file mode 100644
index 0000000..98e0918
--- /dev/null
+++ b/tajo-core-tests/src/test/resources/results/TestQueryOnOrcFile/testTimezone4.result
@@ -0,0 +1,5 @@
+t_timestamp,t_date
+-------------------------------
+1980-03-31 11:50:30.01,1980-04-01
+1980-03-31 11:50:30,1980-04-01
+1980-03-31 11:50:30,1980-04-01
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-core-tests/src/test/resources/results/TestSelectQuery/testTimezonedORCTable.result
----------------------------------------------------------------------
diff --git a/tajo-core-tests/src/test/resources/results/TestSelectQuery/testTimezonedORCTable.result b/tajo-core-tests/src/test/resources/results/TestSelectQuery/testTimezonedORCTable.result
deleted file mode 100644
index 39f593b..0000000
--- a/tajo-core-tests/src/test/resources/results/TestSelectQuery/testTimezonedORCTable.result
+++ /dev/null
@@ -1,5 +0,0 @@
-t_timestamp,t_date
--------------------------------
-1980-04-01 01:50:30.01,1980-04-01
-1980-04-01 01:50:30,1980-04-01
-1980-04-01 01:50:30,1980-04-01
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-dist/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-dist/pom.xml b/tajo-dist/pom.xml
index 095f128..652ab84 100644
--- a/tajo-dist/pom.xml
+++ b/tajo-dist/pom.xml
@@ -154,22 +154,14 @@
                       run cp -r ${project.basedir}/src/main/conf .
                       run rm -rf lib/tajo-*-${project.version}.jar
 
-                      run mkdir hive
-                      run mv lib/hive-*.jar hive/
-
+                      run mkdir -p lib
+                      run cp -r $ROOT/tajo-storage/tajo-storage-hdfs/target/lib/hive-*.jar lib/
+                      
                       run mkdir -p share/jdbc-dist
                       run cp -r $ROOT/tajo-jdbc/target/tajo-jdbc-${project.version}-jar-with-dependencies.jar ./share/jdbc-dist/tajo-jdbc-${project.version}.jar
 
                       run mkdir -p extlib
 
-                      if [ -f $ROOT/tajo-catalog/tajo-catalog-drivers/tajo-hive/target/lib/parquet-hive-bundle-*.jar ]
-                      then
-                      run cp -r $ROOT/tajo-catalog/tajo-catalog-drivers/tajo-hive/target/lib/parquet-hive-bundle-*.jar lib/
-                      echo
-                      echo "Tajo installed parquet-hive-bundle library at: ${project.build.directory}/tajo-${project.version}"
-                      echo
-                      fi
-
                       echo
                       echo "Tajo dist layout available at: ${project.build.directory}/tajo-${project.version}"
                       echo

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-dist/src/main/bin/tajo
----------------------------------------------------------------------
diff --git a/tajo-dist/src/main/bin/tajo b/tajo-dist/src/main/bin/tajo
index c08c538..007e960 100755
--- a/tajo-dist/src/main/bin/tajo
+++ b/tajo-dist/src/main/bin/tajo
@@ -300,11 +300,15 @@ if [ ! -z ${HIVE_HOME} ] && [ -d ${HIVE_HOME} ] && [ -d ${HIVE_LIB} ]; then
     CLASSPATH=${CLASSPATH}:$f;
   done
 
-  for f in ${HIVE_LIB}/datanucleus-*.jar; do
+  for f in ${HIVE_LIB}/javax.jdo-*.jar; do
     CLASSPATH=${CLASSPATH}:$f;
   done
-else
-  for f in $TAJO_HOME/hive/*.jar; do
+
+  for f in ${HIVE_LIB}/log4j-core-*.jar; do
+    CLASSPATH=${CLASSPATH}:$f;
+  done
+
+  for f in ${HIVE_LIB}/datanucleus-*.jar; do
     CLASSPATH=${CLASSPATH}:$f;
   done
 fi

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-project/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-project/pom.xml b/tajo-project/pom.xml
index cd86d3b..27fa66b 100644
--- a/tajo-project/pom.xml
+++ b/tajo-project/pom.xml
@@ -36,10 +36,11 @@
     <hadoop.version>2.7.2</hadoop.version>
     <protobuf.version>2.5.0</protobuf.version>
     <hbase.version>1.1.1</hbase.version>
-    <hive.version>1.1.0</hive.version>
+    <hive.version>2.0.0</hive.version>
     <netty.version>4.0.34.Final</netty.version>
     <jersey.version>2.6</jersey.version>
     <jetty.version>6.1.26</jetty.version>
+    <parquet.version>1.8.1</parquet.version>
     <tajo.root>${project.parent.relativePath}/..</tajo.root>
     <extra.source.path>src/main/hadoop-${hadoop.version}</extra.source.path>
   </properties>

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-storage/tajo-storage-common/src/main/resources/storage-default.xml
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/main/resources/storage-default.xml b/tajo-storage/tajo-storage-common/src/main/resources/storage-default.xml
index 7f4661b..2454714 100644
--- a/tajo-storage/tajo-storage-common/src/main/resources/storage-default.xml
+++ b/tajo-storage/tajo-storage-common/src/main/resources/storage-default.xml
@@ -130,7 +130,7 @@
 
   <property>
     <name>tajo.storage.scanner-handler.orc.class</name>
-    <value>org.apache.tajo.storage.orc.ORCScanner</value>
+    <value>org.apache.tajo.storage.orc.OrcScanner</value>
   </property>
 
   <property>

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-storage/tajo-storage-common/src/test/resources/storage-default.xml
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-common/src/test/resources/storage-default.xml b/tajo-storage/tajo-storage-common/src/test/resources/storage-default.xml
index 934dd01..1c4530a 100644
--- a/tajo-storage/tajo-storage-common/src/test/resources/storage-default.xml
+++ b/tajo-storage/tajo-storage-common/src/test/resources/storage-default.xml
@@ -132,7 +132,7 @@
 
   <property>
     <name>tajo.storage.scanner-handler.orc.class</name>
-    <value>org.apache.tajo.storage.orc.ORCScanner</value>
+    <value>org.apache.tajo.storage.orc.OrcScanner</value>
   </property>
 
   <property>

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-storage/tajo-storage-hdfs/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/pom.xml b/tajo-storage/tajo-storage-hdfs/pom.xml
index 5f66395..aa6e6a6 100644
--- a/tajo-storage/tajo-storage-hdfs/pom.xml
+++ b/tajo-storage/tajo-storage-hdfs/pom.xml
@@ -34,7 +34,6 @@
   <properties>
     <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
     <project.reporting.outputEncoding>UTF-8</project.reporting.outputEncoding>
-    <parquet.version>1.8.1</parquet.version>
   </properties>
 
   <repositories>
@@ -129,7 +128,6 @@
                 <argument>--proto_path=../../tajo-catalog/tajo-catalog-common/src/main/proto</argument>
                 <argument>--java_out=target/generated-sources/proto</argument>
                 <argument>src/main/proto/StorageFragmentProtos.proto</argument>
-                <argument>src/main/proto/orc_proto.proto</argument>
               </arguments>
             </configuration>
             <goals>
@@ -161,6 +159,26 @@
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-surefire-report-plugin</artifactId>
       </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-dependency-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>copy-dependencies</id>
+            <phase>package</phase>
+            <goals>
+              <goal>copy-dependencies</goal>
+            </goals>
+            <configuration>
+              <includeScope>runtime</includeScope>
+              <outputDirectory>${project.build.directory}/lib</outputDirectory>
+              <overWriteReleases>false</overWriteReleases>
+              <overWriteSnapshots>false</overWriteSnapshots>
+              <overWriteIfNewer>true</overWriteIfNewer>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
     </plugins>
   </build>
 
@@ -345,10 +363,16 @@
       <artifactId>netty-buffer</artifactId>
     </dependency>
     <dependency>
-      <groupId>com.facebook.presto</groupId>
-      <artifactId>presto-orc</artifactId>
-      <version>0.141</version>
+      <groupId>org.apache.hive</groupId>
+      <artifactId>hive-orc</artifactId>
+      <version>${hive.version}</version>
     </dependency>
+    <dependency>
+      <groupId>org.apache.hive</groupId>
+      <artifactId>hive-storage-api</artifactId>
+      <version>${hive.version}</version>
+    </dependency>
+
   </dependencies>
 
   <profiles>

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/ORCAppender.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/ORCAppender.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/ORCAppender.java
index 7999d02..b27c640 100644
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/ORCAppender.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/ORCAppender.java
@@ -20,6 +20,9 @@ package org.apache.tajo.storage.orc;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
+import org.apache.orc.CompressionKind;
+import org.apache.orc.OrcConf;
+import org.apache.orc.TypeDescription;
 import org.apache.tajo.TajoConstants;
 import org.apache.tajo.TaskAttemptId;
 import org.apache.tajo.catalog.Schema;
@@ -29,12 +32,13 @@ import org.apache.tajo.storage.FileAppender;
 import org.apache.tajo.storage.StorageConstants;
 import org.apache.tajo.storage.TableStatistics;
 import org.apache.tajo.storage.Tuple;
-import org.apache.tajo.storage.orc.objectinspector.ObjectInspectorFactory;
-import org.apache.tajo.storage.thirdparty.orc.CompressionKind;
 import org.apache.tajo.storage.thirdparty.orc.OrcFile;
+import org.apache.tajo.storage.thirdparty.orc.OrcFile.EncodingStrategy;
+import org.apache.tajo.storage.thirdparty.orc.OrcUtils;
 import org.apache.tajo.storage.thirdparty.orc.Writer;
 
 import java.io.IOException;
+import java.util.Properties;
 import java.util.TimeZone;
 
 public class ORCAppender extends FileAppender {
@@ -46,21 +50,14 @@ public class ORCAppender extends FileAppender {
                      TableMeta meta, Path workDir) {
     super(conf, taskAttemptId, schema, meta, workDir);
 
-    timezone = TimeZone.getTimeZone(meta.getProperty(StorageConstants.TIMEZONE,
-        TajoConstants.DEFAULT_SYSTEM_TIMEZONE));
+    timezone = meta.containsProperty(StorageConstants.TIMEZONE) ?
+        TimeZone.getTimeZone(meta.getProperty(StorageConstants.TIMEZONE)) :
+        TimeZone.getDefault();
   }
 
   @Override
   public void init() throws IOException {
-    writer = OrcFile.createWriter(workDir.getFileSystem(conf), path, conf,
-      ObjectInspectorFactory.buildStructObjectInspector(schema),
-      Long.parseLong(meta.getProperty(StorageConstants.ORC_STRIPE_SIZE,
-        StorageConstants.DEFAULT_ORC_STRIPE_SIZE)), getCompressionKind(),
-      Integer.parseInt(meta.getProperty(StorageConstants.ORC_BUFFER_SIZE,
-        StorageConstants.DEFAULT_ORC_BUFFER_SIZE)),
-      Integer.parseInt(meta.getProperty(StorageConstants.ORC_ROW_INDEX_STRIDE,
-        StorageConstants.DEFAULT_ORC_ROW_INDEX_STRIDE)),
-      timezone);
+    writer = OrcFile.createWriter(path, buildWriterOptions(conf, meta, schema), timezone);
 
     if (tableStatsEnabled) {
       this.stats = new TableStatistics(schema, columnStatsEnabled);
@@ -90,7 +87,6 @@ public class ORCAppender extends FileAppender {
   public void close() throws IOException {
     writer.close();
 
-    // TODO: getOffset is not implemented yet
 //    if (tableStatsEnabled) {
 //      stats.setNumBytes(getOffset());
 //    }
@@ -107,24 +103,81 @@ public class ORCAppender extends FileAppender {
 
   @Override
   public long getEstimatedOutputSize() throws IOException {
-    return writer.getRawDataSize() * writer.getNumberOfRows();
+    return writer.getRawDataSize();
   }
 
-  private CompressionKind getCompressionKind() {
-    String kindstr = meta.getProperty(StorageConstants.ORC_COMPRESSION, StorageConstants.DEFAULT_ORC_COMPRESSION_KIND);
+  private static OrcFile.WriterOptions buildWriterOptions(Configuration conf, TableMeta meta, Schema schema) {
+    return OrcFile.writerOptions(conf)
+        .setSchema(OrcUtils.convertSchema(schema))
+        .compress(getCompressionKind(meta))
+        .stripeSize(Long.parseLong(meta.getProperty(OrcConf.STRIPE_SIZE.getAttribute(),
+            String.valueOf(OrcConf.STRIPE_SIZE.getDefaultValue()))))
+        .blockSize(Long.parseLong(meta.getProperty(OrcConf.BLOCK_SIZE.getAttribute(),
+            String.valueOf(OrcConf.BLOCK_SIZE.getDefaultValue()))))
+        .rowIndexStride(Integer.parseInt(meta.getProperty(OrcConf.ROW_INDEX_STRIDE.getAttribute(),
+            String.valueOf(OrcConf.ROW_INDEX_STRIDE.getDefaultValue()))))
+        .bufferSize(Integer.parseInt(meta.getProperty(OrcConf.BUFFER_SIZE.getAttribute(),
+            String.valueOf(OrcConf.BUFFER_SIZE.getDefaultValue()))))
+        .blockPadding(Boolean.parseBoolean(meta.getProperty(OrcConf.BLOCK_PADDING.getAttribute(),
+            String.valueOf(OrcConf.BLOCK_PADDING.getDefaultValue()))))
+        .encodingStrategy(EncodingStrategy.valueOf(meta.getProperty(OrcConf.ENCODING_STRATEGY.getAttribute(),
+            String.valueOf(OrcConf.ENCODING_STRATEGY.getDefaultValue()))))
+        .bloomFilterFpp(Double.parseDouble(meta.getProperty(OrcConf.BLOOM_FILTER_FPP.getAttribute(),
+            String.valueOf(OrcConf.BLOOM_FILTER_FPP.getDefaultValue()))))
+        .bloomFilterColumns(meta.getProperty(OrcConf.BLOOM_FILTER_COLUMNS.getAttribute(),
+            String.valueOf(OrcConf.BLOOM_FILTER_COLUMNS.getDefaultValue())));
+  }
+
+  private static CompressionKind getCompressionKind(TableMeta meta) {
+    String kindstr = meta.getProperty(OrcConf.COMPRESS.getAttribute(),
+        String.valueOf(OrcConf.COMPRESS.getDefaultValue()));
 
-    if (kindstr.equalsIgnoreCase(StorageConstants.ORC_COMPRESSION_KIND_ZIP)) {
+    if (kindstr.equalsIgnoreCase(CompressionKind.ZLIB.name())) {
       return CompressionKind.ZLIB;
     }
 
-    if (kindstr.equalsIgnoreCase(StorageConstants.ORC_COMPRESSION_KIND_SNAPPY)) {
+    if (kindstr.equalsIgnoreCase(CompressionKind.SNAPPY.name())) {
       return CompressionKind.SNAPPY;
     }
 
-    if (kindstr.equalsIgnoreCase(StorageConstants.ORC_COMPRESSION_KIND_LZO)) {
+    if (kindstr.equalsIgnoreCase(CompressionKind.LZO.name())) {
       return CompressionKind.LZO;
     }
 
     return CompressionKind.NONE;
   }
+
+  /**
+   * Options for creating ORC file writers.
+   */
+  public static class WriterOptions extends OrcFile.WriterOptions {
+    // Setting the default batch size to 1000 makes the memory check at 5000
+    // rows work the same as the row by row writer. (If it was the default 1024,
+    // the smallest stripe size would be 5120 rows, which changes the output
+    // of some of the tests.)
+    private int batchSize = 1000;
+
+    public WriterOptions(Properties tableProperties, Configuration conf) {
+      super(tableProperties, conf);
+    }
+
+    /**
+     * Set the schema for the file. This is a required parameter.
+     * @param schema the schema for the file.
+     * @return this
+     */
+    public WriterOptions setSchema(TypeDescription schema) {
+      super.setSchema(schema);
+      return this;
+    }
+
+    protected WriterOptions batchSize(int maxSize) {
+      batchSize = maxSize;
+      return this;
+    }
+
+    int getBatchSize() {
+      return batchSize;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/ORCScanner.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/ORCScanner.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/ORCScanner.java
deleted file mode 100644
index 0a4ebc6..0000000
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/ORCScanner.java
+++ /dev/null
@@ -1,332 +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.tajo.storage.orc;
-
-import com.facebook.presto.orc.OrcDataSource;
-import com.facebook.presto.orc.OrcPredicate;
-import com.facebook.presto.orc.OrcReader;
-import com.facebook.presto.orc.OrcRecordReader;
-import com.facebook.presto.orc.memory.AggregatedMemoryContext;
-import com.facebook.presto.orc.metadata.OrcMetadataReader;
-import com.facebook.presto.spi.block.Block;
-import com.facebook.presto.spi.type.*;
-import com.google.protobuf.InvalidProtocolBufferException;
-import io.airlift.units.DataSize;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.tajo.TajoConstants;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.catalog.TableMeta;
-import org.apache.tajo.common.TajoDataTypes;
-import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.datum.*;
-import org.apache.tajo.exception.NotImplementedException;
-import org.apache.tajo.exception.TajoRuntimeException;
-import org.apache.tajo.plan.expr.EvalNode;
-import org.apache.tajo.storage.FileScanner;
-import org.apache.tajo.storage.StorageConstants;
-import org.apache.tajo.storage.Tuple;
-import org.apache.tajo.storage.VTuple;
-import org.apache.tajo.storage.fragment.Fragment;
-import org.apache.tajo.storage.thirdparty.orc.HdfsOrcDataSource;
-import org.apache.tajo.util.datetime.DateTimeUtil;
-import org.joda.time.DateTimeZone;
-
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.TimeZone;
-
-/**
- * OrcScanner for reading ORC files
- */
-public class ORCScanner extends FileScanner {
-  private static final Log LOG = LogFactory.getLog(ORCScanner.class);
-  private OrcRecordReader recordReader;
-  private Block[] blocks;
-  private int currentPosInBatch = 0;
-  private int batchSize = 0;
-  private Tuple outTuple;
-  private AggregatedMemoryContext aggrMemoryContext = new AggregatedMemoryContext();
-
-  public ORCScanner(Configuration conf, final Schema schema, final TableMeta meta, final Fragment fragment) {
-    super(conf, schema, meta, fragment);
-  }
-
-  private FileSystem fs;
-  private FSDataInputStream fis;
-
-  private static class ColumnInfo {
-    TajoDataTypes.DataType type;
-    int id;
-  }
-
-  /**
-   * Temporary array for caching column info
-   */
-  private ColumnInfo [] targetColInfo;
-
-  @Override
-  public void init() throws IOException {
-    OrcReader orcReader;
-    DataSize maxMergeDistance = new DataSize(Double.parseDouble(meta.getProperty(StorageConstants.ORC_MAX_MERGE_DISTANCE,
-            StorageConstants.DEFAULT_ORC_MAX_MERGE_DISTANCE)), DataSize.Unit.BYTE);
-    DataSize maxReadSize = new DataSize(Double.parseDouble(meta.getProperty(StorageConstants.ORC_MAX_READ_BUFFER_SIZE,
-        StorageConstants.DEFAULT_ORC_MAX_READ_BUFFER_SIZE)), DataSize.Unit.BYTE);
-
-    if (targets == null) {
-      targets = schema.toArray();
-    }
-
-    outTuple = new VTuple(targets.length);
-
-    Path path = fragment.getPath();
-
-    if(fs == null) {
-      fs = FileScanner.getFileSystem((TajoConf)conf, path);
-    }
-
-    if(fis == null) {
-      fis = fs.open(path);
-    }
-
-    OrcDataSource orcDataSource = new HdfsOrcDataSource(
-        this.fragment.getPath().toString(),
-        fis,
-        fs.getFileStatus(path).getLen(),
-        maxMergeDistance,
-        maxReadSize);
-
-    targetColInfo = new ColumnInfo[targets.length];
-    for (int i=0; i<targets.length; i++) {
-      ColumnInfo cinfo = new ColumnInfo();
-      cinfo.type = targets[i].getDataType();
-      cinfo.id = schema.getColumnId(targets[i].getQualifiedName());
-      targetColInfo[i] = cinfo;
-    }
-
-    // creating blocks for buffering
-    blocks = new Block[targetColInfo.length];
-
-    Map<Integer, Type> columnMap = new HashMap<>();
-    for (ColumnInfo colInfo: targetColInfo) {
-      columnMap.put(colInfo.id, createFBtypeByTajoType(colInfo.type));
-    }
-
-    orcReader = new OrcReader(orcDataSource, new OrcMetadataReader(), maxMergeDistance, maxReadSize);
-
-    TimeZone timezone = TimeZone.getTimeZone(meta.getProperty(StorageConstants.TIMEZONE,
-      TajoConstants.DEFAULT_SYSTEM_TIMEZONE));
-
-    // TODO: make OrcPredicate useful
-    // presto-orc uses joda timezone, so it needs to be converted.
-    recordReader = orcReader.createRecordReader(columnMap, OrcPredicate.TRUE,
-        fragment.getStartKey(), fragment.getLength(), DateTimeZone.forTimeZone(timezone), aggrMemoryContext);
-
-    super.init();
-    LOG.debug("file fragment { path: " + fragment.getPath() +
-      ", start offset: " + fragment.getStartKey() +
-      ", length: " + fragment.getLength() + "}");
-  }
-
-  @Override
-  public Tuple next() throws IOException {
-    if (currentPosInBatch == batchSize) {
-      getNextBatch();
-
-      // EOF
-      if (batchSize == -1) {
-        return null;
-      }
-    }
-
-    for (int i=0; i<targetColInfo.length; i++) {
-      outTuple.put(i, createValueDatum(blocks[i], targetColInfo[i].type));
-    }
-
-    currentPosInBatch++;
-
-    return outTuple;
-  }
-
-  private Type createFBtypeByTajoType(TajoDataTypes.DataType type) {
-    switch(type.getType()) {
-      case BOOLEAN:
-        return BooleanType.BOOLEAN;
-
-      case INT1:
-      case INT2:
-      case INT4:
-      case INT8:
-      case INET4:
-      case NULL_TYPE: // meaningless
-        return BigintType.BIGINT;
-
-      case TIMESTAMP:
-        return TimestampType.TIMESTAMP;
-
-      case DATE:
-        return DateType.DATE;
-
-      case FLOAT4:
-      case FLOAT8:
-        return DoubleType.DOUBLE;
-
-      case CHAR:
-      case TEXT:
-        return VarcharType.VARCHAR;
-
-      case BLOB:
-      case PROTOBUF:
-        return VarbinaryType.VARBINARY;
-
-      default:
-        throw new TajoRuntimeException(new NotImplementedException(type.getType().name() + " for orc"));
-    }
-  }
-
-  // TODO: support more types
-  private Datum createValueDatum(Block block, TajoDataTypes.DataType type) {
-    if (block.isNull(currentPosInBatch))
-      return NullDatum.get();
-
-    // NOTE: block.get***() methods are determined by the type size wich is in createFBtypeByTajoType()
-    switch (type.getType()) {
-      case INT1:
-        return DatumFactory.createInt2((short)block.getLong(currentPosInBatch, 0));
-
-      case INT2:
-        return DatumFactory.createInt2((short)block.getLong(currentPosInBatch, 0));
-
-      case INT4:
-        return DatumFactory.createInt4((int)block.getLong(currentPosInBatch, 0));
-
-      case INT8:
-        return DatumFactory.createInt8(block.getLong(currentPosInBatch, 0));
-
-      case FLOAT4:
-        return DatumFactory.createFloat4((float)block.getDouble(currentPosInBatch, 0));
-
-      case FLOAT8:
-        return DatumFactory.createFloat8(block.getDouble(currentPosInBatch, 0));
-
-      case BOOLEAN:
-        return DatumFactory.createBool(block.getByte(currentPosInBatch, 0) != 0);
-
-      case CHAR:
-        return DatumFactory.createChar(block.getSlice(currentPosInBatch, 0,
-            block.getLength(currentPosInBatch)).getBytes());
-
-      case TEXT:
-        return DatumFactory.createText(block.getSlice(currentPosInBatch, 0,
-            block.getLength(currentPosInBatch)).getBytes());
-
-      case BLOB:
-        return DatumFactory.createBlob(block.getSlice(currentPosInBatch, 0,
-            block.getLength(currentPosInBatch)).getBytes());
-
-      case PROTOBUF:
-        try {
-          return ProtobufDatumFactory.createDatum(type, block.getSlice(currentPosInBatch, 0,
-              block.getLength(currentPosInBatch)).getBytes());
-        } catch (InvalidProtocolBufferException e) {
-          LOG.error("ERROR", e);
-          return NullDatum.get();
-        }
-
-      case TIMESTAMP:
-        return DatumFactory.createTimestamp(
-            DateTimeUtil.javaTimeToJulianTime(block.getLong(currentPosInBatch, 0)));
-
-      case DATE:
-        return DatumFactory.createDate(
-            block.getInt(currentPosInBatch, 0) + DateTimeUtil.DAYS_FROM_JULIAN_TO_EPOCH);
-
-      case INET4:
-        return DatumFactory.createInet4((int)block.getLong(currentPosInBatch, 0));
-
-      case NULL_TYPE:
-        return NullDatum.get();
-
-      default:
-        throw new TajoRuntimeException(new NotImplementedException(type.getType().name() + " for orc"));
-    }
-  }
-
-  /**
-   * Fetch next batch from ORC file and write to block data structure as many as batch size
-   *
-   * @throws IOException
-   */
-  private void getNextBatch() throws IOException {
-    batchSize = recordReader.nextBatch();
-
-    // end of file
-    if (batchSize == -1)
-      return;
-
-    for (int i=0; i<targetColInfo.length; i++) {
-      blocks[i] = recordReader.readBlock(createFBtypeByTajoType(targetColInfo[i].type), targetColInfo[i].id);
-    }
-
-    currentPosInBatch = 0;
-  }
-
-  @Override
-  public float getProgress() {
-    if(!inited) return super.getProgress();
-
-    return recordReader.getProgress();
-  }
-
-  @Override
-  public void reset() throws IOException {
-  }
-
-  @Override
-  public void close() throws IOException {
-    if (recordReader != null) {
-      recordReader.close();
-    }
-  }
-
-  @Override
-  public boolean isProjectable() {
-    return true;
-  }
-
-  @Override
-  public boolean isSelectable() {
-    return false;
-  }
-
-  @Override
-  public void setFilter(EvalNode filter) {
-    // TODO: implement it
-  }
-
-  @Override
-  public boolean isSplittable() {
-    return true;
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/OrcScanner.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/OrcScanner.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/OrcScanner.java
new file mode 100644
index 0000000..c8aa67b
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/OrcScanner.java
@@ -0,0 +1,460 @@
+/*
+ * 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.tajo.storage.orc;
+
+import com.google.common.collect.Lists;
+import com.google.protobuf.CodedInputStream;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
+import org.apache.orc.*;
+import org.apache.orc.Reader.Options;
+import org.apache.orc.impl.BufferChunk;
+import org.apache.orc.impl.InStream;
+import org.apache.tajo.TajoConstants;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.plan.expr.EvalNode;
+import org.apache.tajo.storage.FileScanner;
+import org.apache.tajo.storage.StorageConstants;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.storage.fragment.Fragment;
+import org.apache.tajo.storage.thirdparty.orc.OrcRecordReader;
+import org.apache.tajo.storage.thirdparty.orc.OrcUtils;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.TimeZone;
+
+public class OrcScanner extends FileScanner {
+  private static final Log LOG = LogFactory.getLog(OrcScanner.class);
+
+  private static final int DIRECTORY_SIZE_GUESS = 16 * 1024;
+
+  protected final FileSystem fileSystem;
+  private final long maxLength = Long.MAX_VALUE;
+  protected final Path path;
+  protected org.apache.orc.CompressionKind compressionKind;
+  protected CompressionCodec codec;
+  protected int bufferSize;
+  private List<OrcProto.StripeStatistics> stripeStats;
+  private int metadataSize;
+  protected List<OrcProto.Type> types;
+  private List<OrcProto.UserMetadataItem> userMetadata;
+  private List<OrcProto.ColumnStatistics> fileStats;
+  private List<StripeInformation> stripes;
+  protected int rowIndexStride;
+  private long contentLength, numberOfRows;
+
+  private List<Integer> versionList;
+
+  //serialized footer - Keeping this around for use by getFileMetaInfo()
+  // will help avoid cpu cycles spend in deserializing at cost of increased
+  // memory footprint.
+  private ByteBuffer footerByteBuffer;
+  // 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 ByteBuffer footerMetaAndPsBuffer;
+
+  private OrcRecordReader recordReader;
+
+  private long recordCount = 0;
+
+  /**
+   * 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 len = OrcFile.MAGIC.length();
+    if (psLen < len + 1) {
+      throw new IOException("Malformed ORC file " + path +
+          ". Invalid postscript length " + psLen);
+    }
+    int offset = buffer.arrayOffset() + buffer.position() + buffer.limit() - 1 - len;
+    byte[] array = buffer.array();
+    // now look for the magic string at the end of the postscript.
+    if (!Text.decode(array, offset, len).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[len];
+      in.readFully(0, header, 0, len);
+      // if it isn't there, this isn't an ORC file
+      if (!Text.decode(header, 0 , len).equals(OrcFile.MAGIC)) {
+        throw new IOException("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(Log 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.");
+      }
+    }
+  }
+
+  public OrcScanner(Configuration conf, Schema schema, TableMeta meta, Fragment fragment) throws IOException {
+    super(conf, schema, meta, fragment);
+
+    this.path = this.fragment.getPath();
+    this.fileSystem = this.path.getFileSystem(conf);
+  }
+
+  private static FileMetaInfo extractMetaInfoFromFooter(FileSystem fs,
+                                                        Path path,
+                                                        long maxFileLength
+  ) throws IOException {
+    FSDataInputStream file = fs.open(path);
+
+    // 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);
+    ByteBuffer 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;
+    OrcProto.PostScript ps = extractPostScript(buffer, path, psLen, psOffset);
+
+    int footerSize = (int) ps.getFooterLength();
+    int metadataSize = (int) ps.getMetadataLength();
+
+    //check if extra bytes need to be read
+    ByteBuffer fullFooterBuffer = null;
+    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
+    buffer.mark();
+
+    file.close();
+
+    return new FileMetaInfo(
+        ps.getCompression().toString(),
+        (int) ps.getCompressionBlockSize(),
+        (int) ps.getMetadataLength(),
+        buffer,
+        ps.getVersionList(),
+        org.apache.orc.OrcFile.WriterVersion.FUTURE,
+        fullFooterBuffer
+    );
+  }
+
+  public OrcRecordReader createRecordReader() throws IOException {
+    return new OrcRecordReader(this.stripes, fileSystem, schema, targets, fragment, types, codec, bufferSize,
+        rowIndexStride, buildReaderOptions(meta), conf,
+        TimeZone.getTimeZone(meta.getProperty(StorageConstants.TIMEZONE, TajoConstants.DEFAULT_SYSTEM_TIMEZONE)));
+  }
+
+  private static Options buildReaderOptions(TableMeta meta) {
+    return new Options()
+        .useZeroCopy(Boolean.parseBoolean(meta.getProperty(OrcConf.USE_ZEROCOPY.getAttribute(),
+            String.valueOf(OrcConf.USE_ZEROCOPY.getDefaultValue()))))
+        .skipCorruptRecords(Boolean.parseBoolean(meta.getProperty(OrcConf.SKIP_CORRUPT_DATA.getAttribute(),
+            String.valueOf(OrcConf.SKIP_CORRUPT_DATA.getDefaultValue()))));
+  }
+
+  @Override
+  public void init() throws IOException {
+    FileMetaInfo footerMetaData = extractMetaInfoFromFooter(fileSystem, path, maxLength);
+    this.footerMetaAndPsBuffer = footerMetaData.footerMetaAndPsBuffer;
+    MetaInfoObjExtractor rInfo =
+        new MetaInfoObjExtractor(footerMetaData.compressionType,
+            footerMetaData.bufferSize,
+            footerMetaData.metadataSize,
+            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.versionList = footerMetaData.versionList;
+    this.stripes = convertProtoStripesToStripes(rInfo.footer.getStripesList());
+
+    recordReader = createRecordReader();
+
+    super.init();
+  }
+
+  @Override
+  public Tuple next() throws IOException {
+    Tuple next = recordReader.next();
+    if (next != null) {
+      recordCount++;
+    }
+    return next;
+  }
+
+  @Override
+  public void reset() throws IOException {
+    // TODO: improve this
+    this.close();
+    recordReader = createRecordReader();
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (recordReader != null) {
+      recordReader.close();
+      tableStats.setNumBytes(recordReader.getNumBytes());
+      tableStats.setNumRows(recordCount);
+    }
+  }
+
+  @Override
+  public boolean isProjectable() {
+    return true;
+  }
+
+  @Override
+  public boolean isSelectable() {
+    return false;
+  }
+
+  @Override
+  public void setFilter(EvalNode filter) {
+    // TODO: implement this
+  }
+
+  @Override
+  public float getProgress() {
+    return inited ? recordReader.getProgress() : super.getProgress();
+  }
+
+  @Override
+  public boolean isSplittable() {
+    return true;
+  }
+
+  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 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.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.newArrayList(new BufferChunk(bb, 0)), metadataSize, codec, bufferSize));
+  }
+
+  /**
+   * 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);
+      this.bufferSize = bufferSize;
+      this.codec = OrcUtils.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);
+    }
+  }
+
+  public static class StripeInformationImpl
+      implements org.apache.orc.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();
+    }
+  }
+
+  private static List<StripeInformation> convertProtoStripesToStripes(
+      List<OrcProto.StripeInformation> stripes) {
+    List<StripeInformation> result = new ArrayList<>(stripes.size());
+    for (OrcProto.StripeInformation info : stripes) {
+      result.add(new StripeInformationImpl(info));
+    }
+    return result;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/objectinspector/ObjectInspectorFactory.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/objectinspector/ObjectInspectorFactory.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/objectinspector/ObjectInspectorFactory.java
deleted file mode 100644
index 061ba0d..0000000
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/orc/objectinspector/ObjectInspectorFactory.java
+++ /dev/null
@@ -1,91 +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.tajo.storage.orc.objectinspector;
-
-import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.common.TajoDataTypes;
-import org.apache.tajo.exception.UnsupportedException;
-
-public class ObjectInspectorFactory {
-
-  public static StructObjectInspector buildStructObjectInspector(Schema schema) {
-    StructObjectInspector structOI = new TajoStructObjectInspector(schema);
-    return structOI;
-  }
-
-  public static ObjectInspector buildObjectInspectorByType(TajoDataTypes.Type dataType) throws UnsupportedException {
-    ObjectInspector oi = null;
-
-    switch(dataType) {
-      case BOOLEAN:
-        oi = new TajoBooleanObjectInspector();
-        break;
-
-      case INT2:
-        oi = new TajoShortObjectInspector();
-        break;
-
-      case INET4:
-      case INT4:
-        oi = new TajoIntObjectInspector();
-        break;
-
-      case INT8:
-        oi = new TajoLongObjectInspector();
-        break;
-
-      case FLOAT4:
-        oi = new TajoFloatObjectInspector();
-        break;
-
-      case FLOAT8:
-        oi = new TajoDoubleObjectInspector();
-        break;
-
-      case TEXT:
-      case CHAR:
-        oi = new TajoStringObjectInspector();
-        break;
-
-      case TIMESTAMP:
-        oi = new TajoTimestampObjectInspector();
-        break;
-
-      case DATE:
-        oi = new TajoDateObjectInspector();
-        break;
-
-      case BLOB:
-      case PROTOBUF:
-        oi = new TajoBlobObjectInspector();
-        break;
-
-      case NULL_TYPE:
-        oi = new TajoNullObjectInspector();
-        break;
-
-      default:
-        throw new UnsupportedException(dataType.name()+" is not supported yet in OrcAppender");
-    }
-
-    return oi;
-  }
-}


[5/7] tajo git commit: TAJO-2102: Migrate to Apache Orc from Presto's one.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/DecimalColumnStatistics.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/DecimalColumnStatistics.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/DecimalColumnStatistics.java
deleted file mode 100644
index 27cdac2..0000000
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/DecimalColumnStatistics.java
+++ /dev/null
@@ -1,45 +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.tajo.storage.thirdparty.orc;
-
-import org.apache.hadoop.hive.common.type.HiveDecimal;
-
-/**
- * Statistics for decimal columns.
- */
-public interface DecimalColumnStatistics extends ColumnStatistics {
-
-  /**
-   * Get the minimum value for the column.
-   * @return the minimum value
-   */
-  HiveDecimal getMinimum();
-
-  /**
-   * Get the maximum value for the column.
-   * @return the maximum value
-   */
-  HiveDecimal getMaximum();
-
-  /**
-   * Get the sum of the values of the column.
-   * @return the sum
-   */
-  HiveDecimal getSum();
-
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/DirectDecompressionCodec.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/DirectDecompressionCodec.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/DirectDecompressionCodec.java
deleted file mode 100644
index 5333052..0000000
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/DirectDecompressionCodec.java
+++ /dev/null
@@ -1,26 +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.tajo.storage.thirdparty.orc;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-public interface DirectDecompressionCodec extends CompressionCodec {
-  public boolean isAvailable();
-  public void directDecompress(ByteBuffer in, ByteBuffer out) throws IOException;
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/DoubleColumnStatistics.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/DoubleColumnStatistics.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/DoubleColumnStatistics.java
deleted file mode 100644
index ddce8f7..0000000
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/DoubleColumnStatistics.java
+++ /dev/null
@@ -1,44 +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.tajo.storage.thirdparty.orc;
-
-/**
- * Statistics for float and double columns.
- */
-public interface DoubleColumnStatistics extends ColumnStatistics {
-
-  /**
-   * Get the smallest value in the column. Only defined if getNumberOfValues
-   * is non-zero.
-   * @return the minimum
-   */
-  double getMinimum();
-
-  /**
-   * Get the largest value in the column. Only defined if getNumberOfValues
-   * is non-zero.
-   * @return the maximum
-   */
-  double getMaximum();
-
-  /**
-   * Get the sum of the values in the column.
-   * @return the sum
-   */
-  double getSum();
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/DynamicByteArray.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/DynamicByteArray.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/DynamicByteArray.java
deleted file mode 100644
index 1d44f77..0000000
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/DynamicByteArray.java
+++ /dev/null
@@ -1,303 +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.tajo.storage.thirdparty.orc;
-
-import org.apache.hadoop.io.Text;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.nio.ByteBuffer;
-
-/**
- * A class that is a growable array of bytes. Growth is managed in terms of
- * chunks that are allocated when needed.
- */
-final class DynamicByteArray {
-  static final int DEFAULT_CHUNKSIZE = 32 * 1024;
-  static final int DEFAULT_NUM_CHUNKS = 128;
-
-  private final int chunkSize;        // our allocation sizes
-  private byte[][] data;              // the real data
-  private int length;                 // max set element index +1
-  private int initializedChunks = 0;  // the number of chunks created
-
-  public DynamicByteArray() {
-    this(DEFAULT_NUM_CHUNKS, DEFAULT_CHUNKSIZE);
-  }
-
-  public DynamicByteArray(int numChunks, int chunkSize) {
-    if (chunkSize == 0) {
-      throw new IllegalArgumentException("bad chunksize");
-    }
-    this.chunkSize = chunkSize;
-    data = new byte[numChunks][];
-  }
-
-  /**
-   * Ensure that the given index is valid.
-   */
-  private void grow(int chunkIndex) {
-    if (chunkIndex >= initializedChunks) {
-      if (chunkIndex >= data.length) {
-        int newSize = Math.max(chunkIndex + 1, 2 * data.length);
-        byte[][] newChunk = new byte[newSize][];
-        System.arraycopy(data, 0, newChunk, 0, data.length);
-        data = newChunk;
-      }
-      for(int i=initializedChunks; i <= chunkIndex; ++i) {
-        data[i] = new byte[chunkSize];
-      }
-      initializedChunks = chunkIndex + 1;
-    }
-  }
-
-  public byte get(int index) {
-    if (index >= length) {
-      throw new IndexOutOfBoundsException("Index " + index +
-                                            " is outside of 0.." +
-                                            (length - 1));
-    }
-    int i = index / chunkSize;
-    int j = index % chunkSize;
-    return data[i][j];
-  }
-
-  public void set(int index, byte value) {
-    int i = index / chunkSize;
-    int j = index % chunkSize;
-    grow(i);
-    if (index >= length) {
-      length = index + 1;
-    }
-    data[i][j] = value;
-  }
-
-  public int add(byte value) {
-    int i = length / chunkSize;
-    int j = length % chunkSize;
-    grow(i);
-    data[i][j] = value;
-    int result = length;
-    length += 1;
-    return result;
-  }
-
-  /**
-   * Copy a slice of a byte array into our buffer.
-   * @param value the array to copy from
-   * @param valueOffset the first location to copy from value
-   * @param valueLength the number of bytes to copy from value
-   * @return the offset of the start of the value
-   */
-  public int add(byte[] value, int valueOffset, int valueLength) {
-    int i = length / chunkSize;
-    int j = length % chunkSize;
-    grow((length + valueLength) / chunkSize);
-    int remaining = valueLength;
-    while (remaining > 0) {
-      int size = Math.min(remaining, chunkSize - j);
-      System.arraycopy(value, valueOffset, data[i], j, size);
-      remaining -= size;
-      valueOffset += size;
-      i += 1;
-      j = 0;
-    }
-    int result = length;
-    length += valueLength;
-    return result;
-  }
-
-  /**
-   * Read the entire stream into this array.
-   * @param in the stream to read from
-   * @throws IOException
-   */
-  public void readAll(InputStream in) throws IOException {
-    int currentChunk = length / chunkSize;
-    int currentOffset = length % chunkSize;
-    grow(currentChunk);
-    int currentLength = in.read(data[currentChunk], currentOffset,
-      chunkSize - currentOffset);
-    while (currentLength > 0) {
-      length += currentLength;
-      currentOffset = length % chunkSize;
-      if (currentOffset == 0) {
-        currentChunk = length / chunkSize;
-        grow(currentChunk);
-      }
-      currentLength = in.read(data[currentChunk], currentOffset,
-        chunkSize - currentOffset);
-    }
-  }
-
-  /**
-   * Byte compare a set of bytes against the bytes in this dynamic array.
-   * @param other source of the other bytes
-   * @param otherOffset start offset in the other array
-   * @param otherLength number of bytes in the other array
-   * @param ourOffset the offset in our array
-   * @param ourLength the number of bytes in our array
-   * @return negative for less, 0 for equal, positive for greater
-   */
-  public int compare(byte[] other, int otherOffset, int otherLength,
-                     int ourOffset, int ourLength) {
-    int currentChunk = ourOffset / chunkSize;
-    int currentOffset = ourOffset % chunkSize;
-    int maxLength = Math.min(otherLength, ourLength);
-    while (maxLength > 0 &&
-      other[otherOffset] == data[currentChunk][currentOffset]) {
-      otherOffset += 1;
-      currentOffset += 1;
-      if (currentOffset == chunkSize) {
-        currentChunk += 1;
-        currentOffset = 0;
-      }
-      maxLength -= 1;
-    }
-    if (maxLength == 0) {
-      return otherLength - ourLength;
-    }
-    int otherByte = 0xff & other[otherOffset];
-    int ourByte = 0xff & data[currentChunk][currentOffset];
-    return otherByte > ourByte ? 1 : -1;
-  }
-
-  /**
-   * Get the size of the array.
-   * @return the number of bytes in the array
-   */
-  public int size() {
-    return length;
-  }
-
-  /**
-   * Clear the array to its original pristine state.
-   */
-  public void clear() {
-    length = 0;
-    for(int i=0; i < data.length; ++i) {
-      data[i] = null;
-    }
-    initializedChunks = 0;
-  }
-
-  /**
-   * Set a text value from the bytes in this dynamic array.
-   * @param result the value to set
-   * @param offset the start of the bytes to copy
-   * @param length the number of bytes to copy
-   */
-  public void setText(Text result, int offset, int length) {
-    result.clear();
-    int currentChunk = offset / chunkSize;
-    int currentOffset = offset % chunkSize;
-    int currentLength = Math.min(length, chunkSize - currentOffset);
-    while (length > 0) {
-      result.append(data[currentChunk], currentOffset, currentLength);
-      length -= currentLength;
-      currentChunk += 1;
-      currentOffset = 0;
-      currentLength = Math.min(length, chunkSize - currentOffset);
-    }
-  }
-
-  /**
-   * Write out a range of this dynamic array to an output stream.
-   * @param out the stream to write to
-   * @param offset the first offset to write
-   * @param length the number of bytes to write
-   * @throws IOException
-   */
-  public void write(OutputStream out, int offset,
-                    int length) throws IOException {
-    int currentChunk = offset / chunkSize;
-    int currentOffset = offset % chunkSize;
-    while (length > 0) {
-      int currentLength = Math.min(length, chunkSize - currentOffset);
-      out.write(data[currentChunk], currentOffset, currentLength);
-      length -= currentLength;
-      currentChunk += 1;
-      currentOffset = 0;
-    }
-  }
-
-  @Override
-  public String toString() {
-    int i;
-    StringBuilder sb = new StringBuilder(length * 3);
-
-    sb.append('{');
-    int l = length - 1;
-    for (i=0; i<l; i++) {
-      sb.append(Integer.toHexString(get(i)));
-      sb.append(',');
-    }
-    sb.append(get(i));
-    sb.append('}');
-
-    return sb.toString();
-  }
-
-  public void setByteBuffer(ByteBuffer result, int offset, int length) {
-    result.clear();
-    int currentChunk = offset / chunkSize;
-    int currentOffset = offset % chunkSize;
-    int currentLength = Math.min(length, chunkSize - currentOffset);
-    while (length > 0) {
-      result.put(data[currentChunk], currentOffset, currentLength);
-      length -= currentLength;
-      currentChunk += 1;
-      currentOffset = 0;
-      currentLength = Math.min(length, chunkSize - currentOffset);
-    }
-  }
-
-  /**
-   * Gets all the bytes of the array.
-   *
-   * @return Bytes of the array
-   */
-  public byte[] get() {
-    byte[] result = null;
-    if (length > 0) {
-      int currentChunk = 0;
-      int currentOffset = 0;
-      int currentLength = Math.min(length, chunkSize);
-      int destOffset = 0;
-      result = new byte[length];
-      int totalLength = length;
-      while (totalLength > 0) {
-        System.arraycopy(data[currentChunk], currentOffset, result, destOffset, currentLength);
-        destOffset += currentLength;
-        totalLength -= currentLength;
-        currentChunk += 1;
-        currentOffset = 0;
-        currentLength = Math.min(totalLength, chunkSize - currentOffset);
-      }
-    }
-    return result;
-  }
-
-  /**
-   * Get the size of the buffers.
-   */
-  public long getSizeInBytes() {
-    return initializedChunks * chunkSize;
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/DynamicIntArray.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/DynamicIntArray.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/DynamicIntArray.java
deleted file mode 100644
index a347706..0000000
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/DynamicIntArray.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.tajo.storage.thirdparty.orc;
-
-/**
- * Dynamic int array that uses primitive types and chunks to avoid copying
- * large number of integers when it resizes.
- *
- * The motivation for this class is memory optimization, i.e. space efficient
- * storage of potentially huge arrays without good a-priori size guesses.
- *
- * The API of this class is between a primitive array and a AbstractList. It's
- * not a Collection implementation because it handles primitive types, but the
- * API could be extended to support iterators and the like.
- *
- * NOTE: Like standard Collection implementations/arrays, this class is not
- * synchronized.
- */
-final class DynamicIntArray {
-  static final int DEFAULT_CHUNKSIZE = 8 * 1024;
-  static final int INIT_CHUNKS = 128;
-
-  private final int chunkSize;       // our allocation size
-  private int[][] data;              // the real data
-  private int length;                // max set element index +1
-  private int initializedChunks = 0; // the number of created chunks
-
-  public DynamicIntArray() {
-    this(DEFAULT_CHUNKSIZE);
-  }
-
-  public DynamicIntArray(int chunkSize) {
-    this.chunkSize = chunkSize;
-
-    data = new int[INIT_CHUNKS][];
-  }
-
-  /**
-   * Ensure that the given index is valid.
-   */
-  private void grow(int chunkIndex) {
-    if (chunkIndex >= initializedChunks) {
-      if (chunkIndex >= data.length) {
-        int newSize = Math.max(chunkIndex + 1, 2 * data.length);
-        int[][] newChunk = new int[newSize][];
-        System.arraycopy(data, 0, newChunk, 0, data.length);
-        data = newChunk;
-      }
-      for (int i=initializedChunks; i <= chunkIndex; ++i) {
-        data[i] = new int[chunkSize];
-      }
-      initializedChunks = chunkIndex + 1;
-    }
-  }
-
-  public int get(int index) {
-    if (index >= length) {
-      throw new IndexOutOfBoundsException("Index " + index +
-                                            " is outside of 0.." +
-                                            (length - 1));
-    }
-    int i = index / chunkSize;
-    int j = index % chunkSize;
-    return data[i][j];
-  }
-
-  public void set(int index, int value) {
-    int i = index / chunkSize;
-    int j = index % chunkSize;
-    grow(i);
-    if (index >= length) {
-      length = index + 1;
-    }
-    data[i][j] = value;
-  }
-
-  public void increment(int index, int value) {
-    int i = index / chunkSize;
-    int j = index % chunkSize;
-    grow(i);
-    if (index >= length) {
-      length = index + 1;
-    }
-    data[i][j] += value;
-  }
-
-  public void add(int value) {
-    int i = length / chunkSize;
-    int j = length % chunkSize;
-    grow(i);
-    data[i][j] = value;
-    length += 1;
-  }
-
-  public int size() {
-    return length;
-  }
-
-  public void clear() {
-    length = 0;
-    for(int i=0; i < data.length; ++i) {
-      data[i] = null;
-    }
-    initializedChunks = 0;
-  }
-
-  public String toString() {
-    int i;
-    StringBuilder sb = new StringBuilder(length * 4);
-
-    sb.append('{');
-    int l = length - 1;
-    for (i=0; i<l; i++) {
-      sb.append(get(i));
-      sb.append(',');
-    }
-    sb.append(get(i));
-    sb.append('}');
-
-    return sb.toString();
-  }
-
-  public int getSizeInBytes() {
-    return 4 * initializedChunks * chunkSize;
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/HdfsOrcDataSource.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/HdfsOrcDataSource.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/HdfsOrcDataSource.java
deleted file mode 100644
index 5357f51..0000000
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/HdfsOrcDataSource.java
+++ /dev/null
@@ -1,133 +0,0 @@
-
-/*
- * 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.tajo.storage.thirdparty.orc;
-
-import com.facebook.presto.orc.DiskRange;
-import com.facebook.presto.orc.OrcDataSource;
-import com.google.common.collect.ImmutableMap;
-import io.airlift.slice.BasicSliceInput;
-import io.airlift.slice.FixedLengthSliceInput;
-import io.airlift.units.DataSize;
-import org.apache.hadoop.fs.FSDataInputStream;
-
-import java.io.IOException;
-import java.util.LinkedHashMap;
-import java.util.Map;
-import java.util.Map.Entry;
-
-import static com.facebook.presto.orc.OrcDataSourceUtils.getDiskRangeSlice;
-import static com.facebook.presto.orc.OrcDataSourceUtils.mergeAdjacentDiskRanges;
-import static com.google.common.base.Preconditions.checkArgument;
-import static com.google.common.base.Preconditions.checkNotNull;
-
-/**
- * HDFS File data source class for Orc Reader
- *
- * Most of code is from Presto
- */
-public class HdfsOrcDataSource
-  implements OrcDataSource
-{
-  private final FSDataInputStream inputStream;
-  private final String path;
-  private final long size;
-  private final DataSize maxMergeDistance;
-  private final DataSize maxReadSize;
-  private long readTimeNanos;
-
-  public HdfsOrcDataSource(String path, FSDataInputStream inputStream, long size,
-                           DataSize maxMergeDistance, DataSize maxReadSize)
-  {
-    this.path = checkNotNull(path, "path is null");
-    this.inputStream = checkNotNull(inputStream, "inputStream is null");
-    this.size = size;
-    checkArgument(size >= 0, "size is negative");
-
-    this.maxMergeDistance = checkNotNull(maxMergeDistance, "maxMergeDistance is null");
-    this.maxReadSize = checkNotNull(maxReadSize, "maxMergeDistance is null");
-  }
-
-  @Override
-  public void close()
-    throws IOException
-  {
-    inputStream.close();
-  }
-
-  @Override
-  public long getReadTimeNanos()
-  {
-    return readTimeNanos;
-  }
-
-  @Override
-  public long getSize()
-  {
-    return size;
-  }
-
-  @Override
-  public void readFully(long position, byte[] buffer)
-    throws IOException
-  {
-    readFully(position, buffer, 0, buffer.length);
-  }
-
-  @Override
-  public void readFully(long position, byte[] buffer, int bufferOffset, int bufferLength)
-    throws IOException
-  {
-    long start = System.nanoTime();
-
-    inputStream.readFully(position, buffer, bufferOffset, bufferLength);
-    readTimeNanos += System.nanoTime() - start;
-  }
-
-  @Override
-  public <K> Map<K, FixedLengthSliceInput> readFully(Map<K, DiskRange> diskRanges)
-    throws IOException
-  {
-    checkNotNull(diskRanges, "diskRanges is null");
-
-    if (diskRanges.isEmpty()) {
-      return ImmutableMap.of();
-    }
-
-    Iterable<DiskRange> mergedRanges = mergeAdjacentDiskRanges(diskRanges.values(), maxMergeDistance, maxReadSize);
-
-    // read ranges
-    Map<DiskRange, byte[]> buffers = new LinkedHashMap<>();
-    for (DiskRange mergedRange : mergedRanges) {
-      // read full range in one request
-      byte[] buffer = new byte[mergedRange.getLength()];
-      readFully(mergedRange.getOffset(), buffer);
-      buffers.put(mergedRange, buffer);
-    }
-
-    ImmutableMap.Builder<K, FixedLengthSliceInput> slices = ImmutableMap.builder();
-    diskRanges.forEach((K key, DiskRange range) ->
-        slices.put(key, new BasicSliceInput(getDiskRangeSlice(range, buffers))));
-
-    return slices.build();
-  }
-
-  @Override
-  public String toString()
-  {
-    return path;
-  }
-}
-
-

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/IntegerColumnStatistics.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/IntegerColumnStatistics.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/IntegerColumnStatistics.java
deleted file mode 100644
index 208454f..0000000
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/IntegerColumnStatistics.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.tajo.storage.thirdparty.orc;
-
-/**
- * Statistics for all of the integer columns, such as byte, short, int, and
- * long.
- */
-public interface IntegerColumnStatistics extends ColumnStatistics {
-  /**
-   * Get the smallest value in the column. Only defined if getNumberOfValues
-   * is non-zero.
-   * @return the minimum
-   */
-  long getMinimum();
-
-  /**
-   * Get the largest value in the column. Only defined if getNumberOfValues
-   * is non-zero.
-   * @return the maximum
-   */
-  long getMaximum();
-
-  /**
-   * Is the sum defined? If the sum overflowed the counter this will be false.
-   * @return is the sum available
-   */
-  boolean isSumDefined();
-
-  /**
-   * Get the sum of the column. Only valid if isSumDefined returns true.
-   * @return the sum of the column
-   */
-  long getSum();
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/IntegerWriter.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/IntegerWriter.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/IntegerWriter.java
deleted file mode 100644
index 6872882..0000000
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/IntegerWriter.java
+++ /dev/null
@@ -1,47 +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.tajo.storage.thirdparty.orc;
-
-import java.io.IOException;
-
-/**
- * Interface for writing integers.
- */
-interface IntegerWriter {
-
-  /**
-   * Get position from the stream.
-   * @param recorder
-   * @throws IOException
-   */
-  void getPosition(PositionRecorder recorder) throws IOException;
-
-  /**
-   * Write the integer value
-   * @param value
-   * @throws IOException
-   */
-  void write(long value) throws IOException;
-
-  /**
-   * Flush the buffer
-   * @throws IOException
-   */
-  void flush() throws IOException;
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/MemoryManager.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/MemoryManager.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/MemoryManager.java
deleted file mode 100644
index 79af80f..0000000
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/MemoryManager.java
+++ /dev/null
@@ -1,212 +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.tajo.storage.thirdparty.orc;
-
-import com.google.common.base.Preconditions;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hive.conf.HiveConf;
-
-import java.io.IOException;
-import java.lang.management.ManagementFactory;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.concurrent.locks.ReentrantLock;
-
-/**
- * Implements a memory manager that keeps a global context of how many ORC
- * writers there are and manages the memory between them. For use cases with
- * dynamic partitions, it is easy to end up with many writers in the same task.
- * By managing the size of each allocation, we try to cut down the size of each
- * allocation and keep the task from running out of memory.
- *
- * This class is not thread safe, but is re-entrant - ensure creation and all
- * invocations are triggered from the same thread.
- */
-class MemoryManager {
-
-  private static final Log LOG = LogFactory.getLog(MemoryManager.class);
-
-  /**
-   * How often should we check the memory sizes? Measured in rows added
-   * to all of the writers.
-   */
-  private static final int ROWS_BETWEEN_CHECKS = 5000;
-  private final long totalMemoryPool;
-  private final Map<Path, WriterInfo> writerList =
-          new HashMap<>();
-  private long totalAllocation = 0;
-  private double currentScale = 1;
-  private int rowsAddedSinceCheck = 0;
-  private final OwnedLock ownerLock = new OwnedLock();
-
-  @SuppressWarnings("serial")
-  private static class OwnedLock extends ReentrantLock {
-    public Thread getOwner() {
-      return super.getOwner();
-    }
-  }
-
-  private static class WriterInfo {
-    long allocation;
-    Callback callback;
-    WriterInfo(long allocation, Callback callback) {
-      this.allocation = allocation;
-      this.callback = callback;
-    }
-  }
-
-  public interface Callback {
-    /**
-     * The writer needs to check its memory usage
-     * @param newScale the current scale factor for memory allocations
-     * @return true if the writer was over the limit
-     * @throws IOException
-     */
-    boolean checkMemory(double newScale) throws IOException;
-  }
-
-  /**
-   * Create the memory manager.
-   * @param conf use the configuration to find the maximum size of the memory
-   *             pool.
-   */
-  MemoryManager(Configuration conf) {
-    HiveConf.ConfVars poolVar = HiveConf.ConfVars.HIVE_ORC_FILE_MEMORY_POOL;
-    double maxLoad = conf.getFloat(poolVar.varname, poolVar.defaultFloatVal);
-    totalMemoryPool = Math.round(ManagementFactory.getMemoryMXBean().
-        getHeapMemoryUsage().getMax() * maxLoad);
-    ownerLock.lock();
-  }
-
-  /**
-   * Light weight thread-safety check for multi-threaded access patterns
-   */
-  private void checkOwner() {
-    Preconditions.checkArgument(ownerLock.isHeldByCurrentThread(),
-      "Owner thread expected %s, got %s",
-      ownerLock.getOwner(),
-      Thread.currentThread());
-  }
-
-  /**
-   * Add a new writer's memory allocation to the pool. We use the path
-   * as a unique key to ensure that we don't get duplicates.
-   * @param path the file that is being written
-   * @param requestedAllocation the requested buffer size
-   */
-  void addWriter(Path path, long requestedAllocation,
-                              Callback callback) throws IOException {
-    checkOwner();
-    WriterInfo oldVal = writerList.get(path);
-    // this should always be null, but we handle the case where the memory
-    // manager wasn't told that a writer wasn't still in use and the task
-    // starts writing to the same path.
-    if (oldVal == null) {
-      oldVal = new WriterInfo(requestedAllocation, callback);
-      writerList.put(path, oldVal);
-      totalAllocation += requestedAllocation;
-    } else {
-      // handle a new writer that is writing to the same path
-      totalAllocation += requestedAllocation - oldVal.allocation;
-      oldVal.allocation = requestedAllocation;
-      oldVal.callback = callback;
-    }
-    updateScale(true);
-  }
-
-  /**
-   * Remove the given writer from the pool.
-   * @param path the file that has been closed
-   */
-  void removeWriter(Path path) throws IOException {
-    checkOwner();
-    WriterInfo val = writerList.get(path);
-    if (val != null) {
-      writerList.remove(path);
-      totalAllocation -= val.allocation;
-      if (writerList.isEmpty()) {
-        rowsAddedSinceCheck = 0;
-      }
-      updateScale(false);
-    }
-    if(writerList.isEmpty()) {
-      rowsAddedSinceCheck = 0;
-    }
-  }
-
-  /**
-   * Get the total pool size that is available for ORC writers.
-   * @return the number of bytes in the pool
-   */
-  long getTotalMemoryPool() {
-    return totalMemoryPool;
-  }
-
-  /**
-   * The scaling factor for each allocation to ensure that the pool isn't
-   * oversubscribed.
-   * @return a fraction between 0.0 and 1.0 of the requested size that is
-   * available for each writer.
-   */
-  double getAllocationScale() {
-    return currentScale;
-  }
-
-  /**
-   * Give the memory manager an opportunity for doing a memory check.
-   * @throws IOException
-   */
-  void addedRow() throws IOException {
-    if (++rowsAddedSinceCheck >= ROWS_BETWEEN_CHECKS) {
-      notifyWriters();
-    }
-  }
-
-  /**
-   * Notify all of the writers that they should check their memory usage.
-   * @throws IOException
-   */
-  void notifyWriters() throws IOException {
-    checkOwner();
-    LOG.debug("Notifying writers after " + rowsAddedSinceCheck);
-    for(WriterInfo writer: writerList.values()) {
-      boolean flushed = writer.callback.checkMemory(currentScale);
-      if (LOG.isDebugEnabled() && flushed) {
-        LOG.debug("flushed " + writer.toString());
-      }
-    }
-    rowsAddedSinceCheck = 0;
-  }
-
-  /**
-   * Update the currentScale based on the current allocation and pool size.
-   * This also updates the notificationTrigger.
-   * @param isAllocate is this an allocation?
-   */
-  private void updateScale(boolean isAllocate) throws IOException {
-    if (totalAllocation <= totalMemoryPool) {
-      currentScale = 1;
-    } else {
-      currentScale = (double) totalMemoryPool / totalAllocation;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/Metadata.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/Metadata.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/Metadata.java
deleted file mode 100644
index dfa4c36..0000000
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/Metadata.java
+++ /dev/null
@@ -1,45 +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.tajo.storage.thirdparty.orc;
-
-import com.google.common.collect.Lists;
-
-import java.util.List;
-
-public class Metadata {
-
-  private final OrcProto.Metadata metadata;
-
-  Metadata(OrcProto.Metadata m) {
-    this.metadata = m;
-  }
-
-  /**
-   * Return list of stripe level column statistics
-   *
-   * @return list of stripe statistics
-   */
-  public List<StripeStatistics> getStripeStatistics() {
-    List<StripeStatistics> result = Lists.newArrayList();
-    for (OrcProto.StripeStatistics ss : metadata.getStripeStatsList()) {
-      result.add(new StripeStatistics(ss.getColStatsList()));
-    }
-    return result;
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/OrcFile.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/OrcFile.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/OrcFile.java
index a291953..8f26d21 100644
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/OrcFile.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/OrcFile.java
@@ -21,11 +21,15 @@ package org.apache.tajo.storage.thirdparty.orc;
 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 static org.apache.tajo.storage.thirdparty.orc.OrcConf.ConfVars.*;
+import org.apache.orc.CompressionKind;
+import org.apache.orc.FileMetaInfo;
+import org.apache.orc.FileMetadata;
+import org.apache.orc.TypeDescription;
+import org.apache.orc.impl.MemoryManager;
+import org.apache.tajo.storage.orc.ORCAppender;
 
 import java.io.IOException;
+import java.util.Properties;
 import java.util.TimeZone;
 
 /**
@@ -50,9 +54,9 @@ public final class OrcFile {
    * prevent the new reader from reading ORC files generated by any released
    * version of Hive.
    */
-  public static enum Version {
+  public enum Version {
     V_0_11("0.11", 0, 11),
-      V_0_12("0.12", 0, 12);
+    V_0_12("0.12", 0, 12);
 
     public static final Version CURRENT = V_0_12;
 
@@ -102,9 +106,14 @@ public final class OrcFile {
    * For bugs in the writer, but the old readers already read the new data
    * correctly, bump this version instead of the Version.
    */
-  public static enum WriterVersion {
+  public enum WriterVersion {
     ORIGINAL(0),
-      HIVE_8732(1); // corrupted stripe/file maximum column statistics
+    HIVE_8732(1), // corrupted stripe/file maximum column statistics
+    HIVE_4243(2), // use real column names from Hive tables
+    HIVE_12055(3), // vectorized writer
+
+    // Don't use any magic numbers here except for the below:
+    FUTURE(Integer.MAX_VALUE); // a version from a future writer
 
     private final int id;
 
@@ -112,67 +121,111 @@ public final class OrcFile {
       return id;
     }
 
-    private WriterVersion(int id) {
+    WriterVersion(int id) {
       this.id = id;
     }
+
+    private static final WriterVersion[] values;
+    static {
+      // Assumes few non-negative values close to zero.
+      int max = Integer.MIN_VALUE;
+      for (WriterVersion v : WriterVersion.values()) {
+        if (v.id < 0) throw new AssertionError();
+        if (v.id > max && FUTURE.id != v.id) {
+          max = v.id;
+        }
+      }
+      values = new WriterVersion[max + 1];
+      for (WriterVersion v : WriterVersion.values()) {
+        if (v.id < values.length) {
+          values[v.id] = v;
+        }
+      }
+    }
+
+    public static WriterVersion from(int val) {
+      if (val == FUTURE.id) return FUTURE; // Special handling for the magic value.
+      return values[val];
+    }
   }
+  public static final WriterVersion CURRENT_WRITER = WriterVersion.HIVE_12055;
 
-  public static enum EncodingStrategy {
+  public enum EncodingStrategy {
     SPEED, COMPRESSION;
   }
 
-  public static enum CompressionStrategy {
+  public enum CompressionStrategy {
     SPEED, COMPRESSION;
   }
 
-  // Note : these string definitions for table properties are deprecated,
-  // and retained only for backward compatibility, please do not add to
-  // them, add to OrcTableProperties below instead
-  @Deprecated public static final String COMPRESSION = "orc.compress";
-  @Deprecated public static final String COMPRESSION_BLOCK_SIZE = "orc.compress.size";
-  @Deprecated public static final String STRIPE_SIZE = "orc.stripe.size";
-  @Deprecated public static final String ROW_INDEX_STRIDE = "orc.row.index.stride";
-  @Deprecated public static final String ENABLE_INDEXES = "orc.create.index";
-  @Deprecated public static final String BLOCK_PADDING = "orc.block.padding";
+  // unused
+  private OrcFile() {}
 
-  /**
-   * Enum container for all orc table properties.
-   * If introducing a new orc-specific table property,
-   * add it here.
-   */
-  public static enum OrcTableProperties {
-    COMPRESSION("orc.compress"),
-    COMPRESSION_BLOCK_SIZE("orc.compress.size"),
-    STRIPE_SIZE("orc.stripe.size"),
-    BLOCK_SIZE("orc.block.size"),
-    ROW_INDEX_STRIDE("orc.row.index.stride"),
-    ENABLE_INDEXES("orc.create.index"),
-    BLOCK_PADDING("orc.block.padding"),
-    ENCODING_STRATEGY("orc.encoding.strategy"),
-    BLOOM_FILTER_COLUMNS("orc.bloom.filter.columns"),
-    BLOOM_FILTER_FPP("orc.bloom.filter.fpp");
+  public static class ReaderOptions {
+    private final Configuration conf;
+    private FileSystem filesystem;
+    private FileMetaInfo fileMetaInfo; // TODO: this comes from some place.
+    private long maxLength = Long.MAX_VALUE;
+    private FileMetadata fullFileMetadata; // Propagate from LLAP cache.
+
+    public ReaderOptions(Configuration conf) {
+      this.conf = conf;
+    }
+
+    public ReaderOptions fileMetaInfo(FileMetaInfo info) {
+      fileMetaInfo = info;
+      return this;
+    }
+
+    public ReaderOptions filesystem(FileSystem fs) {
+      this.filesystem = fs;
+      return this;
+    }
+
+    public ReaderOptions maxLength(long val) {
+      maxLength = val;
+      return this;
+    }
 
-    private final String propName;
+    public ReaderOptions fileMetadata(FileMetadata metadata) {
+      this.fullFileMetadata = metadata;
+      return this;
+    }
+
+    public Configuration getConfiguration() {
+      return conf;
+    }
 
-    OrcTableProperties(String propName) {
-      this.propName = propName;
+    public FileSystem getFilesystem() {
+      return filesystem;
     }
 
-    public String getPropName(){
-      return this.propName;
+    public FileMetaInfo getFileMetaInfo() {
+      return fileMetaInfo;
+    }
+
+    public long getMaxLength() {
+      return maxLength;
+    }
+
+    public FileMetadata getFileMetadata() {
+      return fullFileMetadata;
     }
   }
 
-  // unused
-  private OrcFile() {}
+  public static ReaderOptions readerOptions(Configuration conf) {
+    return new ReaderOptions(conf);
+  }
+
+
 
-  public static interface WriterContext {
+  public interface WriterContext {
     Writer getWriter();
   }
 
-  public static interface WriterCallback {
-    public void preStripeWrite(WriterContext context) throws IOException;
-    public void preFooterWrite(WriterContext context) throws IOException;
+  public interface WriterCallback {
+    void preStripeWrite(WriterContext context) throws IOException;
+    void preFooterWrite(WriterContext context) throws IOException;
   }
 
   /**
@@ -181,7 +234,7 @@ public final class OrcFile {
   public static class WriterOptions {
     private final Configuration configuration;
     private FileSystem fileSystemValue = null;
-    private ObjectInspector inspectorValue = null;
+    private TypeDescription schema = null;
     private long stripeSizeValue;
     private long blockSizeValue;
     private int rowIndexStrideValue;
@@ -193,45 +246,42 @@ public final class OrcFile {
     private WriterCallback callback;
     private EncodingStrategy encodingStrategy;
     private CompressionStrategy compressionStrategy;
-    private float paddingTolerance;
+    private double paddingTolerance;
     private String bloomFilterColumns;
     private double bloomFilterFpp;
-    private TimeZone timezone;
 
-    WriterOptions(Configuration conf) {
+    protected WriterOptions(Properties tableProperties, Configuration conf) {
       configuration = conf;
-      memoryManagerValue = getMemoryManager(conf);
-      stripeSizeValue = OrcConf.getLongVar(conf, HIVE_ORC_DEFAULT_STRIPE_SIZE);
-      blockSizeValue = OrcConf.getLongVar(conf, HIVE_ORC_DEFAULT_BLOCK_SIZE);
-      rowIndexStrideValue = OrcConf.getIntVar(conf, HIVE_ORC_DEFAULT_ROW_INDEX_STRIDE);
-      bufferSizeValue = OrcConf.getIntVar(conf, HIVE_ORC_DEFAULT_BUFFER_SIZE);
-      blockPaddingValue = OrcConf.getBoolVar(conf, HIVE_ORC_DEFAULT_BLOCK_PADDING);
-      compressValue = CompressionKind.valueOf(OrcConf.getVar(conf, HIVE_ORC_DEFAULT_COMPRESS));
-      String versionName = OrcConf.getVar(conf, HIVE_ORC_WRITE_FORMAT);
-      if (versionName == null) {
-        versionValue = Version.CURRENT;
-      } else {
-        versionValue = Version.byName(versionName);
-      }
-      String enString =
-          conf.get(OrcConf.ConfVars.HIVE_ORC_ENCODING_STRATEGY.varname);
-      if (enString == null) {
-        encodingStrategy = EncodingStrategy.SPEED;
-      } else {
-        encodingStrategy = EncodingStrategy.valueOf(enString);
-      }
-
-      String compString = conf
-          .get(OrcConf.ConfVars.HIVE_ORC_COMPRESSION_STRATEGY.varname);
-      if (compString == null) {
-        compressionStrategy = CompressionStrategy.SPEED;
-      } else {
-        compressionStrategy = CompressionStrategy.valueOf(compString);
-      }
-
-      paddingTolerance = conf.getFloat(OrcConf.ConfVars.HIVE_ORC_BLOCK_PADDING_TOLERANCE.varname,
-          OrcConf.ConfVars.HIVE_ORC_BLOCK_PADDING_TOLERANCE.defaultFloatVal);
-      bloomFilterFpp = BloomFilterIO.DEFAULT_FPP;
+      memoryManagerValue = getStaticMemoryManager(conf);
+      stripeSizeValue = org.apache.orc.OrcConf.STRIPE_SIZE.getLong(tableProperties, conf);
+      blockSizeValue = org.apache.orc.OrcConf.BLOCK_SIZE.getLong(tableProperties, conf);
+      rowIndexStrideValue =
+          (int) org.apache.orc.OrcConf.ROW_INDEX_STRIDE.getLong(tableProperties, conf);
+      bufferSizeValue = (int) org.apache.orc.OrcConf.BUFFER_SIZE.getLong(tableProperties,
+          conf);
+      blockPaddingValue =
+          org.apache.orc.OrcConf.BLOCK_PADDING.getBoolean(tableProperties, conf);
+      compressValue =
+          CompressionKind.valueOf(org.apache.orc.OrcConf.COMPRESS.getString(tableProperties,
+              conf));
+      String versionName = org.apache.orc.OrcConf.WRITE_FORMAT.getString(tableProperties,
+          conf);
+      versionValue = Version.byName(versionName);
+      String enString = org.apache.orc.OrcConf.ENCODING_STRATEGY.getString(tableProperties,
+          conf);
+      encodingStrategy = EncodingStrategy.valueOf(enString);
+
+      String compString =
+          org.apache.orc.OrcConf.COMPRESSION_STRATEGY.getString(tableProperties, conf);
+      compressionStrategy = CompressionStrategy.valueOf(compString);
+
+      paddingTolerance =
+          org.apache.orc.OrcConf.BLOCK_PADDING_TOLERANCE.getDouble(tableProperties, conf);
+
+      bloomFilterColumns = org.apache.orc.OrcConf.BLOOM_FILTER_COLUMNS.getString(tableProperties,
+          conf);
+      bloomFilterFpp = org.apache.orc.OrcConf.BLOOM_FILTER_FPP.getDouble(tableProperties,
+          conf);
     }
 
     /**
@@ -302,7 +352,7 @@ public final class OrcFile {
     /**
      * Sets the tolerance for block padding as a percentage of stripe size.
      */
-    public WriterOptions paddingTolerance(float value) {
+    public WriterOptions paddingTolerance(double value) {
       paddingTolerance = value;
       return this;
     }
@@ -318,7 +368,7 @@ public final class OrcFile {
     /**
      * Specify the false positive probability for bloom filter.
      * @param fpp - false positive probability
-     * @return
+     * @return this
      */
     public WriterOptions bloomFilterFpp(double fpp) {
       bloomFilterFpp = fpp;
@@ -334,11 +384,12 @@ public final class OrcFile {
     }
 
     /**
-     * A required option that sets the object inspector for the rows. Used
-     * to determine the schema for the file.
+     * Set the schema for the file. This is a required parameter.
+     * @param schema the schema for the file.
+     * @return this
      */
-    public WriterOptions inspector(ObjectInspector value) {
-      inspectorValue = value;
+    public WriterOptions setSchema(TypeDescription schema) {
+      this.schema = schema;
       return this;
     }
 
@@ -353,7 +404,7 @@ public final class OrcFile {
     /**
      * Add a listener for when the stripe and file are about to be closed.
      * @param callback the object to be called when the stripe is closed
-     * @return
+     * @return this
      */
     public WriterOptions callback(WriterCallback callback) {
       this.callback = callback;
@@ -363,25 +414,112 @@ public final class OrcFile {
     /**
      * A package local option to set the memory manager.
      */
-    WriterOptions memory(MemoryManager value) {
+    protected WriterOptions memory(MemoryManager value) {
       memoryManagerValue = value;
       return this;
     }
 
-    /**
-     * Tajo-specific
-     */
-    WriterOptions timezone(TimeZone value) {
-      timezone = value;
-      return this;
+    public boolean getBlockPadding() {
+      return blockPaddingValue;
+    }
+
+    public long getBlockSize() {
+      return blockSizeValue;
+    }
+
+    public String getBloomFilterColumns() {
+      return bloomFilterColumns;
     }
+
+    public FileSystem getFileSystem() {
+      return fileSystemValue;
+    }
+
+    public Configuration getConfiguration() {
+      return configuration;
+    }
+
+    public TypeDescription getSchema() {
+      return schema;
+    }
+
+    public long getStripeSize() {
+      return stripeSizeValue;
+    }
+
+    public CompressionKind getCompress() {
+      return compressValue;
+    }
+
+    public WriterCallback getCallback() {
+      return callback;
+    }
+
+    public Version getVersion() {
+      return versionValue;
+    }
+
+    public MemoryManager getMemoryManager() {
+      return memoryManagerValue;
+    }
+
+    public int getBufferSize() {
+      return bufferSizeValue;
+    }
+
+    public int getRowIndexStride() {
+      return rowIndexStrideValue;
+    }
+
+    public CompressionStrategy getCompressionStrategy() {
+      return compressionStrategy;
+    }
+
+    public EncodingStrategy getEncodingStrategy() {
+      return encodingStrategy;
+    }
+
+    public double getPaddingTolerance() {
+      return paddingTolerance;
+    }
+
+    public double getBloomFilterFpp() {
+      return bloomFilterFpp;
+    }
+  }
+
+  /**
+   * Create a set of writer options based on a configuration.
+   * @param conf the configuration to use for values
+   * @return A WriterOptions object that can be modified
+   */
+  public static ORCAppender.WriterOptions writerOptions(Configuration conf) {
+    return new ORCAppender.WriterOptions(null, conf);
   }
 
   /**
-   * Create a default set of write options that can be modified.
+   * Create a set of write options based on a set of table properties and
+   * configuration.
+   * @param tableProperties the properties of the table
+   * @param conf the configuration of the query
+   * @return a WriterOptions object that can be modified
    */
-  public static WriterOptions writerOptions(Configuration conf) {
-    return new WriterOptions(conf);
+  public static WriterOptions writerOptions(Properties tableProperties,
+                                            Configuration conf) {
+    return new WriterOptions(tableProperties, conf);
+  }
+
+  private static synchronized MemoryManager getStaticMemoryManager(
+      final Configuration conf) {
+    if (memoryManager == null) {
+      memoryManager = new ThreadLocal<MemoryManager>() {
+        @Override
+        protected MemoryManager initialValue() {
+          return new MemoryManager(conf);
+        }
+      };
+    }
+    return memoryManager.get();
   }
 
   /**
@@ -393,54 +531,13 @@ public final class OrcFile {
    * @throws IOException
    */
   public static Writer createWriter(Path path,
-                                    WriterOptions opts
-                                    ) throws IOException {
-    FileSystem fs = opts.fileSystemValue == null ?
-      path.getFileSystem(opts.configuration) : opts.fileSystemValue;
-
-    return new WriterImpl(fs, path, opts.configuration, opts.inspectorValue,
-                          opts.stripeSizeValue, opts.compressValue,
-                          opts.bufferSizeValue, opts.rowIndexStrideValue,
-                          opts.memoryManagerValue, opts.blockPaddingValue,
-                          opts.versionValue, opts.callback,
-                          opts.encodingStrategy, opts.compressionStrategy,
-                          opts.paddingTolerance, opts.blockSizeValue,
-                          opts.bloomFilterColumns, opts.bloomFilterFpp,
-                          opts.timezone);
-  }
+                                    WriterOptions opts,
+                                    TimeZone timeZone
+  ) throws IOException {
+    FileSystem fs = opts.getFileSystem() == null ?
+        path.getFileSystem(opts.getConfiguration()) : opts.getFileSystem();
 
-  /**
-   * Create an ORC file writer. This method is provided for API backward
-   * compatability with Hive 0.11.
-   * @param fs file system
-   * @param path filename to write to
-   * @param inspector the ObjectInspector that inspects the rows
-   * @param stripeSize the number of bytes in a stripe
-   * @param compress how to compress the file
-   * @param bufferSize the number of bytes to compress at once
-   * @param rowIndexStride the number of rows between row index entries or
-   *                       0 to suppress all indexes
-   * @return a new ORC file writer
-   * @throws IOException
-   */
-  public static Writer createWriter(FileSystem fs,
-                                    Path path,
-                                    Configuration conf,
-                                    ObjectInspector inspector,
-                                    long stripeSize,
-                                    CompressionKind compress,
-                                    int bufferSize,
-                                    int rowIndexStride,
-                                    TimeZone timeZone) throws IOException {
-    return createWriter(path,
-                        writerOptions(conf)
-                        .fileSystem(fs)
-                        .inspector(inspector)
-                        .stripeSize(stripeSize)
-                        .compress(compress)
-                        .bufferSize(bufferSize)
-                        .rowIndexStride(rowIndexStride)
-                        .timezone(timeZone));
+    return new WriterImpl(fs, path, opts, timeZone);
   }
 
   private static ThreadLocal<MemoryManager> memoryManager = null;

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/OrcRecordReader.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/OrcRecordReader.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/OrcRecordReader.java
new file mode 100644
index 0000000..7194bf4
--- /dev/null
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/OrcRecordReader.java
@@ -0,0 +1,454 @@
+/*
+ * 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.tajo.storage.thirdparty.orc;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+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.orc.*;
+import org.apache.orc.OrcProto;
+import org.apache.orc.impl.*;
+import org.apache.orc.impl.StreamName;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.storage.VTuple;
+import org.apache.tajo.storage.fragment.FileFragment;
+import org.apache.tajo.storage.thirdparty.orc.TreeReaderFactory.DatumTreeReader;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.*;
+
+public class OrcRecordReader implements Closeable {
+
+  private final Log LOG = LogFactory.getLog(OrcRecordReader.class);
+
+  private final Path path;
+  private final long firstRow;
+  private final List<StripeInformation> stripes = new ArrayList<>();
+  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<org.apache.orc.impl.StreamName, InStream> streams = new HashMap<>();
+  DiskRangeList bufferChunks = null;
+  private final TreeReaderFactory.DatumTreeReader[] reader;
+  private final OrcProto.RowIndex[] indexes;
+  private final OrcProto.BloomFilterIndex[] bloomFilterIndices;
+  private final Configuration conf;
+  private final MetadataReader metadata;
+  private final DataReader dataReader;
+  private final Tuple result;
+
+  public OrcRecordReader(List<StripeInformation> stripes,
+                         FileSystem fileSystem,
+                         Schema schema,
+                         Column[] target,
+                         FileFragment fragment,
+                         List<OrcProto.Type> types,
+                         CompressionCodec codec,
+                         int bufferSize,
+                         long strideRate,
+                         Reader.Options options,
+                         Configuration conf,
+                         TimeZone timeZone) throws IOException {
+
+    result = new VTuple(target.length);
+
+    this.conf = conf;
+    this.path = fragment.getPath();
+    this.codec = codec;
+    this.types = types;
+    this.bufferSize = bufferSize;
+    this.included = new boolean[schema.size() + 1];
+    included[0] = target.length > 0; // always include root column except when target schema size is 0
+    Schema targetSchema = new Schema(target);
+    for (int i = 1; i < included.length; i++) {
+      included[i] = targetSchema.contains(schema.getColumn(i - 1));
+    }
+    this.rowIndexStride = strideRate;
+    this.metadata = new MetadataReaderImpl(fileSystem, path, codec, bufferSize, types.size());
+
+    long rows = 0;
+    long skippedRows = 0;
+    long offset = fragment.getStartKey();
+    long maxOffset = fragment.getStartKey() + fragment.getLength();
+    for(StripeInformation stripe: stripes) {
+      long stripeStart = stripe.getOffset();
+      if (offset > stripeStart) {
+        skippedRows += stripe.getNumberOfRows();
+      } else if (stripeStart < maxOffset) {
+        this.stripes.add(stripe);
+        rows += stripe.getNumberOfRows();
+      }
+    }
+
+    // TODO: we could change the ctor to pass this externally
+    this.dataReader = RecordReaderUtils.createDefaultDataReader(fileSystem, path, options.getUseZeroCopy(), codec);
+    this.dataReader.open();
+
+    firstRow = skippedRows;
+    totalRowCount = rows;
+
+    reader = new DatumTreeReader[target.length];
+    for (int i = 0; i < reader.length; i++) {
+      reader[i] = TreeReaderFactory.createTreeReader(timeZone, schema.getColumnId(target[i].getQualifiedName()), target[i],
+          options.getSkipCorruptRecords());
+    }
+
+    indexes = new OrcProto.RowIndex[types.size()];
+    bloomFilterIndices = new OrcProto.BloomFilterIndex[types.size()];
+    advanceToNextRow(reader, 0L, true);
+  }
+
+  /**
+   * 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 includedColumns   which columns are needed
+   * @param doMergeBuffers
+   * @return the list of disk ranges that will be loaded
+   */
+  static DiskRangeList planReadPartialDataStreams
+  (List<OrcProto.Stream> streamList,
+   boolean[] includedColumns,
+   boolean doMergeBuffers) {
+    long offset = 0;
+    // figure out which columns have a present stream
+    DiskRangeList.CreateHelper list = new DiskRangeList.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() &&
+          (org.apache.orc.impl.StreamName.getArea(streamKind) == org.apache.orc.impl.StreamName.Area.DATA) &&
+          includedColumns[column]) {
+        RecordReaderUtils.addEntireStreamToRanges(offset, length, list, doMergeBuffers);
+      }
+      offset += length;
+    }
+    return list.extract();
+  }
+
+  void createStreams(List<OrcProto.Stream> streamDescriptions,
+                     DiskRangeList ranges,
+                     boolean[] includeColumn,
+                     CompressionCodec codec,
+                     int bufferSize,
+                     Map<org.apache.orc.impl.StreamName, InStream> streams) throws IOException {
+    long streamOffset = 0;
+    for (OrcProto.Stream streamDesc : streamDescriptions) {
+      int column = streamDesc.getColumn();
+      if ((includeColumn != null && !includeColumn[column]) ||
+          streamDesc.hasKind() &&
+              (org.apache.orc.impl.StreamName.getArea(streamDesc.getKind()) != org.apache.orc.impl.StreamName.Area.DATA)) {
+        streamOffset += streamDesc.getLength();
+        continue;
+      }
+      List<DiskRange> buffers = RecordReaderUtils.getStreamBuffers(
+          ranges, streamOffset, streamDesc.getLength());
+      org.apache.orc.impl.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, included, 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);
+  }
+
+  /**
+   * 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;
+
+    if (nextRowInStripe >= rowCountInStripe) {
+      if (canAdvanceStripe) {
+        advanceStripe();
+      }
+      return canAdvanceStripe;
+    }
+    if (nextRowInStripe != rowInStripe) {
+      if (rowIndexStride != 0) {
+        int rowGroup = (int) (nextRowInStripe / rowIndexStride);
+        seekToRowEntry(reader, rowGroup);
+        for (TreeReaderFactory.TreeReader eachReader : reader) {
+          eachReader.skipRows(nextRowInStripe - rowGroup * rowIndexStride);
+        }
+      } else {
+        for (TreeReaderFactory.TreeReader eachReader : reader) {
+          eachReader.skipRows(nextRowInStripe - rowInStripe);
+        }
+      }
+      rowInStripe = nextRowInStripe;
+    }
+    return true;
+  }
+
+  public boolean hasNext() throws IOException {
+    return rowInStripe < rowCountInStripe;
+  }
+
+  public Tuple next() throws IOException {
+    if (hasNext()) {
+      try {
+        for (int i = 0; i < reader.length; i++) {
+          result.put(i, reader[i].next());
+        }
+        // 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);
+      }
+    } else {
+      return null;
+    }
+  }
+
+  /**
+   * 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();
+    }
+  }
+
+  /**
+   * Read the current stripe into memory.
+   *
+   * @throws IOException
+   */
+  private void readStripe() throws IOException {
+    StripeInformation stripe = beginReadStripe();
+
+    // 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) {
+        readAllDataStreams(stripe);
+      } else {
+        readPartialDataStreams(stripe);
+      }
+
+      for (TreeReaderFactory.TreeReader eachReader : reader) {
+        eachReader.startStripe(streams, stripeFooter);
+      }
+      // if we skipped the first row group, move the pointers forward
+      if (rowInStripe != 0) {
+        seekToRowEntry(reader, (int) (rowInStripe / rowIndexStride));
+      }
+    }
+  }
+
+  private void clearStreams() throws IOException {
+    // 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();
+  }
+
+  OrcProto.StripeFooter readStripeFooter(StripeInformation stripe) throws IOException {
+    return metadata.readStripeFooter(stripe);
+  }
+
+  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, included, codec, bufferSize, streams);
+  }
+
+  public long getRowNumber() {
+    return rowInStripe + rowBaseInStripe + firstRow;
+  }
+
+  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");
+  }
+
+  OrcIndex readRowIndex(
+      int stripeIndex, boolean[] included) throws IOException {
+    return readRowIndex(stripeIndex, included, null, null);
+  }
+
+  OrcIndex readRowIndex(int stripeIndex, boolean[] included, OrcProto.RowIndex[] indexes,
+                        OrcProto.BloomFilterIndex[] bloomFilterIndex) 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;
+    }
+    return metadata.readRowIndex(stripe, stripeFooter, included, indexes, null,
+        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));
+      }
+    }
+    for (TreeReaderFactory.TreeReader eachReader : reader) {
+      eachReader.seek(index);
+    }
+  }
+
+  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);
+
+    // if we aren't to the right row yet, advance in the stripe.
+    advanceToNextRow(reader, rowNumber, true);
+  }
+
+  public long getNumBytes() {
+    return ((RecordReaderUtils.DefaultDataReader)dataReader).getReadBytes();
+  }
+
+  @Override
+  public void close() throws IOException {
+    clearStreams();
+    dataReader.close();
+  }
+
+  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++);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/OrcUtils.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/OrcUtils.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/OrcUtils.java
index 3a474dd..b8d3f52 100644
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/OrcUtils.java
+++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/OrcUtils.java
@@ -17,185 +17,101 @@
  */
 package org.apache.tajo.storage.thirdparty.orc;
 
-import com.google.common.collect.Lists;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hive.serde2.objectinspector.*;
-
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
+import org.apache.orc.CompressionCodec;
+import org.apache.orc.TypeDescription;
+import org.apache.orc.impl.SnappyCodec;
+import org.apache.orc.impl.ZlibCodec;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.TypeDesc;
+import org.apache.tajo.exception.TajoRuntimeException;
+import org.apache.tajo.exception.UnsupportedDataTypeException;
 
 public class OrcUtils {
   private static final Log LOG = LogFactory.getLog(OrcUtils.class);
 
-  /**
-   * Returns selected columns as a boolean array with true value set for specified column names.
-   * The result will contain number of elements equal to flattened number of columns.
-   * For example:
-   * selectedColumns - a,b,c
-   * allColumns - a,b,c,d
-   * If column c is a complex type, say list<string> and other types are primitives then result will
-   * be [false, true, true, true, true, true, false]
-   * Index 0 is the root element of the struct which is set to false by default, index 1,2
-   * corresponds to columns a and b. Index 3,4 correspond to column c which is list<string> and
-   * index 5 correspond to column d. After flattening list<string> gets 2 columns.
-   *
-   * @param selectedColumns - comma separated list of selected column names
-   * @param allColumns      - comma separated list of all column names
-   * @param inspector       - object inspector
-   * @return - boolean array with true value set for the specified column names
-   */
-  public static boolean[] includeColumns(String selectedColumns, String allColumns,
-      ObjectInspector inspector) {
-    int numFlattenedCols = getFlattenedColumnsCount(inspector);
-    boolean[] results = new boolean[numFlattenedCols];
-    if ("*".equals(selectedColumns)) {
-      Arrays.fill(results, true);
-      return results;
-    }
-    if (selectedColumns != null && !selectedColumns.isEmpty()) {
-      includeColumnsImpl(results, selectedColumns.toLowerCase(), allColumns, inspector);
-    }
-    return results;
-  }
-
-  private static void includeColumnsImpl(boolean[] includeColumns, String selectedColumns,
-      String allColumns,
-      ObjectInspector inspector) {
-      Map<String, List<Integer>> columnSpanMap = getColumnSpan(allColumns, inspector);
-      LOG.info("columnSpanMap: " + columnSpanMap);
-
-      String[] selCols = selectedColumns.split(",");
-      for (String sc : selCols) {
-        if (columnSpanMap.containsKey(sc)) {
-          List<Integer> colSpan = columnSpanMap.get(sc);
-          int start = colSpan.get(0);
-          int end = colSpan.get(1);
-          for (int i = start; i <= end; i++) {
-            includeColumns[i] = true;
+  public static org.apache.orc.CompressionCodec createCodec(org.apache.orc.CompressionKind kind) {
+    switch (kind) {
+      case NONE:
+        return null;
+      case ZLIB:
+        return new ZlibCodec();
+      case SNAPPY:
+        return new SnappyCodec();
+      case LZO:
+        try {
+          ClassLoader loader = Thread.currentThread().getContextClassLoader();
+          if (loader == null) {
+            throw new RuntimeException("error while getting a class loader");
           }
+          @SuppressWarnings("unchecked")
+          Class<? extends org.apache.orc.CompressionCodec> lzo =
+              (Class<? extends CompressionCodec>)
+                  loader.loadClass("org.apache.hadoop.hive.ql.io.orc.LzoCodec");
+          return lzo.newInstance();
+        } catch (ClassNotFoundException e) {
+          throw new IllegalArgumentException("LZO is not available.", e);
+        } catch (InstantiationException e) {
+          throw new IllegalArgumentException("Problem initializing LZO", e);
+        } catch (IllegalAccessException e) {
+          throw new IllegalArgumentException("Insufficient access to LZO", e);
         }
-      }
-
-      LOG.info("includeColumns: " + Arrays.toString(includeColumns));
+      default:
+        throw new IllegalArgumentException("Unknown compression codec: " +
+            kind);
     }
+  }
 
-  private static Map<String, List<Integer>> getColumnSpan(String allColumns,
-      ObjectInspector inspector) {
-    // map that contains the column span for each column. Column span is the number of columns
-    // required after flattening. For a given object inspector this map contains the start column
-    // id and end column id (both inclusive) after flattening.
-    // EXAMPLE:
-    // schema: struct<a:int, b:float, c:map<string,int>>
-    // column span map for the above struct will be
-    // a => [1,1], b => [2,2], c => [3,5]
-    Map<String, List<Integer>> columnSpanMap = new HashMap<>();
-    if (allColumns != null) {
-      String[] columns = allColumns.split(",");
-      int startIdx = 0;
-      int endIdx = 0;
-      if (inspector instanceof StructObjectInspector) {
-        StructObjectInspector soi = (StructObjectInspector) inspector;
-        List<? extends StructField> fields = soi.getAllStructFieldRefs();
-        for (int i = 0; i < fields.size(); i++) {
-          StructField sf = fields.get(i);
-
-          // we get the type (category) from object inspector but column name from the argument.
-          // The reason for this is hive (FileSinkOperator) does not pass the actual column names,
-          // instead it passes the internal column names (_col1,_col2).
-          ObjectInspector sfOI = sf.getFieldObjectInspector();
-          String colName = columns[i];
+  public static TypeDescription convertSchema(Schema schema) {
+    TypeDescription description = TypeDescription.createStruct();
 
-          startIdx = endIdx + 1;
-          switch (sfOI.getCategory()) {
-            case PRIMITIVE:
-              endIdx += 1;
-              break;
-            case STRUCT:
-              endIdx += 1;
-              StructObjectInspector structInsp = (StructObjectInspector) sfOI;
-              List<? extends StructField> structFields = structInsp.getAllStructFieldRefs();
-              for (StructField structField : structFields) {
-                endIdx += getFlattenedColumnsCount(structField.getFieldObjectInspector());
-              }
-              break;
-            case MAP:
-              endIdx += 1;
-              MapObjectInspector mapInsp = (MapObjectInspector) sfOI;
-              endIdx += getFlattenedColumnsCount(mapInsp.getMapKeyObjectInspector());
-              endIdx += getFlattenedColumnsCount(mapInsp.getMapValueObjectInspector());
-              break;
-            case LIST:
-              endIdx += 1;
-              ListObjectInspector listInsp = (ListObjectInspector) sfOI;
-              endIdx += getFlattenedColumnsCount(listInsp.getListElementObjectInspector());
-              break;
-            case UNION:
-              endIdx += 1;
-              UnionObjectInspector unionInsp = (UnionObjectInspector) sfOI;
-              List<ObjectInspector> choices = unionInsp.getObjectInspectors();
-              for (ObjectInspector choice : choices) {
-                endIdx += getFlattenedColumnsCount(choice);
-              }
-              break;
-            default:
-              throw new IllegalArgumentException("Bad category: " +
-                  inspector.getCategory());
-          }
-
-          columnSpanMap.put(colName, Lists.newArrayList(startIdx, endIdx));
-        }
-      }
+    for (Column eachColumn : schema.getRootColumns()) {
+      description.addField(eachColumn.getQualifiedName(),
+          convertTypeInfo(eachColumn.getTypeDesc()));
     }
-    return columnSpanMap;
+    return description;
   }
 
-  /**
-   * Returns the number of columns after flatting complex types.
-   *
-   * @param inspector - object inspector
-   * @return
-   */
-  public static int getFlattenedColumnsCount(ObjectInspector inspector) {
-    int numWriters = 0;
-    switch (inspector.getCategory()) {
-      case PRIMITIVE:
-        numWriters += 1;
-        break;
-      case STRUCT:
-        numWriters += 1;
-        StructObjectInspector structInsp = (StructObjectInspector) inspector;
-        List<? extends StructField> fields = structInsp.getAllStructFieldRefs();
-        for (StructField field : fields) {
-          numWriters += getFlattenedColumnsCount(field.getFieldObjectInspector());
-        }
-        break;
-      case MAP:
-        numWriters += 1;
-        MapObjectInspector mapInsp = (MapObjectInspector) inspector;
-        numWriters += getFlattenedColumnsCount(mapInsp.getMapKeyObjectInspector());
-        numWriters += getFlattenedColumnsCount(mapInsp.getMapValueObjectInspector());
-        break;
-      case LIST:
-        numWriters += 1;
-        ListObjectInspector listInsp = (ListObjectInspector) inspector;
-        numWriters += getFlattenedColumnsCount(listInsp.getListElementObjectInspector());
-        break;
-      case UNION:
-        numWriters += 1;
-        UnionObjectInspector unionInsp = (UnionObjectInspector) inspector;
-        List<ObjectInspector> choices = unionInsp.getObjectInspectors();
-        for (ObjectInspector choice : choices) {
-          numWriters += getFlattenedColumnsCount(choice);
+  public static TypeDescription convertTypeInfo(TypeDesc desc) {
+    switch (desc.getDataType().getType()) {
+      case BOOLEAN:
+        return TypeDescription.createBoolean();
+      case BIT:
+        return TypeDescription.createByte();
+      case INT2:
+        return TypeDescription.createShort();
+      case INT4:
+      case INET4:
+        return TypeDescription.createInt();
+      case INT8:
+        return TypeDescription.createLong();
+      case FLOAT4:
+        return TypeDescription.createFloat();
+      case FLOAT8:
+        return TypeDescription.createDouble();
+      case TEXT:
+        return TypeDescription.createString();
+      case DATE:
+        return TypeDescription.createDate();
+      case TIMESTAMP:
+        return TypeDescription.createTimestamp();
+      case BLOB:
+        return TypeDescription.createBinary();
+      case CHAR:
+        return TypeDescription.createChar()
+            .withMaxLength(desc.getDataType().getLength());
+      case RECORD: {
+        TypeDescription result = TypeDescription.createStruct();
+        for (Column eachColumn : desc.getNestedSchema().getRootColumns()) {
+          result.addField(eachColumn.getQualifiedName(),
+              convertTypeInfo(eachColumn.getTypeDesc()));
         }
-        break;
+        return result;
+      }
       default:
-        throw new IllegalArgumentException("Bad category: " +
-            inspector.getCategory());
+        throw new TajoRuntimeException(new UnsupportedDataTypeException(desc.getDataType().getType().name()));
     }
-    return numWriters;
   }
-
 }


[3/7] tajo git commit: TAJO-2102: Migrate to Apache Orc from Presto's one.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/SerializationUtils.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/SerializationUtils.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/SerializationUtils.java
deleted file mode 100644
index 53687b7..0000000
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/SerializationUtils.java
+++ /dev/null
@@ -1,844 +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.tajo.storage.thirdparty.orc;
-
-import java.io.EOFException;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.math.BigInteger;
-
-final class SerializationUtils {
-
-  private final static int BUFFER_SIZE = 64;
-  private final byte[] readBuffer;
-  private final byte[] writeBuffer;
-
-  public SerializationUtils() {
-    this.readBuffer = new byte[BUFFER_SIZE];
-    this.writeBuffer = new byte[BUFFER_SIZE];
-  }
-
-  void writeVulong(OutputStream output, long value) throws IOException {
-    while (true) {
-      if ((value & ~0x7f) == 0) {
-        output.write((byte) value);
-        return;
-      } else {
-        output.write((byte) (0x80 | (value & 0x7f)));
-        value >>>= 7;
-      }
-    }
-  }
-
-  void writeVslong(OutputStream output, long value) throws IOException {
-    writeVulong(output, (value << 1) ^ (value >> 63));
-  }
-
-
-  long readVulong(InputStream in) throws IOException {
-    long result = 0;
-    long b;
-    int offset = 0;
-    do {
-      b = in.read();
-      if (b == -1) {
-        throw new EOFException("Reading Vulong past EOF");
-      }
-      result |= (0x7f & b) << offset;
-      offset += 7;
-    } while (b >= 0x80);
-    return result;
-  }
-
-  long readVslong(InputStream in) throws IOException {
-    long result = readVulong(in);
-    return (result >>> 1) ^ -(result & 1);
-  }
-
-  float readFloat(InputStream in) throws IOException {
-    int ser = in.read() | (in.read() << 8) | (in.read() << 16) |
-      (in.read() << 24);
-    return Float.intBitsToFloat(ser);
-  }
-
-  void writeFloat(OutputStream output, float value) throws IOException {
-    int ser = Float.floatToIntBits(value);
-    output.write(ser & 0xff);
-    output.write((ser >> 8) & 0xff);
-    output.write((ser >> 16) & 0xff);
-    output.write((ser >> 24) & 0xff);
-  }
-
-  double readDouble(InputStream in) throws IOException {
-    return Double.longBitsToDouble(readLongLE(in));
-  }
-
-  long readLongLE(InputStream in) throws IOException {
-    in.read(readBuffer, 0, 8);
-    return (((readBuffer[0] & 0xff) << 0)
-        + ((readBuffer[1] & 0xff) << 8)
-        + ((readBuffer[2] & 0xff) << 16)
-        + ((long) (readBuffer[3] & 0xff) << 24)
-        + ((long) (readBuffer[4] & 0xff) << 32)
-        + ((long) (readBuffer[5] & 0xff) << 40)
-        + ((long) (readBuffer[6] & 0xff) << 48)
-        + ((long) (readBuffer[7] & 0xff) << 56));
-  }
-
-  void writeDouble(OutputStream output, double value) throws IOException {
-    writeLongLE(output, Double.doubleToLongBits(value));
-  }
-
-  private void writeLongLE(OutputStream output, long value) throws IOException {
-    writeBuffer[0] = (byte) ((value >> 0)  & 0xff);
-    writeBuffer[1] = (byte) ((value >> 8)  & 0xff);
-    writeBuffer[2] = (byte) ((value >> 16) & 0xff);
-    writeBuffer[3] = (byte) ((value >> 24) & 0xff);
-    writeBuffer[4] = (byte) ((value >> 32) & 0xff);
-    writeBuffer[5] = (byte) ((value >> 40) & 0xff);
-    writeBuffer[6] = (byte) ((value >> 48) & 0xff);
-    writeBuffer[7] = (byte) ((value >> 56) & 0xff);
-    output.write(writeBuffer, 0, 8);
-  }
-
-  /**
-   * Write the arbitrarily sized signed BigInteger in vint format.
-   *
-   * Signed integers are encoded using the low bit as the sign bit using zigzag
-   * encoding.
-   *
-   * Each byte uses the low 7 bits for data and the high bit for stop/continue.
-   *
-   * Bytes are stored LSB first.
-   * @param output the stream to write to
-   * @param value the value to output
-   * @throws IOException
-   */
-  static void writeBigInteger(OutputStream output,
-                              BigInteger value) throws IOException {
-    // encode the signed number as a positive integer
-    value = value.shiftLeft(1);
-    int sign = value.signum();
-    if (sign < 0) {
-      value = value.negate();
-      value = value.subtract(BigInteger.ONE);
-    }
-    int length = value.bitLength();
-    while (true) {
-      long lowBits = value.longValue() & 0x7fffffffffffffffL;
-      length -= 63;
-      // write out the next 63 bits worth of data
-      for(int i=0; i < 9; ++i) {
-        // if this is the last byte, leave the high bit off
-        if (length <= 0 && (lowBits & ~0x7f) == 0) {
-          output.write((byte) lowBits);
-          return;
-        } else {
-          output.write((byte) (0x80 | (lowBits & 0x7f)));
-          lowBits >>>= 7;
-        }
-      }
-      value = value.shiftRight(63);
-    }
-  }
-
-  /**
-   * Read the signed arbitrary sized BigInteger BigInteger in vint format
-   * @param input the stream to read from
-   * @return the read BigInteger
-   * @throws IOException
-   */
-  static BigInteger readBigInteger(InputStream input) throws IOException {
-    BigInteger result = BigInteger.ZERO;
-    long work = 0;
-    int offset = 0;
-    long b;
-    do {
-      b = input.read();
-      if (b == -1) {
-        throw new EOFException("Reading BigInteger past EOF from " + input);
-      }
-      work |= (0x7f & b) << (offset % 63);
-      offset += 7;
-      // if we've read 63 bits, roll them into the result
-      if (offset == 63) {
-        result = BigInteger.valueOf(work);
-        work = 0;
-      } else if (offset % 63 == 0) {
-        result = result.or(BigInteger.valueOf(work).shiftLeft(offset-63));
-        work = 0;
-      }
-    } while (b >= 0x80);
-    if (work != 0) {
-      result = result.or(BigInteger.valueOf(work).shiftLeft((offset/63)*63));
-    }
-    // convert back to a signed number
-    boolean isNegative = result.testBit(0);
-    if (isNegative) {
-      result = result.add(BigInteger.ONE);
-      result = result.negate();
-    }
-    result = result.shiftRight(1);
-    return result;
-  }
-
-  enum FixedBitSizes {
-    ONE, TWO, THREE, FOUR, FIVE, SIX, SEVEN, EIGHT, NINE, TEN, ELEVEN, TWELVE,
-    THIRTEEN, FOURTEEN, FIFTEEN, SIXTEEN, SEVENTEEN, EIGHTEEN, NINETEEN,
-    TWENTY, TWENTYONE, TWENTYTWO, TWENTYTHREE, TWENTYFOUR, TWENTYSIX,
-    TWENTYEIGHT, THIRTY, THIRTYTWO, FORTY, FORTYEIGHT, FIFTYSIX, SIXTYFOUR;
-  }
-
-  /**
-   * Count the number of bits required to encode the given value
-   * @param value
-   * @return bits required to store value
-   */
-  int findClosestNumBits(long value) {
-    int count = 0;
-    while (value != 0) {
-      count++;
-      value = value >>> 1;
-    }
-    return getClosestFixedBits(count);
-  }
-
-  /**
-   * zigzag encode the given value
-   * @param val
-   * @return zigzag encoded value
-   */
-  long zigzagEncode(long val) {
-    return (val << 1) ^ (val >> 63);
-  }
-
-  /**
-   * zigzag decode the given value
-   * @param val
-   * @return zizag decoded value
-   */
-  long zigzagDecode(long val) {
-    return (val >>> 1) ^ -(val & 1);
-  }
-
-  /**
-   * Compute the bits required to represent pth percentile value
-   * @param data - array
-   * @param p - percentile value (>=0.0 to <=1.0)
-   * @return pth percentile bits
-   */
-  int percentileBits(long[] data, int offset, int length, double p) {
-    if ((p > 1.0) || (p <= 0.0)) {
-      return -1;
-    }
-
-    // histogram that store the encoded bit requirement for each values.
-    // maximum number of bits that can encoded is 32 (refer FixedBitSizes)
-    int[] hist = new int[32];
-
-    // compute the histogram
-    for(int i = offset; i < (offset + length); i++) {
-      int idx = encodeBitWidth(findClosestNumBits(data[i]));
-      hist[idx] += 1;
-    }
-
-    int perLen = (int) (length * (1.0 - p));
-
-    // return the bits required by pth percentile length
-    for(int i = hist.length - 1; i >= 0; i--) {
-      perLen -= hist[i];
-      if (perLen < 0) {
-        return decodeBitWidth(i);
-      }
-    }
-
-    return 0;
-  }
-
-  /**
-   * Calculate the number of bytes required
-   * @param n - number of values
-   * @param numBits - bit width
-   * @return number of bytes required
-   */
-  int getTotalBytesRequired(int n, int numBits) {
-    return (n * numBits + 7) / 8;
-  }
-
-  /**
-   * For a given fixed bit this function will return the closest available fixed
-   * bit
-   * @param n
-   * @return closest valid fixed bit
-   */
-  int getClosestFixedBits(int n) {
-    if (n == 0) {
-      return 1;
-    }
-
-    if (n >= 1 && n <= 24) {
-      return n;
-    } else if (n > 24 && n <= 26) {
-      return 26;
-    } else if (n > 26 && n <= 28) {
-      return 28;
-    } else if (n > 28 && n <= 30) {
-      return 30;
-    } else if (n > 30 && n <= 32) {
-      return 32;
-    } else if (n > 32 && n <= 40) {
-      return 40;
-    } else if (n > 40 && n <= 48) {
-      return 48;
-    } else if (n > 48 && n <= 56) {
-      return 56;
-    } else {
-      return 64;
-    }
-  }
-
-  public int getClosestAlignedFixedBits(int n) {
-    if (n == 0 ||  n == 1) {
-      return 1;
-    } else if (n > 1 && n <= 2) {
-      return 2;
-    } else if (n > 2 && n <= 4) {
-      return 4;
-    } else if (n > 4 && n <= 8) {
-      return 8;
-    } else if (n > 8 && n <= 16) {
-      return 16;
-    } else if (n > 16 && n <= 24) {
-      return 24;
-    } else if (n > 24 && n <= 32) {
-      return 32;
-    } else if (n > 32 && n <= 40) {
-      return 40;
-    } else if (n > 40 && n <= 48) {
-      return 48;
-    } else if (n > 48 && n <= 56) {
-      return 56;
-    } else {
-      return 64;
-    }
-  }
-
-  /**
-   * Finds the closest available fixed bit width match and returns its encoded
-   * value (ordinal)
-   * @param n - fixed bit width to encode
-   * @return encoded fixed bit width
-   */
-  int encodeBitWidth(int n) {
-    n = getClosestFixedBits(n);
-
-    if (n >= 1 && n <= 24) {
-      return n - 1;
-    } else if (n > 24 && n <= 26) {
-      return FixedBitSizes.TWENTYSIX.ordinal();
-    } else if (n > 26 && n <= 28) {
-      return FixedBitSizes.TWENTYEIGHT.ordinal();
-    } else if (n > 28 && n <= 30) {
-      return FixedBitSizes.THIRTY.ordinal();
-    } else if (n > 30 && n <= 32) {
-      return FixedBitSizes.THIRTYTWO.ordinal();
-    } else if (n > 32 && n <= 40) {
-      return FixedBitSizes.FORTY.ordinal();
-    } else if (n > 40 && n <= 48) {
-      return FixedBitSizes.FORTYEIGHT.ordinal();
-    } else if (n > 48 && n <= 56) {
-      return FixedBitSizes.FIFTYSIX.ordinal();
-    } else {
-      return FixedBitSizes.SIXTYFOUR.ordinal();
-    }
-  }
-
-  /**
-   * Decodes the ordinal fixed bit value to actual fixed bit width value
-   * @param n - encoded fixed bit width
-   * @return decoded fixed bit width
-   */
-  int decodeBitWidth(int n) {
-    if (n >= FixedBitSizes.ONE.ordinal()
-        && n <= FixedBitSizes.TWENTYFOUR.ordinal()) {
-      return n + 1;
-    } else if (n == FixedBitSizes.TWENTYSIX.ordinal()) {
-      return 26;
-    } else if (n == FixedBitSizes.TWENTYEIGHT.ordinal()) {
-      return 28;
-    } else if (n == FixedBitSizes.THIRTY.ordinal()) {
-      return 30;
-    } else if (n == FixedBitSizes.THIRTYTWO.ordinal()) {
-      return 32;
-    } else if (n == FixedBitSizes.FORTY.ordinal()) {
-      return 40;
-    } else if (n == FixedBitSizes.FORTYEIGHT.ordinal()) {
-      return 48;
-    } else if (n == FixedBitSizes.FIFTYSIX.ordinal()) {
-      return 56;
-    } else {
-      return 64;
-    }
-  }
-
-  /**
-   * Bitpack and write the input values to underlying output stream
-   * @param input - values to write
-   * @param offset - offset
-   * @param len - length
-   * @param bitSize - bit width
-   * @param output - output stream
-   * @throws IOException
-   */
-  void writeInts(long[] input, int offset, int len, int bitSize,
-                        OutputStream output) throws IOException {
-    if (input == null || input.length < 1 || offset < 0 || len < 1
-        || bitSize < 1) {
-      return;
-    }
-
-    switch (bitSize) {
-    case 1:
-      unrolledBitPack1(input, offset, len, output);
-      return;
-    case 2:
-      unrolledBitPack2(input, offset, len, output);
-      return;
-    case 4:
-      unrolledBitPack4(input, offset, len, output);
-      return;
-    case 8:
-      unrolledBitPack8(input, offset, len, output);
-      return;
-    case 16:
-      unrolledBitPack16(input, offset, len, output);
-      return;
-    case 24:
-      unrolledBitPack24(input, offset, len, output);
-      return;
-    case 32:
-      unrolledBitPack32(input, offset, len, output);
-      return;
-    case 40:
-      unrolledBitPack40(input, offset, len, output);
-      return;
-    case 48:
-      unrolledBitPack48(input, offset, len, output);
-      return;
-    case 56:
-      unrolledBitPack56(input, offset, len, output);
-      return;
-    case 64:
-      unrolledBitPack64(input, offset, len, output);
-      return;
-    default:
-      break;
-    }
-
-    int bitsLeft = 8;
-    byte current = 0;
-    for(int i = offset; i < (offset + len); i++) {
-      long value = input[i];
-      int bitsToWrite = bitSize;
-      while (bitsToWrite > bitsLeft) {
-        // add the bits to the bottom of the current word
-        current |= value >>> (bitsToWrite - bitsLeft);
-        // subtract out the bits we just added
-        bitsToWrite -= bitsLeft;
-        // zero out the bits above bitsToWrite
-        value &= (1L << bitsToWrite) - 1;
-        output.write(current);
-        current = 0;
-        bitsLeft = 8;
-      }
-      bitsLeft -= bitsToWrite;
-      current |= value << bitsLeft;
-      if (bitsLeft == 0) {
-        output.write(current);
-        current = 0;
-        bitsLeft = 8;
-      }
-    }
-
-    // flush
-    if (bitsLeft != 8) {
-      output.write(current);
-      current = 0;
-      bitsLeft = 8;
-    }
-  }
-
-  private void unrolledBitPack1(long[] input, int offset, int len,
-      OutputStream output) throws IOException {
-    final int numHops = 8;
-    final int remainder = len % numHops;
-    final int endOffset = offset + len;
-    final int endUnroll = endOffset - remainder;
-    int val = 0;
-    for (int i = offset; i < endUnroll; i = i + numHops) {
-      val = (int) (val | ((input[i] & 1) << 7)
-          | ((input[i + 1] & 1) << 6)
-          | ((input[i + 2] & 1) << 5)
-          | ((input[i + 3] & 1) << 4)
-          | ((input[i + 4] & 1) << 3)
-          | ((input[i + 5] & 1) << 2)
-          | ((input[i + 6] & 1) << 1)
-          | (input[i + 7]) & 1);
-      output.write(val);
-      val = 0;
-    }
-
-    if (remainder > 0) {
-      int startShift = 7;
-      for (int i = endUnroll; i < endOffset; i++) {
-        val = (int) (val | (input[i] & 1) << startShift);
-        startShift -= 1;
-      }
-      output.write(val);
-    }
-  }
-
-  private void unrolledBitPack2(long[] input, int offset, int len,
-      OutputStream output) throws IOException {
-    final int numHops = 4;
-    final int remainder = len % numHops;
-    final int endOffset = offset + len;
-    final int endUnroll = endOffset - remainder;
-    int val = 0;
-    for (int i = offset; i < endUnroll; i = i + numHops) {
-      val = (int) (val | ((input[i] & 3) << 6)
-          | ((input[i + 1] & 3) << 4)
-          | ((input[i + 2] & 3) << 2)
-          | (input[i + 3]) & 3);
-      output.write(val);
-      val = 0;
-    }
-
-    if (remainder > 0) {
-      int startShift = 6;
-      for (int i = endUnroll; i < endOffset; i++) {
-        val = (int) (val | (input[i] & 3) << startShift);
-        startShift -= 2;
-      }
-      output.write(val);
-    }
-  }
-
-  private void unrolledBitPack4(long[] input, int offset, int len,
-      OutputStream output) throws IOException {
-    final int numHops = 2;
-    final int remainder = len % numHops;
-    final int endOffset = offset + len;
-    final int endUnroll = endOffset - remainder;
-    int val = 0;
-    for (int i = offset; i < endUnroll; i = i + numHops) {
-      val = (int) (val | ((input[i] & 15) << 4) | (input[i + 1]) & 15);
-      output.write(val);
-      val = 0;
-    }
-
-    if (remainder > 0) {
-      int startShift = 4;
-      for (int i = endUnroll; i < endOffset; i++) {
-        val = (int) (val | (input[i] & 15) << startShift);
-        startShift -= 4;
-      }
-      output.write(val);
-    }
-  }
-
-  private void unrolledBitPack8(long[] input, int offset, int len,
-      OutputStream output) throws IOException {
-    unrolledBitPackBytes(input, offset, len, output, 1);
-  }
-
-  private void unrolledBitPack16(long[] input, int offset, int len,
-      OutputStream output) throws IOException {
-    unrolledBitPackBytes(input, offset, len, output, 2);
-  }
-
-  private void unrolledBitPack24(long[] input, int offset, int len,
-      OutputStream output) throws IOException {
-    unrolledBitPackBytes(input, offset, len, output, 3);
-  }
-
-  private void unrolledBitPack32(long[] input, int offset, int len,
-      OutputStream output) throws IOException {
-    unrolledBitPackBytes(input, offset, len, output, 4);
-  }
-
-  private void unrolledBitPack40(long[] input, int offset, int len,
-      OutputStream output) throws IOException {
-    unrolledBitPackBytes(input, offset, len, output, 5);
-  }
-
-  private void unrolledBitPack48(long[] input, int offset, int len,
-      OutputStream output) throws IOException {
-    unrolledBitPackBytes(input, offset, len, output, 6);
-  }
-
-  private void unrolledBitPack56(long[] input, int offset, int len,
-      OutputStream output) throws IOException {
-    unrolledBitPackBytes(input, offset, len, output, 7);
-  }
-
-  private void unrolledBitPack64(long[] input, int offset, int len,
-      OutputStream output) throws IOException {
-    unrolledBitPackBytes(input, offset, len, output, 8);
-  }
-
-  private void unrolledBitPackBytes(long[] input, int offset, int len, OutputStream output, int numBytes) throws IOException {
-    final int numHops = 8;
-    final int remainder = len % numHops;
-    final int endOffset = offset + len;
-    final int endUnroll = endOffset - remainder;
-    int i = offset;
-    for (; i < endUnroll; i = i + numHops) {
-      writeLongBE(output, input, i, numHops, numBytes);
-    }
-
-    if (remainder > 0) {
-      writeRemainingLongs(output, i, input, remainder, numBytes);
-    }
-  }
-
-  private void writeRemainingLongs(OutputStream output, int offset, long[] input, int remainder,
-      int numBytes) throws IOException {
-    final int numHops = remainder;
-
-    int idx = 0;
-    switch (numBytes) {
-    case 1:
-      while (remainder > 0) {
-        writeBuffer[idx] = (byte) (input[offset + idx] & 255);
-        remainder--;
-        idx++;
-      }
-      break;
-    case 2:
-      while (remainder > 0) {
-        writeLongBE2(output, input[offset + idx], idx * 2);
-        remainder--;
-        idx++;
-      }
-      break;
-    case 3:
-      while (remainder > 0) {
-        writeLongBE3(output, input[offset + idx], idx * 3);
-        remainder--;
-        idx++;
-      }
-      break;
-    case 4:
-      while (remainder > 0) {
-        writeLongBE4(output, input[offset + idx], idx * 4);
-        remainder--;
-        idx++;
-      }
-      break;
-    case 5:
-      while (remainder > 0) {
-        writeLongBE5(output, input[offset + idx], idx * 5);
-        remainder--;
-        idx++;
-      }
-      break;
-    case 6:
-      while (remainder > 0) {
-        writeLongBE6(output, input[offset + idx], idx * 6);
-        remainder--;
-        idx++;
-      }
-      break;
-    case 7:
-      while (remainder > 0) {
-        writeLongBE7(output, input[offset + idx], idx * 7);
-        remainder--;
-        idx++;
-      }
-      break;
-    case 8:
-      while (remainder > 0) {
-        writeLongBE8(output, input[offset + idx], idx * 8);
-        remainder--;
-        idx++;
-      }
-      break;
-    default:
-      break;
-    }
-
-    final int toWrite = numHops * numBytes;
-    output.write(writeBuffer, 0, toWrite);
-  }
-
-  private void writeLongBE(OutputStream output, long[] input, int offset, int numHops, int numBytes) throws IOException {
-
-    switch (numBytes) {
-    case 1:
-      writeBuffer[0] = (byte) (input[offset + 0] & 255);
-      writeBuffer[1] = (byte) (input[offset + 1] & 255);
-      writeBuffer[2] = (byte) (input[offset + 2] & 255);
-      writeBuffer[3] = (byte) (input[offset + 3] & 255);
-      writeBuffer[4] = (byte) (input[offset + 4] & 255);
-      writeBuffer[5] = (byte) (input[offset + 5] & 255);
-      writeBuffer[6] = (byte) (input[offset + 6] & 255);
-      writeBuffer[7] = (byte) (input[offset + 7] & 255);
-      break;
-    case 2:
-      writeLongBE2(output, input[offset + 0], 0);
-      writeLongBE2(output, input[offset + 1], 2);
-      writeLongBE2(output, input[offset + 2], 4);
-      writeLongBE2(output, input[offset + 3], 6);
-      writeLongBE2(output, input[offset + 4], 8);
-      writeLongBE2(output, input[offset + 5], 10);
-      writeLongBE2(output, input[offset + 6], 12);
-      writeLongBE2(output, input[offset + 7], 14);
-      break;
-    case 3:
-      writeLongBE3(output, input[offset + 0], 0);
-      writeLongBE3(output, input[offset + 1], 3);
-      writeLongBE3(output, input[offset + 2], 6);
-      writeLongBE3(output, input[offset + 3], 9);
-      writeLongBE3(output, input[offset + 4], 12);
-      writeLongBE3(output, input[offset + 5], 15);
-      writeLongBE3(output, input[offset + 6], 18);
-      writeLongBE3(output, input[offset + 7], 21);
-      break;
-    case 4:
-      writeLongBE4(output, input[offset + 0], 0);
-      writeLongBE4(output, input[offset + 1], 4);
-      writeLongBE4(output, input[offset + 2], 8);
-      writeLongBE4(output, input[offset + 3], 12);
-      writeLongBE4(output, input[offset + 4], 16);
-      writeLongBE4(output, input[offset + 5], 20);
-      writeLongBE4(output, input[offset + 6], 24);
-      writeLongBE4(output, input[offset + 7], 28);
-      break;
-    case 5:
-      writeLongBE5(output, input[offset + 0], 0);
-      writeLongBE5(output, input[offset + 1], 5);
-      writeLongBE5(output, input[offset + 2], 10);
-      writeLongBE5(output, input[offset + 3], 15);
-      writeLongBE5(output, input[offset + 4], 20);
-      writeLongBE5(output, input[offset + 5], 25);
-      writeLongBE5(output, input[offset + 6], 30);
-      writeLongBE5(output, input[offset + 7], 35);
-      break;
-    case 6:
-      writeLongBE6(output, input[offset + 0], 0);
-      writeLongBE6(output, input[offset + 1], 6);
-      writeLongBE6(output, input[offset + 2], 12);
-      writeLongBE6(output, input[offset + 3], 18);
-      writeLongBE6(output, input[offset + 4], 24);
-      writeLongBE6(output, input[offset + 5], 30);
-      writeLongBE6(output, input[offset + 6], 36);
-      writeLongBE6(output, input[offset + 7], 42);
-      break;
-    case 7:
-      writeLongBE7(output, input[offset + 0], 0);
-      writeLongBE7(output, input[offset + 1], 7);
-      writeLongBE7(output, input[offset + 2], 14);
-      writeLongBE7(output, input[offset + 3], 21);
-      writeLongBE7(output, input[offset + 4], 28);
-      writeLongBE7(output, input[offset + 5], 35);
-      writeLongBE7(output, input[offset + 6], 42);
-      writeLongBE7(output, input[offset + 7], 49);
-      break;
-    case 8:
-      writeLongBE8(output, input[offset + 0], 0);
-      writeLongBE8(output, input[offset + 1], 8);
-      writeLongBE8(output, input[offset + 2], 16);
-      writeLongBE8(output, input[offset + 3], 24);
-      writeLongBE8(output, input[offset + 4], 32);
-      writeLongBE8(output, input[offset + 5], 40);
-      writeLongBE8(output, input[offset + 6], 48);
-      writeLongBE8(output, input[offset + 7], 56);
-      break;
-      default:
-        break;
-    }
-
-    final int toWrite = numHops * numBytes;
-    output.write(writeBuffer, 0, toWrite);
-  }
-
-  private void writeLongBE2(OutputStream output, long val, int wbOffset) {
-    writeBuffer[wbOffset + 0] =  (byte) (val >>> 8);
-    writeBuffer[wbOffset + 1] =  (byte) (val >>> 0);
-  }
-
-  private void writeLongBE3(OutputStream output, long val, int wbOffset) {
-    writeBuffer[wbOffset + 0] =  (byte) (val >>> 16);
-    writeBuffer[wbOffset + 1] =  (byte) (val >>> 8);
-    writeBuffer[wbOffset + 2] =  (byte) (val >>> 0);
-  }
-
-  private void writeLongBE4(OutputStream output, long val, int wbOffset) {
-    writeBuffer[wbOffset + 0] =  (byte) (val >>> 24);
-    writeBuffer[wbOffset + 1] =  (byte) (val >>> 16);
-    writeBuffer[wbOffset + 2] =  (byte) (val >>> 8);
-    writeBuffer[wbOffset + 3] =  (byte) (val >>> 0);
-  }
-
-  private void writeLongBE5(OutputStream output, long val, int wbOffset) {
-    writeBuffer[wbOffset + 0] =  (byte) (val >>> 32);
-    writeBuffer[wbOffset + 1] =  (byte) (val >>> 24);
-    writeBuffer[wbOffset + 2] =  (byte) (val >>> 16);
-    writeBuffer[wbOffset + 3] =  (byte) (val >>> 8);
-    writeBuffer[wbOffset + 4] =  (byte) (val >>> 0);
-  }
-
-  private void writeLongBE6(OutputStream output, long val, int wbOffset) {
-    writeBuffer[wbOffset + 0] =  (byte) (val >>> 40);
-    writeBuffer[wbOffset + 1] =  (byte) (val >>> 32);
-    writeBuffer[wbOffset + 2] =  (byte) (val >>> 24);
-    writeBuffer[wbOffset + 3] =  (byte) (val >>> 16);
-    writeBuffer[wbOffset + 4] =  (byte) (val >>> 8);
-    writeBuffer[wbOffset + 5] =  (byte) (val >>> 0);
-  }
-
-  private void writeLongBE7(OutputStream output, long val, int wbOffset) {
-    writeBuffer[wbOffset + 0] =  (byte) (val >>> 48);
-    writeBuffer[wbOffset + 1] =  (byte) (val >>> 40);
-    writeBuffer[wbOffset + 2] =  (byte) (val >>> 32);
-    writeBuffer[wbOffset + 3] =  (byte) (val >>> 24);
-    writeBuffer[wbOffset + 4] =  (byte) (val >>> 16);
-    writeBuffer[wbOffset + 5] =  (byte) (val >>> 8);
-    writeBuffer[wbOffset + 6] =  (byte) (val >>> 0);
-  }
-
-  private void writeLongBE8(OutputStream output, long val, int wbOffset) {
-    writeBuffer[wbOffset + 0] =  (byte) (val >>> 56);
-    writeBuffer[wbOffset + 1] =  (byte) (val >>> 48);
-    writeBuffer[wbOffset + 2] =  (byte) (val >>> 40);
-    writeBuffer[wbOffset + 3] =  (byte) (val >>> 32);
-    writeBuffer[wbOffset + 4] =  (byte) (val >>> 24);
-    writeBuffer[wbOffset + 5] =  (byte) (val >>> 16);
-    writeBuffer[wbOffset + 6] =  (byte) (val >>> 8);
-    writeBuffer[wbOffset + 7] =  (byte) (val >>> 0);
-  }
-
-  // Do not want to use Guava LongMath.checkedSubtract() here as it will throw
-  // ArithmeticException in case of overflow
-  public boolean isSafeSubtract(long left, long right) {
-    return (left ^ right) >= 0 | (left ^ (left - right)) >= 0;
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/SnappyCodec.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/SnappyCodec.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/SnappyCodec.java
deleted file mode 100644
index 285a32a..0000000
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/SnappyCodec.java
+++ /dev/null
@@ -1,109 +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.tajo.storage.thirdparty.orc;
-
-import org.apache.hadoop.hive.shims.HadoopShims.DirectCompressionType;
-import org.apache.hadoop.hive.shims.HadoopShims.DirectDecompressorShim;
-import org.apache.hadoop.hive.shims.ShimLoader;
-import org.iq80.snappy.Snappy;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.EnumSet;
-
-class SnappyCodec implements CompressionCodec, DirectDecompressionCodec {
-
-  Boolean direct = null;
-
-  @Override
-  public boolean compress(ByteBuffer in, ByteBuffer out,
-                          ByteBuffer overflow) throws IOException {
-    int inBytes = in.remaining();
-    // I should work on a patch for Snappy to support an overflow buffer
-    // to prevent the extra buffer copy.
-    byte[] compressed = new byte[Snappy.maxCompressedLength(inBytes)];
-    int outBytes =
-        Snappy.compress(in.array(), in.arrayOffset() + in.position(), inBytes,
-          compressed, 0);
-    if (outBytes < inBytes) {
-      int remaining = out.remaining();
-      if (remaining >= outBytes) {
-        System.arraycopy(compressed, 0, out.array(), out.arrayOffset() +
-            out.position(), outBytes);
-        out.position(out.position() + outBytes);
-      } else {
-        System.arraycopy(compressed, 0, out.array(), out.arrayOffset() +
-            out.position(), remaining);
-        out.position(out.limit());
-        System.arraycopy(compressed, remaining, overflow.array(),
-            overflow.arrayOffset(), outBytes - remaining);
-        overflow.position(outBytes - remaining);
-      }
-      return true;
-    } else {
-      return false;
-    }
-  }
-
-  @Override
-  public void decompress(ByteBuffer in, ByteBuffer out) throws IOException {
-    if(in.isDirect() && out.isDirect()) {
-      directDecompress(in, out);
-      return;
-    }
-    int inOffset = in.position();
-    int uncompressLen =
-        Snappy.uncompress(in.array(), in.arrayOffset() + inOffset,
-          in.limit() - inOffset, out.array(), out.arrayOffset() + out.position());
-    out.position(uncompressLen + out.position());
-    out.flip();
-  }
-
-  @Override
-  public boolean isAvailable() {
-    if (direct == null) {
-      try {
-        if (ShimLoader.getHadoopShims().getDirectDecompressor(
-            DirectCompressionType.SNAPPY) != null) {
-          direct = Boolean.valueOf(true);
-        } else {
-          direct = Boolean.valueOf(false);
-        }
-      } catch (UnsatisfiedLinkError ule) {
-        direct = Boolean.valueOf(false);
-      }
-    }
-    return direct.booleanValue();
-  }
-
-  @Override
-  public void directDecompress(ByteBuffer in, ByteBuffer out)
-      throws IOException {
-    DirectDecompressorShim decompressShim = ShimLoader.getHadoopShims()
-        .getDirectDecompressor(DirectCompressionType.SNAPPY);
-    decompressShim.decompress(in, out);
-    out.flip(); // flip for read
-  }
-
-  @Override
-  public CompressionCodec modify(EnumSet<Modifier> modifiers) {
-    // snappy allows no modifications
-    return this;
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/StreamName.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/StreamName.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/StreamName.java
deleted file mode 100644
index 3821645..0000000
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/StreamName.java
+++ /dev/null
@@ -1,95 +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.tajo.storage.thirdparty.orc;
-
-/**
- * The name of a stream within a stripe.
- */
-class StreamName implements Comparable<StreamName> {
-  private final int column;
-  private final OrcProto.Stream.Kind kind;
-
-  public enum Area {
-    DATA, INDEX
-  }
-
-  public StreamName(int column, OrcProto.Stream.Kind kind) {
-    this.column = column;
-    this.kind = kind;
-  }
-
-  public boolean equals(Object obj) {
-    if (obj != null && obj instanceof StreamName) {
-      StreamName other = (StreamName) obj;
-      return other.column == column && other.kind == kind;
-    } else {
-      return false;
-    }
-  }
-
-  @Override
-  public int compareTo(StreamName streamName) {
-    if (streamName == null) {
-      return -1;
-    }
-    Area area = getArea(kind);
-    Area otherArea = StreamName.getArea(streamName.kind);
-    if (area != otherArea) {
-      return -area.compareTo(otherArea);
-    }
-    if (column != streamName.column) {
-      return column < streamName.column ? -1 : 1;
-    }
-    return kind.compareTo(streamName.kind);
-  }
-
-  public int getColumn() {
-    return column;
-  }
-
-  public OrcProto.Stream.Kind getKind() {
-    return kind;
-  }
-
-  public Area getArea() {
-    return getArea(kind);
-  }
-
-  public static Area getArea(OrcProto.Stream.Kind kind) {
-    switch (kind) {
-      case ROW_INDEX:
-      case DICTIONARY_COUNT:
-      case BLOOM_FILTER:
-        return Area.INDEX;
-      default:
-        return Area.DATA;
-    }
-  }
-
-  @Override
-  public String toString() {
-    return "Stream for column " + column + " kind " + kind;
-  }
-
-  @Override
-  public int hashCode() {
-    return column * 101 + kind.getNumber();
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/StringColumnStatistics.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/StringColumnStatistics.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/StringColumnStatistics.java
deleted file mode 100644
index 4248664..0000000
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/StringColumnStatistics.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.tajo.storage.thirdparty.orc;
-
-/**
- * Statistics for string columns.
- */
-public interface StringColumnStatistics extends ColumnStatistics {
-  /**
-   * Get the minimum string.
-   * @return the minimum
-   */
-  String getMinimum();
-
-  /**
-   * Get the maximum string.
-   * @return the maximum
-   */
-  String getMaximum();
-
-  /**
-   * Get the total length of all strings
-   * @return the sum (total length)
-   */
-  long getSum();
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/StringRedBlackTree.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/StringRedBlackTree.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/StringRedBlackTree.java
deleted file mode 100644
index 8835cef..0000000
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/StringRedBlackTree.java
+++ /dev/null
@@ -1,202 +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.tajo.storage.thirdparty.orc;
-
-import org.apache.hadoop.io.Text;
-
-import java.io.IOException;
-import java.io.OutputStream;
-
-/**
- * A red-black tree that stores strings. The strings are stored as UTF-8 bytes
- * and an offset for each entry.
- */
-class StringRedBlackTree extends RedBlackTree {
-  private final DynamicByteArray byteArray = new DynamicByteArray();
-  private final DynamicIntArray keyOffsets;
-  private String newKey;
-
-  public StringRedBlackTree(int initialCapacity) {
-    super(initialCapacity);
-    keyOffsets = new DynamicIntArray(initialCapacity);
-  }
-
-  public int add(String value) {
-    newKey = value;
-    return addNewKey();
-  }
-
-  private int addNewKey() {
-    // if the newKey is actually new, add it to our byteArray and store the offset & length
-    if (add()) {
-      int len = newKey.length();
-      keyOffsets.add(byteArray.add(newKey.getBytes(), 0, len));
-    }
-    return lastAdd;
-  }
-
-  public int add(Text value) {
-    newKey = value.toString();
-    return addNewKey();
-  }
-
-  @Override
-  protected int compareValue(int position) {
-    int start = keyOffsets.get(position);
-    int end;
-    if (position + 1 == keyOffsets.size()) {
-      end = byteArray.size();
-    } else {
-      end = keyOffsets.get(position+1);
-    }
-    return byteArray.compare(newKey.getBytes(), 0, newKey.length(),
-                             start, end - start);
-  }
-
-  /**
-   * The information about each node.
-   */
-  public interface VisitorContext {
-    /**
-     * Get the position where the key was originally added.
-     * @return the number returned by add.
-     */
-    int getOriginalPosition();
-
-    /**
-     * Write the bytes for the string to the given output stream.
-     * @param out the stream to write to.
-     * @throws IOException
-     */
-    void writeBytes(OutputStream out) throws IOException;
-
-    /**
-     * Get the original string.
-     * @return the string
-     */
-    Text getText();
-
-    /**
-     * Get the number of bytes.
-     * @return the string's length in bytes
-     */
-    int getLength();
-  }
-
-  /**
-   * The interface for visitors.
-   */
-  public interface Visitor {
-    /**
-     * Called once for each node of the tree in sort order.
-     * @param context the information about each node
-     * @throws IOException
-     */
-    void visit(VisitorContext context) throws IOException;
-  }
-
-  private class VisitorContextImpl implements VisitorContext {
-    private int originalPosition;
-    private int start;
-    private int end;
-    private final Text text = new Text();
-
-    public int getOriginalPosition() {
-      return originalPosition;
-    }
-
-    public Text getText() {
-      byteArray.setText(text, start, end - start);
-      return text;
-    }
-
-    public void writeBytes(OutputStream out) throws IOException {
-      byteArray.write(out, start, end - start);
-    }
-
-    public int getLength() {
-      return end - start;
-    }
-
-    void setPosition(int position) {
-      originalPosition = position;
-      start = keyOffsets.get(originalPosition);
-      if (position + 1 == keyOffsets.size()) {
-        end = byteArray.size();
-      } else {
-        end = keyOffsets.get(originalPosition + 1);
-      }
-    }
-  }
-
-  private void recurse(int node, Visitor visitor, VisitorContextImpl context
-                      ) throws IOException {
-    if (node != NULL) {
-      recurse(getLeft(node), visitor, context);
-      context.setPosition(node);
-      visitor.visit(context);
-      recurse(getRight(node), visitor, context);
-    }
-  }
-
-  /**
-   * Visit all of the nodes in the tree in sorted order.
-   * @param visitor the action to be applied to each node
-   * @throws IOException
-   */
-  public void visit(Visitor visitor) throws IOException {
-    recurse(root, visitor, new VisitorContextImpl());
-  }
-
-  /**
-   * Reset the table to empty.
-   */
-  public void clear() {
-    super.clear();
-    byteArray.clear();
-    keyOffsets.clear();
-  }
-
-  public void getText(Text result, int originalPosition) {
-    int offset = keyOffsets.get(originalPosition);
-    int length;
-    if (originalPosition + 1 == keyOffsets.size()) {
-      length = byteArray.size() - offset;
-    } else {
-      length = keyOffsets.get(originalPosition + 1) - offset;
-    }
-    byteArray.setText(result, offset, length);
-  }
-
-  /**
-   * Get the size of the character data in the table.
-   * @return the bytes used by the table
-   */
-  public int getCharacterSize() {
-    return byteArray.size();
-  }
-
-  /**
-   * Calculate the approximate size in memory.
-   * @return the number of bytes used in storing the tree.
-   */
-  public long getSizeInBytes() {
-    return byteArray.getSizeInBytes() + keyOffsets.getSizeInBytes() +
-      super.getSizeInBytes();
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/StripeInformation.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/StripeInformation.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/StripeInformation.java
deleted file mode 100644
index 62819c1..0000000
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/StripeInformation.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.tajo.storage.thirdparty.orc;
-
-/**
- * Information about the stripes in an ORC file that is provided by the Reader.
- */
-public interface StripeInformation {
-  /**
-   * Get the byte offset of the start of the stripe.
-   * @return the bytes from the start of the file
-   */
-  long getOffset();
-
-  /**
-   * Get the total length of the stripe in bytes.
-   * @return the number of bytes in the stripe
-   */
-  long getLength();
-
-  /**
-   * Get the length of the stripe's indexes.
-   * @return the number of bytes in the index
-   */
-  long getIndexLength();
-
-  /**
-   * Get the length of the stripe's data.
-   * @return the number of bytes in the stripe
-   */
-  long getDataLength();
-
-  /**
-   * Get the length of the stripe's tail section, which contains its index.
-   * @return the number of bytes in the tail
-   */
-  long getFooterLength();
-
-  /**
-   * Get the number of rows in the stripe.
-   * @return a count of the number of rows
-   */
-  long getNumberOfRows();
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/StripeStatistics.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/StripeStatistics.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/StripeStatistics.java
deleted file mode 100644
index 013fc8e..0000000
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/StripeStatistics.java
+++ /dev/null
@@ -1,42 +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.tajo.storage.thirdparty.orc;
-
-import java.util.List;
-
-public class StripeStatistics {
-  private final List<OrcProto.ColumnStatistics> cs;
-
-  StripeStatistics(List<OrcProto.ColumnStatistics> list) {
-    this.cs = list;
-  }
-
-  /**
-   * Return list of column statistics
-   *
-   * @return column stats
-   */
-  public ColumnStatistics[] getColumnStatistics() {
-    ColumnStatistics[] result = new ColumnStatistics[cs.size()];
-    for (int i = 0; i < result.length; ++i) {
-      result[i] = ColumnStatisticsImpl.deserialize(cs.get(i));
-    }
-    return result;
-  }
-}

http://git-wip-us.apache.org/repos/asf/tajo/blob/68263585/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/TimestampColumnStatistics.java
----------------------------------------------------------------------
diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/TimestampColumnStatistics.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/TimestampColumnStatistics.java
deleted file mode 100644
index 6fad0ac..0000000
--- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/thirdparty/orc/TimestampColumnStatistics.java
+++ /dev/null
@@ -1,38 +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.tajo.storage.thirdparty.orc;
-
-import java.sql.Timestamp;
-
-/**
- * Statistics for Timestamp columns.
- */
-public interface TimestampColumnStatistics extends ColumnStatistics {
-  /**
-   * Get the minimum value for the column.
-   * @return minimum value
-   */
-  Timestamp getMinimum();
-
-  /**
-   * Get the maximum value for the column.
-   * @return maximum value
-   */
-  Timestamp getMaximum();
-}