You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@orc.apache.org by om...@apache.org on 2016/07/01 17:56:19 UTC

[4/8] orc git commit: HIVE-14000 Changing a numeric type column causes values other than NULL.

HIVE-14000 Changing a numeric type column causes values other than NULL.

Signed-off-by: Owen O'Malley <om...@apache.org>


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

Branch: refs/heads/branch-1.1
Commit: 63829ed0bce7def4d3f5b760346d22318feb51dd
Parents: 6638772
Author: Owen O'Malley <om...@apache.org>
Authored: Thu Jun 30 10:21:24 2016 -0700
Committer: Owen O'Malley <om...@apache.org>
Committed: Fri Jul 1 09:33:02 2016 -0700

----------------------------------------------------------------------
 .../orc/impl/ConvertTreeReaderFactory.java      | 305 ++++++++++---------
 .../hadoop/hive/ql/util/TimestampUtils.java     |  70 +++--
 2 files changed, 206 insertions(+), 169 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/orc/blob/63829ed0/java/core/src/java/org/apache/orc/impl/ConvertTreeReaderFactory.java
----------------------------------------------------------------------
diff --git a/java/core/src/java/org/apache/orc/impl/ConvertTreeReaderFactory.java b/java/core/src/java/org/apache/orc/impl/ConvertTreeReaderFactory.java
index 3ba56f7..753e5bc 100644
--- a/java/core/src/java/org/apache/orc/impl/ConvertTreeReaderFactory.java
+++ b/java/core/src/java/org/apache/orc/impl/ConvertTreeReaderFactory.java
@@ -35,8 +35,6 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.StringExpr;
 import org.apache.hadoop.hive.ql.util.TimestampUtils;
 import org.apache.hadoop.hive.serde2.io.DateWritable;
 import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
-import org.apache.hadoop.io.BytesWritable;
-import org.apache.hadoop.io.FloatWritable;
 import org.apache.orc.OrcProto;
 import org.apache.orc.TypeDescription;
 import org.apache.orc.TypeDescription.Category;
@@ -263,6 +261,22 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
       return string;
     }
 
+    private static final double MIN_LONG_AS_DOUBLE = -0x1p63;
+    /*
+     * We cannot store Long.MAX_VALUE as a double without losing precision. Instead, we store
+     * Long.MAX_VALUE + 1 == -Long.MIN_VALUE, and then offset all comparisons by 1.
+     */
+    private static final double MAX_LONG_AS_DOUBLE_PLUS_ONE = 0x1p63;
+
+    public boolean doubleCanFitInLong(double doubleValue) {
+
+      // Borrowed from Guava DoubleMath.roundToLong except do not want dependency on Guava and we
+      // don't want to catch an exception.
+
+      return ((MIN_LONG_AS_DOUBLE - doubleValue < 1.0) &&
+              (doubleValue < MAX_LONG_AS_DOUBLE_PLUS_ONE));
+    }
+
     @Override
     void checkEncoding(OrcProto.ColumnEncoding encoding) throws IOException {
       // Pass-thru.
@@ -336,20 +350,44 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
       }
     }
 
-    public long downCastAnyInteger(long input, TypeDescription readerType) {
-      switch (readerType.getCategory()) {
+    public void downCastAnyInteger(LongColumnVector longColVector, int elementNum,
+        TypeDescription readerType) {
+      downCastAnyInteger(longColVector, elementNum, longColVector.vector[elementNum], readerType);
+    }
+
+    public void downCastAnyInteger(LongColumnVector longColVector, int elementNum, long inputLong,
+        TypeDescription readerType) {
+      long[] vector = longColVector.vector;
+      long outputLong;
+      Category readerCategory = readerType.getCategory();
+      switch (readerCategory) {
       case BOOLEAN:
-        return input == 0 ? 0 : 1;
+        // No data loss for boolean.
+        vector[elementNum] = inputLong == 0 ? 0 : 1;
+        return;
       case BYTE:
-        return (byte) input;
+        outputLong = (byte) inputLong;
+        break;
       case SHORT:
-        return (short) input;
+        outputLong = (short) inputLong;
+        break;
       case INT:
-        return (int) input;
+        outputLong = (int) inputLong;
+        break;
       case LONG:
-        return input;
+        // No data loss for long.
+        vector[elementNum] = inputLong;
+        return;
       default:
-        throw new RuntimeException("Unexpected type kind " + readerType.getCategory().name());
+        throw new RuntimeException("Unexpected type kind " + readerCategory.name());
+      }
+
+      if (outputLong != inputLong) {
+        // Data loss.
+        longColVector.isNull[elementNum] = true;
+        longColVector.noNulls = false;
+      } else {
+        vector[elementNum] = outputLong;
       }
     }
 
@@ -439,25 +477,22 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
       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);
+            downCastAnyInteger(resultColVector, 0, readerType);
           } else {
-            resultColVector.noNulls = false;
-            resultColVector.isNull[0] = true;
+            // Result remains null.
           }
         } else if (resultColVector.noNulls){
           for (int i = 0; i < batchSize; i++) {
-            resultVector[i] = downCastAnyInteger(resultVector[i], readerType);
+            downCastAnyInteger(resultColVector, i, readerType);
           }
         } else {
           for (int i = 0; i < batchSize; i++) {
             if (!resultColVector.isNull[i]) {
-              resultVector[i] = downCastAnyInteger(resultVector[i], readerType);
+              downCastAnyInteger(resultColVector, i, readerType);
             } else {
-              resultColVector.noNulls = false;
-              resultColVector.isNull[i] = true;
+              // Result remains null.
             }
           }
         }
@@ -470,7 +505,6 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
     private FloatTreeReader floatTreeReader;
 
     private final TypeDescription readerType;
-    private FloatWritable floatResult;
     private DoubleColumnVector doubleColVector;
     private LongColumnVector longColVector;
 
@@ -480,15 +514,19 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
       this.readerType = readerType;
       floatTreeReader = new FloatTreeReader(columnId);
       setConvertTreeReader(floatTreeReader);
-      floatResult = new FloatWritable();
     }
 
     @Override
     public void setConvertVectorElement(int elementNum) throws IOException {
-      float floatValue = (float) doubleColVector.vector[elementNum];
-      longColVector.vector[elementNum] =
-          downCastAnyInteger(
-              (long) floatValue, readerType);
+      double doubleValue = doubleColVector.vector[elementNum];
+      if (!doubleCanFitInLong(doubleValue)) {
+        longColVector.isNull[elementNum] = true;
+        longColVector.noNulls = false;
+      } else {
+        // UNDONE: Does the overflow check above using double really work here for float?
+        float floatValue = (float) doubleValue;
+        downCastAnyInteger(longColVector, elementNum, (long) floatValue, readerType);
+      }
     }
 
     @Override
@@ -525,9 +563,13 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
 
     @Override
     public void setConvertVectorElement(int elementNum) throws IOException {
-      longColVector.vector[elementNum] =
-          downCastAnyInteger(
-              (long) doubleColVector.vector[elementNum], readerType);
+      double doubleValue = doubleColVector.vector[elementNum];
+      if (!doubleCanFitInLong(doubleValue)) {
+        longColVector.isNull[elementNum] = true;
+        longColVector.noNulls = false;
+      } else {
+        downCastAnyInteger(longColVector, elementNum, (long) doubleValue, readerType);
+      }
     }
 
     @Override
@@ -553,7 +595,6 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
     private final int precision;
     private final int scale;
     private final TypeDescription readerType;
-    private HiveDecimalWritable hiveDecimalResult;
     private DecimalColumnVector decimalColVector;
     private LongColumnVector longColVector;
 
@@ -565,15 +606,21 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
       this.readerType = readerType;
       decimalTreeReader = new DecimalTreeReader(columnId, precision, scale);
       setConvertTreeReader(decimalTreeReader);
-      hiveDecimalResult = new HiveDecimalWritable();
     }
 
+    private static HiveDecimal DECIMAL_MAX_LONG = HiveDecimal.create(Long.MAX_VALUE);
+    private static HiveDecimal DECIMAL_MIN_LONG = HiveDecimal.create(Long.MIN_VALUE);
+
     @Override
     public void setConvertVectorElement(int elementNum) throws IOException {
-      longColVector.vector[elementNum] =
-          downCastAnyInteger(
-              decimalColVector.vector[elementNum].getHiveDecimal().longValue(),
-              readerType);
+      HiveDecimal decimalValue = decimalColVector.vector[elementNum].getHiveDecimal();
+      if (decimalValue.compareTo(DECIMAL_MAX_LONG) > 0 ||
+          decimalValue.compareTo(DECIMAL_MIN_LONG) < 0) {
+        longColVector.isNull[elementNum] = true;
+        longColVector.noNulls = false;
+      } else {
+        downCastAnyInteger(longColVector, elementNum, decimalValue.longValue(), readerType);
+      }
     }
 
     @Override
@@ -596,7 +643,6 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
 
     private TreeReader stringGroupTreeReader;
 
-    private final TypeDescription fileType;
     private final TypeDescription readerType;
     private BytesColumnVector bytesColVector;
     private LongColumnVector longColVector;
@@ -604,7 +650,6 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
     AnyIntegerFromStringGroupTreeReader(int columnId, TypeDescription fileType,
         TypeDescription readerType) throws IOException {
       super(columnId);
-      this.fileType = fileType;
       this.readerType = readerType;
       stringGroupTreeReader = getStringGroupTreeReader(columnId, fileType);
       setConvertTreeReader(stringGroupTreeReader);
@@ -615,8 +660,7 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
       String string = stringFromBytesColumnVectorEntry(bytesColVector, elementNum);
       long longValue = parseLongFromString(string);
       if (!getIsParseError()) {
-        longColVector.vector[elementNum] =
-            downCastAnyInteger(longValue, readerType);
+        downCastAnyInteger(longColVector, elementNum, longValue, readerType);
       } else {
         longColVector.noNulls = false;
         longColVector.isNull[elementNum] = true;
@@ -660,8 +704,7 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
       // Use TimestampWritable's getSeconds.
       long longValue = TimestampUtils.millisToSeconds(
           timestampColVector.asScratchTimestamp(elementNum).getTime());
-      longColVector.vector[elementNum] =
-          downCastAnyInteger(longValue, readerType);
+      downCastAnyInteger(longColVector, elementNum, longValue, readerType);
     }
 
     @Override
@@ -745,8 +788,7 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
         if (resultColVector.noNulls || !resultColVector.isNull[0]) {
           resultVector[0] = (float) resultVector[0];
         } else {
-          resultColVector.noNulls = false;
-          resultColVector.isNull[0] = true;
+          // Remains null.
         }
       } else if (resultColVector.noNulls){
         for (int i = 0; i < batchSize; i++) {
@@ -757,8 +799,7 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
           if (!resultColVector.isNull[i]) {
             resultVector[i] = (float) resultVector[i];
           } else {
-            resultColVector.noNulls = false;
-            resultColVector.isNull[i] = true;
+            // Remains null.
           }
         }
       }
@@ -771,8 +812,6 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
 
     private final int precision;
     private final int scale;
-    private final TypeDescription readerType;
-    private HiveDecimalWritable hiveDecimalResult;
     private DecimalColumnVector decimalColVector;
     private DoubleColumnVector doubleColVector;
 
@@ -781,10 +820,8 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
       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
@@ -813,14 +850,12 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
 
     private TreeReader stringGroupTreeReader;
 
-    private final TypeDescription fileType;
     private BytesColumnVector bytesColVector;
     private DoubleColumnVector doubleColVector;
 
     FloatFromStringGroupTreeReader(int columnId, TypeDescription fileType)
         throws IOException {
       super(columnId);
-      this.fileType = fileType;
       stringGroupTreeReader = getStringGroupTreeReader(columnId, fileType);
       setConvertTreeReader(stringGroupTreeReader);
     }
@@ -858,14 +893,11 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
 
     private TimestampTreeReader timestampTreeReader;
 
-    private final TypeDescription readerType;
     private TimestampColumnVector timestampColVector;
     private DoubleColumnVector doubleColVector;
 
-    FloatFromTimestampTreeReader(int columnId, TypeDescription readerType,
-        boolean skipCorrupt) throws IOException {
+    FloatFromTimestampTreeReader(int columnId, boolean skipCorrupt) throws IOException {
       super(columnId);
-      this.readerType = readerType;
       timestampTreeReader = new TimestampTreeReader(columnId, skipCorrupt);
       setConvertTreeReader(timestampTreeReader);
     }
@@ -940,13 +972,10 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
 
     private FloatTreeReader floatTreeReader;
 
-    private FloatWritable floatResult;
-
     DoubleFromFloatTreeReader(int columnId) throws IOException {
       super(columnId);
       floatTreeReader = new FloatTreeReader(columnId);
       setConvertTreeReader(floatTreeReader);
-      floatResult = new FloatWritable();
     }
 
     @Override
@@ -964,20 +993,15 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
 
     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 {
+    DoubleFromDecimalTreeReader(int columnId, TypeDescription fileType) 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
@@ -1006,14 +1030,12 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
 
     private TreeReader stringGroupTreeReader;
 
-    private final TypeDescription fileType;
     private BytesColumnVector bytesColVector;
     private DoubleColumnVector doubleColVector;
 
     DoubleFromStringGroupTreeReader(int columnId, TypeDescription fileType)
         throws IOException {
       super(columnId);
-      this.fileType = fileType;
       stringGroupTreeReader = getStringGroupTreeReader(columnId, fileType);
       setConvertTreeReader(stringGroupTreeReader);
     }
@@ -1050,14 +1072,11 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
 
     private TimestampTreeReader timestampTreeReader;
 
-    private final TypeDescription readerType;
     private TimestampColumnVector timestampColVector;
     private DoubleColumnVector doubleColVector;
 
-    DoubleFromTimestampTreeReader(int columnId, TypeDescription readerType,
-        boolean skipCorrupt) throws IOException {
+    DoubleFromTimestampTreeReader(int columnId, boolean skipCorrupt) throws IOException {
       super(columnId);
-      this.readerType = readerType;
       timestampTreeReader = new TimestampTreeReader(columnId, skipCorrupt);
       setConvertTreeReader(timestampTreeReader);
     }
@@ -1088,16 +1107,12 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
 
     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 {
+    DecimalFromAnyIntegerTreeReader(int columnId, TypeDescription fileType, boolean skipCorrupt)
+        throws IOException {
       super(columnId);
-      this.precision = readerType.getPrecision();
-      this.scale = readerType.getScale();
       anyIntegerAsLongTreeReader =
           new AnyIntegerTreeReader(columnId, fileType, skipCorrupt);
       setConvertTreeReader(anyIntegerAsLongTreeReader);
@@ -1106,8 +1121,8 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
     @Override
     public void setConvertVectorElement(int elementNum) {
       long longValue = longColVector.vector[elementNum];
-      HiveDecimalWritable hiveDecimalWritable =
-          new HiveDecimalWritable(longValue);
+      HiveDecimalWritable hiveDecimalWritable = new HiveDecimalWritable(longValue);
+      // The DecimalColumnVector will enforce precision and scale and set the entry to null when out of bounds.
       decimalColVector.set(elementNum, hiveDecimalWritable);
     }
 
@@ -1131,30 +1146,25 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
 
     private FloatTreeReader floatTreeReader;
 
-    private int precision;
-    private int scale;
-    private FloatWritable floatResult;
     private DoubleColumnVector doubleColVector;
     private DecimalColumnVector decimalColVector;
 
     DecimalFromFloatTreeReader(int columnId, TypeDescription readerType)
         throws IOException {
       super(columnId);
-      this.precision = readerType.getPrecision();
-      this.scale = readerType.getScale();
       floatTreeReader = new FloatTreeReader(columnId);
       setConvertTreeReader(floatTreeReader);
-      floatResult = new FloatWritable();
     }
 
     @Override
     public void setConvertVectorElement(int elementNum) throws IOException {
       float floatValue = (float) doubleColVector.vector[elementNum];
       if (!Float.isNaN(floatValue)) {
-        HiveDecimal value =
+        HiveDecimal decimalValue =
             HiveDecimal.create(Float.toString(floatValue));
-        if (value != null) {
-          decimalColVector.set(elementNum, value);
+        if (decimalValue != null) {
+          // The DecimalColumnVector will enforce precision and scale and set the entry to null when out of bounds.
+          decimalColVector.set(elementNum, decimalValue);
         } else {
           decimalColVector.noNulls = false;
           decimalColVector.isNull[elementNum] = true;
@@ -1227,14 +1237,12 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
 
     private TreeReader stringGroupTreeReader;
 
-    private final TypeDescription fileType;
     private BytesColumnVector bytesColVector;
     private DecimalColumnVector decimalColVector;
 
     DecimalFromStringGroupTreeReader(int columnId, TypeDescription fileType,
         TypeDescription readerType) throws IOException {
       super(columnId);
-      this.fileType = fileType;
       stringGroupTreeReader = getStringGroupTreeReader(columnId, fileType);
       setConvertTreeReader(stringGroupTreeReader);
     }
@@ -1244,6 +1252,7 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
       String string = stringFromBytesColumnVectorEntry(bytesColVector, elementNum);
       HiveDecimal value = parseDecimalFromString(string);
       if (value != null) {
+        // The DecimalColumnVector will enforce precision and scale and set the entry to null when out of bounds.
         decimalColVector.set(elementNum, value);
       } else {
         decimalColVector.noNulls = false;
@@ -1271,18 +1280,11 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
 
     private TimestampTreeReader timestampTreeReader;
 
-    private final TypeDescription readerType;
     private TimestampColumnVector timestampColVector;
-    private int precision;
-    private int scale;
     private DecimalColumnVector decimalColVector;
 
-    DecimalFromTimestampTreeReader(int columnId, TypeDescription readerType,
-        boolean skipCorrupt) throws IOException {
+    DecimalFromTimestampTreeReader(int columnId, boolean skipCorrupt) throws IOException {
       super(columnId);
-      this.readerType = readerType;
-      this.precision = readerType.getPrecision();
-      this.scale = readerType.getScale();
       timestampTreeReader = new TimestampTreeReader(columnId, skipCorrupt);
       setConvertTreeReader(timestampTreeReader);
     }
@@ -1293,6 +1295,7 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
           timestampColVector.asScratchTimestamp(elementNum));
       HiveDecimal value = HiveDecimal.create(Double.toString(doubleValue));
       if (value != null) {
+        // The DecimalColumnVector will enforce precision and scale and set the entry to null when out of bounds.
         decimalColVector.set(elementNum, value);
       } else {
         decimalColVector.noNulls = false;
@@ -1316,11 +1319,61 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
     }
   }
 
+  public static class DecimalFromDecimalTreeReader extends ConvertTreeReader {
+
+    private DecimalTreeReader decimalTreeReader;
+
+    private DecimalColumnVector fileDecimalColVector;
+    private int filePrecision;
+    private int fileScale;
+    private int readerPrecision;
+    private int readerScale;
+    private DecimalColumnVector decimalColVector;
+
+    DecimalFromDecimalTreeReader(int columnId, TypeDescription fileType, TypeDescription readerType)
+        throws IOException {
+      super(columnId);
+      filePrecision = fileType.getPrecision();
+      fileScale = fileType.getScale();
+      readerPrecision = readerType.getPrecision();
+      readerScale = readerType.getScale();
+      decimalTreeReader = new DecimalTreeReader(columnId, filePrecision, fileScale);
+      setConvertTreeReader(decimalTreeReader);
+    }
+
+    @Override
+    public void setConvertVectorElement(int elementNum) throws IOException {
+
+      HiveDecimalWritable valueWritable = HiveDecimalWritable.enforcePrecisionScale(
+          fileDecimalColVector.vector[elementNum], readerPrecision, readerScale);
+      if (valueWritable != null) {
+        decimalColVector.set(elementNum, valueWritable);
+      } else {
+        decimalColVector.noNulls = false;
+        decimalColVector.isNull[elementNum] = true;
+      }
+    }
+
+    @Override
+    public void nextVector(ColumnVector previousVector,
+                           boolean[] isNull,
+                           final int batchSize) throws IOException {
+      if (fileDecimalColVector == null) {
+        // Allocate column vector for file; cast column vector for reader.
+        fileDecimalColVector = new DecimalColumnVector(filePrecision, fileScale);
+        decimalColVector = (DecimalColumnVector) previousVector;
+      }
+      // Read present/isNull stream
+      decimalTreeReader.nextVector(fileDecimalColVector, isNull, batchSize);
+
+      convertVector(fileDecimalColVector, 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;
@@ -1328,7 +1381,6 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
     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);
@@ -1364,7 +1416,6 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
     private FloatTreeReader floatTreeReader;
 
     private final TypeDescription readerType;
-    private FloatWritable floatResult;
     private DoubleColumnVector doubleColVector;
     private BytesColumnVector bytesColVector;
 
@@ -1375,7 +1426,6 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
       this.readerType = readerType;
       floatTreeReader = new FloatTreeReader(columnId);
       setConvertTreeReader(floatTreeReader);
-      floatResult = new FloatWritable();
     }
 
     @Override
@@ -1544,7 +1594,6 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
     private final TypeDescription readerType;
     private LongColumnVector longColVector;
     private BytesColumnVector bytesColVector;
-    private DateWritable dateWritableResult;
     private Date date;
 
     StringGroupFromDateTreeReader(int columnId, TypeDescription readerType,
@@ -1553,7 +1602,6 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
       this.readerType = readerType;
       dateTreeReader = new DateTreeReader(columnId);
       setConvertTreeReader(dateTreeReader);
-      dateWritableResult = new DateWritable();
       date = new Date(0);
     }
 
@@ -1585,13 +1633,11 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
 
     private TreeReader stringGroupTreeReader;
 
-    private final TypeDescription fileType;
     private final TypeDescription readerType;
 
     StringGroupFromStringGroupTreeReader(int columnId, TypeDescription fileType,
         TypeDescription readerType) throws IOException {
       super(columnId);
-      this.fileType = fileType;
       this.readerType = readerType;
       stringGroupTreeReader = getStringGroupTreeReader(columnId, fileType);
       setConvertTreeReader(stringGroupTreeReader);
@@ -1609,8 +1655,7 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
         if (resultColVector.noNulls || !resultColVector.isNull[0]) {
           convertStringGroupVectorElement(resultColVector, 0, readerType);
         } else {
-          resultColVector.noNulls = false;
-          resultColVector.isNull[0] = true;
+          // Remains null.
         }
       } else if (resultColVector.noNulls){
         for (int i = 0; i < batchSize; i++) {
@@ -1621,8 +1666,7 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
           if (!resultColVector.isNull[i]) {
             convertStringGroupVectorElement(resultColVector, i, readerType);
           } else {
-            resultColVector.noNulls = false;
-            resultColVector.isNull[i] = true;
+            // Remains null.
           }
         }
       }
@@ -1634,7 +1678,6 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
     private BinaryTreeReader binaryTreeReader;
 
     private final TypeDescription readerType;
-    private BytesWritable binaryWritableResult;
     private BytesColumnVector inBytesColVector;
     private BytesColumnVector outBytesColVector;
 
@@ -1644,7 +1687,6 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
       this.readerType = readerType;
       binaryTreeReader = new BinaryTreeReader(columnId);
       setConvertTreeReader(binaryTreeReader);
-      binaryWritableResult = new BytesWritable();
     }
 
     @Override
@@ -1725,7 +1767,6 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
 
     private FloatTreeReader floatTreeReader;
 
-    private FloatWritable floatResult;
     private DoubleColumnVector doubleColVector;
     private TimestampColumnVector timestampColVector;
 
@@ -1734,14 +1775,14 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
       super(columnId);
       floatTreeReader = new FloatTreeReader(columnId);
       setConvertTreeReader(floatTreeReader);
-      floatResult = new FloatWritable();
     }
 
     @Override
     public void setConvertVectorElement(int elementNum) {
       float floatValue = (float) doubleColVector.vector[elementNum];
-      timestampColVector.set(elementNum,
-          TimestampUtils.doubleToTimestamp(floatValue));
+      Timestamp timestampValue = TimestampUtils.doubleToTimestamp(floatValue);
+      // The TimestampColumnVector will set the entry to null when a null timestamp is passed in.
+      timestampColVector.set(elementNum, timestampValue);
     }
 
     @Override
@@ -1777,8 +1818,9 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
     @Override
     public void setConvertVectorElement(int elementNum) {
       double doubleValue = doubleColVector.vector[elementNum];
-      timestampColVector.set(elementNum,
-          TimestampUtils.doubleToTimestamp(doubleValue));
+      Timestamp timestampValue = TimestampUtils.doubleToTimestamp(doubleValue);
+      // The TimestampColumnVector will set the entry to null when a null timestamp is passed in.
+      timestampColVector.set(elementNum, timestampValue);
     }
 
     @Override
@@ -1803,7 +1845,6 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
 
     private final int precision;
     private final int scale;
-    private HiveDecimalWritable hiveDecimalResult;
     private DecimalColumnVector decimalColVector;
     private TimestampColumnVector timestampColVector;
 
@@ -1814,14 +1855,14 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
       this.scale = fileType.getScale();
       decimalTreeReader = new DecimalTreeReader(columnId, precision, scale);
       setConvertTreeReader(decimalTreeReader);
-      hiveDecimalResult = new HiveDecimalWritable();
     }
 
     @Override
     public void setConvertVectorElement(int elementNum) {
       Timestamp timestampValue =
-          TimestampUtils.decimalToTimestamp(
-              decimalColVector.vector[elementNum].getHiveDecimal());
+            TimestampUtils.decimalToTimestamp(
+                decimalColVector.vector[elementNum].getHiveDecimal());
+      // The TimestampColumnVector will set the entry to null when a null timestamp is passed in.
       timestampColVector.set(elementNum, timestampValue);
     }
 
@@ -1845,14 +1886,12 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
 
     private TreeReader stringGroupTreeReader;
 
-    private final TypeDescription fileType;
     private BytesColumnVector bytesColVector;
     private TimestampColumnVector timestampColVector;
 
     TimestampFromStringGroupTreeReader(int columnId, TypeDescription fileType)
         throws IOException {
       super(columnId);
-      this.fileType = fileType;
       stringGroupTreeReader = getStringGroupTreeReader(columnId, fileType);
       setConvertTreeReader(stringGroupTreeReader);
     }
@@ -1890,7 +1929,6 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
 
     private DateTreeReader dateTreeReader;
 
-    private DateWritable doubleResult;
     private LongColumnVector longColVector;
     private TimestampColumnVector timestampColVector;
 
@@ -1899,7 +1937,6 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
       super(columnId);
       dateTreeReader = new DateTreeReader(columnId);
       setConvertTreeReader(dateTreeReader);
-      doubleResult = new DateWritable();
     }
 
     @Override
@@ -1929,14 +1966,12 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
 
     private TreeReader stringGroupTreeReader;
 
-    private final TypeDescription fileType;
     private BytesColumnVector bytesColVector;
     private LongColumnVector longColVector;
 
     DateFromStringGroupTreeReader(int columnId, TypeDescription fileType)
         throws IOException {
       super(columnId);
-      this.fileType = fileType;
       stringGroupTreeReader = getStringGroupTreeReader(columnId, fileType);
       setConvertTreeReader(stringGroupTreeReader);
     }
@@ -1974,14 +2009,11 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
 
     private TimestampTreeReader timestampTreeReader;
 
-    private final TypeDescription readerType;
     private TimestampColumnVector timestampColVector;
     private LongColumnVector longColVector;
 
-    DateFromTimestampTreeReader(int columnId, TypeDescription readerType,
-        boolean skipCorrupt) throws IOException {
+    DateFromTimestampTreeReader(int columnId, boolean skipCorrupt) throws IOException {
       super(columnId);
-      this.readerType = readerType;
       timestampTreeReader = new TimestampTreeReader(columnId, skipCorrupt);
       setConvertTreeReader(timestampTreeReader);
     }
@@ -2014,12 +2046,9 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
 
     private TreeReader stringGroupTreeReader;
 
-    private final TypeDescription fileType;
-
     BinaryFromStringGroupTreeReader(int columnId, TypeDescription fileType)
         throws IOException {
       super(columnId);
-      this.fileType = fileType;
       stringGroupTreeReader = getStringGroupTreeReader(columnId, fileType);
       setConvertTreeReader(stringGroupTreeReader);
     }
@@ -2064,7 +2093,7 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
           skipCorrupt);
 
     case DECIMAL:
-      return new DecimalFromAnyIntegerTreeReader(columnId, fileType, readerType, skipCorrupt);
+      return new DecimalFromAnyIntegerTreeReader(columnId, fileType, skipCorrupt);
 
     case STRING:
     case CHAR:
@@ -2208,7 +2237,7 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
       return new FloatFromDecimalTreeReader(columnId, fileType, readerType);
 
     case DOUBLE:
-      return new DoubleFromDecimalTreeReader(columnId, fileType, readerType);
+      return new DoubleFromDecimalTreeReader(columnId, fileType);
 
     case STRING:
     case CHAR:
@@ -2424,13 +2453,13 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
       return new AnyIntegerFromTimestampTreeReader(columnId, readerType, skipCorrupt);
 
     case FLOAT:
-      return new FloatFromTimestampTreeReader(columnId, readerType, skipCorrupt);
+      return new FloatFromTimestampTreeReader(columnId, skipCorrupt);
 
     case DOUBLE:
-      return new DoubleFromTimestampTreeReader(columnId, readerType, skipCorrupt);
+      return new DoubleFromTimestampTreeReader(columnId, skipCorrupt);
 
     case DECIMAL:
-      return new DecimalFromTimestampTreeReader(columnId, readerType, skipCorrupt);
+      return new DecimalFromTimestampTreeReader(columnId, skipCorrupt);
 
     case STRING:
     case CHAR:
@@ -2442,7 +2471,7 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
           readerType.getCategory() + " to self needed");
 
     case DATE:
-      return new DateFromTimestampTreeReader(columnId, readerType, skipCorrupt);
+      return new DateFromTimestampTreeReader(columnId, skipCorrupt);
 
     // Not currently supported conversion(s):
     case BINARY:

http://git-wip-us.apache.org/repos/asf/orc/blob/63829ed0/java/storage-api/src/java/org/apache/hadoop/hive/ql/util/TimestampUtils.java
----------------------------------------------------------------------
diff --git a/java/storage-api/src/java/org/apache/hadoop/hive/ql/util/TimestampUtils.java b/java/storage-api/src/java/org/apache/hadoop/hive/ql/util/TimestampUtils.java
index 189ead5..41db9ca 100644
--- a/java/storage-api/src/java/org/apache/hadoop/hive/ql/util/TimestampUtils.java
+++ b/java/storage-api/src/java/org/apache/hadoop/hive/ql/util/TimestampUtils.java
@@ -39,45 +39,53 @@ public class TimestampUtils {
   }
 
   public static Timestamp doubleToTimestamp(double f) {
-    long seconds = (long) f;
-
-    // We must ensure the exactness of the double's fractional portion.
-    // 0.6 as the fraction part will be converted to 0.59999... and
-    // significantly reduce the savings from binary serialization
-    BigDecimal bd;
     try {
-      bd = new BigDecimal(String.valueOf(f));
+      long seconds = (long) f;
+
+      // We must ensure the exactness of the double's fractional portion.
+      // 0.6 as the fraction part will be converted to 0.59999... and
+      // significantly reduce the savings from binary serialization
+      BigDecimal bd = new BigDecimal(String.valueOf(f));
+
+      bd = bd.subtract(new BigDecimal(seconds)).multiply(new BigDecimal(1000000000));
+      int nanos = bd.intValue();
+
+      // Convert to millis
+      long millis = seconds * 1000;
+      if (nanos < 0) {
+        millis -= 1000;
+        nanos += 1000000000;
+      }
+      Timestamp t = new Timestamp(millis);
+
+      // Set remaining fractional portion to nanos
+      t.setNanos(nanos);
+      return t;
     } catch (NumberFormatException nfe) {
       return null;
+    } catch (IllegalArgumentException iae) {
+      return null;
     }
-    bd = bd.subtract(new BigDecimal(seconds)).multiply(new BigDecimal(1000000000));
-    int nanos = bd.intValue();
-
-    // Convert to millis
-    long millis = seconds * 1000;
-    if (nanos < 0) {
-      millis -= 1000;
-      nanos += 1000000000;
-    }
-    Timestamp t = new Timestamp(millis);
-
-    // Set remaining fractional portion to nanos
-    t.setNanos(nanos);
-    return t;
   }
 
   public static Timestamp decimalToTimestamp(HiveDecimal d) {
-    BigDecimal nanoInstant = d.bigDecimalValue().multiply(BILLION_BIG_DECIMAL);
-    int nanos = nanoInstant.remainder(BILLION_BIG_DECIMAL).intValue();
-    if (nanos < 0) {
-      nanos += 1000000000;
-    }
-    long seconds =
-        nanoInstant.subtract(new BigDecimal(nanos)).divide(BILLION_BIG_DECIMAL).longValue();
-    Timestamp t = new Timestamp(seconds * 1000);
-    t.setNanos(nanos);
+    try {
+      BigDecimal nanoInstant = d.bigDecimalValue().multiply(BILLION_BIG_DECIMAL);
+      int nanos = nanoInstant.remainder(BILLION_BIG_DECIMAL).intValue();
+      if (nanos < 0) {
+        nanos += 1000000000;
+      }
+      long seconds =
+          nanoInstant.subtract(new BigDecimal(nanos)).divide(BILLION_BIG_DECIMAL).longValue();
+      Timestamp t = new Timestamp(seconds * 1000);
+      t.setNanos(nanos);
 
-    return t;
+      return t;
+    } catch (NumberFormatException nfe) {
+      return null;
+    } catch (IllegalArgumentException iae) {
+      return null;
+    }
   }
 
   /**