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

[41/45] hive git commit: HIVE-13178: Enhance ORC Schema Evolution to handle more standard data type conversions (Matt McCline, reviewed by Prasanth Jayachandran)

http://git-wip-us.apache.org/repos/asf/hive/blob/a16058e1/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
new file mode 100644
index 0000000..74a097e
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/ConvertTreeReaderFactory.java
@@ -0,0 +1,3750 @@
+/**
+ * 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) doubleColVector

<TRUNCATED>