You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by mm...@apache.org on 2016/04/10 08:59:37 UTC

[05/12] 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/ca11c393/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/ca11c393/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/ca11c393/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/ca11c393/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/ca11c393/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/ca11c393/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/ca11c393/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/ca11c393/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/ca11c393/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/ca11c393/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/ca11c393/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) {

http://git-wip-us.apache.org/repos/asf/hive/blob/ca11c393/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorTypeCasts.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorTypeCasts.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorTypeCasts.java
index 038e382..1e41fce 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorTypeCasts.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorTypeCasts.java
@@ -23,13 +23,19 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
 import java.io.UnsupportedEncodingException;
+import java.math.BigDecimal;
+import java.math.MathContext;
+import java.math.RoundingMode;
+import java.sql.Timestamp;
 import java.util.Arrays;
+import java.util.Random;
+import java.util.concurrent.TimeUnit;
 
 import junit.framework.Assert;
 
 import org.apache.hadoop.hive.common.type.Decimal128;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
-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.DecimalColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
@@ -39,6 +45,7 @@ import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.*;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.*;
 import org.apache.hadoop.hive.serde2.io.TimestampWritable;
+import org.apache.hadoop.hive.serde2.typeinfo.HiveDecimalUtils;
 import org.junit.Test;
 
 /**
@@ -84,8 +91,8 @@ public class TestVectorTypeCasts {
     b.cols[0].noNulls = true;
     VectorExpression expr = new CastDoubleToTimestamp(0, 1);
     expr.evaluate(b);
-    Assert.assertEquals(0.0, resultV.getTimestampSecondsWithFractionalNanos(3));
-    Assert.assertEquals(0.5d, resultV.getTimestampSecondsWithFractionalNanos(4));
+    Assert.assertEquals(0.0, TimestampWritable.getDouble(resultV.asScratchTimestamp(3)));
+    Assert.assertEquals(0.5d, TimestampWritable.getDouble(resultV.asScratchTimestamp(4)));
   }
 
   @Test
@@ -103,39 +110,51 @@ public class TestVectorTypeCasts {
 
   @Test
   public void testCastLongToTimestamp() {
-    VectorizedRowBatch b = TestVectorMathFunctions.getVectorizedRowBatchLongInTimestampOut();
+    long[] longValues = new long[500];
+    VectorizedRowBatch b = TestVectorMathFunctions.getVectorizedRowBatchLongInTimestampOut(longValues);
     TimestampColumnVector resultV = (TimestampColumnVector) b.cols[1];
     b.cols[0].noNulls = true;
     VectorExpression expr = new CastLongToTimestamp(0, 1);
     expr.evaluate(b);
-    Assert.assertEquals(-2, resultV.getTimestampSeconds(0));
-    Assert.assertEquals(2, resultV.getTimestampSeconds(1));
+    for (int i = 0; i < longValues.length; i++) {
+      Timestamp timestamp = resultV.asScratchTimestamp(i);
+      long actual = TimestampWritable.getLong(timestamp);
+      assertEquals(actual, longValues[i]);
+    }
   }
 
   @Test
   public void testCastTimestampToLong() {
-    VectorizedRowBatch b = TestVectorMathFunctions.getVectorizedRowBatchTimestampInLongOut();
+    long[] longValues = new long[500];
+    VectorizedRowBatch b = TestVectorMathFunctions.getVectorizedRowBatchTimestampInLongOut(longValues);
     TimestampColumnVector inV = (TimestampColumnVector) b.cols[0];
-    inV.set(0, new PisaTimestamp(0, PisaTimestamp.NANOSECONDS_PER_SECOND));  // Make one entry produce interesting result
-      // (1 sec after epoch).
-
     LongColumnVector resultV = (LongColumnVector) b.cols[1];
     b.cols[0].noNulls = true;
     VectorExpression expr = new CastTimestampToLong(0, 1);
     expr.evaluate(b);
-    Assert.assertEquals(1, resultV.vector[0]);
+    for (int i = 0; i < longValues.length; i++) {
+      long actual = resultV.vector[i];
+      long timestampLong = inV.getTimestampAsLong(i);
+      if (actual != timestampLong) {
+        assertTrue(false);
+      }
+    }
   }
 
   @Test
   public void testCastTimestampToDouble() {
-    VectorizedRowBatch b = TestVectorMathFunctions.getVectorizedRowBatchTimestampInDoubleOut();
+    double[] doubleValues = new double[500];
+    VectorizedRowBatch b = TestVectorMathFunctions.getVectorizedRowBatchTimestampInDoubleOut(doubleValues);
     TimestampColumnVector inV = (TimestampColumnVector) b.cols[0];
     DoubleColumnVector resultV = (DoubleColumnVector) b.cols[1];
     b.cols[0].noNulls = true;
     VectorExpression expr = new CastTimestampToDouble(0, 1);
     expr.evaluate(b);
-    Assert.assertEquals(-1E-9D , resultV.vector[1]);
-    Assert.assertEquals(1E-9D, resultV.vector[3]);
+    for (int i = 0; i < doubleValues.length; i++) {
+      double actual = resultV.vector[i];
+      double doubleValue = TimestampWritable.getDouble(inV.asScratchTimestamp(i));
+      assertEquals(actual, doubleValue, 0.000000001F);
+    }
   }
 
   public byte[] toBytes(String s) {
@@ -356,16 +375,19 @@ public class TestVectorTypeCasts {
 
   @Test
   public void testCastDecimalToTimestamp() {
-    VectorizedRowBatch b = getBatchDecimalTimestamp();
+    double[] doubleValues = new double[500];
+    VectorizedRowBatch b = getBatchDecimalTimestamp(doubleValues);
     VectorExpression expr = new CastDecimalToTimestamp(0, 1);
     expr.evaluate(b);
     TimestampColumnVector r = (TimestampColumnVector) b.cols[1];
-    assertEquals(1111111111L, r.getNanoOfDay(0));
-    assertEquals(0L, r.getEpochDay(0));
-    assertEquals(-2222222222L, r.getNanoOfDay(1));
-    assertEquals(0L, r.getEpochDay(1));
-    assertEquals(999999999L, r.getNanoOfDay(2));
-    assertEquals(365L, r.getEpochDay(2));
+    for (int i = 0; i < doubleValues.length; i++) {
+      Timestamp timestamp = r.asScratchTimestamp(i);
+      double asDouble = TimestampWritable.getDouble(timestamp);
+      double expectedDouble = doubleValues[i];
+      if (expectedDouble != asDouble) {
+        assertTrue(false);
+      }
+    }
   }
 
   private VectorizedRowBatch getBatchDecimalLong2() {
@@ -384,19 +406,25 @@ public class TestVectorTypeCasts {
     return b;
   }
 
-  private VectorizedRowBatch getBatchDecimalTimestamp() {
+  private VectorizedRowBatch getBatchDecimalTimestamp(double[] doubleValues) {
     VectorizedRowBatch b = new VectorizedRowBatch(2);
     DecimalColumnVector dv;
-    short scale = 9;
-    b.cols[0] = dv = new DecimalColumnVector(18, scale);
-    b.cols[1] = new TimestampColumnVector();
-
-    b.size = 3;
-
-    dv.vector[0].set(HiveDecimal.create("1.111111111").setScale(scale));
-    dv.vector[1].set(HiveDecimal.create("-2.222222222").setScale(scale));
-    dv.vector[2].set(HiveDecimal.create("31536000.999999999").setScale(scale));
-
+    b.cols[0] = dv = new DecimalColumnVector(doubleValues.length, HiveDecimal.SYSTEM_DEFAULT_PRECISION, HiveDecimal.SYSTEM_DEFAULT_SCALE);
+    b.cols[1] = new TimestampColumnVector(doubleValues.length);
+    dv.noNulls = true;
+    Random r = new Random(94830);
+    for (int i = 0; i < doubleValues.length; i++) {
+      long millis = RandomTypeUtil.randomMillis(r);
+      Timestamp ts = new Timestamp(millis);
+      int nanos = RandomTypeUtil.randomNanos(r);
+      ts.setNanos(nanos);
+      TimestampWritable tsw = new TimestampWritable(ts);
+      double asDouble = tsw.getDouble();
+      doubleValues[i] = asDouble;
+      HiveDecimal hiveDecimal = HiveDecimal.create(new BigDecimal(asDouble));
+      dv.set(i, hiveDecimal);
+    }
+    b.size = doubleValues.length;
     return b;
   }
 
@@ -422,14 +450,44 @@ public class TestVectorTypeCasts {
     return b;
   }
 
-  private VectorizedRowBatch getBatchTimestampDecimal() {
+
+  public static final long NANOSECONDS_PER_SECOND = TimeUnit.SECONDS.toNanos(1);
+  public static final long MILLISECONDS_PER_SECOND = TimeUnit.SECONDS.toMillis(1);
+  public static final long NANOSECONDS_PER_MILLISSECOND = TimeUnit.MILLISECONDS.toNanos(1);
+
+  private VectorizedRowBatch getBatchTimestampDecimal(HiveDecimal[] hiveDecimalValues) {
+    Random r = new Random(994);
     VectorizedRowBatch b = new VectorizedRowBatch(2);
     TimestampColumnVector tcv;
-    b.cols[0] = tcv = new TimestampColumnVector();
-    b.cols[1] = new DecimalColumnVector(18, 2);
-    tcv.set(0, new PisaTimestamp( 0, 0));
-    tcv.set(1, new PisaTimestamp( 0, -1));
-    tcv.set(2, new PisaTimestamp( 99999999999999L / PisaTimestamp.NANOSECONDS_PER_DAY, 99999999999999L % PisaTimestamp.NANOSECONDS_PER_DAY));
+    b.cols[0] = tcv = new TimestampColumnVector(hiveDecimalValues.length);
+    b.cols[1] = new DecimalColumnVector(hiveDecimalValues.length, HiveDecimal.SYSTEM_DEFAULT_PRECISION, HiveDecimal.SYSTEM_DEFAULT_SCALE);
+    for (int i = 0; i < hiveDecimalValues.length; i++) {
+      int optionalNanos = 0;
+      switch (r.nextInt(4)) {
+      case 0:
+        // No nanos.
+        break;
+      case 1:
+        optionalNanos = r.nextInt((int) NANOSECONDS_PER_SECOND);
+        break;
+      case 2:
+        // Limit to milliseconds only...
+        optionalNanos = r.nextInt((int) MILLISECONDS_PER_SECOND) * (int) NANOSECONDS_PER_MILLISSECOND;
+        break;
+      case 3:
+        // Limit to below milliseconds only...
+        optionalNanos = r.nextInt((int) NANOSECONDS_PER_MILLISSECOND);
+        break;
+      }
+      long millis = RandomTypeUtil.randomMillis(r);
+      Timestamp ts = new Timestamp(millis);
+      ts.setNanos(optionalNanos);
+      TimestampWritable tsw = new TimestampWritable(ts);
+      hiveDecimalValues[i] = tsw.getHiveDecimal();
+
+      tcv.set(i, ts);
+    }
+    b.size = hiveDecimalValues.length;
     return b;
   }
 
@@ -440,9 +498,18 @@ public class TestVectorTypeCasts {
     expr.evaluate(b);
     DecimalColumnVector r = (DecimalColumnVector) b.cols[1];
 
-    assertTrue(r.vector[0].getHiveDecimal().equals(HiveDecimal.create("0.0")));
-    assertTrue(r.vector[1].getHiveDecimal().equals(HiveDecimal.create("-1.0")));
-    assertTrue(r.vector[2].getHiveDecimal().equals(HiveDecimal.create("99999999999999")));
+    HiveDecimal hd0 = HiveDecimal.create("0.0");
+    if (!hd0.equals(r.vector[0].getHiveDecimal())) {
+      assertTrue(false);
+    }
+    HiveDecimal hd1 = HiveDecimal.create("-1.0");
+    if (!hd1.equals(r.vector[1].getHiveDecimal())) {
+      assertTrue(false);
+    }
+    HiveDecimal hd2 = HiveDecimal.create("99999999999999");
+    if (!hd2.equals(r.vector[2].getHiveDecimal())) {
+      assertTrue(false);
+    }
   }
 
   private VectorizedRowBatch getBatchDoubleDecimal() {
@@ -496,25 +563,37 @@ public class TestVectorTypeCasts {
 
     // The input timestamps are stored as long values
     // measured in nanoseconds from the epoch.
-    VectorizedRowBatch b = getBatchTimestampDecimal();
+    HiveDecimal[] hiveDecimalValues = new HiveDecimal[500];
+    VectorizedRowBatch b = getBatchTimestampDecimal(hiveDecimalValues);
     VectorExpression expr = new CastTimestampToDecimal(0, 1);
     TimestampColumnVector inT = (TimestampColumnVector) b.cols[0];
-    inT.set(1, new PisaTimestamp(0, -1990000000L));
     expr.evaluate(b);
     DecimalColumnVector r = (DecimalColumnVector) b.cols[1];
-    assertTrue(r.vector[0].getHiveDecimal().equals(HiveDecimal.create("0.00")));
-    assertTrue(r.vector[1].getHiveDecimal().equals(HiveDecimal.create("-1.99")));
-    assertTrue(r.vector[2].getHiveDecimal().equals(HiveDecimal.create("100000.00")));
+    for (int i = 0; i < hiveDecimalValues.length; i++) {
+      HiveDecimal hiveDecimal = r.vector[i].getHiveDecimal();
+      HiveDecimal expectedHiveDecimal = hiveDecimalValues[i];
+      if (!hiveDecimal.equals(expectedHiveDecimal)) {
+        assertTrue(false);
+      }
+    }
 
     // Try again with a value that won't fit in 5 digits, to make
     // sure that NULL is produced.
-    b = getBatchTimestampDecimalPrec5Scale2();
+    b.cols[1] = r = new DecimalColumnVector(hiveDecimalValues.length, 5, 2);
     expr.evaluate(b);
     r = (DecimalColumnVector) b.cols[1];
-    assertFalse(r.noNulls);
-    assertFalse(r.isNull[0]);
-    assertFalse(r.isNull[1]);
-    assertTrue(r.isNull[2]);
+    for (int i = 0; i < hiveDecimalValues.length; i++) {
+      HiveDecimal hiveDecimal = r.vector[i].getHiveDecimal();
+      HiveDecimal expectedHiveDecimal = hiveDecimalValues[i];
+      if (HiveDecimal.enforcePrecisionScale(expectedHiveDecimal, 5, 2) == null) {
+        assertTrue(r.isNull[i]);
+      } else {
+        assertTrue(!r.isNull[i]);
+        if (!hiveDecimal.equals(expectedHiveDecimal)) {
+          assertTrue(false);
+        }
+      }
+    }
   }
 
   /* This batch has output decimal column precision 5 and scale 2.
@@ -533,41 +612,6 @@ public class TestVectorTypeCasts {
     return b;
   }
 
-  private VectorizedRowBatch getBatchTimestampDecimalPrec5Scale2() {
-    VectorizedRowBatch b = new VectorizedRowBatch(2);
-    TimestampColumnVector tcv;
-    b.cols[0] = tcv = new TimestampColumnVector();
-    b.cols[1] = new DecimalColumnVector(5, 2);
-    tcv.set(0, new PisaTimestamp(0, 0));
-    tcv.set(1, new PisaTimestamp(0, -1));
-    tcv.set(2, new PisaTimestamp(99999999999999L / PisaTimestamp.NANOSECONDS_PER_DAY, 99999999999999L % PisaTimestamp.NANOSECONDS_PER_DAY));
-    return b;
-  }
-
-  /*
-  @Test
-  public void testCastDecimalToDecimal() {
-
-    // test casting from one precision and scale to another.
-    VectorizedRowBatch b = getBatchDecimalDecimal();
-    VectorExpression expr = new CastDecimalToDecimal(0, 1);
-    expr.evaluate(b);
-    DecimalColumnVector r = (DecimalColumnVector) b.cols[1];
-    assertTrue(r.vector[0].getHiveDecimal().equals(HiveDecimal.create("10.00", (short) 2)));
-    assertFalse(r.noNulls);
-    assertTrue(r.isNull[1]);
-
-    // test an increase in precision/scale
-    b = getBatchDecimalDecimal();
-    expr = new CastDecimalToDecimal(1, 0);
-    expr.evaluate(b);
-    r = (DecimalColumnVector) b.cols[0];
-    assertTrue(r.vector[0].getHiveDecimal().equals(HiveDecimal.create("100.01", (short) 4)));
-    assertTrue(r.vector[1].getHiveDecimal().equals(HiveDecimal.create("-200.02", (short) 4)));
-    assertTrue(r.noNulls);
-  }
-  */
-
   private VectorizedRowBatch getBatchDecimalDecimal() {
     VectorizedRowBatch b = new VectorizedRowBatch(2);
 

http://git-wip-us.apache.org/repos/asf/hive/blob/ca11c393/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/util/FakeVectorRowBatchFromObjectIterables.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/util/FakeVectorRowBatchFromObjectIterables.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/util/FakeVectorRowBatchFromObjectIterables.java
index ab86082..98849c3 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/util/FakeVectorRowBatchFromObjectIterables.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/util/FakeVectorRowBatchFromObjectIterables.java
@@ -27,7 +27,6 @@ import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
 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.ColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector;
@@ -111,7 +110,7 @@ public class FakeVectorRowBatchFromObjectIterables extends FakeVectorRowBatchBas
               Object value) {
             TimestampColumnVector lcv = (TimestampColumnVector) columnVector;
             Timestamp t = (Timestamp) value;
-            lcv.set(row, new PisaTimestamp(t));
+            lcv.set(row, t);
           }
         };
 

http://git-wip-us.apache.org/repos/asf/hive/blob/ca11c393/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/util/VectorizedRowGroupGenUtil.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/util/VectorizedRowGroupGenUtil.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/util/VectorizedRowGroupGenUtil.java
index 649e52b..84717b1 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/util/VectorizedRowGroupGenUtil.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/util/VectorizedRowGroupGenUtil.java
@@ -22,7 +22,6 @@ import java.sql.Timestamp;
 import java.util.Random;
 
 import org.apache.hadoop.hive.common.type.HiveDecimal;
-import org.apache.hadoop.hive.common.type.PisaTimestamp;
 import org.apache.hadoop.hive.common.type.RandomTypeUtil;
 import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
@@ -81,7 +80,7 @@ public class VectorizedRowGroupGenUtil {
   }
 
   public static TimestampColumnVector generateTimestampColumnVector(
-      boolean nulls, boolean repeating, int size, Random rand) {
+      boolean nulls, boolean repeating, int size, Random rand, Timestamp[] timestampValues) {
     TimestampColumnVector tcv = new TimestampColumnVector(size);
 
     tcv.noNulls = !nulls;
@@ -95,10 +94,17 @@ public class VectorizedRowGroupGenUtil {
       if(nulls && (repeating || i % nullFrequency == 0)) {
         tcv.isNull[i] = true;
         tcv.setNullValue(i);
-
+        timestampValues[i] = null;
       }else {
         tcv.isNull[i] = false;
-        tcv.set(i, repeating ? repeatingTimestamp : RandomTypeUtil.getRandTimestamp(rand));
+        if (!repeating) {
+          Timestamp randomTimestamp = RandomTypeUtil.getRandTimestamp(rand);
+          tcv.set(i,  randomTimestamp);
+          timestampValues[i] = randomTimestamp;
+        } else {
+          tcv.set(i, repeatingTimestamp);
+          timestampValues[i] = repeatingTimestamp;
+        }
       }
     }
     return tcv;

http://git-wip-us.apache.org/repos/asf/hive/blob/ca11c393/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestInputOutputFormat.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestInputOutputFormat.java b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestInputOutputFormat.java
index 6f84708..15fcdb6 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestInputOutputFormat.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestInputOutputFormat.java
@@ -1850,7 +1850,7 @@ public class TestInputOutputFormat {
       long millis = (long) i * MILLIS_IN_DAY;
       millis -= LOCAL_TIMEZONE.getOffset(millis);
       assertEquals("checking timestamp " + i, millis,
-          timestampColumn.getTimestampMilliseconds(i));
+          timestampColumn.getTime(i));
     }
     assertEquals(false, reader.next(key, value));
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/ca11c393/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcFile.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcFile.java b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcFile.java
index 3843c6d..1a97a6d 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcFile.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcFile.java
@@ -541,7 +541,10 @@ public class TestOrcFile {
     int idx = 0;
     while (rows.hasNext()) {
       Object row = rows.next(null);
-      assertEquals(tslist.get(idx++).getNanos(), ((TimestampWritable) row).getNanos());
+      Timestamp tlistTimestamp = tslist.get(idx++);
+      if (tlistTimestamp.getNanos() != ((TimestampWritable) row).getNanos()) {
+        assertTrue(false);
+      }
     }
     assertEquals(0, writer.getSchema().getMaximumId());
     boolean[] expected = new boolean[] {false};

http://git-wip-us.apache.org/repos/asf/hive/blob/ca11c393/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestVectorOrcFile.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestVectorOrcFile.java b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestVectorOrcFile.java
index 4ca20c5..a82d672 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestVectorOrcFile.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestVectorOrcFile.java
@@ -24,7 +24,6 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 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;
@@ -527,7 +526,7 @@ public class TestVectorOrcFile {
     batch.size = tslist.size();
     for (int i=0; i < tslist.size(); ++i) {
       Timestamp ts = tslist.get(i);
-      vec.set(i, new PisaTimestamp(ts));
+      vec.set(i, ts);
     }
     writer.addRowBatch(batch);
     writer.close();
@@ -1345,8 +1344,8 @@ public class TestVectorOrcFile {
       for (int ms = 1000; ms < 2000; ++ms) {
         TimestampColumnVector timestampColVector = (TimestampColumnVector) batch.cols[0];
         timestampColVector.set(ms - 1000,
-            new PisaTimestamp(Timestamp.valueOf(year +
-                "-05-05 12:34:56." + ms)));
+            Timestamp.valueOf(year +
+                "-05-05 12:34:56." + ms));
         ((LongColumnVector) batch.cols[1]).vector[ms - 1000] =
             new DateWritable(new Date(year - 1900, 11, 25)).getDays();
       }
@@ -1385,7 +1384,7 @@ public class TestVectorOrcFile {
     UnionColumnVector union = (UnionColumnVector) batch.cols[1];
     if (ts != null) {
       TimestampColumnVector timestampColVector = (TimestampColumnVector) batch.cols[0];
-      timestampColVector.set(rowId, new PisaTimestamp(ts));
+      timestampColVector.set(rowId, ts);
     } else {
       batch.cols[0].isNull[rowId] = true;
       batch.cols[0].noNulls = false;
@@ -2178,8 +2177,8 @@ public class TestVectorOrcFile {
     ((LongColumnVector) batch.cols[6]).vector[0] =
         new DateWritable(new Date(111, 6, 1)).getDays();
     ((TimestampColumnVector) batch.cols[7]).set(0,
-        new PisaTimestamp(new Timestamp(115, 9, 23, 10, 11, 59,
-            999999999)));
+        new Timestamp(115, 9, 23, 10, 11, 59,
+            999999999));
     ((DecimalColumnVector) batch.cols[8]).vector[0] =
         new HiveDecimalWritable("1.234567");
     ((BytesColumnVector) batch.cols[9]).setVal(0, "Echelon".getBytes());
@@ -2234,9 +2233,8 @@ public class TestVectorOrcFile {
           new DateWritable(new Date(111, 6, 1)).getDays() + r;
 
       Timestamp ts = new Timestamp(115, 9, 23, 10, 11, 59, 999999999);
-      PisaTimestamp pisaTimestamp = new PisaTimestamp(ts);
-      pisaTimestamp.addSeconds(pisaTimestamp, r, pisaTimestamp);
-      ((TimestampColumnVector) batch.cols[7]).set(r, pisaTimestamp);
+      ts.setTime(ts.getTime() + r * 1000);
+      ((TimestampColumnVector) batch.cols[7]).set(r, ts);
       ((DecimalColumnVector) batch.cols[8]).vector[r] =
           new HiveDecimalWritable("1.234567");
       ((BytesColumnVector) batch.cols[9]).setVal(r,
@@ -2378,8 +2376,10 @@ public class TestVectorOrcFile {
           row.getFieldValue(5).toString());
       assertEquals("row " + r, new Date(111, 6, 1 + r).toString(),
           row.getFieldValue(6).toString());
+      Timestamp ts = new Timestamp(115, 9, 23, 10, 11, 59, 999999999);
+      ts.setTime(ts.getTime() + r * 1000);
       assertEquals("row " + r,
-          new Timestamp(115, 9, 23, 10, 11, 59 + r, 999999999).toString(),
+          ts.toString(),
           row.getFieldValue(7).toString());
       assertEquals("row " + r, "1.234567", row.getFieldValue(8).toString());
       assertEquals("row " + r, Integer.toString(r),

http://git-wip-us.apache.org/repos/asf/hive/blob/ca11c393/ql/src/test/queries/clientpositive/vector_interval_arithmetic.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/vector_interval_arithmetic.q b/ql/src/test/queries/clientpositive/vector_interval_arithmetic.q
new file mode 100644
index 0000000..40c4c03
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/vector_interval_arithmetic.q
@@ -0,0 +1,174 @@
+set hive.cli.print.header=true;
+set hive.explain.user=false;
+set hive.fetch.task.conversion=none;
+
+create table unique_timestamps (tsval timestamp) STORED AS TEXTFILE;
+
+LOAD DATA LOCAL INPATH '../../data/files/timestamps.txt' OVERWRITE INTO TABLE unique_timestamps;
+
+create table interval_arithmetic_1 (dateval date, tsval timestamp) stored as orc;
+insert overwrite table interval_arithmetic_1
+  select cast(tsval as date), tsval from unique_timestamps;
+
+SET hive.vectorized.execution.enabled=true;
+
+-- interval year-month arithmetic
+explain
+select
+  dateval,
+  dateval - interval '2-2' year to month,
+  dateval - interval '-2-2' year to month,
+  dateval + interval '2-2' year to month,
+  dateval + interval '-2-2' year to month,
+  - interval '2-2' year to month + dateval,
+  interval '2-2' year to month + dateval
+from interval_arithmetic_1
+order by dateval;
+
+select
+  dateval,
+  dateval - interval '2-2' year to month,
+  dateval - interval '-2-2' year to month,
+  dateval + interval '2-2' year to month,
+  dateval + interval '-2-2' year to month,
+  - interval '2-2' year to month + dateval,
+  interval '2-2' year to month + dateval
+from interval_arithmetic_1
+order by dateval;
+
+explain
+select
+  dateval,
+  dateval - date '1999-06-07',
+  date '1999-06-07' - dateval,
+  dateval - dateval
+from interval_arithmetic_1
+order by dateval;
+
+select
+  dateval,
+  dateval - date '1999-06-07',
+  date '1999-06-07' - dateval,
+  dateval - dateval
+from interval_arithmetic_1
+order by dateval;
+
+explain
+select
+  tsval,
+  tsval - interval '2-2' year to month,
+  tsval - interval '-2-2' year to month,
+  tsval + interval '2-2' year to month,
+  tsval + interval '-2-2' year to month,
+  - interval '2-2' year to month + tsval,
+  interval '2-2' year to month + tsval
+from interval_arithmetic_1
+order by tsval;
+
+select
+  tsval,
+  tsval - interval '2-2' year to month,
+  tsval - interval '-2-2' year to month,
+  tsval + interval '2-2' year to month,
+  tsval + interval '-2-2' year to month,
+  - interval '2-2' year to month + tsval,
+  interval '2-2' year to month + tsval
+from interval_arithmetic_1
+order by tsval;
+
+explain
+select
+  interval '2-2' year to month + interval '3-3' year to month,
+  interval '2-2' year to month - interval '3-3' year to month
+from interval_arithmetic_1
+order by interval '2-2' year to month + interval '3-3' year to month
+limit 2;
+
+select
+  interval '2-2' year to month + interval '3-3' year to month,
+  interval '2-2' year to month - interval '3-3' year to month
+from interval_arithmetic_1
+order by interval '2-2' year to month + interval '3-3' year to month
+limit 2;
+
+
+-- interval day-time arithmetic
+explain
+select
+  dateval,
+  dateval - interval '99 11:22:33.123456789' day to second,
+  dateval - interval '-99 11:22:33.123456789' day to second,
+  dateval + interval '99 11:22:33.123456789' day to second,
+  dateval + interval '-99 11:22:33.123456789' day to second,
+  -interval '99 11:22:33.123456789' day to second + dateval,
+  interval '99 11:22:33.123456789' day to second + dateval
+from interval_arithmetic_1
+order by dateval;
+
+select
+  dateval,
+  dateval - interval '99 11:22:33.123456789' day to second,
+  dateval - interval '-99 11:22:33.123456789' day to second,
+  dateval + interval '99 11:22:33.123456789' day to second,
+  dateval + interval '-99 11:22:33.123456789' day to second,
+  -interval '99 11:22:33.123456789' day to second + dateval,
+  interval '99 11:22:33.123456789' day to second + dateval
+from interval_arithmetic_1
+order by dateval;
+
+explain
+select
+  dateval,
+  tsval,
+  dateval - tsval,
+  tsval - dateval,
+  tsval - tsval
+from interval_arithmetic_1
+order by dateval;
+
+select
+  dateval,
+  tsval,
+  dateval - tsval,
+  tsval - dateval,
+  tsval - tsval
+from interval_arithmetic_1
+order by dateval;
+
+explain
+select
+  tsval,
+  tsval - interval '99 11:22:33.123456789' day to second,
+  tsval - interval '-99 11:22:33.123456789' day to second,
+  tsval + interval '99 11:22:33.123456789' day to second,
+  tsval + interval '-99 11:22:33.123456789' day to second,
+  -interval '99 11:22:33.123456789' day to second + tsval,
+  interval '99 11:22:33.123456789' day to second + tsval
+from interval_arithmetic_1
+order by tsval;
+
+select
+  tsval,
+  tsval - interval '99 11:22:33.123456789' day to second,
+  tsval - interval '-99 11:22:33.123456789' day to second,
+  tsval + interval '99 11:22:33.123456789' day to second,
+  tsval + interval '-99 11:22:33.123456789' day to second,
+  -interval '99 11:22:33.123456789' day to second + tsval,
+  interval '99 11:22:33.123456789' day to second + tsval
+from interval_arithmetic_1
+order by tsval;
+
+explain
+select
+  interval '99 11:22:33.123456789' day to second + interval '10 9:8:7.123456789' day to second,
+  interval '99 11:22:33.123456789' day to second - interval '10 9:8:7.123456789' day to second
+from interval_arithmetic_1
+limit 2;
+
+select
+  interval '99 11:22:33.123456789' day to second + interval '10 9:8:7.123456789' day to second,
+  interval '99 11:22:33.123456789' day to second - interval '10 9:8:7.123456789' day to second
+from interval_arithmetic_1
+limit 2;
+
+drop table interval_arithmetic_1;