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/22 17:46:20 UTC

[04/35] hive git commit: HIVE-12192: 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/c95136a0/serde/src/java/org/apache/hadoop/hive/serde2/io/TimestampWritableV2.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/io/TimestampWritableV2.java b/serde/src/java/org/apache/hadoop/hive/serde2/io/TimestampWritableV2.java
new file mode 100644
index 0000000..9aa7f19
--- /dev/null
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/io/TimestampWritableV2.java
@@ -0,0 +1,625 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.serde2.io;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.time.format.DateTimeFormatter;
+
+import org.apache.hadoop.hive.common.type.HiveDecimal;
+import org.apache.hadoop.hive.common.type.Timestamp;
+import org.apache.hadoop.hive.common.type.TimestampUtils;
+import org.apache.hadoop.hive.serde2.ByteStream.RandomAccessOutput;
+import org.apache.hadoop.hive.serde2.lazybinary.LazyBinaryUtils;
+import org.apache.hadoop.hive.serde2.lazybinary.LazyBinaryUtils.VInt;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.io.WritableUtils;
+
+/**
+ * TimestampWritableV2
+ *
+ * Timestamps are of the format
+ *    YYYY-MM-DD HH:MM:SS.[fff...]
+ *
+ * We encode Unix timestamp in seconds in 4 bytes, using the MSB to signify
+ * whether the timestamp has a fractional portion.
+ *
+ * The fractional portion is reversed, and encoded as a VInt
+ * so timestamps with less precision use fewer bytes.
+ *
+ *      0.1    -> 1
+ *      0.01   -> 10
+ *      0.001  -> 100
+ *
+ */
+public class TimestampWritableV2 implements WritableComparable<TimestampWritableV2> {
+
+  static final public byte[] nullBytes = {0x0, 0x0, 0x0, 0x0};
+
+  private static final int DECIMAL_OR_SECOND_VINT_FLAG = 0x80000000;
+  private static final int LOWEST_31_BITS_OF_SEC_MASK = 0x7fffffff;
+
+  private static final long SEVEN_BYTE_LONG_SIGN_FLIP = 0xff80L << 48;
+
+
+  /** The maximum number of bytes required for a TimestampWritableV2 */
+  public static final int MAX_BYTES = 13;
+
+  public static final int BINARY_SORTABLE_LENGTH = 11;
+
+  public static final DateTimeFormatter DATE_TIME_FORMAT = DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss");
+
+  private Timestamp timestamp = new Timestamp();
+
+  /**
+   * true if data is stored in timestamp field rather than byte arrays.
+   *      allows for lazy conversion to bytes when necessary
+   * false otherwise
+   */
+  private boolean bytesEmpty;
+  private boolean timestampEmpty;
+
+  /* Allow use of external byte[] for efficiency */
+  private byte[] currentBytes;
+  private final byte[] internalBytes = new byte[MAX_BYTES];
+  private byte[] externalBytes;
+  private int offset;
+
+  /* Constructors */
+  public TimestampWritableV2() {
+    bytesEmpty = false;
+    currentBytes = internalBytes;
+    offset = 0;
+
+    clearTimestamp();
+  }
+
+  public TimestampWritableV2(byte[] bytes, int offset) {
+    set(bytes, offset);
+  }
+
+  public TimestampWritableV2(TimestampWritableV2 t) {
+    this(t.getBytes(), 0);
+  }
+
+  public TimestampWritableV2(Timestamp t) {
+    set(t);
+  }
+
+  public void set(byte[] bytes, int offset) {
+    externalBytes = bytes;
+    this.offset = offset;
+    bytesEmpty = false;
+    currentBytes = externalBytes;
+
+    clearTimestamp();
+  }
+
+  public void set(Timestamp t) {
+    if (t == null) {
+      timestamp.set(null);
+      return;
+    }
+    timestamp.set(t);
+    bytesEmpty = true;
+    timestampEmpty = false;
+  }
+
+  public void set(TimestampWritableV2 t) {
+    if (t.bytesEmpty) {
+      set(t.getTimestamp());
+      return;
+    }
+    if (t.currentBytes == t.externalBytes) {
+      set(t.currentBytes, t.offset);
+    } else {
+      set(t.currentBytes, 0);
+    }
+  }
+
+  public static void updateTimestamp(Timestamp timestamp, long secondsAsMillis, int nanos) {
+    timestamp.setTimeInMillis(secondsAsMillis, nanos);
+  }
+
+  public void setInternal(long secondsAsMillis, int nanos) {
+    // This is our way of documenting that we are MUTATING the contents of
+    // this writable's internal timestamp.
+    updateTimestamp(timestamp, secondsAsMillis, nanos);
+
+    bytesEmpty = true;
+    timestampEmpty = false;
+  }
+
+  private void clearTimestamp() {
+    timestampEmpty = true;
+  }
+
+  public void writeToByteStream(RandomAccessOutput byteStream) {
+    checkBytes();
+    byteStream.write(currentBytes, offset, getTotalLength());
+  }
+
+  /**
+   *
+   * @return seconds corresponding to this TimestampWritableV2
+   */
+  public long getSeconds() {
+    if (!timestampEmpty) {
+      return timestamp.toEpochSecond();
+    } else if (!bytesEmpty) {
+      return TimestampWritableV2.getSeconds(currentBytes, offset);
+    } else {
+      throw new IllegalStateException("Both timestamp and bytes are empty");
+    }
+  }
+
+  /**
+   *
+   * @return nanoseconds in this TimestampWritableV2
+   */
+  public int getNanos() {
+    if (!timestampEmpty) {
+      return timestamp.getNanos();
+    } else if (!bytesEmpty) {
+      return hasDecimalOrSecondVInt() ?
+          TimestampWritableV2.getNanos(currentBytes, offset + 4) : 0;
+    } else {
+      throw new IllegalStateException("Both timestamp and bytes are empty");
+    }
+  }
+
+  /**
+   * @return length of serialized TimestampWritableV2 data. As a side effect, populates the internal
+   *         byte array if empty.
+   */
+  int getTotalLength() {
+    checkBytes();
+    return getTotalLength(currentBytes, offset);
+  }
+
+  public static int getTotalLength(byte[] bytes, int offset) {
+    int len = 4;
+    if (hasDecimalOrSecondVInt(bytes[offset])) {
+      int firstVIntLen = WritableUtils.decodeVIntSize(bytes[offset + 4]);
+      len += firstVIntLen;
+      if (hasSecondVInt(bytes[offset + 4])) {
+        len += WritableUtils.decodeVIntSize(bytes[offset + 4 + firstVIntLen]);
+      }
+    }
+    return len;
+  }
+
+  public Timestamp getTimestamp() {
+    if (timestampEmpty) {
+      populateTimestamp();
+    }
+    return timestamp;
+  }
+
+  /**
+   * Used to create copies of objects
+   * @return a copy of the internal TimestampWritableV2 byte[]
+   */
+  public byte[] getBytes() {
+    checkBytes();
+
+    int len = getTotalLength();
+    byte[] b = new byte[len];
+
+    System.arraycopy(currentBytes, offset, b, 0, len);
+    return b;
+  }
+
+  /**
+   * @return byte[] representation of TimestampWritableV2 that is binary
+   * sortable (7 bytes for seconds, 4 bytes for nanoseconds)
+   */
+  public byte[] getBinarySortable() {
+    byte[] b = new byte[BINARY_SORTABLE_LENGTH];
+    int nanos = getNanos();
+    // We flip the highest-order bit of the seven-byte representation of seconds to make negative
+    // values come before positive ones.
+    long seconds = getSeconds() ^ SEVEN_BYTE_LONG_SIGN_FLIP;
+    sevenByteLongToBytes(seconds, b, 0);
+    intToBytes(nanos, b, 7);
+    return b;
+  }
+
+  /**
+   * Given a byte[] that has binary sortable data, initialize the internal
+   * structures to hold that data
+   * @param bytes the byte array that holds the binary sortable representation
+   * @param binSortOffset offset of the binary-sortable representation within the buffer.
+   */
+  public void setBinarySortable(byte[] bytes, int binSortOffset) {
+    // Flip the sign bit (and unused bits of the high-order byte) of the seven-byte long back.
+    long seconds = readSevenByteLong(bytes, binSortOffset) ^ SEVEN_BYTE_LONG_SIGN_FLIP;
+    int nanos = bytesToInt(bytes, binSortOffset + 7);
+    int firstInt = (int) seconds;
+    boolean hasSecondVInt = seconds < 0 || seconds > Integer.MAX_VALUE;
+    if (nanos != 0 || hasSecondVInt) {
+      firstInt |= DECIMAL_OR_SECOND_VINT_FLAG;
+    } else {
+      firstInt &= LOWEST_31_BITS_OF_SEC_MASK;
+    }
+
+    intToBytes(firstInt, internalBytes, 0);
+    setNanosBytes(nanos, internalBytes, 4, hasSecondVInt);
+    if (hasSecondVInt) {
+      LazyBinaryUtils.writeVLongToByteArray(internalBytes,
+          4 + WritableUtils.decodeVIntSize(internalBytes[4]),
+          seconds >> 31);
+    }
+
+    currentBytes = internalBytes;
+    this.offset = 0;
+  }
+
+  /**
+   * The data of TimestampWritableV2 can be stored either in a byte[]
+   * or in a Timestamp object. Calling this method ensures that the byte[]
+   * is populated from the Timestamp object if previously empty.
+   */
+  private void checkBytes() {
+    if (bytesEmpty) {
+      // Populate byte[] from Timestamp
+      convertTimestampToBytes(timestamp, internalBytes, 0);
+      offset = 0;
+      currentBytes = internalBytes;
+      bytesEmpty = false;
+    }
+  }
+
+  /**
+   *
+   * @return double representation of the timestamp, accurate to nanoseconds
+   */
+  public double getDouble() {
+    double seconds, nanos;
+    if (bytesEmpty) {
+      seconds = timestamp.toEpochSecond();
+      nanos = timestamp.getNanos();
+    } else {
+      seconds = getSeconds();
+      nanos = getNanos();
+    }
+    return seconds + nanos / 1000000000;
+  }
+
+  public static long getLong(Timestamp timestamp) {
+    return timestamp.toEpochSecond();
+  }
+
+  public void readFields(DataInput in) throws IOException {
+    in.readFully(internalBytes, 0, 4);
+    if (TimestampWritableV2.hasDecimalOrSecondVInt(internalBytes[0])) {
+      in.readFully(internalBytes, 4, 1);
+      int len = (byte) WritableUtils.decodeVIntSize(internalBytes[4]);
+      if (len > 1) {
+        in.readFully(internalBytes, 5, len-1);
+      }
+
+      long vlong = LazyBinaryUtils.readVLongFromByteArray(internalBytes, 4);
+      if (vlong < -1000000000 || vlong > 999999999) {
+        throw new IOException(
+            "Invalid first vint value (encoded nanoseconds) of a TimestampWritableV2: " + vlong +
+            ", expected to be between -1000000000 and 999999999.");
+        // Note that -1000000000 is a valid value corresponding to a nanosecond timestamp
+        // of 999999999, because if the second VInt is present, we use the value
+        // (-reversedNanoseconds - 1) as the second VInt.
+      }
+      if (vlong < 0) {
+        // This indicates there is a second VInt containing the additional bits of the seconds
+        // field.
+        in.readFully(internalBytes, 4 + len, 1);
+        int secondVIntLen = (byte) WritableUtils.decodeVIntSize(internalBytes[4 + len]);
+        if (secondVIntLen > 1) {
+          in.readFully(internalBytes, 5 + len, secondVIntLen - 1);
+        }
+      }
+    }
+    currentBytes = internalBytes;
+    this.offset = 0;
+  }
+
+  public void write(DataOutput out) throws IOException {
+    checkBytes();
+    out.write(currentBytes, offset, getTotalLength());
+  }
+
+  public int compareTo(TimestampWritableV2 t) {
+    checkBytes();
+    long s1 = this.getSeconds();
+    long s2 = t.getSeconds();
+    if (s1 == s2) {
+      int n1 = this.getNanos();
+      int n2 = t.getNanos();
+      if (n1 == n2) {
+        return 0;
+      }
+      return n1 - n2;
+    } else {
+      return s1 < s2 ? -1 : 1;
+    }
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    return compareTo((TimestampWritableV2) o) == 0;
+  }
+
+  @Override
+  public String toString() {
+    if (timestampEmpty) {
+      populateTimestamp();
+    }
+
+    if (timestamp.getNanos() > 0) {
+      return timestamp.toString();
+    }
+
+    String timestampString = timestamp.toString();
+    if (timestampString.length() > 19) {
+      if (timestampString.length() == 21) {
+        if (timestampString.substring(19).compareTo(".0") == 0) {
+          return timestamp.format(DATE_TIME_FORMAT);
+        }
+      }
+      return timestamp.format(DATE_TIME_FORMAT) + timestampString.substring(19);
+    }
+
+    return timestamp.format(DATE_TIME_FORMAT);
+  }
+
+  @Override
+  public int hashCode() {
+    long seconds = getSeconds();
+    seconds <<= 30;  // the nanosecond part fits in 30 bits
+    seconds |= getNanos();
+    return (int) ((seconds >>> 32) ^ seconds);
+  }
+
+  private void populateTimestamp() {
+    long seconds = getSeconds();
+    int nanos = getNanos();
+    timestamp.setTimeInSeconds(seconds, nanos);
+  }
+
+  /** Static methods **/
+
+  /**
+   * Gets seconds stored as integer at bytes[offset]
+   * @param bytes
+   * @param offset
+   * @return the number of seconds
+   */
+  public static long getSeconds(byte[] bytes, int offset) {
+    int lowest31BitsOfSecondsAndFlag = bytesToInt(bytes, offset);
+    if (lowest31BitsOfSecondsAndFlag >= 0 ||  // the "has decimal or second VInt" flag is not set
+        !hasSecondVInt(bytes[offset + 4])) {
+      // The entire seconds field is stored in the first 4 bytes.
+      return lowest31BitsOfSecondsAndFlag & LOWEST_31_BITS_OF_SEC_MASK;
+    }
+
+    // We compose the seconds field from two parts. The lowest 31 bits come from the first four
+    // bytes. The higher-order bits come from the second VInt that follows the nanos field.
+    return ((long) (lowest31BitsOfSecondsAndFlag & LOWEST_31_BITS_OF_SEC_MASK)) |
+           (LazyBinaryUtils.readVLongFromByteArray(bytes,
+               offset + 4 + WritableUtils.decodeVIntSize(bytes[offset + 4])) << 31);
+  }
+
+  public static int getNanos(byte[] bytes, int offset) {
+    VInt vInt = LazyBinaryUtils.threadLocalVInt.get();
+    LazyBinaryUtils.readVInt(bytes, offset, vInt);
+    int val = vInt.value;
+    if (val < 0) {
+      // This means there is a second VInt present that specifies additional bits of the timestamp.
+      // The reversed nanoseconds value is still encoded in this VInt.
+      val = -val - 1;
+    }
+    int len = (int) Math.floor(Math.log10(val)) + 1;
+
+    // Reverse the value
+    int tmp = 0;
+    while (val != 0) {
+      tmp *= 10;
+      tmp += val % 10;
+      val /= 10;
+    }
+    val = tmp;
+
+    if (len < 9) {
+      val *= Math.pow(10, 9 - len);
+    }
+    return val;
+  }
+
+  /**
+   * Writes a Timestamp's serialized value to byte array b at the given offset
+   * @param t to convert to bytes
+   * @param b destination byte array
+   * @param offset destination offset in the byte array
+   */
+  public static void convertTimestampToBytes(Timestamp t, byte[] b,
+      int offset) {
+    long seconds = t.toEpochSecond();
+    int nanos = t.getNanos();
+
+    boolean hasSecondVInt = seconds < 0 || seconds > Integer.MAX_VALUE;
+    boolean hasDecimal = setNanosBytes(nanos, b, offset+4, hasSecondVInt);
+
+    int firstInt = (int) seconds;
+    if (hasDecimal || hasSecondVInt) {
+      firstInt |= DECIMAL_OR_SECOND_VINT_FLAG;
+    } else {
+      firstInt &= LOWEST_31_BITS_OF_SEC_MASK;
+    }
+    intToBytes(firstInt, b, offset);
+
+    if (hasSecondVInt) {
+      LazyBinaryUtils.writeVLongToByteArray(b,
+          offset + 4 + WritableUtils.decodeVIntSize(b[offset + 4]),
+          seconds >> 31);
+    }
+  }
+
+  /**
+   * Given an integer representing nanoseconds, write its serialized
+   * value to the byte array b at offset
+   *
+   * @param nanos
+   * @param b
+   * @param offset
+   * @return
+   */
+  private static boolean setNanosBytes(int nanos, byte[] b, int offset, boolean hasSecondVInt) {
+    int decimal = 0;
+    if (nanos != 0) {
+      int counter = 0;
+      while (counter < 9) {
+        decimal *= 10;
+        decimal += nanos % 10;
+        nanos /= 10;
+        counter++;
+      }
+    }
+
+    if (hasSecondVInt || decimal != 0) {
+      // We use the sign of the reversed-nanoseconds field to indicate that there is a second VInt
+      // present.
+      LazyBinaryUtils.writeVLongToByteArray(b, offset, hasSecondVInt ? (-decimal - 1) : decimal);
+    }
+    return decimal != 0;
+  }
+
+  public HiveDecimal getHiveDecimal() {
+    if (timestampEmpty) {
+      populateTimestamp();
+    }
+    return getHiveDecimal(timestamp);
+  }
+
+  public static HiveDecimal getHiveDecimal(Timestamp timestamp) {
+    // The BigDecimal class recommends not converting directly from double to BigDecimal,
+    // so we convert through a string...
+    Double timestampDouble = TimestampUtils.getDouble(timestamp);
+    HiveDecimal result = HiveDecimal.create(timestampDouble.toString());
+    return result;
+  }
+
+  /**
+   * Converts the time in seconds or milliseconds to a timestamp.
+   * @param time time in seconds or in milliseconds
+   * @return the timestamp
+   */
+  public static Timestamp longToTimestamp(long time, boolean intToTimestampInSeconds) {
+    // If the time is in seconds, converts it to milliseconds first.
+    if (intToTimestampInSeconds) {
+      return Timestamp.ofEpochSecond(time);
+    }
+    return Timestamp.ofEpochMilli(time);
+  }
+
+  public static void setTimestamp(Timestamp t, byte[] bytes, int offset) {
+    long seconds = getSeconds(bytes, offset);
+    int nanos;
+    if (hasDecimalOrSecondVInt(bytes[offset])) {
+      nanos = getNanos(bytes, offset + 4);
+    } else {
+      nanos = 0;
+    }
+    t.setTimeInSeconds(seconds, nanos);
+  }
+
+  public static Timestamp createTimestamp(byte[] bytes, int offset) {
+    Timestamp t = new Timestamp();
+    TimestampWritableV2.setTimestamp(t, bytes, offset);
+    return t;
+  }
+
+  private static boolean hasDecimalOrSecondVInt(byte b) {
+    return (b >> 7) != 0;
+  }
+
+  private static boolean hasSecondVInt(byte b) {
+    return WritableUtils.isNegativeVInt(b);
+  }
+
+  private final boolean hasDecimalOrSecondVInt() {
+    return hasDecimalOrSecondVInt(currentBytes[offset]);
+  }
+
+  public final boolean hasDecimal() {
+    return hasDecimalOrSecondVInt() || currentBytes[offset + 4] != -1;
+    // If the first byte of the VInt is -1, the VInt itself is -1, indicating that there is a
+    // second VInt but the nanoseconds field is actually 0.
+  }
+
+  /**
+   * Writes <code>value</code> into <code>dest</code> at <code>offset</code>
+   * @param value
+   * @param dest
+   * @param offset
+   */
+  private static void intToBytes(int value, byte[] dest, int offset) {
+    dest[offset] = (byte) ((value >> 24) & 0xFF);
+    dest[offset+1] = (byte) ((value >> 16) & 0xFF);
+    dest[offset+2] = (byte) ((value >> 8) & 0xFF);
+    dest[offset+3] = (byte) (value & 0xFF);
+  }
+
+  /**
+   * Writes <code>value</code> into <code>dest</code> at <code>offset</code> as a seven-byte
+   * serialized long number.
+   */
+  static void sevenByteLongToBytes(long value, byte[] dest, int offset) {
+    dest[offset] = (byte) ((value >> 48) & 0xFF);
+    dest[offset+1] = (byte) ((value >> 40) & 0xFF);
+    dest[offset+2] = (byte) ((value >> 32) & 0xFF);
+    dest[offset+3] = (byte) ((value >> 24) & 0xFF);
+    dest[offset+4] = (byte) ((value >> 16) & 0xFF);
+    dest[offset+5] = (byte) ((value >> 8) & 0xFF);
+    dest[offset+6] = (byte) (value & 0xFF);
+  }
+
+  /**
+   *
+   * @param bytes
+   * @param offset
+   * @return integer represented by the four bytes in <code>bytes</code>
+   *  beginning at <code>offset</code>
+   */
+  private static int bytesToInt(byte[] bytes, int offset) {
+    return ((0xFF & bytes[offset]) << 24)
+        | ((0xFF & bytes[offset+1]) << 16)
+        | ((0xFF & bytes[offset+2]) << 8)
+        | (0xFF & bytes[offset+3]);
+  }
+
+  static long readSevenByteLong(byte[] bytes, int offset) {
+    // We need to shift everything 8 bits left and then shift back to populate the sign field.
+    return (((0xFFL & bytes[offset]) << 56)
+        | ((0xFFL & bytes[offset+1]) << 48)
+        | ((0xFFL & bytes[offset+2]) << 40)
+        | ((0xFFL & bytes[offset+3]) << 32)
+        | ((0xFFL & bytes[offset+4]) << 24)
+        | ((0xFFL & bytes[offset+5]) << 16)
+        | ((0xFFL & bytes[offset+6]) << 8)) >> 8;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c95136a0/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyDate.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyDate.java b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyDate.java
index c50cd40..e464339 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyDate.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyDate.java
@@ -20,11 +20,11 @@ package org.apache.hadoop.hive.serde2.lazy;
 import java.io.IOException;
 import java.io.OutputStream;
 import java.nio.ByteBuffer;
-import java.sql.Date;
 
+import org.apache.hadoop.hive.common.type.Date;
+import org.apache.hadoop.hive.serde2.io.DateWritableV2;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.apache.hadoop.hive.serde2.io.DateWritable;
 import org.apache.hadoop.hive.serde2.lazy.objectinspector.primitive.LazyDateObjectInspector;
 import org.apache.hadoop.io.Text;
 
@@ -36,17 +36,17 @@ import org.apache.hadoop.io.Text;
  *    YYYY-MM-DD
  *
  */
-public class LazyDate extends LazyPrimitive<LazyDateObjectInspector, DateWritable> {
+public class LazyDate extends LazyPrimitive<LazyDateObjectInspector, DateWritableV2> {
   private static final Logger LOG = LoggerFactory.getLogger(LazyDate.class);
 
   public LazyDate(LazyDateObjectInspector oi) {
     super(oi);
-    data = new DateWritable();
+    data = new DateWritableV2();
   }
 
   public LazyDate(LazyDate copy) {
     super(copy);
-    data = new DateWritable(copy.data);
+    data = new DateWritableV2(copy.data);
   }
 
   /**
@@ -81,7 +81,7 @@ public class LazyDate extends LazyPrimitive<LazyDateObjectInspector, DateWritabl
    *          The Date to write
    * @throws IOException
    */
-  public static void writeUTF8(OutputStream out, DateWritable d)
+  public static void writeUTF8(OutputStream out, DateWritableV2 d)
       throws IOException {
     ByteBuffer b = Text.encode(d.toString());
     out.write(b.array(), 0, b.limit());

http://git-wip-us.apache.org/repos/asf/hive/blob/c95136a0/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyTimestamp.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyTimestamp.java b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyTimestamp.java
index ee801ee..3473c56 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyTimestamp.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyTimestamp.java
@@ -20,11 +20,11 @@ package org.apache.hadoop.hive.serde2.lazy;
 import java.io.IOException;
 import java.io.OutputStream;
 import java.io.UnsupportedEncodingException;
-import java.sql.Timestamp;
 
+import org.apache.hadoop.hive.serde2.io.TimestampWritableV2;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.apache.hadoop.hive.serde2.io.TimestampWritable;
+import org.apache.hadoop.hive.common.type.Timestamp;
 import org.apache.hadoop.hive.serde2.lazy.objectinspector.primitive.LazyTimestampObjectInspector;
 
 /**
@@ -35,17 +35,17 @@ import org.apache.hadoop.hive.serde2.lazy.objectinspector.primitive.LazyTimestam
  *    YYYY-MM-DD HH:MM:SS.[fff...]
  *
  */
-public class LazyTimestamp extends LazyPrimitive<LazyTimestampObjectInspector, TimestampWritable> {
+public class LazyTimestamp extends LazyPrimitive<LazyTimestampObjectInspector, TimestampWritableV2> {
   private static final Logger LOG = LoggerFactory.getLogger(LazyTimestamp.class);
 
   public LazyTimestamp(LazyTimestampObjectInspector oi) {
     super(oi);
-    data = new TimestampWritable();
+    data = new TimestampWritableV2();
   }
 
   public LazyTimestamp(LazyTimestamp copy) {
     super(copy);
-    data = new TimestampWritable(copy.data);
+    data = new TimestampWritableV2(copy.data);
   }
 
   /**
@@ -94,18 +94,18 @@ public class LazyTimestamp extends LazyPrimitive<LazyTimestampObjectInspector, T
    *          The Timestamp to write
    * @throws IOException
    */
-  public static void writeUTF8(OutputStream out, TimestampWritable i)
+  public static void writeUTF8(OutputStream out, TimestampWritableV2 i)
       throws IOException {
     if (i == null) {
       // Serialize as time 0
-      out.write(TimestampWritable.nullBytes);
+      out.write(TimestampWritableV2.nullBytes);
     } else {
       out.write(i.toString().getBytes("US-ASCII"));
     }
   }
 
   @Override
-  public TimestampWritable getWritableObject() {
+  public TimestampWritableV2 getWritableObject() {
     return data;
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/c95136a0/serde/src/java/org/apache/hadoop/hive/serde2/lazy/VerifyLazy.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/VerifyLazy.java b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/VerifyLazy.java
index 17c0357..14ff6d2 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/VerifyLazy.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/VerifyLazy.java
@@ -17,20 +17,20 @@
  */
 package org.apache.hadoop.hive.serde2.lazy;
 
-import java.sql.Date;
-import java.sql.Timestamp;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
 
+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.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;
@@ -38,7 +38,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.lazybinary.LazyBinaryArray;
 import org.apache.hadoop.hive.serde2.lazybinary.LazyBinaryMap;
 import org.apache.hadoop.hive.serde2.lazybinary.LazyBinaryStruct;
@@ -301,7 +301,7 @@ public class VerifyLazy {
             throw new RuntimeException("Expected LazyDate");
           }
           Date value = ((LazyDate) primitiveObject).getWritableObject().get();
-          Date expected = ((DateWritable) expectedObject).get();
+          Date expected = ((DateWritableV2) expectedObject).get();
           if (!value.equals(expected)) {
             throw new RuntimeException("Date field mismatch (expected " + expected + " found " + value + ")");
           }
@@ -310,10 +310,10 @@ public class VerifyLazy {
       case TIMESTAMP:
         {
           if (!(primitiveObject instanceof LazyTimestamp)) {
-            throw new RuntimeException("TimestampWritable expected writable not TimestampWritable");
+            throw new RuntimeException("TimestampWritableV2 expected writable not TimestampWritableV2");
           }
           Timestamp value = ((LazyTimestamp) primitiveObject).getWritableObject().getTimestamp();
-          Timestamp expected = ((TimestampWritable) expectedObject).getTimestamp();
+          Timestamp expected = ((TimestampWritableV2) expectedObject).getTimestamp();
           if (!value.equals(expected)) {
             throw new RuntimeException("Timestamp field mismatch (expected " + expected + " found " + value + ")");
           }

http://git-wip-us.apache.org/repos/asf/hive/blob/c95136a0/serde/src/java/org/apache/hadoop/hive/serde2/lazy/fast/LazySimpleDeserializeRead.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/fast/LazySimpleDeserializeRead.java b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/fast/LazySimpleDeserializeRead.java
index 1890f18..45c44da 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/fast/LazySimpleDeserializeRead.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/fast/LazySimpleDeserializeRead.java
@@ -21,10 +21,10 @@ package org.apache.hadoop.hive.serde2.lazy.fast;
 import java.io.IOException;
 import java.nio.charset.CharacterCodingException;
 import java.nio.charset.StandardCharsets;
-import java.sql.Date;
 import java.util.Arrays;
 import java.util.List;
 
+import org.apache.hadoop.hive.common.type.Date;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hive.common.type.DataTypePhysicalVariation;

http://git-wip-us.apache.org/repos/asf/hive/blob/c95136a0/serde/src/java/org/apache/hadoop/hive/serde2/lazy/fast/LazySimpleSerializeWrite.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/fast/LazySimpleSerializeWrite.java b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/fast/LazySimpleSerializeWrite.java
index 356326c..a42d6f4 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/fast/LazySimpleSerializeWrite.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/fast/LazySimpleSerializeWrite.java
@@ -20,14 +20,14 @@ package org.apache.hadoop.hive.serde2.lazy.fast;
 
 import java.io.IOException;
 import java.nio.ByteBuffer;
-import java.sql.Date;
-import java.sql.Timestamp;
 import java.util.ArrayDeque;
 import java.util.Deque;
 import java.util.List;
 import java.util.Map;
 
 import org.apache.commons.codec.binary.Base64;
+import org.apache.hadoop.hive.common.type.Date;
+import org.apache.hadoop.hive.serde2.io.DateWritableV2;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hive.common.type.HiveChar;
@@ -35,12 +35,12 @@ 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.ByteStream.Output;
-import org.apache.hadoop.hive.serde2.io.DateWritable;
 import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 import org.apache.hadoop.hive.serde2.io.HiveIntervalDayTimeWritable;
 import org.apache.hadoop.hive.serde2.io.HiveIntervalYearMonthWritable;
-import org.apache.hadoop.hive.serde2.io.TimestampWritable;
+import org.apache.hadoop.hive.serde2.io.TimestampWritableV2;
 import org.apache.hadoop.hive.serde2.lazy.LazyDate;
 import org.apache.hadoop.hive.serde2.lazy.LazyHiveDecimal;
 import org.apache.hadoop.hive.serde2.lazy.LazyHiveIntervalDayTime;
@@ -77,8 +77,8 @@ public final class LazySimpleSerializeWrite implements SerializeWrite {
   private Deque<Integer> indexStack = new ArrayDeque<Integer>();
 
   // For thread safety, we allocate private writable objects for our use only.
-  private DateWritable dateWritable;
-  private TimestampWritable timestampWritable;
+  private DateWritableV2 dateWritable;
+  private TimestampWritableV2 timestampWritable;
   private HiveIntervalYearMonthWritable hiveIntervalYearMonthWritable;
   private HiveIntervalDayTimeWritable hiveIntervalDayTimeWritable;
   private HiveIntervalDayTime hiveIntervalDayTime;
@@ -299,7 +299,7 @@ public final class LazySimpleSerializeWrite implements SerializeWrite {
   public void writeDate(Date date) throws IOException {
     beginPrimitive();
     if (dateWritable == null) {
-      dateWritable = new DateWritable();
+      dateWritable = new DateWritableV2();
     }
     dateWritable.set(date);
     LazyDate.writeUTF8(output, dateWritable);
@@ -311,7 +311,7 @@ public final class LazySimpleSerializeWrite implements SerializeWrite {
   public void writeDate(int dateAsDays) throws IOException {
     beginPrimitive();
     if (dateWritable == null) {
-      dateWritable = new DateWritable();
+      dateWritable = new DateWritableV2();
     }
     dateWritable.set(dateAsDays);
     LazyDate.writeUTF8(output, dateWritable);
@@ -325,7 +325,7 @@ public final class LazySimpleSerializeWrite implements SerializeWrite {
   public void writeTimestamp(Timestamp v) throws IOException {
     beginPrimitive();
     if (timestampWritable == null) {
-      timestampWritable = new TimestampWritable();
+      timestampWritable = new TimestampWritableV2();
     }
     timestampWritable.set(v);
     LazyTimestamp.writeUTF8(output, timestampWritable);

http://git-wip-us.apache.org/repos/asf/hive/blob/c95136a0/serde/src/java/org/apache/hadoop/hive/serde2/lazy/objectinspector/primitive/LazyDateObjectInspector.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/objectinspector/primitive/LazyDateObjectInspector.java b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/objectinspector/primitive/LazyDateObjectInspector.java
index 3bc4ff7..e356d23 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/objectinspector/primitive/LazyDateObjectInspector.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/objectinspector/primitive/LazyDateObjectInspector.java
@@ -17,18 +17,17 @@
  */
 package org.apache.hadoop.hive.serde2.lazy.objectinspector.primitive;
 
-import java.sql.Date;
-
-import org.apache.hadoop.hive.serde2.io.DateWritable;
+import org.apache.hadoop.hive.common.type.Date;
+import org.apache.hadoop.hive.serde2.io.DateWritableV2;
 import org.apache.hadoop.hive.serde2.lazy.LazyDate;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.DateObjectInspector;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
 
 /**
- * A WritableDateObjectInspector inspects a DateWritable Object.
+ * A WritableDateObjectInspector inspects a DateWritableV2 Object.
  */
 public class LazyDateObjectInspector
-    extends AbstractPrimitiveLazyObjectInspector<DateWritable>
+    extends AbstractPrimitiveLazyObjectInspector<DateWritableV2>
     implements DateObjectInspector {
 
   protected LazyDateObjectInspector() {

http://git-wip-us.apache.org/repos/asf/hive/blob/c95136a0/serde/src/java/org/apache/hadoop/hive/serde2/lazy/objectinspector/primitive/LazyTimestampObjectInspector.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/objectinspector/primitive/LazyTimestampObjectInspector.java b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/objectinspector/primitive/LazyTimestampObjectInspector.java
index e0f993e..a10a722 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/objectinspector/primitive/LazyTimestampObjectInspector.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/objectinspector/primitive/LazyTimestampObjectInspector.java
@@ -17,17 +17,17 @@
  */
 package org.apache.hadoop.hive.serde2.lazy.objectinspector.primitive;
 
-import java.sql.Timestamp;
 import java.util.List;
 
-import org.apache.hadoop.hive.serde2.io.TimestampWritable;
+import org.apache.hadoop.hive.common.type.Timestamp;
+import org.apache.hadoop.hive.serde2.io.TimestampWritableV2;
 import org.apache.hadoop.hive.serde2.lazy.LazyTimestamp;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.TimestampObjectInspector;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
 import org.apache.hive.common.util.TimestampParser;
 
 public class LazyTimestampObjectInspector
-    extends AbstractPrimitiveLazyObjectInspector<TimestampWritable>
+    extends AbstractPrimitiveLazyObjectInspector<TimestampWritableV2>
     implements TimestampObjectInspector {
 
   protected List<String> timestampFormats = null;

http://git-wip-us.apache.org/repos/asf/hive/blob/c95136a0/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryDate.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryDate.java b/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryDate.java
index 2952e26..fbfe961 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryDate.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryDate.java
@@ -17,9 +17,9 @@
  */
 package org.apache.hadoop.hive.serde2.lazybinary;
 
+import org.apache.hadoop.hive.serde2.io.DateWritableV2;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.apache.hadoop.hive.serde2.io.DateWritable;
 import org.apache.hadoop.hive.serde2.lazy.ByteArrayRef;
 import org.apache.hadoop.hive.serde2.lazybinary.LazyBinaryUtils.VInt;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableDateObjectInspector;
@@ -29,17 +29,17 @@ import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableDateObjec
  *
  */
 public class LazyBinaryDate extends
-    LazyBinaryPrimitive<WritableDateObjectInspector, DateWritable> {
+    LazyBinaryPrimitive<WritableDateObjectInspector, DateWritableV2> {
   static final Logger LOG = LoggerFactory.getLogger(LazyBinaryDate.class);
 
   LazyBinaryDate(WritableDateObjectInspector oi) {
     super(oi);
-    data = new DateWritable();
+    data = new DateWritableV2();
   }
 
   LazyBinaryDate(LazyBinaryDate copy) {
     super(copy);
-    data = new DateWritable(copy.data);
+    data = new DateWritableV2(copy.data);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hive/blob/c95136a0/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinarySerDe.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinarySerDe.java b/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinarySerDe.java
index 24704a1..660080c 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinarySerDe.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinarySerDe.java
@@ -24,6 +24,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Properties;
 
+import org.apache.hadoop.hive.serde2.io.DateWritableV2;
 import org.apache.hadoop.hive.serde2.io.TimestampLocalTZWritable;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.TimestampLocalTZObjectInspector;
 import org.slf4j.Logger;
@@ -37,12 +38,11 @@ import org.apache.hadoop.hive.serde2.SerDeException;
 import org.apache.hadoop.hive.serde2.SerDeSpec;
 import org.apache.hadoop.hive.serde2.SerDeStats;
 import org.apache.hadoop.hive.serde2.SerDeUtils;
-import org.apache.hadoop.hive.serde2.io.DateWritable;
 import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
 import org.apache.hadoop.hive.serde2.io.HiveIntervalDayTimeWritable;
 import org.apache.hadoop.hive.serde2.io.HiveIntervalYearMonthWritable;
-import org.apache.hadoop.hive.serde2.io.TimestampWritable;
+import org.apache.hadoop.hive.serde2.io.TimestampWritableV2;
 import org.apache.hadoop.hive.serde2.lazy.ByteArrayRef;
 import org.apache.hadoop.hive.serde2.objectinspector.ListObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.MapObjectInspector;
@@ -318,7 +318,7 @@ public class LazyBinarySerDe extends AbstractSerDe {
   }
 
   public static void writeDateToByteStream(RandomAccessOutput byteStream,
-                                            DateWritable date) {
+                                            DateWritableV2 date) {
     LazyBinaryUtils.writeVInt(byteStream, date.getDays());
   }
 
@@ -505,13 +505,13 @@ public class LazyBinarySerDe extends AbstractSerDe {
       }
 
       case DATE: {
-        DateWritable d = ((DateObjectInspector) poi).getPrimitiveWritableObject(obj);
+        DateWritableV2 d = ((DateObjectInspector) poi).getPrimitiveWritableObject(obj);
         writeDateToByteStream(byteStream, d);
         return;
       }
       case TIMESTAMP: {
         TimestampObjectInspector toi = (TimestampObjectInspector) poi;
-        TimestampWritable t = toi.getPrimitiveWritableObject(obj);
+        TimestampWritableV2 t = toi.getPrimitiveWritableObject(obj);
         t.writeToByteStream(byteStream);
         return;
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/c95136a0/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinarySerDe2.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinarySerDe2.java b/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinarySerDe2.java
index b328508..3e06892 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinarySerDe2.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinarySerDe2.java
@@ -27,11 +27,11 @@ import org.apache.hadoop.hive.serde.serdeConstants;
 import org.apache.hadoop.hive.serde2.ByteStream.RandomAccessOutput;
 import org.apache.hadoop.hive.serde2.SerDeException;
 import org.apache.hadoop.hive.serde2.SerDeSpec;
-import org.apache.hadoop.hive.serde2.io.DateWritable;
+import org.apache.hadoop.hive.serde2.io.DateWritableV2;
 import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 import org.apache.hadoop.hive.serde2.io.HiveIntervalDayTimeWritable;
 import org.apache.hadoop.hive.serde2.io.HiveIntervalYearMonthWritable;
-import org.apache.hadoop.hive.serde2.io.TimestampWritable;
+import org.apache.hadoop.hive.serde2.io.TimestampWritableV2;
 import org.apache.hadoop.hive.serde2.objectinspector.ListObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.MapObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
@@ -337,7 +337,7 @@ public class LazyBinarySerDe2 extends LazyBinarySerDe {
     @Override
     void serialize(RandomAccessOutput byteStream, Object obj, ObjectInspector objInspector,
         boolean skipLengthPrefix, BooleanRef warnedOnceNullMapKey) {
-      DateWritable d = ((DateObjectInspector) objInspector).getPrimitiveWritableObject(obj);
+      DateWritableV2 d = ((DateObjectInspector) objInspector).getPrimitiveWritableObject(obj);
       LazyBinarySerDe.writeDateToByteStream(byteStream, d);
     }
   }
@@ -347,7 +347,7 @@ public class LazyBinarySerDe2 extends LazyBinarySerDe {
     void serialize(RandomAccessOutput byteStream, Object obj, ObjectInspector objInspector,
         boolean skipLengthPrefix, BooleanRef warnedOnceNullMapKey) {
       TimestampObjectInspector toi = (TimestampObjectInspector) objInspector;
-      TimestampWritable t = toi.getPrimitiveWritableObject(obj);
+      TimestampWritableV2 t = toi.getPrimitiveWritableObject(obj);
       t.writeToByteStream(byteStream);
     }
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/c95136a0/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryTimestamp.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryTimestamp.java b/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryTimestamp.java
index fa5461a..724a167 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryTimestamp.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryTimestamp.java
@@ -17,9 +17,9 @@
  */
 package org.apache.hadoop.hive.serde2.lazybinary;
 
+import org.apache.hadoop.hive.serde2.io.TimestampWritableV2;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.apache.hadoop.hive.serde2.io.TimestampWritable;
 import org.apache.hadoop.hive.serde2.lazy.ByteArrayRef;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableTimestampObjectInspector;
 
@@ -29,17 +29,17 @@ import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableTimestamp
  *
  */
 public class LazyBinaryTimestamp extends
-    LazyBinaryPrimitive<WritableTimestampObjectInspector, TimestampWritable> {
+    LazyBinaryPrimitive<WritableTimestampObjectInspector, TimestampWritableV2> {
   static final Logger LOG = LoggerFactory.getLogger(LazyBinaryTimestamp.class);
 
   LazyBinaryTimestamp(WritableTimestampObjectInspector oi) {
     super(oi);
-    data = new TimestampWritable();
+    data = new TimestampWritableV2();
   }
 
   LazyBinaryTimestamp(LazyBinaryTimestamp copy) {
     super(copy);
-    data = new TimestampWritable(copy.data);
+    data = new TimestampWritableV2(copy.data);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hive/blob/c95136a0/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryUtils.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryUtils.java b/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryUtils.java
index ee1e2e6..eb028e3 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryUtils.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryUtils.java
@@ -23,7 +23,7 @@ import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.hadoop.hive.serde2.ByteStream.RandomAccessOutput;
 import org.apache.hadoop.hive.serde2.io.TimestampLocalTZWritable;
-import org.apache.hadoop.hive.serde2.io.TimestampWritable;
+import org.apache.hadoop.hive.serde2.io.TimestampWritableV2;
 import org.apache.hadoop.hive.serde2.lazybinary.objectinspector.LazyBinaryObjectInspectorFactory;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category;
@@ -203,7 +203,7 @@ public final class LazyBinaryUtils {
         break;
       case TIMESTAMP:
         recordInfo.elementOffset = 0;
-        recordInfo.elementSize = TimestampWritable.getTotalLength(bytes, offset);
+        recordInfo.elementSize = TimestampWritableV2.getTotalLength(bytes, offset);
         break;
       case TIMESTAMPLOCALTZ:
         recordInfo.elementOffset = 0;

http://git-wip-us.apache.org/repos/asf/hive/blob/c95136a0/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/fast/LazyBinaryDeserializeRead.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/fast/LazyBinaryDeserializeRead.java b/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/fast/LazyBinaryDeserializeRead.java
index 340f322..000dfed 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/fast/LazyBinaryDeserializeRead.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/fast/LazyBinaryDeserializeRead.java
@@ -26,7 +26,7 @@ import java.util.Deque;
 import java.util.List;
 
 import org.apache.hadoop.hive.serde2.fast.DeserializeRead;
-import org.apache.hadoop.hive.serde2.io.TimestampWritable;
+import org.apache.hadoop.hive.serde2.io.TimestampWritableV2;
 import org.apache.hadoop.hive.serde2.lazybinary.LazyBinaryUtils;
 import org.apache.hadoop.hive.serde2.lazybinary.LazyBinaryUtils.VInt;
 import org.apache.hadoop.hive.serde2.lazybinary.LazyBinaryUtils.VLong;
@@ -303,7 +303,7 @@ public final class LazyBinaryDeserializeRead extends DeserializeRead {
       break;
     case TIMESTAMP:
       {
-        int length = TimestampWritable.getTotalLength(bytes, offset);
+        int length = TimestampWritableV2.getTotalLength(bytes, offset);
         int saveStart = offset;
         offset += length;
         // Last item -- ok to be at end.

http://git-wip-us.apache.org/repos/asf/hive/blob/c95136a0/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/fast/LazyBinarySerializeWrite.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/fast/LazyBinarySerializeWrite.java b/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/fast/LazyBinarySerializeWrite.java
index cd4e619..ec56b82 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/fast/LazyBinarySerializeWrite.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/fast/LazyBinarySerializeWrite.java
@@ -19,14 +19,14 @@
 package org.apache.hadoop.hive.serde2.lazybinary.fast;
 
 import java.io.IOException;
-import java.sql.Date;
-import java.sql.Timestamp;
 import java.util.ArrayDeque;
 import java.util.Deque;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.hadoop.hive.common.type.Date;
 import org.apache.hadoop.hive.serde2.ByteStream;
+import org.apache.hadoop.hive.serde2.io.TimestampWritableV2;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -35,12 +35,12 @@ 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.ByteStream.Output;
-import org.apache.hadoop.hive.serde2.io.DateWritable;
+import org.apache.hadoop.hive.serde2.io.DateWritableV2;
 import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 import org.apache.hadoop.hive.serde2.io.HiveIntervalDayTimeWritable;
 import org.apache.hadoop.hive.serde2.io.HiveIntervalYearMonthWritable;
-import org.apache.hadoop.hive.serde2.io.TimestampWritable;
 import org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe;
 import org.apache.hadoop.hive.serde2.lazybinary.LazyBinaryUtils;
 import org.apache.hadoop.hive.serde2.fast.SerializeWrite;
@@ -64,7 +64,7 @@ public class LazyBinarySerializeWrite implements SerializeWrite {
   private boolean skipLengthPrefix = false;
 
   // For thread safety, we allocate private writable objects for our use only.
-  private TimestampWritable timestampWritable;
+  private TimestampWritableV2 timestampWritable;
   private HiveIntervalYearMonthWritable hiveIntervalYearMonthWritable;
   private HiveIntervalDayTimeWritable hiveIntervalDayTimeWritable;
   private HiveIntervalDayTime hiveIntervalDayTime;
@@ -308,7 +308,7 @@ public class LazyBinarySerializeWrite implements SerializeWrite {
   @Override
   public void writeDate(Date date) throws IOException {
     beginElement();
-    writeVInt(DateWritable.dateToDays(date));
+    writeVInt(DateWritableV2.dateToDays(date));
     finishElement();
   }
 
@@ -327,7 +327,7 @@ public class LazyBinarySerializeWrite implements SerializeWrite {
   public void writeTimestamp(Timestamp v) throws IOException {
     beginElement();
     if (timestampWritable == null) {
-      timestampWritable = new TimestampWritable();
+      timestampWritable = new TimestampWritableV2();
     }
     timestampWritable.set(v);
     timestampWritable.writeToByteStream(output);

http://git-wip-us.apache.org/repos/asf/hive/blob/c95136a0/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/ObjectInspectorUtils.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/ObjectInspectorUtils.java b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/ObjectInspectorUtils.java
index a442cb1..9393fb8 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/ObjectInspectorUtils.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/ObjectInspectorUtils.java
@@ -29,9 +29,9 @@ import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 
-import org.apache.hadoop.hive.serde2.ByteStream;
-import org.apache.hadoop.hive.serde2.binarysortable.BinarySortableSerDe;
+import org.apache.hadoop.hive.serde2.io.DateWritableV2;
 import org.apache.hadoop.hive.serde2.io.TimestampLocalTZWritable;
+import org.apache.hadoop.hive.serde2.io.TimestampWritableV2;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.SettableTimestampLocalTZObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.TimestampLocalTZObjectInspector;
 import org.apache.hive.common.util.Murmur3;
@@ -39,13 +39,11 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hive.serde.serdeConstants;
 import org.apache.hadoop.hive.serde2.SerDeException;
-import org.apache.hadoop.hive.serde2.io.DateWritable;
 import org.apache.hadoop.hive.serde2.io.HiveCharWritable;
 import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 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.TimestampWritable;
 import org.apache.hadoop.hive.serde2.lazy.LazyDouble;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory.ObjectInspectorOptions;
@@ -720,7 +718,7 @@ public final class ObjectInspectorUtils {
       case DATE:
         return ((DateObjectInspector) poi).getPrimitiveWritableObject(o).hashCode();
       case TIMESTAMP:
-        TimestampWritable t = ((TimestampObjectInspector) poi)
+        TimestampWritableV2 t = ((TimestampObjectInspector) poi)
             .getPrimitiveWritableObject(o);
         return t.hashCode();
       case TIMESTAMPLOCALTZ:
@@ -857,7 +855,7 @@ public final class ObjectInspectorUtils {
             byteBuffer.putInt(((DateObjectInspector) poi).getPrimitiveWritableObject(o).getDays());
             return Murmur3.hash32(byteBuffer.array(), 4);
           case TIMESTAMP: {
-            TimestampWritable t = ((TimestampObjectInspector) poi)
+            TimestampWritableV2 t = ((TimestampObjectInspector) poi)
                     .getPrimitiveWritableObject(o);
             return Murmur3.hash32(t.getBytes());
           }
@@ -1112,16 +1110,16 @@ public final class ObjectInspectorUtils {
       }
 
       case DATE: {
-        DateWritable d1 = ((DateObjectInspector) poi1)
+        DateWritableV2 d1 = ((DateObjectInspector) poi1)
             .getPrimitiveWritableObject(o1);
-        DateWritable d2 = ((DateObjectInspector) poi2)
+        DateWritableV2 d2 = ((DateObjectInspector) poi2)
             .getPrimitiveWritableObject(o2);
         return d1.compareTo(d2);
       }
       case TIMESTAMP: {
-        TimestampWritable t1 = ((TimestampObjectInspector) poi1)
+        TimestampWritableV2 t1 = ((TimestampObjectInspector) poi1)
             .getPrimitiveWritableObject(o1);
-        TimestampWritable t2 = ((TimestampObjectInspector) poi2)
+        TimestampWritableV2 t2 = ((TimestampObjectInspector) poi2)
             .getPrimitiveWritableObject(o2);
         return t1.compareTo(t2);
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/c95136a0/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/DateObjectInspector.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/DateObjectInspector.java b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/DateObjectInspector.java
index 93a18f7..f58364c 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/DateObjectInspector.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/DateObjectInspector.java
@@ -17,11 +17,10 @@
  */
 package org.apache.hadoop.hive.serde2.objectinspector.primitive;
 
-import java.sql.Date;
-
 import org.apache.hadoop.hive.common.classification.InterfaceAudience;
 import org.apache.hadoop.hive.common.classification.InterfaceStability;
-import org.apache.hadoop.hive.serde2.io.DateWritable;
+import org.apache.hadoop.hive.common.type.Date;
+import org.apache.hadoop.hive.serde2.io.DateWritableV2;
 import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
 
 /**
@@ -31,7 +30,7 @@ import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
 @InterfaceStability.Stable
 public interface DateObjectInspector extends PrimitiveObjectInspector {
 
-  DateWritable getPrimitiveWritableObject(Object o);
+  DateWritableV2 getPrimitiveWritableObject(Object o);
 
   Date getPrimitiveJavaObject(Object o);
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/c95136a0/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantDateObjectInspector.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantDateObjectInspector.java b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantDateObjectInspector.java
index bd86c22..7dc3d07 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantDateObjectInspector.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantDateObjectInspector.java
@@ -17,9 +17,8 @@
  */
 package org.apache.hadoop.hive.serde2.objectinspector.primitive;
 
-import java.sql.Date;
-
-import org.apache.hadoop.hive.serde2.io.DateWritable;
+import org.apache.hadoop.hive.common.type.Date;
+import org.apache.hadoop.hive.serde2.io.DateWritableV2;
 import org.apache.hadoop.hive.serde2.objectinspector.ConstantObjectInspector;
 
 public class JavaConstantDateObjectInspector extends JavaDateObjectInspector
@@ -36,6 +35,6 @@ public class JavaConstantDateObjectInspector extends JavaDateObjectInspector
     if (value==null) {
       return null;
     }
-    return new DateWritable(value);
+    return new DateWritableV2(value);
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/c95136a0/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantTimestampObjectInspector.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantTimestampObjectInspector.java b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantTimestampObjectInspector.java
index 2453bc6..4da7299 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantTimestampObjectInspector.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaConstantTimestampObjectInspector.java
@@ -17,9 +17,8 @@
  */
 package org.apache.hadoop.hive.serde2.objectinspector.primitive;
 
-import java.sql.Timestamp;
-
-import org.apache.hadoop.hive.serde2.io.TimestampWritable;
+import org.apache.hadoop.hive.common.type.Timestamp;
+import org.apache.hadoop.hive.serde2.io.TimestampWritableV2;
 import org.apache.hadoop.hive.serde2.objectinspector.ConstantObjectInspector;
 
 public class JavaConstantTimestampObjectInspector extends
@@ -36,6 +35,6 @@ public class JavaConstantTimestampObjectInspector extends
     if (value==null) {
       return null;
     }
-    return new TimestampWritable(value);
+    return new TimestampWritableV2(value);
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/c95136a0/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaDateObjectInspector.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaDateObjectInspector.java b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaDateObjectInspector.java
index d93d719..4cf0a60 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaDateObjectInspector.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaDateObjectInspector.java
@@ -17,9 +17,8 @@
  */
 package org.apache.hadoop.hive.serde2.objectinspector.primitive;
 
-import java.sql.Date;
-
-import org.apache.hadoop.hive.serde2.io.DateWritable;
+import org.apache.hadoop.hive.common.type.Date;
+import org.apache.hadoop.hive.serde2.io.DateWritableV2;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
 
 /**
@@ -33,8 +32,8 @@ public class JavaDateObjectInspector
     super(TypeInfoFactory.dateTypeInfo);
   }
 
-  public DateWritable getPrimitiveWritableObject(Object o) {
-    return o == null ? null : new DateWritable((Date) o);
+  public DateWritableV2 getPrimitiveWritableObject(Object o) {
+    return o == null ? null : new DateWritableV2((Date) o);
   }
 
   @Override
@@ -50,20 +49,34 @@ public class JavaDateObjectInspector
     if (value == null) {
       return null;
     }
-    ((Date) o).setTime(value.getTime());
+    ((Date) o).setTimeInDays(value.toEpochDay());
     return o;
   }
 
-  public Object set(Object o, DateWritable d) {
+  @Deprecated
+  public Object set(Object o, java.sql.Date value) {
+    if (value == null) {
+      return null;
+    }
+    ((Date) o).setTimeInMillis(value.getTime());
+    return o;
+  }
+
+  public Object set(Object o, DateWritableV2 d) {
     if (d == null) {
       return null;
     }
-    ((Date) o).setTime(d.get().getTime());
+    ((Date) o).setTimeInDays(d.get().toEpochDay());
     return o;
   }
 
+  @Deprecated
+  public Object create(java.sql.Date value) {
+    return Date.ofEpochMilli(value.getTime());
+  }
+
   public Object create(Date value) {
-    return new Date(value.getTime());
+    return Date.ofEpochDay(value.toEpochDay());
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/c95136a0/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaTimestampObjectInspector.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaTimestampObjectInspector.java b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaTimestampObjectInspector.java
index 1e805ba..47719c8 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaTimestampObjectInspector.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/JavaTimestampObjectInspector.java
@@ -17,9 +17,8 @@
  */
 package org.apache.hadoop.hive.serde2.objectinspector.primitive;
 
-import java.sql.Timestamp;
-
-import org.apache.hadoop.hive.serde2.io.TimestampWritable;
+import org.apache.hadoop.hive.common.type.Timestamp;
+import org.apache.hadoop.hive.serde2.io.TimestampWritableV2;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
 
 public class JavaTimestampObjectInspector
@@ -30,8 +29,8 @@ public class JavaTimestampObjectInspector
     super(TypeInfoFactory.timestampTypeInfo);
   }
 
-  public TimestampWritable getPrimitiveWritableObject(Object o) {
-    return o == null ? null : new TimestampWritable((Timestamp) o);
+  public TimestampWritableV2 getPrimitiveWritableObject(Object o) {
+    return o == null ? null : new TimestampWritableV2((Timestamp) o);
   }
 
   @Override
@@ -45,43 +44,54 @@ public class JavaTimestampObjectInspector
       return null;
     }
     Timestamp source = (Timestamp) o;
-    Timestamp copy = new Timestamp(source.getTime());
-    copy.setNanos(source.getNanos());
-    return copy;
+    return new Timestamp(source);
   }
 
   public Timestamp get(Object o) {
     return (Timestamp) o;
   }
 
+  @Deprecated
+  public Object set(Object o, java.sql.Timestamp value) {
+    if (value == null) {
+      return null;
+    }
+    ((Timestamp) o).setTimeInMillis(value.getTime(), value.getNanos());
+    return o;
+  }
+
   public Object set(Object o, Timestamp value) {
     if (value == null) {
       return null;
     }
-    ((Timestamp) o).setTime(value.getTime());
+    ((Timestamp) o).set(value);
     return o;
   }
 
   public Object set(Object o, byte[] bytes, int offset) {
-    TimestampWritable.setTimestamp((Timestamp) o, bytes, offset);
+    TimestampWritableV2.setTimestamp((Timestamp) o, bytes, offset);
     return o;
   }
 
-  public Object set(Object o, TimestampWritable tw) {
+  public Object set(Object o, TimestampWritableV2 tw) {
     if (tw == null) {
       return null;
     }
     Timestamp t = (Timestamp) o;
-    t.setTime(tw.getTimestamp().getTime());
-    t.setNanos(tw.getTimestamp().getNanos());
+    t.set(tw.getTimestamp());
     return t;
   }
 
+  @Deprecated
+  public Object create(java.sql.Timestamp value) {
+    return Timestamp.ofEpochMilli(value.getTime(), value.getNanos());
+  }
+
   public Object create(Timestamp value) {
-    return new Timestamp(value.getTime());
+    return new Timestamp(value);
   }
 
   public Object create(byte[] bytes, int offset) {
-    return TimestampWritable.createTimestamp(bytes, offset);
+    return TimestampWritableV2.createTimestamp(bytes, offset);
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/c95136a0/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorConverter.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorConverter.java b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorConverter.java
index ba20a2c..1e12cca 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorConverter.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorConverter.java
@@ -18,15 +18,15 @@
 
 package org.apache.hadoop.hive.serde2.objectinspector.primitive;
 
-import java.sql.Date;
-import java.sql.Timestamp;
 import java.time.ZoneId;
 
+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.HiveIntervalYearMonth;
 import org.apache.hadoop.hive.common.type.HiveIntervalDayTime;
 import org.apache.hadoop.hive.common.type.HiveVarchar;
+import org.apache.hadoop.hive.common.type.Timestamp;
 import org.apache.hadoop.hive.common.type.TimestampTZ;
 import org.apache.hadoop.hive.serde2.ByteStream;
 import org.apache.hadoop.hive.serde2.lazy.LazyInteger;
@@ -255,7 +255,7 @@ public class PrimitiveObjectInspectorConverter {
         SettableDateObjectInspector outputOI) {
       this.inputOI = inputOI;
       this.outputOI = outputOI;
-      r = outputOI.create(new Date(0));
+      r = outputOI.create(new Date());
     }
 
     public Object convert(Object input) {
@@ -277,7 +277,7 @@ public class PrimitiveObjectInspectorConverter {
         SettableTimestampObjectInspector outputOI) {
       this.inputOI = inputOI;
       this.outputOI = outputOI;
-      r = outputOI.create(new Timestamp(0));
+      r = outputOI.create(new Timestamp());
     }
 
     public void setIntToTimestampInSeconds(boolean intToTimestampInSeconds) {

http://git-wip-us.apache.org/repos/asf/hive/blob/c95136a0/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorFactory.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorFactory.java b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorFactory.java
index 10af3dc..51a0aed 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorFactory.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorFactory.java
@@ -24,7 +24,7 @@ import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.hadoop.hive.serde.serdeConstants;
 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;
@@ -33,7 +33,7 @@ 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.TimestampLocalTZWritable;
-import org.apache.hadoop.hive.serde2.io.TimestampWritable;
+import org.apache.hadoop.hive.serde2.io.TimestampWritableV2;
 import org.apache.hadoop.hive.serde2.objectinspector.ConstantObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory;
@@ -346,9 +346,9 @@ public final class PrimitiveObjectInspectorFactory {
       return new WritableConstantHiveVarcharObjectInspector((VarcharTypeInfo)typeInfo,
           (HiveVarcharWritable)value);
     case DATE:
-      return new WritableConstantDateObjectInspector((DateWritable)value);
+      return new WritableConstantDateObjectInspector((DateWritableV2)value);
     case TIMESTAMP:
-      return new WritableConstantTimestampObjectInspector((TimestampWritable)value);
+      return new WritableConstantTimestampObjectInspector((TimestampWritableV2)value);
     case TIMESTAMPLOCALTZ:
       return new WritableConstantTimestampLocalTZObjectInspector((TimestampLocalTZTypeInfo)typeInfo, (TimestampLocalTZWritable) value);
     case INTERVAL_YEAR_MONTH:

http://git-wip-us.apache.org/repos/asf/hive/blob/c95136a0/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorUtils.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorUtils.java b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorUtils.java
index 8cf0744..6362f2e 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorUtils.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorUtils.java
@@ -23,8 +23,6 @@ import java.io.DataOutput;
 import java.io.IOException;
 import java.nio.charset.CharacterCodingException;
 import java.nio.charset.StandardCharsets;
-import java.sql.Date;
-import java.sql.Timestamp;
 import java.time.DateTimeException;
 import java.time.ZoneId;
 import java.util.HashMap;
@@ -32,18 +30,19 @@ import java.util.Map;
 
 import org.apache.hadoop.hive.common.classification.InterfaceAudience;
 import org.apache.hadoop.hive.common.classification.InterfaceStability;
-import org.apache.hadoop.hive.common.type.TimestampTZ;
-import org.apache.hadoop.hive.common.type.TimestampTZUtil;
-import org.apache.hadoop.hive.ql.util.TimestampUtils;
-import org.apache.hadoop.hive.serde2.io.TimestampLocalTZWritable;
+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.HiveIntervalYearMonth;
 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.common.type.TimestampTZ;
+import org.apache.hadoop.hive.common.type.TimestampTZUtil;
+import org.apache.hadoop.hive.common.type.TimestampUtils;
 import org.apache.hadoop.hive.serde.serdeConstants;
 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;
@@ -51,7 +50,8 @@ 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.TimestampLocalTZWritable;
+import org.apache.hadoop.hive.serde2.io.TimestampWritableV2;
 import org.apache.hadoop.hive.serde2.lazy.LazyInteger;
 import org.apache.hadoop.hive.serde2.lazy.LazyLong;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
@@ -229,10 +229,10 @@ public final class PrimitiveObjectInspectorUtils {
       Short.class, ShortWritable.class);
   public static final PrimitiveTypeEntry dateTypeEntry = new PrimitiveTypeEntry(
       PrimitiveCategory.DATE, serdeConstants.DATE_TYPE_NAME, null,
-      Date.class, DateWritable.class);
+      Date.class, DateWritableV2.class);
   public static final PrimitiveTypeEntry timestampTypeEntry = new PrimitiveTypeEntry(
       PrimitiveCategory.TIMESTAMP, serdeConstants.TIMESTAMP_TYPE_NAME, null,
-      Timestamp.class, TimestampWritable.class);
+      Timestamp.class, TimestampWritableV2.class);
   public static final PrimitiveTypeEntry timestampTZTypeEntry = new PrimitiveTypeEntry(
       PrimitiveCategory.TIMESTAMPLOCALTZ, serdeConstants.TIMESTAMPLOCALTZ_TYPE_NAME, null,
       TimestampTZ.class, TimestampLocalTZWritable.class);
@@ -1126,7 +1126,7 @@ public final class PrimitiveObjectInspectorUtils {
       } catch (IllegalArgumentException e) {
         Timestamp ts = getTimestampFromString(s);
         if (ts != null) {
-          result = new Date(ts.getTime());
+          result = Date.ofEpochMilli(ts.toEpochMilli());
         } else {
           result = null;
         }
@@ -1140,7 +1140,7 @@ public final class PrimitiveObjectInspectorUtils {
       } catch (IllegalArgumentException e) {
         Timestamp ts = getTimestampFromString(val);
         if (ts != null) {
-          result = new Date(ts.getTime());
+          result = Date.ofEpochMilli(ts.toEpochMilli());
         } else {
           result = null;
         }
@@ -1151,7 +1151,7 @@ public final class PrimitiveObjectInspectorUtils {
       result = ((DateObjectInspector) oi).getPrimitiveWritableObject(o).get();
       break;
     case TIMESTAMP:
-      result = DateWritable.timeToDate(
+      result = DateWritableV2.timeToDate(
           ((TimestampObjectInspector) oi).getPrimitiveWritableObject(o).getSeconds());
       break;
     case TIMESTAMPLOCALTZ:
@@ -1187,23 +1187,23 @@ public final class PrimitiveObjectInspectorUtils {
       break;
     case BOOLEAN:
       longValue = ((BooleanObjectInspector) inputOI).get(o) ? 1 : 0;
-      result = TimestampWritable.longToTimestamp(longValue, intToTimestampInSeconds);
+      result = TimestampWritableV2.longToTimestamp(longValue, intToTimestampInSeconds);
       break;
     case BYTE:
       longValue = ((ByteObjectInspector) inputOI).get(o);
-      result = TimestampWritable.longToTimestamp(longValue, intToTimestampInSeconds);
+      result = TimestampWritableV2.longToTimestamp(longValue, intToTimestampInSeconds);
       break;
     case SHORT:
       longValue = ((ShortObjectInspector) inputOI).get(o);
-      result = TimestampWritable.longToTimestamp(longValue, intToTimestampInSeconds);
+      result = TimestampWritableV2.longToTimestamp(longValue, intToTimestampInSeconds);
       break;
     case INT:
       longValue = ((IntObjectInspector) inputOI).get(o);
-      result = TimestampWritable.longToTimestamp(longValue, intToTimestampInSeconds);
+      result = TimestampWritableV2.longToTimestamp(longValue, intToTimestampInSeconds);
       break;
     case LONG:
       longValue = ((LongObjectInspector) inputOI).get(o);
-      result = TimestampWritable.longToTimestamp(longValue, intToTimestampInSeconds);
+      result = TimestampWritableV2.longToTimestamp(longValue, intToTimestampInSeconds);
       break;
     case FLOAT:
       result = TimestampUtils.doubleToTimestamp(((FloatObjectInspector) inputOI).get(o));
@@ -1212,8 +1212,8 @@ public final class PrimitiveObjectInspectorUtils {
       result = TimestampUtils.doubleToTimestamp(((DoubleObjectInspector) inputOI).get(o));
       break;
     case DECIMAL:
-      result = TimestampUtils.decimalToTimestamp(((HiveDecimalObjectInspector) inputOI)
-                                                    .getPrimitiveJavaObject(o));
+      result = TimestampUtils.decimalToTimestamp(
+          ((HiveDecimalObjectInspector) inputOI).getPrimitiveJavaObject(o));
       break;
     case STRING:
       StringObjectInspector soi = (StringObjectInspector) inputOI;
@@ -1225,8 +1225,8 @@ public final class PrimitiveObjectInspectorUtils {
       result = getTimestampFromString(getString(o, inputOI));
       break;
     case DATE:
-      result = new Timestamp(
-          ((DateObjectInspector) inputOI).getPrimitiveWritableObject(o).get().getTime());
+      result = Timestamp.ofEpochMilli(
+          ((DateObjectInspector) inputOI).getPrimitiveWritableObject(o).get().toEpochMilli());
       break;
     case TIMESTAMP:
       result = ((TimestampObjectInspector) inputOI).getPrimitiveWritableObject(o).getTimestamp();
@@ -1247,23 +1247,25 @@ public final class PrimitiveObjectInspectorUtils {
     return result;
   }
 
-  static Timestamp getTimestampFromString(String s) {
+  public static Timestamp getTimestampFromString(String s) {
     Timestamp result;
     s = s.trim();
     s = trimNanoTimestamp(s);
 
-    int firstSpace = s.indexOf(' ');
-    if (firstSpace < 0) {
-      s = s.concat(" 00:00:00");
-    }
     try {
       result = Timestamp.valueOf(s);
     } catch (IllegalArgumentException e) {
       // Let's try to parse it as timestamp with time zone and transform
       try {
-        result = Timestamp.from(TimestampTZUtil.parse(s).getZonedDateTime().toInstant());
+        result = Timestamp.valueOf(TimestampTZUtil.parse(s).getZonedDateTime()
+            .toLocalDateTime().toString());
       } catch (DateTimeException e2) {
-        result = null;
+        // Last try: we try to parse it as date and transform
+        try {
+          result = Timestamp.ofEpochMilli(Date.valueOf(s).toEpochMilli());
+        } catch (IllegalArgumentException e3) {
+          result = null;
+        }
       }
     }
     return result;

http://git-wip-us.apache.org/repos/asf/hive/blob/c95136a0/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/SettableDateObjectInspector.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/SettableDateObjectInspector.java b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/SettableDateObjectInspector.java
index 831411d..725d5cd 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/SettableDateObjectInspector.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/SettableDateObjectInspector.java
@@ -17,17 +17,22 @@
  */
 package org.apache.hadoop.hive.serde2.objectinspector.primitive;
 
-import java.sql.Date;
-
-import org.apache.hadoop.hive.serde2.io.DateWritable;
+import org.apache.hadoop.hive.common.type.Date;
+import org.apache.hadoop.hive.serde2.io.DateWritableV2;
 
 /**
  * A SettableDecimalObjectInspector can set a Date value to an object.
  */
 public interface SettableDateObjectInspector extends DateObjectInspector {
+  @Deprecated
+  Object set(Object o, java.sql.Date d);
+
   Object set(Object o, Date d);
 
-  Object set(Object o, DateWritable d);
+  Object set(Object o, DateWritableV2 d);
+
+  @Deprecated
+  Object create(java.sql.Date d);
 
   Object create(Date d);
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/c95136a0/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/SettableTimestampObjectInspector.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/SettableTimestampObjectInspector.java b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/SettableTimestampObjectInspector.java
index c676a62..65cae16 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/SettableTimestampObjectInspector.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/SettableTimestampObjectInspector.java
@@ -17,20 +17,25 @@
  */
 package org.apache.hadoop.hive.serde2.objectinspector.primitive;
 
-import java.sql.Timestamp;
-
-import org.apache.hadoop.hive.serde2.io.TimestampWritable;
+import org.apache.hadoop.hive.common.type.Timestamp;
+import org.apache.hadoop.hive.serde2.io.TimestampWritableV2;
 
 
 public interface SettableTimestampObjectInspector extends TimestampObjectInspector {
 
   Object set(Object o, byte[] bytes, int offset);
 
+  @Deprecated
+  Object set(Object o, java.sql.Timestamp t);
+
   Object set(Object o, Timestamp t);
 
-  Object set(Object o, TimestampWritable t);
+  Object set(Object o, TimestampWritableV2 t);
 
   Object create(byte[] bytes, int offset);
 
+  @Deprecated
+  Object create (java.sql.Timestamp t);
+
   Object create (Timestamp t);
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/c95136a0/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/TimestampObjectInspector.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/TimestampObjectInspector.java b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/TimestampObjectInspector.java
index f277232..6eb2aac 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/TimestampObjectInspector.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/TimestampObjectInspector.java
@@ -17,18 +17,17 @@
  */
 package org.apache.hadoop.hive.serde2.objectinspector.primitive;
 
-import java.sql.Timestamp;
-
 import org.apache.hadoop.hive.common.classification.InterfaceAudience;
 import org.apache.hadoop.hive.common.classification.InterfaceStability;
-import org.apache.hadoop.hive.serde2.io.TimestampWritable;
+import org.apache.hadoop.hive.common.type.Timestamp;
+import org.apache.hadoop.hive.serde2.io.TimestampWritableV2;
 import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
 
 @InterfaceAudience.Public
 @InterfaceStability.Stable
 public interface TimestampObjectInspector extends PrimitiveObjectInspector {
 
-  TimestampWritable getPrimitiveWritableObject(Object o);
+  TimestampWritableV2 getPrimitiveWritableObject(Object o);
 
   Timestamp getPrimitiveJavaObject(Object o);
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/c95136a0/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableConstantDateObjectInspector.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableConstantDateObjectInspector.java b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableConstantDateObjectInspector.java
index 290fcd3..7e6cc8d 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableConstantDateObjectInspector.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableConstantDateObjectInspector.java
@@ -17,10 +17,9 @@
  */
 package org.apache.hadoop.hive.serde2.objectinspector.primitive;
 
+import org.apache.hadoop.hive.serde2.io.DateWritableV2;
 import org.apache.hadoop.hive.serde2.objectinspector.ConstantObjectInspector;
 
-import org.apache.hadoop.hive.serde2.io.DateWritable;
-
 /**
  * A WritableConstantDateObjectInspector is a WritableDateObjectInspector
  * that implements ConstantObjectInspector.
@@ -29,18 +28,18 @@ public class WritableConstantDateObjectInspector extends
     WritableDateObjectInspector implements
     ConstantObjectInspector {
 
-  private DateWritable value;
+  private DateWritableV2 value;
 
   protected WritableConstantDateObjectInspector() {
     super();
   }
-  WritableConstantDateObjectInspector(DateWritable value) {
+  WritableConstantDateObjectInspector(DateWritableV2 value) {
     super();
     this.value = value;
   }
 
   @Override
-  public DateWritable getWritableConstantValue() {
+  public DateWritableV2 getWritableConstantValue() {
     return value;
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/c95136a0/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableConstantTimestampObjectInspector.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableConstantTimestampObjectInspector.java b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableConstantTimestampObjectInspector.java
index dc8fedf..9428421 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableConstantTimestampObjectInspector.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableConstantTimestampObjectInspector.java
@@ -17,10 +17,9 @@
  */
 package org.apache.hadoop.hive.serde2.objectinspector.primitive;
 
+import org.apache.hadoop.hive.serde2.io.TimestampWritableV2;
 import org.apache.hadoop.hive.serde2.objectinspector.ConstantObjectInspector;
 
-import org.apache.hadoop.hive.serde2.io.TimestampWritable;
-
 /**
  * A WritableConstantTimestampObjectInspector is a WritableTimestampObjectInspector
  * that implements ConstantObjectInspector.
@@ -29,18 +28,18 @@ public class WritableConstantTimestampObjectInspector extends
     WritableTimestampObjectInspector implements
     ConstantObjectInspector {
 
-  private TimestampWritable value;
+  private TimestampWritableV2 value;
 
   protected WritableConstantTimestampObjectInspector() {
     super();
   }
-  WritableConstantTimestampObjectInspector(TimestampWritable value) {
+  WritableConstantTimestampObjectInspector(TimestampWritableV2 value) {
     super();
     this.value = value;
   }
 
   @Override
-  public TimestampWritable getWritableConstantValue() {
+  public TimestampWritableV2 getWritableConstantValue() {
     return value;
   }
 }