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:02:26 UTC

[03/16] hive git commit: HIVE-9862 Vectorized execution corrupts timestamp values (Matt McCline, reviewed by Jason Dere)

http://git-wip-us.apache.org/repos/asf/hive/blob/4a479d0c/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/RandomRowObjectSource.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/RandomRowObjectSource.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/RandomRowObjectSource.java
index 6f5d1a0..2d4baa0 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/RandomRowObjectSource.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/RandomRowObjectSource.java
@@ -33,6 +33,7 @@ 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.RandomTypeUtil;
 import org.apache.hadoop.hive.serde.serdeConstants;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
@@ -258,7 +259,7 @@ public class RandomRowObjectSource {
     case BINARY:
       return getRandBinary(r, 1 + r.nextInt(100));
     case TIMESTAMP:
-      return getRandTimestamp(r);
+      return RandomTypeUtil.getRandTimestamp(r);
     case INTERVAL_YEAR_MONTH:
       return getRandIntervalYearMonth(r);
     case INTERVAL_DAY_TIME:
@@ -355,24 +356,6 @@ public class RandomRowObjectSource {
     return dateVal;
   }
 
-  public static Timestamp getRandTimestamp(Random r) {
-    String optionalNanos = "";
-    if (r.nextInt(2) == 1) {
-      optionalNanos = String.format(".%09d",
-          Integer.valueOf(0 + r.nextInt(DateUtils.NANOS_PER_SEC)));
-    }
-    String timestampStr = String.format("%d-%02d-%02d %02d:%02d:%02d%s",
-        Integer.valueOf(1970 + r.nextInt(200)),  // year
-        Integer.valueOf(1 + r.nextInt(12)),      // month
-        Integer.valueOf(1 + r.nextInt(28)),      // day
-        Integer.valueOf(0 + r.nextInt(24)),      // hour
-        Integer.valueOf(0 + r.nextInt(60)),      // minute
-        Integer.valueOf(0 + r.nextInt(60)),      // second
-        optionalNanos);
-    Timestamp timestampVal = Timestamp.valueOf(timestampStr);
-    return timestampVal;
-  }
-
   public static HiveIntervalYearMonth getRandIntervalYearMonth(Random r) {
     String yearMonthSignStr = r.nextInt(2) == 0 ? "" : "-";
     String intervalYearMonthStr = String.format("%s%d-%d",

http://git-wip-us.apache.org/repos/asf/hive/blob/4a479d0c/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorizationContext.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorizationContext.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorizationContext.java
index 515ea7b..e4c7529 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorizationContext.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorizationContext.java
@@ -50,6 +50,10 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.IfExprStringGroupColumn
 import org.apache.hadoop.hive.ql.exec.vector.expressions.IfExprStringGroupColumnVarCharScalar;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.IfExprStringScalarStringGroupColumn;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.IfExprStringScalarStringScalar;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.IfExprTimestampColumnColumn;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.IfExprTimestampColumnScalar;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.IfExprTimestampScalarColumn;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.IfExprTimestampScalarScalar;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.IfExprVarCharScalarStringGroupColumn;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.IsNotNull;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.IsNull;
@@ -67,11 +71,13 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.StringLTrim;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.StringLower;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.StringUpper;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorUDFUnixTimeStampLong;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorUDFYearLong;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorUDFUnixTimeStampDate;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorUDFUnixTimeStampTimestamp;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorUDFYearDate;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.FilterStringColumnInList;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.FilterLongColumnInList;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.FilterDoubleColumnInList;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorUDFYearTimestamp;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.IfExprLongColumnLongScalar;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.IfExprLongScalarLongScalar;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.IfExprLongScalarLongColumn;
@@ -151,10 +157,10 @@ public class TestVectorizationContext {
 
   @Test
   public void testVectorExpressionDescriptor() {
-    VectorUDFUnixTimeStampLong v1 = new VectorUDFUnixTimeStampLong();
+    VectorUDFUnixTimeStampDate v1 = new VectorUDFUnixTimeStampDate();
     VectorExpressionDescriptor.Builder builder1 = new VectorExpressionDescriptor.Builder();
     VectorExpressionDescriptor.Descriptor d1 = builder1.setMode(VectorExpressionDescriptor.Mode.PROJECTION)
-        .setNumArguments(1).setArgumentTypes(VectorExpressionDescriptor.ArgumentType.INT_DATETIME_INTERVAL_FAMILY)
+        .setNumArguments(1).setArgumentTypes(VectorExpressionDescriptor.ArgumentType.INT_DATE_INTERVAL_YEAR_MONTH)
         .setInputExpressionTypes(VectorExpressionDescriptor.InputExpressionType.COLUMN).build();
     assertTrue(d1.matches(v1.getDescriptor()));
 
@@ -1086,14 +1092,14 @@ public class TestVectorizationContext {
     tsFuncExpr.setGenericUDF(gudfBridge);
     tsFuncExpr.setChildren(children);
     VectorExpression ve = vc.getVectorExpression(tsFuncExpr);
-    Assert.assertEquals(VectorUDFYearLong.class, ve.getClass());
+    Assert.assertEquals(VectorUDFYearTimestamp.class, ve.getClass());
 
     //GenericUDFToUnixTimeStamp
     GenericUDFToUnixTimeStamp gudf = new GenericUDFToUnixTimeStamp();
     tsFuncExpr.setGenericUDF(gudf);
     tsFuncExpr.setTypeInfo(TypeInfoFactory.longTypeInfo);
     ve = vc.getVectorExpression(tsFuncExpr);
-    Assert.assertEquals(VectorUDFUnixTimeStampLong.class, ve.getClass());
+    Assert.assertEquals(VectorUDFUnixTimeStampTimestamp.class, ve.getClass());
   }
 
   @Test
@@ -1353,7 +1359,7 @@ public class TestVectorizationContext {
     children1.set(1, col2Expr);
     children1.set(2, col3Expr);
     ve = vc.getVectorExpression(exprDesc);
-    assertTrue(ve instanceof IfExprLongColumnLongColumn);
+    assertTrue(ve instanceof IfExprTimestampColumnColumn);
 
     // timestamp column/scalar IF where scalar is really a CAST of a constant to timestamp.
     ExprNodeGenericFuncDesc f = new ExprNodeGenericFuncDesc();
@@ -1368,20 +1374,20 @@ public class TestVectorizationContext {
     // We check for two different classes below because initially the result
     // is IfExprLongColumnLongColumn but in the future if the system is enhanced
     // with constant folding then the result will be IfExprLongColumnLongScalar.
-    assertTrue(IfExprLongColumnLongColumn.class == ve.getClass()
-               || IfExprLongColumnLongScalar.class == ve.getClass());
+    assertTrue(IfExprTimestampColumnColumn.class == ve.getClass()
+               || IfExprTimestampColumnScalar.class == ve.getClass());
 
     // timestamp scalar/scalar
     children1.set(1, f);
     ve = vc.getVectorExpression(exprDesc);
-    assertTrue(IfExprLongColumnLongColumn.class == ve.getClass()
-        || IfExprLongScalarLongScalar.class == ve.getClass());
+    assertTrue(IfExprTimestampColumnColumn.class == ve.getClass()
+        || IfExprTimestampScalarScalar.class == ve.getClass());
 
     // timestamp scalar/column
     children1.set(2, col3Expr);
     ve = vc.getVectorExpression(exprDesc);
-    assertTrue(IfExprLongColumnLongColumn.class == ve.getClass()
-        || IfExprLongScalarLongColumn.class == ve.getClass());
+    assertTrue(IfExprTimestampColumnColumn.class == ve.getClass()
+        || IfExprTimestampScalarColumn.class == ve.getClass());
 
     // test for boolean type
     col2Expr = new  ExprNodeColumnDesc(Boolean.class, "col2", "table", false);

http://git-wip-us.apache.org/repos/asf/hive/blob/4a479d0c/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestConstantVectorExpression.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestConstantVectorExpression.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestConstantVectorExpression.java
index c2bf85a..7b07293 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestConstantVectorExpression.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestConstantVectorExpression.java
@@ -44,9 +44,9 @@ public class TestConstantVectorExpression {
     String str = "alpha";
     ConstantVectorExpression bytesCve = new ConstantVectorExpression(2, str.getBytes());
     HiveDecimal decVal = HiveDecimal.create("25.8");
-    ConstantVectorExpression decimalCve = new ConstantVectorExpression(3, decVal);
+    ConstantVectorExpression decimalCve = new ConstantVectorExpression(3, decVal, "decimal");
     ConstantVectorExpression nullCve = new ConstantVectorExpression(4, "string", true);
-    
+
     int size = 20;
     VectorizedRowBatch vrg = VectorizedRowGroupGenUtil.getVectorizedRowBatch(size, 5, 0);
 

http://git-wip-us.apache.org/repos/asf/hive/blob/4a479d0c/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorDateExpressions.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorDateExpressions.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorDateExpressions.java
index 9c4a751..58cecc1 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorDateExpressions.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorDateExpressions.java
@@ -114,7 +114,7 @@ public class TestVectorDateExpressions {
 
   private void verifyUDFYear(VectorizedRowBatch batch) {
     VectorExpression udf = null;
-    udf = new VectorUDFYearLong(0, 1);
+    udf = new VectorUDFYearDate(0, 1);
     udf.setInputTypes(VectorExpression.Type.DATE);
     udf.evaluate(batch);
     final int in = 0;
@@ -176,7 +176,7 @@ public class TestVectorDateExpressions {
 
   private void verifyUDFDayOfMonth(VectorizedRowBatch batch) {
     VectorExpression udf = null;
-    udf = new VectorUDFDayOfMonthLong(0, 1);
+    udf = new VectorUDFDayOfMonthDate(0, 1);
     udf.setInputTypes(VectorExpression.Type.DATE);
     udf.evaluate(batch);
     final int in = 0;
@@ -238,7 +238,7 @@ public class TestVectorDateExpressions {
 
   private void verifyUDFMonth(VectorizedRowBatch batch) {
     VectorExpression udf;
-      udf = new VectorUDFMonthLong(0, 1);
+      udf = new VectorUDFMonthDate(0, 1);
     udf.setInputTypes(VectorExpression.Type.DATE);
     udf.evaluate(batch);
     final int in = 0;
@@ -314,7 +314,7 @@ public class TestVectorDateExpressions {
 
   private void verifyUDFUnixTimeStamp(VectorizedRowBatch batch) {
     VectorExpression udf;
-    udf = new VectorUDFUnixTimeStampLong(0, 1);
+    udf = new VectorUDFUnixTimeStampDate(0, 1);
     udf.setInputTypes(VectorExpression.Type.DATE);
     udf.evaluate(batch);
     final int in = 0;
@@ -376,7 +376,7 @@ public class TestVectorDateExpressions {
 
   private void verifyUDFWeekOfYear(VectorizedRowBatch batch) {
     VectorExpression udf;
-    udf = new VectorUDFWeekOfYearLong(0, 1);
+    udf = new VectorUDFWeekOfYearDate(0, 1);
     udf.setInputTypes(VectorExpression.Type.DATE);
     udf.evaluate(batch);
     final int in = 0;

http://git-wip-us.apache.org/repos/asf/hive/blob/4a479d0c/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 6523e7b..fc38dd3 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
@@ -25,14 +25,13 @@ import java.util.Random;
 
 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.HiveVarchar;
 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;
 import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.TimestampUtils;
+import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.util.VectorizedRowGroupGenUtil;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc;
@@ -45,7 +44,6 @@ import org.apache.hadoop.hive.serde2.io.ShortWritable;
 import org.apache.hadoop.hive.serde2.io.TimestampWritable;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
-import org.apache.hadoop.hive.serde2.objectinspector.StandardStructObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
 import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo;
@@ -84,6 +82,11 @@ public class TestVectorExpressionWriters {
     return null;
   }
 
+
+  private Writable getWritableValue(TypeInfo ti, Timestamp value) {
+    return new TimestampWritable(value);
+  }
+
   private Writable getWritableValue(TypeInfo ti, HiveDecimal value) {
     return new HiveDecimalWritable(value);
   }
@@ -113,7 +116,6 @@ public class TestVectorExpressionWriters {
       return new BooleanWritable( value == 0 ? false : true);
     } else if (ti.equals(TypeInfoFactory.timestampTypeInfo)) {
       Timestamp ts = new Timestamp(value);
-      TimestampUtils.assignTimeInNanoSec(value, ts);
       TimestampWritable tw = new TimestampWritable(ts);
       return tw;
     }
@@ -199,13 +201,6 @@ public class TestVectorExpressionWriters {
       Writable w = (Writable) vew.writeValue(lcv, i);
       if (w != null) {
         Writable expected = getWritableValue(type, lcv.vector[i]);
-        if (expected instanceof TimestampWritable) {
-          TimestampWritable t1 = (TimestampWritable) expected;
-          TimestampWritable t2 = (TimestampWritable) w;
-          Assert.assertTrue(t1.getNanos() == t2.getNanos());
-          Assert.assertTrue(t1.getSeconds() == t2.getSeconds());
-          continue;
-        }
         Assert.assertEquals(expected, w);
       } else {
         Assert.assertTrue(lcv.isNull[i]);
@@ -226,20 +221,56 @@ public class TestVectorExpressionWriters {
       values[i] = vew.setValue(values[i], lcv, i);
       if (values[i] != null) {
         Writable expected = getWritableValue(type, lcv.vector[i]);
-        if (expected instanceof TimestampWritable) {
-          TimestampWritable t1 = (TimestampWritable) expected;
-          TimestampWritable t2 = (TimestampWritable) values[i];
-          Assert.assertTrue(t1.getNanos() == t2.getNanos());
-          Assert.assertTrue(t1.getSeconds() == t2.getSeconds());
-          continue;
-        }
         Assert.assertEquals(expected, values[i]);
       } else {
         Assert.assertTrue(lcv.isNull[i]);
       }
     }
   }
-  
+
+  private void testWriterTimestamp(TypeInfo type) throws HiveException {
+    TimestampColumnVector tcv = VectorizedRowGroupGenUtil.generateTimestampColumnVector(true, false,
+        vectorSize, new Random(10));
+    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));
+        TimestampWritable t1 = (TimestampWritable) expected;
+        TimestampWritable t2 = (TimestampWritable) w;
+        Assert.assertTrue(t1.getNanos() == t2.getNanos());
+        Assert.assertTrue(t1.getSeconds() == t2.getSeconds());
+       } else {
+        Assert.assertTrue(tcv.isNull[i]);
+      }
+    }
+  }
+
+  private void testSetterTimestamp(TypeInfo type) throws HiveException {
+    TimestampColumnVector tcv = VectorizedRowGroupGenUtil.generateTimestampColumnVector(true, false,
+        vectorSize, new Random(10));
+    tcv.isNull[3] = true;
+
+    Object[] values = new Object[this.vectorSize];
+
+    VectorExpressionWriter vew = getWriter(type);
+    for (int i = 0; i < vectorSize; i++) {
+      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);
+        TimestampWritable t1 = (TimestampWritable) expected;
+        TimestampWritable t2 = (TimestampWritable) values[i];
+        Assert.assertTrue(t1.getNanos() == t2.getNanos());
+        Assert.assertTrue(t1.getSeconds() == t2.getSeconds());
+      } else {
+        Assert.assertTrue(tcv.isNull[i]);
+      }
+    }
+  }
+
   private StructObjectInspector genStructOI() {
     ArrayList<String> fieldNames1 = new ArrayList<String>();
     fieldNames1.add("theInt");
@@ -427,14 +458,14 @@ public class TestVectorExpressionWriters {
 
   @Test
   public void testVectorExpressionWriterTimestamp() throws HiveException {
-    testWriterLong(TypeInfoFactory.timestampTypeInfo);
+    testWriterTimestamp(TypeInfoFactory.timestampTypeInfo);
   }
 
   @Test
   public void testVectorExpressionSetterTimestamp() throws HiveException {
-    testSetterLong(TypeInfoFactory.timestampTypeInfo);
+    testSetterTimestamp(TypeInfoFactory.timestampTypeInfo);
   }
-  
+
   @Test
   public void testVectorExpressionWriterByte() throws HiveException {
     testWriterLong(TypeInfoFactory.byteTypeInfo);
@@ -469,67 +500,9 @@ public class TestVectorExpressionWriters {
   public void testVectorExpressionWriterBinary() throws HiveException {
     testWriterText(TypeInfoFactory.binaryTypeInfo);
   }
-  
+
   @Test
   public void testVectorExpressionSetterBinary() throws HiveException {
     testSetterText(TypeInfoFactory.binaryTypeInfo);
   }
-
-  @Test
-  public void testTimeStampUtils(){
-    Timestamp ts = new Timestamp(0);
-
-    // Convert positive nanoseconds to timestamp object.
-    TimestampUtils.assignTimeInNanoSec(1234567891, ts);
-    Assert.assertEquals(234567891, ts.getNanos());
-    Assert.assertEquals(1234567891, TimestampUtils.getTimeNanoSec(ts));
-
-    // Test negative nanoseconds
-    TimestampUtils.assignTimeInNanoSec(-1234567891, ts);
-    Assert.assertEquals((1000000000-234567891), ts.getNanos());
-    Assert.assertEquals(-1234567891, TimestampUtils.getTimeNanoSec(ts));
-
-    // Test positive value smaller than a second.
-    TimestampUtils.assignTimeInNanoSec(234567891, ts);
-    Assert.assertEquals(234567891, ts.getNanos());
-    Assert.assertEquals(234567891, TimestampUtils.getTimeNanoSec(ts));
-
-    // Test negative value smaller than a second.
-    TimestampUtils.assignTimeInNanoSec(-234567891, ts);
-    Assert.assertEquals((1000000000-234567891), ts.getNanos());
-    Assert.assertEquals(-234567891, TimestampUtils.getTimeNanoSec(ts));
-
-    // Test a positive long timestamp
-    long big = 152414813551296L;
-    TimestampUtils.assignTimeInNanoSec(big, ts);
-    Assert.assertEquals(big % 1000000000, ts.getNanos());
-    Assert.assertEquals(big, TimestampUtils.getTimeNanoSec(ts));
-
-    // Test a negative long timestamp
-    big = -152414813551296L;
-    TimestampUtils.assignTimeInNanoSec(big, ts);
-    Assert.assertEquals((1000000000 + (big % 1000000000)), ts.getNanos());
-    Assert.assertEquals(big, TimestampUtils.getTimeNanoSec(ts));
-
-    // big/1000000 will yield zero nanoseconds
-    big = -1794750230000828416L;
-    ts = new Timestamp(0);
-    TimestampUtils.assignTimeInNanoSec(big, ts);
-    Assert.assertEquals((1000000000 + big % 1000000000), ts.getNanos());
-    Assert.assertEquals(big, TimestampUtils.getTimeNanoSec(ts));
-
-    // Very small nanosecond part
-    big = 1700000000000000016L;
-    ts = new Timestamp(0);
-    TimestampUtils.assignTimeInNanoSec(big, ts);
-    Assert.assertEquals(big % 1000000000, ts.getNanos());
-    Assert.assertEquals(big, TimestampUtils.getTimeNanoSec(ts));
-
-    // Very small nanosecond part
-    big = -1700000000000000016L;
-    ts = new Timestamp(0);
-    TimestampUtils.assignTimeInNanoSec(big, ts);
-    Assert.assertEquals((1000000000 + big % 1000000000), ts.getNanos());
-    Assert.assertEquals(big, TimestampUtils.getTimeNanoSec(ts));
-  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/4a479d0c/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 3841317..819cc27 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,11 +25,12 @@ 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;
 import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.TimestampUtils;
+import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.FilterDecimalColGreaterEqualDecimalColumn;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.FilterDecimalColLessDecimalScalar;
@@ -49,6 +50,8 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.FilterStringColumnN
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.FilterDecimalColEqualDecimalScalar;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.FilterDecimalColEqualDecimalColumn;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.FilterDecimalScalarEqualDecimalColumn;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.FilterTimestampColumnBetween;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.FilterTimestampColumnNotBetween;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.LongColAddLongScalar;
 import org.apache.hadoop.hive.ql.exec.vector.util.VectorizedRowGroupGenUtil;
 import org.junit.Assert;
@@ -586,23 +589,23 @@ public class TestVectorFilterExpressions {
 
   @Test
   public void testFilterTimestampBetween() {
-    int seed = 17;
-    VectorizedRowBatch vrb = VectorizedRowGroupGenUtil.getVectorizedRowBatch(
-        5, 2, seed);
-    LongColumnVector lcv0 = (LongColumnVector) vrb.cols[0];
-    long startTS = 0; // the epoch
-    long endTS = TimestampUtils.getTimeNanoSec(
-        Timestamp.valueOf("2013-11-05 00:00:00.000000000"));
+
+    VectorizedRowBatch vrb = new VectorizedRowBatch(1);
+    vrb.cols[0] = new TimestampColumnVector();
+
+    TimestampColumnVector lcv0 = (TimestampColumnVector) vrb.cols[0];
+    Timestamp startTS = new Timestamp(0); // the epoch
+    Timestamp endTS = Timestamp.valueOf("2013-11-05 00:00:00.000000000");
 
     Timestamp ts0 = Timestamp.valueOf("1963-11-06 00:00:00.000");
-    lcv0.vector[0] = TimestampUtils.getTimeNanoSec(ts0);
+    lcv0.set(0, ts0);
     Timestamp ts1 = Timestamp.valueOf("1983-11-06 00:00:00.000");
-    lcv0.vector[1] = TimestampUtils.getTimeNanoSec(ts1);
+    lcv0.set(1, ts1);
     Timestamp ts2 = Timestamp.valueOf("2099-11-06 00:00:00.000");
-    lcv0.vector[2] = TimestampUtils.getTimeNanoSec(ts2);
+    lcv0.set(2, ts2);
     vrb.size = 3;
 
-    VectorExpression expr1 = new FilterLongColumnBetween(0, startTS, endTS);
+    VectorExpression expr1 = new FilterTimestampColumnBetween(0, startTS, endTS);
     expr1.evaluate(vrb);
     assertEquals(1, vrb.size);
     assertEquals(true, vrb.selectedInUse);
@@ -611,24 +614,22 @@ public class TestVectorFilterExpressions {
 
   @Test
   public void testFilterTimestampNotBetween() {
-    int seed = 17;
-    VectorizedRowBatch vrb = VectorizedRowGroupGenUtil.getVectorizedRowBatch(
-        5, 2, seed);
-    LongColumnVector lcv0 = (LongColumnVector) vrb.cols[0];
-    long startTS = TimestampUtils.getTimeNanoSec(
-        Timestamp.valueOf("2013-11-05 00:00:00.000000000"));
-    long endTS = TimestampUtils.getTimeNanoSec(
-        Timestamp.valueOf("2013-11-05 00:00:00.000000010"));
+    VectorizedRowBatch vrb = new VectorizedRowBatch(1);
+    vrb.cols[0] = new TimestampColumnVector();
+
+    TimestampColumnVector lcv0 = (TimestampColumnVector) vrb.cols[0];
+    Timestamp startTS = Timestamp.valueOf("2013-11-05 00:00:00.000000000");
+    Timestamp endTS = Timestamp.valueOf("2013-11-05 00:00:00.000000010");
 
     Timestamp ts0 = Timestamp.valueOf("2013-11-04 00:00:00.000000000");
-    lcv0.vector[0] = TimestampUtils.getTimeNanoSec(ts0);
+    lcv0.set(0, ts0);
     Timestamp ts1 = Timestamp.valueOf("2013-11-05 00:00:00.000000002");
-    lcv0.vector[1] = TimestampUtils.getTimeNanoSec(ts1);
+    lcv0.set(1, ts1);
     Timestamp ts2 = Timestamp.valueOf("2099-11-06 00:00:00.000");
-    lcv0.vector[2] = TimestampUtils.getTimeNanoSec(ts2);
+    lcv0.set(2, ts2);
     vrb.size = 3;
 
-    VectorExpression expr1 = new FilterLongColumnNotBetween(0, startTS, endTS);
+    VectorExpression expr1 = new FilterTimestampColumnNotBetween(0, startTS, endTS);
     expr1.evaluate(vrb);
     assertEquals(2, vrb.size);
     assertEquals(true, vrb.selectedInUse);

http://git-wip-us.apache.org/repos/asf/hive/blob/4a479d0c/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorGenericDateExpressions.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorGenericDateExpressions.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorGenericDateExpressions.java
index 74f4671..3f2b031 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorGenericDateExpressions.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorGenericDateExpressions.java
@@ -22,6 +22,7 @@ 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.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.TestVectorizedRowBatch;
+import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.serde2.io.DateWritable;
 import org.junit.Assert;
@@ -29,6 +30,7 @@ import org.junit.Test;
 
 import java.io.UnsupportedEncodingException;
 import java.sql.Date;
+import java.sql.Timestamp;
 import java.text.SimpleDateFormat;
 import java.util.Arrays;
 import java.util.List;
@@ -53,21 +55,21 @@ public class TestVectorGenericDateExpressions {
     return vector;
   }
 
-  private LongColumnVector toTimestamp(LongColumnVector date) {
-    LongColumnVector vector = new LongColumnVector(size);
+  private TimestampColumnVector toTimestamp(LongColumnVector date) {
+    TimestampColumnVector vector = new TimestampColumnVector(size);
     for (int i = 0; i < size; i++) {
       if (date.isNull[i]) {
         vector.isNull[i] = true;
         vector.noNulls = false;
       } else {
-        vector.vector[i] = toTimestamp(date.vector[i]);
+        vector.set(i, toTimestamp(date.vector[i]));
       }
     }
     return vector;
   }
 
-  private long toTimestamp(long date) {
-    return DateWritable.daysToMillis((int) date) * 1000000;
+  private Timestamp toTimestamp(long date) {
+    return new Timestamp(DateWritable.daysToMillis((int) date));
   }
 
   private BytesColumnVector toString(LongColumnVector date) {
@@ -474,7 +476,7 @@ public class TestVectorGenericDateExpressions {
     }
     VectorizedRowBatch batch = new VectorizedRowBatch(2, 1);
 
-    udf = new VectorUDFDateDiffScalarCol(0, 0, 1);
+    udf = new VectorUDFDateDiffScalarCol(new Timestamp(0), 0, 1);
     udf.setInputTypes(VectorExpression.Type.TIMESTAMP, VectorExpression.Type.STRING);
     batch.cols[0] = new BytesColumnVector(1);
     batch.cols[1] = new LongColumnVector(1);
@@ -615,7 +617,7 @@ public class TestVectorGenericDateExpressions {
 
     udf.setInputTypes(VectorExpression.Type.STRING, VectorExpression.Type.TIMESTAMP);
     batch.cols[0] = new BytesColumnVector(1);
-    batch.cols[1] = new LongColumnVector(1);
+    batch.cols[1] = new TimestampColumnVector(1);
     batch.cols[2] = new LongColumnVector(1);
     bcv = (BytesColumnVector) batch.cols[0];
     bcv.vector[0] = bytes;
@@ -625,7 +627,7 @@ public class TestVectorGenericDateExpressions {
     Assert.assertEquals(batch.cols[2].isNull[0], true);
 
     udf.setInputTypes(VectorExpression.Type.TIMESTAMP, VectorExpression.Type.STRING);
-    batch.cols[0] = new LongColumnVector(1);
+    batch.cols[0] = new TimestampColumnVector(1);
     batch.cols[1] = new BytesColumnVector(1);
     batch.cols[2] = new LongColumnVector(1);
     bcv = (BytesColumnVector) batch.cols[1];
@@ -640,6 +642,8 @@ public class TestVectorGenericDateExpressions {
     VectorExpression udf;
     if (colType == VectorExpression.Type.STRING) {
       udf = new VectorUDFDateString(0, 1);
+    } else if (colType == VectorExpression.Type.TIMESTAMP) {
+      udf = new VectorUDFDateTimestamp(0, 1);
     } else {
       udf = new VectorUDFDateLong(0, 1);
     }
@@ -708,6 +712,8 @@ public class TestVectorGenericDateExpressions {
         colType == VectorExpression.Type.CHAR ||
         colType == VectorExpression.Type.VARCHAR) {
       udf = new CastStringToDate(0, 1);
+    } else if (colType == VectorExpression.Type.TIMESTAMP) {
+      udf = new CastTimestampToDate(0, 1);
     } else {
       udf = new CastLongToDate(0, 1);
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/4a479d0c/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 77365a8..c14eb4a 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
@@ -23,9 +23,11 @@ import java.util.Arrays;
 
 import junit.framework.Assert;
 
+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.DoubleColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.FuncACosDoubleToDouble;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.FuncASinDoubleToDouble;
@@ -130,6 +132,27 @@ public class TestVectorMathFunctions {
     return batch;
   }
 
+  public static VectorizedRowBatch getVectorizedRowBatchDoubleInTimestampOut() {
+    VectorizedRowBatch batch = new VectorizedRowBatch(2);
+    TimestampColumnVector tcv;
+    DoubleColumnVector dcv;
+    tcv = new TimestampColumnVector();
+    dcv = new DoubleColumnVector();
+    dcv.vector[0] = -1.5d;
+    dcv.vector[1] = -0.5d;
+    dcv.vector[2] = -0.1d;
+    dcv.vector[3] = 0d;
+    dcv.vector[4] = 0.5d;
+    dcv.vector[5] = 0.7d;
+    dcv.vector[6] = 1.5d;
+
+    batch.cols[0] = dcv;
+    batch.cols[1] = tcv;
+
+    batch.size = 7;
+    return batch;
+  }
+
   public static VectorizedRowBatch getVectorizedRowBatchDoubleInDoubleOut() {
     VectorizedRowBatch batch = new VectorizedRowBatch(2);
     DoubleColumnVector inV;
@@ -171,6 +194,25 @@ public class TestVectorMathFunctions {
     return batch;
   }
 
+  public static VectorizedRowBatch getVectorizedRowBatchTimestampInDoubleOut() {
+    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));
+
+    batch.cols[0] = tcv;
+    batch.cols[1] = dcv;
+
+    batch.size = 5;
+    return batch;
+  }
+
   public static VectorizedRowBatch getVectorizedRowBatchLongInLongOut() {
     VectorizedRowBatch batch = new VectorizedRowBatch(2);
     LongColumnVector inV, outV;
@@ -186,6 +228,38 @@ public class TestVectorMathFunctions {
     return batch;
   }
 
+  public static VectorizedRowBatch getVectorizedRowBatchTimestampInLongOut() {
+    VectorizedRowBatch batch = new VectorizedRowBatch(2);
+    TimestampColumnVector inV;
+    LongColumnVector outV;
+    inV = new TimestampColumnVector();
+    outV = new LongColumnVector();
+    inV.setTimestampSeconds(0, 2);
+    inV.setTimestampSeconds(1, 2);
+
+    batch.cols[0] = inV;
+    batch.cols[1] = outV;
+
+    batch.size = 2;
+    return batch;
+  }
+
+  public static VectorizedRowBatch getVectorizedRowBatchLongInTimestampOut() {
+    VectorizedRowBatch batch = new VectorizedRowBatch(2);
+    LongColumnVector inV;
+    TimestampColumnVector outV;
+    inV = new LongColumnVector();
+    outV = new TimestampColumnVector();
+    inV.vector[0] = -2;
+    inV.vector[1] = 2;
+
+    batch.cols[0] = inV;
+    batch.cols[1] = outV;
+
+    batch.size = 2;
+    return batch;
+  }
+
   public static VectorizedRowBatch getBatchForStringMath() {
     VectorizedRowBatch batch = new VectorizedRowBatch(3);
     LongColumnVector inL;

http://git-wip-us.apache.org/repos/asf/hive/blob/4a479d0c/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 4a4ce27..375f369 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,10 +32,13 @@ 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;
 import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.TestVectorizedRowBatch;
+import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.udf.UDFDayOfMonth;
 import org.apache.hadoop.hive.ql.udf.UDFHour;
@@ -56,51 +59,42 @@ import org.junit.Test;
 public class TestVectorTimestampExpressions {
   private SimpleDateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
 
-  /* copied over from VectorUDFTimestampFieldLong */
-  private TimestampWritable toTimestampWritable(long nanos) {
-    long ms = (nanos / (1000 * 1000 * 1000)) * 1000;
-    /* the milliseconds should be kept in nanos */
-    long ns = nanos % (1000*1000*1000);
-    if (ns < 0) {
-      /*
-       * The nano seconds are always positive,
-       * but the milliseconds can be negative
-       */
-      ms -= 1000;
-      ns += 1000*1000*1000;
-    }
-    Timestamp ts = new Timestamp(ms);
-    ts.setNanos((int) ns);
-    return new TimestampWritable(ts);
-  }
-
-  private long[] getAllBoundaries() {
-    List<Long> boundaries = new ArrayList<Long>(1);
+  private Timestamp[] getAllBoundaries(int minYear, int maxYear) {
+     ArrayList<Timestamp> boundaries = new ArrayList<Timestamp>(1);
     Calendar c = Calendar.getInstance();
     c.setTimeInMillis(0); // c.set doesn't reset millis
-    for (int year = 1902; year <= 2038; year++) {
+    for (int year = minYear; year <= maxYear; year++) {
       c.set(year, Calendar.JANUARY, 1, 0, 0, 0);
-      long exactly = c.getTimeInMillis() * 1000 * 1000;
+      if (c.get(Calendar.YEAR) < 0 || c.get(Calendar.YEAR) >= 10000) {
+        continue;
+      }
+      long exactly = c.getTimeInMillis();
       /* one second before and after */
-      long before = exactly - 1000 * 1000 * 1000;
-      long after = exactly + 1000 * 1000 * 1000;
-      boundaries.add(Long.valueOf(before));
-      boundaries.add(Long.valueOf(exactly));
-      boundaries.add(Long.valueOf(after));
+      long before = exactly - 1000;
+      long after = exactly + 1000;
+      if (minYear != 0) {
+        boundaries.add(new Timestamp(before));
+      }
+      boundaries.add(new Timestamp(exactly));
+      if (year != maxYear) {
+        boundaries.add(new Timestamp(after));
+      }
     }
-    Long[] indices = boundaries.toArray(new Long[1]);
-    return ArrayUtils.toPrimitive(indices);
+    return boundaries.toArray(new Timestamp[0]);
   }
 
-  private VectorizedRowBatch getVectorizedRandomRowBatchLong2(int seed, int size) {
+  private Timestamp[] getAllBoundaries() {
+    return getAllBoundaries(0000, 9999);
+  }
+
+  private VectorizedRowBatch getVectorizedRandomRowBatchTimestampLong(int seed, int size) {
     VectorizedRowBatch batch = new VectorizedRowBatch(2, size);
-    LongColumnVector lcv = new LongColumnVector(size);
+    TimestampColumnVector tcv = new TimestampColumnVector(size);
     Random rand = new Random(seed);
     for (int i = 0; i < size; i++) {
-      /* all 32 bit numbers qualify & multiply up to get nano-seconds */
-      lcv.vector[i] = (long)(1000*1000*1000*rand.nextInt());
+      tcv.set(i, RandomTypeUtil.getRandTimestamp(rand));
     }
-    batch.cols[0] = lcv;
+    batch.cols[0] = tcv;
     batch.cols[1] = new LongColumnVector(size);
     batch.size = size;
     return batch;
@@ -112,7 +106,7 @@ public class TestVectorTimestampExpressions {
     Random rand = new Random(seed);
     for (int i = 0; i < size; i++) {
       /* all 32 bit numbers qualify & multiply up to get nano-seconds */
-      byte[] encoded = encodeTime(1000 * 1000 * 1000 * rand.nextInt());
+      byte[] encoded = encodeTime(RandomTypeUtil.getRandTimestamp(rand));
       bcv.vector[i] = encoded;
       bcv.start[i] = 0;
       bcv.length[i] = encoded.length;
@@ -125,8 +119,8 @@ public class TestVectorTimestampExpressions {
 
   private VectorizedRowBatch getVectorizedRandomRowBatch(int seed, int size, TestType testType) {
     switch (testType) {
-      case LONG2:
-        return getVectorizedRandomRowBatchLong2(seed, size);
+      case TIMESTAMP_LONG:
+        return getVectorizedRandomRowBatchTimestampLong(seed, size);
       case STRING_LONG:
         return getVectorizedRandomRowBatchStringLong(seed, size);
       default:
@@ -137,13 +131,13 @@ public class TestVectorTimestampExpressions {
   /*
    * Input array is used to fill the entire size of the vector row batch
    */
-  private VectorizedRowBatch getVectorizedRowBatchLong2(long[] inputs, int size) {
+  private VectorizedRowBatch getVectorizedRowBatchTimestampLong(Timestamp[] inputs, int size) {
     VectorizedRowBatch batch = new VectorizedRowBatch(2, size);
-    LongColumnVector lcv = new LongColumnVector(size);
+    TimestampColumnVector tcv = new TimestampColumnVector(size);
     for (int i = 0; i < size; i++) {
-      lcv.vector[i] = inputs[i % inputs.length];
+      tcv.set(i, inputs[i % inputs.length]);
     }
-    batch.cols[0] = lcv;
+    batch.cols[0] = tcv;
     batch.cols[1] = new LongColumnVector(size);
     batch.size = size;
     return batch;
@@ -152,7 +146,7 @@ public class TestVectorTimestampExpressions {
   /*
    * Input array is used to fill the entire size of the vector row batch
    */
-  private VectorizedRowBatch getVectorizedRowBatchStringLong(long[] inputs, int size) {
+  private VectorizedRowBatch getVectorizedRowBatchStringLong(Timestamp[] inputs, int size) {
     VectorizedRowBatch batch = new VectorizedRowBatch(2, size);
     BytesColumnVector bcv = new BytesColumnVector(size);
     for (int i = 0; i < size; i++) {
@@ -181,10 +175,10 @@ public class TestVectorTimestampExpressions {
     return batch;
   }
 
-  private VectorizedRowBatch getVectorizedRowBatch(long[] inputs, int size, TestType testType) {
+  private VectorizedRowBatch getVectorizedRowBatch(Timestamp[] inputs, int size, TestType testType) {
     switch (testType) {
-      case LONG2:
-        return getVectorizedRowBatchLong2(inputs, size);
+      case TIMESTAMP_LONG:
+        return getVectorizedRowBatchTimestampLong(inputs, size);
       case STRING_LONG:
         return getVectorizedRowBatchStringLong(inputs, size);
       default:
@@ -192,10 +186,11 @@ public class TestVectorTimestampExpressions {
     }
   }
 
-  private byte[] encodeTime(long time) {
+  private byte[] encodeTime(Timestamp timestamp) {
     ByteBuffer encoded;
+    long time = timestamp.getTime();
     try {
-      String formatted = dateFormat.format(new Date(time / (1000 * 1000)));
+      String formatted = dateFormat.format(new Date(time));
       encoded = Text.encode(formatted);
     } catch (CharacterCodingException e) {
       throw new RuntimeException(e);
@@ -203,17 +198,17 @@ public class TestVectorTimestampExpressions {
     return Arrays.copyOf(encoded.array(), encoded.limit());
   }
 
-  private long decodeTime(byte[] time) {
+  private Timestamp decodeTime(byte[] time) {
     try {
-      return dateFormat.parse(Text.decode(time)).getTime() * 1000 * 1000;
+      return new Timestamp(dateFormat.parse(Text.decode(time)).getTime());
     } catch (Exception e) {
       throw new RuntimeException(e);
     }
   }
 
-  private long readVectorElementAt(ColumnVector col, int i) {
-    if (col instanceof LongColumnVector) {
-      return ((LongColumnVector) col).vector[i];
+  private Timestamp readVectorElementAt(ColumnVector col, int i) {
+    if (col instanceof TimestampColumnVector) {
+      return ((TimestampColumnVector) col).asScratchTimestamp(i);
     }
     if (col instanceof BytesColumnVector) {
       byte[] timeBytes = ((BytesColumnVector) col).vector[i];
@@ -223,20 +218,24 @@ public class TestVectorTimestampExpressions {
   }
 
   private enum TestType {
-    LONG2, STRING_LONG
+    TIMESTAMP_LONG, STRING_LONG
   }
 
-  private void compareToUDFYearLong(long t, int y) {
+  private void compareToUDFYearLong(Timestamp t, int y) {
     UDFYear udf = new UDFYear();
-    TimestampWritable tsw = toTimestampWritable(t);
+    TimestampWritable tsw = new TimestampWritable(t);
     IntWritable res = udf.evaluate(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 verifyUDFYear(VectorizedRowBatch batch, TestType testType) {
     VectorExpression udf = null;
-    if (testType == TestType.LONG2) {
-      udf = new VectorUDFYearLong(0, 1);
+    if (testType == TestType.TIMESTAMP_LONG) {
+      udf = new VectorUDFYearTimestamp(0, 1);
       udf.setInputTypes(VectorExpression.Type.TIMESTAMP);
     } else {
       udf = new VectorUDFYearString(0, 1);
@@ -251,7 +250,7 @@ public class TestVectorTimestampExpressions {
         if (!batch.cols[in].noNulls) {
           Assert.assertEquals(batch.cols[out].isNull[i], batch.cols[in].isNull[i]);
         }
-        long t = readVectorElementAt(batch.cols[in], i);
+        Timestamp t = readVectorElementAt(batch.cols[in], i);
         long y = ((LongColumnVector) batch.cols[out]).vector[i];
         compareToUDFYearLong(t, (int) y);
       } else {
@@ -261,7 +260,7 @@ public class TestVectorTimestampExpressions {
   }
 
   private void testVectorUDFYear(TestType testType) {
-    VectorizedRowBatch batch = getVectorizedRowBatch(new long[] {0},
+    VectorizedRowBatch batch = getVectorizedRowBatch(new Timestamp[] {new Timestamp(0)},
             VectorizedRowBatch.DEFAULT_SIZE, testType);
     Assert.assertTrue(((LongColumnVector) batch.cols[1]).noNulls);
     Assert.assertFalse(((LongColumnVector) batch.cols[1]).isRepeating);
@@ -269,7 +268,7 @@ public class TestVectorTimestampExpressions {
     TestVectorizedRowBatch.addRandomNulls(batch.cols[0]);
     verifyUDFYear(batch, testType);
 
-    long[] boundaries = getAllBoundaries();
+    Timestamp[] boundaries = getAllBoundaries();
     batch = getVectorizedRowBatch(boundaries, boundaries.length, testType);
     verifyUDFYear(batch, testType);
     TestVectorizedRowBatch.addRandomNulls(batch.cols[0]);
@@ -277,14 +276,14 @@ public class TestVectorTimestampExpressions {
     TestVectorizedRowBatch.addRandomNulls(batch.cols[1]);
     verifyUDFYear(batch, testType);
 
-    batch = getVectorizedRowBatch(new long[] {0}, 1, testType);
+    batch = getVectorizedRowBatch(new Timestamp[] {new Timestamp(0)}, 1, testType);
     batch.cols[0].isRepeating = true;
     verifyUDFYear(batch, testType);
     batch.cols[0].noNulls = false;
     batch.cols[0].isNull[0] = true;
     verifyUDFYear(batch, testType);
 
-    batch = getVectorizedRowBatch(new long[] {0}, 1, testType);
+    batch = getVectorizedRowBatch(new Timestamp[] {new Timestamp(0)}, 1, testType);
     batch.cols[0].isRepeating = true;
     batch.selectedInUse = true;
     batch.selected = new int[] {42};
@@ -302,8 +301,8 @@ public class TestVectorTimestampExpressions {
   }
 
   @Test
-  public void testVectorUDFYearLong() {
-    testVectorUDFYear(TestType.LONG2);
+  public void testVectorUDFYearTimestamp() {
+    testVectorUDFYear(TestType.TIMESTAMP_LONG);
   }
 
   @Test
@@ -318,17 +317,17 @@ public class TestVectorTimestampExpressions {
     Assert.assertEquals(true, lcv.isNull[0]);
   }
 
-  private void compareToUDFDayOfMonthLong(long t, int y) {
+  private void compareToUDFDayOfMonthLong(Timestamp t, int y) {
     UDFDayOfMonth udf = new UDFDayOfMonth();
-    TimestampWritable tsw = toTimestampWritable(t);
+    TimestampWritable tsw = new TimestampWritable(t);
     IntWritable res = udf.evaluate(tsw);
     Assert.assertEquals(res.get(), y);
   }
 
   private void verifyUDFDayOfMonth(VectorizedRowBatch batch, TestType testType) {
     VectorExpression udf = null;
-    if (testType == TestType.LONG2) {
-      udf = new VectorUDFDayOfMonthLong(0, 1);
+    if (testType == TestType.TIMESTAMP_LONG) {
+      udf = new VectorUDFDayOfMonthTimestamp(0, 1);
       udf.setInputTypes(VectorExpression.Type.TIMESTAMP);
     } else {
       udf = new VectorUDFDayOfMonthString(0, 1);
@@ -343,7 +342,7 @@ public class TestVectorTimestampExpressions {
         if (!batch.cols[in].noNulls) {
           Assert.assertEquals(batch.cols[out].isNull[i], batch.cols[in].isNull[i]);
         }
-        long t = readVectorElementAt(batch.cols[in], i);
+        Timestamp t = readVectorElementAt(batch.cols[in], i);
         long y = ((LongColumnVector) batch.cols[out]).vector[i];
         compareToUDFDayOfMonthLong(t, (int) y);
       } else {
@@ -353,7 +352,7 @@ public class TestVectorTimestampExpressions {
   }
 
   private void testVectorUDFDayOfMonth(TestType testType) {
-    VectorizedRowBatch batch = getVectorizedRowBatch(new long[] {0},
+    VectorizedRowBatch batch = getVectorizedRowBatch(new Timestamp[] {new Timestamp(0)},
             VectorizedRowBatch.DEFAULT_SIZE, testType);
     Assert.assertTrue(((LongColumnVector) batch.cols[1]).noNulls);
     Assert.assertFalse(((LongColumnVector) batch.cols[1]).isRepeating);
@@ -361,7 +360,7 @@ public class TestVectorTimestampExpressions {
     TestVectorizedRowBatch.addRandomNulls(batch.cols[0]);
     verifyUDFDayOfMonth(batch, testType);
 
-    long[] boundaries = getAllBoundaries();
+    Timestamp[] boundaries = getAllBoundaries();
     batch = getVectorizedRowBatch(boundaries, boundaries.length, testType);
     verifyUDFDayOfMonth(batch, testType);
     TestVectorizedRowBatch.addRandomNulls(batch.cols[0]);
@@ -369,14 +368,14 @@ public class TestVectorTimestampExpressions {
     TestVectorizedRowBatch.addRandomNulls(batch.cols[1]);
     verifyUDFDayOfMonth(batch, testType);
 
-    batch = getVectorizedRowBatch(new long[] {0}, 1, testType);
+    batch = getVectorizedRowBatch(new Timestamp[] {new Timestamp(0)}, 1, testType);
     batch.cols[0].isRepeating = true;
     verifyUDFDayOfMonth(batch, testType);
     batch.cols[0].noNulls = false;
     batch.cols[0].isNull[0] = true;
     verifyUDFDayOfMonth(batch, testType);
 
-    batch = getVectorizedRowBatch(new long[] {0}, 1, testType);
+    batch = getVectorizedRowBatch(new Timestamp[] {new Timestamp(0)}, 1, testType);
     batch.cols[0].isRepeating = true;
     batch.selectedInUse = true;
     batch.selected = new int[] {42};
@@ -394,8 +393,8 @@ public class TestVectorTimestampExpressions {
   }
 
   @Test
-  public void testVectorUDFDayOfMonthLong() {
-    testVectorUDFDayOfMonth(TestType.LONG2);
+  public void testVectorUDFDayOfMonthTimestamp() {
+    testVectorUDFDayOfMonth(TestType.TIMESTAMP_LONG);
   }
 
   @Test
@@ -403,17 +402,17 @@ public class TestVectorTimestampExpressions {
     testVectorUDFDayOfMonth(TestType.STRING_LONG);
   }
 
-  private void compareToUDFHourLong(long t, int y) {
+  private void compareToUDFHourLong(Timestamp t, int y) {
     UDFHour udf = new UDFHour();
-    TimestampWritable tsw = toTimestampWritable(t);
+    TimestampWritable tsw = new TimestampWritable(t);
     IntWritable res = udf.evaluate(tsw);
     Assert.assertEquals(res.get(), y);
   }
 
   private void verifyUDFHour(VectorizedRowBatch batch, TestType testType) {
     VectorExpression udf = null;
-    if (testType == TestType.LONG2) {
-      udf = new VectorUDFHourLong(0, 1);
+    if (testType == TestType.TIMESTAMP_LONG) {
+      udf = new VectorUDFHourTimestamp(0, 1);
       udf.setInputTypes(VectorExpression.Type.TIMESTAMP);
     } else {
       udf = new VectorUDFHourString(0, 1);
@@ -428,7 +427,7 @@ public class TestVectorTimestampExpressions {
         if (!batch.cols[in].noNulls) {
           Assert.assertEquals(batch.cols[out].isNull[i], batch.cols[in].isNull[i]);
         }
-        long t = readVectorElementAt(batch.cols[in], i);
+        Timestamp t = readVectorElementAt(batch.cols[in], i);
         long y = ((LongColumnVector) batch.cols[out]).vector[i];
         compareToUDFHourLong(t, (int) y);
       } else {
@@ -438,7 +437,7 @@ public class TestVectorTimestampExpressions {
   }
 
   private void testVectorUDFHour(TestType testType) {
-    VectorizedRowBatch batch = getVectorizedRowBatch(new long[] {0},
+    VectorizedRowBatch batch = getVectorizedRowBatch(new Timestamp[] {new Timestamp(0)},
             VectorizedRowBatch.DEFAULT_SIZE, testType);
     Assert.assertTrue(((LongColumnVector) batch.cols[1]).noNulls);
     Assert.assertFalse(((LongColumnVector) batch.cols[1]).isRepeating);
@@ -446,7 +445,7 @@ public class TestVectorTimestampExpressions {
     TestVectorizedRowBatch.addRandomNulls(batch.cols[0]);
     verifyUDFHour(batch, testType);
 
-    long[] boundaries = getAllBoundaries();
+    Timestamp[] boundaries = getAllBoundaries();
     batch = getVectorizedRowBatch(boundaries, boundaries.length, testType);
     verifyUDFHour(batch, testType);
     TestVectorizedRowBatch.addRandomNulls(batch.cols[0]);
@@ -454,14 +453,14 @@ public class TestVectorTimestampExpressions {
     TestVectorizedRowBatch.addRandomNulls(batch.cols[1]);
     verifyUDFHour(batch, testType);
 
-    batch = getVectorizedRowBatch(new long[] {0}, 1, testType);
+    batch = getVectorizedRowBatch(new Timestamp[] {new Timestamp(0)}, 1, testType);
     batch.cols[0].isRepeating = true;
     verifyUDFHour(batch, testType);
     batch.cols[0].noNulls = false;
     batch.cols[0].isNull[0] = true;
     verifyUDFHour(batch, testType);
 
-    batch = getVectorizedRowBatch(new long[] {0}, 1, testType);
+    batch = getVectorizedRowBatch(new Timestamp[] {new Timestamp(0)}, 1, testType);
     batch.cols[0].isRepeating = true;
     batch.selectedInUse = true;
     batch.selected = new int[] {42};
@@ -479,8 +478,8 @@ public class TestVectorTimestampExpressions {
   }
 
   @Test
-  public void testVectorUDFHourLong() {
-    testVectorUDFHour(TestType.LONG2);
+  public void testVectorUDFHourTimestamp() {
+    testVectorUDFHour(TestType.TIMESTAMP_LONG);
   }
 
   @Test
@@ -488,17 +487,17 @@ public class TestVectorTimestampExpressions {
     testVectorUDFHour(TestType.STRING_LONG);
   }
 
-  private void compareToUDFMinuteLong(long t, int y) {
+  private void compareToUDFMinuteLong(Timestamp t, int y) {
     UDFMinute udf = new UDFMinute();
-    TimestampWritable tsw = toTimestampWritable(t);
+    TimestampWritable tsw = new TimestampWritable(t);
     IntWritable res = udf.evaluate(tsw);
     Assert.assertEquals(res.get(), y);
   }
 
   private void verifyUDFMinute(VectorizedRowBatch batch, TestType testType) {
     VectorExpression udf = null;
-    if (testType == TestType.LONG2) {
-      udf = new VectorUDFMinuteLong(0, 1);
+    if (testType == TestType.TIMESTAMP_LONG) {
+      udf = new VectorUDFMinuteTimestamp(0, 1);
       udf.setInputTypes(VectorExpression.Type.TIMESTAMP);
     } else {
       udf = new VectorUDFMinuteString(0, 1);
@@ -513,7 +512,7 @@ public class TestVectorTimestampExpressions {
         if (!batch.cols[in].noNulls) {
           Assert.assertEquals(batch.cols[out].isNull[i], batch.cols[in].isNull[i]);
         }
-        long t = readVectorElementAt(batch.cols[in], i);
+        Timestamp t = readVectorElementAt(batch.cols[in], i);
         long y = ((LongColumnVector) batch.cols[out]).vector[i];
         compareToUDFMinuteLong(t, (int) y);
       } else {
@@ -523,7 +522,7 @@ public class TestVectorTimestampExpressions {
   }
 
   private void testVectorUDFMinute(TestType testType) {
-    VectorizedRowBatch batch = getVectorizedRowBatch(new long[] {0},
+    VectorizedRowBatch batch = getVectorizedRowBatch(new Timestamp[] {new Timestamp(0)},
             VectorizedRowBatch.DEFAULT_SIZE, testType);
     Assert.assertTrue(((LongColumnVector) batch.cols[1]).noNulls);
     Assert.assertFalse(((LongColumnVector) batch.cols[1]).isRepeating);
@@ -531,7 +530,7 @@ public class TestVectorTimestampExpressions {
     TestVectorizedRowBatch.addRandomNulls(batch.cols[0]);
     verifyUDFMinute(batch, testType);
 
-    long[] boundaries = getAllBoundaries();
+    Timestamp[] boundaries = getAllBoundaries();
     batch = getVectorizedRowBatch(boundaries, boundaries.length, testType);
     verifyUDFMinute(batch, testType);
     TestVectorizedRowBatch.addRandomNulls(batch.cols[0]);
@@ -539,14 +538,14 @@ public class TestVectorTimestampExpressions {
     TestVectorizedRowBatch.addRandomNulls(batch.cols[1]);
     verifyUDFMinute(batch, testType);
 
-    batch = getVectorizedRowBatch(new long[] {0}, 1, testType);
+    batch = getVectorizedRowBatch(new Timestamp[] {new Timestamp(0)}, 1, testType);
     batch.cols[0].isRepeating = true;
     verifyUDFMinute(batch, testType);
     batch.cols[0].noNulls = false;
     batch.cols[0].isNull[0] = true;
     verifyUDFMinute(batch, testType);
 
-    batch = getVectorizedRowBatch(new long[] {0}, 1, testType);
+    batch = getVectorizedRowBatch(new Timestamp[] {new Timestamp(0)}, 1, testType);
     batch.cols[0].isRepeating = true;
     batch.selectedInUse = true;
     batch.selected = new int[] {42};
@@ -565,7 +564,7 @@ public class TestVectorTimestampExpressions {
 
   @Test
   public void testVectorUDFMinuteLong() {
-    testVectorUDFMinute(TestType.LONG2);
+    testVectorUDFMinute(TestType.TIMESTAMP_LONG);
   }
 
   @Test
@@ -573,17 +572,17 @@ public class TestVectorTimestampExpressions {
     testVectorUDFMinute(TestType.STRING_LONG);
   }
 
-  private void compareToUDFMonthLong(long t, int y) {
+  private void compareToUDFMonthLong(Timestamp t, int y) {
     UDFMonth udf = new UDFMonth();
-    TimestampWritable tsw = toTimestampWritable(t);
+    TimestampWritable tsw = new TimestampWritable(t);
     IntWritable res = udf.evaluate(tsw);
     Assert.assertEquals(res.get(), y);
   }
 
   private void verifyUDFMonth(VectorizedRowBatch batch, TestType testType) {
     VectorExpression udf;
-    if (testType == TestType.LONG2) {
-      udf = new VectorUDFMonthLong(0, 1);
+    if (testType == TestType.TIMESTAMP_LONG) {
+      udf = new VectorUDFMonthTimestamp(0, 1);
       udf.setInputTypes(VectorExpression.Type.TIMESTAMP);
     } else {
       udf = new VectorUDFMonthString(0, 1);
@@ -598,7 +597,7 @@ public class TestVectorTimestampExpressions {
         if (!batch.cols[in].noNulls) {
           Assert.assertEquals(batch.cols[out].isNull[i], batch.cols[in].isNull[i]);
         }
-        long t = readVectorElementAt(batch.cols[in], i);
+        Timestamp t = readVectorElementAt(batch.cols[in], i);
         long y = ((LongColumnVector) batch.cols[out]).vector[i];
         compareToUDFMonthLong(t, (int) y);
       } else {
@@ -608,7 +607,7 @@ public class TestVectorTimestampExpressions {
   }
 
   private void testVectorUDFMonth(TestType testType) {
-    VectorizedRowBatch batch = getVectorizedRowBatch(new long[] {0},
+    VectorizedRowBatch batch = getVectorizedRowBatch(new Timestamp[] {new Timestamp(0)},
             VectorizedRowBatch.DEFAULT_SIZE, testType);
     Assert.assertTrue(((LongColumnVector) batch.cols[1]).noNulls);
     Assert.assertFalse(((LongColumnVector) batch.cols[1]).isRepeating);
@@ -616,7 +615,7 @@ public class TestVectorTimestampExpressions {
     TestVectorizedRowBatch.addRandomNulls(batch.cols[0]);
     verifyUDFMonth(batch, testType);
 
-    long[] boundaries = getAllBoundaries();
+    Timestamp[] boundaries = getAllBoundaries();
     batch = getVectorizedRowBatch(boundaries, boundaries.length, testType);
     verifyUDFMonth(batch, testType);
     TestVectorizedRowBatch.addRandomNulls(batch.cols[0]);
@@ -624,14 +623,14 @@ public class TestVectorTimestampExpressions {
     TestVectorizedRowBatch.addRandomNulls(batch.cols[1]);
     verifyUDFMonth(batch, testType);
 
-    batch = getVectorizedRowBatch(new long[] {0}, 1, testType);
+    batch = getVectorizedRowBatch(new Timestamp[] {new Timestamp(0)}, 1, testType);
     batch.cols[0].isRepeating = true;
     verifyUDFMonth(batch, testType);
     batch.cols[0].noNulls = false;
     batch.cols[0].isNull[0] = true;
     verifyUDFMonth(batch, testType);
 
-    batch = getVectorizedRowBatch(new long[] {0}, 1, testType);
+    batch = getVectorizedRowBatch(new Timestamp[] {new Timestamp(0)}, 1, testType);
     batch.cols[0].isRepeating = true;
     batch.selectedInUse = true;
     batch.selected = new int[] {42};
@@ -649,8 +648,8 @@ public class TestVectorTimestampExpressions {
   }
 
   @Test
-  public void testVectorUDFMonthLong() {
-    testVectorUDFMonth(TestType.LONG2);
+  public void testVectorUDFMonthTimestamp() {
+    testVectorUDFMonth(TestType.TIMESTAMP_LONG);
   }
 
   @Test
@@ -658,17 +657,17 @@ public class TestVectorTimestampExpressions {
     testVectorUDFMonth(TestType.STRING_LONG);
   }
 
-  private void compareToUDFSecondLong(long t, int y) {
+  private void compareToUDFSecondLong(Timestamp t, int y) {
     UDFSecond udf = new UDFSecond();
-    TimestampWritable tsw = toTimestampWritable(t);
+    TimestampWritable tsw = new TimestampWritable(t);
     IntWritable res = udf.evaluate(tsw);
     Assert.assertEquals(res.get(), y);
   }
 
   private void verifyUDFSecond(VectorizedRowBatch batch, TestType testType) {
     VectorExpression udf;
-    if (testType == TestType.LONG2) {
-      udf = new VectorUDFSecondLong(0, 1);
+    if (testType == TestType.TIMESTAMP_LONG) {
+      udf = new VectorUDFSecondTimestamp(0, 1);
       udf.setInputTypes(VectorExpression.Type.TIMESTAMP);
     } else {
       udf = new VectorUDFSecondString(0, 1);
@@ -683,7 +682,7 @@ public class TestVectorTimestampExpressions {
         if (!batch.cols[in].noNulls) {
           Assert.assertEquals(batch.cols[out].isNull[i], batch.cols[in].isNull[i]);
         }
-        long t = readVectorElementAt(batch.cols[in], i);
+        Timestamp t = readVectorElementAt(batch.cols[in], i);
         long y = ((LongColumnVector) batch.cols[out]).vector[i];
         compareToUDFSecondLong(t, (int) y);
       } else {
@@ -693,7 +692,7 @@ public class TestVectorTimestampExpressions {
   }
 
   private void testVectorUDFSecond(TestType testType) {
-    VectorizedRowBatch batch = getVectorizedRowBatch(new long[] {0},
+    VectorizedRowBatch batch = getVectorizedRowBatch(new Timestamp[] {new Timestamp(0)},
             VectorizedRowBatch.DEFAULT_SIZE, testType);
     Assert.assertTrue(((LongColumnVector) batch.cols[1]).noNulls);
     Assert.assertFalse(((LongColumnVector) batch.cols[1]).isRepeating);
@@ -701,7 +700,7 @@ public class TestVectorTimestampExpressions {
     TestVectorizedRowBatch.addRandomNulls(batch.cols[0]);
     verifyUDFSecond(batch, testType);
 
-    long[] boundaries = getAllBoundaries();
+    Timestamp[] boundaries = getAllBoundaries();
     batch = getVectorizedRowBatch(boundaries, boundaries.length, testType);
     verifyUDFSecond(batch, testType);
     TestVectorizedRowBatch.addRandomNulls(batch.cols[0]);
@@ -709,14 +708,14 @@ public class TestVectorTimestampExpressions {
     TestVectorizedRowBatch.addRandomNulls(batch.cols[1]);
     verifyUDFSecond(batch, testType);
 
-    batch = getVectorizedRowBatch(new long[] {0}, 1, testType);
+    batch = getVectorizedRowBatch(new Timestamp[] {new Timestamp(0)}, 1, testType);
     batch.cols[0].isRepeating = true;
     verifyUDFSecond(batch, testType);
     batch.cols[0].noNulls = false;
     batch.cols[0].isNull[0] = true;
     verifyUDFSecond(batch, testType);
 
-    batch = getVectorizedRowBatch(new long[] {0}, 1, testType);
+    batch = getVectorizedRowBatch(new Timestamp[] {new Timestamp(0)}, 1, testType);
     batch.cols[0].isRepeating = true;
     batch.selectedInUse = true;
     batch.selected = new int[] {42};
@@ -735,7 +734,7 @@ public class TestVectorTimestampExpressions {
 
   @Test
   public void testVectorUDFSecondLong() {
-    testVectorUDFSecond(TestType.LONG2);
+    testVectorUDFSecond(TestType.TIMESTAMP_LONG);
   }
 
   @Test
@@ -753,11 +752,11 @@ public class TestVectorTimestampExpressions {
     }
   }
 
-  private void compareToUDFUnixTimeStampLong(long t, long y) {
-    TimestampWritable tsw = toTimestampWritable(t);
+  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 %d, %d\n", res.get(), y, t,
+      System.out.printf("%d vs %d for %s, %d\n", res.get(), y, t.toString(),
           tsw.getTimestamp().getTime()/1000);
     }
 
@@ -766,8 +765,8 @@ public class TestVectorTimestampExpressions {
 
   private void verifyUDFUnixTimeStamp(VectorizedRowBatch batch, TestType testType) {
     VectorExpression udf;
-    if (testType == TestType.LONG2) {
-      udf = new VectorUDFUnixTimeStampLong(0, 1);
+    if (testType == TestType.TIMESTAMP_LONG) {
+      udf = new VectorUDFUnixTimeStampTimestamp(0, 1);
       udf.setInputTypes(VectorExpression.Type.TIMESTAMP);
     } else {
       udf = new VectorUDFUnixTimeStampString(0, 1);
@@ -782,7 +781,7 @@ public class TestVectorTimestampExpressions {
         if (!batch.cols[out].noNulls) {
           Assert.assertEquals(batch.cols[out].isNull[i], batch.cols[in].isNull[i]);
         }
-        long t = readVectorElementAt(batch.cols[in], i);
+        Timestamp t = readVectorElementAt(batch.cols[in], i);
         long y = ((LongColumnVector) batch.cols[out]).vector[i];
         compareToUDFUnixTimeStampLong(t, y);
       } else {
@@ -792,7 +791,7 @@ public class TestVectorTimestampExpressions {
   }
 
   private void testVectorUDFUnixTimeStamp(TestType testType) {
-    VectorizedRowBatch batch = getVectorizedRowBatch(new long[] {0},
+    VectorizedRowBatch batch = getVectorizedRowBatch(new Timestamp[] {new Timestamp(0)},
             VectorizedRowBatch.DEFAULT_SIZE, testType);
     Assert.assertTrue(((LongColumnVector) batch.cols[1]).noNulls);
     Assert.assertFalse(((LongColumnVector) batch.cols[1]).isRepeating);
@@ -800,7 +799,7 @@ public class TestVectorTimestampExpressions {
     TestVectorizedRowBatch.addRandomNulls(batch.cols[0]);
     verifyUDFUnixTimeStamp(batch, testType);
 
-    long[] boundaries = getAllBoundaries();
+    Timestamp[] boundaries = getAllBoundaries();
     batch = getVectorizedRowBatch(boundaries, boundaries.length, testType);
     verifyUDFUnixTimeStamp(batch, testType);
     TestVectorizedRowBatch.addRandomNulls(batch.cols[0]);
@@ -808,14 +807,14 @@ public class TestVectorTimestampExpressions {
     TestVectorizedRowBatch.addRandomNulls(batch.cols[1]);
     verifyUDFUnixTimeStamp(batch, testType);
 
-    batch = getVectorizedRowBatch(new long[] {0}, 1, testType);
+    batch = getVectorizedRowBatch(new Timestamp[] {new Timestamp(0)}, 1, testType);
     batch.cols[0].isRepeating = true;
     verifyUDFUnixTimeStamp(batch, testType);
     batch.cols[0].noNulls = false;
     batch.cols[0].isNull[0] = true;
     verifyUDFUnixTimeStamp(batch, testType);
 
-    batch = getVectorizedRowBatch(new long[] {0}, 1, testType);
+    batch = getVectorizedRowBatch(new Timestamp[] {new Timestamp(0)}, 1, testType);
     batch.cols[0].isRepeating = true;
     batch.selectedInUse = true;
     batch.selected = new int[] {42};
@@ -833,8 +832,8 @@ public class TestVectorTimestampExpressions {
   }
 
   @Test
-  public void testVectorUDFUnixTimeStampLong() {
-    testVectorUDFUnixTimeStamp(TestType.LONG2);
+  public void testVectorUDFUnixTimeStampTimestamp() {
+    testVectorUDFUnixTimeStamp(TestType.TIMESTAMP_LONG);
   }
 
   @Test
@@ -842,17 +841,17 @@ public class TestVectorTimestampExpressions {
     testVectorUDFUnixTimeStamp(TestType.STRING_LONG);
   }
 
-  private void compareToUDFWeekOfYearLong(long t, int y) {
+  private void compareToUDFWeekOfYearLong(Timestamp t, int y) {
     UDFWeekOfYear udf = new UDFWeekOfYear();
-    TimestampWritable tsw = toTimestampWritable(t);
+    TimestampWritable tsw = new TimestampWritable(t);
     IntWritable res = udf.evaluate(tsw);
     Assert.assertEquals(res.get(), y);
   }
 
   private void verifyUDFWeekOfYear(VectorizedRowBatch batch, TestType testType) {
     VectorExpression udf;
-    if (testType == TestType.LONG2) {
-      udf = new VectorUDFWeekOfYearLong(0, 1);
+    if (testType == TestType.TIMESTAMP_LONG) {
+      udf = new VectorUDFWeekOfYearTimestamp(0, 1);
       udf.setInputTypes(VectorExpression.Type.TIMESTAMP);
     } else {
       udf = new VectorUDFWeekOfYearString(0, 1);
@@ -864,7 +863,7 @@ public class TestVectorTimestampExpressions {
 
     for (int i = 0; i < batch.size; i++) {
       if (batch.cols[in].noNulls || !batch.cols[in].isNull[i]) {
-        long t = readVectorElementAt(batch.cols[in], i);
+        Timestamp t = readVectorElementAt(batch.cols[in], i);
         long y = ((LongColumnVector) batch.cols[out]).vector[i];
         compareToUDFWeekOfYearLong(t, (int) y);
       } else {
@@ -874,7 +873,7 @@ public class TestVectorTimestampExpressions {
   }
 
   private void testVectorUDFWeekOfYear(TestType testType) {
-    VectorizedRowBatch batch = getVectorizedRowBatch(new long[] {0},
+    VectorizedRowBatch batch = getVectorizedRowBatch(new Timestamp[] {new Timestamp(0)},
             VectorizedRowBatch.DEFAULT_SIZE, testType);
     Assert.assertTrue(((LongColumnVector) batch.cols[1]).noNulls);
     Assert.assertFalse(((LongColumnVector) batch.cols[1]).isRepeating);
@@ -882,7 +881,7 @@ public class TestVectorTimestampExpressions {
     TestVectorizedRowBatch.addRandomNulls(batch.cols[0]);
     verifyUDFWeekOfYear(batch, testType);
 
-    long[] boundaries = getAllBoundaries();
+    Timestamp[] boundaries = getAllBoundaries();
     batch = getVectorizedRowBatch(boundaries, boundaries.length, testType);
     verifyUDFWeekOfYear(batch, testType);
     TestVectorizedRowBatch.addRandomNulls(batch.cols[0]);
@@ -890,14 +889,14 @@ public class TestVectorTimestampExpressions {
     TestVectorizedRowBatch.addRandomNulls(batch.cols[1]);
     verifyUDFWeekOfYear(batch, testType);
 
-    batch = getVectorizedRowBatch(new long[] {0}, 1, testType);
+    batch = getVectorizedRowBatch(new Timestamp[] {new Timestamp(0)}, 1, testType);
     batch.cols[0].isRepeating = true;
     verifyUDFWeekOfYear(batch, testType);
     batch.cols[0].noNulls = false;
     batch.cols[0].isNull[0] = true;
     verifyUDFWeekOfYear(batch, testType);
 
-    batch = getVectorizedRowBatch(new long[] {0}, 1, testType);
+    batch = getVectorizedRowBatch(new Timestamp[] {new Timestamp(0)}, 1, testType);
     batch.cols[0].isRepeating = true;
     batch.selectedInUse = true;
     batch.selected = new int[] {42};
@@ -915,8 +914,8 @@ public class TestVectorTimestampExpressions {
   }
 
   @Test
-  public void testVectorUDFWeekOfYearLong() {
-    testVectorUDFWeekOfYear(TestType.LONG2);
+  public void testVectorUDFWeekOfYearTimestamp() {
+    testVectorUDFWeekOfYear(TestType.TIMESTAMP_LONG);
   }
 
   @Test
@@ -926,12 +925,13 @@ public class TestVectorTimestampExpressions {
 
   public static void main(String[] args) {
     TestVectorTimestampExpressions self = new TestVectorTimestampExpressions();
-    self.testVectorUDFYearLong();
-    self.testVectorUDFMonthLong();
-    self.testVectorUDFDayOfMonthLong();
-    self.testVectorUDFHourLong();
-    self.testVectorUDFWeekOfYearLong();
-    self.testVectorUDFUnixTimeStampLong();
+    self.testVectorUDFYearTimestamp();
+    self.testVectorUDFMonthTimestamp();
+    self.testVectorUDFDayOfMonthTimestamp();
+    self.testVectorUDFHourTimestamp();
+    self.testVectorUDFWeekOfYearTimestamp();
+    self.testVectorUDFUnixTimeStampTimestamp();
+
     self.testVectorUDFYearString();
     self.testVectorUDFMonthString();
     self.testVectorUDFDayOfMonthString();

http://git-wip-us.apache.org/repos/asf/hive/blob/4a479d0c/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 0e23680..038e382 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
@@ -29,13 +29,16 @@ 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.ql.exec.vector.BytesColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.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.junit.Test;
 
 /**
@@ -43,9 +46,6 @@ import org.junit.Test;
  */
 public class TestVectorTypeCasts {
 
-  // Number of nanoseconds in one second
-  private static final long NANOS_PER_SECOND = 1000000000;
-
   @Test
   public void testVectorCastLongToDouble() {
     VectorizedRowBatch b = TestVectorMathFunctions.getVectorizedRowBatchLongInDoubleOut();
@@ -79,13 +79,13 @@ public class TestVectorTypeCasts {
 
   @Test
   public void testCastDoubleToTimestamp() {
-    VectorizedRowBatch b = TestVectorMathFunctions.getVectorizedRowBatchDoubleInLongOut();
-    LongColumnVector resultV = (LongColumnVector) b.cols[1];
+    VectorizedRowBatch b = TestVectorMathFunctions.getVectorizedRowBatchDoubleInTimestampOut();
+    TimestampColumnVector resultV = (TimestampColumnVector) b.cols[1];
     b.cols[0].noNulls = true;
-    VectorExpression expr = new CastDoubleToTimestampViaDoubleToLong(0, 1);
+    VectorExpression expr = new CastDoubleToTimestamp(0, 1);
     expr.evaluate(b);
-    Assert.assertEquals(0, resultV.vector[3]);
-    Assert.assertEquals((long) (0.5d * NANOS_PER_SECOND), resultV.vector[4]);
+    Assert.assertEquals(0.0, resultV.getTimestampSecondsWithFractionalNanos(3));
+    Assert.assertEquals(0.5d, resultV.getTimestampSecondsWithFractionalNanos(4));
   }
 
   @Test
@@ -103,36 +103,36 @@ public class TestVectorTypeCasts {
 
   @Test
   public void testCastLongToTimestamp() {
-    VectorizedRowBatch b = TestVectorMathFunctions.getVectorizedRowBatchLongInLongOut();
-    LongColumnVector resultV = (LongColumnVector) b.cols[1];
+    VectorizedRowBatch b = TestVectorMathFunctions.getVectorizedRowBatchLongInTimestampOut();
+    TimestampColumnVector resultV = (TimestampColumnVector) b.cols[1];
     b.cols[0].noNulls = true;
-    VectorExpression expr = new CastLongToTimestampViaLongToLong(0, 1);
+    VectorExpression expr = new CastLongToTimestamp(0, 1);
     expr.evaluate(b);
-    Assert.assertEquals(-2 * NANOS_PER_SECOND, resultV.vector[0]);
-    Assert.assertEquals(2 * NANOS_PER_SECOND, resultV.vector[1]);
+    Assert.assertEquals(-2, resultV.getTimestampSeconds(0));
+    Assert.assertEquals(2, resultV.getTimestampSeconds(1));
   }
 
   @Test
   public void testCastTimestampToLong() {
-    VectorizedRowBatch b = TestVectorMathFunctions.getVectorizedRowBatchLongInLongOut();
-    LongColumnVector inV = (LongColumnVector) b.cols[0];
-    inV.vector[0] = NANOS_PER_SECOND;  // Make one entry produce interesting result
+    VectorizedRowBatch b = TestVectorMathFunctions.getVectorizedRowBatchTimestampInLongOut();
+    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 CastTimestampToLongViaLongToLong(0, 1);
+    VectorExpression expr = new CastTimestampToLong(0, 1);
     expr.evaluate(b);
     Assert.assertEquals(1, resultV.vector[0]);
   }
 
   @Test
   public void testCastTimestampToDouble() {
-    VectorizedRowBatch b = TestVectorMathFunctions.getVectorizedRowBatchLongInDoubleOut();
-    LongColumnVector inV = (LongColumnVector) b.cols[0];
+    VectorizedRowBatch b = TestVectorMathFunctions.getVectorizedRowBatchTimestampInDoubleOut();
+    TimestampColumnVector inV = (TimestampColumnVector) b.cols[0];
     DoubleColumnVector resultV = (DoubleColumnVector) b.cols[1];
     b.cols[0].noNulls = true;
-    VectorExpression expr = new CastTimestampToDoubleViaLongToDouble(0, 1);
+    VectorExpression expr = new CastTimestampToDouble(0, 1);
     expr.evaluate(b);
     Assert.assertEquals(-1E-9D , resultV.vector[1]);
     Assert.assertEquals(1E-9D, resultV.vector[3]);
@@ -356,13 +356,16 @@ public class TestVectorTypeCasts {
 
   @Test
   public void testCastDecimalToTimestamp() {
-    VectorizedRowBatch b = getBatchDecimalLong2();
+    VectorizedRowBatch b = getBatchDecimalTimestamp();
     VectorExpression expr = new CastDecimalToTimestamp(0, 1);
     expr.evaluate(b);
-    LongColumnVector r = (LongColumnVector) b.cols[1];
-    assertEquals(1111111111L, r.vector[0]);
-    assertEquals(-2222222222L, r.vector[1]);
-    assertEquals(31536000999999999L, r.vector[2]);
+    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));
   }
 
   private VectorizedRowBatch getBatchDecimalLong2() {
@@ -381,6 +384,22 @@ public class TestVectorTypeCasts {
     return b;
   }
 
+  private VectorizedRowBatch getBatchDecimalTimestamp() {
+    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));
+
+    return b;
+  }
+
   @Test
   public void testCastLongToDecimal() {
     VectorizedRowBatch b = getBatchLongDecimal();
@@ -403,6 +422,17 @@ public class TestVectorTypeCasts {
     return b;
   }
 
+  private VectorizedRowBatch getBatchTimestampDecimal() {
+    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));
+    return b;
+  }
+
   @Test
   public void testCastDoubleToDecimal() {
     VectorizedRowBatch b = getBatchDoubleDecimal();
@@ -466,10 +496,10 @@ public class TestVectorTypeCasts {
 
     // The input timestamps are stored as long values
     // measured in nanoseconds from the epoch.
-    VectorizedRowBatch b = getBatchLongDecimal();
+    VectorizedRowBatch b = getBatchTimestampDecimal();
     VectorExpression expr = new CastTimestampToDecimal(0, 1);
-    LongColumnVector inL = (LongColumnVector) b.cols[0];
-    inL.vector[1] = -1990000000L;
+    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")));
@@ -478,7 +508,7 @@ public class TestVectorTypeCasts {
 
     // Try again with a value that won't fit in 5 digits, to make
     // sure that NULL is produced.
-    b = getBatchLongDecimalPrec5Scale2();
+    b = getBatchTimestampDecimalPrec5Scale2();
     expr.evaluate(b);
     r = (DecimalColumnVector) b.cols[1];
     assertFalse(r.noNulls);
@@ -503,6 +533,17 @@ 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() {

http://git-wip-us.apache.org/repos/asf/hive/blob/4a479d0c/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 bbda9a3..ab86082 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,12 +27,13 @@ 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;
 import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.TimestampUtils;
+import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 
@@ -101,16 +102,16 @@ public class FakeVectorRowBatchFromObjectIterables extends FakeVectorRowBatchBas
           }
         };
       } else if (types[i].equalsIgnoreCase("timestamp")) {
-        batch.cols[i] = new LongColumnVector(batchSize);
+        batch.cols[i] = new TimestampColumnVector(batchSize);
         columnAssign[i] = new ColumnVectorAssign() {
           @Override
           public void assign(
               ColumnVector columnVector,
               int row,
               Object value) {
-            LongColumnVector lcv = (LongColumnVector) columnVector;
+            TimestampColumnVector lcv = (TimestampColumnVector) columnVector;
             Timestamp t = (Timestamp) value;
-            lcv.vector[row] = TimestampUtils.getTimeNanoSec(t);
+            lcv.set(row, new PisaTimestamp(t));
           }
         };
 

http://git-wip-us.apache.org/repos/asf/hive/blob/4a479d0c/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 5d79f9c..649e52b 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
@@ -18,13 +18,16 @@
 
 package org.apache.hadoop.hive.ql.exec.vector.util;
 
+import java.sql.Timestamp;
 import java.util.Random;
 
-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.DecimalColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo;
@@ -77,6 +80,30 @@ public class VectorizedRowGroupGenUtil {
     return lcv;
   }
 
+  public static TimestampColumnVector generateTimestampColumnVector(
+      boolean nulls, boolean repeating, int size, Random rand) {
+    TimestampColumnVector tcv = new TimestampColumnVector(size);
+
+    tcv.noNulls = !nulls;
+    tcv.isRepeating = repeating;
+
+    Timestamp repeatingTimestamp = RandomTypeUtil.getRandTimestamp(rand);
+
+    int nullFrequency = generateNullFrequency(rand);
+
+    for(int i = 0; i < size; i++) {
+      if(nulls && (repeating || i % nullFrequency == 0)) {
+        tcv.isNull[i] = true;
+        tcv.setNullValue(i);
+
+      }else {
+        tcv.isNull[i] = false;
+        tcv.set(i, repeating ? repeatingTimestamp : RandomTypeUtil.getRandTimestamp(rand));
+      }
+    }
+    return tcv;
+  }
+
   public static DoubleColumnVector generateDoubleColumnVector(boolean nulls,
       boolean repeating, int size, Random rand) {
     DoubleColumnVector dcv = new DoubleColumnVector(size);

http://git-wip-us.apache.org/repos/asf/hive/blob/4a479d0c/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 a438fc4..6f84708 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
@@ -60,6 +60,7 @@ 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;
 import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatchCtx;
 import org.apache.hadoop.hive.ql.io.AcidInputFormat;
@@ -1827,7 +1828,7 @@ public class TestInputOutputFormat {
     BytesColumnVector stringColumn = (BytesColumnVector) value.cols[7];
     DecimalColumnVector decimalColumn = (DecimalColumnVector) value.cols[8];
     LongColumnVector dateColumn = (LongColumnVector) value.cols[9];
-    LongColumnVector timestampColumn = (LongColumnVector) value.cols[10];
+    TimestampColumnVector timestampColumn = (TimestampColumnVector) value.cols[10];
     for(int i=0; i < 100; i++) {
       assertEquals("checking boolean " + i, i % 2 == 0 ? 1 : 0,
           booleanColumn.vector[i]);
@@ -1848,8 +1849,8 @@ public class TestInputOutputFormat {
       assertEquals("checking date " + i, i, dateColumn.vector[i]);
       long millis = (long) i * MILLIS_IN_DAY;
       millis -= LOCAL_TIMEZONE.getOffset(millis);
-      assertEquals("checking timestamp " + i, millis * 1000000L,
-          timestampColumn.vector[i]);
+      assertEquals("checking timestamp " + i, millis,
+          timestampColumn.getTimestampMilliseconds(i));
     }
     assertEquals(false, reader.next(key, value));
   }