You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by jc...@apache.org on 2018/06/28 15:45:12 UTC

[28/34] hive git commit: HIVE-20007: Hive should carry out timestamp computations in UTC (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

http://git-wip-us.apache.org/repos/asf/hive/blob/ae008b79/ql/src/java/org/apache/hadoop/hive/ql/util/DateTimeMath.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/util/DateTimeMath.java b/ql/src/java/org/apache/hadoop/hive/ql/util/DateTimeMath.java
index 9a097af..16babbf 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/util/DateTimeMath.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/util/DateTimeMath.java
@@ -17,17 +17,17 @@
  */
 package org.apache.hadoop.hive.ql.util;
 
-import java.sql.Date;
-import java.sql.Timestamp;
+import org.apache.hadoop.hive.common.type.Date;
+import org.apache.hadoop.hive.common.type.HiveIntervalDayTime;
+import org.apache.hadoop.hive.common.type.HiveIntervalYearMonth;
+import org.apache.hadoop.hive.common.type.Timestamp;
+import org.apache.hadoop.hive.serde2.io.DateWritableV2;
+import org.apache.hive.common.util.DateUtils;
+
 import java.util.Calendar;
 import java.util.TimeZone;
 import java.util.concurrent.TimeUnit;
 
-import org.apache.hadoop.hive.common.type.HiveIntervalYearMonth;
-import org.apache.hadoop.hive.common.type.HiveIntervalDayTime;
-import org.apache.hadoop.hive.serde2.io.DateWritable;
-import org.apache.hive.common.util.DateUtils;
-
 
 public class DateTimeMath {
 
@@ -49,7 +49,6 @@ public class DateTimeMath {
   }
 
   protected Calendar calUtc = Calendar.getInstance(TimeZone.getTimeZone("UTC"));
-  protected Calendar calLocal = Calendar.getInstance();
   protected NanosResult nanosResult = new NanosResult();
 
   //
@@ -62,39 +61,22 @@ public class DateTimeMath {
    * @param months
    * @return
    */
-  public long addMonthsToMillisUtc(long millis, int months) {
+  public long addMonthsToMillis(long millis, int months) {
     calUtc.setTimeInMillis(millis);
     calUtc.add(Calendar.MONTH, months);
     return calUtc.getTimeInMillis();
   }
 
-  /**
-   * Perform month arithmetic to millis value using local time zone.
-   * @param millis
-   * @param months
-   * @return
-   */
-  public long addMonthsToMillisLocal(long millis, int months) {
-    calLocal.setTimeInMillis(millis);
-    calLocal.add(Calendar.MONTH, months);
-    return calLocal.getTimeInMillis();
-  }
-
-  public long addMonthsToNanosUtc(long nanos, int months) {
-    long result = addMonthsToMillisUtc(nanos / 1000000, months) * 1000000 + (nanos % 1000000);
-    return result;
-  }
-
-  public long addMonthsToNanosLocal(long nanos, int months) {
-    long result = addMonthsToMillisLocal(nanos / 1000000, months) * 1000000 + (nanos % 1000000);
+  public long addMonthsToNanos(long nanos, int months) {
+    long result = addMonthsToMillis(nanos / 1000000, months) * 1000000 + (nanos % 1000000);
     return result;
   }
 
   public long addMonthsToDays(long days, int months) {
-    long millis = DateWritable.daysToMillis((int) days);
-    millis = addMonthsToMillisLocal(millis, months);
+    long millis = DateWritableV2.daysToMillis((int) days);
+    millis = addMonthsToMillis(millis, months);
     // Convert millis result back to days
-    return DateWritable.millisToDays(millis);
+    return DateWritableV2.millisToDays(millis);
   }
 
   public Timestamp add(Timestamp ts, HiveIntervalYearMonth interval) {
@@ -102,7 +84,19 @@ public class DateTimeMath {
       return null;
     }
 
-    Timestamp tsResult = new Timestamp(0);
+    Timestamp tsResult = new Timestamp();
+    add(ts, interval, tsResult);
+
+    return tsResult;
+  }
+
+  @Deprecated
+  public java.sql.Timestamp add(java.sql.Timestamp ts, HiveIntervalYearMonth interval) {
+    if (ts == null || interval == null) {
+      return null;
+    }
+
+    java.sql.Timestamp tsResult = new java.sql.Timestamp(0);
     add(ts, interval, tsResult);
 
     return tsResult;
@@ -113,9 +107,21 @@ public class DateTimeMath {
       return false;
     }
 
+    long resultMillis = addMonthsToMillis(ts.toEpochMilli(), interval.getTotalMonths());
+    result.setTimeInMillis(resultMillis, ts.getNanos());
+
+    return true;
+  }
+
+  @Deprecated
+  public boolean add(java.sql.Timestamp ts, HiveIntervalYearMonth interval, java.sql.Timestamp result) {
+    if (ts == null || interval == null) {
+      return false;
+    }
+
     // Attempt to match Oracle semantics for timestamp arithmetic,
     // where timestamp arithmetic is done in UTC, then converted back to local timezone
-    long resultMillis = addMonthsToMillisUtc(ts.getTime(), interval.getTotalMonths());
+    long resultMillis = addMonthsToMillis(ts.getTime(), interval.getTotalMonths());
     result.setTime(resultMillis);
     result.setNanos(ts.getNanos());
 
@@ -127,7 +133,19 @@ public class DateTimeMath {
       return null;
     }
 
-    Timestamp tsResult = new Timestamp(0);
+    Timestamp tsResult = new Timestamp();
+    add(interval, ts, tsResult);
+
+    return tsResult;
+  }
+
+  @Deprecated
+  public java.sql.Timestamp add(HiveIntervalYearMonth interval, java.sql.Timestamp ts) {
+    if (ts == null || interval == null) {
+      return null;
+    }
+
+    java.sql.Timestamp tsResult = new java.sql.Timestamp(0);
     add(interval, ts, tsResult);
 
     return tsResult;
@@ -138,9 +156,19 @@ public class DateTimeMath {
       return false;
     }
 
-    // Attempt to match Oracle semantics for timestamp arithmetic,
-    // where timestamp arithmetic is done in UTC, then converted back to local timezone
-    long resultMillis = addMonthsToMillisUtc(ts.getTime(), interval.getTotalMonths());
+    long resultMillis = addMonthsToMillis(ts.toEpochMilli(), interval.getTotalMonths());
+    result.setTimeInMillis(resultMillis, ts.getNanos());
+
+    return true;
+  }
+
+  @Deprecated
+  public boolean add(HiveIntervalYearMonth interval, java.sql.Timestamp ts, java.sql.Timestamp result) {
+    if (ts == null || interval == null) {
+      return false;
+    }
+
+    long resultMillis = addMonthsToMillis(ts.getTime(), interval.getTotalMonths());
     result.setTime(resultMillis);
     result.setNanos(ts.getNanos());
 
@@ -152,7 +180,19 @@ public class DateTimeMath {
       return null;
     }
 
-    Date dtResult = new Date(0);
+    Date dtResult = new Date();
+    add(dt, interval, dtResult);
+
+    return dtResult;
+  }
+
+  @Deprecated
+  public java.sql.Date add(java.sql.Date dt, HiveIntervalYearMonth interval) {
+    if (dt == null || interval == null) {
+      return null;
+    }
+
+    java.sql.Date dtResult = new java.sql.Date(0);
     add(dt, interval, dtResult);
 
     return dtResult;
@@ -163,9 +203,18 @@ public class DateTimeMath {
       return false;
     }
 
-    // Since Date millis value is in local timezone representation, do date arithmetic
-    // using local timezone so the time remains at the start of the day.
-    long resultMillis = addMonthsToMillisLocal(dt.getTime(), interval.getTotalMonths());
+    long resultMillis = addMonthsToMillis(dt.toEpochMilli(), interval.getTotalMonths());
+    result.setTimeInMillis(resultMillis);
+    return true;
+  }
+
+  @Deprecated
+  public boolean add(java.sql.Date dt, HiveIntervalYearMonth interval, java.sql.Date result) {
+    if (dt == null || interval == null) {
+      return false;
+    }
+
+    long resultMillis = addMonthsToMillis(dt.getTime(), interval.getTotalMonths());
     result.setTime(resultMillis);
     return true;
   }
@@ -175,7 +224,19 @@ public class DateTimeMath {
       return null;
     }
 
-    Date dtResult = new Date(0);
+    Date dtResult = new Date();
+    add(interval, dt, dtResult);
+
+    return dtResult;
+  }
+
+  @Deprecated
+  public java.sql.Date add(HiveIntervalYearMonth interval, java.sql.Date dt) {
+    if (dt == null || interval == null) {
+      return null;
+    }
+
+    java.sql.Date dtResult = new java.sql.Date(0);
     add(interval, dt, dtResult);
 
     return dtResult;
@@ -186,9 +247,18 @@ public class DateTimeMath {
       return false;
     }
 
-    // Since Date millis value is in local timezone representation, do date arithmetic
-    // using local timezone so the time remains at the start of the day.
-    long resultMillis = addMonthsToMillisLocal(dt.getTime(), interval.getTotalMonths());
+    long resultMillis = addMonthsToMillis(dt.toEpochMilli(), interval.getTotalMonths());
+    result.setTimeInMillis(resultMillis);
+    return true;
+  }
+
+  @Deprecated
+  public boolean add(HiveIntervalYearMonth interval, java.sql.Date dt, java.sql.Date result) {
+    if (dt == null || interval == null) {
+      return false;
+    }
+
+    long resultMillis = addMonthsToMillis(dt.getTime(), interval.getTotalMonths());
     result.setTime(resultMillis);
     return true;
   }
@@ -208,7 +278,19 @@ public class DateTimeMath {
       return null;
     }
 
-    Timestamp tsResult = new Timestamp(0);
+    Timestamp tsResult = new Timestamp();
+    subtract(left, right, tsResult);
+
+    return tsResult;
+  }
+
+  @Deprecated
+  public java.sql.Timestamp subtract(java.sql.Timestamp left, HiveIntervalYearMonth right) {
+    if (left == null || right == null) {
+      return null;
+    }
+
+    java.sql.Timestamp tsResult = new java.sql.Timestamp(0);
     subtract(left, right, tsResult);
 
     return tsResult;
@@ -221,12 +303,32 @@ public class DateTimeMath {
     return add(left, right.negate(), result);
   }
 
+  @Deprecated
+  public boolean subtract(java.sql.Timestamp left, HiveIntervalYearMonth right, java.sql.Timestamp result) {
+    if (left == null || right == null) {
+      return false;
+    }
+    return add(left, right.negate(), result);
+  }
+
   public Date subtract(Date left, HiveIntervalYearMonth right) {
     if (left == null || right == null) {
       return null;
     }
 
-    Date dtResult = new Date(0);
+    Date dtResult = new Date();
+    subtract(left, right, dtResult);
+
+    return dtResult;
+  }
+
+  @Deprecated
+  public java.sql.Date subtract(java.sql.Date left, HiveIntervalYearMonth right) {
+    if (left == null || right == null) {
+      return null;
+    }
+
+    java.sql.Date dtResult = new java.sql.Date(0);
     subtract(left, right, dtResult);
 
     return dtResult;
@@ -239,6 +341,14 @@ public class DateTimeMath {
     return add(left, right.negate(), result);
   }
 
+  @Deprecated
+  public boolean subtract(java.sql.Date left, HiveIntervalYearMonth right, java.sql.Date result) {
+    if (left == null || right == null) {
+      return false;
+    }
+    return add(left, right.negate(), result);
+  }
+
   public HiveIntervalYearMonth subtract(HiveIntervalYearMonth left, HiveIntervalYearMonth right) {
     if (left == null || right == null) {
       return null;
@@ -255,7 +365,19 @@ public class DateTimeMath {
       return null;
     }
 
-    Timestamp tsResult = new Timestamp(0);
+    Timestamp tsResult = new Timestamp();
+    add(ts, interval, tsResult);
+
+    return tsResult;
+  }
+
+  @Deprecated
+  public java.sql.Timestamp add(java.sql.Timestamp ts, HiveIntervalDayTime interval) {
+    if (ts == null || interval == null) {
+      return null;
+    }
+
+    java.sql.Timestamp tsResult = new java.sql.Timestamp(0);
     add(ts, interval, tsResult);
 
     return tsResult;
@@ -269,6 +391,21 @@ public class DateTimeMath {
 
     nanosResult.addNanos(ts.getNanos(), interval.getNanos());
 
+    long newMillis = ts.toEpochMilli()
+        + TimeUnit.SECONDS.toMillis(interval.getTotalSeconds() + nanosResult.seconds);
+    result.setTimeInMillis(newMillis, nanosResult.nanos);
+    return true;
+  }
+
+  @Deprecated
+  public boolean add(java.sql.Timestamp ts, HiveIntervalDayTime interval,
+      java.sql.Timestamp result) {
+    if (ts == null || interval == null) {
+      return false;
+    }
+
+    nanosResult.addNanos(ts.getNanos(), interval.getNanos());
+
     long newMillis = ts.getTime()
         + TimeUnit.SECONDS.toMillis(interval.getTotalSeconds() + nanosResult.seconds);
     result.setTime(newMillis);
@@ -281,7 +418,18 @@ public class DateTimeMath {
       return null;
     }
 
-    Timestamp tsResult = new Timestamp(0);
+    Timestamp tsResult = new Timestamp();
+    add(interval, ts, tsResult);
+    return tsResult;
+  }
+
+  @Deprecated
+  public java.sql.Timestamp add(HiveIntervalDayTime interval, java.sql.Timestamp ts) {
+    if (ts == null || interval == null) {
+      return null;
+    }
+
+    java.sql.Timestamp tsResult = new java.sql.Timestamp(0);
     add(interval, ts, tsResult);
     return tsResult;
   }
@@ -294,6 +442,21 @@ public class DateTimeMath {
 
     nanosResult.addNanos(ts.getNanos(), interval.getNanos());
 
+    long newMillis = ts.toEpochMilli()
+        + TimeUnit.SECONDS.toMillis(interval.getTotalSeconds() + nanosResult.seconds);
+    result.setTimeInMillis(newMillis, nanosResult.nanos);
+    return true;
+  }
+
+  @Deprecated
+  public boolean add(HiveIntervalDayTime interval, java.sql.Timestamp ts,
+      java.sql.Timestamp result) {
+    if (ts == null || interval == null) {
+      return false;
+    }
+
+    nanosResult.addNanos(ts.getNanos(), interval.getNanos());
+
     long newMillis = ts.getTime()
         + TimeUnit.SECONDS.toMillis(interval.getTotalSeconds() + nanosResult.seconds);
     result.setTime(newMillis);
@@ -332,6 +495,14 @@ public class DateTimeMath {
     return add(left, right.negate());
   }
 
+  @Deprecated
+  public java.sql.Timestamp subtract(java.sql.Timestamp left, HiveIntervalDayTime right) {
+    if (left == null || right == null) {
+      return null;
+    }
+    return add(left, right.negate());
+  }
+
   public boolean subtract(Timestamp left, HiveIntervalDayTime right, Timestamp result) {
     if (left == null || right == null) {
       return false;
@@ -339,6 +510,14 @@ public class DateTimeMath {
     return add(left, right.negate(), result);
   }
 
+  @Deprecated
+  public boolean subtract(java.sql.Timestamp left, HiveIntervalDayTime right, java.sql.Timestamp result) {
+    if (left == null || right == null) {
+      return false;
+    }
+    return add(left, right.negate(), result);
+  }
+
   public HiveIntervalDayTime subtract(HiveIntervalDayTime left, HiveIntervalDayTime right) {
     if (left == null || right == null) {
       return null;
@@ -365,6 +544,18 @@ public class DateTimeMath {
     return result;
   }
 
+  @Deprecated
+  public HiveIntervalDayTime subtract(java.sql.Timestamp left, java.sql.Timestamp right) {
+    if (left == null || right == null) {
+      return null;
+    }
+
+    HiveIntervalDayTime result = new HiveIntervalDayTime();
+    subtract(left, right, result);
+
+    return result;
+  }
+
   public boolean subtract(Timestamp left, Timestamp right,
       HiveIntervalDayTime result) {
     if (left == null || right == null) {
@@ -373,6 +564,21 @@ public class DateTimeMath {
 
     nanosResult.addNanos(left.getNanos(), -(right.getNanos()));
 
+    long totalSeconds = TimeUnit.MILLISECONDS.toSeconds(left.toEpochMilli())
+        - TimeUnit.MILLISECONDS.toSeconds(right.toEpochMilli()) + nanosResult.seconds;
+    result.set(totalSeconds, nanosResult.nanos);
+    return true;
+  }
+
+  @Deprecated
+  public boolean subtract(java.sql.Timestamp left, java.sql.Timestamp right,
+      HiveIntervalDayTime result) {
+    if (left == null || right == null) {
+      return false;
+    }
+
+    nanosResult.addNanos(left.getNanos(), -(right.getNanos()));
+
     long totalSeconds = TimeUnit.MILLISECONDS.toSeconds(left.getTime())
         - TimeUnit.MILLISECONDS.toSeconds(right.getTime()) + nanosResult.seconds;
     result.set(totalSeconds, nanosResult.nanos);

http://git-wip-us.apache.org/repos/asf/hive/blob/ae008b79/ql/src/test/org/apache/hadoop/hive/ql/exec/TestFunctionRegistry.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestFunctionRegistry.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestFunctionRegistry.java
index 613d7a8..40d60f3 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestFunctionRegistry.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestFunctionRegistry.java
@@ -28,21 +28,19 @@ import junit.framework.TestCase;
 import org.apache.hadoop.hive.common.type.HiveVarchar;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.exec.FunctionInfo.FunctionResource;
-import org.apache.hadoop.hive.ql.exec.FunctionInfo.FunctionType;
 import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc;
 import org.apache.hadoop.hive.ql.session.SessionState;
-import org.apache.hadoop.hive.ql.udf.UDFAscii;
 import org.apache.hadoop.hive.ql.udf.UDFLn;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFMax;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDF;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFConcat;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFCurrentTimestamp;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDTFExplode;
-import org.apache.hadoop.hive.serde2.io.DateWritable;
+import org.apache.hadoop.hive.serde2.io.DateWritableV2;
 import org.apache.hadoop.hive.serde2.io.DoubleWritable;
 import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
-import org.apache.hadoop.hive.serde2.io.TimestampWritable;
+import org.apache.hadoop.hive.serde2.io.TimestampWritableV2;
 import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory;
 import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
@@ -62,10 +60,10 @@ public class TestFunctionRegistry extends TestCase {
     public void one(IntWritable x, HiveDecimalWritable y) {}
     public void one(IntWritable x, DoubleWritable y) {}
     public void one(IntWritable x, IntWritable y) {}
-    public void mismatch(DateWritable x, HiveDecimalWritable y) {}
-    public void mismatch(TimestampWritable x, HiveDecimalWritable y) {}
+    public void mismatch(DateWritableV2 x, HiveDecimalWritable y) {}
+    public void mismatch(TimestampWritableV2 x, HiveDecimalWritable y) {}
     public void mismatch(BytesWritable x, DoubleWritable y) {}
-    public void typeaffinity1(DateWritable x) {}
+    public void typeaffinity1(DateWritableV2 x) {}
     public void typeaffinity1(DoubleWritable x) {};
     public void typeaffinity1(Text x) {}
     public void typeaffinity2(IntWritable x) {}
@@ -158,8 +156,8 @@ public class TestFunctionRegistry extends TestCase {
     typeAffinity("typeaffinity1", TypeInfoFactory.floatTypeInfo, 1, DoubleWritable.class);
 
     // Prefer date type arguments over other method signatures
-    typeAffinity("typeaffinity1", TypeInfoFactory.dateTypeInfo, 1, DateWritable.class);
-    typeAffinity("typeaffinity1", TypeInfoFactory.timestampTypeInfo, 1, DateWritable.class);
+    typeAffinity("typeaffinity1", TypeInfoFactory.dateTypeInfo, 1, DateWritableV2.class);
+    typeAffinity("typeaffinity1", TypeInfoFactory.timestampTypeInfo, 1, DateWritableV2.class);
 
     // String type affinity
     typeAffinity("typeaffinity1", TypeInfoFactory.stringTypeInfo, 1, Text.class);

http://git-wip-us.apache.org/repos/asf/hive/blob/ae008b79/ql/src/test/org/apache/hadoop/hive/ql/exec/TestUtilities.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestUtilities.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestUtilities.java
index fdc268c..90eb45b 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestUtilities.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestUtilities.java
@@ -36,7 +36,6 @@ import static org.mockito.Mockito.when;
 import java.io.File;
 import java.io.IOException;
 import java.io.Serializable;
-import java.sql.Timestamp;
 import java.util.ArrayList;
 import java.util.HashSet;
 import java.util.LinkedHashMap;
@@ -55,6 +54,7 @@ import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.common.type.Timestamp;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
 import org.apache.hadoop.hive.ql.Context;
@@ -132,8 +132,7 @@ public class TestUtilities {
 
   @Test
   public void testSerializeTimestamp() {
-    Timestamp ts = new Timestamp(1374554702000L);
-    ts.setNanos(123456);
+    Timestamp ts = Timestamp.ofEpochMilli(1374554702000L, 123456);
     ExprNodeConstantDesc constant = new ExprNodeConstantDesc(ts);
     List<ExprNodeDesc> children = new ArrayList<ExprNodeDesc>(1);
     children.add(constant);

http://git-wip-us.apache.org/repos/asf/hive/blob/ae008b79/ql/src/test/org/apache/hadoop/hive/ql/exec/persistence/TestRowContainer.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/persistence/TestRowContainer.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/persistence/TestRowContainer.java
index c8ae73a..7e5e5fb 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/persistence/TestRowContainer.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/persistence/TestRowContainer.java
@@ -19,17 +19,17 @@ package org.apache.hadoop.hive.ql.exec.persistence;
 import static org.junit.Assert.assertEquals;
 
 import java.io.IOException;
-import java.sql.Timestamp;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Properties;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.common.type.Timestamp;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.serde.serdeConstants;
 import org.apache.hadoop.hive.serde2.SerDeException;
 import org.apache.hadoop.hive.serde2.SerDeUtils;
-import org.apache.hadoop.hive.serde2.io.TimestampWritable;
+import org.apache.hadoop.hive.serde2.io.TimestampWritableV2;
 import org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils;
 import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
@@ -55,7 +55,7 @@ public class TestRowContainer {
       ObjectInspectorUtils.getStandardObjectInspector(serde.getObjectInspector()));
     result.setTableDesc(
       PTFRowContainer.createTableDesc((StructObjectInspector) serde.getObjectInspector()));
-    TimestampWritable key = new TimestampWritable(new Timestamp(10));
+    TimestampWritableV2 key = new TimestampWritableV2(Timestamp.ofEpochMilli(10));
     result.setKeyObject(Lists.newArrayList(key));
     List<Writable> row;
     // will trigger 2 spills

http://git-wip-us.apache.org/repos/asf/hive/blob/ae008b79/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestTimestampWritableAndColumnVector.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestTimestampWritableAndColumnVector.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestTimestampWritableAndColumnVector.java
index f163289..6fd8e09 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestTimestampWritableAndColumnVector.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestTimestampWritableAndColumnVector.java
@@ -23,7 +23,7 @@ import org.junit.Test;
 import java.sql.Timestamp;
 import java.util.Random;
 
-import org.apache.hadoop.hive.common.type.RandomTypeUtil;
+import org.apache.hadoop.hive.serde2.RandomTypeUtil;
 import org.apache.hadoop.hive.ql.util.TimestampUtils;
 
 import static org.junit.Assert.*;
@@ -45,7 +45,7 @@ public class TestTimestampWritableAndColumnVector {
     Timestamp[] randTimestamps = new Timestamp[VectorizedRowBatch.DEFAULT_SIZE];
 
     for (int i = 0; i < VectorizedRowBatch.DEFAULT_SIZE; i++) {
-      Timestamp randTimestamp = RandomTypeUtil.getRandTimestamp(r);
+      Timestamp randTimestamp = RandomTypeUtil.getRandTimestamp(r).toSqlTimestamp();
       randTimestamps[i] = randTimestamp;
       timestampColVector.set(i, randTimestamp);
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/ae008b79/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorGroupByOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorGroupByOperator.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorGroupByOperator.java
index e65288b..ffdc410 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorGroupByOperator.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorGroupByOperator.java
@@ -71,7 +71,7 @@ import org.apache.hadoop.hive.serde2.io.ByteWritable;
 import org.apache.hadoop.hive.serde2.io.DoubleWritable;
 import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 import org.apache.hadoop.hive.serde2.io.ShortWritable;
-import org.apache.hadoop.hive.serde2.io.TimestampWritable;
+import org.apache.hadoop.hive.serde2.io.TimestampWritableV2;
 import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory;
 import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
@@ -1867,8 +1867,8 @@ public class TestVectorGroupByOperator {
           } else if (key instanceof LongWritable) {
             LongWritable lwKey = (LongWritable)key;
             keyValue.add(lwKey.get());
-          } else if (key instanceof TimestampWritable) {
-            TimestampWritable twKey = (TimestampWritable)key;
+          } else if (key instanceof TimestampWritableV2) {
+            TimestampWritableV2 twKey = (TimestampWritableV2)key;
             keyValue.add(twKey.getTimestamp());
           } else if (key instanceof DoubleWritable) {
             DoubleWritable dwKey = (DoubleWritable)key;
@@ -1988,9 +1988,9 @@ public class TestVectorGroupByOperator {
         } else if (key instanceof LongWritable) {
           LongWritable lwKey = (LongWritable)key;
           keyValue = lwKey.get();
-        } else if (key instanceof TimestampWritable) {
-          TimestampWritable twKey = (TimestampWritable)key;
-          keyValue = twKey.getTimestamp();
+        } else if (key instanceof TimestampWritableV2) {
+          TimestampWritableV2 twKey = (TimestampWritableV2)key;
+          keyValue = twKey.getTimestamp().toSqlTimestamp();
         } else if (key instanceof DoubleWritable) {
           DoubleWritable dwKey = (DoubleWritable)key;
           keyValue = dwKey.get();

http://git-wip-us.apache.org/repos/asf/hive/blob/ae008b79/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 f51b8bb..70a481d 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
@@ -21,12 +21,12 @@ package org.apache.hadoop.hive.ql.exec.vector;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
-import java.sql.Timestamp;
 import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.hadoop.hive.common.type.HiveChar;
 import org.apache.hadoop.hive.common.type.HiveVarchar;
+import org.apache.hadoop.hive.common.type.Timestamp;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.BRoundWithNumDigitsDoubleToDouble;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.ColAndCol;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.ColOrCol;

http://git-wip-us.apache.org/repos/asf/hive/blob/ae008b79/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/VectorRandomRowSource.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/VectorRandomRowSource.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/VectorRandomRowSource.java
index ae91b73..6181ae8 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/VectorRandomRowSource.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/VectorRandomRowSource.java
@@ -18,11 +18,10 @@
 
 package org.apache.hadoop.hive.ql.exec.vector;
 
-import java.sql.Date;
-import java.sql.Timestamp;
 import java.text.DateFormat;
 import java.text.SimpleDateFormat;
 import java.text.ParseException;
+
 import java.util.ArrayList;
 import java.util.HashSet;
 import java.util.List;
@@ -32,12 +31,14 @@ import java.util.Set;
 import org.apache.commons.lang.StringUtils;
 
 import org.apache.hadoop.hive.common.type.DataTypePhysicalVariation;
+import org.apache.hadoop.hive.common.type.Date;
 import org.apache.hadoop.hive.common.type.HiveChar;
 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.common.type.Timestamp;
+import org.apache.hadoop.hive.serde2.RandomTypeUtil;
 import org.apache.hadoop.hive.serde2.io.HiveCharWritable;
 import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 import org.apache.hadoop.hive.serde2.io.HiveVarcharWritable;
@@ -1298,15 +1299,11 @@ public class VectorRandomRowSource {
   }
 
   public static String randomPrimitiveDateStringObject(Random r) {
-    Date randomDate = RandomTypeUtil.getRandDate(r);
-    String randomDateString = randomDate.toString();
-    return randomDateString;
+    return RandomTypeUtil.getRandDate(r).toString();
   }
 
   public static String randomPrimitiveTimestampStringObject(Random r) {
-    Timestamp randomTimestamp = RandomTypeUtil.getRandTimestamp(r);
-    String randomTimestampString = randomTimestamp.toString();
-    return randomTimestampString;
+    return RandomTypeUtil.getRandTimestamp(r).toString();
   }
 
   public static HiveChar getRandHiveChar(Random r, CharTypeInfo charTypeInfo) {

http://git-wip-us.apache.org/repos/asf/hive/blob/ae008b79/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/VectorVerifyFast.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/VectorVerifyFast.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/VectorVerifyFast.java
index ec5ad23..458aae8 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/VectorVerifyFast.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/VectorVerifyFast.java
@@ -19,15 +19,17 @@
 package org.apache.hadoop.hive.ql.exec.vector;
 
 import junit.framework.TestCase;
+import org.apache.hadoop.hive.common.type.Date;
 import org.apache.hadoop.hive.common.type.HiveChar;
 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.Timestamp;
 import org.apache.hadoop.hive.serde2.fast.DeserializeRead;
 import org.apache.hadoop.hive.serde2.fast.SerializeWrite;
 import org.apache.hadoop.hive.serde2.io.ByteWritable;
-import org.apache.hadoop.hive.serde2.io.DateWritable;
+import org.apache.hadoop.hive.serde2.io.DateWritableV2;
 import org.apache.hadoop.hive.serde2.io.DoubleWritable;
 import org.apache.hadoop.hive.serde2.io.HiveCharWritable;
 import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
@@ -35,7 +37,7 @@ import org.apache.hadoop.hive.serde2.io.HiveIntervalDayTimeWritable;
 import org.apache.hadoop.hive.serde2.io.HiveIntervalYearMonthWritable;
 import org.apache.hadoop.hive.serde2.io.HiveVarcharWritable;
 import org.apache.hadoop.hive.serde2.io.ShortWritable;
-import org.apache.hadoop.hive.serde2.io.TimestampWritable;
+import org.apache.hadoop.hive.serde2.io.TimestampWritableV2;
 import org.apache.hadoop.hive.serde2.objectinspector.StandardUnionObjectInspector;
 import org.apache.hadoop.hive.serde2.typeinfo.CharTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo;
@@ -56,8 +58,6 @@ import org.apache.hadoop.io.Text;
 
 import java.io.IOException;
 import java.nio.charset.StandardCharsets;
-import java.sql.Date;
-import java.sql.Timestamp;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashMap;
@@ -240,7 +240,7 @@ public class VectorVerifyFast {
       case DATE:
       {
         Date value = deserializeRead.currentDateWritable.get();
-        Date expected = ((DateWritable) object).get();
+        Date expected = ((DateWritableV2) object).get();
         if (!value.equals(expected)) {
           TestCase.fail("Date field mismatch (expected " + expected.toString() + " found " + value.toString() + ")");
         }
@@ -249,7 +249,7 @@ public class VectorVerifyFast {
       case TIMESTAMP:
       {
         Timestamp value = deserializeRead.currentTimestampWritable.getTimestamp();
-        Timestamp expected = ((TimestampWritable) object).getTimestamp();
+        Timestamp expected = ((TimestampWritableV2) object).getTimestamp();
         if (!value.equals(expected)) {
           TestCase.fail("Timestamp field mismatch (expected " + expected.toString() + " found " + value.toString() + ")");
         }
@@ -390,13 +390,13 @@ public class VectorVerifyFast {
       break;
       case DATE:
       {
-        Date value = ((DateWritable) object).get();
+        Date value = ((DateWritableV2) object).get();
         serializeWrite.writeDate(value);
       }
       break;
       case TIMESTAMP:
       {
-        Timestamp value = ((TimestampWritable) object).getTimestamp();
+        Timestamp value = ((TimestampWritableV2) object).getTimestamp();
         serializeWrite.writeTimestamp(value);
       }
       break;
@@ -567,9 +567,9 @@ public class VectorVerifyFast {
     case DECIMAL:
       return new HiveDecimalWritable(deserializeRead.currentHiveDecimalWritable);
     case DATE:
-      return new DateWritable(deserializeRead.currentDateWritable);
+      return new DateWritableV2(deserializeRead.currentDateWritable);
     case TIMESTAMP:
-      return new TimestampWritable(deserializeRead.currentTimestampWritable);
+      return new TimestampWritableV2(deserializeRead.currentTimestampWritable);
     case INTERVAL_YEAR_MONTH:
       return new HiveIntervalYearMonthWritable(deserializeRead.currentHiveIntervalYearMonthWritable);
     case INTERVAL_DAY_TIME:

http://git-wip-us.apache.org/repos/asf/hive/blob/ae008b79/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 ffe9c81..b5ad22c 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
@@ -18,6 +18,13 @@
 
 package org.apache.hadoop.hive.ql.exec.vector.expressions;
 
+import org.apache.hadoop.hive.ql.udf.UDFDayOfMonth;
+import org.apache.hadoop.hive.ql.udf.UDFMonth;
+import org.apache.hadoop.hive.ql.udf.UDFYear;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDF;
+import org.apache.hadoop.hive.serde2.io.TimestampWritableV2;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
 import org.junit.Assert;
 import org.apache.commons.lang.ArrayUtils;
 import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
@@ -25,12 +32,8 @@ 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.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
-import org.apache.hadoop.hive.ql.udf.UDFDayOfMonth;
-import org.apache.hadoop.hive.ql.udf.UDFMonth;
 import org.apache.hadoop.hive.ql.udf.UDFWeekOfYear;
-import org.apache.hadoop.hive.ql.udf.UDFYear;
-import org.apache.hadoop.hive.serde2.io.DateWritable;
-import org.apache.hadoop.hive.serde2.io.TimestampWritable;
+import org.apache.hadoop.hive.serde2.io.DateWritableV2;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
 import org.apache.hadoop.io.IntWritable;
@@ -38,34 +41,33 @@ import org.apache.hadoop.io.LongWritable;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
-import org.junit.internal.runners.statements.Fail;
 
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
 
-import java.sql.Timestamp;
 import java.util.ArrayList;
 import java.util.Calendar;
 import java.util.List;
 import java.util.Random;
+import java.util.TimeZone;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
-import java.util.concurrent.ThreadFactory;
 
 public class TestVectorDateExpressions {
 
   private ExecutorService runner;
 
   /* copied over from VectorUDFTimestampFieldLong */
-  private TimestampWritable toTimestampWritable(long daysSinceEpoch) {
-    Timestamp ts = new Timestamp(DateWritable.daysToMillis((int) daysSinceEpoch));
-    return new TimestampWritable(ts);
+  private TimestampWritableV2 toTimestampWritable(long daysSinceEpoch) {
+    return new TimestampWritableV2(
+        org.apache.hadoop.hive.common.type.Timestamp.ofEpochMilli(
+            DateWritableV2.daysToMillis((int) daysSinceEpoch)));
   }
 
   private int[] getAllBoundaries() {
     List<Integer> boundaries = new ArrayList<Integer>(1);
-    Calendar c = Calendar.getInstance();
+    Calendar c = Calendar.getInstance(TimeZone.getTimeZone("UTC"));
     c.setTimeInMillis(0); // c.set doesn't reset millis
     for (int year = 1902; year <= 2038; year++) {
       c.set(year, Calendar.JANUARY, 1, 0, 0, 0);
@@ -108,10 +110,12 @@ public class TestVectorDateExpressions {
     return batch;
   }
 
-  private void compareToUDFYearDate(long t, int y) {
+  private void compareToUDFYearDate(long t, int y) throws HiveException {
     UDFYear udf = new UDFYear();
-    TimestampWritable tsw = toTimestampWritable(t);
-    IntWritable res = udf.evaluate(tsw);
+    udf.initialize(new ObjectInspector[]{PrimitiveObjectInspectorFactory.writableTimestampObjectInspector});
+    TimestampWritableV2 tsw = toTimestampWritable(t);
+    IntWritable res = (IntWritable) udf.evaluate(
+        new GenericUDF.DeferredObject[]{new GenericUDF.DeferredJavaObject(tsw)});
     Assert.assertEquals(res.get(), y);
   }
 
@@ -170,10 +174,12 @@ public class TestVectorDateExpressions {
     verifyUDFYear(batch);
   }
 
-  private void compareToUDFDayOfMonthDate(long t, int y) {
+  private void compareToUDFDayOfMonthDate(long t, int y) throws HiveException {
     UDFDayOfMonth udf = new UDFDayOfMonth();
-    TimestampWritable tsw = toTimestampWritable(t);
-    IntWritable res = udf.evaluate(tsw);
+    udf.initialize(new ObjectInspector[]{PrimitiveObjectInspectorFactory.writableTimestampObjectInspector});
+    TimestampWritableV2 tsw = toTimestampWritable(t);
+    IntWritable res = (IntWritable) udf.evaluate(
+        new GenericUDF.DeferredObject[]{new GenericUDF.DeferredJavaObject(tsw)});
     Assert.assertEquals(res.get(), y);
   }
 
@@ -232,10 +238,12 @@ public class TestVectorDateExpressions {
     verifyUDFDayOfMonth(batch);
   }
 
-  private void compareToUDFMonthDate(long t, int y) {
+  private void compareToUDFMonthDate(long t, int y) throws HiveException {
     UDFMonth udf = new UDFMonth();
-    TimestampWritable tsw = toTimestampWritable(t);
-    IntWritable res = udf.evaluate(tsw);
+    udf.initialize(new ObjectInspector[]{PrimitiveObjectInspectorFactory.writableTimestampObjectInspector});
+    TimestampWritableV2 tsw = toTimestampWritable(t);
+    IntWritable res = (IntWritable) udf.evaluate(
+        new GenericUDF.DeferredObject[]{new GenericUDF.DeferredJavaObject(tsw)});
     Assert.assertEquals(res.get(), y);
   }
 
@@ -294,7 +302,7 @@ public class TestVectorDateExpressions {
     verifyUDFMonth(batch);
   }
 
-  private LongWritable getLongWritable(TimestampWritable i) {
+  private LongWritable getLongWritable(TimestampWritableV2 i) {
     LongWritable result = new LongWritable();
     if (i == null) {
       return null;
@@ -305,11 +313,11 @@ public class TestVectorDateExpressions {
   }
 
   private void compareToUDFUnixTimeStampDate(long t, long y) {
-    TimestampWritable tsw = toTimestampWritable(t);
+    TimestampWritableV2 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);
+              tsw.getTimestamp().toEpochMilli()/1000);
     }
 
     Assert.assertEquals(res.get(), y);
@@ -372,7 +380,7 @@ public class TestVectorDateExpressions {
 
   private void compareToUDFWeekOfYearDate(long t, int y) {
     UDFWeekOfYear udf = new UDFWeekOfYear();
-    TimestampWritable tsw = toTimestampWritable(t);
+    TimestampWritableV2 tsw = toTimestampWritable(t);
     IntWritable res = udf.evaluate(tsw);
     Assert.assertEquals(res.get(), y);
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/ae008b79/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 0bae9b4..aa91344 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
@@ -46,7 +46,7 @@ import org.apache.hadoop.hive.serde2.io.DoubleWritable;
 import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 import org.apache.hadoop.hive.serde2.io.HiveVarcharWritable;
 import org.apache.hadoop.hive.serde2.io.ShortWritable;
-import org.apache.hadoop.hive.serde2.io.TimestampWritable;
+import org.apache.hadoop.hive.serde2.io.TimestampWritableV2;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
 import org.apache.hadoop.hive.serde2.objectinspector.SettableListObjectInspector;
@@ -91,7 +91,8 @@ public class TestVectorExpressionWriters {
 
 
   private Writable getWritableValue(TypeInfo ti, Timestamp value) {
-    return new TimestampWritable(value);
+    return new TimestampWritableV2(
+        org.apache.hadoop.hive.common.type.Timestamp.ofEpochMilli(value.getTime(), value.getNanos()));
   }
 
   private Writable getWritableValue(TypeInfo ti, HiveDecimal value) {
@@ -123,7 +124,8 @@ public class TestVectorExpressionWriters {
       return new BooleanWritable( value == 0 ? false : true);
     } else if (ti.equals(TypeInfoFactory.timestampTypeInfo)) {
       Timestamp ts = new Timestamp(value);
-      TimestampWritable tw = new TimestampWritable(ts);
+      TimestampWritableV2 tw = new TimestampWritableV2(
+          org.apache.hadoop.hive.common.type.Timestamp.ofEpochMilli(ts.getTime(), ts.getNanos()));
       return tw;
     }
     return null;
@@ -246,8 +248,8 @@ public class TestVectorExpressionWriters {
       Writable w = (Writable) vew.writeValue(tcv, i);
       if (w != null) {
         Writable expected = getWritableValue(type, timestampValues[i]);
-        TimestampWritable t1 = (TimestampWritable) expected;
-        TimestampWritable t2 = (TimestampWritable) w;
+        TimestampWritableV2 t1 = (TimestampWritableV2) expected;
+        TimestampWritableV2 t2 = (TimestampWritableV2) w;
         Assert.assertTrue(t1.equals(t2));
        } else {
         Assert.assertTrue(tcv.isNull[i]);
@@ -270,8 +272,8 @@ public class TestVectorExpressionWriters {
       values[i] = vew.setValue(values[i], tcv, i);
       if (values[i] != null) {
         Writable expected = getWritableValue(type, timestampValues[i]);
-        TimestampWritable t1 = (TimestampWritable) expected;
-        TimestampWritable t2 = (TimestampWritable) values[i];
+        TimestampWritableV2 t1 = (TimestampWritableV2) expected;
+        TimestampWritableV2 t2 = (TimestampWritableV2) values[i];
         Assert.assertTrue(t1.equals(t2));
       } else {
         Assert.assertTrue(tcv.isNull[i]);

http://git-wip-us.apache.org/repos/asf/hive/blob/ae008b79/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 9792951..e7884b2 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
@@ -25,7 +25,7 @@ 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.metadata.HiveException;
-import org.apache.hadoop.hive.serde2.io.DateWritable;
+import org.apache.hadoop.hive.serde2.io.DateWritableV2;
 import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
@@ -40,16 +40,23 @@ import java.text.SimpleDateFormat;
 import java.util.Arrays;
 import java.util.List;
 import java.util.Random;
+import java.util.TimeZone;
 
 public class TestVectorGenericDateExpressions {
 
   private Charset utf8 = StandardCharsets.UTF_8;
   private int size = 200;
   private Random random = new Random();
-  private SimpleDateFormat formatter = new SimpleDateFormat("yyyy-MM-dd");
+  private SimpleDateFormat formatter = getFormatter();
   private List<PrimitiveCategory> dateTimestampStringTypes =
       Arrays.<PrimitiveCategory>asList(PrimitiveCategory.DATE, PrimitiveCategory.TIMESTAMP, PrimitiveCategory.STRING);
 
+  private static SimpleDateFormat getFormatter() {
+    SimpleDateFormat formatter = new SimpleDateFormat("yyyy-MM-dd");
+    formatter.setTimeZone(TimeZone.getTimeZone("UTC"));
+    return formatter;
+  }
+
   private long newRandom(int i) {
     return random.nextInt(i);
   }
@@ -88,7 +95,7 @@ public class TestVectorGenericDateExpressions {
   }
 
   private Timestamp toTimestamp(long date) {
-    return new Timestamp(DateWritable.daysToMillis((int) date));
+    return new Timestamp(DateWritableV2.daysToMillis((int) date));
   }
 
   private BytesColumnVector toString(LongColumnVector date) {
@@ -107,7 +114,7 @@ public class TestVectorGenericDateExpressions {
   }
 
   private byte[] toString(long date) {
-    String formatted = formatter.format(new Date(DateWritable.daysToMillis((int) date)));
+    String formatted = formatter.format(new Date(DateWritableV2.daysToMillis((int) date)));
     return formatted.getBytes(utf8);
   }
 
@@ -668,7 +675,7 @@ public class TestVectorGenericDateExpressions {
       if (date.isNull[i]) {
         Assert.assertTrue(output.isNull[i]);
       } else {
-        String expected = formatter.format(new Date(DateWritable.daysToMillis((int) date.vector[i])));
+        String expected = formatter.format(new Date(DateWritableV2.daysToMillis((int) date.vector[i])));
         Assert.assertEquals(expected, actual);
       }
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/ae008b79/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 e81844c..14d1343 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
@@ -20,13 +20,13 @@ package org.apache.hadoop.hive.ql.exec.vector.expressions;
 
 import java.io.UnsupportedEncodingException;
 import java.nio.charset.StandardCharsets;
-import java.sql.Timestamp;
 import java.util.Arrays;
 import java.util.Random;
 
 import junit.framework.Assert;
 
-import org.apache.hadoop.hive.common.type.RandomTypeUtil;
+import org.apache.hadoop.hive.serde2.RandomTypeUtil;
+import org.apache.hadoop.hive.common.type.Timestamp;
 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;
@@ -56,7 +56,7 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.FuncSinDoubleToDoub
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.FuncSqrtDoubleToDouble;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.FuncTanDoubleToDouble;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
-import org.apache.hadoop.hive.serde2.io.TimestampWritable;
+import org.apache.hadoop.hive.serde2.io.TimestampWritableV2;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
 import org.junit.Test;
 
@@ -287,8 +287,8 @@ public class TestVectorMathFunctions {
     outV = new LongColumnVector(longValues.length);
     for (int i = 0; i < longValues.length; i++) {
       Timestamp randTimestamp = RandomTypeUtil.getRandTimestamp(r);
-      longValues[i] = TimestampWritable.getLong(randTimestamp);
-      inV.set(0, randTimestamp);
+      longValues[i] = TimestampWritableV2.getLong(randTimestamp);
+      inV.set(0, randTimestamp.toSqlTimestamp());
     }
 
     batch.cols[0] = inV;

http://git-wip-us.apache.org/repos/asf/hive/blob/ae008b79/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 f6dbd67..650bac4 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
@@ -26,13 +26,19 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Calendar;
 import java.util.Date;
-import java.util.List;
 import java.util.Random;
+import java.util.TimeZone;
 
 import junit.framework.Assert;
 
-import org.apache.commons.lang.ArrayUtils;
-import org.apache.hadoop.hive.common.type.RandomTypeUtil;
+import org.apache.hadoop.hive.ql.udf.UDFDayOfMonth;
+import org.apache.hadoop.hive.ql.udf.UDFHour;
+import org.apache.hadoop.hive.ql.udf.UDFMinute;
+import org.apache.hadoop.hive.ql.udf.UDFMonth;
+import org.apache.hadoop.hive.ql.udf.UDFSecond;
+import org.apache.hadoop.hive.ql.udf.UDFYear;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDF;
+import org.apache.hadoop.hive.serde2.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;
@@ -40,18 +46,13 @@ 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.metadata.HiveException;
-import org.apache.hadoop.hive.ql.udf.UDFDayOfMonth;
-import org.apache.hadoop.hive.ql.udf.UDFHour;
-import org.apache.hadoop.hive.ql.udf.UDFMinute;
-import org.apache.hadoop.hive.ql.udf.UDFMonth;
-import org.apache.hadoop.hive.ql.udf.UDFSecond;
 import org.apache.hadoop.hive.ql.udf.UDFWeekOfYear;
-import org.apache.hadoop.hive.ql.udf.UDFYear;
-import org.apache.hadoop.hive.serde2.io.TimestampWritable;
+import org.apache.hadoop.hive.serde2.io.TimestampWritableV2;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
 import org.apache.hadoop.io.IntWritable;
-import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
 import org.junit.Test;
 
@@ -60,7 +61,13 @@ import org.junit.Test;
  */
 public class TestVectorTimestampExpressions {
 
-  private SimpleDateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
+  private SimpleDateFormat dateFormat = getFormatter();
+
+  private static SimpleDateFormat getFormatter() {
+    SimpleDateFormat formatter = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
+    formatter.setTimeZone(TimeZone.getTimeZone("UTC"));
+    return formatter;
+  }
 
   private Timestamp[] getAllBoundaries(int minYear, int maxYear) {
      ArrayList<Timestamp> boundaries = new ArrayList<Timestamp>(1);
@@ -95,7 +102,7 @@ public class TestVectorTimestampExpressions {
     TimestampColumnVector tcv = new TimestampColumnVector(size);
     Random rand = new Random(seed);
     for (int i = 0; i < size; i++) {
-      tcv.set(i, RandomTypeUtil.getRandTimestamp(rand));
+      tcv.set(i, RandomTypeUtil.getRandTimestamp(rand).toSqlTimestamp());
     }
     batch.cols[0] = tcv;
     batch.cols[1] = new LongColumnVector(size);
@@ -109,7 +116,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(RandomTypeUtil.getRandTimestamp(rand));
+      byte[] encoded = encodeTime(RandomTypeUtil.getRandTimestamp(rand).toSqlTimestamp());
       bcv.vector[i] = encoded;
       bcv.start[i] = 0;
       bcv.length[i] = encoded.length;
@@ -224,14 +231,13 @@ public class TestVectorTimestampExpressions {
     TIMESTAMP_LONG, STRING_LONG
   }
 
-  private void compareToUDFYearLong(Timestamp t, int y) {
+  private void compareToUDFYearLong(Timestamp t, int y) throws HiveException {
     UDFYear udf = new UDFYear();
-    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);
-    }
+    udf.initialize(new ObjectInspector[]{PrimitiveObjectInspectorFactory.writableTimestampObjectInspector});
+    TimestampWritableV2 tsw = new TimestampWritableV2(
+        org.apache.hadoop.hive.common.type.Timestamp.ofEpochMilli(t.getTime(), t.getNanos()));
+    IntWritable res = (IntWritable) udf.evaluate(
+        new GenericUDF.DeferredObject[]{new GenericUDF.DeferredJavaObject(tsw)});
     Assert.assertEquals(res.get(), y);
   }
 
@@ -322,10 +328,13 @@ public class TestVectorTimestampExpressions {
     Assert.assertEquals(true, lcv.isNull[0]);
   }
 
-  private void compareToUDFDayOfMonthLong(Timestamp t, int y) {
+  private void compareToUDFDayOfMonthLong(Timestamp t, int y) throws HiveException {
     UDFDayOfMonth udf = new UDFDayOfMonth();
-    TimestampWritable tsw = new TimestampWritable(t);
-    IntWritable res = udf.evaluate(tsw);
+    udf.initialize(new ObjectInspector[]{PrimitiveObjectInspectorFactory.writableTimestampObjectInspector});
+    TimestampWritableV2 tsw = new TimestampWritableV2(
+        org.apache.hadoop.hive.common.type.Timestamp.ofEpochMilli(t.getTime(), t.getNanos()));
+    IntWritable res = (IntWritable) udf.evaluate(
+        new GenericUDF.DeferredObject[]{new GenericUDF.DeferredJavaObject(tsw)});
     Assert.assertEquals(res.get(), y);
   }
 
@@ -409,10 +418,13 @@ public class TestVectorTimestampExpressions {
     testVectorUDFDayOfMonth(TestType.STRING_LONG);
   }
 
-  private void compareToUDFHourLong(Timestamp t, int y) {
+  private void compareToUDFHourLong(Timestamp t, int y) throws HiveException {
     UDFHour udf = new UDFHour();
-    TimestampWritable tsw = new TimestampWritable(t);
-    IntWritable res = udf.evaluate(tsw);
+    udf.initialize(new ObjectInspector[]{PrimitiveObjectInspectorFactory.writableTimestampObjectInspector});
+    TimestampWritableV2 tsw = new TimestampWritableV2(
+        org.apache.hadoop.hive.common.type.Timestamp.ofEpochMilli(t.getTime(), t.getNanos()));
+    IntWritable res = (IntWritable) udf.evaluate(
+        new GenericUDF.DeferredObject[]{new GenericUDF.DeferredJavaObject(tsw)});
     Assert.assertEquals(res.get(), y);
   }
 
@@ -495,10 +507,13 @@ public class TestVectorTimestampExpressions {
     testVectorUDFHour(TestType.STRING_LONG);
   }
 
-  private void compareToUDFMinuteLong(Timestamp t, int y) {
+  private void compareToUDFMinuteLong(Timestamp t, int y) throws HiveException {
     UDFMinute udf = new UDFMinute();
-    TimestampWritable tsw = new TimestampWritable(t);
-    IntWritable res = udf.evaluate(tsw);
+    udf.initialize(new ObjectInspector[]{PrimitiveObjectInspectorFactory.writableTimestampObjectInspector});
+    TimestampWritableV2 tsw = new TimestampWritableV2(
+        org.apache.hadoop.hive.common.type.Timestamp.ofEpochMilli(t.getTime(), t.getNanos()));
+    IntWritable res = (IntWritable) udf.evaluate(
+        new GenericUDF.DeferredObject[]{new GenericUDF.DeferredJavaObject(tsw)});
     Assert.assertEquals(res.get(), y);
   }
 
@@ -582,10 +597,13 @@ public class TestVectorTimestampExpressions {
     testVectorUDFMinute(TestType.STRING_LONG);
   }
 
-  private void compareToUDFMonthLong(Timestamp t, int y) {
+  private void compareToUDFMonthLong(Timestamp t, int y) throws HiveException {
     UDFMonth udf = new UDFMonth();
-    TimestampWritable tsw = new TimestampWritable(t);
-    IntWritable res = udf.evaluate(tsw);
+    udf.initialize(new ObjectInspector[]{PrimitiveObjectInspectorFactory.writableTimestampObjectInspector});
+    TimestampWritableV2 tsw = new TimestampWritableV2(
+        org.apache.hadoop.hive.common.type.Timestamp.ofEpochMilli(t.getTime(), t.getNanos()));
+    IntWritable res = (IntWritable) udf.evaluate(
+        new GenericUDF.DeferredObject[]{new GenericUDF.DeferredJavaObject(tsw)});
     Assert.assertEquals(res.get(), y);
   }
 
@@ -668,10 +686,13 @@ public class TestVectorTimestampExpressions {
     testVectorUDFMonth(TestType.STRING_LONG);
   }
 
-  private void compareToUDFSecondLong(Timestamp t, int y) {
+  private void compareToUDFSecondLong(Timestamp t, int y) throws HiveException {
     UDFSecond udf = new UDFSecond();
-    TimestampWritable tsw = new TimestampWritable(t);
-    IntWritable res = udf.evaluate(tsw);
+    udf.initialize(new ObjectInspector[]{PrimitiveObjectInspectorFactory.writableTimestampObjectInspector});
+    TimestampWritableV2 tsw = new TimestampWritableV2(
+        org.apache.hadoop.hive.common.type.Timestamp.ofEpochMilli(t.getTime(), t.getNanos()));
+    IntWritable res = (IntWritable) udf.evaluate(
+        new GenericUDF.DeferredObject[]{new GenericUDF.DeferredJavaObject(tsw)});
     Assert.assertEquals(res.get(), y);
   }
 
@@ -844,7 +865,8 @@ public class TestVectorTimestampExpressions {
 
   private void compareToUDFWeekOfYearLong(Timestamp t, int y) {
     UDFWeekOfYear udf = new UDFWeekOfYear();
-    TimestampWritable tsw = new TimestampWritable(t);
+    TimestampWritableV2 tsw = new TimestampWritableV2(
+        org.apache.hadoop.hive.common.type.Timestamp.ofEpochMilli(t.getTime(), t.getNanos()));
     IntWritable res = udf.evaluate(tsw);
     Assert.assertEquals(res.get(), y);
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/ae008b79/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorTimestampExtract.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorTimestampExtract.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorTimestampExtract.java
index fa8f465..58e3fa3 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorTimestampExtract.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorTimestampExtract.java
@@ -18,32 +18,20 @@
 
 package org.apache.hadoop.hive.ql.exec.vector.expressions;
 
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-import java.util.Random;
-
+import junit.framework.Assert;
 import org.apache.hadoop.hive.common.type.DataTypePhysicalVariation;
-import org.apache.hadoop.hive.common.type.HiveChar;
-import org.apache.hadoop.hive.common.type.HiveVarchar;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluator;
 import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluatorFactory;
-import org.apache.hadoop.hive.ql.exec.UDF;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExtractRow;
 import org.apache.hadoop.hive.ql.exec.vector.VectorRandomBatchSource;
 import org.apache.hadoop.hive.ql.exec.vector.VectorRandomRowSource;
+import org.apache.hadoop.hive.ql.exec.vector.VectorRandomRowSource.GenerationSpec;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizationContext;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatchCtx;
-import org.apache.hadoop.hive.ql.exec.vector.VectorRandomRowSource.GenerationSpec;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.IdentityExpression;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.TestVectorDateAddSub.ColumnScalarMode;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
-import org.apache.hadoop.hive.ql.metadata.VirtualColumn;
 import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc;
-import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc;
 import org.apache.hadoop.hive.ql.udf.UDFDayOfMonth;
@@ -56,35 +44,24 @@ import org.apache.hadoop.hive.ql.udf.UDFWeekOfYear;
 import org.apache.hadoop.hive.ql.udf.UDFYear;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDF;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFBridge;
-import org.apache.hadoop.hive.ql.udf.generic.GenericUDFDateAdd;
-import org.apache.hadoop.hive.ql.udf.generic.GenericUDFDateSub;
-import org.apache.hadoop.hive.ql.udf.generic.GenericUDFIf;
-import org.apache.hadoop.hive.ql.udf.generic.GenericUDFWhen;
-import org.apache.hadoop.hive.serde2.io.DateWritable;
-import org.apache.hadoop.hive.serde2.io.DoubleWritable;
-import org.apache.hadoop.hive.serde2.io.HiveCharWritable;
-import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
-import org.apache.hadoop.hive.serde2.io.HiveVarcharWritable;
-import org.apache.hadoop.hive.serde2.io.TimestampWritable;
+import org.apache.hadoop.hive.serde2.io.DateWritableV2;
+import org.apache.hadoop.hive.serde2.io.TimestampWritableV2;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption;
 import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory;
-import org.apache.hadoop.hive.serde2.typeinfo.CharTypeInfo;
-import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
-import org.apache.hadoop.hive.serde2.typeinfo.VarcharTypeInfo;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.LongWritable;
-
-import junit.framework.Assert;
-
-import org.junit.Ignore;
 import org.junit.Test;
 
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Random;
+
 public class TestVectorTimestampExtract {
 
   @Test
@@ -108,14 +85,6 @@ public class TestVectorTimestampExtract {
     doTimestampExtractTests(random, "string");
   }
 
-  public enum TimestampExtractTestMode {
-    ROW_MODE,
-    ADAPTOR,
-    VECTOR_EXPRESSION;
-
-    static final int count = values().length;
-  }
-
   private void doTimestampExtractTests(Random random, String typeName)
       throws Exception {
 
@@ -192,13 +161,15 @@ public class TestVectorTimestampExtract {
       return;
     }
 
-    final UDF udf;
+    final GenericUDF udf;
     switch (extractFunctionName) {
     case "day":
       udf = new UDFDayOfMonth();
       break;
     case "dayofweek":
-      udf = new UDFDayOfWeek();
+      GenericUDFBridge dayOfWeekUDFBridge = new GenericUDFBridge();
+      dayOfWeekUDFBridge.setUdfClassName(UDFDayOfWeek.class.getName());
+      udf = dayOfWeekUDFBridge;
       break;
     case "hour":
       udf = new UDFHour();
@@ -213,7 +184,9 @@ public class TestVectorTimestampExtract {
       udf = new UDFSecond();
       break;
     case "yearweek":
-      udf = new UDFWeekOfYear();
+      GenericUDFBridge weekOfYearUDFBridge = new GenericUDFBridge();
+      weekOfYearUDFBridge.setUdfClassName(UDFWeekOfYear.class.getName());
+      udf = weekOfYearUDFBridge;
       break;
     case "year":
       udf = new UDFYear();
@@ -222,11 +195,8 @@ public class TestVectorTimestampExtract {
       throw new RuntimeException("Unexpected extract function name " + extractFunctionName);
     }
 
-    GenericUDFBridge genericUDFBridge = new GenericUDFBridge();
-    genericUDFBridge.setUdfClassName(udf.getClass().getName());
-
     ExprNodeGenericFuncDesc exprDesc =
-        new ExprNodeGenericFuncDesc(TypeInfoFactory.intTypeInfo, genericUDFBridge, children);
+        new ExprNodeGenericFuncDesc(TypeInfoFactory.intTypeInfo, udf, children);
 
     final int rowCount = randomRows.length;
     Object[][] resultObjectsArray = new Object[TimestampExtractTestMode.count][];
@@ -342,10 +312,10 @@ public class TestVectorTimestampExtract {
       Object result;
       switch (dateTimeStringPrimitiveCategory) {
       case TIMESTAMP:
-        result = evaluator.evaluate((TimestampWritable) object);
+        result = evaluator.evaluate((TimestampWritableV2) object);
         break;
       case DATE:
-        result = evaluator.evaluate((DateWritable) object);
+        result = evaluator.evaluate((DateWritableV2) object);
         break;
       case STRING:
         {
@@ -462,4 +432,12 @@ public class TestVectorTimestampExtract {
 
     return true;
   }
+
+  public enum TimestampExtractTestMode {
+    ROW_MODE,
+    ADAPTOR,
+    VECTOR_EXPRESSION;
+
+    static final int count = values().length;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/ae008b79/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 58ed151..4ed087e 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
@@ -22,12 +22,8 @@ import static org.junit.Assert.assertEquals;
 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;
 
@@ -35,7 +31,7 @@ import junit.framework.Assert;
 
 import org.apache.hadoop.hive.common.type.DataTypePhysicalVariation;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
-import org.apache.hadoop.hive.common.type.RandomTypeUtil;
+import org.apache.hadoop.hive.serde2.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;
@@ -43,12 +39,10 @@ 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.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.util.TimestampUtils;
-import org.apache.hadoop.hive.serde2.io.DateWritable;
-import org.apache.hadoop.hive.serde2.io.TimestampWritable;
-import org.apache.hadoop.hive.serde2.typeinfo.HiveDecimalUtils;
+import org.apache.hadoop.hive.serde2.io.DateWritableV2;
+import org.apache.hadoop.hive.serde2.io.TimestampWritableV2;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
 import org.junit.Test;
@@ -88,7 +82,7 @@ public class TestVectorTypeCasts {
     expr.evaluate(b);
     for (int i = 0; i < intValues.length; i++) {
       Timestamp timestamp = resultV.asScratchTimestamp(i);
-      long actual = DateWritable.millisToDays(timestamp.getTime());
+      long actual = DateWritableV2.millisToDays(timestamp.getTime());
       assertEquals(actual, intValues[i]);
     }
   }
@@ -155,7 +149,8 @@ public class TestVectorTypeCasts {
     expr.evaluate(b);
     for (int i = 0; i < longValues.length; i++) {
       Timestamp timestamp = resultV.asScratchTimestamp(i);
-      long actual = TimestampWritable.getLong(timestamp);
+      long actual = TimestampWritableV2.getLong(
+          org.apache.hadoop.hive.common.type.Timestamp.ofEpochMilli(timestamp.getTime(), timestamp.getNanos()));
       assertEquals(actual, longValues[i]);
     }
   }
@@ -518,7 +513,8 @@ public class TestVectorTypeCasts {
       Timestamp ts = new Timestamp(millis);
       int nanos = RandomTypeUtil.randomNanos(r);
       ts.setNanos(nanos);
-      TimestampWritable tsw = new TimestampWritable(ts);
+      TimestampWritableV2 tsw = new TimestampWritableV2(
+          org.apache.hadoop.hive.common.type.Timestamp.ofEpochMilli(ts.getTime(), ts.getNanos()));
       double asDouble = tsw.getDouble();
       doubleValues[i] = asDouble;
       HiveDecimal hiveDecimal = HiveDecimal.create(new BigDecimal(asDouble));
@@ -582,7 +578,8 @@ public class TestVectorTypeCasts {
       long millis = RandomTypeUtil.randomMillis(r);
       Timestamp ts = new Timestamp(millis);
       ts.setNanos(optionalNanos);
-      TimestampWritable tsw = new TimestampWritable(ts);
+      TimestampWritableV2 tsw = new TimestampWritableV2(
+          org.apache.hadoop.hive.common.type.Timestamp.ofEpochMilli(ts.getTime(), ts.getNanos()));
       hiveDecimalValues[i] = tsw.getHiveDecimal();
 
       tcv.set(i, ts);

http://git-wip-us.apache.org/repos/asf/hive/blob/ae008b79/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VerifyFastRow.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VerifyFastRow.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VerifyFastRow.java
index f257363..c908f66 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VerifyFastRow.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VerifyFastRow.java
@@ -19,8 +19,6 @@ package org.apache.hadoop.hive.ql.exec.vector.mapjoin.fast;
 
 import java.io.IOException;
 import java.nio.charset.StandardCharsets;
-import java.sql.Date;
-import java.sql.Timestamp;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.LinkedHashMap;
@@ -29,15 +27,17 @@ import java.util.Map;
 
 import junit.framework.TestCase;
 
+import org.apache.hadoop.hive.common.type.Date;
 import org.apache.hadoop.hive.common.type.HiveChar;
 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.Timestamp;
 import org.apache.hadoop.hive.serde2.fast.DeserializeRead;
 import org.apache.hadoop.hive.serde2.fast.SerializeWrite;
 import org.apache.hadoop.hive.serde2.io.ByteWritable;
-import org.apache.hadoop.hive.serde2.io.DateWritable;
+import org.apache.hadoop.hive.serde2.io.DateWritableV2;
 import org.apache.hadoop.hive.serde2.io.DoubleWritable;
 import org.apache.hadoop.hive.serde2.io.HiveCharWritable;
 import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
@@ -45,7 +45,7 @@ import org.apache.hadoop.hive.serde2.io.HiveIntervalDayTimeWritable;
 import org.apache.hadoop.hive.serde2.io.HiveIntervalYearMonthWritable;
 import org.apache.hadoop.hive.serde2.io.HiveVarcharWritable;
 import org.apache.hadoop.hive.serde2.io.ShortWritable;
-import org.apache.hadoop.hive.serde2.io.TimestampWritable;
+import org.apache.hadoop.hive.serde2.io.TimestampWritableV2;
 import org.apache.hadoop.hive.serde2.objectinspector.StandardUnionObjectInspector;
 import org.apache.hadoop.hive.serde2.typeinfo.CharTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo;
@@ -63,7 +63,6 @@ import org.apache.hadoop.io.FloatWritable;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
 
 /**
  * (Copy of VerifyFast from serde).
@@ -245,7 +244,7 @@ public class VerifyFastRow {
         case DATE:
           {
             Date value = deserializeRead.currentDateWritable.get();
-            Date expected = ((DateWritable) object).get();
+            Date expected = ((DateWritableV2) object).get();
             if (!value.equals(expected)) {
               TestCase.fail("Date field mismatch (expected " + expected.toString() + " found " + value.toString() + ")");
             }
@@ -254,7 +253,7 @@ public class VerifyFastRow {
         case TIMESTAMP:
           {
             Timestamp value = deserializeRead.currentTimestampWritable.getTimestamp();
-            Timestamp expected = ((TimestampWritable) object).getTimestamp();
+            Timestamp expected = ((TimestampWritableV2) object).getTimestamp();
             if (!value.equals(expected)) {
               TestCase.fail("Timestamp field mismatch (expected " + expected.toString() + " found " + value.toString() + ")");
             }
@@ -395,13 +394,13 @@ public class VerifyFastRow {
           break;
         case DATE:
           {
-            Date value = ((DateWritable) object).get();
+            Date value = ((DateWritableV2) object).get();
             serializeWrite.writeDate(value);
           }
           break;
         case TIMESTAMP:
           {
-            Timestamp value = ((TimestampWritable) object).getTimestamp();
+            Timestamp value = ((TimestampWritableV2) object).getTimestamp();
             serializeWrite.writeTimestamp(value);
           }
           break;
@@ -572,9 +571,9 @@ public class VerifyFastRow {
     case DECIMAL:
       return new HiveDecimalWritable(deserializeRead.currentHiveDecimalWritable);
     case DATE:
-      return new DateWritable(deserializeRead.currentDateWritable);
+      return new DateWritableV2(deserializeRead.currentDateWritable);
     case TIMESTAMP:
-      return new TimestampWritable(deserializeRead.currentTimestampWritable);
+      return new TimestampWritableV2(deserializeRead.currentTimestampWritable);
     case INTERVAL_YEAR_MONTH:
       return new HiveIntervalYearMonthWritable(deserializeRead.currentHiveIntervalYearMonthWritable);
     case INTERVAL_DAY_TIME:

http://git-wip-us.apache.org/repos/asf/hive/blob/ae008b79/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 3fe8b09..878d286 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,7 @@ import java.sql.Timestamp;
 import java.util.Random;
 
 import org.apache.hadoop.hive.common.type.HiveDecimal;
-import org.apache.hadoop.hive.common.type.RandomTypeUtil;
+import org.apache.hadoop.hive.serde2.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;
@@ -86,7 +86,7 @@ public class VectorizedRowGroupGenUtil {
     tcv.noNulls = !nulls;
     tcv.isRepeating = repeating;
 
-    Timestamp repeatingTimestamp = RandomTypeUtil.getRandTimestamp(rand);
+    Timestamp repeatingTimestamp = RandomTypeUtil.getRandTimestamp(rand).toSqlTimestamp();
 
     int nullFrequency = generateNullFrequency(rand);
 
@@ -98,7 +98,7 @@ public class VectorizedRowGroupGenUtil {
       }else {
         tcv.isNull[i] = false;
         if (!repeating) {
-          Timestamp randomTimestamp = RandomTypeUtil.getRandTimestamp(rand);
+          Timestamp randomTimestamp = RandomTypeUtil.getRandTimestamp(rand).toSqlTimestamp();
           tcv.set(i,  randomTimestamp);
           timestampValues[i] = randomTimestamp;
         } else {

http://git-wip-us.apache.org/repos/asf/hive/blob/ae008b79/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/util/batchgen/VectorColumnGroupGenerator.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/util/batchgen/VectorColumnGroupGenerator.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/util/batchgen/VectorColumnGroupGenerator.java
index 1064b19..9bf9d9d 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/util/batchgen/VectorColumnGroupGenerator.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/util/batchgen/VectorColumnGroupGenerator.java
@@ -22,7 +22,7 @@ import java.sql.Timestamp;
 import java.util.Arrays;
 import java.util.Random;
 
-import org.apache.hadoop.hive.common.type.RandomTypeUtil;
+import org.apache.hadoop.hive.serde2.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;
@@ -31,8 +31,6 @@ 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.util.batchgen.VectorBatchGenerator.GenerateType;
 import org.apache.hadoop.hive.ql.exec.vector.util.batchgen.VectorBatchGenerator.GenerateType.GenerateCategory;
-import org.apache.hadoop.io.BooleanWritable;
-import org.apache.hadoop.io.Text;
 
 public class VectorColumnGroupGenerator {
 
@@ -232,7 +230,7 @@ public class VectorColumnGroupGenerator {
 
     case TIMESTAMP:
       {
-        Timestamp value = RandomTypeUtil.getRandTimestamp(random);
+        Timestamp value = RandomTypeUtil.getRandTimestamp(random).toSqlTimestamp();
         ((Timestamp[]) array)[rowIndex] = value;
       }
       break;

http://git-wip-us.apache.org/repos/asf/hive/blob/ae008b79/ql/src/test/org/apache/hadoop/hive/ql/io/arrow/TestArrowColumnarBatchSerDe.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/arrow/TestArrowColumnarBatchSerDe.java b/ql/src/test/org/apache/hadoop/hive/ql/io/arrow/TestArrowColumnarBatchSerDe.java
index ce25c3e..c9a5812 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/arrow/TestArrowColumnarBatchSerDe.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/io/arrow/TestArrowColumnarBatchSerDe.java
@@ -26,13 +26,14 @@ 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.Timestamp;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.serde.serdeConstants;
 import org.apache.hadoop.hive.serde2.AbstractSerDe;
 import org.apache.hadoop.hive.serde2.SerDeException;
 import org.apache.hadoop.hive.serde2.SerDeUtils;
 import org.apache.hadoop.hive.serde2.io.ByteWritable;
-import org.apache.hadoop.hive.serde2.io.DateWritable;
+import org.apache.hadoop.hive.serde2.io.DateWritableV2;
 import org.apache.hadoop.hive.serde2.io.DoubleWritable;
 import org.apache.hadoop.hive.serde2.io.HiveCharWritable;
 import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
@@ -40,7 +41,7 @@ import org.apache.hadoop.hive.serde2.io.HiveIntervalDayTimeWritable;
 import org.apache.hadoop.hive.serde2.io.HiveIntervalYearMonthWritable;
 import org.apache.hadoop.hive.serde2.io.HiveVarcharWritable;
 import org.apache.hadoop.hive.serde2.io.ShortWritable;
-import org.apache.hadoop.hive.serde2.io.TimestampWritable;
+import org.apache.hadoop.hive.serde2.io.TimestampWritableV2;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.StructField;
@@ -57,7 +58,6 @@ import org.apache.hadoop.io.Text;
 import org.junit.Before;
 import org.junit.Test;
 
-import java.sql.Timestamp;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
@@ -111,20 +111,20 @@ public class TestArrowColumnarBatchSerDe {
   private final static Timestamp NEGATIVE_TIMESTAMP_WITHOUT_NANOS;
 
   static {
-    TIMESTAMP = new Timestamp(TIME_IN_MILLIS);
-    NEGATIVE_TIMESTAMP_WITHOUT_NANOS = new Timestamp(NEGATIVE_TIME_IN_MILLIS);
+    TIMESTAMP = Timestamp.ofEpochMilli(TIME_IN_MILLIS);
+    NEGATIVE_TIMESTAMP_WITHOUT_NANOS = Timestamp.ofEpochMilli(NEGATIVE_TIME_IN_MILLIS);
   }
 
   private final static Object[][] DTI_ROWS = {
       {
-          new DateWritable(DateWritable.millisToDays(TIME_IN_MILLIS)),
-          new TimestampWritable(TIMESTAMP),
+          new DateWritableV2(DateWritableV2.millisToDays(TIME_IN_MILLIS)),
+          new TimestampWritableV2(TIMESTAMP),
           new HiveIntervalYearMonthWritable(new HiveIntervalYearMonth(1, 2)),
           new HiveIntervalDayTimeWritable(new HiveIntervalDayTime(1, 2, 3, 4, 5_000_000))
       },
       {
-          new DateWritable(DateWritable.millisToDays(NEGATIVE_TIME_IN_MILLIS)),
-          new TimestampWritable(NEGATIVE_TIMESTAMP_WITHOUT_NANOS),
+          new DateWritableV2(DateWritableV2.millisToDays(NEGATIVE_TIME_IN_MILLIS)),
+          new TimestampWritableV2(NEGATIVE_TIMESTAMP_WITHOUT_NANOS),
           null,
           null
       },
@@ -364,10 +364,10 @@ public class TestArrowColumnarBatchSerDe {
                         newArrayList(text("hello")),
                         input -> text(input.toString().toUpperCase())),
                     intW(0))), // c16:array<struct<m:map<string,string>,n:int>>
-            new TimestampWritable(TIMESTAMP), // c17:timestamp
+            new TimestampWritableV2(TIMESTAMP), // c17:timestamp
             decimalW(HiveDecimal.create(0, 0)), // c18:decimal(16,7)
             new BytesWritable("Hello".getBytes()), // c19:binary
-            new DateWritable(123), // c20:date
+            new DateWritableV2(123), // c20:date
             varcharW("x", 20), // c21:varchar(20)
             charW("y", 15), // c22:char(15)
             new BytesWritable("world!".getBytes()), // c23:binary
@@ -508,9 +508,9 @@ public class TestArrowColumnarBatchSerDe {
     Object[][] rows = new Object[size][];
     for (int i = 0; i < size; i++) {
       long millis = ((long) rand.nextInt(Integer.MAX_VALUE)) * 1000;
-      Timestamp timestamp = new Timestamp(rand.nextBoolean() ? millis : -millis);
+      Timestamp timestamp = Timestamp.ofEpochMilli(rand.nextBoolean() ? millis : -millis);
       timestamp.setNanos(rand.nextInt(1000) * 1000);
-      rows[i] = new Object[] {new TimestampWritable(timestamp)};
+      rows[i] = new Object[] {new TimestampWritableV2(timestamp)};
     }
 
     initAndSerializeAndDeserialize(schema, rows);

http://git-wip-us.apache.org/repos/asf/hive/blob/ae008b79/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 fe475f6..208aeb5 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
@@ -28,8 +28,6 @@ import java.net.URISyntaxException;
 import java.nio.ByteBuffer;
 import java.nio.charset.StandardCharsets;
 import java.security.PrivilegedExceptionAction;
-import java.sql.Date;
-import java.sql.Timestamp;
 import java.text.SimpleDateFormat;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -50,7 +48,9 @@ import org.apache.hadoop.fs.*;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hive.common.ValidTxnList;
 import org.apache.hadoop.hive.common.ValidWriteIdList;
+import org.apache.hadoop.hive.common.type.Date;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
+import org.apache.hadoop.hive.common.type.Timestamp;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
@@ -181,8 +181,8 @@ public class TestInputOutputFormat {
       decimalValue = HiveDecimal.create(x);
       long millisUtc = x * MILLIS_IN_DAY;
       millisUtc -= LOCAL_TIMEZONE.getOffset(millisUtc);
-      dateValue = new Date(millisUtc);
-      timestampValue = new Timestamp(millisUtc);
+      dateValue = Date.ofEpochMilli(millisUtc);
+      timestampValue = Timestamp.ofEpochMilli(millisUtc);
     }
 
     @Override