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

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

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

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

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

http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/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 c88f6d8..85923a8 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
@@ -1946,7 +1946,7 @@ public class TestInputOutputFormat {
       long millis = (long) i * MILLIS_IN_DAY;
       millis -= LOCAL_TIMEZONE.getOffset(millis);
       assertEquals("checking timestamp " + i, millis,
-          timestampColumn.getTimestampMilliseconds(i));
+          timestampColumn.getTime(i));
     }
     assertEquals(false, reader.next(key, value));
   }

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

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

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