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

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

http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/ql/src/gen/vectorization/ExpressionTemplates/TimestampScalarArithmeticTimestampColumnBase.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/TimestampScalarArithmeticTimestampColumnBase.txt b/ql/src/gen/vectorization/ExpressionTemplates/TimestampScalarArithmeticTimestampColumnBase.txt
deleted file mode 100644
index 0ff9226..0000000
--- a/ql/src/gen/vectorization/ExpressionTemplates/TimestampScalarArithmeticTimestampColumnBase.txt
+++ /dev/null
@@ -1,139 +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.ql.exec.vector.expressions.gen;
-
-import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
-import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
-import org.apache.hadoop.hive.ql.exec.vector.*;
-
-import org.apache.hadoop.hive.common.type.PisaTimestamp;
-/*
- * Because of the templatized nature of the code, either or both
- * of these ColumnVector imports may be needed. Listing both of them
- * rather than using ....vectorization.*;
- */
-import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.NullUtil;
-import org.apache.hadoop.hive.ql.util.DateTimeMath;
-
-/**
- * Generated from template TimestampScalarArithmeticTimestampColumnBase.txt.
- * Implements a vectorized arithmetic operator with a scalar on the left and a
- * column vector on the right. The result is output to an output column vector.
- */
-public abstract class <BaseClassName> extends VectorExpression {
-
-  private static final long serialVersionUID = 1L;
-
-  private int colNum;
-  private PisaTimestamp value;
-  private int outputColumn;
-  private PisaTimestamp scratchPisaTimestamp;
-  private DateTimeMath dtm = new DateTimeMath();
-
-  public <BaseClassName>(PisaTimestamp value, int colNum, int outputColumn) {
-    this.colNum = colNum;
-    this.value = value;
-    this.outputColumn = outputColumn;
-    scratchPisaTimestamp = new PisaTimestamp();
-  }
-
-  public <BaseClassName>() {
-  }
-
-  @Override
-  /**
-   * Method to evaluate scalar-column operation in vectorized fashion.
-   *
-   * @batch a package of rows with each column stored in a vector
-   */
-  public void evaluate(VectorizedRowBatch batch) {
-
-    if (childExpressions != null) {
-      super.evaluateChildren(batch);
-    }
-
-    // Input #2 is type timestamp/interval_day_time.
-    TimestampColumnVector inputColVector2 = (TimestampColumnVector) batch.cols[colNum];
-
-    // Output is type timestamp/interval_day_time.
-    TimestampColumnVector outputColVector = (TimestampColumnVector) batch.cols[outputColumn];
-
-    int[] sel = batch.selected;
-    boolean[] inputIsNull = inputColVector2.isNull;
-    boolean[] outputIsNull = outputColVector.isNull;
-    outputColVector.noNulls = inputColVector2.noNulls;
-    outputColVector.isRepeating = inputColVector2.isRepeating;
-    int n = batch.size;
-
-    // return immediately if batch is empty
-    if (n == 0) {
-      return;
-    }
-
-    if (inputColVector2.isRepeating) {
-       outputColVector.<OperatorMethod>(
-         value, inputColVector2.asScratchPisaTimestamp(0), 0);
-
-      // Even if there are no nulls, we always copy over entry 0. Simplifies code.
-      outputIsNull[0] = inputIsNull[0];
-    } else if (inputColVector2.noNulls) {
-      if (batch.selectedInUse) {
-        for(int j = 0; j != n; j++) {
-          int i = sel[j];
-          outputColVector.<OperatorMethod>(
-            value, inputColVector2.asScratchPisaTimestamp(i), i);
-        }
-      } else {
-        for(int i = 0; i != n; i++) {
-          outputColVector.<OperatorMethod>(
-            value, inputColVector2.asScratchPisaTimestamp(i), i);
-        }
-      }
-    } else {                         /* there are nulls */
-      if (batch.selectedInUse) {
-        for(int j = 0; j != n; j++) {
-          int i = sel[j];
-          outputColVector.<OperatorMethod>(
-            value, inputColVector2.asScratchPisaTimestamp(i), i);
-          outputIsNull[i] = inputIsNull[i];
-        }
-      } else {
-        for(int i = 0; i != n; i++) {
-          outputColVector.<OperatorMethod>(
-            value, inputColVector2.asScratchPisaTimestamp(i), i);
-        }
-        System.arraycopy(inputIsNull, 0, outputIsNull, 0, n);
-      }
-    }
-
-    NullUtil.setNullOutputEntriesColScalar(outputColVector, batch.selectedInUse, sel, n);
-  }
-
-  @Override
-  public int getOutputColumn() {
-    return outputColumn;
-  }
-
-  @Override
-  public String getOutputType() {
-    return "timestamp";
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/ql/src/gen/vectorization/ExpressionTemplates/TimestampScalarCompareLongDoubleColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/TimestampScalarCompareLongDoubleColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/TimestampScalarCompareLongDoubleColumn.txt
index 9e855e8..6815b5b 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/TimestampScalarCompareLongDoubleColumn.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/TimestampScalarCompareLongDoubleColumn.txt
@@ -19,8 +19,8 @@
 package org.apache.hadoop.hive.ql.exec.vector.expressions.gen;
 
 import java.sql.Timestamp;
-import org.apache.hadoop.hive.common.type.PisaTimestamp;
 
+import org.apache.hadoop.hive.ql.exec.vector.*;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.*;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
@@ -35,7 +35,7 @@ public class <ClassName> extends <BaseClassName> {
   private static final long serialVersionUID = 1L;
 
   public <ClassName>(Timestamp value, int colNum, int outputColumn) {
-    super(new PisaTimestamp(value).<GetTimestampLongDoubleMethod>(), colNum, outputColumn);
+    super(TimestampColumnVector.<GetTimestampLongDoubleMethod>(value), colNum, outputColumn);
   }
 
   public <ClassName>() {

http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/ql/src/gen/vectorization/ExpressionTemplates/TimestampScalarCompareTimestampColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/TimestampScalarCompareTimestampColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/TimestampScalarCompareTimestampColumn.txt
index df9f3c9..6506c93 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/TimestampScalarCompareTimestampColumn.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/TimestampScalarCompareTimestampColumn.txt
@@ -21,26 +21,117 @@ package org.apache.hadoop.hive.ql.exec.vector.expressions.gen;
 import java.sql.Timestamp;
 
 import org.apache.hadoop.hive.common.type.HiveIntervalDayTime;
-import org.apache.hadoop.hive.common.type.PisaTimestamp;
-
+import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.*;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
 
-
 /**
- * Generated from template TimestampColumnCompareTimestampScalar.txt, which covers comparison
- * expressions between a datetime/interval column and a scalar of the same type. The boolean output
- * is stored in a separate boolean column.
+ * Generated from template ScalarCompareTimestamp.txt, which covers comparison
+ * expressions between a long/double scalar and a column. The boolean output is stored in a
+ * separate boolean column.
  */
-public class <ClassName> extends <BaseClassName> {
+public class <ClassName> extends VectorExpression {
 
   private static final long serialVersionUID = 1L;
 
-  public <ClassName>(Timestamp value, int colNum, int outputColumn) {
-    super(new PisaTimestamp(value), colNum, outputColumn);
+  private int colNum;
+  private <HiveOperandType> value;
+  private int outputColumn;
+
+  public <ClassName>(<HiveOperandType> value, int colNum, int outputColumn) {
+    this.colNum = colNum;
+    this.value = value;
+    this.outputColumn = outputColumn;
   }
 
   public <ClassName>() {
-    super();
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+     // Input #2 is type <OperandType>.
+    <InputColumnVectorType> inputColVector2 = (<InputColumnVectorType>) batch.cols[colNum];
+
+    LongColumnVector outputColVector = (LongColumnVector) batch.cols[outputColumn];
+
+    int[] sel = batch.selected;
+    boolean[] nullPos = inputColVector2.isNull;
+    boolean[] outNulls = outputColVector.isNull;
+    int n = batch.size;
+    long[] outputVector = outputColVector.vector;
+
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+
+    outputColVector.isRepeating = false;
+    outputColVector.noNulls = inputColVector2.noNulls;
+    if (inputColVector2.noNulls) {
+      if (inputColVector2.isRepeating) {
+        //All must be selected otherwise size would be zero
+        //Repeating property will not change.
+        outputVector[0] = inputColVector2.compareTo(value, 0) <OperatorSymbol> 0 ? 1 : 0;
+        outputColVector.isRepeating = true;
+      } else if (batch.selectedInUse) {
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = inputColVector2.compareTo(value, i) <OperatorSymbol> 0 ? 1 : 0;
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = inputColVector2.compareTo(value, i) <OperatorSymbol> 0 ? 1 : 0;
+        }
+      }
+    } else {
+      if (inputColVector2.isRepeating) {
+        //All must be selected otherwise size would be zero
+        //Repeating property will not change.
+        if (!nullPos[0]) {
+          outputVector[0] = inputColVector2.compareTo(value, 0) <OperatorSymbol> 0 ? 1 : 0;
+          outNulls[0] = false;
+        } else {
+          outNulls[0] = true;
+        }
+        outputColVector.isRepeating = true;
+      } else if (batch.selectedInUse) {
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          if (!nullPos[i]) {
+            outputVector[i] = inputColVector2.compareTo(value, i) <OperatorSymbol> 0 ? 1 : 0;
+            outNulls[i] = false;
+          } else {
+            //comparison with null is null
+            outNulls[i] = true;
+          }
+        }
+      } else {
+        System.arraycopy(nullPos, 0, outNulls, 0, n);
+        for(int i = 0; i != n; i++) {
+          if (!nullPos[i]) {
+            outputVector[i] = inputColVector2.compareTo(value, i) <OperatorSymbol> 0 ? 1 : 0;
+          }
+        }
+      }
+    }
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return outputColumn;
+  }
+
+  @Override
+  public String getOutputType() {
+    return "long";
   }
 
   @Override
@@ -50,8 +141,8 @@ public class <ClassName> extends <BaseClassName> {
             VectorExpressionDescriptor.Mode.PROJECTION)
         .setNumArguments(2)
         .setArgumentTypes(
-            VectorExpressionDescriptor.ArgumentType.getType("timestamp"),
-            VectorExpressionDescriptor.ArgumentType.getType("timestamp"))
+            VectorExpressionDescriptor.ArgumentType.getType("<OperandType>"),
+            VectorExpressionDescriptor.ArgumentType.getType("<OperandType>"))
         .setInputExpressionTypes(
             VectorExpressionDescriptor.InputExpressionType.SCALAR,
             VectorExpressionDescriptor.InputExpressionType.COLUMN).build();

http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/ql/src/gen/vectorization/ExpressionTemplates/TimestampScalarCompareTimestampColumnBase.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/TimestampScalarCompareTimestampColumnBase.txt b/ql/src/gen/vectorization/ExpressionTemplates/TimestampScalarCompareTimestampColumnBase.txt
deleted file mode 100644
index bd345e7..0000000
--- a/ql/src/gen/vectorization/ExpressionTemplates/TimestampScalarCompareTimestampColumnBase.txt
+++ /dev/null
@@ -1,132 +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.ql.exec.vector.expressions.gen;
-
-import java.sql.Timestamp;
-import org.apache.hadoop.hive.common.type.PisaTimestamp;
-
-import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
-import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
-import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
-
-/**
- * Generated from template ScalarCompareTimestamp.txt, which covers comparison
- * expressions between a long/double scalar and a column. The boolean output is stored in a
- * separate boolean column.
- */
-public abstract class <ClassName> extends VectorExpression {
-
-  private static final long serialVersionUID = 1L;
-
-  private int colNum;
-  private PisaTimestamp value;
-  private int outputColumn;
-
-  public <ClassName>(PisaTimestamp value, int colNum, int outputColumn) {
-    this.colNum = colNum;
-    this.value = value;
-    this.outputColumn = outputColumn;
-  }
-
-  public <ClassName>() {
-  }
-
-  @Override
-  public void evaluate(VectorizedRowBatch batch) {
-
-    if (childExpressions != null) {
-      super.evaluateChildren(batch);
-    }
-
-    TimestampColumnVector inputColVector2 = (TimestampColumnVector) batch.cols[colNum];
-    LongColumnVector outputColVector = (LongColumnVector) batch.cols[outputColumn];
-    int[] sel = batch.selected;
-    boolean[] nullPos = inputColVector2.isNull;
-    boolean[] outNulls = outputColVector.isNull;
-    int n = batch.size;
-    long[] outputVector = outputColVector.vector;
-
-    // return immediately if batch is empty
-    if (n == 0) {
-      return;
-    }
-
-    outputColVector.isRepeating = false;
-    outputColVector.noNulls = inputColVector2.noNulls;
-    if (inputColVector2.noNulls) {
-      if (inputColVector2.isRepeating) {
-        //All must be selected otherwise size would be zero
-        //Repeating property will not change.
-        outputVector[0] = inputColVector2.compareTo(value, 0) <OperatorSymbol> 0 ? 1 : 0;
-        outputColVector.isRepeating = true;
-      } else if (batch.selectedInUse) {
-        for(int j=0; j != n; j++) {
-          int i = sel[j];
-          outputVector[i] = inputColVector2.compareTo(value, i) <OperatorSymbol> 0 ? 1 : 0;
-        }
-      } else {
-        for(int i = 0; i != n; i++) {
-          outputVector[i] = inputColVector2.compareTo(value, i) <OperatorSymbol> 0 ? 1 : 0;
-        }
-      }
-    } else {
-      if (inputColVector2.isRepeating) {
-        //All must be selected otherwise size would be zero
-        //Repeating property will not change.
-        if (!nullPos[0]) {
-          outputVector[0] = inputColVector2.compareTo(value, 0) <OperatorSymbol> 0 ? 1 : 0;
-          outNulls[0] = false;
-        } else {
-          outNulls[0] = true;
-        }
-        outputColVector.isRepeating = true;
-      } else if (batch.selectedInUse) {
-        for(int j=0; j != n; j++) {
-          int i = sel[j];
-          if (!nullPos[i]) {
-            outputVector[i] = inputColVector2.compareTo(value, i) <OperatorSymbol> 0 ? 1 : 0;
-            outNulls[i] = false;
-          } else {
-            //comparison with null is null
-            outNulls[i] = true;
-          }
-        }
-      } else {
-        System.arraycopy(nullPos, 0, outNulls, 0, n);
-        for(int i = 0; i != n; i++) {
-          if (!nullPos[i]) {
-            outputVector[i] = inputColVector2.compareTo(value, i) <OperatorSymbol> 0 ? 1 : 0;
-          }
-        }
-      }
-    }
-  }
-
-  @Override
-  public int getOutputColumn() {
-    return outputColumn;
-  }
-
-  @Override
-  public String getOutputType() {
-    return "long";
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFMinMaxIntervalDayTime.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFMinMaxIntervalDayTime.txt b/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFMinMaxIntervalDayTime.txt
new file mode 100644
index 0000000..3cdf7e2
--- /dev/null
+++ b/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFMinMaxIntervalDayTime.txt
@@ -0,0 +1,454 @@
+/**
+ * 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.expressions.aggregates.gen;
+
+import org.apache.hadoop.hive.ql.exec.Description;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.aggregates.VectorAggregateExpression;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpressionWriter;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpressionWriterFactory;
+import org.apache.hadoop.hive.ql.exec.vector.VectorAggregationBufferRow;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.exec.vector.IntervalDayTimeColumnVector;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.plan.AggregationDesc;
+import org.apache.hadoop.hive.ql.util.JavaDataModel;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.common.type.HiveIntervalDayTime;
+
+/**
+* <ClassName>. Vectorized implementation for MIN/MAX aggregates.
+*/
+@Description(name = "<DescriptionName>",
+    value = "<DescriptionValue>")
+public class <ClassName> extends VectorAggregateExpression {
+
+    private static final long serialVersionUID = 1L;
+
+    /**
+     * class for storing the current aggregate value.
+     */
+    static private final class Aggregation implements AggregationBuffer {
+
+      private static final long serialVersionUID = 1L;
+
+      transient private final HiveIntervalDayTime value;
+
+      /**
+      * Value is explicitly (re)initialized in reset()
+      */
+      transient private boolean isNull = true;
+
+      public Aggregation() {
+        value = new HiveIntervalDayTime();
+      }
+
+      public void checkValue(IntervalDayTimeColumnVector colVector, int index) {
+        if (isNull) {
+          isNull = false;
+          colVector.intervalDayTimeUpdate(this.value, index);
+        } else if (colVector.compareTo(this.value, index) <OperatorSymbol> 0) {
+          colVector.intervalDayTimeUpdate(this.value, index);
+        }
+      }
+
+      @Override
+      public int getVariableSize() {
+        throw new UnsupportedOperationException();
+      }
+
+      @Override
+      public void reset () {
+        isNull = true;
+        this.value.set(0, 0);
+      }
+    }
+
+    private VectorExpression inputExpression;
+    private transient VectorExpressionWriter resultWriter;
+
+    public <ClassName>(VectorExpression inputExpression) {
+      this();
+      this.inputExpression = inputExpression;
+    }
+
+    public <ClassName>() {
+      super();
+    }
+
+    @Override
+    public void init(AggregationDesc desc) throws HiveException {
+      resultWriter = VectorExpressionWriterFactory.genVectorExpressionWritable(
+          desc.getParameters().get(0));
+    }
+
+    private Aggregation getCurrentAggregationBuffer(
+        VectorAggregationBufferRow[] aggregationBufferSets,
+        int aggregrateIndex,
+        int row) {
+      VectorAggregationBufferRow mySet = aggregationBufferSets[row];
+      Aggregation myagg = (Aggregation) mySet.getAggregationBuffer(aggregrateIndex);
+      return myagg;
+    }
+
+    @Override
+    public void aggregateInputSelection(
+      VectorAggregationBufferRow[] aggregationBufferSets,
+      int aggregrateIndex,
+      VectorizedRowBatch batch) throws HiveException {
+
+      int batchSize = batch.size;
+
+      if (batchSize == 0) {
+        return;
+      }
+
+      inputExpression.evaluate(batch);
+
+      IntervalDayTimeColumnVector inputColVector = (IntervalDayTimeColumnVector)batch.
+        cols[this.inputExpression.getOutputColumn()];
+
+      if (inputColVector.noNulls) {
+        if (inputColVector.isRepeating) {
+          iterateNoNullsRepeatingWithAggregationSelection(
+            aggregationBufferSets, aggregrateIndex,
+            inputColVector, batchSize);
+        } else {
+          if (batch.selectedInUse) {
+            iterateNoNullsSelectionWithAggregationSelection(
+              aggregationBufferSets, aggregrateIndex,
+              inputColVector, batch.selected, batchSize);
+          } else {
+            iterateNoNullsWithAggregationSelection(
+              aggregationBufferSets, aggregrateIndex,
+              inputColVector, batchSize);
+          }
+        }
+      } else {
+        if (inputColVector.isRepeating) {
+          if (batch.selectedInUse) {
+            iterateHasNullsRepeatingSelectionWithAggregationSelection(
+              aggregationBufferSets, aggregrateIndex,
+              inputColVector, batchSize, batch.selected, inputColVector.isNull);
+          } else {
+            iterateHasNullsRepeatingWithAggregationSelection(
+              aggregationBufferSets, aggregrateIndex,
+              inputColVector, batchSize, inputColVector.isNull);
+          }
+        } else {
+          if (batch.selectedInUse) {
+            iterateHasNullsSelectionWithAggregationSelection(
+              aggregationBufferSets, aggregrateIndex,
+              inputColVector, batchSize, batch.selected, inputColVector.isNull);
+          } else {
+            iterateHasNullsWithAggregationSelection(
+              aggregationBufferSets, aggregrateIndex,
+              inputColVector, batchSize, inputColVector.isNull);
+          }
+        }
+      }
+    }
+
+    private void iterateNoNullsRepeatingWithAggregationSelection(
+      VectorAggregationBufferRow[] aggregationBufferSets,
+      int aggregrateIndex,
+      IntervalDayTimeColumnVector inputColVector,
+      int batchSize) {
+
+      for (int i=0; i < batchSize; ++i) {
+        Aggregation myagg = getCurrentAggregationBuffer(
+          aggregationBufferSets,
+          aggregrateIndex,
+          i);
+        // Repeating use index 0.
+        myagg.checkValue(inputColVector, 0);
+      }
+    }
+
+    private void iterateNoNullsSelectionWithAggregationSelection(
+      VectorAggregationBufferRow[] aggregationBufferSets,
+      int aggregrateIndex,
+      IntervalDayTimeColumnVector inputColVector,
+      int[] selection,
+      int batchSize) {
+
+      for (int i=0; i < batchSize; ++i) {
+        Aggregation myagg = getCurrentAggregationBuffer(
+          aggregationBufferSets,
+          aggregrateIndex,
+          i);
+        myagg.checkValue(inputColVector, selection[i]);
+      }
+    }
+
+    private void iterateNoNullsWithAggregationSelection(
+      VectorAggregationBufferRow[] aggregationBufferSets,
+      int aggregrateIndex,
+      IntervalDayTimeColumnVector inputColVector,
+      int batchSize) {
+      for (int i=0; i < batchSize; ++i) {
+        Aggregation myagg = getCurrentAggregationBuffer(
+          aggregationBufferSets,
+          aggregrateIndex,
+          i);
+        myagg.checkValue(inputColVector, i);
+      }
+    }
+
+    private void iterateHasNullsRepeatingSelectionWithAggregationSelection(
+      VectorAggregationBufferRow[] aggregationBufferSets,
+      int aggregrateIndex,
+      IntervalDayTimeColumnVector inputColVector,
+      int batchSize,
+      int[] selection,
+      boolean[] isNull) {
+
+      for (int i=0; i < batchSize; ++i) {
+        if (!isNull[selection[i]]) {
+          Aggregation myagg = getCurrentAggregationBuffer(
+            aggregationBufferSets,
+            aggregrateIndex,
+            i);
+          // Repeating use index 0.
+          myagg.checkValue(inputColVector, 0);
+        }
+      }
+
+    }
+
+    private void iterateHasNullsRepeatingWithAggregationSelection(
+      VectorAggregationBufferRow[] aggregationBufferSets,
+      int aggregrateIndex,
+      IntervalDayTimeColumnVector inputColVector,
+      int batchSize,
+      boolean[] isNull) {
+
+      for (int i=0; i < batchSize; ++i) {
+        if (!isNull[i]) {
+          Aggregation myagg = getCurrentAggregationBuffer(
+            aggregationBufferSets,
+            aggregrateIndex,
+            i);
+          // Repeating use index 0.
+          myagg.checkValue(inputColVector, 0);
+        }
+      }
+    }
+
+    private void iterateHasNullsSelectionWithAggregationSelection(
+      VectorAggregationBufferRow[] aggregationBufferSets,
+      int aggregrateIndex,
+      IntervalDayTimeColumnVector inputColVector,
+      int batchSize,
+      int[] selection,
+      boolean[] isNull) {
+
+      for (int j=0; j < batchSize; ++j) {
+        int i = selection[j];
+        if (!isNull[i]) {
+          Aggregation myagg = getCurrentAggregationBuffer(
+            aggregationBufferSets,
+            aggregrateIndex,
+            j);
+          myagg.checkValue(inputColVector, i);
+        }
+      }
+   }
+
+    private void iterateHasNullsWithAggregationSelection(
+      VectorAggregationBufferRow[] aggregationBufferSets,
+      int aggregrateIndex,
+      IntervalDayTimeColumnVector inputColVector,
+      int batchSize,
+      boolean[] isNull) {
+
+      for (int i=0; i < batchSize; ++i) {
+        if (!isNull[i]) {
+          Aggregation myagg = getCurrentAggregationBuffer(
+            aggregationBufferSets,
+            aggregrateIndex,
+            i);
+          myagg.checkValue(inputColVector, i);
+        }
+      }
+   }
+
+    @Override
+    public void aggregateInput(AggregationBuffer agg, VectorizedRowBatch batch)
+      throws HiveException {
+
+        inputExpression.evaluate(batch);
+
+        IntervalDayTimeColumnVector inputColVector = (IntervalDayTimeColumnVector)batch.
+            cols[this.inputExpression.getOutputColumn()];
+
+        int batchSize = batch.size;
+
+        if (batchSize == 0) {
+          return;
+        }
+
+        Aggregation myagg = (Aggregation)agg;
+
+        if (inputColVector.isRepeating) {
+          if (inputColVector.noNulls &&
+            (myagg.isNull || (inputColVector.compareTo(myagg.value, 0) <OperatorSymbol> 0))) {
+            myagg.isNull = false;
+            inputColVector.intervalDayTimeUpdate(myagg.value, 0);
+          }
+          return;
+        }
+
+        if (!batch.selectedInUse && inputColVector.noNulls) {
+          iterateNoSelectionNoNulls(myagg, inputColVector, batchSize);
+        }
+        else if (!batch.selectedInUse) {
+          iterateNoSelectionHasNulls(myagg, inputColVector,
+            batchSize, inputColVector.isNull);
+        }
+        else if (inputColVector.noNulls){
+          iterateSelectionNoNulls(myagg, inputColVector, batchSize, batch.selected);
+        }
+        else {
+          iterateSelectionHasNulls(myagg, inputColVector,
+            batchSize, inputColVector.isNull, batch.selected);
+        }
+    }
+
+    private void iterateSelectionHasNulls(
+        Aggregation myagg,
+        IntervalDayTimeColumnVector inputColVector,
+        int batchSize,
+        boolean[] isNull,
+        int[] selected) {
+
+      for (int j=0; j< batchSize; ++j) {
+        int i = selected[j];
+        if (!isNull[i]) {
+          if (myagg.isNull) {
+            myagg.isNull = false;
+            inputColVector.intervalDayTimeUpdate(myagg.value, i);
+          }
+          else if (inputColVector.compareTo(myagg.value, i) <OperatorSymbol> 0) {
+            inputColVector.intervalDayTimeUpdate(myagg.value, i);
+          }
+        }
+      }
+    }
+
+    private void iterateSelectionNoNulls(
+        Aggregation myagg,
+        IntervalDayTimeColumnVector inputColVector,
+        int batchSize,
+        int[] selected) {
+
+      if (myagg.isNull) {
+        inputColVector.intervalDayTimeUpdate(myagg.value, selected[0]);
+        myagg.isNull = false;
+      }
+
+      for (int i=0; i< batchSize; ++i) {
+        int sel = selected[i];
+        if (inputColVector.compareTo(myagg.value, sel) <OperatorSymbol> 0) {
+          inputColVector.intervalDayTimeUpdate(myagg.value, sel);
+        }
+      }
+    }
+
+    private void iterateNoSelectionHasNulls(
+        Aggregation myagg,
+        IntervalDayTimeColumnVector inputColVector,
+        int batchSize,
+        boolean[] isNull) {
+
+      for(int i=0;i<batchSize;++i) {
+        if (!isNull[i]) {
+          if (myagg.isNull) {
+            inputColVector.intervalDayTimeUpdate(myagg.value, i);
+            myagg.isNull = false;
+          }
+          else if (inputColVector.compareTo(myagg.value, i) <OperatorSymbol> 0) {
+            inputColVector.intervalDayTimeUpdate(myagg.value, i);
+          }
+        }
+      }
+    }
+
+    private void iterateNoSelectionNoNulls(
+        Aggregation myagg,
+        IntervalDayTimeColumnVector inputColVector,
+        int batchSize) {
+      if (myagg.isNull) {
+        inputColVector.intervalDayTimeUpdate(myagg.value, 0);
+        myagg.isNull = false;
+      }
+
+      for (int i=0;i<batchSize;++i) {
+        if (inputColVector.compareTo(myagg.value, i) <OperatorSymbol> 0) {
+          inputColVector.intervalDayTimeUpdate(myagg.value, i);
+        }
+      }
+    }
+
+    @Override
+    public AggregationBuffer getNewAggregationBuffer() throws HiveException {
+      return new Aggregation();
+    }
+
+    @Override
+    public void reset(AggregationBuffer agg) throws HiveException {
+      Aggregation myAgg = (Aggregation) agg;
+      myAgg.reset();
+    }
+
+    @Override
+    public Object evaluateOutput(
+        AggregationBuffer agg) throws HiveException {
+    Aggregation myagg = (Aggregation) agg;
+      if (myagg.isNull) {
+        return null;
+      }
+      else {
+        return resultWriter.writeValue(myagg.value);
+      }
+    }
+
+    @Override
+    public ObjectInspector getOutputObjectInspector() {
+      return resultWriter.getObjectInspector();
+    }
+
+    @Override
+    public int getAggregationBufferFixedSize() {
+    JavaDataModel model = JavaDataModel.get();
+    return JavaDataModel.alignUp(
+      model.object() +
+      model.primitive2(),
+      model.memoryAlign());
+  }
+
+  public VectorExpression getInputExpression() {
+    return inputExpression;
+  }
+
+  public void setInputExpression(VectorExpression inputExpression) {
+    this.inputExpression = inputExpression;
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFMinMaxTimestamp.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFMinMaxTimestamp.txt b/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFMinMaxTimestamp.txt
index 3cdf405..7e34965 100644
--- a/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFMinMaxTimestamp.txt
+++ b/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFMinMaxTimestamp.txt
@@ -18,7 +18,8 @@
 
 package org.apache.hadoop.hive.ql.exec.vector.expressions.aggregates.gen;
 
-import org.apache.hadoop.hive.common.type.PisaTimestamp;
+import java.sql.Timestamp;
+
 import org.apache.hadoop.hive.ql.exec.Description;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.aggregates.VectorAggregateExpression;
@@ -49,7 +50,7 @@ public class <ClassName> extends VectorAggregateExpression {
 
       private static final long serialVersionUID = 1L;
 
-      transient private final PisaTimestamp value;
+      transient private final Timestamp value;
 
       /**
       * Value is explicitly (re)initialized in reset()
@@ -57,15 +58,15 @@ public class <ClassName> extends VectorAggregateExpression {
       transient private boolean isNull = true;
 
       public Aggregation() {
-        value = new PisaTimestamp();
+        value = new Timestamp(0);
       }
 
       public void checkValue(TimestampColumnVector colVector, int index) {
         if (isNull) {
           isNull = false;
-          colVector.pisaTimestampUpdate(this.value, index);
+          colVector.timestampUpdate(this.value, index);
         } else if (colVector.compareTo(this.value, index) <OperatorSymbol> 0) {
-          colVector.pisaTimestampUpdate(this.value, index);
+          colVector.timestampUpdate(this.value, index);
         }
       }
 
@@ -77,7 +78,7 @@ public class <ClassName> extends VectorAggregateExpression {
       @Override
       public void reset () {
         isNull = true;
-        this.value.reset();
+        this.value.setTime(0);
       }
     }
 
@@ -311,7 +312,7 @@ public class <ClassName> extends VectorAggregateExpression {
           if (inputColVector.noNulls &&
             (myagg.isNull || (inputColVector.compareTo(myagg.value, 0) <OperatorSymbol> 0))) {
             myagg.isNull = false;
-            inputColVector.pisaTimestampUpdate(myagg.value, 0);
+            inputColVector.timestampUpdate(myagg.value, 0);
           }
           return;
         }
@@ -344,10 +345,10 @@ public class <ClassName> extends VectorAggregateExpression {
         if (!isNull[i]) {
           if (myagg.isNull) {
             myagg.isNull = false;
-            inputColVector.pisaTimestampUpdate(myagg.value, i);
+            inputColVector.timestampUpdate(myagg.value, i);
           }
           else if (inputColVector.compareTo(myagg.value, i) <OperatorSymbol> 0) {
-            inputColVector.pisaTimestampUpdate(myagg.value, i);
+            inputColVector.timestampUpdate(myagg.value, i);
           }
         }
       }
@@ -360,14 +361,14 @@ public class <ClassName> extends VectorAggregateExpression {
         int[] selected) {
 
       if (myagg.isNull) {
-        inputColVector.pisaTimestampUpdate(myagg.value, selected[0]);
+        inputColVector.timestampUpdate(myagg.value, selected[0]);
         myagg.isNull = false;
       }
 
       for (int i=0; i< batchSize; ++i) {
         int sel = selected[i];
         if (inputColVector.compareTo(myagg.value, sel) <OperatorSymbol> 0) {
-          inputColVector.pisaTimestampUpdate(myagg.value, sel);
+          inputColVector.timestampUpdate(myagg.value, sel);
         }
       }
     }
@@ -381,11 +382,11 @@ public class <ClassName> extends VectorAggregateExpression {
       for(int i=0;i<batchSize;++i) {
         if (!isNull[i]) {
           if (myagg.isNull) {
-            inputColVector.pisaTimestampUpdate(myagg.value, i);
+            inputColVector.timestampUpdate(myagg.value, i);
             myagg.isNull = false;
           }
           else if (inputColVector.compareTo(myagg.value, i) <OperatorSymbol> 0) {
-            inputColVector.pisaTimestampUpdate(myagg.value, i);
+            inputColVector.timestampUpdate(myagg.value, i);
           }
         }
       }
@@ -396,13 +397,13 @@ public class <ClassName> extends VectorAggregateExpression {
         TimestampColumnVector inputColVector,
         int batchSize) {
       if (myagg.isNull) {
-        inputColVector.pisaTimestampUpdate(myagg.value, 0);
+        inputColVector.timestampUpdate(myagg.value, 0);
         myagg.isNull = false;
       }
 
       for (int i=0;i<batchSize;++i) {
         if (inputColVector.compareTo(myagg.value, i) <OperatorSymbol> 0) {
-          inputColVector.pisaTimestampUpdate(myagg.value, i);
+          inputColVector.timestampUpdate(myagg.value, i);
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/TimestampUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/TimestampUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/TimestampUtils.java
index 5de055c..bb795fa 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/TimestampUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/TimestampUtils.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hive.ql.exec.vector;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.hadoop.hive.serde2.io.DateWritable;
+import org.apache.hadoop.hive.serde2.io.HiveIntervalDayTimeWritable;
 import org.apache.hadoop.hive.serde2.io.TimestampWritable;
 
 public final class TimestampUtils {
@@ -38,4 +39,11 @@ public final class TimestampUtils {
     timestampWritable.set(timestampColVector.asScratchTimestamp(elementNum));
     return timestampWritable;
   }
+
+  public static HiveIntervalDayTimeWritable intervalDayTimeColumnVectorWritable(
+      IntervalDayTimeColumnVector intervalDayTimeColVector, int elementNum,
+      HiveIntervalDayTimeWritable intervalDayTimeWritable) {
+    intervalDayTimeWritable.set(intervalDayTimeColVector.asScratchIntervalDayTime(elementNum));
+    return intervalDayTimeWritable;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorAssignRow.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorAssignRow.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorAssignRow.java
index 965c027..de0300a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorAssignRow.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorAssignRow.java
@@ -288,7 +288,26 @@ public abstract class VectorAssignRow {
     }
   }
 
-  private class IntervalDayTimeAssigner extends AbstractTimestampAssigner {
+  private abstract class AbstractIntervalDayTimeAssigner extends Assigner {
+
+    protected IntervalDayTimeColumnVector colVector;
+
+    AbstractIntervalDayTimeAssigner(int columnIndex) {
+      super(columnIndex);
+    }
+
+    @Override
+    void setColumnVector(VectorizedRowBatch batch) {
+      colVector = (IntervalDayTimeColumnVector) batch.cols[columnIndex];
+    }
+
+    @Override
+    void forgetColumnVector() {
+      colVector = null;
+    }
+  }
+
+  private class IntervalDayTimeAssigner extends AbstractIntervalDayTimeAssigner {
 
     IntervalDayTimeAssigner(int columnIndex) {
       super(columnIndex);
@@ -301,7 +320,7 @@ public abstract class VectorAssignRow {
       } else {
         HiveIntervalDayTimeWritable idtw = (HiveIntervalDayTimeWritable) object;
         HiveIntervalDayTime idt = idtw.getHiveIntervalDayTime();
-        colVector.set(batchIndex, idt.pisaTimestampUpdate(colVector.useScratchPisaTimestamp()));
+        colVector.set(batchIndex, idt);
         colVector.isNull[batchIndex] = false;
       }
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorColumnAssignFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorColumnAssignFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorColumnAssignFactory.java
index 463c8a6..96b8f78 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorColumnAssignFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorColumnAssignFactory.java
@@ -25,6 +25,7 @@ import java.util.Map;
 
 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.HiveVarchar;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.serde2.io.ByteWritable;
@@ -176,6 +177,16 @@ public class VectorColumnAssignFactory {
     }
   }
 
+  private static abstract class VectorIntervalDayTimeColumnAssign
+  extends VectorColumnAssignVectorBase<IntervalDayTimeColumnVector> {
+
+    protected void assignIntervalDayTime(HiveIntervalDayTime value, int index) {
+      outCol.set(index, value);
+    }
+    protected void assignIntervalDayTime(HiveIntervalDayTimeWritable tw, int index) {
+      outCol.set(index, tw.getHiveIntervalDayTime());
+    }
+  }
 
   public static VectorColumnAssign[] buildAssigners(VectorizedRowBatch outputBatch)
       throws HiveException {
@@ -364,7 +375,7 @@ public class VectorColumnAssignFactory {
           }
         }.init(outputBatch, (LongColumnVector) destCol);
         break;
-      case INTERVAL_DAY_TIME:outVCA = new VectorLongColumnAssign() {
+      case INTERVAL_DAY_TIME:outVCA = new VectorIntervalDayTimeColumnAssign() {
         @Override
         public void assignObjectValue(Object val, int destIndex) throws HiveException {
           if (val == null) {
@@ -372,12 +383,12 @@ public class VectorColumnAssignFactory {
           }
           else {
             HiveIntervalDayTimeWritable bw = (HiveIntervalDayTimeWritable) val;
-            assignLong(
-                DateUtils.getIntervalDayTimeTotalNanos(bw.getHiveIntervalDayTime()),
+            assignIntervalDayTime(
+                bw.getHiveIntervalDayTime(),
                 destIndex);
           }
         }
-      }.init(outputBatch, (LongColumnVector) destCol);
+      }.init(outputBatch, (IntervalDayTimeColumnVector) destCol);
       break;
       default:
         throw new HiveException("Incompatible Long vector column and primitive category " +

http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorColumnSetInfo.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorColumnSetInfo.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorColumnSetInfo.java
index 0949145..935b47b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorColumnSetInfo.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorColumnSetInfo.java
@@ -60,6 +60,11 @@ public class VectorColumnSetInfo {
   protected int[] timestampIndices;
 
   /**
+   * indices of INTERVAL_DAY_TIME primitive keys.
+   */
+  protected int[] intervalDayTimeIndices;
+
+  /**
    * Helper class for looking up a key value based on key index.
    */
   public class KeyLookupHelper {
@@ -68,12 +73,13 @@ public class VectorColumnSetInfo {
     public int stringIndex;
     public int decimalIndex;
     public int timestampIndex;
+    public int intervalDayTimeIndex;
 
     private static final int INDEX_UNUSED = -1;
 
     private void resetIndices() {
         this.longIndex = this.doubleIndex = this.stringIndex = this.decimalIndex =
-            timestampIndex = INDEX_UNUSED;
+            timestampIndex = intervalDayTimeIndex = INDEX_UNUSED;
     }
     public void setLong(int index) {
       resetIndices();
@@ -99,6 +105,11 @@ public class VectorColumnSetInfo {
       resetIndices();
       this.timestampIndex= index;
     }
+
+    public void setIntervalDayTime(int index) {
+      resetIndices();
+      this.intervalDayTimeIndex= index;
+    }
   }
 
   /**
@@ -114,6 +125,7 @@ public class VectorColumnSetInfo {
   protected int stringIndicesIndex;
   protected int decimalIndicesIndex;
   protected int timestampIndicesIndex;
+  protected int intervalDayTimeIndicesIndex;
 
   protected VectorColumnSetInfo(int keyCount) {
     this.keyCount = keyCount;
@@ -130,6 +142,8 @@ public class VectorColumnSetInfo {
     decimalIndicesIndex = 0;
     timestampIndices = new int[this.keyCount];
     timestampIndicesIndex = 0;
+    intervalDayTimeIndices = new int[this.keyCount];
+    intervalDayTimeIndicesIndex = 0;
     indexLookup = new KeyLookupHelper[this.keyCount];
   }
 
@@ -172,6 +186,12 @@ public class VectorColumnSetInfo {
       ++timestampIndicesIndex;
       break;
 
+    case INTERVAL_DAY_TIME:
+      intervalDayTimeIndices[intervalDayTimeIndicesIndex] = addIndex;
+      indexLookup[addIndex].setIntervalDayTime(intervalDayTimeIndicesIndex);
+      ++intervalDayTimeIndicesIndex;
+      break;
+
     default:
       throw new HiveException("Unexpected column vector type " + columnVectorType);
     }
@@ -185,5 +205,6 @@ public class VectorColumnSetInfo {
     stringIndices = Arrays.copyOf(stringIndices, stringIndicesIndex);
     decimalIndices = Arrays.copyOf(decimalIndices, decimalIndicesIndex);
     timestampIndices = Arrays.copyOf(timestampIndices, timestampIndicesIndex);
+    intervalDayTimeIndices = Arrays.copyOf(intervalDayTimeIndices, intervalDayTimeIndicesIndex);
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorCopyRow.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorCopyRow.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorCopyRow.java
index 73476a3..c8e0284 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorCopyRow.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorCopyRow.java
@@ -223,6 +223,34 @@ public class VectorCopyRow {
       }
     }
   }
+
+  private class IntervalDayTimeCopyRow extends CopyRow {
+
+    IntervalDayTimeCopyRow(int inColumnIndex, int outColumnIndex) {
+      super(inColumnIndex, outColumnIndex);
+    }
+
+    @Override
+    void copy(VectorizedRowBatch inBatch, int inBatchIndex, VectorizedRowBatch outBatch, int outBatchIndex) {
+      IntervalDayTimeColumnVector inColVector = (IntervalDayTimeColumnVector) inBatch.cols[inColumnIndex];
+      IntervalDayTimeColumnVector outColVector = (IntervalDayTimeColumnVector) outBatch.cols[outColumnIndex];
+
+      if (inColVector.isRepeating) {
+        if (inColVector.noNulls || !inColVector.isNull[0]) {
+          outColVector.setElement(outBatchIndex, 0, inColVector);
+        } else {
+          VectorizedBatchUtil.setNullColIsNullValue(outColVector, outBatchIndex);
+        }
+      } else {
+        if (inColVector.noNulls || !inColVector.isNull[inBatchIndex]) {
+          outColVector.setElement(outBatchIndex, inBatchIndex, inColVector);
+        } else {
+          VectorizedBatchUtil.setNullColIsNullValue(outColVector, outBatchIndex);
+        }
+      }
+    }
+  }
+
   private CopyRow[] subRowToBatchCopiersByValue;
   private CopyRow[] subRowToBatchCopiersByReference;
 
@@ -250,6 +278,10 @@ public class VectorCopyRow {
         copyRowByValue = new TimestampCopyRow(inputColumn, outputColumn);
         break;
 
+      case INTERVAL_DAY_TIME:
+        copyRowByValue = new IntervalDayTimeCopyRow(inputColumn, outputColumn);
+        break;
+
       case DOUBLE:
         copyRowByValue = new DoubleCopyRow(inputColumn, outputColumn);
         break;

http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorDeserializeRow.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorDeserializeRow.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorDeserializeRow.java
index 50881e7..3eadc12 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorDeserializeRow.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorDeserializeRow.java
@@ -264,7 +264,14 @@ public final class VectorDeserializeRow<T extends DeserializeRead> {
     }
   }
 
-  private class IntervalDayTimeReader extends AbstractTimestampReader {
+  private abstract class AbstractIntervalDayTimeReader extends Reader<T> {
+
+    AbstractIntervalDayTimeReader(int columnIndex) {
+      super(columnIndex);
+    }
+  }
+
+  private class IntervalDayTimeReader extends AbstractIntervalDayTimeReader {
 
     DeserializeRead.ReadIntervalDayTimeResults readIntervalDayTimeResults;
 
@@ -275,14 +282,14 @@ public final class VectorDeserializeRow<T extends DeserializeRead> {
 
     @Override
     void apply(VectorizedRowBatch batch, int batchIndex) throws IOException {
-      TimestampColumnVector colVector = (TimestampColumnVector) batch.cols[columnIndex];
+      IntervalDayTimeColumnVector colVector = (IntervalDayTimeColumnVector) batch.cols[columnIndex];
 
       if (deserializeRead.readCheckNull()) {
         VectorizedBatchUtil.setNullColIsNullValue(colVector, batchIndex);
       } else {
         deserializeRead.readIntervalDayTime(readIntervalDayTimeResults);
         HiveIntervalDayTime idt = readIntervalDayTimeResults.getHiveIntervalDayTime();
-        colVector.set(batchIndex, idt.pisaTimestampUpdate(colVector.useScratchPisaTimestamp()));
+        colVector.set(batchIndex, idt);
         colVector.isNull[batchIndex] = false;
       }
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorExpressionDescriptor.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorExpressionDescriptor.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorExpressionDescriptor.java
index 0b9ad55..7b3f781 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorExpressionDescriptor.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorExpressionDescriptor.java
@@ -59,6 +59,9 @@ public class VectorExpressionDescriptor {
   // TimestampColumnVector -->
   //    TIMESTAMP
   //
+  // IntervalDayTimeColumnVector -->
+  //    INTERVAL_DAY_TIME
+  //
   public enum ArgumentType {
     NONE                    (0x000),
     INT_FAMILY              (0x001),
@@ -76,7 +79,6 @@ public class VectorExpressionDescriptor {
     INTERVAL_FAMILY         (INTERVAL_YEAR_MONTH.value | INTERVAL_DAY_TIME.value),
     INT_INTERVAL_YEAR_MONTH     (INT_FAMILY.value | INTERVAL_YEAR_MONTH.value),
     INT_DATE_INTERVAL_YEAR_MONTH  (INT_FAMILY.value | DATE.value | INTERVAL_YEAR_MONTH.value),
-    TIMESTAMP_INTERVAL_DAY_TIME (TIMESTAMP.value | INTERVAL_DAY_TIME.value),
     STRING_DATETIME_FAMILY  (STRING_FAMILY.value | DATETIME_FAMILY.value),
     ALL_FAMILY              (0xFFF);
 
@@ -346,7 +348,7 @@ public class VectorExpressionDescriptor {
           return ve;
         }
       } catch (Exception ex) {
-        throw new HiveException(ex);
+        throw new HiveException("Could not instantiate VectorExpression class " + ve.getSimpleName(), ex);
       }
     }
     if (LOG.isDebugEnabled()) {

http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorExtractRow.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorExtractRow.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorExtractRow.java
index 622f4a3..e883f38 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorExtractRow.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorExtractRow.java
@@ -32,7 +32,6 @@ 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.PisaTimestamp;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.serde2.io.DateWritable;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
@@ -323,7 +322,26 @@ public abstract class VectorExtractRow {
     }
   }
 
-  private class IntervalDayTimeExtractor extends AbstractTimestampExtractor {
+  private abstract class AbstractIntervalDayTimeExtractor extends Extractor {
+
+    protected IntervalDayTimeColumnVector colVector;
+
+    AbstractIntervalDayTimeExtractor(int columnIndex) {
+      super(columnIndex);
+    }
+
+    @Override
+    void setColumnVector(VectorizedRowBatch batch) {
+      colVector = (IntervalDayTimeColumnVector) batch.cols[columnIndex];
+    }
+
+    @Override
+    void forgetColumnVector() {
+      colVector = null;
+    }
+  }
+
+  private class IntervalDayTimeExtractor extends AbstractIntervalDayTimeExtractor {
 
     private HiveIntervalDayTime hiveIntervalDayTime;
 
@@ -337,7 +355,7 @@ public abstract class VectorExtractRow {
     Object extract(int batchIndex) {
       int adjustedIndex = (colVector.isRepeating ? 0 : batchIndex);
       if (colVector.noNulls || !colVector.isNull[adjustedIndex]) {
-        hiveIntervalDayTime.set(colVector.asScratchPisaTimestamp(adjustedIndex));
+        hiveIntervalDayTime.set(colVector.asScratchIntervalDayTime(adjustedIndex));
         PrimitiveObjectInspectorFactory.writableHiveIntervalDayTimeObjectInspector.set(object, hiveIntervalDayTime);
         return object;
       } else {

http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorGroupKeyHelper.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorGroupKeyHelper.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorGroupKeyHelper.java
index 9f0ac11..50d0452 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorGroupKeyHelper.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorGroupKeyHelper.java
@@ -131,5 +131,17 @@ public class VectorGroupKeyHelper extends VectorColumnSetInfo {
         outputColumnVector.isNull[outputBatch.size] = true;
       }
     }
+    for(int i=0;i<intervalDayTimeIndices.length; ++i) {
+      int keyIndex = intervalDayTimeIndices[i];
+      IntervalDayTimeColumnVector inputColumnVector = (IntervalDayTimeColumnVector) inputBatch.cols[keyIndex];
+      IntervalDayTimeColumnVector outputColumnVector = (IntervalDayTimeColumnVector) outputBatch.cols[keyIndex];
+      if (inputColumnVector.noNulls || !inputColumnVector.isNull[0]) {
+
+        outputColumnVector.setElement(outputBatch.size, 0, inputColumnVector);
+      } else {
+        outputColumnVector.noNulls = false;
+        outputColumnVector.isNull[outputBatch.size] = true;
+      }
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorHashKeyWrapper.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorHashKeyWrapper.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorHashKeyWrapper.java
index b5d8164..8a101a6 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorHashKeyWrapper.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorHashKeyWrapper.java
@@ -18,16 +18,16 @@
 
 package org.apache.hadoop.hive.ql.exec.vector;
 
+import java.sql.Timestamp;
 import java.util.Arrays;
 
-import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
-import org.apache.hadoop.hive.serde2.io.TimestampWritable;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
-import org.apache.hadoop.hive.common.type.PisaTimestamp;
+import org.apache.hadoop.hive.common.type.HiveIntervalDayTime;
 import org.apache.hadoop.hive.ql.exec.KeyWrapper;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.StringExpr;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.util.JavaDataModel;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 
 /**
@@ -44,7 +44,8 @@ public class VectorHashKeyWrapper extends KeyWrapper {
   private static final double[] EMPTY_DOUBLE_ARRAY = new double[0];
   private static final byte[][] EMPTY_BYTES_ARRAY = new byte[0][];
   private static final HiveDecimalWritable[] EMPTY_DECIMAL_ARRAY = new HiveDecimalWritable[0];
-  private static final PisaTimestamp[] EMPTY_TIMESTAMP_ARRAY = new PisaTimestamp[0];
+  private static final Timestamp[] EMPTY_TIMESTAMP_ARRAY = new Timestamp[0];
+  private static final HiveIntervalDayTime[] EMPTY_INTERVAL_DAY_TIME_ARRAY = new HiveIntervalDayTime[0];
 
   private long[] longValues;
   private double[] doubleValues;
@@ -55,17 +56,21 @@ public class VectorHashKeyWrapper extends KeyWrapper {
 
   private HiveDecimalWritable[] decimalValues;
 
-  private PisaTimestamp[] timestampValues;
+  private Timestamp[] timestampValues;
+
+  private HiveIntervalDayTime[] intervalDayTimeValues;
 
   private boolean[] isNull;
   private int hashcode;
 
   public VectorHashKeyWrapper(int longValuesCount, int doubleValuesCount,
-          int byteValuesCount, int decimalValuesCount, int timestampValuesCount) {
+          int byteValuesCount, int decimalValuesCount, int timestampValuesCount,
+          int intervalDayTimeValuesCount) {
     longValues = longValuesCount > 0 ? new long[longValuesCount] : EMPTY_LONG_ARRAY;
     doubleValues = doubleValuesCount > 0 ? new double[doubleValuesCount] : EMPTY_DOUBLE_ARRAY;
     decimalValues = decimalValuesCount > 0 ? new HiveDecimalWritable[decimalValuesCount] : EMPTY_DECIMAL_ARRAY;
-    timestampValues = timestampValuesCount > 0 ? new PisaTimestamp[timestampValuesCount] : EMPTY_TIMESTAMP_ARRAY;
+    timestampValues = timestampValuesCount > 0 ? new Timestamp[timestampValuesCount] : EMPTY_TIMESTAMP_ARRAY;
+    intervalDayTimeValues = intervalDayTimeValuesCount > 0 ? new HiveIntervalDayTime[intervalDayTimeValuesCount] : EMPTY_INTERVAL_DAY_TIME_ARRAY;
     for(int i = 0; i < decimalValuesCount; ++i) {
       decimalValues[i] = new HiveDecimalWritable(HiveDecimal.ZERO);
     }
@@ -79,10 +84,13 @@ public class VectorHashKeyWrapper extends KeyWrapper {
       byteLengths = EMPTY_INT_ARRAY;
     }
     for(int i = 0; i < timestampValuesCount; ++i) {
-      timestampValues[i] = new PisaTimestamp();
+      timestampValues[i] = new Timestamp(0);
+    }
+    for(int i = 0; i < intervalDayTimeValuesCount; ++i) {
+      intervalDayTimeValues[i] = new HiveIntervalDayTime();
     }
     isNull = new boolean[longValuesCount + doubleValuesCount + byteValuesCount +
-                         decimalValuesCount + timestampValuesCount];
+                         decimalValuesCount + timestampValuesCount + intervalDayTimeValuesCount];
     hashcode = 0;
   }
 
@@ -108,6 +116,10 @@ public class VectorHashKeyWrapper extends KeyWrapper {
       hashcode ^= timestampValues[i].hashCode();
     }
 
+    for (int i = 0; i < intervalDayTimeValues.length; i++) {
+      hashcode ^= intervalDayTimeValues[i].hashCode();
+    }
+
     // This code, with branches and all, is not executed if there are no string keys
     for (int i = 0; i < byteValues.length; ++i) {
       /*
@@ -146,6 +158,7 @@ public class VectorHashKeyWrapper extends KeyWrapper {
           Arrays.equals(doubleValues, keyThat.doubleValues) &&
           Arrays.equals(decimalValues,  keyThat.decimalValues) &&
           Arrays.equals(timestampValues,  keyThat.timestampValues) &&
+          Arrays.equals(intervalDayTimeValues,  keyThat.intervalDayTimeValues) &&
           Arrays.equals(isNull, keyThat.isNull) &&
           byteValues.length == keyThat.byteValues.length &&
           (0 == byteValues.length || bytesEquals(keyThat));
@@ -212,14 +225,21 @@ public class VectorHashKeyWrapper extends KeyWrapper {
       clone.byteLengths = EMPTY_INT_ARRAY;
     }
     if (timestampValues.length > 0) {
-      clone.timestampValues = new PisaTimestamp[timestampValues.length];
+      clone.timestampValues = new Timestamp[timestampValues.length];
       for(int i = 0; i < timestampValues.length; ++i) {
-        clone.timestampValues[i] = new PisaTimestamp();
-        clone.timestampValues[i].update(timestampValues[i]);
+        clone.timestampValues[i] = (Timestamp) timestampValues[i].clone();
       }
     } else {
       clone.timestampValues = EMPTY_TIMESTAMP_ARRAY;
     }
+    if (intervalDayTimeValues.length > 0) {
+      clone.intervalDayTimeValues = new HiveIntervalDayTime[intervalDayTimeValues.length];
+      for(int i = 0; i < intervalDayTimeValues.length; ++i) {
+        clone.intervalDayTimeValues[i] = (HiveIntervalDayTime) intervalDayTimeValues[i].clone();
+      }
+    } else {
+      clone.intervalDayTimeValues = EMPTY_INTERVAL_DAY_TIME_ARRAY;
+    }
 
     clone.hashcode = hashcode;
     assert clone.equals(this);
@@ -281,14 +301,14 @@ public class VectorHashKeyWrapper extends KeyWrapper {
       isNull[longValues.length + doubleValues.length + byteValues.length + index] = true;
   }
 
-  public void assignTimestamp(int index, PisaTimestamp value) {
-    timestampValues[index].update(value);
+  public void assignTimestamp(int index, Timestamp value) {
+    timestampValues[index] = value;
     isNull[longValues.length + doubleValues.length + byteValues.length +
            decimalValues.length + index] = false;
   }
 
   public void assignTimestamp(int index, TimestampColumnVector colVector, int elementNum) {
-    colVector.pisaTimestampUpdate(timestampValues[index], elementNum);
+    colVector.timestampUpdate(timestampValues[index], elementNum);
     isNull[longValues.length + doubleValues.length + byteValues.length +
            decimalValues.length + index] = false;
   }
@@ -298,15 +318,33 @@ public class VectorHashKeyWrapper extends KeyWrapper {
              decimalValues.length + index] = true;
   }
 
+  public void assignIntervalDayTime(int index, HiveIntervalDayTime value) {
+    intervalDayTimeValues[index].set(value);
+    isNull[longValues.length + doubleValues.length + byteValues.length +
+           decimalValues.length + timestampValues.length + index] = false;
+  }
+
+  public void assignIntervalDayTime(int index, IntervalDayTimeColumnVector colVector, int elementNum) {
+    intervalDayTimeValues[index].set(colVector.asScratchIntervalDayTime(elementNum));
+    isNull[longValues.length + doubleValues.length + byteValues.length +
+           decimalValues.length + timestampValues.length + index] = false;
+  }
+
+  public void assignNullIntervalDayTime(int index) {
+      isNull[longValues.length + doubleValues.length + byteValues.length +
+             decimalValues.length + timestampValues.length + index] = true;
+  }
+
   @Override
   public String toString()
   {
-    return String.format("%d[%s] %d[%s] %d[%s] %d[%s] %d[%s]",
+    return String.format("%d[%s] %d[%s] %d[%s] %d[%s] %d[%s] %d[%s]",
         longValues.length, Arrays.toString(longValues),
         doubleValues.length, Arrays.toString(doubleValues),
         byteValues.length, Arrays.toString(byteValues),
         decimalValues.length, Arrays.toString(decimalValues),
-        timestampValues.length, Arrays.toString(timestampValues));
+        timestampValues.length, Arrays.toString(timestampValues),
+        intervalDayTimeValues.length, Arrays.toString(intervalDayTimeValues));
   }
 
   public boolean getIsLongNull(int i) {
@@ -364,9 +402,17 @@ public class VectorHashKeyWrapper extends KeyWrapper {
                   decimalValues.length + i];
   }
 
-  public PisaTimestamp getTimestamp(int i) {
+  public Timestamp getTimestamp(int i) {
     return timestampValues[i];
   }
 
+  public boolean getIsIntervalDayTimeNull(int i) {
+    return isNull[longValues.length + doubleValues.length + byteValues.length +
+                  decimalValues.length + timestampValues.length + i];
+  }
+
+  public HiveIntervalDayTime getIntervalDayTime(int i) {
+    return intervalDayTimeValues[i];
+  }
 }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorHashKeyWrapperBatch.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorHashKeyWrapperBatch.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorHashKeyWrapperBatch.java
index 1c34124..bfd26ae 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorHashKeyWrapperBatch.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorHashKeyWrapperBatch.java
@@ -198,6 +198,28 @@ public class VectorHashKeyWrapperBatch extends VectorColumnSetInfo {
             columnVector.noNulls, columnVector.isRepeating, batch.selectedInUse));
       }
     }
+    for(int i=0;i<intervalDayTimeIndices.length; ++i) {
+      int keyIndex = intervalDayTimeIndices[i];
+      int columnIndex = keyExpressions[keyIndex].getOutputColumn();
+      IntervalDayTimeColumnVector columnVector = (IntervalDayTimeColumnVector) batch.cols[columnIndex];
+      if (columnVector.noNulls && !columnVector.isRepeating && !batch.selectedInUse) {
+        assignIntervalDayTimeNoNullsNoRepeatingNoSelection(i, batch.size, columnVector);
+      } else if (columnVector.noNulls && !columnVector.isRepeating && batch.selectedInUse) {
+        assignIntervalDayTimeNoNullsNoRepeatingSelection(i, batch.size, columnVector, batch.selected);
+      } else if (columnVector.noNulls && columnVector.isRepeating) {
+        assignIntervalDayTimeNoNullsRepeating(i, batch.size, columnVector);
+      } else if (!columnVector.noNulls && !columnVector.isRepeating && !batch.selectedInUse) {
+        assignIntervalDayTimeNullsNoRepeatingNoSelection(i, batch.size, columnVector);
+      } else if (!columnVector.noNulls && columnVector.isRepeating) {
+        assignIntervalDayTimeNullsRepeating(i, batch.size, columnVector);
+      } else if (!columnVector.noNulls && !columnVector.isRepeating && batch.selectedInUse) {
+        assignIntervalDayTimeNullsNoRepeatingSelection (i, batch.size, columnVector, batch.selected);
+      } else {
+        throw new HiveException (String.format(
+            "Unimplemented intervalDayTime null/repeat/selected combination %b/%b/%b",
+            columnVector.noNulls, columnVector.isRepeating, batch.selectedInUse));
+      }
+    }
     for(int i=0;i<batch.size;++i) {
       vectorHashKeyWrappers[i].setHashKey();
     }
@@ -596,6 +618,81 @@ public class VectorHashKeyWrapperBatch extends VectorColumnSetInfo {
       vectorHashKeyWrappers[r].assignTimestamp(index, columnVector, r);
     }
   }
+
+  /**
+   * Helper method to assign values from a vector column into the key wrapper.
+   * Optimized for IntervalDayTime type, possible nulls, no repeat values, batch selection vector.
+   */
+  private void assignIntervalDayTimeNullsNoRepeatingSelection(int index, int size,
+      IntervalDayTimeColumnVector columnVector, int[] selected) {
+    for(int i = 0; i < size; ++i) {
+      int row = selected[i];
+      if (!columnVector.isNull[row]) {
+        vectorHashKeyWrappers[i].assignIntervalDayTime(index, columnVector, row);
+      } else {
+        vectorHashKeyWrappers[i].assignNullIntervalDayTime(index);
+      }
+    }
+  }
+
+  /**
+   * Helper method to assign values from a vector column into the key wrapper.
+   * Optimized for IntervalDayTime type, repeat null values.
+   */
+  private void assignIntervalDayTimeNullsRepeating(int index, int size,
+      IntervalDayTimeColumnVector columnVector) {
+    for(int r = 0; r < size; ++r) {
+      vectorHashKeyWrappers[r].assignNullIntervalDayTime(index);
+    }
+  }
+
+  /**
+   * Helper method to assign values from a vector column into the key wrapper.
+   * Optimized for IntervalDayTime type, possible nulls, repeat values.
+   */
+  private void assignIntervalDayTimeNullsNoRepeatingNoSelection(int index, int size,
+      IntervalDayTimeColumnVector columnVector) {
+    for(int r = 0; r < size; ++r) {
+      if (!columnVector.isNull[r]) {
+        vectorHashKeyWrappers[r].assignIntervalDayTime(index, columnVector, r);
+      } else {
+        vectorHashKeyWrappers[r].assignNullIntervalDayTime(index);
+      }
+    }
+  }
+
+  /**
+   * Helper method to assign values from a vector column into the key wrapper.
+   * Optimized for IntervalDayTime type, no nulls, repeat values, no selection vector.
+   */
+  private void assignIntervalDayTimeNoNullsRepeating(int index, int size, IntervalDayTimeColumnVector columnVector) {
+    for(int r = 0; r < size; ++r) {
+      vectorHashKeyWrappers[r].assignIntervalDayTime(index, columnVector, 0);
+    }
+  }
+
+  /**
+   * Helper method to assign values from a vector column into the key wrapper.
+   * Optimized for IntervalDayTime type, no nulls, no repeat values, batch selection vector.
+   */
+  private void assignIntervalDayTimeNoNullsNoRepeatingSelection(int index, int size,
+      IntervalDayTimeColumnVector columnVector, int[] selected) {
+    for(int r = 0; r < size; ++r) {
+      vectorHashKeyWrappers[r].assignIntervalDayTime(index, columnVector, selected[r]);
+    }
+  }
+
+  /**
+   * Helper method to assign values from a vector column into the key wrapper.
+   * Optimized for IntervalDayTime type, no nulls, no repeat values, no selection vector.
+   */
+  private void assignIntervalDayTimeNoNullsNoRepeatingNoSelection(int index, int size,
+      IntervalDayTimeColumnVector columnVector) {
+    for(int r = 0; r < size; ++r) {
+      vectorHashKeyWrappers[r].assignIntervalDayTime(index, columnVector, r);
+    }
+  }
+
   /**
    * Prepares a VectorHashKeyWrapperBatch to work for a specific set of keys.
    * Computes the fast access lookup indices, preallocates all needed internal arrays.
@@ -638,6 +735,7 @@ public class VectorHashKeyWrapperBatch extends VectorColumnSetInfo {
     compiledKeyWrapperBatch.keysFixedSize += model.lengthForObjectArrayOfSize(compiledKeyWrapperBatch.stringIndices.length);
     compiledKeyWrapperBatch.keysFixedSize += model.lengthForObjectArrayOfSize(compiledKeyWrapperBatch.decimalIndices.length);
     compiledKeyWrapperBatch.keysFixedSize += model.lengthForObjectArrayOfSize(compiledKeyWrapperBatch.timestampIndices.length);
+    compiledKeyWrapperBatch.keysFixedSize += model.lengthForObjectArrayOfSize(compiledKeyWrapperBatch.intervalDayTimeIndices.length);
     compiledKeyWrapperBatch.keysFixedSize += model.lengthForIntArrayOfSize(compiledKeyWrapperBatch.longIndices.length) * 2;
     compiledKeyWrapperBatch.keysFixedSize +=
         model.lengthForBooleanArrayOfSize(keyExpressions.length);
@@ -647,7 +745,8 @@ public class VectorHashKeyWrapperBatch extends VectorColumnSetInfo {
 
   public VectorHashKeyWrapper allocateKeyWrapper() {
     return new VectorHashKeyWrapper(longIndices.length, doubleIndices.length,
-        stringIndices.length, decimalIndices.length, timestampIndices.length);
+        stringIndices.length, decimalIndices.length, timestampIndices.length,
+        intervalDayTimeIndices.length);
   }
 
   /**
@@ -679,12 +778,15 @@ public class VectorHashKeyWrapperBatch extends VectorColumnSetInfo {
       return kw.getIsTimestampNull(klh.timestampIndex)? null :
           keyOutputWriter.writeValue(
                 kw.getTimestamp(klh.timestampIndex));
-    }
-    else {
+    } else if (klh.intervalDayTimeIndex >= 0) {
+      return kw.getIsIntervalDayTimeNull(klh.intervalDayTimeIndex)? null :
+        keyOutputWriter.writeValue(
+              kw.getIntervalDayTime(klh.intervalDayTimeIndex));
+    } else {
       throw new HiveException(String.format(
-          "Internal inconsistent KeyLookupHelper at index [%d]:%d %d %d %d %d",
+          "Internal inconsistent KeyLookupHelper at index [%d]:%d %d %d %d %d %d",
           i, klh.longIndex, klh.doubleIndex, klh.stringIndex, klh.decimalIndex,
-          klh.timestampIndex));
+          klh.timestampIndex, klh.intervalDayTimeIndex));
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorSerializeRow.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorSerializeRow.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorSerializeRow.java
index dea38e8..6af3d99 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorSerializeRow.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorSerializeRow.java
@@ -23,7 +23,6 @@ import java.sql.Timestamp;
 import java.util.List;
 
 import org.apache.hadoop.hive.common.type.HiveIntervalDayTime;
-import org.apache.hadoop.hive.common.type.PisaTimestamp;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.serde2.ByteStream.Output;
 import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
@@ -332,11 +331,11 @@ public final class VectorSerializeRow<T extends SerializeWrite> {
 
     @Override
     boolean apply(VectorizedRowBatch batch, int batchIndex) throws IOException {
-      TimestampColumnVector colVector = (TimestampColumnVector) batch.cols[columnIndex];
+      IntervalDayTimeColumnVector colVector = (IntervalDayTimeColumnVector) batch.cols[columnIndex];
 
       if (colVector.isRepeating) {
         if (colVector.noNulls || !colVector.isNull[0]) {
-          hiveIntervalDayTime.set(colVector.asScratchPisaTimestamp(0));
+          hiveIntervalDayTime.set(colVector.asScratchIntervalDayTime(0));
           serializeWrite.writeHiveIntervalDayTime(hiveIntervalDayTime);
           return true;
         } else {
@@ -345,7 +344,7 @@ public final class VectorSerializeRow<T extends SerializeWrite> {
         }
       } else {
         if (colVector.noNulls || !colVector.isNull[batchIndex]) {
-          hiveIntervalDayTime.set(colVector.asScratchPisaTimestamp(batchIndex));
+          hiveIntervalDayTime.set(colVector.asScratchIntervalDayTime(batchIndex));
           serializeWrite.writeHiveIntervalDayTime(hiveIntervalDayTime);
           return true;
         } else {

http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java
index 3f95be2..0552f9d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java
@@ -68,11 +68,13 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.aggregates.gen.VectorUD
 import org.apache.hadoop.hive.ql.exec.vector.expressions.aggregates.gen.VectorUDAFMaxLong;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.aggregates.gen.VectorUDAFMaxString;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.aggregates.gen.VectorUDAFMaxTimestamp;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.aggregates.gen.VectorUDAFMaxIntervalDayTime;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.aggregates.gen.VectorUDAFMinDecimal;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.aggregates.gen.VectorUDAFMinDouble;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.aggregates.gen.VectorUDAFMinLong;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.aggregates.gen.VectorUDAFMinString;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.aggregates.gen.VectorUDAFMinTimestamp;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.aggregates.gen.VectorUDAFMinIntervalDayTime;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.aggregates.gen.VectorUDAFStdPopDecimal;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.aggregates.gen.VectorUDAFStdPopDouble;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.aggregates.gen.VectorUDAFStdPopLong;
@@ -2333,10 +2335,12 @@ public class VectorizationContext {
           case INTERVAL_YEAR_MONTH:
             return ColumnVector.Type.LONG;
 
-          case INTERVAL_DAY_TIME:
           case TIMESTAMP:
             return ColumnVector.Type.TIMESTAMP;
 
+          case INTERVAL_DAY_TIME:
+            return ColumnVector.Type.INTERVAL_DAY_TIME;
+
           case FLOAT:
           case DOUBLE:
             return ColumnVector.Type.DOUBLE;
@@ -2369,19 +2373,20 @@ public class VectorizationContext {
     add(new AggregateDefinition("min",         VectorExpressionDescriptor.ArgumentType.FLOAT_FAMILY,           null,                          VectorUDAFMinDouble.class));
     add(new AggregateDefinition("min",         VectorExpressionDescriptor.ArgumentType.STRING_FAMILY,          null,                          VectorUDAFMinString.class));
     add(new AggregateDefinition("min",         VectorExpressionDescriptor.ArgumentType.DECIMAL,                null,                          VectorUDAFMinDecimal.class));
-    add(new AggregateDefinition("min",         VectorExpressionDescriptor.ArgumentType.TIMESTAMP_INTERVAL_DAY_TIME,     null,                          VectorUDAFMinTimestamp.class));
+    add(new AggregateDefinition("min",         VectorExpressionDescriptor.ArgumentType.TIMESTAMP,              null,                          VectorUDAFMinTimestamp.class));
     add(new AggregateDefinition("max",         VectorExpressionDescriptor.ArgumentType.INT_DATE_INTERVAL_YEAR_MONTH,    null,                          VectorUDAFMaxLong.class));
     add(new AggregateDefinition("max",         VectorExpressionDescriptor.ArgumentType.FLOAT_FAMILY,           null,                          VectorUDAFMaxDouble.class));
     add(new AggregateDefinition("max",         VectorExpressionDescriptor.ArgumentType.STRING_FAMILY,          null,                          VectorUDAFMaxString.class));
     add(new AggregateDefinition("max",         VectorExpressionDescriptor.ArgumentType.DECIMAL,                null,                          VectorUDAFMaxDecimal.class));
-    add(new AggregateDefinition("max",         VectorExpressionDescriptor.ArgumentType.TIMESTAMP_INTERVAL_DAY_TIME,     null,                          VectorUDAFMaxTimestamp.class));
+    add(new AggregateDefinition("max",         VectorExpressionDescriptor.ArgumentType.TIMESTAMP,              null,                          VectorUDAFMaxTimestamp.class));
     add(new AggregateDefinition("count",       VectorExpressionDescriptor.ArgumentType.NONE,                   GroupByDesc.Mode.HASH,         VectorUDAFCountStar.class));
     add(new AggregateDefinition("count",       VectorExpressionDescriptor.ArgumentType.INT_DATE_INTERVAL_YEAR_MONTH,    GroupByDesc.Mode.HASH,         VectorUDAFCount.class));
     add(new AggregateDefinition("count",       VectorExpressionDescriptor.ArgumentType.INT_FAMILY,             GroupByDesc.Mode.MERGEPARTIAL, VectorUDAFCountMerge.class));
     add(new AggregateDefinition("count",       VectorExpressionDescriptor.ArgumentType.FLOAT_FAMILY,           GroupByDesc.Mode.HASH,         VectorUDAFCount.class));
     add(new AggregateDefinition("count",       VectorExpressionDescriptor.ArgumentType.STRING_FAMILY,          GroupByDesc.Mode.HASH,         VectorUDAFCount.class));
     add(new AggregateDefinition("count",       VectorExpressionDescriptor.ArgumentType.DECIMAL,                GroupByDesc.Mode.HASH,         VectorUDAFCount.class));
-    add(new AggregateDefinition("count",       VectorExpressionDescriptor.ArgumentType.TIMESTAMP_INTERVAL_DAY_TIME,     GroupByDesc.Mode.HASH,         VectorUDAFCount.class));
+    add(new AggregateDefinition("count",       VectorExpressionDescriptor.ArgumentType.TIMESTAMP,              GroupByDesc.Mode.HASH,         VectorUDAFCount.class));
+    add(new AggregateDefinition("count",       VectorExpressionDescriptor.ArgumentType.INTERVAL_DAY_TIME,      GroupByDesc.Mode.HASH,         VectorUDAFCount.class));
     add(new AggregateDefinition("sum",         VectorExpressionDescriptor.ArgumentType.INT_FAMILY,             null,                          VectorUDAFSumLong.class));
     add(new AggregateDefinition("sum",         VectorExpressionDescriptor.ArgumentType.FLOAT_FAMILY,           null,                          VectorUDAFSumDouble.class));
     add(new AggregateDefinition("sum",         VectorExpressionDescriptor.ArgumentType.DECIMAL,                null,                          VectorUDAFSumDecimal.class));

http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedBatchUtil.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedBatchUtil.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedBatchUtil.java
index a68d0cc..be04da8 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedBatchUtil.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedBatchUtil.java
@@ -144,9 +144,10 @@ public class VectorizedBatchUtil {
           case DATE:
           case INTERVAL_YEAR_MONTH:
             return new LongColumnVector(VectorizedRowBatch.DEFAULT_SIZE);
-          case INTERVAL_DAY_TIME:
           case TIMESTAMP:
             return new TimestampColumnVector(VectorizedRowBatch.DEFAULT_SIZE);
+          case INTERVAL_DAY_TIME:
+            return new IntervalDayTimeColumnVector(VectorizedRowBatch.DEFAULT_SIZE);
           case FLOAT:
           case DOUBLE:
             return new DoubleColumnVector(VectorizedRowBatch.DEFAULT_SIZE);
@@ -417,14 +418,14 @@ public class VectorizedBatchUtil {
     }
       break;
     case INTERVAL_DAY_TIME: {
-      LongColumnVector lcv = (LongColumnVector) batch.cols[offset + colIndex];
+      IntervalDayTimeColumnVector icv = (IntervalDayTimeColumnVector) batch.cols[offset + colIndex];
       if (writableCol != null) {
-        HiveIntervalDayTime i = ((HiveIntervalDayTimeWritable) writableCol).getHiveIntervalDayTime();
-        lcv.vector[rowIndex] = DateUtils.getIntervalDayTimeTotalNanos(i);
-        lcv.isNull[rowIndex] = false;
+        HiveIntervalDayTime idt = ((HiveIntervalDayTimeWritable) writableCol).getHiveIntervalDayTime();
+        icv.set(rowIndex, idt);
+        icv.isNull[rowIndex] = false;
       } else {
-        lcv.vector[rowIndex] = 1;
-        setNullColIsNullValue(lcv, rowIndex);
+        icv.setNullValue(rowIndex);
+        setNullColIsNullValue(icv, rowIndex);
       }
     }
       break;
@@ -585,6 +586,8 @@ public class VectorizedBatchUtil {
           decColVector.scale);
     } else if (source instanceof TimestampColumnVector) {
       return new TimestampColumnVector(((TimestampColumnVector) source).getLength());
+    } else if (source instanceof IntervalDayTimeColumnVector) {
+      return new IntervalDayTimeColumnVector(((IntervalDayTimeColumnVector) source).getLength());
     } else if (source instanceof ListColumnVector) {
       ListColumnVector src = (ListColumnVector) source;
       ColumnVector child = cloneColumnVector(src.child);
@@ -688,6 +691,9 @@ public class VectorizedBatchUtil {
             Timestamp timestamp = new Timestamp(0);
             ((TimestampColumnVector) colVector).timestampUpdate(timestamp, index);
             sb.append(timestamp.toString());
+          } else if (colVector instanceof IntervalDayTimeColumnVector) {
+            HiveIntervalDayTime intervalDayTime = ((IntervalDayTimeColumnVector) colVector).asScratchIntervalDayTime(index);
+            sb.append(intervalDayTime.toString());
           } else {
             sb.append("Unknown");
           }

http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedRowBatchCtx.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedRowBatchCtx.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedRowBatchCtx.java
index 7e79e1e..0724191 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedRowBatchCtx.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedRowBatchCtx.java
@@ -400,14 +400,14 @@ public class VectorizedRowBatchCtx {
         }
 
         case INTERVAL_DAY_TIME: {
-          TimestampColumnVector tcv = (TimestampColumnVector) batch.cols[colIndex];
+          IntervalDayTimeColumnVector icv = (IntervalDayTimeColumnVector) batch.cols[colIndex];
           if (value == null) {
-            tcv.noNulls = false;
-            tcv.isNull[0] = true;
-            tcv.isRepeating = true;
+            icv.noNulls = false;
+            icv.isNull[0] = true;
+            icv.isRepeating = true;
           } else {
-            tcv.fill(((HiveIntervalDayTime) value).pisaTimestampUpdate(tcv.useScratchPisaTimestamp()));
-            tcv.isNull[0] = false;
+            icv.fill(((HiveIntervalDayTime) value));
+            icv.isNull[0] = false;
           }
         }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToTimestamp.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToTimestamp.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToTimestamp.java
index 2b0068d..6225ade 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToTimestamp.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToTimestamp.java
@@ -44,7 +44,6 @@ public class CastDecimalToTimestamp extends FuncDecimalToTimestamp {
 
   @Override
   protected void func(TimestampColumnVector outV, DecimalColumnVector inV,  int i) {
-    Timestamp timestamp = TimestampWritable.decimalToTimestamp(inV.vector[i].getHiveDecimal());
-    outV.set(i, timestamp);
+    outV.set(i, TimestampWritable.decimalToTimestamp(inV.vector[i].getHiveDecimal()));
   }
 }