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/19 12:13:00 UTC

[06/20] hive git commit: HIVE-9862 Vectorized execution corrupts timestamp values (Matt McCline, reviewed by Jason Dere) 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/130293e5/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 419254b..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,10 +32,12 @@ import java.util.Random;
 import junit.framework.Assert;
 
 import org.apache.commons.lang.ArrayUtils;
+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 +58,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 Timestamp[] getAllBoundaries() {
+    return getAllBoundaries(RandomTypeUtil.MIN_YEAR, RandomTypeUtil.MAX_YEAR);
   }
 
-  private VectorizedRowBatch getVectorizedRandomRowBatchLong2(int seed, int size) {
+  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 +105,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 +118,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 +130,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 +145,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 +174,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 +185,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 +197,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 +217,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 +249,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 +259,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 +267,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,8 +275,17 @@ 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 Timestamp[] {new Timestamp(0)}, 1, testType);
     batch.cols[0].isRepeating = true;
+    batch.selectedInUse = true;
+    batch.selected = new int[] {42};
     verifyUDFYear(batch, testType);
     batch.cols[0].noNulls = false;
     batch.cols[0].isNull[0] = true;
@@ -293,8 +300,8 @@ public class TestVectorTimestampExpressions {
   }
 
   @Test
-  public void testVectorUDFYearLong() {
-    testVectorUDFYear(TestType.LONG2);
+  public void testVectorUDFYearTimestamp() {
+    testVectorUDFYear(TestType.TIMESTAMP_LONG);
   }
 
   @Test
@@ -309,17 +316,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);
@@ -334,7 +341,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 {
@@ -344,7 +351,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);
@@ -352,7 +359,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]);
@@ -360,13 +367,22 @@ 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 Timestamp[] {new Timestamp(0)}, 1, testType);
+    batch.cols[0].isRepeating = true;
+    batch.selectedInUse = true;
+    batch.selected = new int[] {42};
+    verifyUDFDayOfMonth(batch, testType);
+    batch.cols[0].noNulls = false;
+    batch.cols[0].isNull[0] = true;
+    verifyUDFDayOfMonth(batch, testType);
+
     batch = getVectorizedRandomRowBatch(200, VectorizedRowBatch.DEFAULT_SIZE, testType);
     verifyUDFDayOfMonth(batch, testType);
     TestVectorizedRowBatch.addRandomNulls(batch.cols[0]);
@@ -376,8 +392,8 @@ public class TestVectorTimestampExpressions {
   }
 
   @Test
-  public void testVectorUDFDayOfMonthLong() {
-    testVectorUDFDayOfMonth(TestType.LONG2);
+  public void testVectorUDFDayOfMonthTimestamp() {
+    testVectorUDFDayOfMonth(TestType.TIMESTAMP_LONG);
   }
 
   @Test
@@ -385,17 +401,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);
@@ -410,7 +426,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 {
@@ -420,7 +436,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);
@@ -428,7 +444,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]);
@@ -436,13 +452,22 @@ 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 Timestamp[] {new Timestamp(0)}, 1, testType);
+    batch.cols[0].isRepeating = true;
+    batch.selectedInUse = true;
+    batch.selected = new int[] {42};
+    verifyUDFHour(batch, testType);
+    batch.cols[0].noNulls = false;
+    batch.cols[0].isNull[0] = true;
+    verifyUDFHour(batch, testType);
+
     batch = getVectorizedRandomRowBatch(200, VectorizedRowBatch.DEFAULT_SIZE, testType);
     verifyUDFHour(batch, testType);
     TestVectorizedRowBatch.addRandomNulls(batch.cols[0]);
@@ -452,8 +477,8 @@ public class TestVectorTimestampExpressions {
   }
 
   @Test
-  public void testVectorUDFHourLong() {
-    testVectorUDFHour(TestType.LONG2);
+  public void testVectorUDFHourTimestamp() {
+    testVectorUDFHour(TestType.TIMESTAMP_LONG);
   }
 
   @Test
@@ -461,17 +486,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);
@@ -486,7 +511,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 {
@@ -496,7 +521,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);
@@ -504,7 +529,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]);
@@ -512,13 +537,22 @@ 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 Timestamp[] {new Timestamp(0)}, 1, testType);
+    batch.cols[0].isRepeating = true;
+    batch.selectedInUse = true;
+    batch.selected = new int[] {42};
+    verifyUDFMinute(batch, testType);
+    batch.cols[0].noNulls = false;
+    batch.cols[0].isNull[0] = true;
+    verifyUDFMinute(batch, testType);
+
     batch = getVectorizedRandomRowBatch(200, VectorizedRowBatch.DEFAULT_SIZE, testType);
     verifyUDFMinute(batch, testType);
     TestVectorizedRowBatch.addRandomNulls(batch.cols[0]);
@@ -529,7 +563,7 @@ public class TestVectorTimestampExpressions {
 
   @Test
   public void testVectorUDFMinuteLong() {
-    testVectorUDFMinute(TestType.LONG2);
+    testVectorUDFMinute(TestType.TIMESTAMP_LONG);
   }
 
   @Test
@@ -537,17 +571,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);
@@ -562,7 +596,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 {
@@ -572,7 +606,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);
@@ -580,7 +614,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]);
@@ -588,13 +622,22 @@ 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 Timestamp[] {new Timestamp(0)}, 1, testType);
+    batch.cols[0].isRepeating = true;
+    batch.selectedInUse = true;
+    batch.selected = new int[] {42};
+    verifyUDFMonth(batch, testType);
+    batch.cols[0].noNulls = false;
+    batch.cols[0].isNull[0] = true;
+    verifyUDFMonth(batch, testType);
+
     batch = getVectorizedRandomRowBatch(200, VectorizedRowBatch.DEFAULT_SIZE, testType);
     verifyUDFMonth(batch, testType);
     TestVectorizedRowBatch.addRandomNulls(batch.cols[0]);
@@ -604,8 +647,8 @@ public class TestVectorTimestampExpressions {
   }
 
   @Test
-  public void testVectorUDFMonthLong() {
-    testVectorUDFMonth(TestType.LONG2);
+  public void testVectorUDFMonthTimestamp() {
+    testVectorUDFMonth(TestType.TIMESTAMP_LONG);
   }
 
   @Test
@@ -613,17 +656,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);
@@ -638,7 +681,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 {
@@ -648,7 +691,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);
@@ -656,7 +699,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]);
@@ -664,13 +707,22 @@ 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 Timestamp[] {new Timestamp(0)}, 1, testType);
+    batch.cols[0].isRepeating = true;
+    batch.selectedInUse = true;
+    batch.selected = new int[] {42};
+    verifyUDFSecond(batch, testType);
+    batch.cols[0].noNulls = false;
+    batch.cols[0].isNull[0] = true;
+    verifyUDFSecond(batch, testType);
+
     batch = getVectorizedRandomRowBatch(200, VectorizedRowBatch.DEFAULT_SIZE, testType);
     verifyUDFSecond(batch, testType);
     TestVectorizedRowBatch.addRandomNulls(batch.cols[0]);
@@ -681,7 +733,7 @@ public class TestVectorTimestampExpressions {
 
   @Test
   public void testVectorUDFSecondLong() {
-    testVectorUDFSecond(TestType.LONG2);
+    testVectorUDFSecond(TestType.TIMESTAMP_LONG);
   }
 
   @Test
@@ -689,31 +741,18 @@ 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(long t, long y) {
-    TimestampWritable tsw = toTimestampWritable(t);
-    LongWritable res = getLongWritable(tsw);
-    if(res.get() != y) {
-      System.out.printf("%d vs %d for %d, %d\n", res.get(), y, t,
-          tsw.getTimestamp().getTime()/1000);
-    }
-
-    Assert.assertEquals(res.get(), y);
-  }
-
   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);
@@ -728,7 +767,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 {
@@ -738,7 +777,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);
@@ -746,7 +785,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]);
@@ -754,13 +793,22 @@ 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 Timestamp[] {new Timestamp(0)}, 1, testType);
+    batch.cols[0].isRepeating = true;
+    batch.selectedInUse = true;
+    batch.selected = new int[] {42};
+    verifyUDFUnixTimeStamp(batch, testType);
+    batch.cols[0].noNulls = false;
+    batch.cols[0].isNull[0] = true;
+
+    verifyUDFUnixTimeStamp(batch, testType);
     batch = getVectorizedRandomRowBatch(200, VectorizedRowBatch.DEFAULT_SIZE, testType);
     verifyUDFUnixTimeStamp(batch, testType);
     TestVectorizedRowBatch.addRandomNulls(batch.cols[0]);
@@ -770,8 +818,8 @@ public class TestVectorTimestampExpressions {
   }
 
   @Test
-  public void testVectorUDFUnixTimeStampLong() {
-    testVectorUDFUnixTimeStamp(TestType.LONG2);
+  public void testVectorUDFUnixTimeStampTimestamp() {
+    testVectorUDFUnixTimeStamp(TestType.TIMESTAMP_LONG);
   }
 
   @Test
@@ -779,17 +827,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);
@@ -801,7 +849,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 {
@@ -811,7 +859,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);
@@ -819,7 +867,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]);
@@ -827,8 +875,17 @@ 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 Timestamp[] {new Timestamp(0)}, 1, testType);
     batch.cols[0].isRepeating = true;
+    batch.selectedInUse = true;
+    batch.selected = new int[] {42};
     verifyUDFWeekOfYear(batch, testType);
     batch.cols[0].noNulls = false;
     batch.cols[0].isNull[0] = true;
@@ -843,8 +900,8 @@ public class TestVectorTimestampExpressions {
   }
 
   @Test
-  public void testVectorUDFWeekOfYearLong() {
-    testVectorUDFWeekOfYear(TestType.LONG2);
+  public void testVectorUDFWeekOfYearTimestamp() {
+    testVectorUDFWeekOfYear(TestType.TIMESTAMP_LONG);
   }
 
   @Test
@@ -854,12 +911,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/130293e5/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..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,19 +23,29 @@ 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.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;
 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.apache.hadoop.hive.serde2.typeinfo.HiveDecimalUtils;
 import org.junit.Test;
 
 /**
@@ -43,9 +53,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 +86,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, 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.getVectorizedRowBatchLongInLongOut();
-    LongColumnVector resultV = (LongColumnVector) b.cols[1];
+    long[] longValues = new long[500];
+    VectorizedRowBatch b = TestVectorMathFunctions.getVectorizedRowBatchLongInTimestampOut(longValues);
+    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]);
+    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.getVectorizedRowBatchLongInLongOut();
-    LongColumnVector inV = (LongColumnVector) b.cols[0];
-    inV.vector[0] = NANOS_PER_SECOND;  // Make one entry produce interesting result
-      // (1 sec after epoch).
-
+    long[] longValues = new long[500];
+    VectorizedRowBatch b = TestVectorMathFunctions.getVectorizedRowBatchTimestampInLongOut(longValues);
+    TimestampColumnVector inV = (TimestampColumnVector) b.cols[0];
     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]);
+    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.getVectorizedRowBatchLongInDoubleOut();
-    LongColumnVector inV = (LongColumnVector) b.cols[0];
+    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 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]);
+    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,13 +375,19 @@ public class TestVectorTypeCasts {
 
   @Test
   public void testCastDecimalToTimestamp() {
-    VectorizedRowBatch b = getBatchDecimalLong2();
+    double[] doubleValues = new double[500];
+    VectorizedRowBatch b = getBatchDecimalTimestamp(doubleValues);
     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];
+    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() {
@@ -381,6 +406,28 @@ public class TestVectorTypeCasts {
     return b;
   }
 
+  private VectorizedRowBatch getBatchDecimalTimestamp(double[] doubleValues) {
+    VectorizedRowBatch b = new VectorizedRowBatch(2);
+    DecimalColumnVector dv;
+    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;
+  }
+
   @Test
   public void testCastLongToDecimal() {
     VectorizedRowBatch b = getBatchLongDecimal();
@@ -403,6 +450,47 @@ public class TestVectorTypeCasts {
     return b;
   }
 
+
+  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(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;
+  }
+
   @Test
   public void testCastDoubleToDecimal() {
     VectorizedRowBatch b = getBatchDoubleDecimal();
@@ -410,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() {
@@ -466,25 +563,37 @@ public class TestVectorTypeCasts {
 
     // The input timestamps are stored as long values
     // measured in nanoseconds from the epoch.
-    VectorizedRowBatch b = getBatchLongDecimal();
+    HiveDecimal[] hiveDecimalValues = new HiveDecimal[500];
+    VectorizedRowBatch b = getBatchTimestampDecimal(hiveDecimalValues);
     VectorExpression expr = new CastTimestampToDecimal(0, 1);
-    LongColumnVector inL = (LongColumnVector) b.cols[0];
-    inL.vector[1] = -1990000000L;
+    TimestampColumnVector inT = (TimestampColumnVector) b.cols[0];
     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 = getBatchLongDecimalPrec5Scale2();
+    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.
@@ -503,30 +612,6 @@ public class TestVectorTypeCasts {
     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/130293e5/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..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
@@ -32,7 +32,7 @@ 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 +101,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, t);
           }
         };
 

http://git-wip-us.apache.org/repos/asf/hive/blob/130293e5/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..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
@@ -18,13 +18,15 @@
 
 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.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 +79,37 @@ public class VectorizedRowGroupGenUtil {
     return lcv;
   }
 
+  public static TimestampColumnVector generateTimestampColumnVector(
+      boolean nulls, boolean repeating, int size, Random rand, Timestamp[] timestampValues) {
+    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);
+        timestampValues[i] = null;
+      }else {
+        tcv.isNull[i] = false;
+        if (!repeating) {
+          Timestamp randomTimestamp = RandomTypeUtil.getRandTimestamp(rand);
+          tcv.set(i,  randomTimestamp);
+          timestampValues[i] = randomTimestamp;
+        } else {
+          tcv.set(i, repeatingTimestamp);
+          timestampValues[i] = repeatingTimestamp;
+        }
+      }
+    }
+    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/130293e5/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 a345884..c0d912d 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
@@ -58,6 +58,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;
@@ -1779,7 +1780,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]);
@@ -1800,8 +1801,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.getTime(i));
     }
     assertEquals(false, reader.next(key, value));
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/130293e5/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 137c39f..e526913 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
@@ -525,7 +525,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(1, OrcUtils.getFlattenedColumnsCount(inspector));
     boolean[] expected = new boolean[] {false};