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

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

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

<TRUNCATED>