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

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

http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/serde/src/java/org/apache/hadoop/hive/serde2/io/TimestampWritable.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/io/TimestampWritable.java b/serde/src/java/org/apache/hadoop/hive/serde2/io/TimestampWritable.java
index fdc64e7..305fdbe 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/io/TimestampWritable.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/io/TimestampWritable.java
@@ -28,7 +28,6 @@ import java.text.SimpleDateFormat;
 import java.util.Date;
 
 import org.apache.hadoop.hive.common.type.HiveDecimal;
-import org.apache.hadoop.hive.common.type.PisaTimestamp;
 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;
@@ -323,7 +322,20 @@ public class TimestampWritable implements WritableComparable<TimestampWritable>
     return seconds + nanos / 1000000000;
   }
 
+  public static long getLong(Timestamp timestamp) {
+    return timestamp.getTime() / 1000;
+  }
 
+  /**
+  *
+  * @return double representation of the timestamp, accurate to nanoseconds
+  */
+ public static double getDouble(Timestamp timestamp) {
+   double seconds, nanos;
+   seconds = millisToSeconds(timestamp.getTime());
+   nanos = timestamp.getNanos();
+   return seconds + nanos / 1000000000;
+ }
 
   public void readFields(DataInput in) throws IOException {
     in.readFully(internalBytes, 0, 4);
@@ -543,6 +555,21 @@ public class TimestampWritable implements WritableComparable<TimestampWritable>
     return t;
   }
 
+  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 = TimestampWritable.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
@@ -553,6 +580,17 @@ public class TimestampWritable implements WritableComparable<TimestampWritable>
       return new Timestamp(intToTimestampInSeconds ?  time * 1000 : time);
   }
 
+  /**
+   * Converts the time in seconds or milliseconds to a timestamp.
+   * @param time time in seconds or in milliseconds
+   * @return the timestamp
+   */
+  public static void setTimestampFromLong(Timestamp timestamp, long time,
+      boolean intToTimestampInSeconds) {
+      // If the time is in seconds, converts it to milliseconds first.
+    timestamp.setTime(intToTimestampInSeconds ?  time * 1000 : time);
+  }
+
   public static Timestamp doubleToTimestamp(double f) {
     long seconds = (long) f;
 
@@ -576,6 +614,37 @@ public class TimestampWritable implements WritableComparable<TimestampWritable>
     return t;
   }
 
+  public static void setTimestampFromDouble(Timestamp timestamp, double f) {
+    // Otherwise, BigDecimal throws an exception.  (Support vector operations that sometimes
+    // do work on double Not-a-Number NaN values).
+    if (Double.isNaN(f)) {
+      timestamp.setTime(0);
+      return;
+    }
+    // Algorithm used by TimestampWritable.doubleToTimestamp method.
+    // Allocates a BigDecimal object!
+
+    long seconds = (long) f;
+
+    // We must ensure the exactness of the double's fractional portion.
+    // 0.6 as the fraction part will be converted to 0.59999... and
+    // significantly reduce the savings from binary serialization
+    BigDecimal bd = new BigDecimal(String.valueOf(f));
+    bd = bd.subtract(new BigDecimal(seconds)).multiply(new BigDecimal(1000000000));
+    int nanos = bd.intValue();
+
+    // Convert to millis
+    long millis = seconds * 1000;
+    if (nanos < 0) {
+      millis -= 1000;
+      nanos += 1000000000;
+    }
+    timestamp.setTime(millis);
+
+    // Set remaining fractional portion to nanos
+    timestamp.setNanos(nanos);
+  }
+
   public static void setTimestamp(Timestamp t, byte[] bytes, int offset) {
     boolean hasDecimalOrSecondVInt = hasDecimalOrSecondVInt(bytes[offset]);
     long seconds = (long) TimestampWritable.getSeconds(bytes, offset);

http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/storage-api/src/java/org/apache/hadoop/hive/common/type/HiveIntervalDayTime.java
----------------------------------------------------------------------
diff --git a/storage-api/src/java/org/apache/hadoop/hive/common/type/HiveIntervalDayTime.java b/storage-api/src/java/org/apache/hadoop/hive/common/type/HiveIntervalDayTime.java
new file mode 100644
index 0000000..b891e27
--- /dev/null
+++ b/storage-api/src/java/org/apache/hadoop/hive/common/type/HiveIntervalDayTime.java
@@ -0,0 +1,253 @@
+/**
+ * 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.common.type;
+
+import java.math.BigDecimal;
+import java.sql.Timestamp;
+import java.util.Date;
+import java.util.concurrent.TimeUnit;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import org.apache.hive.common.util.IntervalDayTimeUtils;
+
+import sun.util.calendar.BaseCalendar;
+
+/**
+ * Day-time interval type representing an offset in days/hours/minutes/seconds,
+ * with nanosecond precision.
+ * 1 day = 24 hours = 1440 minutes = 86400 seconds
+ */
+public class HiveIntervalDayTime implements Comparable<HiveIntervalDayTime> {
+
+  // days/hours/minutes/seconds all represented as seconds
+  protected long totalSeconds;
+  protected int nanos;
+
+  public HiveIntervalDayTime() {
+  }
+
+  public HiveIntervalDayTime(int days, int hours, int minutes, int seconds, int nanos) {
+    set(days, hours, minutes, seconds, nanos);
+  }
+
+  public HiveIntervalDayTime(long seconds, int nanos) {
+    set(seconds, nanos);
+  }
+
+  public HiveIntervalDayTime(BigDecimal seconds) {
+    set(seconds);
+  }
+
+  public HiveIntervalDayTime(HiveIntervalDayTime other) {
+    set(other.totalSeconds, other.nanos);
+  }
+
+  public int getDays() {
+    return (int) TimeUnit.SECONDS.toDays(totalSeconds);
+  }
+
+  public int getHours() {
+    return (int) (TimeUnit.SECONDS.toHours(totalSeconds) % TimeUnit.DAYS.toHours(1));
+  }
+
+  public int getMinutes() {
+    return (int) (TimeUnit.SECONDS.toMinutes(totalSeconds) % TimeUnit.HOURS.toMinutes(1));
+  }
+
+  public int getSeconds() {
+    return (int) (totalSeconds % TimeUnit.MINUTES.toSeconds(1));
+  }
+
+  public int getNanos() {
+    return nanos;
+  }
+
+  /**
+   * Returns days/hours/minutes all converted into seconds.
+   * Nanos still need to be retrieved using getNanos()
+   * @return
+   */
+  public long getTotalSeconds() {
+    return totalSeconds;
+  }
+
+  /**
+   *
+   * @return double representation of the interval day time, accurate to nanoseconds
+   */
+  public double getDouble() {
+    return totalSeconds + nanos / 1000000000;
+  }
+
+  /**
+   * Ensures that the seconds and nanoseconds fields have consistent sign
+   */
+  protected void normalizeSecondsAndNanos() {
+    if (totalSeconds > 0 && nanos < 0) {
+      --totalSeconds;
+      nanos += IntervalDayTimeUtils.NANOS_PER_SEC;
+    } else if (totalSeconds < 0 && nanos > 0) {
+      ++totalSeconds;
+      nanos -= IntervalDayTimeUtils.NANOS_PER_SEC;
+    }
+  }
+
+  public void set(int days, int hours, int minutes, int seconds, int nanos) {
+    long totalSeconds = seconds;
+    totalSeconds += TimeUnit.DAYS.toSeconds(days);
+    totalSeconds += TimeUnit.HOURS.toSeconds(hours);
+    totalSeconds += TimeUnit.MINUTES.toSeconds(minutes);
+    totalSeconds += TimeUnit.NANOSECONDS.toSeconds(nanos);
+    nanos = nanos % IntervalDayTimeUtils.NANOS_PER_SEC;
+
+    this.totalSeconds = totalSeconds;
+    this.nanos = nanos;
+
+    normalizeSecondsAndNanos();
+  }
+
+  public void set(long seconds, int nanos) {
+    this.totalSeconds = seconds;
+    this.nanos = nanos;
+    normalizeSecondsAndNanos();
+  }
+
+  public void set(BigDecimal totalSecondsBd) {
+    long totalSeconds = totalSecondsBd.longValue();
+    BigDecimal fractionalSecs = totalSecondsBd.remainder(BigDecimal.ONE);
+    int nanos = fractionalSecs.multiply(IntervalDayTimeUtils.NANOS_PER_SEC_BD).intValue();
+    set(totalSeconds, nanos);
+  }
+
+  public void set(HiveIntervalDayTime other) {
+    set(other.getTotalSeconds(), other.getNanos());
+  }
+
+  public HiveIntervalDayTime negate() {
+    return new HiveIntervalDayTime(-getTotalSeconds(), -getNanos());
+  }
+
+  @Override
+  public int compareTo(HiveIntervalDayTime other) {
+    long cmp = this.totalSeconds - other.totalSeconds;
+    if (cmp == 0) {
+      cmp = this.nanos - other.nanos;
+    }
+    if (cmp != 0) {
+      cmp = cmp > 0 ? 1 : -1;
+    }
+    return (int) cmp;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (!(obj instanceof HiveIntervalDayTime)) {
+      return false;
+    }
+    return 0 == compareTo((HiveIntervalDayTime) obj);
+  }
+
+  /**
+   * Return a copy of this object.
+   */
+  public Object clone() {
+      return new HiveIntervalDayTime(totalSeconds, nanos);
+  }
+
+  @Override
+  public int hashCode() {
+    return new HashCodeBuilder().append(totalSeconds).append(nanos).toHashCode();
+  }
+
+  @Override
+  public String toString() {
+    // If normalize() was used, then day-hour-minute-second-nanos should have the same sign.
+    // This is currently working with that assumption.
+    boolean isNegative = (totalSeconds < 0 || nanos < 0);
+    String daySecondSignStr = isNegative ? "-" : "";
+
+    return String.format("%s%d %02d:%02d:%02d.%09d",
+        daySecondSignStr, Math.abs(getDays()),
+        Math.abs(getHours()), Math.abs(getMinutes()),
+        Math.abs(getSeconds()), Math.abs(getNanos()));
+  }
+
+  public static HiveIntervalDayTime valueOf(String strVal) {
+    HiveIntervalDayTime result = null;
+    if (strVal == null) {
+      throw new IllegalArgumentException("Interval day-time string was null");
+    }
+    Matcher patternMatcher = PATTERN_MATCHER.get();
+    patternMatcher.reset(strVal);
+    if (patternMatcher.matches()) {
+      // Parse out the individual parts
+      try {
+        // Sign - whether interval is positive or negative
+        int sign = 1;
+        String field = patternMatcher.group(1);
+        if (field != null && field.equals("-")) {
+          sign = -1;
+        }
+        int days = sign *
+            IntervalDayTimeUtils.parseNumericValueWithRange("day", patternMatcher.group(2),
+                0, Integer.MAX_VALUE);
+        byte hours = (byte) (sign *
+            IntervalDayTimeUtils.parseNumericValueWithRange("hour", patternMatcher.group(3), 0, 23));
+        byte minutes = (byte) (sign *
+            IntervalDayTimeUtils.parseNumericValueWithRange("minute", patternMatcher.group(4), 0, 59));
+        int seconds = 0;
+        int nanos = 0;
+        field = patternMatcher.group(5);
+        if (field != null) {
+          BigDecimal bdSeconds = new BigDecimal(field);
+          if (bdSeconds.compareTo(IntervalDayTimeUtils.MAX_INT_BD) > 0) {
+            throw new IllegalArgumentException("seconds value of " + bdSeconds + " too large");
+          }
+          seconds = sign * bdSeconds.intValue();
+          nanos = sign * bdSeconds.subtract(new BigDecimal(bdSeconds.toBigInteger()))
+              .multiply(IntervalDayTimeUtils.NANOS_PER_SEC_BD).intValue();
+        }
+
+        result = new HiveIntervalDayTime(days, hours, minutes, seconds, nanos);
+      } catch (Exception err) {
+        throw new IllegalArgumentException("Error parsing interval day-time string: " + strVal, err);
+      }
+    } else {
+      throw new IllegalArgumentException(
+          "Interval string does not match day-time format of 'd h:m:s.n': " + strVal);
+    }
+
+    return result;
+  }
+
+  // Simple pattern: D H:M:S.nnnnnnnnn
+  private final static String PARSE_PATTERN =
+      "([+|-])?(\\d+) (\\d+):(\\d+):((\\d+)(\\.(\\d+))?)";
+
+  private static final ThreadLocal<Matcher> PATTERN_MATCHER = new ThreadLocal<Matcher>() {
+      @Override
+      protected Matcher initialValue() {
+        return Pattern.compile(PARSE_PATTERN).matcher("");
+      }
+  };
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/storage-api/src/java/org/apache/hadoop/hive/common/type/PisaTimestamp.java
----------------------------------------------------------------------
diff --git a/storage-api/src/java/org/apache/hadoop/hive/common/type/PisaTimestamp.java b/storage-api/src/java/org/apache/hadoop/hive/common/type/PisaTimestamp.java
deleted file mode 100644
index ac1e38a..0000000
--- a/storage-api/src/java/org/apache/hadoop/hive/common/type/PisaTimestamp.java
+++ /dev/null
@@ -1,609 +0,0 @@
-/**
- * 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.common.type;
-
-import java.math.BigDecimal;
-import java.sql.Timestamp;
-import java.util.Random;
-import java.util.concurrent.TimeUnit;
-
-import com.google.common.base.Preconditions;
-
-/**
- * Pisa project is named after the famous Leonardo of Pisa, or better known as Fibanacci.
- *
- * A Pisa timestamp is a timestamp without a time-zone (i.e. local) in the ISO-8601 calendar system,
- * such as 2007-12-03 10:15:30.0123456789, with accuracy to the nanosecond (1 billionth of a
- * second).
- *
- * Pisa timestamps use the same starting point as a java.sql.Timestamp -- the number of nanoseconds
- * since the epoch (1970-01-01, or the day Unix roared awake) where negative numbers represent
- * earlier days.
- *
- * However, we use the PisaTimestamp class which has different design requirements than
- * java.sql.Timestamp.  It is designed to be mutable and NOT thread-safe to avoid high memory
- * allocation / garbage collection costs.  And, provides for ease of use by our vectorization
- * code to avoid the high CPU data cache miss cost for small objects, too.  We do this by allowing
- * the epoch day and nano of day to be stored externally (i.e. vector arrays).
- *
- * And, importantly, PisaTimestamp is a light-weight class similar to the epochDay/NanoOfDay of
- * the newer Java 8 LocalDateTime class, except the timestamp is *indifferent* to timezone.
- *
- * A common usage would be to treat it as UTC.
- *
- * You can work with days, seconds, milliseconds, nanoseconds, etc.  But to work with months you
- * will need to convert to an external timestamp object and use calendars, etc.
- * *
- * The storage for a PisaTimestamp is:
- *
- *        long epochDay
- *            // The number of days since 1970-01-01 (==> similar to Java 8 LocalDate).
- *        long nanoOfDay
- *            // The number of nanoseconds within the day, with the range of
- *            //  0 to 24 * 60 * 60 * 1,000,000,000 - 1 (==> similar to Java 8 LocalTime).
- *
- * Both epochDay and nanoOfDay are signed.
- *
- * We when both epochDay and nanoOfDay are non-zero, we will maintain them so they have the
- * same sign.
- *
- */
-
-public class PisaTimestamp {
-
-  private static final long serialVersionUID = 1L;
-
-  private long epochDay;
-  private long nanoOfDay;
-
-  private Timestamp scratchTimestamp;
-
-  public static final long NANOSECONDS_PER_SECOND = TimeUnit.SECONDS.toNanos(1);
-  public static final long NANOSECONDS_PER_MILLISECOND = TimeUnit.MILLISECONDS.toNanos(1);
-  public static final long NANOSECONDS_PER_DAY = TimeUnit.DAYS.toNanos(1);
-
-  public static final long MILLISECONDS_PER_SECOND = TimeUnit.SECONDS.toMillis(1);
-  public static final long MILLISECONDS_PER_DAY = TimeUnit.DAYS.toMillis(1);
-
-  public static final long SECONDS_PER_DAY = TimeUnit.DAYS.toSeconds(1);
-
-  public static final long MIN_NANO_OF_DAY = -NANOSECONDS_PER_DAY;
-  public static final long MAX_NANO_OF_DAY = NANOSECONDS_PER_DAY;
-
-  public static final BigDecimal BIG_NANOSECONDS_PER_SECOND = new BigDecimal(NANOSECONDS_PER_SECOND);
-
-  public long getEpochDay() {
-    return epochDay;
-  }
-
-  public long getNanoOfDay() {
-    return nanoOfDay;
-  }
-
-  public PisaTimestamp() {
-    epochDay = 0;
-    nanoOfDay = 0;
-    scratchTimestamp = new Timestamp(0);
-  }
-
-  public PisaTimestamp(long epochDay, long nanoOfDay) {
-
-    Preconditions.checkState(validateIntegrity(epochDay, nanoOfDay),
-        "epochDay " + epochDay + ", nanoOfDay " + nanoOfDay + " not valid");
-
-    this.epochDay = epochDay;
-    this.nanoOfDay = nanoOfDay;
-    scratchTimestamp = new Timestamp(0);
-  }
-
-  public PisaTimestamp(Timestamp timestamp) {
-    super();
-    updateFromTimestamp(timestamp);
-  }
-
-  public void reset() {
-    epochDay = 0;
-    nanoOfDay = 0;
-  }
-
-  /**
-   * NOTE: This method validates the integrity rules between epoch day and nano of day,
-   * but not overflow/underflow of epoch day.  Since epoch day overflow/underflow can result
-   * from to client data input, that must be checked manually with <undone> as this
-   * class do not throw data range exceptions as a rule.  It leaves that choice to the caller.
-   * @param epochDay
-   * @param nanoOfDay
-   * @return true if epoch day and nano of day have integrity.
-   */
-  public static boolean validateIntegrity(long epochDay, long nanoOfDay) {
-
-    // Range check nano per day as invariant.
-    if (nanoOfDay >= NANOSECONDS_PER_DAY || nanoOfDay <= -NANOSECONDS_PER_DAY) {
-      return false;
-    }
-
-    // Signs of epoch day and nano of day must match.
-    if (!(epochDay >= 0 && nanoOfDay >= 0 ||
-          epochDay <= 0 && nanoOfDay <= 0)) {
-      return false;
-    }
-
-    return true;
-  }
-
-  /**
-   * Set this PisaTimestamp from another PisaTimestamp.
-   * @param source
-   * @return this
-   */
-  public PisaTimestamp update(PisaTimestamp source) {
-    this.epochDay = source.epochDay;
-    this.nanoOfDay = source.nanoOfDay;
-    return this;
-  }
-
-  /**
-   * Set this PisaTimestamp from a epoch day and nano of day.
-   * @param epochDay
-   * @param nanoOfDay
-   * @return this
-   */
-  public PisaTimestamp update(long epochDay, long nanoOfDay) {
-
-    Preconditions.checkState(validateIntegrity(epochDay, nanoOfDay),
-        "epochDay " + epochDay + ", nanoOfDay " + nanoOfDay + " not valid");
-
-    this.epochDay = epochDay;
-    this.nanoOfDay = nanoOfDay;
-    return this;
-  }
-
-  /**
-   * Set the PisaTimestamp from a Timestamp object.
-   * @param timestamp
-   * @return this
-   */
-  public PisaTimestamp updateFromTimestamp(Timestamp timestamp) {
-
-    long timestampTime = timestamp.getTime();
-    int nanos = timestamp.getNanos();
-
-    /**
-     * Since the Timestamp class always stores nanos as a positive quantity (0 .. 999,999,999),
-     * we have to adjust back the time (subtract) by 1,000,000,000 to get right quantity for
-     * our calculations below.  One thing it ensures is nanoOfDay will be negative.
-     */
-    if (timestampTime < 0 && nanos > 0) {
-      timestampTime -= MILLISECONDS_PER_SECOND;
-    }
-
-    // The Timestamp class does not use the milliseconds part (always 0).  It is covered by nanos.
-    long epochSeconds = timestampTime / MILLISECONDS_PER_SECOND;
-
-    nanoOfDay = (epochSeconds % SECONDS_PER_DAY) * NANOSECONDS_PER_SECOND + nanos;
-    epochDay = epochSeconds / SECONDS_PER_DAY + (nanoOfDay / NANOSECONDS_PER_DAY);
-
-    Preconditions.checkState(validateIntegrity(epochDay, nanoOfDay));
-    return this;
-  }
-
-  /**
-   * Set this PisaTimestamp from a timestamp milliseconds.
-   * @param epochMilliseconds
-   * @return this
-   */
-  public PisaTimestamp updateFromTimestampMilliseconds(long timestampMilliseconds) {
-    /**
-     * The Timestamp class setTime sets both the time (seconds stored as milliseconds) and
-     * the nanos.
-     */
-    scratchTimestamp.setTime(timestampMilliseconds);
-    updateFromTimestamp(scratchTimestamp);
-    return this;
-  }
-
-  /**
-   * Set this PisaTimestamp from a timestamp seconds.
-   * @param epochMilliseconds
-   * @return this
-   */
-  public PisaTimestamp updateFromTimestampSeconds(long timestampSeconds) {
-    return updateFromTimestampMilliseconds(timestampSeconds * MILLISECONDS_PER_SECOND);
-  }
-
-  /**
-   * Set this PisaTimestamp from a timestamp seconds.
-   * @param epochMilliseconds
-   * @return this
-   */
-  public PisaTimestamp updateFromTimestampSecondsWithFractionalNanoseconds(
-      double timestampSecondsWithFractionalNanoseconds) {
-
-    // Otherwise, BigDecimal throws an exception.  (Support vector operations that sometimes
-    // do work on double Not-a-Number NaN values).
-    if (Double.isNaN(timestampSecondsWithFractionalNanoseconds)) {
-      timestampSecondsWithFractionalNanoseconds = 0;
-    }
-    // Algorithm used by TimestampWritable.doubleToTimestamp method.
-    // Allocates a BigDecimal object!
-
-    long seconds = (long) timestampSecondsWithFractionalNanoseconds;
-
-    // We must ensure the exactness of the double's fractional portion.
-    // 0.6 as the fraction part will be converted to 0.59999... and
-    // significantly reduce the savings from binary serialization.
-    BigDecimal bd;
-
-    bd = new BigDecimal(String.valueOf(timestampSecondsWithFractionalNanoseconds));
-    bd = bd.subtract(new BigDecimal(seconds));       // Get the nanos fraction.
-    bd = bd.multiply(BIG_NANOSECONDS_PER_SECOND);    // Make nanos an integer.
-
-    int nanos = bd.intValue();
-
-    // Convert to millis
-    long millis = seconds * 1000;
-    if (nanos < 0) {
-      millis -= 1000;
-      nanos += 1000000000;
-    }
-
-    scratchTimestamp.setTime(millis);
-    scratchTimestamp.setNanos(nanos);
-    updateFromTimestamp(scratchTimestamp);
-    return this;
-  }
-
-  /**
-   * Set this PisaTimestamp from a epoch seconds and signed nanos (-999999999 to 999999999).
-   * @param epochSeconds
-   * @param signedNanos
-   * @return this
-   */
-  public PisaTimestamp updateFromEpochSecondsAndSignedNanos(long epochSeconds, int signedNanos) {
-
-    long nanoOfDay = (epochSeconds % SECONDS_PER_DAY) * NANOSECONDS_PER_SECOND + signedNanos;
-    long epochDay = epochSeconds / SECONDS_PER_DAY + nanoOfDay / NANOSECONDS_PER_DAY;
-
-    Preconditions.checkState(validateIntegrity(epochDay, nanoOfDay));
-
-    this.epochDay = epochDay;
-    this.nanoOfDay = nanoOfDay;
-    return this;
-  }
-
-  /**
-   * Set a scratch PisaTimestamp with this PisaTimestamp's values and return the scratch object.
-   * @param epochDay
-   * @param nanoOfDay
-   */
-  public PisaTimestamp scratchCopy(PisaTimestamp scratch) {
-
-    scratch.epochDay = epochDay;
-    scratch.nanoOfDay = nanoOfDay;
-    return scratch;
-  }
-
-  /**
-   * Set a Timestamp object from this PisaTimestamp.
-   * @param timestamp
-   */
-  public void timestampUpdate(Timestamp timestamp) {
-
-    /*
-     * java.sql.Timestamp consists of a long variable to store milliseconds and an integer variable for nanoseconds.
-     * The long variable is used to store only the full seconds converted to millis. For example for 1234 milliseconds,
-     * 1000 is stored in the long variable, and 234000000 (234 converted to nanoseconds) is stored as nanoseconds.
-     * The negative timestamps are also supported, but nanoseconds must be positive therefore millisecond part is
-     * reduced by one second.
-     */
-
-    long epochSeconds = epochDay * SECONDS_PER_DAY + nanoOfDay / NANOSECONDS_PER_SECOND;
-    long integralSecInMillis;
-    int nanos = (int) (nanoOfDay % NANOSECONDS_PER_SECOND); // The nanoseconds.
-    if (nanos < 0) {
-      nanos = (int) NANOSECONDS_PER_SECOND + nanos; // The positive nano-part that will be added to milliseconds.
-      integralSecInMillis = (epochSeconds - 1) * MILLISECONDS_PER_SECOND; // Reduce by one second.
-    } else {
-      integralSecInMillis = epochSeconds * MILLISECONDS_PER_SECOND; // Full seconds converted to millis.
-    }
-
-    timestamp.setTime(integralSecInMillis);
-    timestamp.setNanos(nanos);
-  }
-
-  /**
-   * Return the scratch timestamp with values from Pisa timestamp.
-   * @return
-   */
-  public Timestamp asScratchTimestamp() {
-    timestampUpdate(scratchTimestamp);
-    return scratchTimestamp;
-  }
-
-  /**
-   * Return the scratch timestamp for use by the caller.
-   * @return
-   */
-  public Timestamp useScratchTimestamp() {
-    return scratchTimestamp;
-  }
-
-  public int compareTo(PisaTimestamp another) {
-
-    if (epochDay == another.epochDay) {
-      if (nanoOfDay == another.nanoOfDay){
-        return 0;
-      } else {
-        return (nanoOfDay < another.nanoOfDay ? -1 : 1);
-      }
-    } else {
-      return (epochDay < another.epochDay ? -1: 1);
-    }
-  }
-
-  public static int compareTo(long epochDay1, long nanoOfDay1, PisaTimestamp another) {
-
-    if (epochDay1 == another.epochDay) {
-      if (nanoOfDay1 == another.nanoOfDay){
-        return 0;
-      } else {
-        return (nanoOfDay1 < another.nanoOfDay ? -1 : 1);
-      }
-    } else {
-      return (epochDay1 < another.epochDay ? -1: 1);
-    }
-  }
-
-  public static int compareTo(PisaTimestamp pisaTimestamp1, long epochDay2, long nanoOfDay2) {
-
-    if (pisaTimestamp1.epochDay == epochDay2) {
-      if (pisaTimestamp1.nanoOfDay == nanoOfDay2){
-        return 0;
-      } else {
-        return (pisaTimestamp1.nanoOfDay < nanoOfDay2 ? -1 : 1);
-      }
-    } else {
-      return (pisaTimestamp1.epochDay < epochDay2 ? -1: 1);
-    }
-  }
-
-  public static int compareTo(long epochDay1, long nanoOfDay1, long epochDay2, long nanoOfDay2) {
-
-    if (epochDay1 == epochDay2) {
-      if (nanoOfDay1 == nanoOfDay2){
-        return 0;
-      } else {
-        return (nanoOfDay1 < nanoOfDay2 ? -1 : 1);
-      }
-    } else {
-      return (epochDay1 < epochDay2 ? -1: 1);
-    }
-  }
-
-
-  /**
-   * Standard equals method override.
-   */
-  @Override
-  public boolean equals(Object obj) {
-    if (obj == null || obj.getClass() != getClass()) {
-      return false;
-    }
-    return equals((PisaTimestamp) obj);
-  }
-
-  public boolean equals(PisaTimestamp other) {
-
-    if (epochDay == other.epochDay) {
-      if  (nanoOfDay == other.nanoOfDay) {
-        return true;
-      } else {
-        return false;
-      }
-    } else {
-        return false;
-    }
-  }
-
-  public static void add(PisaTimestamp pisaTimestamp1, PisaTimestamp pisaTimestamp2,
-      PisaTimestamp result) {
-    add(pisaTimestamp1.epochDay, pisaTimestamp1.nanoOfDay,
-        pisaTimestamp2.epochDay, pisaTimestamp2.nanoOfDay,
-        result);
-  }
-
-  public static void add(long epochDay1, long nanoOfDay1,
-      long epochDay2, long nanoOfDay2,
-      PisaTimestamp result) {
-
-    // Validate integrity rules between epoch day and nano of day.
-    Preconditions.checkState(PisaTimestamp.validateIntegrity(epochDay1, nanoOfDay1));
-    Preconditions.checkState(PisaTimestamp.validateIntegrity(epochDay2, nanoOfDay2));
-
-    long intermediateEpochDay = epochDay1 + epochDay2;
-    long intermediateNanoOfDay = nanoOfDay1 + nanoOfDay2;
-
-    // Normalize so both are positive or both are negative.
-    long normalizedEpochDay;
-    long normalizedNanoOfDay;
-    if (intermediateEpochDay > 0 && intermediateNanoOfDay < 0) {
-      normalizedEpochDay = intermediateEpochDay - 1;
-      normalizedNanoOfDay = intermediateNanoOfDay + NANOSECONDS_PER_DAY;
-    } else if (intermediateEpochDay < 0 && intermediateNanoOfDay > 0) {
-      normalizedEpochDay = intermediateEpochDay + 1;
-      normalizedNanoOfDay = intermediateNanoOfDay - NANOSECONDS_PER_DAY;
-    } else {
-      normalizedEpochDay = intermediateEpochDay;
-      normalizedNanoOfDay = intermediateNanoOfDay;
-    }
-
-    long resultEpochDay;
-    long resultNanoOfDay;
-    if (normalizedNanoOfDay >= NANOSECONDS_PER_DAY || normalizedNanoOfDay <= -NANOSECONDS_PER_DAY) {
-      // Adjust for carry or overflow...
-
-      resultEpochDay = normalizedEpochDay + normalizedNanoOfDay / NANOSECONDS_PER_DAY;
-      resultNanoOfDay = normalizedNanoOfDay % NANOSECONDS_PER_DAY;
-
-    } else {
-      resultEpochDay = normalizedEpochDay;
-      resultNanoOfDay = normalizedNanoOfDay;
-    }
-
-    // The update method will validate integrity rules between epoch day and nano of day,
-    // but not overflow/underflow of epoch day.
-    result.update(resultEpochDay, resultNanoOfDay);
-  }
-
-  public static void addSeconds(PisaTimestamp timestamp1, long epochSeconds, PisaTimestamp result) {
-    long epochDay = epochSeconds / SECONDS_PER_DAY;
-    long nanoOfDay = (epochSeconds % SECONDS_PER_DAY) * NANOSECONDS_PER_SECOND;
-    add(timestamp1.epochDay, timestamp1.nanoOfDay, epochDay, nanoOfDay, result);
-  }
-
-  public static void subtract(PisaTimestamp timestamp1, PisaTimestamp timestamp2,
-      PisaTimestamp result) {
-
-    add(timestamp1.epochDay, timestamp1.nanoOfDay, -timestamp2.epochDay, -timestamp2.nanoOfDay,
-        result);
-  }
-
-  public static void subtract(long epochDay1, long nanoOfDay1,
-      long epochDay2, long nanoOfDay2,
-      PisaTimestamp result) {
-
-    add(epochDay1, nanoOfDay1, -epochDay2, -nanoOfDay2, result);
-  }
-
-  public static void subtractSeconds(PisaTimestamp timestamp1, long epochSeconds,
-      PisaTimestamp result) {
-    long epochDay = epochSeconds / SECONDS_PER_DAY;
-    long nanoOfDay = (epochSeconds % SECONDS_PER_DAY) * NANOSECONDS_PER_SECOND;
-    add(timestamp1.epochDay, timestamp1.nanoOfDay, -epochDay, -nanoOfDay, result);
-  }
-
-  /**
-   * Rounds the number of milliseconds relative to the epoch down to the nearest whole number of
-   * seconds. 500 would round to 0, -500 would round to -1.
-   */
-  public static long timestampMillisToSeconds(long millis) {
-    if (millis >= 0) {
-      return millis / 1000;
-    } else {
-      return (millis - 999) / 1000;
-    }
-  }
-
-  /**
-   * Return a double with the integer part as the seconds and the fractional part as
-   * the nanoseconds the way the Timestamp class does it.
-   * @return seconds.nanoseconds
-   */
-  public double getTimestampSecondsWithFractionalNanos() {
-    // Algorithm must be the same as TimestampWritable.getDouble method.
-    timestampUpdate(scratchTimestamp);
-    double seconds = timestampMillisToSeconds(scratchTimestamp.getTime());
-    double nanos = scratchTimestamp.getNanos();
-    BigDecimal bigSeconds = new BigDecimal(seconds);
-    BigDecimal bigNanos = new BigDecimal(nanos).divide(BIG_NANOSECONDS_PER_SECOND);
-    return bigSeconds.add(bigNanos).doubleValue();
-  }
-
-  /**
-   * Return an integer as the seconds the way the Timestamp class does it.
-   * @return seconds.nanoseconds
-   */
-  public long getTimestampSeconds() {
-    // Algorithm must be the same as TimestampWritable.getSeconds method.
-    timestampUpdate(scratchTimestamp);
-    return timestampMillisToSeconds(scratchTimestamp.getTime());
-  }
-
-  /**
-   * Return an integer as the milliseconds the way the Timestamp class does it.
-   * @return seconds.nanoseconds
-   */
-  public long getTimestampMilliseconds() {
-    timestampUpdate(scratchTimestamp);
-    return scratchTimestamp.getTime();
-  }
-
-  /**
-   * Return the epoch seconds.
-   * @return
-   */
-  public long getEpochSeconds() {
-    return epochDay * SECONDS_PER_DAY + nanoOfDay / NANOSECONDS_PER_SECOND;
-  }
-
-  /**
-   * Return the epoch seconds, given the epoch day and nano of day.
-   * @param epochDay
-   * @param nanoOfDay
-   * @return
-   */
-  public static long getEpochSecondsFromEpochDayAndNanoOfDay(long epochDay, long nanoOfDay) {
-    return epochDay * SECONDS_PER_DAY + nanoOfDay / NANOSECONDS_PER_SECOND;
-  }
-
-  /**
-   * Return the signed nanos (-999999999 to 999999999).
-   * NOTE: Not the same as Timestamp class nanos (which are always positive).
-   */
-  public int getSignedNanos() {
-    return (int) (nanoOfDay % NANOSECONDS_PER_SECOND);
-  }
-
-  /**
-   * Return the signed nanos (-999999999 to 999999999).
-   * NOTE: Not the same as Timestamp class nanos (which are always positive).
-   */
-  public static int getSignedNanos(long nanoOfDay) {
-    return (int) (nanoOfDay % NANOSECONDS_PER_SECOND);
-  }
-
-  /**
-   * Return the epoch milliseconds.
-   * @return
-   */
-  public long getEpochMilliseconds() {
-    return epochDay * MILLISECONDS_PER_DAY + nanoOfDay / NANOSECONDS_PER_MILLISECOND;
-  }
-
-  /**
-   * Return the epoch seconds, given the epoch day and nano of day.
-   * @param epochDay
-   * @param nanoOfDay
-   * @return
-   */
-  public static long getEpochMillisecondsFromEpochDayAndNanoOfDay(long epochDay, long nanoOfDay) {
-    return epochDay * MILLISECONDS_PER_DAY + nanoOfDay / NANOSECONDS_PER_MILLISECOND;
-  }
-
-  @Override
-  public int hashCode() {
-    // UNDONE: We don't want to box the longs just to get the hash codes...
-    return new Long(epochDay).hashCode() ^ new Long(nanoOfDay).hashCode();
-  }
-
-  @Override
-  public String toString() {
-    timestampUpdate(scratchTimestamp);
-    return scratchTimestamp.toString();
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/storage-api/src/java/org/apache/hadoop/hive/common/type/RandomTypeUtil.java
----------------------------------------------------------------------
diff --git a/storage-api/src/java/org/apache/hadoop/hive/common/type/RandomTypeUtil.java b/storage-api/src/java/org/apache/hadoop/hive/common/type/RandomTypeUtil.java
index 13baff4..3fb0cfd 100644
--- a/storage-api/src/java/org/apache/hadoop/hive/common/type/RandomTypeUtil.java
+++ b/storage-api/src/java/org/apache/hadoop/hive/common/type/RandomTypeUtil.java
@@ -18,21 +18,67 @@
 package org.apache.hadoop.hive.common.type;
 
 import java.sql.Timestamp;
+import java.text.DateFormat;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
 import java.util.Random;
 import java.util.concurrent.TimeUnit;
 
 public class RandomTypeUtil {
 
   public static final long NANOSECONDS_PER_SECOND = TimeUnit.SECONDS.toNanos(1);
+  public static final long MILLISECONDS_PER_SECOND = TimeUnit.SECONDS.toMillis(1);
+  public static final long NANOSECONDS_PER_MILLISSECOND = TimeUnit.MILLISECONDS.toNanos(1);
+
+  private static ThreadLocal<DateFormat> DATE_FORMAT =
+      new ThreadLocal<DateFormat>() {
+        @Override
+        protected DateFormat initialValue() {
+          return new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
+        }
+      };
+
+  // We've switched to Joda/Java Calendar which has a more limited time range....
+  public static int MIN_YEAR = 1900;
+  public static int MAX_YEAR = 3000;
+  private static long MIN_FOUR_DIGIT_YEAR_MILLIS = parseToMillis("1900-01-01 00:00:00");
+  private static long MAX_FOUR_DIGIT_YEAR_MILLIS = parseToMillis("3000-01-01 00:00:00");
+
+  private static long parseToMillis(String s) {
+    try {
+      return DATE_FORMAT.get().parse(s).getTime();
+    } catch (ParseException ex) {
+      throw new RuntimeException(ex);
+    }
+  }
 
   public static Timestamp getRandTimestamp(Random r) {
+    return getRandTimestamp(r, MIN_YEAR, MAX_YEAR);
+  }
+
+  public static Timestamp getRandTimestamp(Random r, int minYear, int maxYear) {
     String optionalNanos = "";
-    if (r.nextInt(2) == 1) {
+    switch (r.nextInt(4)) {
+    case 0:
+      // No nanos.
+      break;
+    case 1:
+      optionalNanos = String.format(".%09d",
+          Integer.valueOf(r.nextInt((int) NANOSECONDS_PER_SECOND)));
+      break;
+    case 2:
+      // Limit to milliseconds only...
       optionalNanos = String.format(".%09d",
-          Integer.valueOf(0 + r.nextInt((int) NANOSECONDS_PER_SECOND)));
+          Integer.valueOf(r.nextInt((int) MILLISECONDS_PER_SECOND)) * NANOSECONDS_PER_MILLISSECOND);
+      break;
+    case 3:
+      // Limit to below milliseconds only...
+      optionalNanos = String.format(".%09d",
+          Integer.valueOf(r.nextInt((int) NANOSECONDS_PER_MILLISSECOND)));
+      break;
     }
     String timestampStr = String.format("%04d-%02d-%02d %02d:%02d:%02d%s",
-        Integer.valueOf(0 + r.nextInt(10000)),  // year
+        Integer.valueOf(minYear + r.nextInt(maxYear - minYear + 1)),  // year
         Integer.valueOf(1 + r.nextInt(12)),      // month
         Integer.valueOf(1 + r.nextInt(28)),      // day
         Integer.valueOf(0 + r.nextInt(24)),      // hour
@@ -48,4 +94,22 @@ public class RandomTypeUtil {
     }
     return timestampVal;
   }
+
+  public static long randomMillis(long minMillis, long maxMillis, Random rand) {
+    return minMillis + (long) ((maxMillis - minMillis) * rand.nextDouble());
+  }
+
+  public static long randomMillis(Random rand) {
+    return randomMillis(MIN_FOUR_DIGIT_YEAR_MILLIS, MAX_FOUR_DIGIT_YEAR_MILLIS, rand);
+  }
+
+  public static int randomNanos(Random rand, int decimalDigits) {
+    // Only keep the most significant decimalDigits digits.
+    int nanos = rand.nextInt((int) NANOSECONDS_PER_SECOND);
+    return nanos - nanos % (int) Math.pow(10, 9 - decimalDigits);
+  }
+
+  public static int randomNanos(Random rand) {
+    return randomNanos(rand, 9);
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/ColumnVector.java
----------------------------------------------------------------------
diff --git a/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/ColumnVector.java b/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/ColumnVector.java
index 4ae9c47..c069a5f 100644
--- a/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/ColumnVector.java
+++ b/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/ColumnVector.java
@@ -18,7 +18,6 @@
 
 package org.apache.hadoop.hive.ql.exec.vector;
 
-import java.io.IOException;
 import java.util.Arrays;
 
 /**
@@ -43,6 +42,7 @@ public abstract class ColumnVector {
     BYTES,
     DECIMAL,
     TIMESTAMP,
+    INTERVAL_DAY_TIME,
     STRUCT,
     LIST,
     MAP,

http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/IntervalDayTimeColumnVector.java
----------------------------------------------------------------------
diff --git a/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/IntervalDayTimeColumnVector.java b/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/IntervalDayTimeColumnVector.java
new file mode 100644
index 0000000..39ccea8
--- /dev/null
+++ b/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/IntervalDayTimeColumnVector.java
@@ -0,0 +1,348 @@
+/**
+ * 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.ql.exec.vector;
+
+import java.util.Arrays;
+
+import org.apache.hadoop.hive.common.type.HiveIntervalDayTime;
+import org.apache.hadoop.io.Writable;
+
+/**
+ * This class represents a nullable interval day time column vector capable of handing a
+ * wide range of interval day time values.
+ *
+ * We store the 2 (value) fields of a HiveIntervalDayTime class in primitive arrays.
+ *
+ * We do this to avoid an array of Java HiveIntervalDayTime objects which would have poor storage
+ * and memory access characteristics.
+ *
+ * Generally, the caller will fill in a scratch HiveIntervalDayTime object with values from a row,
+ * work using the scratch HiveIntervalDayTime, and then perhaps update the column vector row
+ * with a result.
+ */
+public class IntervalDayTimeColumnVector extends ColumnVector {
+
+  /*
+   * The storage arrays for this column vector corresponds to the storage of a HiveIntervalDayTime:
+   */
+  private long[] totalSeconds;
+      // The values from HiveIntervalDayTime.getTotalSeconds().
+
+  private int[] nanos;
+      // The values from HiveIntervalDayTime.getNanos().
+
+  /*
+   * Scratch objects.
+   */
+  private final HiveIntervalDayTime scratchIntervalDayTime;
+
+  private Writable scratchWritable;
+      // Supports keeping a HiveIntervalDayTimeWritable object without having to import
+      // that definition...
+
+  /**
+   * Use this constructor by default. All column vectors
+   * should normally be the default size.
+   */
+  public IntervalDayTimeColumnVector() {
+    this(VectorizedRowBatch.DEFAULT_SIZE);
+  }
+
+  /**
+   * Don't use this except for testing purposes.
+   *
+   * @param len the number of rows
+   */
+  public IntervalDayTimeColumnVector(int len) {
+    super(len);
+
+    totalSeconds = new long[len];
+    nanos = new int[len];
+
+    scratchIntervalDayTime = new HiveIntervalDayTime();
+
+    scratchWritable = null;     // Allocated by caller.
+  }
+
+  /**
+   * Return the number of rows.
+   * @return
+   */
+  public int getLength() {
+    return totalSeconds.length;
+  }
+
+  /**
+   * Return a row's HiveIntervalDayTime.getTotalSeconds() value.
+   * We assume the entry has already been NULL checked and isRepeated adjusted.
+   * @param elementNum
+   * @return
+   */
+  public long getTotalSeconds(int elementNum) {
+    return totalSeconds[elementNum];
+  }
+
+  /**
+   * Return a row's HiveIntervalDayTime.getNanos() value.
+   * We assume the entry has already been NULL checked and isRepeated adjusted.
+   * @param elementNum
+   * @return
+   */
+  public long getNanos(int elementNum) {
+    return nanos[elementNum];
+  }
+
+  /**
+   * Return a row's HiveIntervalDayTime.getDouble() value.
+   * We assume the entry has already been NULL checked and isRepeated adjusted.
+   * @param elementNum
+   * @return
+   */
+  public double getDouble(int elementNum) {
+    return asScratchIntervalDayTime(elementNum).getDouble();
+  }
+
+  /**
+   * Set a HiveIntervalDayTime object from a row of the column.
+   * We assume the entry has already been NULL checked and isRepeated adjusted.
+   * @param intervalDayTime
+   * @param elementNum
+   */
+  public void intervalDayTimeUpdate(HiveIntervalDayTime intervalDayTime, int elementNum) {
+    intervalDayTime.set(totalSeconds[elementNum], nanos[elementNum]);
+  }
+
+
+  /**
+   * Return the scratch HiveIntervalDayTime object set from a row.
+   * We assume the entry has already been NULL checked and isRepeated adjusted.
+   * @param elementNum
+   * @return
+   */
+  public HiveIntervalDayTime asScratchIntervalDayTime(int elementNum) {
+    scratchIntervalDayTime.set(totalSeconds[elementNum], nanos[elementNum]);
+    return scratchIntervalDayTime;
+  }
+
+  /**
+   * Return the scratch HiveIntervalDayTime (contents undefined).
+   * @return
+   */
+  public HiveIntervalDayTime getScratchIntervalDayTime() {
+    return scratchIntervalDayTime;
+  }
+
+  /**
+   * Compare row to HiveIntervalDayTime.
+   * We assume the entry has already been NULL checked and isRepeated adjusted.
+   * @param elementNum
+   * @param intervalDayTime
+   * @return -1, 0, 1 standard compareTo values.
+   */
+  public int compareTo(int elementNum, HiveIntervalDayTime intervalDayTime) {
+    return asScratchIntervalDayTime(elementNum).compareTo(intervalDayTime);
+  }
+
+  /**
+   * Compare HiveIntervalDayTime to row.
+   * We assume the entry has already been NULL checked and isRepeated adjusted.
+   * @param intervalDayTime
+   * @param elementNum
+   * @return -1, 0, 1 standard compareTo values.
+   */
+  public int compareTo(HiveIntervalDayTime intervalDayTime, int elementNum) {
+    return intervalDayTime.compareTo(asScratchIntervalDayTime(elementNum));
+  }
+
+  /**
+   * Compare a row to another TimestampColumnVector's row.
+   * @param elementNum1
+   * @param intervalDayTimeColVector2
+   * @param elementNum2
+   * @return
+   */
+  public int compareTo(int elementNum1, IntervalDayTimeColumnVector intervalDayTimeColVector2,
+      int elementNum2) {
+    return asScratchIntervalDayTime(elementNum1).compareTo(
+        intervalDayTimeColVector2.asScratchIntervalDayTime(elementNum2));
+  }
+
+  /**
+   * Compare another TimestampColumnVector's row to a row.
+   * @param intervalDayTimeColVector1
+   * @param elementNum1
+   * @param elementNum2
+   * @return
+   */
+  public int compareTo(IntervalDayTimeColumnVector intervalDayTimeColVector1, int elementNum1,
+      int elementNum2) {
+    return intervalDayTimeColVector1.asScratchIntervalDayTime(elementNum1).compareTo(
+        asScratchIntervalDayTime(elementNum2));
+  }
+
+  @Override
+  public void setElement(int outElementNum, int inputElementNum, ColumnVector inputVector) {
+
+    IntervalDayTimeColumnVector timestampColVector = (IntervalDayTimeColumnVector) inputVector;
+
+    totalSeconds[outElementNum] = timestampColVector.totalSeconds[inputElementNum];
+    nanos[outElementNum] = timestampColVector.nanos[inputElementNum];
+  }
+
+  // Simplify vector by brute-force flattening noNulls and isRepeating
+  // This can be used to reduce combinatorial explosion of code paths in VectorExpressions
+  // with many arguments.
+  public void flatten(boolean selectedInUse, int[] sel, int size) {
+    flattenPush();
+    if (isRepeating) {
+      isRepeating = false;
+      long repeatFastTime = totalSeconds[0];
+      int repeatNanos = nanos[0];
+      if (selectedInUse) {
+        for (int j = 0; j < size; j++) {
+          int i = sel[j];
+          totalSeconds[i] = repeatFastTime;
+          nanos[i] = repeatNanos;
+        }
+      } else {
+        Arrays.fill(totalSeconds, 0, size, repeatFastTime);
+        Arrays.fill(nanos, 0, size, repeatNanos);
+      }
+      flattenRepeatingNulls(selectedInUse, sel, size);
+    }
+    flattenNoNulls(selectedInUse, sel, size);
+  }
+
+  /**
+   * Set a row from a HiveIntervalDayTime.
+   * We assume the entry has already been isRepeated adjusted.
+   * @param elementNum
+   * @param intervalDayTime
+   */
+  public void set(int elementNum, HiveIntervalDayTime intervalDayTime) {
+    this.totalSeconds[elementNum] = intervalDayTime.getTotalSeconds();
+    this.nanos[elementNum] = intervalDayTime.getNanos();
+  }
+
+  /**
+   * Set a row from the current value in the scratch interval day time.
+   * @param elementNum
+   */
+  public void setFromScratchIntervalDayTime(int elementNum) {
+    this.totalSeconds[elementNum] = scratchIntervalDayTime.getTotalSeconds();
+    this.nanos[elementNum] = scratchIntervalDayTime.getNanos();
+  }
+
+  /**
+   * Set row to standard null value(s).
+   * We assume the entry has already been isRepeated adjusted.
+   * @param elementNum
+   */
+  public void setNullValue(int elementNum) {
+    totalSeconds[elementNum] = 0;
+    nanos[elementNum] = 1;
+  }
+
+  // Copy the current object contents into the output. Only copy selected entries,
+  // as indicated by selectedInUse and the sel array.
+  public void copySelected(
+      boolean selectedInUse, int[] sel, int size, IntervalDayTimeColumnVector output) {
+
+    // Output has nulls if and only if input has nulls.
+    output.noNulls = noNulls;
+    output.isRepeating = false;
+
+    // Handle repeating case
+    if (isRepeating) {
+      output.totalSeconds[0] = totalSeconds[0];
+      output.nanos[0] = nanos[0];
+      output.isNull[0] = isNull[0];
+      output.isRepeating = true;
+      return;
+    }
+
+    // Handle normal case
+
+    // Copy data values over
+    if (selectedInUse) {
+      for (int j = 0; j < size; j++) {
+        int i = sel[j];
+        output.totalSeconds[i] = totalSeconds[i];
+        output.nanos[i] = nanos[i];
+      }
+    }
+    else {
+      System.arraycopy(totalSeconds, 0, output.totalSeconds, 0, size);
+      System.arraycopy(nanos, 0, output.nanos, 0, size);
+    }
+
+    // Copy nulls over if needed
+    if (!noNulls) {
+      if (selectedInUse) {
+        for (int j = 0; j < size; j++) {
+          int i = sel[j];
+          output.isNull[i] = isNull[i];
+        }
+      }
+      else {
+        System.arraycopy(isNull, 0, output.isNull, 0, size);
+      }
+    }
+  }
+
+  /**
+   * Fill all the vector entries with a HiveIntervalDayTime.
+   * @param intervalDayTime
+   */
+  public void fill(HiveIntervalDayTime intervalDayTime) {
+    noNulls = true;
+    isRepeating = true;
+    totalSeconds[0] = intervalDayTime.getTotalSeconds();
+    nanos[0] = intervalDayTime.getNanos();
+  }
+
+  /**
+   * Return a convenience writable object stored by this column vector.
+   * Supports keeping a TimestampWritable object without having to import that definition...
+   * @return
+   */
+  public Writable getScratchWritable() {
+    return scratchWritable;
+  }
+
+  /**
+   * Set the convenience writable object stored by this column vector
+   * @param scratchWritable
+   */
+  public void setScratchWritable(Writable scratchWritable) {
+    this.scratchWritable = scratchWritable;
+  }
+
+  @Override
+  public void stringifyValue(StringBuilder buffer, int row) {
+    if (isRepeating) {
+      row = 0;
+    }
+    if (noNulls || !isNull[row]) {
+      scratchIntervalDayTime.set(totalSeconds[row], nanos[row]);
+      buffer.append(scratchIntervalDayTime.toString());
+    } else {
+      buffer.append("null");
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/TimestampColumnVector.java
----------------------------------------------------------------------
diff --git a/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/TimestampColumnVector.java b/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/TimestampColumnVector.java
index b73a0d2..c0dd5ed 100644
--- a/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/TimestampColumnVector.java
+++ b/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/TimestampColumnVector.java
@@ -20,37 +20,35 @@ package org.apache.hadoop.hive.ql.exec.vector;
 import java.sql.Timestamp;
 import java.util.Arrays;
 
-import org.apache.hadoop.hive.common.type.PisaTimestamp;
-import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.io.Writable;
 
-import com.google.common.base.Preconditions;
-
 /**
  * This class represents a nullable timestamp column vector capable of handing a wide range of
  * timestamp values.
  *
- * We use the PisaTimestamp which is designed to be mutable and avoid the heavy memory allocation
- * and CPU data cache miss costs.
+ * We store the 2 (value) fields of a Timestamp class in primitive arrays.
+ *
+ * We do this to avoid an array of Java Timestamp objects which would have poor storage
+ * and memory access characteristics.
+ *
+ * Generally, the caller will fill in a scratch timestamp object with values from a row, work
+ * using the scratch timestamp, and then perhaps update the column vector row with a result.
  */
 public class TimestampColumnVector extends ColumnVector {
 
   /*
-   * The storage arrays for this column vector corresponds to the storage of a PisaTimestamp:
+   * The storage arrays for this column vector corresponds to the storage of a Timestamp:
    */
-  private long[] epochDay;
-      // An array of the number of days since 1970-01-01 (similar to Java 8 LocalDate).
+  public long[] time;
+      // The values from Timestamp.getTime().
 
-  private long[] nanoOfDay;
-      // An array of the number of nanoseconds within the day, with the range of
-      // 0 to 24 * 60 * 60 * 1,000,000,000 - 1 (similar to Java 8 LocalTime).
+  public int[] nanos;
+      // The values from Timestamp.getNanos().
 
   /*
    * Scratch objects.
    */
-  private PisaTimestamp scratchPisaTimestamp;
-      // Convenience scratch Pisa timestamp object.
+  private final Timestamp scratchTimestamp;
 
   private Writable scratchWritable;
       // Supports keeping a TimestampWritable object without having to import that definition...
@@ -71,10 +69,10 @@ public class TimestampColumnVector extends ColumnVector {
   public TimestampColumnVector(int len) {
     super(len);
 
-    epochDay = new long[len];
-    nanoOfDay = new long[len];
+    time = new long[len];
+    nanos = new int[len];
 
-    scratchPisaTimestamp = new PisaTimestamp();
+    scratchTimestamp = new Timestamp(0);
 
     scratchWritable = null;     // Allocated by caller.
   }
@@ -84,48 +82,27 @@ public class TimestampColumnVector extends ColumnVector {
    * @return
    */
   public int getLength() {
-    return epochDay.length;
+    return time.length;
   }
 
   /**
-   * Returnt a row's epoch day.
+   * Return a row's Timestamp.getTime() value.
    * We assume the entry has already been NULL checked and isRepeated adjusted.
    * @param elementNum
    * @return
    */
-  public long getEpochDay(int elementNum) {
-    return epochDay[elementNum];
+  public long getTime(int elementNum) {
+    return time[elementNum];
   }
 
   /**
-   * Return a row's nano of day.
+   * Return a row's Timestamp.getNanos() value.
    * We assume the entry has already been NULL checked and isRepeated adjusted.
    * @param elementNum
    * @return
    */
-  public long getNanoOfDay(int elementNum) {
-    return nanoOfDay[elementNum];
-  }
-
-  /**
-   * Get a scratch PisaTimestamp object from a row of the column.
-   * We assume the entry has already been NULL checked and isRepeated adjusted.
-   * @param elementNum
-   * @return scratch
-   */
-  public PisaTimestamp asScratchPisaTimestamp(int elementNum) {
-    scratchPisaTimestamp.update(epochDay[elementNum], nanoOfDay[elementNum]);
-    return scratchPisaTimestamp;
-  }
-
-  /**
-   * Set a PisaTimestamp object from a row of the column.
-   * We assume the entry has already been NULL checked and isRepeated adjusted.
-   * @param pisaTimestamp
-   * @param elementNum
-   */
-  public void pisaTimestampUpdate(PisaTimestamp pisaTimestamp, int elementNum) {
-    pisaTimestamp.update(epochDay[elementNum], nanoOfDay[elementNum]);
+  public int getNanos(int elementNum) {
+    return nanos[elementNum];
   }
 
   /**
@@ -135,154 +112,133 @@ public class TimestampColumnVector extends ColumnVector {
    * @param elementNum
    */
   public void timestampUpdate(Timestamp timestamp, int elementNum) {
-    scratchPisaTimestamp.update(epochDay[elementNum], nanoOfDay[elementNum]);
-    scratchPisaTimestamp.timestampUpdate(timestamp);
+    timestamp.setTime(time[elementNum]);
+    timestamp.setNanos(nanos[elementNum]);
   }
 
   /**
-   * Compare row to PisaTimestamp.
+   * Return the scratch Timestamp object set from a row.
    * We assume the entry has already been NULL checked and isRepeated adjusted.
    * @param elementNum
-   * @param pisaTimestamp
-   * @return -1, 0, 1 standard compareTo values.
-   */
-  public int compareTo(int elementNum, PisaTimestamp pisaTimestamp) {
-    return PisaTimestamp.compareTo(epochDay[elementNum], nanoOfDay[elementNum], pisaTimestamp);
-  }
-
-  /**
-   * Compare PisaTimestamp to row.
-   * We assume the entry has already been NULL checked and isRepeated adjusted.
-   * @param pisaTimestamp
-   * @param elementNum
-   * @return -1, 0, 1 standard compareTo values.
+   * @return
    */
-  public int compareTo(PisaTimestamp pisaTimestamp, int elementNum) {
-    return PisaTimestamp.compareTo(pisaTimestamp, epochDay[elementNum], nanoOfDay[elementNum]);
+  public Timestamp asScratchTimestamp(int elementNum) {
+    scratchTimestamp.setTime(time[elementNum]);
+    scratchTimestamp.setNanos(nanos[elementNum]);
+    return scratchTimestamp;
   }
 
   /**
-   * Compare a row to another TimestampColumnVector's row.
-   * @param elementNum1
-   * @param timestampColVector2
-   * @param elementNum2
+   * Return the scratch timestamp (contents undefined).
    * @return
    */
-  public int compareTo(int elementNum1, TimestampColumnVector timestampColVector2,
-      int elementNum2) {
-    return PisaTimestamp.compareTo(
-        epochDay[elementNum1], nanoOfDay[elementNum1],
-        timestampColVector2.epochDay[elementNum2], timestampColVector2.nanoOfDay[elementNum2]);
+  public Timestamp getScratchTimestamp() {
+    return scratchTimestamp;
   }
 
   /**
-   * Compare another TimestampColumnVector's row to a row.
-   * @param timestampColVector1
-   * @param elementNum1
-   * @param elementNum2
+   * Return a long representation of a Timestamp.
+   * @param elementNum
    * @return
    */
-  public int compareTo(TimestampColumnVector timestampColVector1, int elementNum1,
-      int elementNum2) {
-    return PisaTimestamp.compareTo(
-        timestampColVector1.epochDay[elementNum1], timestampColVector1.nanoOfDay[elementNum1],
-        epochDay[elementNum2], nanoOfDay[elementNum2]);
-  }
-
-  public void add(PisaTimestamp timestamp1, PisaTimestamp timestamp2, int resultElementNum) {
-    PisaTimestamp.add(timestamp1, timestamp2, scratchPisaTimestamp);
-    epochDay[resultElementNum] = scratchPisaTimestamp.getEpochDay();
-    nanoOfDay[resultElementNum] = scratchPisaTimestamp.getNanoOfDay();
-  }
-
-  public void subtract(PisaTimestamp timestamp1, PisaTimestamp timestamp2, int resultElementNum) {
-    PisaTimestamp.subtract(timestamp1, timestamp2, scratchPisaTimestamp);
-    epochDay[resultElementNum] = scratchPisaTimestamp.getEpochDay();
-    nanoOfDay[resultElementNum] = scratchPisaTimestamp.getNanoOfDay();
+  public long getTimestampAsLong(int elementNum) {
+    scratchTimestamp.setTime(time[elementNum]);
+    scratchTimestamp.setNanos(nanos[elementNum]);
+    return getTimestampAsLong(scratchTimestamp);
   }
 
   /**
-   * Return row as a double with the integer part as the seconds and the fractional part as
-   * the nanoseconds the way the Timestamp class does it.
-   * We assume the entry has already been NULL checked and isRepeated adjusted.
-   * @param elementNum
-   * @return seconds.nanoseconds
+   * Return a long representation of a Timestamp.
+   * @param timestamp
+   * @return
    */
-  public double getTimestampSecondsWithFractionalNanos(int elementNum) {
-    scratchPisaTimestamp.update(epochDay[elementNum], nanoOfDay[elementNum]);
-    return scratchPisaTimestamp.getTimestampSecondsWithFractionalNanos();
+  public static long getTimestampAsLong(Timestamp timestamp) {
+    return millisToSeconds(timestamp.getTime());
   }
 
+  // Copy of TimestampWritable.millisToSeconds
   /**
-   * Return row as integer as the seconds the way the Timestamp class does it.
-   * We assume the entry has already been NULL checked and isRepeated adjusted.
-   * @param elementNum
-   * @return seconds
+   * Rounds the number of milliseconds relative to the epoch down to the nearest whole number of
+   * seconds. 500 would round to 0, -500 would round to -1.
    */
-  public long getTimestampSeconds(int elementNum) {
-    scratchPisaTimestamp.update(epochDay[elementNum], nanoOfDay[elementNum]);
-    return scratchPisaTimestamp.getTimestampSeconds();
+  private static long millisToSeconds(long millis) {
+    if (millis >= 0) {
+      return millis / 1000;
+    } else {
+      return (millis - 999) / 1000;
+    }
   }
 
-
   /**
-   * Return row as milliseconds the way the Timestamp class does it.
-   * We assume the entry has already been NULL checked and isRepeated adjusted.
+   * Return a double representation of a Timestamp.
    * @param elementNum
    * @return
    */
-  public long getTimestampMilliseconds(int elementNum) {
-    scratchPisaTimestamp.update(epochDay[elementNum], nanoOfDay[elementNum]);
-    return scratchPisaTimestamp.getTimestampMilliseconds();
+  public double getDouble(int elementNum) {
+    scratchTimestamp.setTime(time[elementNum]);
+    scratchTimestamp.setNanos(nanos[elementNum]);
+    return getDouble(scratchTimestamp);
   }
 
   /**
-   * Return row as epoch seconds.
-   * We assume the entry has already been NULL checked and isRepeated adjusted.
+   * Return a double representation of a Timestamp.
    * @param elementNum
    * @return
    */
-  public long getEpochSeconds(int elementNum) {
-    return PisaTimestamp.getEpochSecondsFromEpochDayAndNanoOfDay(epochDay[elementNum], nanoOfDay[elementNum]);
+  public static double getDouble(Timestamp timestamp) {
+    // Same algorithm as TimestampWritable (not currently import-able here).
+    double seconds, nanos;
+    seconds = millisToSeconds(timestamp.getTime());
+    nanos = timestamp.getNanos();
+    return seconds + nanos / 1000000000;
   }
 
   /**
-   * Return row as epoch milliseconds.
+   * Compare row to Timestamp.
    * We assume the entry has already been NULL checked and isRepeated adjusted.
    * @param elementNum
-   * @return
+   * @param timestamp
+   * @return -1, 0, 1 standard compareTo values.
    */
-  public long getEpochMilliseconds(int elementNum) {
-    return PisaTimestamp.getEpochMillisecondsFromEpochDayAndNanoOfDay(epochDay[elementNum], nanoOfDay[elementNum]);
+  public int compareTo(int elementNum, Timestamp timestamp) {
+    return asScratchTimestamp(elementNum).compareTo(timestamp);
   }
 
   /**
-   * Return row as signed nanos (-999999999 to 999999999).
-   * NOTE: This is not the same as the Timestamp class nanos (which is always positive).
+   * Compare Timestamp to row.
    * We assume the entry has already been NULL checked and isRepeated adjusted.
+   * @param timestamp
    * @param elementNum
-   * @return
+   * @return -1, 0, 1 standard compareTo values.
    */
-  public int getSignedNanos(int elementNum) {
-    return PisaTimestamp.getSignedNanos(nanoOfDay[elementNum]);
+  public int compareTo(Timestamp timestamp, int elementNum) {
+    return timestamp.compareTo(asScratchTimestamp(elementNum));
   }
 
   /**
-   * Get scratch timestamp with value of a row.
-   * @param elementNum
+   * Compare a row to another TimestampColumnVector's row.
+   * @param elementNum1
+   * @param timestampColVector2
+   * @param elementNum2
    * @return
    */
-  public Timestamp asScratchTimestamp(int elementNum) {
-    scratchPisaTimestamp.update(epochDay[elementNum], nanoOfDay[elementNum]);
-    return scratchPisaTimestamp.asScratchTimestamp();
+  public int compareTo(int elementNum1, TimestampColumnVector timestampColVector2,
+      int elementNum2) {
+    return asScratchTimestamp(elementNum1).compareTo(
+        timestampColVector2.asScratchTimestamp(elementNum2));
   }
 
   /**
-   * Get scratch Pisa timestamp for use by the caller.
+   * Compare another TimestampColumnVector's row to a row.
+   * @param timestampColVector1
+   * @param elementNum1
+   * @param elementNum2
    * @return
    */
-  public PisaTimestamp useScratchPisaTimestamp() {
-    return scratchPisaTimestamp;
+  public int compareTo(TimestampColumnVector timestampColVector1, int elementNum1,
+      int elementNum2) {
+    return timestampColVector1.asScratchTimestamp(elementNum1).compareTo(
+        asScratchTimestamp(elementNum2));
   }
 
   @Override
@@ -290,8 +246,8 @@ public class TimestampColumnVector extends ColumnVector {
 
     TimestampColumnVector timestampColVector = (TimestampColumnVector) inputVector;
 
-    epochDay[outElementNum] = timestampColVector.epochDay[inputElementNum];
-    nanoOfDay[outElementNum] = timestampColVector.nanoOfDay[inputElementNum];
+    time[outElementNum] = timestampColVector.time[inputElementNum];
+    nanos[outElementNum] = timestampColVector.nanos[inputElementNum];
   }
 
   // Simplify vector by brute-force flattening noNulls and isRepeating
@@ -301,17 +257,17 @@ public class TimestampColumnVector extends ColumnVector {
     flattenPush();
     if (isRepeating) {
       isRepeating = false;
-      long repeatEpochDay = epochDay[0];
-      long repeatNanoOfDay = nanoOfDay[0];
+      long repeatFastTime = time[0];
+      int repeatNanos = nanos[0];
       if (selectedInUse) {
         for (int j = 0; j < size; j++) {
           int i = sel[j];
-          epochDay[i] = repeatEpochDay;
-          nanoOfDay[i] = repeatNanoOfDay;
+          time[i] = repeatFastTime;
+          nanos[i] = repeatNanos;
         }
       } else {
-        Arrays.fill(epochDay, 0, size, repeatEpochDay);
-        Arrays.fill(nanoOfDay, 0, size, repeatNanoOfDay);
+        Arrays.fill(time, 0, size, repeatFastTime);
+        Arrays.fill(nanos, 0, size, repeatNanos);
       }
       flattenRepeatingNulls(selectedInUse, sel, size);
     }
@@ -319,71 +275,23 @@ public class TimestampColumnVector extends ColumnVector {
   }
 
   /**
-   * Set a row from a PisaTimestamp.
-   * We assume the entry has already been isRepeated adjusted.
-   * @param elementNum
-   * @param pisaTimestamp
-   */
-  public void set(int elementNum, PisaTimestamp pisaTimestamp) {
-    this.epochDay[elementNum] = pisaTimestamp.getEpochDay();
-    this.nanoOfDay[elementNum] = pisaTimestamp.getNanoOfDay();
-  }
-
-  /**
    * Set a row from a timestamp.
    * We assume the entry has already been isRepeated adjusted.
    * @param elementNum
    * @param timestamp
    */
   public void set(int elementNum, Timestamp timestamp) {
-    scratchPisaTimestamp.updateFromTimestamp(timestamp);
-    this.epochDay[elementNum] = scratchPisaTimestamp.getEpochDay();
-    this.nanoOfDay[elementNum] = scratchPisaTimestamp.getNanoOfDay();
+    this.time[elementNum] = timestamp.getTime();
+    this.nanos[elementNum] = timestamp.getNanos();
   }
 
   /**
-   * Set a row from a epoch seconds and signed nanos (-999999999 to 999999999).
+   * Set a row from the current value in the scratch timestamp.
    * @param elementNum
-   * @param epochSeconds
-   * @param signedNanos
    */
-  public void setEpochSecondsAndSignedNanos(int elementNum, long epochSeconds, int signedNanos) {
-    scratchPisaTimestamp.updateFromEpochSecondsAndSignedNanos(epochSeconds, signedNanos);
-    set(elementNum, scratchPisaTimestamp);
-  }
-
-  /**
-   * Set a row from timestamp milliseconds.
-   * We assume the entry has already been isRepeated adjusted.
-   * @param elementNum
-   * @param timestampMilliseconds
-   */
-  public void setTimestampMilliseconds(int elementNum, long timestampMilliseconds) {
-    scratchPisaTimestamp.updateFromTimestampMilliseconds(timestampMilliseconds);
-    set(elementNum, scratchPisaTimestamp.useScratchTimestamp());
-  }
-
-  /**
-   * Set a row from timestamp seconds.
-   * We assume the entry has already been isRepeated adjusted.
-   * @param elementNum
-   * @param timestamp
-   */
-  public void setTimestampSeconds(int elementNum, long timestampSeconds) {
-    scratchPisaTimestamp.updateFromTimestampSeconds(timestampSeconds);
-    set(elementNum, scratchPisaTimestamp);
-  }
-
-  /**
-   * Set a row from a double timestamp seconds with fractional nanoseconds.
-   * We assume the entry has already been isRepeated adjusted.
-   * @param elementNum
-   * @param timestamp
-   */
-  public void setTimestampSecondsWithFractionalNanoseconds(int elementNum,
-      double secondsWithFractionalNanoseconds) {
-    scratchPisaTimestamp.updateFromTimestampSecondsWithFractionalNanoseconds(secondsWithFractionalNanoseconds);
-    set(elementNum, scratchPisaTimestamp);
+  public void setFromScratchTimestamp(int elementNum) {
+    this.time[elementNum] = scratchTimestamp.getTime();
+    this.nanos[elementNum] = scratchTimestamp.getNanos();
   }
 
   /**
@@ -392,8 +300,8 @@ public class TimestampColumnVector extends ColumnVector {
    * @param elementNum
    */
   public void setNullValue(int elementNum) {
-    epochDay[elementNum] = 0;
-    nanoOfDay[elementNum] = 1;
+    time[elementNum] = 0;
+    nanos[elementNum] = 1;
   }
 
   // Copy the current object contents into the output. Only copy selected entries,
@@ -407,8 +315,8 @@ public class TimestampColumnVector extends ColumnVector {
 
     // Handle repeating case
     if (isRepeating) {
-      output.epochDay[0] = epochDay[0];
-      output.nanoOfDay[0] = nanoOfDay[0];
+      output.time[0] = time[0];
+      output.nanos[0] = nanos[0];
       output.isNull[0] = isNull[0];
       output.isRepeating = true;
       return;
@@ -420,13 +328,13 @@ public class TimestampColumnVector extends ColumnVector {
     if (selectedInUse) {
       for (int j = 0; j < size; j++) {
         int i = sel[j];
-        output.epochDay[i] = epochDay[i];
-        output.nanoOfDay[i] = nanoOfDay[i];
+        output.time[i] = time[i];
+        output.nanos[i] = nanos[i];
       }
     }
     else {
-      System.arraycopy(epochDay, 0, output.epochDay, 0, size);
-      System.arraycopy(nanoOfDay, 0, output.nanoOfDay, 0, size);
+      System.arraycopy(time, 0, output.time, 0, size);
+      System.arraycopy(nanos, 0, output.nanos, 0, size);
     }
 
     // Copy nulls over if needed
@@ -444,26 +352,14 @@ public class TimestampColumnVector extends ColumnVector {
   }
 
   /**
-   * Fill all the vector entries with a PisaTimestamp.
-   * @param pisaTimestamp
-   */
-  public void fill(PisaTimestamp pisaTimestamp) {
-    noNulls = true;
-    isRepeating = true;
-    epochDay[0] = pisaTimestamp.getEpochDay();
-    nanoOfDay[0] = pisaTimestamp.getNanoOfDay();
-  }
-
-  /**
    * Fill all the vector entries with a timestamp.
    * @param timestamp
    */
   public void fill(Timestamp timestamp) {
     noNulls = true;
     isRepeating = true;
-    scratchPisaTimestamp.updateFromTimestamp(timestamp);
-    epochDay[0] = scratchPisaTimestamp.getEpochDay();
-    nanoOfDay[0] = scratchPisaTimestamp.getNanoOfDay();
+    time[0] = timestamp.getTime();
+    nanos[0] = timestamp.getNanos();
   }
 
   /**
@@ -489,8 +385,9 @@ public class TimestampColumnVector extends ColumnVector {
       row = 0;
     }
     if (noNulls || !isNull[row]) {
-      scratchPisaTimestamp.update(epochDay[row], nanoOfDay[row]);
-      buffer.append(scratchPisaTimestamp.toString());
+      scratchTimestamp.setTime(time[row]);
+      scratchTimestamp.setNanos(nanos[row]);
+      buffer.append(scratchTimestamp.toString());
     } else {
       buffer.append("null");
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/storage-api/src/java/org/apache/hive/common/util/IntervalDayTimeUtils.java
----------------------------------------------------------------------
diff --git a/storage-api/src/java/org/apache/hive/common/util/IntervalDayTimeUtils.java b/storage-api/src/java/org/apache/hive/common/util/IntervalDayTimeUtils.java
new file mode 100644
index 0000000..727c1e6
--- /dev/null
+++ b/storage-api/src/java/org/apache/hive/common/util/IntervalDayTimeUtils.java
@@ -0,0 +1,77 @@
+/**
+ * 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.hive.common.util;
+
+import java.math.BigDecimal;
+import java.text.SimpleDateFormat;
+
+import org.apache.hadoop.hive.common.type.HiveIntervalDayTime;
+
+
+/**
+ * DateUtils. Thread-safe class
+ *
+ */
+public class IntervalDayTimeUtils {
+
+  private static final ThreadLocal<SimpleDateFormat> dateFormatLocal = new ThreadLocal<SimpleDateFormat>() {
+    @Override
+    protected SimpleDateFormat initialValue() {
+      return new SimpleDateFormat("yyyy-MM-dd");
+    }
+  };
+
+  public static SimpleDateFormat getDateFormat() {
+    return dateFormatLocal.get();
+  }
+
+  public static final int NANOS_PER_SEC = 1000000000;
+  public static final BigDecimal MAX_INT_BD = new BigDecimal(Integer.MAX_VALUE);
+  public static final BigDecimal NANOS_PER_SEC_BD = new BigDecimal(NANOS_PER_SEC);
+
+  public static int parseNumericValueWithRange(String fieldName,
+      String strVal, int minValue, int maxValue) throws IllegalArgumentException {
+    int result = 0;
+    if (strVal != null) {
+      result = Integer.parseInt(strVal);
+      if (result < minValue || result > maxValue) {
+        throw new IllegalArgumentException(String.format("%s value %d outside range [%d, %d]",
+            fieldName, result, minValue, maxValue));
+      }
+    }
+    return result;
+  }
+
+  public static long getIntervalDayTimeTotalNanos(HiveIntervalDayTime intervalDayTime) {
+    return intervalDayTime.getTotalSeconds() * NANOS_PER_SEC + intervalDayTime.getNanos();
+  }
+
+  public static void setIntervalDayTimeTotalNanos(HiveIntervalDayTime intervalDayTime,
+      long totalNanos) {
+    intervalDayTime.set(totalNanos / NANOS_PER_SEC, (int) (totalNanos % NANOS_PER_SEC));
+  }
+
+  public static long getIntervalDayTimeTotalSecondsFromTotalNanos(long totalNanos) {
+    return totalNanos / NANOS_PER_SEC;
+  }
+
+  public static int getIntervalDayTimeNanosFromTotalNanos(long totalNanos) {
+    return (int) (totalNanos % NANOS_PER_SEC);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/storage-api/src/test/org/apache/hadoop/hive/common/type/TestPisaTimestamp.java
----------------------------------------------------------------------
diff --git a/storage-api/src/test/org/apache/hadoop/hive/common/type/TestPisaTimestamp.java b/storage-api/src/test/org/apache/hadoop/hive/common/type/TestPisaTimestamp.java
deleted file mode 100644
index 8e7395c..0000000
--- a/storage-api/src/test/org/apache/hadoop/hive/common/type/TestPisaTimestamp.java
+++ /dev/null
@@ -1,118 +0,0 @@
-/**
- * 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.common.type;
-
-import org.junit.Test;
-
-import java.math.BigDecimal;
-import java.math.RoundingMode;
-import java.sql.Timestamp;
-import java.util.Random;
-import org.apache.hadoop.hive.common.type.RandomTypeUtil;
-
-import static org.junit.Assert.*;
-
-/**
- * Test for ListColumnVector
- */
-public class TestPisaTimestamp {
-
-  private static int TEST_COUNT = 5000;
-
-  @Test
-  public void testPisaTimestampCreate() throws Exception {
-
-    Random r = new Random(1234);
-
-    for (int i = 0; i < TEST_COUNT; i++) {
-      Timestamp randTimestamp = RandomTypeUtil.getRandTimestamp(r);
-      PisaTimestamp pisaTimestamp = new PisaTimestamp(randTimestamp);
-      Timestamp reconstructedTimestamp = new Timestamp(0);
-      pisaTimestamp.timestampUpdate(reconstructedTimestamp);
-      if (!randTimestamp.equals(reconstructedTimestamp)) {
-        assertTrue(false);
-      }
-    }
-  }
-
-  static BigDecimal BIG_MAX_LONG = new BigDecimal(Long.MAX_VALUE);
-  static BigDecimal BIG_MIN_LONG = new BigDecimal(Long.MIN_VALUE);
-  static BigDecimal BIG_NANOSECONDS_PER_DAY = new BigDecimal(PisaTimestamp.NANOSECONDS_PER_DAY);
-
-  static boolean beyondLongRange = false;
-
-  private BigDecimal[] randomEpochDayAndNanoOfDay(Random r) {
-    double randDouble = (r.nextDouble() - 0.5D) * 2.0D;
-    randDouble *= PisaTimestamp.NANOSECONDS_PER_DAY;
-    randDouble *= 365 * 10000;
-    BigDecimal bigDecimal = new BigDecimal(randDouble);
-    bigDecimal = bigDecimal.setScale(0, RoundingMode.HALF_UP);
-
-    if (bigDecimal.compareTo(BIG_MAX_LONG) > 0 || bigDecimal.compareTo(BIG_MIN_LONG) < 0) {
-      beyondLongRange = true;
-    }
-
-    BigDecimal[] divideAndRemainder = bigDecimal.divideAndRemainder(BIG_NANOSECONDS_PER_DAY);
-
-    return new BigDecimal[] {divideAndRemainder[0], divideAndRemainder[1], bigDecimal};
-  }
-
-  private BigDecimal pisaTimestampToBig(PisaTimestamp pisaTimestamp) {
-    BigDecimal bigNanoOfDay = new BigDecimal(pisaTimestamp.getNanoOfDay());
-
-    BigDecimal bigEpochDay = new BigDecimal(pisaTimestamp.getEpochDay());
-    BigDecimal result = bigEpochDay.multiply(BIG_NANOSECONDS_PER_DAY);
-    result = result.add(bigNanoOfDay);
-    return result;
-  }
-
-  @Test
-  public void testPisaTimestampArithmetic() throws Exception {
-
-    Random r = new Random(1234);
-
-
-    for (int i = 0; i < TEST_COUNT; i++) {
-      BigDecimal[] random1 = randomEpochDayAndNanoOfDay(r);
-      long epochDay1 = random1[0].longValue();
-      long nanoOfDay1 = random1[1].longValue();
-      PisaTimestamp pisa1 = new PisaTimestamp(epochDay1, nanoOfDay1);
-      BigDecimal big1 = random1[2];
-
-      BigDecimal[] random2 = randomEpochDayAndNanoOfDay(r);
-      long epochDay2 = random2[0].longValue();
-      long nanoOfDay2 = random2[1].longValue();
-      PisaTimestamp pisa2 = new PisaTimestamp(epochDay2, nanoOfDay2);
-      BigDecimal big2 = random2[2];
-
-      BigDecimal expectedBig;
-      PisaTimestamp pisaResult = new PisaTimestamp();
-      if (i % 2 == 0) {
-        expectedBig = big1.add(big2);
-        PisaTimestamp.add(pisa1, pisa2, pisaResult);
-      } else {
-        expectedBig = big1.add(big2.negate());
-        PisaTimestamp.subtract(pisa1, pisa2, pisaResult);
-      }
-      BigDecimal resultBig = pisaTimestampToBig(pisaResult);
-      assertEquals(expectedBig, resultBig);
-
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/storage-api/src/test/org/apache/hadoop/hive/ql/exec/vector/TestTimestampColumnVector.java
----------------------------------------------------------------------
diff --git a/storage-api/src/test/org/apache/hadoop/hive/ql/exec/vector/TestTimestampColumnVector.java b/storage-api/src/test/org/apache/hadoop/hive/ql/exec/vector/TestTimestampColumnVector.java
new file mode 100644
index 0000000..6e5d5c8
--- /dev/null
+++ b/storage-api/src/test/org/apache/hadoop/hive/ql/exec/vector/TestTimestampColumnVector.java
@@ -0,0 +1,117 @@
+/**
+ * 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.ql.exec.vector;
+
+import org.junit.Test;
+
+import java.io.PrintWriter;
+import java.math.BigDecimal;
+import java.math.RoundingMode;
+import java.sql.Timestamp;
+import java.util.Date;
+import java.util.Random;
+
+import org.apache.hadoop.hive.common.type.RandomTypeUtil;
+
+import static org.junit.Assert.*;
+
+/**
+ * Test for ListColumnVector
+ */
+public class TestTimestampColumnVector {
+
+  private static int TEST_COUNT = 5000;
+
+  private static int fake = 0;
+
+  @Test
+  public void testSaveAndRetrieve() throws Exception {
+
+    Random r = new Random(1234);
+    TimestampColumnVector timestampColVector = new TimestampColumnVector();
+    Timestamp[] randTimestamps = new Timestamp[VectorizedRowBatch.DEFAULT_SIZE];
+
+    for (int i = 0; i < VectorizedRowBatch.DEFAULT_SIZE; i++) {
+      Timestamp randTimestamp = RandomTypeUtil.getRandTimestamp(r);
+      randTimestamps[i] = randTimestamp;
+      timestampColVector.set(i, randTimestamp);
+    }
+    for (int i = 0; i < VectorizedRowBatch.DEFAULT_SIZE; i++) {
+      Timestamp retrievedTimestamp = timestampColVector.asScratchTimestamp(i);
+      Timestamp randTimestamp = randTimestamps[i];
+      if (!retrievedTimestamp.equals(randTimestamp)) {
+        assertTrue(false);
+      }
+    }
+  }
+
+  @Test
+  public void testTimestampCompare() throws Exception {
+    Random r = new Random(1234);
+    TimestampColumnVector timestampColVector = new TimestampColumnVector();
+    Timestamp[] randTimestamps = new Timestamp[VectorizedRowBatch.DEFAULT_SIZE];
+    Timestamp[] candTimestamps = new Timestamp[VectorizedRowBatch.DEFAULT_SIZE];
+    int[] compareToLeftRights = new int[VectorizedRowBatch.DEFAULT_SIZE];
+    int[] compareToRightLefts = new int[VectorizedRowBatch.DEFAULT_SIZE];
+
+    for (int i = 0; i < VectorizedRowBatch.DEFAULT_SIZE; i++) {
+      Timestamp randTimestamp = RandomTypeUtil.getRandTimestamp(r);
+      randTimestamps[i] = randTimestamp;
+      timestampColVector.set(i, randTimestamp);
+      Timestamp candTimestamp = RandomTypeUtil.getRandTimestamp(r);
+      candTimestamps[i] = candTimestamp;
+      compareToLeftRights[i] = candTimestamp.compareTo(randTimestamp);
+      compareToRightLefts[i] = randTimestamp.compareTo(candTimestamp);
+    }
+
+    for (int i = 0; i < VectorizedRowBatch.DEFAULT_SIZE; i++) {
+      Timestamp retrievedTimestamp = timestampColVector.asScratchTimestamp(i);
+      Timestamp randTimestamp = randTimestamps[i];
+      if (!retrievedTimestamp.equals(randTimestamp)) {
+        assertTrue(false);
+      }
+      Timestamp candTimestamp = candTimestamps[i];
+      int compareToLeftRight = timestampColVector.compareTo(candTimestamp, i);
+      if (compareToLeftRight != compareToLeftRights[i]) {
+        assertTrue(false);
+      }
+      int compareToRightLeft = timestampColVector.compareTo(i, candTimestamp);
+      if (compareToRightLeft != compareToRightLefts[i]) {
+        assertTrue(false);
+      }
+    }
+  }
+
+  /*
+  @Test
+  public void testGenerate() throws Exception {
+    PrintWriter writer = new PrintWriter("/Users/you/timestamps.txt");
+    Random r = new Random(18485);
+    for (int i = 0; i < 25; i++) {
+      Timestamp randTimestamp = RandomTypeUtil.getRandTimestamp(r);
+      writer.println(randTimestamp.toString());
+    }
+    for (int i = 0; i < 25; i++) {
+      Timestamp randTimestamp = RandomTypeUtil.getRandTimestamp(r, 1965, 2025);
+      writer.println(randTimestamp.toString());
+    }
+    writer.close();
+  }
+  */
+}