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/04/04 22:37:04 UTC

[29/50] [abbrv] hive git commit: HIVE-13111: Fix timestamp / interval_day_time wrong results with HIVE-9862 (Matt McCline, reviewed by Jason Dere)

http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/TimestampColumnInList.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/TimestampColumnInList.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/TimestampColumnInList.java
index 2d7d0c2..bc09a3a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/TimestampColumnInList.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/TimestampColumnInList.java
@@ -21,7 +21,6 @@ package org.apache.hadoop.hive.ql.exec.vector.expressions;
 import java.sql.Timestamp;
 import java.util.HashSet;
 
-import org.apache.hadoop.hive.common.type.PisaTimestamp;
 import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor.Descriptor;
 import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
@@ -36,11 +35,8 @@ public class TimestampColumnInList extends VectorExpression implements ITimestam
   private Timestamp[] inListValues;
   private int outputColumn;
 
-  private transient PisaTimestamp scratchTimestamp;
-
-
   // The set object containing the IN list.
-  private transient HashSet<PisaTimestamp> inSet;
+  private transient HashSet<Timestamp> inSet;
 
   public TimestampColumnInList() {
     super();
@@ -64,11 +60,10 @@ public class TimestampColumnInList extends VectorExpression implements ITimestam
     }
 
     if (inSet == null) {
-      inSet = new HashSet<PisaTimestamp>(inListValues.length);
+      inSet = new HashSet<Timestamp>(inListValues.length);
       for (Timestamp val : inListValues) {
-        inSet.add(new PisaTimestamp(val));
+        inSet.add(val);
       }
-      scratchTimestamp = new PisaTimestamp();
     }
 
     TimestampColumnVector inputColVector = (TimestampColumnVector) batch.cols[inputCol];
@@ -91,19 +86,16 @@ public class TimestampColumnInList extends VectorExpression implements ITimestam
 
         // All must be selected otherwise size would be zero
         // Repeating property will not change.
-        inputColVector.pisaTimestampUpdate(scratchTimestamp, 0);
-        outputVector[0] = inSet.contains(scratchTimestamp) ? 1 : 0;
+        outputVector[0] = inSet.contains(inputColVector.asScratchTimestamp(0)) ? 1 : 0;
         outputColVector.isRepeating = true;
       } else if (batch.selectedInUse) {
         for(int j = 0; j != n; j++) {
           int i = sel[j];
-          inputColVector.pisaTimestampUpdate(scratchTimestamp, i);
-          outputVector[i] = inSet.contains(scratchTimestamp) ? 1 : 0;
+          outputVector[i] = inSet.contains(inputColVector.asScratchTimestamp(i)) ? 1 : 0;
         }
       } else {
         for(int i = 0; i != n; i++) {
-          inputColVector.pisaTimestampUpdate(scratchTimestamp, i);
-          outputVector[i] = inSet.contains(scratchTimestamp) ? 1 : 0;
+          outputVector[i] = inSet.contains(inputColVector.asScratchTimestamp(i)) ? 1 : 0;
         }
       }
     } else {
@@ -112,8 +104,7 @@ public class TimestampColumnInList extends VectorExpression implements ITimestam
         //All must be selected otherwise size would be zero
         //Repeating property will not change.
         if (!nullPos[0]) {
-          inputColVector.pisaTimestampUpdate(scratchTimestamp, 0);
-          outputVector[0] = inSet.contains(scratchTimestamp) ? 1 : 0;
+          outputVector[0] = inSet.contains(inputColVector.asScratchTimestamp(0)) ? 1 : 0;
           outNulls[0] = false;
         } else {
           outNulls[0] = true;
@@ -124,16 +115,14 @@ public class TimestampColumnInList extends VectorExpression implements ITimestam
           int i = sel[j];
           outNulls[i] = nullPos[i];
           if (!nullPos[i]) {
-            inputColVector.pisaTimestampUpdate(scratchTimestamp, i);
-            outputVector[i] = inSet.contains(scratchTimestamp) ? 1 : 0;
+            outputVector[i] = inSet.contains(inputColVector.asScratchTimestamp(i)) ? 1 : 0;
           }
         }
       } else {
         System.arraycopy(nullPos, 0, outNulls, 0, n);
         for(int i = 0; i != n; i++) {
           if (!nullPos[i]) {
-            inputColVector.pisaTimestampUpdate(scratchTimestamp, i);
-            outputVector[i] = inSet.contains(scratchTimestamp) ? 1 : 0;
+            outputVector[i] = inSet.contains(inputColVector.asScratchTimestamp(i)) ? 1 : 0;
           }
         }
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorExpressionWriter.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorExpressionWriter.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorExpressionWriter.java
index 326bfb9..85dacd7 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorExpressionWriter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorExpressionWriter.java
@@ -21,10 +21,11 @@ package org.apache.hadoop.hive.ql.exec.vector.expressions;
 import java.sql.Timestamp;
 
 import org.apache.hadoop.hive.common.type.HiveDecimal;
-import org.apache.hadoop.hive.common.type.PisaTimestamp;
+import org.apache.hadoop.hive.common.type.HiveIntervalDayTime;
 import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.hadoop.hive.serde2.io.HiveIntervalDayTimeWritable;
 import org.apache.hadoop.hive.serde2.io.TimestampWritable;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 
@@ -42,7 +43,8 @@ public interface VectorExpressionWriter {
   Object writeValue(HiveDecimal value) throws HiveException;
   Object writeValue(TimestampWritable value) throws HiveException;
   Object writeValue(Timestamp value) throws HiveException;
-  Object writeValue(PisaTimestamp value) throws HiveException;
+  Object writeValue(HiveIntervalDayTimeWritable value) throws HiveException;
+  Object writeValue(HiveIntervalDayTime value) throws HiveException;
   Object setValue(Object row, ColumnVector column, int columnRow) throws HiveException;
   Object initValue(Object ost) throws HiveException;
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorExpressionWriterFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorExpressionWriterFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorExpressionWriterFactory.java
index 9a1d7f3..c20bc68 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorExpressionWriterFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorExpressionWriterFactory.java
@@ -31,7 +31,6 @@ import org.apache.hadoop.hive.common.type.HiveDecimal;
 import org.apache.hadoop.hive.common.type.HiveIntervalDayTime;
 import org.apache.hadoop.hive.common.type.HiveIntervalYearMonth;
 import org.apache.hadoop.hive.common.type.HiveVarchar;
-import org.apache.hadoop.hive.common.type.PisaTimestamp;
 import org.apache.hadoop.hive.ql.exec.vector.*;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
@@ -188,17 +187,39 @@ public final class VectorExpressionWriterFactory {
     }
 
     /**
-     * The base implementation must be overridden by the PisaTimestamp specialization
+     * The base implementation must be overridden by the Timestamp specialization
+     */
+    public Object setValue(Object field, Timestamp value) throws HiveException {
+      throw new HiveException("Internal error: should not reach here");
+    }
+
+    /**
+     * The base implementation must be overridden by the HiveIntervalDayTime specialization
      */
     @Override
-    public Object writeValue(PisaTimestamp value) throws HiveException {
+    public Object writeValue(HiveIntervalDayTimeWritable value) throws HiveException {
       throw new HiveException("Internal error: should not reach here");
     }
 
     /**
-     * The base implementation must be overridden by the Timestamp specialization
+     * The base implementation must be overridden by the HiveIntervalDayTime specialization
      */
-    public Object setValue(Object field, Timestamp value) throws HiveException {
+    @Override
+    public Object writeValue(HiveIntervalDayTime value) throws HiveException {
+      throw new HiveException("Internal error: should not reach here");
+    }
+
+    /**
+     * The base implementation must be overridden by the HiveIntervalDayTime specialization
+     */
+    public Object setValue(Object field, HiveIntervalDayTimeWritable value) throws HiveException {
+      throw new HiveException("Internal error: should not reach here");
+    }
+
+    /**
+     * The base implementation must be overridden by the HiveIntervalDayTime specialization
+     */
+    public Object setValue(Object field, HiveIntervalDayTime value) throws HiveException {
       throw new HiveException("Internal error: should not reach here");
     }
   }
@@ -465,6 +486,66 @@ public final class VectorExpressionWriterFactory {
     }
   }
 
+  /**
+   * Specialized writer for IntervalDayTimeColumnVector. Will throw cast exception
+   * if the wrong vector column is used.
+   */
+  private static abstract class VectorExpressionWriterIntervalDayTime extends VectorExpressionWriterBase {
+    @Override
+    public Object writeValue(ColumnVector column, int row) throws HiveException {
+      IntervalDayTimeColumnVector dcv = (IntervalDayTimeColumnVector) column;
+      HiveIntervalDayTimeWritable intervalDayTimeWritable = (HiveIntervalDayTimeWritable) dcv.getScratchWritable();
+      if (intervalDayTimeWritable == null) {
+        intervalDayTimeWritable = new HiveIntervalDayTimeWritable();
+        dcv.setScratchWritable(intervalDayTimeWritable);
+      }
+      if (dcv.noNulls && !dcv.isRepeating) {
+        return writeValue(TimestampUtils.intervalDayTimeColumnVectorWritable(dcv, row, intervalDayTimeWritable));
+      } else if (dcv.noNulls && dcv.isRepeating) {
+        return writeValue(TimestampUtils.intervalDayTimeColumnVectorWritable(dcv, 0, intervalDayTimeWritable));
+      } else if (!dcv.noNulls && !dcv.isRepeating && !dcv.isNull[row]) {
+        return writeValue(TimestampUtils.intervalDayTimeColumnVectorWritable(dcv, row, intervalDayTimeWritable));
+      } else if (!dcv.noNulls && dcv.isRepeating && !dcv.isNull[0]) {
+        return writeValue(TimestampUtils.intervalDayTimeColumnVectorWritable(dcv, 0, intervalDayTimeWritable));
+      } else if (!dcv.noNulls && dcv.isRepeating && dcv.isNull[0]) {
+        return null;
+      } else if (!dcv.noNulls && !dcv.isRepeating && dcv.isNull[row]) {
+        return null;
+      }
+      throw new HiveException(
+          String.format(
+              "Incorrect null/repeating: row:%d noNulls:%b isRepeating:%b isNull[row]:%b isNull[0]:%b",
+              row, dcv.noNulls, dcv.isRepeating, dcv.isNull[row], dcv.isNull[0]));
+    }
+
+    @Override
+    public Object setValue(Object field, ColumnVector column, int row) throws HiveException {
+      IntervalDayTimeColumnVector dcv = (IntervalDayTimeColumnVector) column;
+      HiveIntervalDayTimeWritable intervalDayTimeWritable = (HiveIntervalDayTimeWritable) dcv.getScratchWritable();
+      if (intervalDayTimeWritable == null) {
+        intervalDayTimeWritable = new HiveIntervalDayTimeWritable();
+        dcv.setScratchWritable(intervalDayTimeWritable);
+      }
+      if (dcv.noNulls && !dcv.isRepeating) {
+        return setValue(field, TimestampUtils.intervalDayTimeColumnVectorWritable(dcv, row, intervalDayTimeWritable));
+      } else if (dcv.noNulls && dcv.isRepeating) {
+        return setValue(field, TimestampUtils.intervalDayTimeColumnVectorWritable(dcv, 0, intervalDayTimeWritable));
+      } else if (!dcv.noNulls && !dcv.isRepeating && !dcv.isNull[row]) {
+        return setValue(field, TimestampUtils.intervalDayTimeColumnVectorWritable(dcv, row, intervalDayTimeWritable));
+      } else if (!dcv.noNulls && !dcv.isRepeating && dcv.isNull[row]) {
+        return null;
+      } else if (!dcv.noNulls && dcv.isRepeating && !dcv.isNull[0]) {
+        return setValue(field, TimestampUtils.intervalDayTimeColumnVectorWritable(dcv, 0, intervalDayTimeWritable));
+      } else if (!dcv.noNulls && dcv.isRepeating && dcv.isNull[0]) {
+        return null;
+      }
+      throw new HiveException(
+          String.format(
+              "Incorrect null/repeating: row:%d noNulls:%b isRepeating:%b isNull[row]:%b isNull[0]:%b",
+              row, dcv.noNulls, dcv.isRepeating, dcv.isNull[row], dcv.isNull[0]));
+    }
+  }
+
     /**
      * Compiles the appropriate vector expression writer based on an expression info (ExprNodeDesc)
      */
@@ -697,8 +778,13 @@ public final class VectorExpressionWriterFactory {
       }
 
       @Override
-      public Object writeValue(PisaTimestamp value) throws HiveException {
-        return ((SettableTimestampObjectInspector) this.objectInspector).set(obj, value.asScratchTimestamp());
+      public Object writeValue(HiveIntervalDayTimeWritable value) throws HiveException {
+        return ((SettableHiveIntervalDayTimeObjectInspector) this.objectInspector).set(obj, value);
+      }
+
+      @Override
+      public Object writeValue(HiveIntervalDayTime value) throws HiveException {
+        return ((SettableHiveIntervalDayTimeObjectInspector) this.objectInspector).set(obj, value);
       }
 
       @Override
@@ -766,53 +852,45 @@ public final class VectorExpressionWriterFactory {
   private static VectorExpressionWriter genVectorExpressionWritableIntervalDayTime(
       SettableHiveIntervalDayTimeObjectInspector fieldObjInspector) throws HiveException {
 
-    return new VectorExpressionWriterTimestamp() {
+    return new VectorExpressionWriterIntervalDayTime() {
       private Object obj;
       private HiveIntervalDayTime interval;
-      private PisaTimestamp pisaTimestamp;
 
       public VectorExpressionWriter init(SettableHiveIntervalDayTimeObjectInspector objInspector)
           throws HiveException {
         super.init(objInspector);
         interval = new HiveIntervalDayTime();
         obj = initValue(null);
-        pisaTimestamp = new PisaTimestamp();
         return this;
       }
 
       @Override
-      public Object writeValue(TimestampWritable value) throws HiveException {
-        interval.set(pisaTimestamp.updateFromTimestamp(value.getTimestamp()));
+      public Object writeValue(HiveIntervalDayTimeWritable value) throws HiveException {
+        interval.set(value.getHiveIntervalDayTime());
         return ((SettableHiveIntervalDayTimeObjectInspector) this.objectInspector).set(obj, interval);
       }
 
       @Override
-      public Object writeValue(Timestamp value) throws HiveException {
-        interval.set(pisaTimestamp.updateFromTimestamp(value));
-        return ((SettableHiveIntervalDayTimeObjectInspector) this.objectInspector).set(obj, interval);
-      }
-
-      @Override
-      public Object writeValue(PisaTimestamp value) throws HiveException {
+      public Object writeValue(HiveIntervalDayTime value) throws HiveException {
         interval.set(value);
         return ((SettableHiveIntervalDayTimeObjectInspector) this.objectInspector).set(obj, interval);
       }
 
       @Override
-      public Object setValue(Object field, TimestampWritable value) {
+      public Object setValue(Object field, HiveIntervalDayTimeWritable value) {
         if (null == field) {
           field = initValue(null);
         }
-        interval.set(pisaTimestamp.updateFromTimestamp(value.getTimestamp()));
+        interval.set(value.getHiveIntervalDayTime());
         return ((SettableHiveIntervalDayTimeObjectInspector) this.objectInspector).set(field, interval);
       }
 
       @Override
-      public Object setValue(Object field, Timestamp value) {
+      public Object setValue(Object field, HiveIntervalDayTime value) {
         if (null == field) {
           field = initValue(null);
         }
-        interval.set(pisaTimestamp.updateFromTimestamp(value));
+        interval.set(value);
         return ((SettableHiveIntervalDayTimeObjectInspector) this.objectInspector).set(field, interval);
       }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateAddColCol.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateAddColCol.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateAddColCol.java
index 9f5c793..05dd93e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateAddColCol.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateAddColCol.java
@@ -166,7 +166,7 @@ public class VectorUDFDateAddColCol extends VectorExpression {
 
   protected byte[] evaluateTimestamp(ColumnVector columnVector, int index, long numDays) {
     TimestampColumnVector tcv = (TimestampColumnVector) columnVector;
-    calendar.setTimeInMillis(tcv.getTimestampMilliseconds(index));
+    calendar.setTimeInMillis(tcv.getTime(index));
     if (isPositive) {
       calendar.add(Calendar.DATE, (int) numDays);
     } else {

http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateAddColScalar.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateAddColScalar.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateAddColScalar.java
index 6390ecd..59ca61e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateAddColScalar.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateAddColScalar.java
@@ -210,7 +210,7 @@ public class VectorUDFDateAddColScalar extends VectorExpression {
 
   protected byte[] evaluateTimestamp(ColumnVector columnVector, int index) {
     TimestampColumnVector tcv = (TimestampColumnVector) columnVector;
-    calendar.setTimeInMillis(tcv.getTimestampMilliseconds(index));
+    calendar.setTimeInMillis(tcv.getTime(index));
     if (isPositive) {
       calendar.add(Calendar.DATE, numDays);
     } else {

http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateDiffColCol.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateDiffColCol.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateDiffColCol.java
index b22c31f..4edf558 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateDiffColCol.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateDiffColCol.java
@@ -275,7 +275,7 @@ public class VectorUDFDateDiffColCol extends VectorExpression {
       output.isRepeating = true;
 
       if (!input.isNull[0]) {
-        date.setTime(input.getTimestampMilliseconds(0));
+        date.setTime(input.getTime(0));
         output.vector[0] = DateWritable.dateToDays(date);
       }
       return;
@@ -288,12 +288,12 @@ public class VectorUDFDateDiffColCol extends VectorExpression {
       if (selectedInUse) {
         for (int j = 0; j < size; j++) {
           int i = sel[j];
-          date.setTime(input.getTimestampMilliseconds(i));
+          date.setTime(input.getTime(i));
           output.vector[i] = DateWritable.dateToDays(date);
         }
       } else {
         for (int i = 0; i < size; i++) {
-          date.setTime(input.getTimestampMilliseconds(i));
+          date.setTime(input.getTime(i));
           output.vector[i] = DateWritable.dateToDays(date);
         }
       }
@@ -312,14 +312,14 @@ public class VectorUDFDateDiffColCol extends VectorExpression {
         for (int j = 0; j < size; j++) {
           int i = sel[j];
           if (!input.isNull[i]) {
-            date.setTime(input.getTimestampMilliseconds(i));
+            date.setTime(input.getTime(i));
             output.vector[i] = DateWritable.dateToDays(date);
           }
         }
       } else {
         for (int i = 0; i < size; i++) {
           if (!input.isNull[i]) {
-            date.setTime(input.getTimestampMilliseconds(i));
+            date.setTime(input.getTime(i));
             output.vector[i] = DateWritable.dateToDays(date);
           }
         }

http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateDiffColScalar.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateDiffColScalar.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateDiffColScalar.java
index ab71b47..71b3887 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateDiffColScalar.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateDiffColScalar.java
@@ -238,7 +238,7 @@ public class VectorUDFDateDiffColScalar extends VectorExpression {
 
   protected int evaluateTimestamp(ColumnVector columnVector, int index) {
     TimestampColumnVector tcv = (TimestampColumnVector) columnVector;
-    date.setTime(tcv.getTimestampMilliseconds(index));
+    date.setTime(tcv.getTime(index));
     return DateWritable.dateToDays(date) - baseDate;
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateDiffScalarCol.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateDiffScalarCol.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateDiffScalarCol.java
index dea5444..c733bc9 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateDiffScalarCol.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateDiffScalarCol.java
@@ -237,7 +237,7 @@ public class VectorUDFDateDiffScalarCol extends VectorExpression {
 
   protected int evaluateTimestamp(ColumnVector columnVector, int index) {
     TimestampColumnVector tcv = (TimestampColumnVector) columnVector;
-    date.setTime(tcv.getTimestampMilliseconds(index));
+    date.setTime(tcv.getTime(index));
     return baseDate - DateWritable.dateToDays(date);
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateTimestamp.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateTimestamp.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateTimestamp.java
index c29e22e..cde0be4 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateTimestamp.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateTimestamp.java
@@ -45,7 +45,7 @@ public class VectorUDFDateTimestamp extends TimestampToStringUnaryUDF {
   protected void func(BytesColumnVector outV, TimestampColumnVector inV, int i) {
     switch (inputTypes[0]) {
       case TIMESTAMP:
-        date.setTime(inV.getTimestampMilliseconds(i));
+        date.setTime(inV.getTime(i));
         break;
 
       default:

http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFUnixTimeStampDate.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFUnixTimeStampDate.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFUnixTimeStampDate.java
index b7c4ff4..3c693af 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFUnixTimeStampDate.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFUnixTimeStampDate.java
@@ -28,15 +28,18 @@ public final class VectorUDFUnixTimeStampDate extends VectorUDFTimestampFieldDat
 
   private static final long serialVersionUID = 1L;
 
+  private DateWritable dateWritable;
+
   @Override
   protected long getDateField(long days) {
-    long ms = DateWritable.daysToMillis((int) days);
-    return ms / 1000;
+    dateWritable.set((int) days);
+    return dateWritable.getTimeInSeconds();
   }
 
   public VectorUDFUnixTimeStampDate(int colNum, int outputColumn) {
     /* not a real field */
     super(-1, colNum, outputColumn);
+    dateWritable = new DateWritable();
   }
 
   public VectorUDFUnixTimeStampDate() {

http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFUnixTimeStampTimestamp.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFUnixTimeStampTimestamp.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFUnixTimeStampTimestamp.java
index e4a31ca..2bd7756 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFUnixTimeStampTimestamp.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFUnixTimeStampTimestamp.java
@@ -18,10 +18,7 @@
 
 package org.apache.hadoop.hive.ql.exec.vector.expressions;
 
-import java.sql.Timestamp;
-
 import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
-import org.apache.hadoop.hive.serde2.io.DateWritable;
 
 /**
  * Return Unix Timestamp.
@@ -33,7 +30,7 @@ public final class VectorUDFUnixTimeStampTimestamp extends VectorUDFTimestampFie
 
   @Override
   protected long getTimestampField(TimestampColumnVector timestampColVector, int elementNum) {
-    return timestampColVector.getTimestampSeconds(elementNum);
+    return timestampColVector.asScratchTimestamp(elementNum).getTime() / 1000;
   }
 
   public VectorUDFUnixTimeStampTimestamp(int colNum, int outputColumn) {

http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFAvgTimestamp.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFAvgTimestamp.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFAvgTimestamp.java
index 5c8db41..d0a1d0d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFAvgTimestamp.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFAvgTimestamp.java
@@ -27,8 +27,6 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.aggregates.VectorAggreg
 import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorAggregationBufferRow;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
-import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.AggregationDesc;
 import org.apache.hadoop.hive.ql.util.JavaDataModel;
@@ -146,7 +144,8 @@ public class VectorUDAFAvgTimestamp extends VectorAggregateExpression {
         if (inputColVector.isRepeating) {
           iterateNoNullsRepeatingWithAggregationSelection(
             aggregationBufferSets, bufferIndex,
-            inputColVector.getTimestampSecondsWithFractionalNanos(0), batchSize);
+            inputColVector.getDouble(0),
+            batchSize);
         } else {
           if (batch.selectedInUse) {
             iterateNoNullsSelectionWithAggregationSelection(
@@ -163,11 +162,11 @@ public class VectorUDAFAvgTimestamp extends VectorAggregateExpression {
           if (batch.selectedInUse) {
             iterateHasNullsRepeatingSelectionWithAggregationSelection(
               aggregationBufferSets, bufferIndex,
-              inputColVector.getTimestampSecondsWithFractionalNanos(0), batchSize, batch.selected, inputColVector.isNull);
+              inputColVector.getDouble(0), batchSize, batch.selected, inputColVector.isNull);
           } else {
             iterateHasNullsRepeatingWithAggregationSelection(
               aggregationBufferSets, bufferIndex,
-              inputColVector.getTimestampSecondsWithFractionalNanos(0), batchSize, inputColVector.isNull);
+              inputColVector.getDouble(0), batchSize, inputColVector.isNull);
           }
         } else {
           if (batch.selectedInUse) {
@@ -210,7 +209,8 @@ public class VectorUDAFAvgTimestamp extends VectorAggregateExpression {
           aggregationBufferSets,
           bufferIndex,
           i);
-        myagg.sumValue(inputColVector.getTimestampSecondsWithFractionalNanos(selection[i]));
+        myagg.sumValue(
+            inputColVector.getDouble(selection[i]));
       }
     }
 
@@ -224,7 +224,7 @@ public class VectorUDAFAvgTimestamp extends VectorAggregateExpression {
           aggregationBufferSets,
           bufferIndex,
           i);
-        myagg.sumValue(inputColVector.getTimestampSecondsWithFractionalNanos(i));
+        myagg.sumValue(inputColVector.getDouble(i));
       }
     }
 
@@ -281,7 +281,7 @@ public class VectorUDAFAvgTimestamp extends VectorAggregateExpression {
             aggregationBufferSets,
             bufferIndex,
             j);
-          myagg.sumValue(inputColVector.getTimestampSecondsWithFractionalNanos(i));
+          myagg.sumValue(inputColVector.getDouble(i));
         }
       }
    }
@@ -296,10 +296,10 @@ public class VectorUDAFAvgTimestamp extends VectorAggregateExpression {
       for (int i=0; i < batchSize; ++i) {
         if (!isNull[i]) {
           Aggregation myagg = getCurrentAggregationBuffer(
-            aggregationBufferSets, 
+            aggregationBufferSets,
             bufferIndex,
             i);
-          myagg.sumValue(inputColVector.getTimestampSecondsWithFractionalNanos(i));
+          myagg.sumValue(inputColVector.getDouble(i));
         }
       }
    }
@@ -328,7 +328,7 @@ public class VectorUDAFAvgTimestamp extends VectorAggregateExpression {
               myagg.sum = 0;
               myagg.count = 0;
             }
-            myagg.sum += inputColVector.getTimestampSecondsWithFractionalNanos(0)*batchSize;
+            myagg.sum += inputColVector.getDouble(0)*batchSize;
             myagg.count += batchSize;
           }
           return;
@@ -358,7 +358,7 @@ public class VectorUDAFAvgTimestamp extends VectorAggregateExpression {
       for (int j=0; j< batchSize; ++j) {
         int i = selected[j];
         if (!isNull[i]) {
-          double value = inputColVector.getTimestampSecondsWithFractionalNanos(i);
+          double value = inputColVector.getDouble(i);
           if (myagg.isNull) {
             myagg.isNull = false;
             myagg.sum = 0;
@@ -381,24 +381,24 @@ public class VectorUDAFAvgTimestamp extends VectorAggregateExpression {
         myagg.sum = 0;
         myagg.count = 0;
       }
-      
+
       for (int i=0; i< batchSize; ++i) {
-        double value = inputColVector.getTimestampSecondsWithFractionalNanos(selected[i]);
+        double value = inputColVector.getDouble(selected[i]);
         myagg.sum += value;
         myagg.count += 1;
       }
     }
 
     private void iterateNoSelectionHasNulls(
-        Aggregation myagg, 
-        TimestampColumnVector inputColVector, 
+        Aggregation myagg,
+        TimestampColumnVector inputColVector,
         int batchSize,
         boolean[] isNull) {
-      
+
       for(int i=0;i<batchSize;++i) {
         if (!isNull[i]) {
-          double value = inputColVector.getTimestampSecondsWithFractionalNanos(i);
-          if (myagg.isNull) { 
+          double value = inputColVector.getDouble(i);
+          if (myagg.isNull) {
             myagg.isNull = false;
             myagg.sum = 0;
             myagg.count = 0;
@@ -420,7 +420,7 @@ public class VectorUDAFAvgTimestamp extends VectorAggregateExpression {
       }
 
       for (int i=0;i<batchSize;++i) {
-        double value = inputColVector.getTimestampSecondsWithFractionalNanos(i);
+        double value = inputColVector.getDouble(i);
         myagg.sum += value;
         myagg.count += 1;
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFStdPopTimestamp.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFStdPopTimestamp.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFStdPopTimestamp.java
index 17906ec..fa25e6a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFStdPopTimestamp.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFStdPopTimestamp.java
@@ -152,7 +152,7 @@ public class VectorUDAFStdPopTimestamp extends VectorAggregateExpression {
       if (inputColVector.isRepeating) {
         if (inputColVector.noNulls || !inputColVector.isNull[0]) {
           iterateRepeatingNoNullsWithAggregationSelection(
-            aggregationBufferSets, aggregateIndex, inputColVector.getTimestampSecondsWithFractionalNanos(0), batchSize);
+            aggregationBufferSets, aggregateIndex, inputColVector.getDouble(0), batchSize);
         }
       }
       else if (!batch.selectedInUse && inputColVector.noNulls) {
@@ -213,7 +213,7 @@ public class VectorUDAFStdPopTimestamp extends VectorAggregateExpression {
           j);
         int i = selected[j];
         if (!isNull[i]) {
-          double value = inputColVector.getTimestampSecondsWithFractionalNanos(i);
+          double value = inputColVector.getDouble(i);
           if (myagg.isNull) {
             myagg.init ();
           }
@@ -239,7 +239,7 @@ public class VectorUDAFStdPopTimestamp extends VectorAggregateExpression {
           aggregationBufferSets,
           aggregateIndex,
           i);
-        double value = inputColVector.getTimestampSecondsWithFractionalNanos(selected[i]);
+        double value = inputColVector.getDouble(selected[i]);
         if (myagg.isNull) {
           myagg.init ();
         }
@@ -265,7 +265,7 @@ public class VectorUDAFStdPopTimestamp extends VectorAggregateExpression {
             aggregationBufferSets,
             aggregateIndex,
           i);
-          double value = inputColVector.getTimestampSecondsWithFractionalNanos(i);
+          double value = inputColVector.getDouble(i);
           if (myagg.isNull) {
             myagg.init ();
           }
@@ -293,7 +293,7 @@ public class VectorUDAFStdPopTimestamp extends VectorAggregateExpression {
         if (myagg.isNull) {
           myagg.init ();
         }
-        double value = inputColVector.getTimestampSecondsWithFractionalNanos(i);
+        double value = inputColVector.getDouble(i);
         myagg.sum += value;
         myagg.count += 1;
         if(myagg.count > 1) {
@@ -322,7 +322,7 @@ public class VectorUDAFStdPopTimestamp extends VectorAggregateExpression {
 
       if (inputColVector.isRepeating) {
         if (inputColVector.noNulls) {
-          iterateRepeatingNoNulls(myagg, inputColVector.getTimestampSecondsWithFractionalNanos(0), batchSize);
+          iterateRepeatingNoNulls(myagg, inputColVector.getDouble(0), batchSize);
         }
       }
       else if (!batch.selectedInUse && inputColVector.noNulls) {
@@ -377,7 +377,7 @@ public class VectorUDAFStdPopTimestamp extends VectorAggregateExpression {
       for (int j=0; j< batchSize; ++j) {
         int i = selected[j];
         if (!isNull[i]) {
-          double value = inputColVector.getTimestampSecondsWithFractionalNanos(i);
+          double value = inputColVector.getDouble(i);
           if (myagg.isNull) {
             myagg.init ();
           }
@@ -401,7 +401,7 @@ public class VectorUDAFStdPopTimestamp extends VectorAggregateExpression {
         myagg.init ();
       }
 
-      double value = inputColVector.getTimestampSecondsWithFractionalNanos(selected[0]);
+      double value = inputColVector.getDouble(selected[0]);
       myagg.sum += value;
       myagg.count += 1;
       if(myagg.count > 1) {
@@ -412,7 +412,7 @@ public class VectorUDAFStdPopTimestamp extends VectorAggregateExpression {
       // i=0 was pulled out to remove the count > 1 check in the loop
       //
       for (int i=1; i< batchSize; ++i) {
-        value = inputColVector.getTimestampSecondsWithFractionalNanos(selected[i]);
+        value = inputColVector.getDouble(selected[i]);
         myagg.sum += value;
         myagg.count += 1;
         double t = myagg.count*value - myagg.sum;
@@ -428,7 +428,7 @@ public class VectorUDAFStdPopTimestamp extends VectorAggregateExpression {
 
       for(int i=0;i<batchSize;++i) {
         if (!isNull[i]) {
-          double value = inputColVector.getTimestampSecondsWithFractionalNanos(i);
+          double value = inputColVector.getDouble(i);
           if (myagg.isNull) {
             myagg.init ();
           }
@@ -451,7 +451,7 @@ public class VectorUDAFStdPopTimestamp extends VectorAggregateExpression {
         myagg.init ();
       }
 
-      double value = inputColVector.getTimestampSecondsWithFractionalNanos(0);
+      double value = inputColVector.getDouble(0);
       myagg.sum += value;
       myagg.count += 1;
 
@@ -462,7 +462,7 @@ public class VectorUDAFStdPopTimestamp extends VectorAggregateExpression {
 
       // i=0 was pulled out to remove count > 1 check
       for (int i=1; i<batchSize; ++i) {
-        value = inputColVector.getTimestampSecondsWithFractionalNanos(i);
+        value = inputColVector.getDouble(i);
         myagg.sum += value;
         myagg.count += 1;
         double t = myagg.count*value - myagg.sum;

http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFStdSampTimestamp.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFStdSampTimestamp.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFStdSampTimestamp.java
index 2e41e47..b3e1fae 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFStdSampTimestamp.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFStdSampTimestamp.java
@@ -26,7 +26,6 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.aggregates.VectorAggregateExpression;
 import org.apache.hadoop.hive.ql.exec.vector.VectorAggregationBufferRow;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
-import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.AggregationDesc;
@@ -38,7 +37,7 @@ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
 
 /**
-* VectorUDAFStdSampDouble. Vectorized implementation for VARIANCE aggregates.
+* VectorUDAFStdSampTimestamp. Vectorized implementation for VARIANCE aggregates.
 */
 @Description(name = "stddev_samp",
     value = "_FUNC_(x) - Returns the sample standard deviation of a set of numbers (vectorized, double)")
@@ -153,7 +152,7 @@ public class VectorUDAFStdSampTimestamp extends VectorAggregateExpression {
       if (inputColVector.isRepeating) {
         if (inputColVector.noNulls || !inputColVector.isNull[0]) {
           iterateRepeatingNoNullsWithAggregationSelection(
-            aggregationBufferSets, aggregateIndex, inputColVector.getTimestampSecondsWithFractionalNanos(0), batchSize);
+            aggregationBufferSets, aggregateIndex, inputColVector.getDouble(0), batchSize);
         }
       }
       else if (!batch.selectedInUse && inputColVector.noNulls) {
@@ -214,7 +213,7 @@ public class VectorUDAFStdSampTimestamp extends VectorAggregateExpression {
           j);
         int i = selected[j];
         if (!isNull[i]) {
-          double value = inputColVector.getTimestampSecondsWithFractionalNanos(i);
+          double value = inputColVector.getDouble(i);
           if (myagg.isNull) {
             myagg.init ();
           }
@@ -240,7 +239,7 @@ public class VectorUDAFStdSampTimestamp extends VectorAggregateExpression {
           aggregationBufferSets,
           aggregateIndex,
           i);
-        double value = inputColVector.getTimestampSecondsWithFractionalNanos(selected[i]);
+        double value = inputColVector.getDouble(selected[i]);
         if (myagg.isNull) {
           myagg.init ();
         }
@@ -266,7 +265,7 @@ public class VectorUDAFStdSampTimestamp extends VectorAggregateExpression {
             aggregationBufferSets,
             aggregateIndex,
           i);
-          double value = inputColVector.getTimestampSecondsWithFractionalNanos(i);
+          double value = inputColVector.getDouble(i);
           if (myagg.isNull) {
             myagg.init ();
           }
@@ -294,7 +293,7 @@ public class VectorUDAFStdSampTimestamp extends VectorAggregateExpression {
         if (myagg.isNull) {
           myagg.init ();
         }
-        double value = inputColVector.getTimestampSecondsWithFractionalNanos(i);
+        double value = inputColVector.getDouble(i);
         myagg.sum += value;
         myagg.count += 1;
         if(myagg.count > 1) {
@@ -323,7 +322,7 @@ public class VectorUDAFStdSampTimestamp extends VectorAggregateExpression {
 
       if (inputColVector.isRepeating) {
         if (inputColVector.noNulls) {
-          iterateRepeatingNoNulls(myagg, inputColVector.getTimestampSecondsWithFractionalNanos(0), batchSize);
+          iterateRepeatingNoNulls(myagg, inputColVector.getDouble(0), batchSize);
         }
       }
       else if (!batch.selectedInUse && inputColVector.noNulls) {
@@ -378,7 +377,7 @@ public class VectorUDAFStdSampTimestamp extends VectorAggregateExpression {
       for (int j=0; j< batchSize; ++j) {
         int i = selected[j];
         if (!isNull[i]) {
-          double value = inputColVector.getTimestampSecondsWithFractionalNanos(i);
+          double value = inputColVector.getDouble(i);
           if (myagg.isNull) {
             myagg.init ();
           }
@@ -402,7 +401,7 @@ public class VectorUDAFStdSampTimestamp extends VectorAggregateExpression {
         myagg.init ();
       }
 
-      double value = inputColVector.getTimestampSecondsWithFractionalNanos(selected[0]);
+      double value = inputColVector.getDouble(selected[0]);
       myagg.sum += value;
       myagg.count += 1;
       if(myagg.count > 1) {
@@ -413,7 +412,7 @@ public class VectorUDAFStdSampTimestamp extends VectorAggregateExpression {
       // i=0 was pulled out to remove the count > 1 check in the loop
       //
       for (int i=1; i< batchSize; ++i) {
-        value = inputColVector.getTimestampSecondsWithFractionalNanos(selected[i]);
+        value = inputColVector.getDouble(selected[i]);
         myagg.sum += value;
         myagg.count += 1;
         double t = myagg.count*value - myagg.sum;
@@ -429,7 +428,7 @@ public class VectorUDAFStdSampTimestamp extends VectorAggregateExpression {
 
       for(int i=0;i<batchSize;++i) {
         if (!isNull[i]) {
-          double value = inputColVector.getTimestampSecondsWithFractionalNanos(i);
+          double value = inputColVector.getDouble(i);
           if (myagg.isNull) {
             myagg.init ();
           }
@@ -452,7 +451,7 @@ public class VectorUDAFStdSampTimestamp extends VectorAggregateExpression {
         myagg.init ();
       }
 
-      double value = inputColVector.getTimestampSecondsWithFractionalNanos(0);
+      double value = inputColVector.getDouble(0);
       myagg.sum += value;
       myagg.count += 1;
 
@@ -463,7 +462,7 @@ public class VectorUDAFStdSampTimestamp extends VectorAggregateExpression {
 
       // i=0 was pulled out to remove count > 1 check
       for (int i=1; i<batchSize; ++i) {
-        value = inputColVector.getTimestampSecondsWithFractionalNanos(i);
+        value = inputColVector.getDouble(i);
         myagg.sum += value;
         myagg.count += 1;
         double t = myagg.count*value - myagg.sum;

http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFVarPopTimestamp.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFVarPopTimestamp.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFVarPopTimestamp.java
index d128b7c..970ec22 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFVarPopTimestamp.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFVarPopTimestamp.java
@@ -152,7 +152,7 @@ public class VectorUDAFVarPopTimestamp extends VectorAggregateExpression {
       if (inputColVector.isRepeating) {
         if (inputColVector.noNulls || !inputColVector.isNull[0]) {
           iterateRepeatingNoNullsWithAggregationSelection(
-            aggregationBufferSets, aggregateIndex, inputColVector.getTimestampSecondsWithFractionalNanos(0), batchSize);
+            aggregationBufferSets, aggregateIndex, inputColVector.getDouble(0), batchSize);
         }
       }
       else if (!batch.selectedInUse && inputColVector.noNulls) {
@@ -213,7 +213,7 @@ public class VectorUDAFVarPopTimestamp extends VectorAggregateExpression {
           j);
         int i = selected[j];
         if (!isNull[i]) {
-          double value = inputColVector.getTimestampSecondsWithFractionalNanos(i);
+          double value = inputColVector.getDouble(i);
           if (myagg.isNull) {
             myagg.init ();
           }
@@ -239,7 +239,7 @@ public class VectorUDAFVarPopTimestamp extends VectorAggregateExpression {
           aggregationBufferSets,
           aggregateIndex,
           i);
-        double value = inputColVector.getTimestampSecondsWithFractionalNanos(selected[i]);
+        double value = inputColVector.getDouble(selected[i]);
         if (myagg.isNull) {
           myagg.init ();
         }
@@ -265,7 +265,7 @@ public class VectorUDAFVarPopTimestamp extends VectorAggregateExpression {
             aggregationBufferSets,
             aggregateIndex,
           i);
-          double value = inputColVector.getTimestampSecondsWithFractionalNanos(i);
+          double value = inputColVector.getDouble(i);
           if (myagg.isNull) {
             myagg.init ();
           }
@@ -293,7 +293,7 @@ public class VectorUDAFVarPopTimestamp extends VectorAggregateExpression {
         if (myagg.isNull) {
           myagg.init ();
         }
-        double value = inputColVector.getTimestampSecondsWithFractionalNanos(i);
+        double value = inputColVector.getDouble(i);
         myagg.sum += value;
         myagg.count += 1;
         if(myagg.count > 1) {
@@ -322,7 +322,7 @@ public class VectorUDAFVarPopTimestamp extends VectorAggregateExpression {
 
       if (inputColVector.isRepeating) {
         if (inputColVector.noNulls) {
-          iterateRepeatingNoNulls(myagg, inputColVector.getTimestampSecondsWithFractionalNanos(0), batchSize);
+          iterateRepeatingNoNulls(myagg, inputColVector.getDouble(0), batchSize);
         }
       }
       else if (!batch.selectedInUse && inputColVector.noNulls) {
@@ -377,7 +377,7 @@ public class VectorUDAFVarPopTimestamp extends VectorAggregateExpression {
       for (int j=0; j< batchSize; ++j) {
         int i = selected[j];
         if (!isNull[i]) {
-          double value = inputColVector.getTimestampSecondsWithFractionalNanos(i);
+          double value = inputColVector.getDouble(i);
           if (myagg.isNull) {
             myagg.init ();
           }
@@ -401,7 +401,7 @@ public class VectorUDAFVarPopTimestamp extends VectorAggregateExpression {
         myagg.init ();
       }
 
-      double value = inputColVector.getTimestampSecondsWithFractionalNanos(selected[0]);
+      double value = inputColVector.getDouble(selected[0]);
       myagg.sum += value;
       myagg.count += 1;
       if(myagg.count > 1) {
@@ -412,7 +412,7 @@ public class VectorUDAFVarPopTimestamp extends VectorAggregateExpression {
       // i=0 was pulled out to remove the count > 1 check in the loop
       //
       for (int i=1; i< batchSize; ++i) {
-        value = inputColVector.getTimestampSecondsWithFractionalNanos(selected[i]);
+        value = inputColVector.getDouble(selected[i]);
         myagg.sum += value;
         myagg.count += 1;
         double t = myagg.count*value - myagg.sum;
@@ -428,7 +428,7 @@ public class VectorUDAFVarPopTimestamp extends VectorAggregateExpression {
 
       for(int i=0;i<batchSize;++i) {
         if (!isNull[i]) {
-          double value = inputColVector.getTimestampSecondsWithFractionalNanos(i);
+          double value = inputColVector.getDouble(i);
           if (myagg.isNull) {
             myagg.init ();
           }
@@ -451,7 +451,7 @@ public class VectorUDAFVarPopTimestamp extends VectorAggregateExpression {
         myagg.init ();
       }
 
-      double value = inputColVector.getTimestampSecondsWithFractionalNanos(0);
+      double value = inputColVector.getDouble(0);
       myagg.sum += value;
       myagg.count += 1;
 
@@ -462,7 +462,7 @@ public class VectorUDAFVarPopTimestamp extends VectorAggregateExpression {
 
       // i=0 was pulled out to remove count > 1 check
       for (int i=1; i<batchSize; ++i) {
-        value = inputColVector.getTimestampSecondsWithFractionalNanos(i);
+        value = inputColVector.getDouble(i);
         myagg.sum += value;
         myagg.count += 1;
         double t = myagg.count*value - myagg.sum;

http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFVarSampTimestamp.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFVarSampTimestamp.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFVarSampTimestamp.java
index cf76f20..9af1a28 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFVarSampTimestamp.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFVarSampTimestamp.java
@@ -152,7 +152,7 @@ public class VectorUDAFVarSampTimestamp extends VectorAggregateExpression {
       if (inputColVector.isRepeating) {
         if (inputColVector.noNulls || !inputColVector.isNull[0]) {
           iterateRepeatingNoNullsWithAggregationSelection(
-            aggregationBufferSets, aggregateIndex, inputColVector.getTimestampSecondsWithFractionalNanos(0), batchSize);
+            aggregationBufferSets, aggregateIndex, inputColVector.getDouble(0), batchSize);
         }
       }
       else if (!batch.selectedInUse && inputColVector.noNulls) {
@@ -213,7 +213,7 @@ public class VectorUDAFVarSampTimestamp extends VectorAggregateExpression {
           j);
         int i = selected[j];
         if (!isNull[i]) {
-          double value = inputColVector.getTimestampSecondsWithFractionalNanos(i);
+          double value = inputColVector.getDouble(i);
           if (myagg.isNull) {
             myagg.init ();
           }
@@ -239,7 +239,7 @@ public class VectorUDAFVarSampTimestamp extends VectorAggregateExpression {
           aggregationBufferSets,
           aggregateIndex,
           i);
-        double value = inputColVector.getTimestampSecondsWithFractionalNanos(selected[i]);
+        double value = inputColVector.getDouble(selected[i]);
         if (myagg.isNull) {
           myagg.init ();
         }
@@ -265,7 +265,7 @@ public class VectorUDAFVarSampTimestamp extends VectorAggregateExpression {
             aggregationBufferSets,
             aggregateIndex,
           i);
-          double value = inputColVector.getTimestampSecondsWithFractionalNanos(i);
+          double value = inputColVector.getDouble(i);
           if (myagg.isNull) {
             myagg.init ();
           }
@@ -293,7 +293,7 @@ public class VectorUDAFVarSampTimestamp extends VectorAggregateExpression {
         if (myagg.isNull) {
           myagg.init ();
         }
-        double value = inputColVector.getTimestampSecondsWithFractionalNanos(i);
+        double value = inputColVector.getDouble(i);
         myagg.sum += value;
         myagg.count += 1;
         if(myagg.count > 1) {
@@ -322,7 +322,7 @@ public class VectorUDAFVarSampTimestamp extends VectorAggregateExpression {
 
       if (inputColVector.isRepeating) {
         if (inputColVector.noNulls) {
-          iterateRepeatingNoNulls(myagg, inputColVector.getTimestampSecondsWithFractionalNanos(0), batchSize);
+          iterateRepeatingNoNulls(myagg, inputColVector.getDouble(0), batchSize);
         }
       }
       else if (!batch.selectedInUse && inputColVector.noNulls) {
@@ -377,7 +377,7 @@ public class VectorUDAFVarSampTimestamp extends VectorAggregateExpression {
       for (int j=0; j< batchSize; ++j) {
         int i = selected[j];
         if (!isNull[i]) {
-          double value = inputColVector.getTimestampSecondsWithFractionalNanos(i);
+          double value = inputColVector.getDouble(i);
           if (myagg.isNull) {
             myagg.init ();
           }
@@ -401,7 +401,7 @@ public class VectorUDAFVarSampTimestamp extends VectorAggregateExpression {
         myagg.init ();
       }
 
-      double value = inputColVector.getTimestampSecondsWithFractionalNanos(selected[0]);
+      double value = inputColVector.getDouble(selected[0]);
       myagg.sum += value;
       myagg.count += 1;
       if(myagg.count > 1) {
@@ -412,7 +412,7 @@ public class VectorUDAFVarSampTimestamp extends VectorAggregateExpression {
       // i=0 was pulled out to remove the count > 1 check in the loop
       //
       for (int i=1; i< batchSize; ++i) {
-        value = inputColVector.getTimestampSecondsWithFractionalNanos(selected[i]);
+        value = inputColVector.getDouble(selected[i]);
         myagg.sum += value;
         myagg.count += 1;
         double t = myagg.count*value - myagg.sum;
@@ -428,7 +428,7 @@ public class VectorUDAFVarSampTimestamp extends VectorAggregateExpression {
 
       for(int i=0;i<batchSize;++i) {
         if (!isNull[i]) {
-          double value = inputColVector.getTimestampSecondsWithFractionalNanos(i);
+          double value = inputColVector.getDouble(i);
           if (myagg.isNull) {
             myagg.init ();
           }
@@ -451,7 +451,7 @@ public class VectorUDAFVarSampTimestamp extends VectorAggregateExpression {
         myagg.init ();
       }
 
-      double value = inputColVector.getTimestampSecondsWithFractionalNanos(0);
+      double value = inputColVector.getDouble(0);
       myagg.sum += value;
       myagg.count += 1;
 
@@ -462,7 +462,7 @@ public class VectorUDAFVarSampTimestamp extends VectorAggregateExpression {
 
       // i=0 was pulled out to remove count > 1 check
       for (int i=1; i<batchSize; ++i) {
-        value = inputColVector.getTimestampSecondsWithFractionalNanos(i);
+        value = inputColVector.getDouble(i);
         myagg.sum += value;
         myagg.count += 1;
         double t = myagg.count*value - myagg.sum;

http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/udf/VectorUDFAdaptor.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/udf/VectorUDFAdaptor.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/udf/VectorUDFAdaptor.java
index d3a0f9f..20cfb89 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/udf/VectorUDFAdaptor.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/udf/VectorUDFAdaptor.java
@@ -297,23 +297,14 @@ public class VectorUDFAdaptor extends VectorExpression {
         lv.vector[i] = ((WritableByteObjectInspector) outputOI).get(value);
       }
     } else if (outputOI instanceof WritableTimestampObjectInspector) {
-      LongColumnVector lv = (LongColumnVector) colVec;
+      TimestampColumnVector tv = (TimestampColumnVector) colVec;
       Timestamp ts;
       if (value instanceof Timestamp) {
         ts = (Timestamp) value;
       } else {
         ts = ((WritableTimestampObjectInspector) outputOI).getPrimitiveJavaObject(value);
       }
-      /* Calculate the number of nanoseconds since the epoch as a long integer. By convention
-       * that is how Timestamp values are operated on in a vector.
-       */
-      long l = ts.getTime() * 1000000  // Shift the milliseconds value over by 6 digits
-                                       // to scale for nanosecond precision.
-                                       // The milliseconds digits will by convention be all 0s.
-            + ts.getNanos() % 1000000; // Add on the remaining nanos.
-                                       // The % 1000000 operation removes the ms values
-                                       // so that the milliseconds are not counted twice.
-      lv.vector[i] = l;
+      tv.set(i, ts);
     } else if (outputOI instanceof WritableDateObjectInspector) {
       LongColumnVector lv = (LongColumnVector) colVec;
       Date ts;

http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/ql/src/java/org/apache/hadoop/hive/ql/util/DateTimeMath.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/util/DateTimeMath.java b/ql/src/java/org/apache/hadoop/hive/ql/util/DateTimeMath.java
index e092ac2..98b1ded 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/util/DateTimeMath.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/util/DateTimeMath.java
@@ -25,7 +25,6 @@ import java.util.concurrent.TimeUnit;
 
 import org.apache.hadoop.hive.common.type.HiveIntervalYearMonth;
 import org.apache.hadoop.hive.common.type.HiveIntervalDayTime;
-import org.apache.hadoop.hive.common.type.PisaTimestamp;
 import org.apache.hadoop.hive.serde2.io.DateWritable;
 import org.apache.hive.common.util.DateUtils;
 
@@ -91,26 +90,6 @@ public class DateTimeMath {
     return result;
   }
 
-  /**
-   * Perform month arithmetic to millis value using local time zone.
-   * @param pisaTimestamp
-   * @param months
-   * @return
-   */
-  public PisaTimestamp addMonthsToPisaTimestamp(PisaTimestamp pisaTimestamp, int months,
-      PisaTimestamp scratchPisaTimestamp) {
-    calLocal.setTimeInMillis(pisaTimestamp.getTimestampMilliseconds());
-    calLocal.add(Calendar.MONTH, months);
-    scratchPisaTimestamp.updateFromTimestampMilliseconds(calLocal.getTimeInMillis());
-
-    // Add in portion of nanos below a millisecond...
-    PisaTimestamp.add(
-        scratchPisaTimestamp.getEpochDay(), scratchPisaTimestamp.getNanoOfDay(),
-        0, pisaTimestamp.getNanoOfDay() % 1000000,
-        scratchPisaTimestamp);
-    return scratchPisaTimestamp;
-  }
-
   public long addMonthsToDays(long days, int months) {
     long millis = DateWritable.daysToMillis((int) days);
     millis = addMonthsToMillisLocal(millis, months);
@@ -123,24 +102,95 @@ public class DateTimeMath {
       return null;
     }
 
+    Timestamp tsResult = new Timestamp(0);
+    add(ts, interval, tsResult);
+
+    return tsResult;
+  }
+
+  public boolean add(Timestamp ts, HiveIntervalYearMonth interval, Timestamp result) {
+    if (ts == null || interval == null) {
+      return false;
+    }
+
     // Attempt to match Oracle semantics for timestamp arithmetic,
     // where timestamp arithmetic is done in UTC, then converted back to local timezone
     long resultMillis = addMonthsToMillisUtc(ts.getTime(), interval.getTotalMonths());
-    Timestamp tsResult = new Timestamp(resultMillis);
-    tsResult.setNanos(ts.getNanos());
+    result.setTime(resultMillis);
+    result.setNanos(ts.getNanos());
+
+    return true;
+  }
+
+  public Timestamp add(HiveIntervalYearMonth interval, Timestamp ts) {
+    if (ts == null || interval == null) {
+      return null;
+    }
+
+    Timestamp tsResult = new Timestamp(0);
+    add(interval, ts, tsResult);
 
     return tsResult;
   }
 
+  public boolean add(HiveIntervalYearMonth interval, Timestamp ts, Timestamp result) {
+    if (ts == null || interval == null) {
+      return false;
+    }
+
+    // Attempt to match Oracle semantics for timestamp arithmetic,
+    // where timestamp arithmetic is done in UTC, then converted back to local timezone
+    long resultMillis = addMonthsToMillisUtc(ts.getTime(), interval.getTotalMonths());
+    result.setTime(resultMillis);
+    result.setNanos(ts.getNanos());
+
+    return true;
+  }
+
   public Date add(Date dt, HiveIntervalYearMonth interval) {
     if (dt == null || interval == null) {
       return null;
     }
 
+    Date dtResult = new Date(0);
+    add(dt, interval, dtResult);
+
+    return dtResult;
+  }
+
+  public boolean add(Date dt, HiveIntervalYearMonth interval, Date result) {
+    if (dt == null || interval == null) {
+      return false;
+    }
+
+    // Since Date millis value is in local timezone representation, do date arithmetic
+    // using local timezone so the time remains at the start of the day.
+    long resultMillis = addMonthsToMillisLocal(dt.getTime(), interval.getTotalMonths());
+    result.setTime(resultMillis);
+    return true;
+  }
+
+  public Date add(HiveIntervalYearMonth interval, Date dt) {
+    if (dt == null || interval == null) {
+      return null;
+    }
+
+    Date dtResult = new Date(0);
+    add(interval, dt, dtResult);
+
+    return dtResult;
+  }
+
+  public boolean add(HiveIntervalYearMonth interval, Date dt, Date result) {
+    if (dt == null || interval == null) {
+      return false;
+    }
+
     // Since Date millis value is in local timezone representation, do date arithmetic
     // using local timezone so the time remains at the start of the day.
     long resultMillis = addMonthsToMillisLocal(dt.getTime(), interval.getTotalMonths());
-    return new Date(resultMillis);
+    result.setTime(resultMillis);
+    return true;
   }
 
   public HiveIntervalYearMonth add(HiveIntervalYearMonth left, HiveIntervalYearMonth right) {
@@ -157,14 +207,36 @@ public class DateTimeMath {
     if (left == null || right == null) {
       return null;
     }
-    return add(left, right.negate());
+
+    Timestamp tsResult = new Timestamp(0);
+    subtract(left, right, tsResult);
+
+    return tsResult;
+  }
+
+  public boolean subtract(Timestamp left, HiveIntervalYearMonth right, Timestamp result) {
+    if (left == null || right == null) {
+      return false;
+    }
+    return add(left, right.negate(), result);
   }
 
   public Date subtract(Date left, HiveIntervalYearMonth right) {
     if (left == null || right == null) {
       return null;
     }
-    return add(left, right.negate());
+
+    Date dtResult = new Date(0);
+    subtract(left, right, dtResult);
+
+    return dtResult;
+  }
+
+  public boolean subtract(Date left, HiveIntervalYearMonth right, Date result) {
+    if (left == null || right == null) {
+      return false;
+    }
+    return add(left, right.negate(), result);
   }
 
   public HiveIntervalYearMonth subtract(HiveIntervalYearMonth left, HiveIntervalYearMonth right) {
@@ -183,26 +255,74 @@ public class DateTimeMath {
       return null;
     }
 
+    Timestamp tsResult = new Timestamp(0);
+    add(ts, interval, tsResult);
+
+    return tsResult;
+  }
+
+  public boolean add(Timestamp ts, HiveIntervalDayTime interval,
+      Timestamp result) {
+    if (ts == null || interval == null) {
+      return false;
+    }
+
     nanosResult.addNanos(ts.getNanos(), interval.getNanos());
 
     long newMillis = ts.getTime()
         + TimeUnit.SECONDS.toMillis(interval.getTotalSeconds() + nanosResult.seconds);
-    Timestamp tsResult = new Timestamp(newMillis);
-    tsResult.setNanos(nanosResult.nanos);
+    result.setTime(newMillis);
+    result.setNanos(nanosResult.nanos);
+    return true;
+  }
+
+  public Timestamp add(HiveIntervalDayTime interval, Timestamp ts) {
+    if (ts == null || interval == null) {
+      return null;
+    }
+
+    Timestamp tsResult = new Timestamp(0);
+    add(interval, ts, tsResult);
     return tsResult;
   }
 
+  public boolean add(HiveIntervalDayTime interval, Timestamp ts,
+      Timestamp result) {
+    if (ts == null || interval == null) {
+      return false;
+    }
+
+    nanosResult.addNanos(ts.getNanos(), interval.getNanos());
+
+    long newMillis = ts.getTime()
+        + TimeUnit.SECONDS.toMillis(interval.getTotalSeconds() + nanosResult.seconds);
+    result.setTime(newMillis);
+    result.setNanos(nanosResult.nanos);
+    return true;
+  }
+
   public HiveIntervalDayTime add(HiveIntervalDayTime left, HiveIntervalDayTime right) {
-    HiveIntervalDayTime result = null;
     if (left == null || right == null) {
       return null;
     }
 
+    HiveIntervalDayTime result = new HiveIntervalDayTime();
+    add(left, right, result);
+ 
+    return result;
+  }
+
+  public boolean add(HiveIntervalDayTime left, HiveIntervalDayTime right,
+      HiveIntervalDayTime result) {
+    if (left == null || right == null) {
+      return false;
+    }
+
     nanosResult.addNanos(left.getNanos(), right.getNanos());
 
     long totalSeconds = left.getTotalSeconds() + right.getTotalSeconds() + nanosResult.seconds;
-    result = new HiveIntervalDayTime(totalSeconds, nanosResult.nanos);
-    return result;
+    result.set(totalSeconds, nanosResult.nanos);
+    return true;
   }
 
   public Timestamp subtract(Timestamp left, HiveIntervalDayTime right) {
@@ -212,6 +332,13 @@ public class DateTimeMath {
     return add(left, right.negate());
   }
 
+  public boolean subtract(Timestamp left, HiveIntervalDayTime right, Timestamp result) {
+    if (left == null || right == null) {
+      return false;
+    }
+    return add(left, right.negate(), result);
+  }
+
   public HiveIntervalDayTime subtract(HiveIntervalDayTime left, HiveIntervalDayTime right) {
     if (left == null || right == null) {
       return null;
@@ -219,17 +346,36 @@ public class DateTimeMath {
     return add(left, right.negate());
   }
 
+  public boolean subtract(HiveIntervalDayTime left, HiveIntervalDayTime right,
+      HiveIntervalDayTime result) {
+    if (left == null || right == null) {
+      return false;
+    }
+    return add(left, right.negate(), result);
+  }
+
   public HiveIntervalDayTime subtract(Timestamp left, Timestamp right) {
-    HiveIntervalDayTime result = null;
     if (left == null || right == null) {
       return null;
     }
 
+    HiveIntervalDayTime result = new HiveIntervalDayTime();
+    subtract(left, right, result);
+
+    return result;
+  }
+
+  public boolean subtract(Timestamp left, Timestamp right,
+      HiveIntervalDayTime result) {
+    if (left == null || right == null) {
+      return false;
+    }
+
     nanosResult.addNanos(left.getNanos(), -(right.getNanos()));
 
     long totalSeconds = TimeUnit.MILLISECONDS.toSeconds(left.getTime())
         - TimeUnit.MILLISECONDS.toSeconds(right.getTime()) + nanosResult.seconds;
-    result = new HiveIntervalDayTime(totalSeconds, nanosResult.nanos);
-    return result;
+    result.set(totalSeconds, nanosResult.nanos);
+    return true;
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestTimestampWritableAndColumnVector.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestTimestampWritableAndColumnVector.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestTimestampWritableAndColumnVector.java
new file mode 100644
index 0000000..6c46257
--- /dev/null
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestTimestampWritableAndColumnVector.java
@@ -0,0 +1,68 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.exec.vector;
+
+import org.junit.Test;
+
+import java.math.BigDecimal;
+import java.math.RoundingMode;
+import java.sql.Timestamp;
+import java.util.Date;
+import java.util.Random;
+
+import org.apache.hadoop.hive.common.type.RandomTypeUtil;
+import org.apache.hadoop.hive.serde2.io.TimestampWritable;
+
+import static org.junit.Assert.*;
+
+/**
+ * Test for ListColumnVector
+ */
+public class TestTimestampWritableAndColumnVector {
+
+  private static int TEST_COUNT = 5000;
+
+  private static int fake = 0;
+
+  @Test
+  public void testDouble() throws Exception {
+
+    Random r = new Random(1234);
+    TimestampColumnVector timestampColVector = new TimestampColumnVector();
+    Timestamp[] randTimestamps = new Timestamp[VectorizedRowBatch.DEFAULT_SIZE];
+
+    for (int i = 0; i < VectorizedRowBatch.DEFAULT_SIZE; i++) {
+      Timestamp randTimestamp = RandomTypeUtil.getRandTimestamp(r);
+      randTimestamps[i] = randTimestamp;
+      timestampColVector.set(i, randTimestamp);
+    }
+    for (int i = 0; i < VectorizedRowBatch.DEFAULT_SIZE; i++) {
+      Timestamp retrievedTimestamp = timestampColVector.asScratchTimestamp(i);
+      Timestamp randTimestamp = randTimestamps[i];
+      if (!retrievedTimestamp.equals(randTimestamp)) {
+        assertTrue(false);
+      }
+      double randDouble = TimestampWritable.getDouble(randTimestamp);
+      double retrievedDouble = timestampColVector.getDouble(i);
+      if (randDouble != retrievedDouble) {
+        assertTrue(false);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorExpressionWriters.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorExpressionWriters.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorExpressionWriters.java
index fc38dd3..02602f4 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorExpressionWriters.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorExpressionWriters.java
@@ -229,18 +229,19 @@ public class TestVectorExpressionWriters {
   }
 
   private void testWriterTimestamp(TypeInfo type) throws HiveException {
-    TimestampColumnVector tcv = VectorizedRowGroupGenUtil.generateTimestampColumnVector(true, false,
-        vectorSize, new Random(10));
+    Timestamp[] timestampValues = new Timestamp[vectorSize];
+    TimestampColumnVector tcv =
+        VectorizedRowGroupGenUtil.generateTimestampColumnVector(true, false,
+        vectorSize, new Random(10), timestampValues);
     tcv.isNull[3] = true;
     VectorExpressionWriter vew = getWriter(type);
     for (int i = 0; i < vectorSize; i++) {
       Writable w = (Writable) vew.writeValue(tcv, i);
       if (w != null) {
-        Writable expected = getWritableValue(type, tcv.asScratchTimestamp(i));
+        Writable expected = getWritableValue(type, timestampValues[i]);
         TimestampWritable t1 = (TimestampWritable) expected;
         TimestampWritable t2 = (TimestampWritable) w;
-        Assert.assertTrue(t1.getNanos() == t2.getNanos());
-        Assert.assertTrue(t1.getSeconds() == t2.getSeconds());
+        Assert.assertTrue(t1.equals(t2));
        } else {
         Assert.assertTrue(tcv.isNull[i]);
       }
@@ -248,8 +249,10 @@ public class TestVectorExpressionWriters {
   }
 
   private void testSetterTimestamp(TypeInfo type) throws HiveException {
-    TimestampColumnVector tcv = VectorizedRowGroupGenUtil.generateTimestampColumnVector(true, false,
-        vectorSize, new Random(10));
+    Timestamp[] timestampValues = new Timestamp[vectorSize];
+    TimestampColumnVector tcv =
+        VectorizedRowGroupGenUtil.generateTimestampColumnVector(true, false,
+        vectorSize, new Random(10), timestampValues);
     tcv.isNull[3] = true;
 
     Object[] values = new Object[this.vectorSize];
@@ -259,12 +262,10 @@ public class TestVectorExpressionWriters {
       values[i] = null;  // setValue() should be able to handle null input
       values[i] = vew.setValue(values[i], tcv, i);
       if (values[i] != null) {
-        Timestamp scratchTimestamp = tcv.asScratchTimestamp(i);
-        Writable expected = getWritableValue(type, scratchTimestamp);
+        Writable expected = getWritableValue(type, timestampValues[i]);
         TimestampWritable t1 = (TimestampWritable) expected;
         TimestampWritable t2 = (TimestampWritable) values[i];
-        Assert.assertTrue(t1.getNanos() == t2.getNanos());
-        Assert.assertTrue(t1.getSeconds() == t2.getSeconds());
+        Assert.assertTrue(t1.equals(t2));
       } else {
         Assert.assertTrue(tcv.isNull[i]);
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorFilterExpressions.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorFilterExpressions.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorFilterExpressions.java
index 819cc27..80f55dc 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorFilterExpressions.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorFilterExpressions.java
@@ -25,7 +25,6 @@ import static org.junit.Assert.assertTrue;
 import java.sql.Timestamp;
 
 import org.apache.hadoop.hive.common.type.HiveDecimal;
-import org.apache.hadoop.hive.common.type.PisaTimestamp;
 import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;

http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorMathFunctions.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorMathFunctions.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorMathFunctions.java
index c14eb4a..31add6e 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorMathFunctions.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorMathFunctions.java
@@ -19,11 +19,13 @@
 package org.apache.hadoop.hive.ql.exec.vector.expressions;
 
 import java.io.UnsupportedEncodingException;
+import java.sql.Timestamp;
 import java.util.Arrays;
+import java.util.Random;
 
 import junit.framework.Assert;
 
-import org.apache.hadoop.hive.common.type.PisaTimestamp;
+import org.apache.hadoop.hive.common.type.RandomTypeUtil;
 import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
@@ -52,6 +54,7 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.FuncSignLongToDoubl
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.FuncSinDoubleToDouble;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.FuncSqrtDoubleToDouble;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.FuncTanDoubleToDouble;
+import org.apache.hadoop.hive.serde2.io.TimestampWritable;
 import org.junit.Test;
 
 
@@ -194,22 +197,22 @@ public class TestVectorMathFunctions {
     return batch;
   }
 
-  public static VectorizedRowBatch getVectorizedRowBatchTimestampInDoubleOut() {
+  public static VectorizedRowBatch getVectorizedRowBatchTimestampInDoubleOut(double[] doubleValues) {
+    Random r = new Random(45993);
     VectorizedRowBatch batch = new VectorizedRowBatch(2);
     TimestampColumnVector tcv;
     DoubleColumnVector dcv;
-    tcv = new TimestampColumnVector();
-    dcv = new DoubleColumnVector();
-    tcv.set(0, new PisaTimestamp(0, -2));
-    tcv.set(1, new PisaTimestamp(0, -1));
-    tcv.set(2, new PisaTimestamp(0, 0));
-    tcv.set(3, new PisaTimestamp(0, 1));
-    tcv.set(4, new PisaTimestamp(0, 2));
+    tcv = new TimestampColumnVector(doubleValues.length);
+    dcv = new DoubleColumnVector(doubleValues.length);
+    for (int i = 0; i < doubleValues.length; i++) {
+      doubleValues[i] = r.nextDouble() % (double) SECONDS_LIMIT;
+      dcv.vector[i] = doubleValues[i];
+    }
 
     batch.cols[0] = tcv;
     batch.cols[1] = dcv;
 
-    batch.size = 5;
+    batch.size = doubleValues.length;
     return batch;
   }
 
@@ -228,35 +231,45 @@ public class TestVectorMathFunctions {
     return batch;
   }
 
-  public static VectorizedRowBatch getVectorizedRowBatchTimestampInLongOut() {
+  public static VectorizedRowBatch getVectorizedRowBatchTimestampInLongOut(long[] longValues) {
+    Random r = new Random(345);
     VectorizedRowBatch batch = new VectorizedRowBatch(2);
     TimestampColumnVector inV;
     LongColumnVector outV;
-    inV = new TimestampColumnVector();
-    outV = new LongColumnVector();
-    inV.setTimestampSeconds(0, 2);
-    inV.setTimestampSeconds(1, 2);
+    inV = new TimestampColumnVector(longValues.length);
+    outV = new LongColumnVector(longValues.length);
+    for (int i = 0; i < longValues.length; i++) {
+      Timestamp randTimestamp = RandomTypeUtil.getRandTimestamp(r);
+      longValues[i] = TimestampWritable.getLong(randTimestamp);
+      inV.set(0, randTimestamp);
+    }
 
     batch.cols[0] = inV;
     batch.cols[1] = outV;
 
-    batch.size = 2;
+    batch.size = longValues.length;
     return batch;
   }
 
-  public static VectorizedRowBatch getVectorizedRowBatchLongInTimestampOut() {
+  static long SECONDS_LIMIT = 60L * 24L * 365L * 9999L;
+
+  public static VectorizedRowBatch getVectorizedRowBatchLongInTimestampOut(long[] longValues) {
+    Random r = new Random(12099);
     VectorizedRowBatch batch = new VectorizedRowBatch(2);
     LongColumnVector inV;
     TimestampColumnVector outV;
     inV = new LongColumnVector();
     outV = new TimestampColumnVector();
-    inV.vector[0] = -2;
-    inV.vector[1] = 2;
+
+    for (int i = 0; i < longValues.length; i++) {
+      longValues[i] = r.nextLong() % SECONDS_LIMIT;
+      inV.vector[i] = longValues[i];
+    }
 
     batch.cols[0] = inV;
     batch.cols[1] = outV;
 
-    batch.size = 2;
+    batch.size = longValues.length;
     return batch;
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorTimestampExpressions.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorTimestampExpressions.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorTimestampExpressions.java
index 375f369..d4f1f6f 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorTimestampExpressions.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorTimestampExpressions.java
@@ -32,7 +32,6 @@ import java.util.Random;
 import junit.framework.Assert;
 
 import org.apache.commons.lang.ArrayUtils;
-import org.apache.hadoop.hive.common.type.PisaTimestamp;
 import org.apache.hadoop.hive.common.type.RandomTypeUtil;
 import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
@@ -84,7 +83,7 @@ public class TestVectorTimestampExpressions {
   }
 
   private Timestamp[] getAllBoundaries() {
-    return getAllBoundaries(0000, 9999);
+    return getAllBoundaries(RandomTypeUtil.MIN_YEAR, RandomTypeUtil.MAX_YEAR);
   }
 
   private VectorizedRowBatch getVectorizedRandomRowBatchTimestampLong(int seed, int size) {
@@ -742,27 +741,14 @@ public class TestVectorTimestampExpressions {
     testVectorUDFSecond(TestType.STRING_LONG);
   }
 
-  private LongWritable getLongWritable(TimestampWritable i) {
-    LongWritable result = new LongWritable();
-    if (i == null) {
-      return null;
-    } else {
-      result.set(i.getSeconds());
-      return result;
+  private void compareToUDFUnixTimeStampLong(Timestamp ts, long y) {
+    long seconds = ts.getTime() / 1000;
+    if(seconds != y) {
+      System.out.printf("%d vs %d for %s\n", seconds, y, ts.toString());
+      Assert.assertTrue(false);
     }
   }
 
-  private void compareToUDFUnixTimeStampLong(Timestamp t, long y) {
-    TimestampWritable tsw = new TimestampWritable(t);
-    LongWritable res = getLongWritable(tsw);
-    if(res.get() != y) {
-      System.out.printf("%d vs %d for %s, %d\n", res.get(), y, t.toString(),
-          tsw.getTimestamp().getTime()/1000);
-    }
-
-    Assert.assertEquals(res.get(), y);
-  }
-
   private void verifyUDFUnixTimeStamp(VectorizedRowBatch batch, TestType testType) {
     VectorExpression udf;
     if (testType == TestType.TIMESTAMP_LONG) {