You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by mm...@apache.org on 2018/02/21 08:54:09 UTC

hive git commit: HIVE-18744: Vectorization: VectorHashKeyWrapperBatch doesn't assign Timestamp values by value (Matt McCline, reviewed by Owen O'Malley, Teddy Choi, and Deepak Jaiswal)

Repository: hive
Updated Branches:
  refs/heads/master 8975924ec -> 9bfdcd761


HIVE-18744: Vectorization: VectorHashKeyWrapperBatch doesn't assign Timestamp values by value (Matt McCline, reviewed by Owen O'Malley, Teddy Choi, and Deepak Jaiswal)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/9bfdcd76
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/9bfdcd76
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/9bfdcd76

Branch: refs/heads/master
Commit: 9bfdcd7618a2a6197112beb64e4a1c7264440f21
Parents: 8975924
Author: Matt McCline <mm...@hortonworks.com>
Authored: Wed Feb 21 00:49:43 2018 -0800
Committer: Matt McCline <mm...@hortonworks.com>
Committed: Wed Feb 21 00:49:43 2018 -0800

----------------------------------------------------------------------
 .../ql/exec/vector/VectorColumnSetInfo.java     |   8 +-
 .../ql/exec/vector/VectorGroupKeyHelper.java    |   3 +-
 .../ql/exec/vector/VectorHashKeyWrapper.java    | 170 +++++++-
 .../exec/vector/VectorHashKeyWrapperBatch.java  | 390 +++++++++----------
 .../vector/TestVectorHashKeyWrapperBatch.java   |  86 ++++
 .../clientpositive/llap/vectorization_15.q.out  |   6 +-
 .../parquet_vectorization_15.q.out              |   6 +-
 .../spark/parquet_vectorization_15.q.out        |   6 +-
 .../clientpositive/spark/vectorization_15.q.out |   6 +-
 .../clientpositive/vectorization_15.q.out       |   6 +-
 .../hive/ql/exec/vector/VectorizedRowBatch.java |  19 +-
 11 files changed, 472 insertions(+), 234 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/9bfdcd76/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 3bfe4f9..7758ac4 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
@@ -21,7 +21,9 @@ package org.apache.hadoop.hive.ql.exec.vector;
 import java.util.Arrays;
 
 import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.ColumnVector.Type;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 
 /**
  * Class to keep information on a set of typed vector columns.  Used by
@@ -75,6 +77,7 @@ public class VectorColumnSetInfo {
   // Given the keyIndex these arrays return:
   //   The ColumnVector.Type,
   //   The type specific index into longIndices, doubleIndices, etc...
+  protected TypeInfo[] typeInfos;
   protected ColumnVector.Type[] columnVectorTypes;
   protected int[] columnTypeSpecificIndices;
 
@@ -96,13 +99,15 @@ public class VectorColumnSetInfo {
     intervalDayTimeIndices = new int[this.keyCount];
     addIntervalDayTimeIndex = 0;
 
+    typeInfos = new TypeInfo[this.keyCount];
     columnVectorTypes = new ColumnVector.Type[this.keyCount];
     columnTypeSpecificIndices = new int[this.keyCount];
   }
 
 
-  protected void addKey(ColumnVector.Type columnVectorType) throws HiveException {
+  protected void addKey(TypeInfo typeInfo) throws HiveException {
 
+    Type columnVectorType = VectorizationContext.getColumnVectorTypeFromTypeInfo(typeInfo);
     switch (columnVectorType) {
     case LONG:
     case DECIMAL_64:
@@ -133,6 +138,7 @@ public class VectorColumnSetInfo {
       throw new HiveException("Unexpected column vector type " + columnVectorType);
     }
 
+    typeInfos[addKeyIndex] = typeInfo;
     columnVectorTypes[addKeyIndex] = columnVectorType;
     addKeyIndex++;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/9bfdcd76/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 6ae6727..82dc4a7 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
@@ -49,8 +49,7 @@ public class VectorGroupKeyHelper extends VectorColumnSetInfo {
       VectorExpression keyExpression = keyExpressions[i];
 
       TypeInfo typeInfo = keyExpression.getOutputTypeInfo();
-      Type columnVectorType = VectorizationContext.getColumnVectorTypeFromTypeInfo(typeInfo);
-      addKey(columnVectorType);
+      addKey(typeInfo);
 
       // The output of the key expression is the input column.
       final int inputColumnNum = keyExpression.getOutputColumnNum();

http://git-wip-us.apache.org/repos/asf/hive/blob/9bfdcd76/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 eb870a7..1f46f2c 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
@@ -20,6 +20,7 @@ package org.apache.hadoop.hive.ql.exec.vector;
 
 import org.apache.hive.common.util.Murmur3;
 
+import java.sql.Date;
 import java.sql.Timestamp;
 import java.util.Arrays;
 
@@ -29,8 +30,11 @@ 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.DateWritable;
 import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 
 import com.google.common.base.Preconditions;
 
@@ -350,7 +354,10 @@ public class VectorHashKeyWrapper extends KeyWrapper {
   }
 
   public void assignTimestamp(int index, Timestamp value) {
-    timestampValues[index] = value;
+    // Do not assign the input value object to the timestampValues array element.
+    // Always copy value using set* methods.
+    timestampValues[index].setTime(value.getTime());
+    timestampValues[index].setNanos(value.getNanos());
   }
 
   public void assignTimestamp(int index, TimestampColumnVector colVector, int elementNum) {
@@ -359,7 +366,9 @@ public class VectorHashKeyWrapper extends KeyWrapper {
 
   public void assignNullTimestamp(int keyIndex, int index) {
     isNull[keyIndex] = true;
-    timestampValues[index] = ZERO_TIMESTAMP; // assign 0 to simplify hashcode
+    // assign 0 to simplify hashcode
+    timestampValues[index].setTime(ZERO_TIMESTAMP.getTime());
+    timestampValues[index].setNanos(ZERO_TIMESTAMP.getNanos());
   }
 
   public void assignIntervalDayTime(int index, HiveIntervalDayTime value) {
@@ -372,7 +381,162 @@ public class VectorHashKeyWrapper extends KeyWrapper {
 
   public void assignNullIntervalDayTime(int keyIndex, int index) {
     isNull[keyIndex] = true;
-    intervalDayTimeValues[index] = ZERO_INTERVALDAYTIME; // assign 0 to simplify hashcode
+    intervalDayTimeValues[index].set(ZERO_INTERVALDAYTIME); // assign 0 to simplify hashcode
+  }
+
+  /*
+   * This method is mainly intended for debug display purposes.
+   */
+  public String stringifyKeys(VectorColumnSetInfo columnSetInfo)
+  {
+    StringBuilder sb = new StringBuilder();
+    boolean isFirstKey = true;
+
+    if (longValues.length > 0) {
+      isFirstKey = false;
+      sb.append("longs ");
+      boolean isFirstValue = true;
+      for (int i = 0; i < columnSetInfo.longIndices.length; i++) {
+        if (isFirstValue) {
+          isFirstValue = false;
+        } else {
+          sb.append(", ");
+        }
+        int keyIndex = columnSetInfo.longIndices[i];
+        if (isNull[keyIndex]) {
+          sb.append("null");
+        } else {
+          sb.append(longValues[i]);
+          PrimitiveTypeInfo primitiveTypeInfo = (PrimitiveTypeInfo) columnSetInfo.typeInfos[keyIndex];
+          // FUTURE: Add INTERVAL_YEAR_MONTH, etc, as desired.
+          switch (primitiveTypeInfo.getPrimitiveCategory()) {
+          case DATE:
+            {
+              Date dt = new Date(0);
+              dt.setTime(DateWritable.daysToMillis((int) longValues[i]));
+              sb.append(" date ");
+              sb.append(dt.toString());
+            }
+            break;
+          default:
+            // Add nothing more.
+            break;
+          }
+        }
+      }
+    }
+    if (doubleValues.length > 0) {
+      if (isFirstKey) {
+        isFirstKey = false;
+      } else {
+        sb.append(", ");
+      }
+      sb.append("doubles ");
+      boolean isFirstValue = true;
+      for (int i = 0; i < columnSetInfo.doubleIndices.length; i++) {
+        if (isFirstValue) {
+          isFirstValue = false;
+        } else {
+          sb.append(", ");
+        }
+        int keyIndex = columnSetInfo.doubleIndices[i];
+        if (isNull[keyIndex]) {
+          sb.append("null");
+        } else {
+          sb.append(doubleValues[i]);
+        }
+      }
+    }
+    if (byteValues.length > 0) {
+      if (isFirstKey) {
+        isFirstKey = false;
+      } else {
+        sb.append(", ");
+      }
+      sb.append("byte lengths ");
+      boolean isFirstValue = true;
+      for (int i = 0; i < columnSetInfo.stringIndices.length; i++) {
+        if (isFirstValue) {
+          isFirstValue = false;
+        } else {
+          sb.append(", ");
+        }
+        int keyIndex = columnSetInfo.stringIndices[i];
+        if (isNull[keyIndex]) {
+          sb.append("null");
+        } else {
+          sb.append(byteLengths[i]);
+        }
+      }
+    }
+    if (decimalValues.length > 0) {
+      if (isFirstKey) {
+        isFirstKey = true;
+      } else {
+        sb.append(", ");
+      }
+      sb.append("decimals ");
+      boolean isFirstValue = true;
+      for (int i = 0; i < columnSetInfo.decimalIndices.length; i++) {
+        if (isFirstValue) {
+          isFirstValue = false;
+        } else {
+          sb.append(", ");
+        }
+        int keyIndex = columnSetInfo.decimalIndices[i];
+        if (isNull[keyIndex]) {
+          sb.append("null");
+        } else {
+          sb.append(decimalValues[i]);
+        }
+      }
+    }
+    if (timestampValues.length > 0) {
+      if (isFirstKey) {
+        isFirstKey = false;
+      } else {
+        sb.append(", ");
+      }
+      sb.append("timestamps ");
+      boolean isFirstValue = true;
+      for (int i = 0; i < columnSetInfo.timestampIndices.length; i++) {
+        if (isFirstValue) {
+          isFirstValue = false;
+        } else {
+          sb.append(", ");
+        }
+        int keyIndex = columnSetInfo.timestampIndices[i];
+        if (isNull[keyIndex]) {
+          sb.append("null");
+        } else {
+          sb.append(timestampValues[i]);
+        }
+      }
+    }
+    if (intervalDayTimeValues.length > 0) {
+      if (isFirstKey) {
+        isFirstKey = false;
+      } else {
+        sb.append(", ");
+      }
+      sb.append("interval day times ");
+      boolean isFirstValue = true;
+      for (int i = 0; i < columnSetInfo.intervalDayTimeIndices.length; i++) {
+        if (isFirstValue) {
+          isFirstValue = false;
+        } else {
+          sb.append(", ");
+        }
+        int keyIndex = columnSetInfo.intervalDayTimeIndices[i];
+        if (isNull[keyIndex]) {
+          sb.append("null");
+        } else {
+          sb.append(intervalDayTimeValues[i]);
+        }
+      }
+    }
+
+    return sb.toString();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/9bfdcd76/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 2b401ac..0e6f8c5 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
@@ -23,6 +23,7 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpressionWriter;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.util.JavaDataModel;
 import org.apache.hadoop.hive.ql.exec.vector.ColumnVector.Type;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 
 /**
  * Class for handling vectorized hash map key wrappers. It evaluates the key columns in a
@@ -106,133 +107,48 @@ public class VectorHashKeyWrapperBatch extends VectorColumnSetInfo {
       keyIndex = longIndices[i];
       columnIndex = keyExpressions[keyIndex].getOutputColumnNum();
       LongColumnVector columnVector = (LongColumnVector) batch.cols[columnIndex];
-      if (columnVector.noNulls && !columnVector.isRepeating && !batch.selectedInUse) {
-        assignLongNoNullsNoRepeatingNoSelection(i, batch.size, columnVector);
-      } else if (columnVector.noNulls && !columnVector.isRepeating && batch.selectedInUse) {
-        assignLongNoNullsNoRepeatingSelection(i, batch.size, columnVector, batch.selected);
-      } else if ((columnVector.noNulls || !columnVector.isNull[0]) && columnVector.isRepeating) {
-        assignLongNoNullsRepeating(i, batch.size, columnVector);
-      } else if (!columnVector.noNulls && !columnVector.isRepeating && !batch.selectedInUse) {
-        assignLongNullsNoRepeatingNoSelection(keyIndex, i, batch.size, columnVector);
-      } else if (!columnVector.noNulls && columnVector.isRepeating) {
-        assignLongNullsRepeating(keyIndex, i, batch.size, columnVector);
-      } else if (!columnVector.noNulls && !columnVector.isRepeating && batch.selectedInUse) {
-        assignLongNullsNoRepeatingSelection (keyIndex, i, batch.size, columnVector, batch.selected);
-      } else {
-        throw new HiveException (String.format(
-            "Unimplemented Long null/repeat/selected combination %b/%b/%b",
-            columnVector.noNulls, columnVector.isRepeating, batch.selectedInUse));
-      }
+
+      evaluateLongColumnVector(batch, columnVector, keyIndex, i);
     }
+
     for(int i=0;i<doubleIndices.length; ++i) {
       keyIndex = doubleIndices[i];
       columnIndex = keyExpressions[keyIndex].getOutputColumnNum();
       DoubleColumnVector columnVector = (DoubleColumnVector) batch.cols[columnIndex];
-      if (columnVector.noNulls && !columnVector.isRepeating && !batch.selectedInUse) {
-        assignDoubleNoNullsNoRepeatingNoSelection(i, batch.size, columnVector);
-      } else if (columnVector.noNulls && !columnVector.isRepeating && batch.selectedInUse) {
-        assignDoubleNoNullsNoRepeatingSelection(i, batch.size, columnVector, batch.selected);
-      } else if ((columnVector.noNulls || !columnVector.isNull[0]) && columnVector.isRepeating) {
-        assignDoubleNoNullsRepeating(i, batch.size, columnVector);
-      } else if (!columnVector.noNulls && !columnVector.isRepeating && !batch.selectedInUse) {
-        assignDoubleNullsNoRepeatingNoSelection(keyIndex, i, batch.size, columnVector);
-      } else if (!columnVector.noNulls && columnVector.isRepeating) {
-        assignDoubleNullsRepeating(keyIndex, i, batch.size, columnVector);
-      } else if (!columnVector.noNulls && !columnVector.isRepeating && batch.selectedInUse) {
-        assignDoubleNullsNoRepeatingSelection(keyIndex, i, batch.size, columnVector, batch.selected);
-      } else {
-        throw new HiveException (String.format(
-            "Unimplemented Double null/repeat/selected combination %b/%b/%b",
-            columnVector.noNulls, columnVector.isRepeating, batch.selectedInUse));
-      }
+
+      evaluateDoubleColumnVector(batch, columnVector, keyIndex, i);
     }
+
     for(int i=0;i<stringIndices.length; ++i) {
       keyIndex = stringIndices[i];
       columnIndex = keyExpressions[keyIndex].getOutputColumnNum();
       BytesColumnVector columnVector = (BytesColumnVector) batch.cols[columnIndex];
-      if (columnVector.noNulls && !columnVector.isRepeating && !batch.selectedInUse) {
-        assignStringNoNullsNoRepeatingNoSelection(i, batch.size, columnVector);
-      } else if (columnVector.noNulls && !columnVector.isRepeating && batch.selectedInUse) {
-        assignStringNoNullsNoRepeatingSelection(i, batch.size, columnVector, batch.selected);
-      } else if ((columnVector.noNulls || !columnVector.isNull[0]) && columnVector.isRepeating) {
-        assignStringNoNullsRepeating(i, batch.size, columnVector);
-      } else if (!columnVector.noNulls && !columnVector.isRepeating && !batch.selectedInUse) {
-        assignStringNullsNoRepeatingNoSelection(keyIndex, i, batch.size, columnVector);
-      } else if (!columnVector.noNulls && columnVector.isRepeating) {
-        assignStringNullsRepeating(keyIndex, i, batch.size, columnVector);
-      } else if (!columnVector.noNulls && !columnVector.isRepeating && batch.selectedInUse) {
-        assignStringNullsNoRepeatingSelection(keyIndex, i, batch.size, columnVector, batch.selected);
-      } else {
-        throw new HiveException (String.format(
-            "Unimplemented String null/repeat/selected combination %b/%b/%b",
-            columnVector.noNulls, columnVector.isRepeating, batch.selectedInUse));
-      }
+
+      evaluateStringColumnVector(batch, columnVector, keyIndex, i);
     }
+
     for(int i=0;i<decimalIndices.length; ++i) {
       keyIndex = decimalIndices[i];
       columnIndex = keyExpressions[keyIndex].getOutputColumnNum();
       DecimalColumnVector columnVector = (DecimalColumnVector) batch.cols[columnIndex];
-      if (columnVector.noNulls && !columnVector.isRepeating && !batch.selectedInUse) {
-        assignDecimalNoNullsNoRepeatingNoSelection(i, batch.size, columnVector);
-      } else if (columnVector.noNulls && !columnVector.isRepeating && batch.selectedInUse) {
-        assignDecimalNoNullsNoRepeatingSelection(i, batch.size, columnVector, batch.selected);
-      } else if ((columnVector.noNulls || !columnVector.isNull[0]) && columnVector.isRepeating) {
-        assignDecimalNoNullsRepeating(i, batch.size, columnVector);
-      } else if (!columnVector.noNulls && !columnVector.isRepeating && !batch.selectedInUse) {
-        assignDecimalNullsNoRepeatingNoSelection(keyIndex, i, batch.size, columnVector);
-      } else if (!columnVector.noNulls && columnVector.isRepeating) {
-        assignDecimalNullsRepeating(keyIndex, i, batch.size, columnVector);
-      } else if (!columnVector.noNulls && !columnVector.isRepeating && batch.selectedInUse) {
-        assignDecimalNullsNoRepeatingSelection(keyIndex, i, batch.size, columnVector, batch.selected);
-      } else {
-        throw new HiveException (String.format(
-            "Unimplemented Decimal null/repeat/selected combination %b/%b/%b",
-            columnVector.noNulls, columnVector.isRepeating, batch.selectedInUse));
-      }
+
+      evaluateDecimalColumnVector(batch, columnVector, keyIndex, i);
     }
+
     for(int i=0;i<timestampIndices.length; ++i) {
       keyIndex = timestampIndices[i];
       columnIndex = keyExpressions[keyIndex].getOutputColumnNum();
       TimestampColumnVector columnVector = (TimestampColumnVector) batch.cols[columnIndex];
-      if (columnVector.noNulls && !columnVector.isRepeating && !batch.selectedInUse) {
-        assignTimestampNoNullsNoRepeatingNoSelection(i, batch.size, columnVector);
-      } else if (columnVector.noNulls && !columnVector.isRepeating && batch.selectedInUse) {
-        assignTimestampNoNullsNoRepeatingSelection(i, batch.size, columnVector, batch.selected);
-      } else if ((columnVector.noNulls || !columnVector.isNull[0]) && columnVector.isRepeating) {
-        assignTimestampNoNullsRepeating(i, batch.size, columnVector);
-      } else if (!columnVector.noNulls && !columnVector.isRepeating && !batch.selectedInUse) {
-        assignTimestampNullsNoRepeatingNoSelection(keyIndex, i, batch.size, columnVector);
-      } else if (!columnVector.noNulls && columnVector.isRepeating) {
-        assignTimestampNullsRepeating(keyIndex, i, batch.size, columnVector);
-      } else if (!columnVector.noNulls && !columnVector.isRepeating && batch.selectedInUse) {
-        assignTimestampNullsNoRepeatingSelection(keyIndex, i, batch.size, columnVector, batch.selected);
-      } else {
-        throw new HiveException (String.format(
-            "Unimplemented timestamp null/repeat/selected combination %b/%b/%b",
-            columnVector.noNulls, columnVector.isRepeating, batch.selectedInUse));
-      }
+
+      evaluateTimestampColumnVector(batch, columnVector, keyIndex, i);
     }
+
     for(int i=0;i<intervalDayTimeIndices.length; ++i) {
       keyIndex = intervalDayTimeIndices[i];
       columnIndex = keyExpressions[keyIndex].getOutputColumnNum();
       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.isNull[0]) && columnVector.isRepeating) {
-        assignIntervalDayTimeNoNullsRepeating(i, batch.size, columnVector);
-      } else if (!columnVector.noNulls && !columnVector.isRepeating && !batch.selectedInUse) {
-        assignIntervalDayTimeNullsNoRepeatingNoSelection(keyIndex, i, batch.size, columnVector);
-      } else if (!columnVector.noNulls && columnVector.isRepeating) {
-        assignIntervalDayTimeNullsRepeating(keyIndex, i, batch.size, columnVector);
-      } else if (!columnVector.noNulls && !columnVector.isRepeating && batch.selectedInUse) {
-        assignIntervalDayTimeNullsNoRepeatingSelection(keyIndex, 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));
-      }
+
+      evaluateIntervalDayTimeColumnVector(batch, columnVector, keyIndex, i);
     }
     for(int i=0;i<batch.size;++i) {
       vectorHashKeyWrappers[i].setHashKey();
@@ -258,24 +174,10 @@ public class VectorHashKeyWrapperBatch extends VectorColumnSetInfo {
       }
       columnIndex = keyExpressions[keyIndex].getOutputColumnNum();
       LongColumnVector columnVector = (LongColumnVector) batch.cols[columnIndex];
-      if (columnVector.noNulls && !columnVector.isRepeating && !batch.selectedInUse) {
-        assignLongNoNullsNoRepeatingNoSelection(i, batch.size, columnVector);
-      } else if (columnVector.noNulls && !columnVector.isRepeating && batch.selectedInUse) {
-        assignLongNoNullsNoRepeatingSelection(i, batch.size, columnVector, batch.selected);
-      } else if ((columnVector.noNulls || !columnVector.isNull[0]) && columnVector.isRepeating) {
-        assignLongNoNullsRepeating(i, batch.size, columnVector);
-      } else if (!columnVector.noNulls && !columnVector.isRepeating && !batch.selectedInUse) {
-        assignLongNullsNoRepeatingNoSelection(keyIndex, i, batch.size, columnVector);
-      } else if (!columnVector.noNulls && columnVector.isRepeating) {
-        assignLongNullsRepeating(keyIndex, i, batch.size, columnVector);
-      } else if (!columnVector.noNulls && !columnVector.isRepeating && batch.selectedInUse) {
-        assignLongNullsNoRepeatingSelection(keyIndex, i, batch.size, columnVector, batch.selected);
-      } else {
-        throw new HiveException (String.format(
-            "Unimplemented Long null/repeat/selected combination %b/%b/%b",
-            columnVector.noNulls, columnVector.isRepeating, batch.selectedInUse));
-      }
+
+      evaluateLongColumnVector(batch, columnVector, keyIndex, i);
     }
+
     for(int i=0;i<doubleIndices.length; ++i) {
       keyIndex = doubleIndices[i];
       if (groupingSetsOverrideIsNulls[keyIndex]) {
@@ -287,24 +189,10 @@ public class VectorHashKeyWrapperBatch extends VectorColumnSetInfo {
       }
       columnIndex = keyExpressions[keyIndex].getOutputColumnNum();
       DoubleColumnVector columnVector = (DoubleColumnVector) batch.cols[columnIndex];
-      if (columnVector.noNulls && !columnVector.isRepeating && !batch.selectedInUse) {
-        assignDoubleNoNullsNoRepeatingNoSelection(i, batch.size, columnVector);
-      } else if (columnVector.noNulls && !columnVector.isRepeating && batch.selectedInUse) {
-        assignDoubleNoNullsNoRepeatingSelection(i, batch.size, columnVector, batch.selected);
-      } else if ((columnVector.noNulls || !columnVector.isNull[0]) && columnVector.isRepeating) {
-        assignDoubleNoNullsRepeating(i, batch.size, columnVector);
-      } else if (!columnVector.noNulls && !columnVector.isRepeating && !batch.selectedInUse) {
-        assignDoubleNullsNoRepeatingNoSelection(keyIndex, i, batch.size, columnVector);
-      } else if (!columnVector.noNulls && columnVector.isRepeating) {
-        assignDoubleNullsRepeating(keyIndex, i, batch.size, columnVector);
-      } else if (!columnVector.noNulls && !columnVector.isRepeating && batch.selectedInUse) {
-        assignDoubleNullsNoRepeatingSelection(keyIndex, i, batch.size, columnVector, batch.selected);
-      } else {
-        throw new HiveException (String.format(
-            "Unimplemented Double null/repeat/selected combination %b/%b/%b",
-            columnVector.noNulls, columnVector.isRepeating, batch.selectedInUse));
-      }
+
+      evaluateDoubleColumnVector(batch, columnVector, keyIndex, i);
     }
+
     for(int i=0;i<stringIndices.length; ++i) {
       keyIndex = stringIndices[i];
       if (groupingSetsOverrideIsNulls[keyIndex]) {
@@ -316,24 +204,10 @@ public class VectorHashKeyWrapperBatch extends VectorColumnSetInfo {
       }
       columnIndex = keyExpressions[keyIndex].getOutputColumnNum();
       BytesColumnVector columnVector = (BytesColumnVector) batch.cols[columnIndex];
-      if (columnVector.noNulls && !columnVector.isRepeating && !batch.selectedInUse) {
-        assignStringNoNullsNoRepeatingNoSelection(i, batch.size, columnVector);
-      } else if (columnVector.noNulls && !columnVector.isRepeating && batch.selectedInUse) {
-        assignStringNoNullsNoRepeatingSelection(i, batch.size, columnVector, batch.selected);
-      } else if ((columnVector.noNulls || !columnVector.isNull[0]) && columnVector.isRepeating) {
-        assignStringNoNullsRepeating(i, batch.size, columnVector);
-      } else if (!columnVector.noNulls && !columnVector.isRepeating && !batch.selectedInUse) {
-        assignStringNullsNoRepeatingNoSelection(keyIndex, i, batch.size, columnVector);
-      } else if (!columnVector.noNulls && columnVector.isRepeating) {
-        assignStringNullsRepeating(keyIndex, i, batch.size, columnVector);
-      } else if (!columnVector.noNulls && !columnVector.isRepeating && batch.selectedInUse) {
-        assignStringNullsNoRepeatingSelection(keyIndex, i, batch.size, columnVector, batch.selected);
-      } else {
-        throw new HiveException (String.format(
-            "Unimplemented String null/repeat/selected combination %b/%b/%b",
-            columnVector.noNulls, columnVector.isRepeating, batch.selectedInUse));
-      }
+
+      evaluateStringColumnVector(batch, columnVector, keyIndex, i);
     }
+
     for(int i=0;i<decimalIndices.length; ++i) {
       keyIndex = decimalIndices[i];
       if (groupingSetsOverrideIsNulls[keyIndex]) {
@@ -345,24 +219,10 @@ public class VectorHashKeyWrapperBatch extends VectorColumnSetInfo {
       }
       columnIndex = keyExpressions[keyIndex].getOutputColumnNum();
       DecimalColumnVector columnVector = (DecimalColumnVector) batch.cols[columnIndex];
-      if (columnVector.noNulls && !columnVector.isRepeating && !batch.selectedInUse) {
-        assignDecimalNoNullsNoRepeatingNoSelection(i, batch.size, columnVector);
-      } else if (columnVector.noNulls && !columnVector.isRepeating && batch.selectedInUse) {
-        assignDecimalNoNullsNoRepeatingSelection(i, batch.size, columnVector, batch.selected);
-      } else if ((columnVector.noNulls || !columnVector.isNull[0]) && columnVector.isRepeating) {
-        assignDecimalNoNullsRepeating(i, batch.size, columnVector);
-      } else if (!columnVector.noNulls && !columnVector.isRepeating && !batch.selectedInUse) {
-        assignDecimalNullsNoRepeatingNoSelection(keyIndex, i, batch.size, columnVector);
-      } else if (!columnVector.noNulls && columnVector.isRepeating) {
-        assignDecimalNullsRepeating(keyIndex, i, batch.size, columnVector);
-      } else if (!columnVector.noNulls && !columnVector.isRepeating && batch.selectedInUse) {
-        assignDecimalNullsNoRepeatingSelection(keyIndex, i, batch.size, columnVector, batch.selected);
-      } else {
-        throw new HiveException (String.format(
-            "Unimplemented Decimal null/repeat/selected combination %b/%b/%b",
-            columnVector.noNulls, columnVector.isRepeating, batch.selectedInUse));
-      }
+
+      evaluateDecimalColumnVector(batch, columnVector, keyIndex, i);
     }
+
     for(int i=0;i<timestampIndices.length; ++i) {
       keyIndex = timestampIndices[i];
       if (groupingSetsOverrideIsNulls[keyIndex]) {
@@ -374,24 +234,10 @@ public class VectorHashKeyWrapperBatch extends VectorColumnSetInfo {
       }
       columnIndex = keyExpressions[keyIndex].getOutputColumnNum();
       TimestampColumnVector columnVector = (TimestampColumnVector) batch.cols[columnIndex];
-      if (columnVector.noNulls && !columnVector.isRepeating && !batch.selectedInUse) {
-        assignTimestampNoNullsNoRepeatingNoSelection(i, batch.size, columnVector);
-      } else if (columnVector.noNulls && !columnVector.isRepeating && batch.selectedInUse) {
-        assignTimestampNoNullsNoRepeatingSelection(i, batch.size, columnVector, batch.selected);
-      } else if ((columnVector.noNulls || !columnVector.isNull[0]) && columnVector.isRepeating) {
-        assignTimestampNoNullsRepeating(i, batch.size, columnVector);
-      } else if (!columnVector.noNulls && !columnVector.isRepeating && !batch.selectedInUse) {
-        assignTimestampNullsNoRepeatingNoSelection(keyIndex, i, batch.size, columnVector);
-      } else if (!columnVector.noNulls && columnVector.isRepeating) {
-        assignTimestampNullsRepeating(keyIndex, i, batch.size, columnVector);
-      } else if (!columnVector.noNulls && !columnVector.isRepeating && batch.selectedInUse) {
-        assignTimestampNullsNoRepeatingSelection(keyIndex, i, batch.size, columnVector, batch.selected);
-      } else {
-        throw new HiveException (String.format(
-            "Unimplemented timestamp null/repeat/selected combination %b/%b/%b",
-            columnVector.noNulls, columnVector.isRepeating, batch.selectedInUse));
-      }
+
+      evaluateTimestampColumnVector(batch, columnVector, keyIndex, i);
     }
+
     for(int i=0;i<intervalDayTimeIndices.length; ++i) {
       keyIndex = intervalDayTimeIndices[i];
       if (groupingSetsOverrideIsNulls[keyIndex]) {
@@ -403,29 +249,153 @@ public class VectorHashKeyWrapperBatch extends VectorColumnSetInfo {
       }
       columnIndex = keyExpressions[keyIndex].getOutputColumnNum();
       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.isNull[0]) && columnVector.isRepeating) {
-        assignIntervalDayTimeNoNullsRepeating(i, batch.size, columnVector);
-      } else if (!columnVector.noNulls && !columnVector.isRepeating && !batch.selectedInUse) {
-        assignIntervalDayTimeNullsNoRepeatingNoSelection(keyIndex, i, batch.size, columnVector);
-      } else if (!columnVector.noNulls && columnVector.isRepeating) {
-        assignIntervalDayTimeNullsRepeating(keyIndex, i, batch.size, columnVector);
-      } else if (!columnVector.noNulls && !columnVector.isRepeating && batch.selectedInUse) {
-        assignIntervalDayTimeNullsNoRepeatingSelection(keyIndex, 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));
-      }
+
+      evaluateIntervalDayTimeColumnVector(batch, columnVector, keyIndex, i);
     }
+
     for(int i=0;i<batch.size;++i) {
       vectorHashKeyWrappers[i].setHashKey();
     }
   }
 
+  private void evaluateLongColumnVector(VectorizedRowBatch batch, LongColumnVector columnVector,
+      int keyIndex, int index) {
+    if (columnVector.isRepeating) {
+      if (columnVector.noNulls || !columnVector.isNull[0]) {
+        assignLongNoNullsRepeating(index, batch.size, columnVector);
+      } else {
+        assignLongNullsRepeating(keyIndex, index, batch.size, columnVector);
+      }
+    } else if (columnVector.noNulls) {
+      if (batch.selectedInUse) {
+        assignLongNoNullsNoRepeatingSelection(index, batch.size, columnVector, batch.selected);
+      } else {
+        assignLongNoNullsNoRepeatingNoSelection(index, batch.size, columnVector);
+      }
+    } else {
+      if (batch.selectedInUse) {
+        assignLongNullsNoRepeatingSelection (keyIndex, index, batch.size, columnVector, batch.selected);
+      } else {
+        assignLongNullsNoRepeatingNoSelection(keyIndex, index, batch.size, columnVector);
+      }
+    }
+  }
+
+  private void evaluateDoubleColumnVector(VectorizedRowBatch batch, DoubleColumnVector columnVector,
+      int keyIndex, int index) {
+    if (columnVector.isRepeating) {
+      if (columnVector.noNulls || !columnVector.isNull[0]) {
+        assignDoubleNoNullsRepeating(index, batch.size, columnVector);
+      } else {
+        assignDoubleNullsRepeating(keyIndex, index, batch.size, columnVector);
+      }
+    } else if (columnVector.noNulls) {
+      if (batch.selectedInUse) {
+        assignDoubleNoNullsNoRepeatingSelection(index, batch.size, columnVector, batch.selected);
+      } else {
+        assignDoubleNoNullsNoRepeatingNoSelection(index, batch.size, columnVector);
+      }
+    } else {
+      if (batch.selectedInUse) {
+        assignDoubleNullsNoRepeatingSelection (keyIndex, index, batch.size, columnVector, batch.selected);
+      } else {
+        assignDoubleNullsNoRepeatingNoSelection(keyIndex, index, batch.size, columnVector);
+      }
+    }
+  }
+
+  private void evaluateStringColumnVector(VectorizedRowBatch batch, BytesColumnVector columnVector,
+      int keyIndex, int index) {
+    if (columnVector.isRepeating) {
+      if (columnVector.noNulls || !columnVector.isNull[0]) {
+        assignStringNoNullsRepeating(index, batch.size, columnVector);
+      } else {
+        assignStringNullsRepeating(keyIndex, index, batch.size, columnVector);
+      }
+    } else if (columnVector.noNulls) {
+      if (batch.selectedInUse) {
+        assignStringNoNullsNoRepeatingSelection(index, batch.size, columnVector, batch.selected);
+      } else {
+        assignStringNoNullsNoRepeatingNoSelection(index, batch.size, columnVector);
+      }
+    } else {
+      if (batch.selectedInUse) {
+        assignStringNullsNoRepeatingSelection (keyIndex, index, batch.size, columnVector, batch.selected);
+      } else {
+        assignStringNullsNoRepeatingNoSelection(keyIndex, index, batch.size, columnVector);
+      }
+    }
+  }
+
+  private void evaluateDecimalColumnVector(VectorizedRowBatch batch, DecimalColumnVector columnVector,
+      int keyIndex, int index) {
+    if (columnVector.isRepeating) {
+      if (columnVector.noNulls || !columnVector.isNull[0]) {
+        assignDecimalNoNullsRepeating(index, batch.size, columnVector);
+      } else {
+        assignDecimalNullsRepeating(keyIndex, index, batch.size, columnVector);
+      }
+    } else if (columnVector.noNulls) {
+      if (batch.selectedInUse) {
+        assignDecimalNoNullsNoRepeatingSelection(index, batch.size, columnVector, batch.selected);
+      } else {
+        assignDecimalNoNullsNoRepeatingNoSelection(index, batch.size, columnVector);
+      }
+    } else {
+      if (batch.selectedInUse) {
+        assignDecimalNullsNoRepeatingSelection (keyIndex, index, batch.size, columnVector, batch.selected);
+      } else {
+        assignDecimalNullsNoRepeatingNoSelection(keyIndex, index, batch.size, columnVector);
+      }
+    }
+  }
+
+  private void evaluateTimestampColumnVector(VectorizedRowBatch batch, TimestampColumnVector columnVector,
+      int keyIndex, int index) {
+    if (columnVector.isRepeating) {
+      if (columnVector.noNulls || !columnVector.isNull[0]) {
+        assignTimestampNoNullsRepeating(index, batch.size, columnVector);
+      } else {
+        assignTimestampNullsRepeating(keyIndex, index, batch.size, columnVector);
+      }
+    } else if (columnVector.noNulls) {
+      if (batch.selectedInUse) {
+        assignTimestampNoNullsNoRepeatingSelection(index, batch.size, columnVector, batch.selected);
+      } else {
+        assignTimestampNoNullsNoRepeatingNoSelection(index, batch.size, columnVector);
+      }
+    } else {
+      if (batch.selectedInUse) {
+        assignTimestampNullsNoRepeatingSelection (keyIndex, index, batch.size, columnVector, batch.selected);
+      } else {
+        assignTimestampNullsNoRepeatingNoSelection(keyIndex, index, batch.size, columnVector);
+      }
+    }
+  }
+
+  private void evaluateIntervalDayTimeColumnVector(VectorizedRowBatch batch, IntervalDayTimeColumnVector columnVector,
+      int keyIndex, int index) {
+    if (columnVector.isRepeating) {
+      if (columnVector.noNulls || !columnVector.isNull[0]) {
+        assignIntervalDayTimeNoNullsRepeating(index, batch.size, columnVector);
+      } else {
+        assignIntervalDayTimeNullsRepeating(keyIndex, index, batch.size, columnVector);
+      }
+    } else if (columnVector.noNulls) {
+      if (batch.selectedInUse) {
+        assignIntervalDayTimeNoNullsNoRepeatingSelection(index, batch.size, columnVector, batch.selected);
+      } else {
+        assignIntervalDayTimeNoNullsNoRepeatingNoSelection(index, batch.size, columnVector);
+      }
+    } else {
+      if (batch.selectedInUse) {
+        assignIntervalDayTimeNullsNoRepeatingSelection (keyIndex, index, batch.size, columnVector, batch.selected);
+      } else {
+        assignIntervalDayTimeNullsNoRepeatingNoSelection(keyIndex, index, batch.size, columnVector);
+      }
+    }
+  }
+
   /**
    * Helper method to assign values from a vector column into the key wrapper.
    * Optimized for string type, possible nulls, no repeat values, batch selection vector.
@@ -906,11 +876,11 @@ public class VectorHashKeyWrapperBatch extends VectorColumnSetInfo {
   {
 
     final int size = keyExpressions.length;
-    ColumnVector.Type[] columnVectorTypes = new ColumnVector.Type[size];
+    TypeInfo[] typeInfos = new TypeInfo[size];
     for (int i = 0; i < size; i++) {
-      columnVectorTypes[i] = keyExpressions[i].getOutputColumnVectorType();
+      typeInfos[i] = keyExpressions[i].getOutputTypeInfo();
     }
-    return compileKeyWrapperBatch(keyExpressions, columnVectorTypes);
+    return compileKeyWrapperBatch(keyExpressions, typeInfos);
   }
 
   /**
@@ -920,7 +890,7 @@ public class VectorHashKeyWrapperBatch extends VectorColumnSetInfo {
    * will be used to generate proper individual VectorKeyHashWrapper objects.
    */
   public static VectorHashKeyWrapperBatch compileKeyWrapperBatch(VectorExpression[] keyExpressions,
-      ColumnVector.Type[] columnVectorTypes)
+      TypeInfo[] typeInfos)
     throws HiveException {
     VectorHashKeyWrapperBatch compiledKeyWrapperBatch = new VectorHashKeyWrapperBatch(keyExpressions.length);
     compiledKeyWrapperBatch.keyExpressions = keyExpressions;
@@ -928,8 +898,8 @@ public class VectorHashKeyWrapperBatch extends VectorColumnSetInfo {
     compiledKeyWrapperBatch.keysFixedSize = 0;
 
     // Inspect the output type of each key expression.
-    for(int i=0; i < columnVectorTypes.length; ++i) {
-      compiledKeyWrapperBatch.addKey(columnVectorTypes[i]);
+    for(int i=0; i < typeInfos.length; ++i) {
+      compiledKeyWrapperBatch.addKey(typeInfos[i]);
     }
     compiledKeyWrapperBatch.finishAdding();
 

http://git-wip-us.apache.org/repos/asf/hive/blob/9bfdcd76/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorHashKeyWrapperBatch.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorHashKeyWrapperBatch.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorHashKeyWrapperBatch.java
new file mode 100644
index 0000000..e349fbd
--- /dev/null
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorHashKeyWrapperBatch.java
@@ -0,0 +1,86 @@
+/*
+ * 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 static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+import java.sql.Timestamp;
+
+import org.junit.Test;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.IdentityExpression;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.exec.vector.util.FakeVectorRowBatchFromObjectIterables;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
+
+/**
+ * Unit test for VectorHashKeyWrapperBatch class.
+ */
+public class TestVectorHashKeyWrapperBatch {
+
+  // Specific test for HIVE-18744 --
+  // Tests Timestamp assignment.
+  @Test
+  public void testVectorHashKeyWrapperBatch() throws HiveException {
+
+    VectorExpression[] keyExpressions =
+        new VectorExpression[] { new IdentityExpression(0) };
+    TypeInfo[] typeInfos =
+        new TypeInfo[] {TypeInfoFactory.timestampTypeInfo};
+    VectorHashKeyWrapperBatch vhkwb =
+        VectorHashKeyWrapperBatch.compileKeyWrapperBatch(
+            keyExpressions,
+            typeInfos);
+
+    VectorizedRowBatch batch = new VectorizedRowBatch(1);
+    batch.selectedInUse = false;
+    batch.size = 10;
+    TimestampColumnVector timestampColVector = new TimestampColumnVector(batch.DEFAULT_SIZE);;
+    batch.cols[0] = timestampColVector;
+    timestampColVector.reset();
+    // Cause Timestamp object to be replaced (in buggy code) with ZERO_TIMESTAMP.
+    timestampColVector.noNulls = false;
+    timestampColVector.isNull[0] = true;
+    Timestamp scratch = new Timestamp(2039);
+    Timestamp ts0 = new Timestamp(2039);
+    scratch.setTime(ts0.getTime());
+    scratch.setNanos(ts0.getNanos());
+    timestampColVector.set(1, scratch);
+    Timestamp ts1 = new Timestamp(33222);
+    scratch.setTime(ts1.getTime());
+    scratch.setNanos(ts1.getNanos());
+    timestampColVector.set(2, scratch);
+    batch.size = 3;
+
+    vhkwb.evaluateBatch(batch);
+    VectorHashKeyWrapper[] vhkwArray = vhkwb.getVectorHashKeyWrappers();
+    VectorHashKeyWrapper vhk = vhkwArray[0];
+    assertTrue(vhk.isNull(0));
+    vhk = vhkwArray[1];
+    assertFalse(vhk.isNull(0));
+    assertEquals(vhk.getTimestamp(0), ts0);
+    vhk = vhkwArray[2];
+    assertFalse(vhk.isNull(0));
+    assertEquals(vhk.getTimestamp(0), ts1);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/9bfdcd76/ql/src/test/results/clientpositive/llap/vectorization_15.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/vectorization_15.q.out b/ql/src/test/results/clientpositive/llap/vectorization_15.q.out
index ae4fe18..9ab5965 100644
--- a/ql/src/test/results/clientpositive/llap/vectorization_15.q.out
+++ b/ql/src/test/results/clientpositive/llap/vectorization_15.q.out
@@ -267,15 +267,15 @@ POSTHOOK: Input: default@alltypesorc
 -51.0	true	NULL	QiOcvR0kt6r7f0R7fiPxQTCU	-51	266531954	1969-12-31 16:00:08.451	NULL	-266531980.28	NULL	NULL	33.0	NULL	0.0	NULL	51	NULL	2.66532E8	-23	266531980.28	0.0
 -51.0	true	NULL	Ybpj38RTTYl7CnJXPNx1g4C	-51	-370919370	1969-12-31 16:00:08.451	NULL	370919343.72	NULL	NULL	33.0	NULL	0.0	NULL	51	NULL	-3.70919296E8	-23	-370919343.72	0.0
 -6.0	NULL	-200.0	NULL	-6	NULL	1969-12-31 15:59:56.094	NULL	NULL	-200.0	-15910.599999999999	3.0	NULL	0.0	-23.0	6	NULL	NULL	-5	NULL	NULL
--62.0	NULL	15601.0	NULL	-62	NULL	1969-12-31 15:59:56.527	NULL	NULL	15601.0	1241106.353	33.0	NULL	0.0	-23.0	62	NULL	NULL	-23	NULL	NULL
+-62.0	NULL	15601.0	NULL	-62	NULL	1969-12-31 16:00:09.889	NULL	NULL	15601.0	1241106.353	33.0	NULL	0.0	-23.0	62	NULL	NULL	-23	NULL	NULL
 11.0	false	NULL	10pO8p1LNx4Y	11	271296824	1969-12-31 16:00:02.351	NULL	-271296850.28	NULL	NULL	0.0	NULL	0.0	NULL	-11	NULL	2.71296832E8	-1	271296850.28	0.0
 11.0	false	NULL	1H6wGP	11	-560827082	1969-12-31 16:00:02.351	NULL	560827055.72	NULL	NULL	0.0	NULL	0.0	NULL	-11	NULL	-5.6082707E8	-1	-560827055.72	0.0
 11.0	false	NULL	2a7V63IL7jK3o	11	-325931647	1969-12-31 16:00:02.351	NULL	325931620.72	NULL	NULL	0.0	NULL	0.0	NULL	-11	NULL	-3.25931648E8	-1	-325931620.72	0.0
 11.0	true	NULL	10	11	92365813	1969-12-31 16:00:02.351	NULL	-92365839.28	NULL	NULL	0.0	NULL	0.0	NULL	-11	NULL	9.2365808E7	-1	92365839.28	0.0
-21.0	NULL	15601.0	NULL	21	NULL	1969-12-31 15:59:56.527	NULL	NULL	15601.0	1241106.353	12.0	NULL	0.0	-23.0	-21	NULL	NULL	-2	NULL	NULL
+21.0	NULL	15601.0	NULL	21	NULL	1969-12-31 16:00:14.256	NULL	NULL	15601.0	1241106.353	12.0	NULL	0.0	-23.0	-21	NULL	NULL	-2	NULL	NULL
 32.0	NULL	-200.0	NULL	32	NULL	1969-12-31 16:00:02.445	NULL	NULL	-200.0	-15910.599999999999	1.0	NULL	0.0	-23.0	-32	NULL	NULL	-23	NULL	NULL
 36.0	NULL	-200.0	NULL	36	NULL	1969-12-31 16:00:00.554	NULL	NULL	-200.0	-15910.599999999999	33.0	NULL	0.0	-23.0	-36	NULL	NULL	-23	NULL	NULL
-5.0	NULL	15601.0	NULL	5	NULL	1969-12-31 15:59:56.527	NULL	NULL	15601.0	1241106.353	3.0	NULL	0.0	-23.0	-5	NULL	NULL	-3	NULL	NULL
+5.0	NULL	15601.0	NULL	5	NULL	1969-12-31 16:00:00.959	NULL	NULL	15601.0	1241106.353	3.0	NULL	0.0	-23.0	-5	NULL	NULL	-3	NULL	NULL
 58.0	NULL	15601.0	NULL	58	NULL	1969-12-31 15:59:56.527	NULL	NULL	15601.0	1241106.353	33.0	NULL	0.0	-23.0	-58	NULL	NULL	-23	NULL	NULL
 8.0	false	NULL	10V3pN5r5lI2qWl2lG103	8	-362835731	1969-12-31 16:00:15.892	NULL	362835704.72	NULL	NULL	1.0	NULL	0.0	NULL	-8	NULL	-3.62835744E8	-7	-362835704.72	0.0
 8.0	false	NULL	10c4qt584m5y6uWT	8	-183000142	1969-12-31 16:00:15.892	NULL	183000115.72	NULL	NULL	1.0	NULL	0.0	NULL	-8	NULL	-1.8300016E8	-7	-183000115.72	0.0

http://git-wip-us.apache.org/repos/asf/hive/blob/9bfdcd76/ql/src/test/results/clientpositive/parquet_vectorization_15.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/parquet_vectorization_15.q.out b/ql/src/test/results/clientpositive/parquet_vectorization_15.q.out
index 71ed777..6ada288 100644
--- a/ql/src/test/results/clientpositive/parquet_vectorization_15.q.out
+++ b/ql/src/test/results/clientpositive/parquet_vectorization_15.q.out
@@ -288,15 +288,15 @@ POSTHOOK: Input: default@alltypesparquet
 -51.0	true	NULL	QiOcvR0kt6r7f0R7fiPxQTCU	-51	266531954	1969-12-31 16:00:08.451	NULL	-266531980.28	NULL	NULL	33.0	NULL	0.0	NULL	51	NULL	2.66532E8	-23	266531980.28	0.0
 -51.0	true	NULL	Ybpj38RTTYl7CnJXPNx1g4C	-51	-370919370	1969-12-31 16:00:08.451	NULL	370919343.72	NULL	NULL	33.0	NULL	0.0	NULL	51	NULL	-3.70919296E8	-23	-370919343.72	0.0
 -6.0	NULL	-200.0	NULL	-6	NULL	1969-12-31 15:59:56.094	NULL	NULL	-200.0	-15910.599999999999	3.0	NULL	0.0	-23.0	6	NULL	NULL	-5	NULL	NULL
--62.0	NULL	15601.0	NULL	-62	NULL	1969-12-31 15:59:56.527	NULL	NULL	15601.0	1241106.353	33.0	NULL	0.0	-23.0	62	NULL	NULL	-23	NULL	NULL
+-62.0	NULL	15601.0	NULL	-62	NULL	1969-12-31 16:00:09.889	NULL	NULL	15601.0	1241106.353	33.0	NULL	0.0	-23.0	62	NULL	NULL	-23	NULL	NULL
 11.0	false	NULL	10pO8p1LNx4Y	11	271296824	1969-12-31 16:00:02.351	NULL	-271296850.28	NULL	NULL	0.0	NULL	0.0	NULL	-11	NULL	2.71296832E8	-1	271296850.28	0.0
 11.0	false	NULL	1H6wGP	11	-560827082	1969-12-31 16:00:02.351	NULL	560827055.72	NULL	NULL	0.0	NULL	0.0	NULL	-11	NULL	-5.6082707E8	-1	-560827055.72	0.0
 11.0	false	NULL	2a7V63IL7jK3o	11	-325931647	1969-12-31 16:00:02.351	NULL	325931620.72	NULL	NULL	0.0	NULL	0.0	NULL	-11	NULL	-3.25931648E8	-1	-325931620.72	0.0
 11.0	true	NULL	10	11	92365813	1969-12-31 16:00:02.351	NULL	-92365839.28	NULL	NULL	0.0	NULL	0.0	NULL	-11	NULL	9.2365808E7	-1	92365839.28	0.0
-21.0	NULL	15601.0	NULL	21	NULL	1969-12-31 15:59:56.527	NULL	NULL	15601.0	1241106.353	12.0	NULL	0.0	-23.0	-21	NULL	NULL	-2	NULL	NULL
+21.0	NULL	15601.0	NULL	21	NULL	1969-12-31 16:00:14.256	NULL	NULL	15601.0	1241106.353	12.0	NULL	0.0	-23.0	-21	NULL	NULL	-2	NULL	NULL
 32.0	NULL	-200.0	NULL	32	NULL	1969-12-31 16:00:02.445	NULL	NULL	-200.0	-15910.599999999999	1.0	NULL	0.0	-23.0	-32	NULL	NULL	-23	NULL	NULL
 36.0	NULL	-200.0	NULL	36	NULL	1969-12-31 16:00:00.554	NULL	NULL	-200.0	-15910.599999999999	33.0	NULL	0.0	-23.0	-36	NULL	NULL	-23	NULL	NULL
-5.0	NULL	15601.0	NULL	5	NULL	1969-12-31 15:59:56.527	NULL	NULL	15601.0	1241106.353	3.0	NULL	0.0	-23.0	-5	NULL	NULL	-3	NULL	NULL
+5.0	NULL	15601.0	NULL	5	NULL	1969-12-31 16:00:00.959	NULL	NULL	15601.0	1241106.353	3.0	NULL	0.0	-23.0	-5	NULL	NULL	-3	NULL	NULL
 58.0	NULL	15601.0	NULL	58	NULL	1969-12-31 15:59:56.527	NULL	NULL	15601.0	1241106.353	33.0	NULL	0.0	-23.0	-58	NULL	NULL	-23	NULL	NULL
 8.0	false	NULL	10V3pN5r5lI2qWl2lG103	8	-362835731	1969-12-31 16:00:15.892	NULL	362835704.72	NULL	NULL	1.0	NULL	0.0	NULL	-8	NULL	-3.62835744E8	-7	-362835704.72	0.0
 8.0	false	NULL	10c4qt584m5y6uWT	8	-183000142	1969-12-31 16:00:15.892	NULL	183000115.72	NULL	NULL	1.0	NULL	0.0	NULL	-8	NULL	-1.8300016E8	-7	-183000115.72	0.0

http://git-wip-us.apache.org/repos/asf/hive/blob/9bfdcd76/ql/src/test/results/clientpositive/spark/parquet_vectorization_15.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/parquet_vectorization_15.q.out b/ql/src/test/results/clientpositive/spark/parquet_vectorization_15.q.out
index c8ac3f2..97cc8de 100644
--- a/ql/src/test/results/clientpositive/spark/parquet_vectorization_15.q.out
+++ b/ql/src/test/results/clientpositive/spark/parquet_vectorization_15.q.out
@@ -263,15 +263,15 @@ POSTHOOK: Input: default@alltypesparquet
 -51.0	true	NULL	QiOcvR0kt6r7f0R7fiPxQTCU	-51	266531954	1969-12-31 16:00:08.451	NULL	-266531980.28	NULL	NULL	33.0	NULL	0.0	NULL	51	NULL	2.66532E8	-23	266531980.28	0.0
 -51.0	true	NULL	Ybpj38RTTYl7CnJXPNx1g4C	-51	-370919370	1969-12-31 16:00:08.451	NULL	370919343.72	NULL	NULL	33.0	NULL	0.0	NULL	51	NULL	-3.70919296E8	-23	-370919343.72	0.0
 -6.0	NULL	-200.0	NULL	-6	NULL	1969-12-31 15:59:56.094	NULL	NULL	-200.0	-15910.599999999999	3.0	NULL	0.0	-23.0	6	NULL	NULL	-5	NULL	NULL
--62.0	NULL	15601.0	NULL	-62	NULL	1969-12-31 15:59:56.527	NULL	NULL	15601.0	1241106.353	33.0	NULL	0.0	-23.0	62	NULL	NULL	-23	NULL	NULL
+-62.0	NULL	15601.0	NULL	-62	NULL	1969-12-31 16:00:09.889	NULL	NULL	15601.0	1241106.353	33.0	NULL	0.0	-23.0	62	NULL	NULL	-23	NULL	NULL
 11.0	false	NULL	10pO8p1LNx4Y	11	271296824	1969-12-31 16:00:02.351	NULL	-271296850.28	NULL	NULL	0.0	NULL	0.0	NULL	-11	NULL	2.71296832E8	-1	271296850.28	0.0
 11.0	false	NULL	1H6wGP	11	-560827082	1969-12-31 16:00:02.351	NULL	560827055.72	NULL	NULL	0.0	NULL	0.0	NULL	-11	NULL	-5.6082707E8	-1	-560827055.72	0.0
 11.0	false	NULL	2a7V63IL7jK3o	11	-325931647	1969-12-31 16:00:02.351	NULL	325931620.72	NULL	NULL	0.0	NULL	0.0	NULL	-11	NULL	-3.25931648E8	-1	-325931620.72	0.0
 11.0	true	NULL	10	11	92365813	1969-12-31 16:00:02.351	NULL	-92365839.28	NULL	NULL	0.0	NULL	0.0	NULL	-11	NULL	9.2365808E7	-1	92365839.28	0.0
-21.0	NULL	15601.0	NULL	21	NULL	1969-12-31 15:59:56.527	NULL	NULL	15601.0	1241106.353	12.0	NULL	0.0	-23.0	-21	NULL	NULL	-2	NULL	NULL
+21.0	NULL	15601.0	NULL	21	NULL	1969-12-31 16:00:14.256	NULL	NULL	15601.0	1241106.353	12.0	NULL	0.0	-23.0	-21	NULL	NULL	-2	NULL	NULL
 32.0	NULL	-200.0	NULL	32	NULL	1969-12-31 16:00:02.445	NULL	NULL	-200.0	-15910.599999999999	1.0	NULL	0.0	-23.0	-32	NULL	NULL	-23	NULL	NULL
 36.0	NULL	-200.0	NULL	36	NULL	1969-12-31 16:00:00.554	NULL	NULL	-200.0	-15910.599999999999	33.0	NULL	0.0	-23.0	-36	NULL	NULL	-23	NULL	NULL
-5.0	NULL	15601.0	NULL	5	NULL	1969-12-31 15:59:56.527	NULL	NULL	15601.0	1241106.353	3.0	NULL	0.0	-23.0	-5	NULL	NULL	-3	NULL	NULL
+5.0	NULL	15601.0	NULL	5	NULL	1969-12-31 16:00:00.959	NULL	NULL	15601.0	1241106.353	3.0	NULL	0.0	-23.0	-5	NULL	NULL	-3	NULL	NULL
 58.0	NULL	15601.0	NULL	58	NULL	1969-12-31 15:59:56.527	NULL	NULL	15601.0	1241106.353	33.0	NULL	0.0	-23.0	-58	NULL	NULL	-23	NULL	NULL
 8.0	false	NULL	10V3pN5r5lI2qWl2lG103	8	-362835731	1969-12-31 16:00:15.892	NULL	362835704.72	NULL	NULL	1.0	NULL	0.0	NULL	-8	NULL	-3.62835744E8	-7	-362835704.72	0.0
 8.0	false	NULL	10c4qt584m5y6uWT	8	-183000142	1969-12-31 16:00:15.892	NULL	183000115.72	NULL	NULL	1.0	NULL	0.0	NULL	-8	NULL	-1.8300016E8	-7	-183000115.72	0.0

http://git-wip-us.apache.org/repos/asf/hive/blob/9bfdcd76/ql/src/test/results/clientpositive/spark/vectorization_15.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/vectorization_15.q.out b/ql/src/test/results/clientpositive/spark/vectorization_15.q.out
index e0953d3..3e7aa0a 100644
--- a/ql/src/test/results/clientpositive/spark/vectorization_15.q.out
+++ b/ql/src/test/results/clientpositive/spark/vectorization_15.q.out
@@ -263,15 +263,15 @@ POSTHOOK: Input: default@alltypesorc
 -51.0	true	NULL	QiOcvR0kt6r7f0R7fiPxQTCU	-51	266531954	1969-12-31 16:00:08.451	NULL	-266531980.28	NULL	NULL	33.0	NULL	0.0	NULL	51	NULL	2.66532E8	-23	266531980.28	0.0
 -51.0	true	NULL	Ybpj38RTTYl7CnJXPNx1g4C	-51	-370919370	1969-12-31 16:00:08.451	NULL	370919343.72	NULL	NULL	33.0	NULL	0.0	NULL	51	NULL	-3.70919296E8	-23	-370919343.72	0.0
 -6.0	NULL	-200.0	NULL	-6	NULL	1969-12-31 15:59:56.094	NULL	NULL	-200.0	-15910.599999999999	3.0	NULL	0.0	-23.0	6	NULL	NULL	-5	NULL	NULL
--62.0	NULL	15601.0	NULL	-62	NULL	1969-12-31 15:59:56.527	NULL	NULL	15601.0	1241106.353	33.0	NULL	0.0	-23.0	62	NULL	NULL	-23	NULL	NULL
+-62.0	NULL	15601.0	NULL	-62	NULL	1969-12-31 16:00:09.889	NULL	NULL	15601.0	1241106.353	33.0	NULL	0.0	-23.0	62	NULL	NULL	-23	NULL	NULL
 11.0	false	NULL	10pO8p1LNx4Y	11	271296824	1969-12-31 16:00:02.351	NULL	-271296850.28	NULL	NULL	0.0	NULL	0.0	NULL	-11	NULL	2.71296832E8	-1	271296850.28	0.0
 11.0	false	NULL	1H6wGP	11	-560827082	1969-12-31 16:00:02.351	NULL	560827055.72	NULL	NULL	0.0	NULL	0.0	NULL	-11	NULL	-5.6082707E8	-1	-560827055.72	0.0
 11.0	false	NULL	2a7V63IL7jK3o	11	-325931647	1969-12-31 16:00:02.351	NULL	325931620.72	NULL	NULL	0.0	NULL	0.0	NULL	-11	NULL	-3.25931648E8	-1	-325931620.72	0.0
 11.0	true	NULL	10	11	92365813	1969-12-31 16:00:02.351	NULL	-92365839.28	NULL	NULL	0.0	NULL	0.0	NULL	-11	NULL	9.2365808E7	-1	92365839.28	0.0
-21.0	NULL	15601.0	NULL	21	NULL	1969-12-31 15:59:56.527	NULL	NULL	15601.0	1241106.353	12.0	NULL	0.0	-23.0	-21	NULL	NULL	-2	NULL	NULL
+21.0	NULL	15601.0	NULL	21	NULL	1969-12-31 16:00:14.256	NULL	NULL	15601.0	1241106.353	12.0	NULL	0.0	-23.0	-21	NULL	NULL	-2	NULL	NULL
 32.0	NULL	-200.0	NULL	32	NULL	1969-12-31 16:00:02.445	NULL	NULL	-200.0	-15910.599999999999	1.0	NULL	0.0	-23.0	-32	NULL	NULL	-23	NULL	NULL
 36.0	NULL	-200.0	NULL	36	NULL	1969-12-31 16:00:00.554	NULL	NULL	-200.0	-15910.599999999999	33.0	NULL	0.0	-23.0	-36	NULL	NULL	-23	NULL	NULL
-5.0	NULL	15601.0	NULL	5	NULL	1969-12-31 15:59:56.527	NULL	NULL	15601.0	1241106.353	3.0	NULL	0.0	-23.0	-5	NULL	NULL	-3	NULL	NULL
+5.0	NULL	15601.0	NULL	5	NULL	1969-12-31 16:00:00.959	NULL	NULL	15601.0	1241106.353	3.0	NULL	0.0	-23.0	-5	NULL	NULL	-3	NULL	NULL
 58.0	NULL	15601.0	NULL	58	NULL	1969-12-31 15:59:56.527	NULL	NULL	15601.0	1241106.353	33.0	NULL	0.0	-23.0	-58	NULL	NULL	-23	NULL	NULL
 8.0	false	NULL	10V3pN5r5lI2qWl2lG103	8	-362835731	1969-12-31 16:00:15.892	NULL	362835704.72	NULL	NULL	1.0	NULL	0.0	NULL	-8	NULL	-3.62835744E8	-7	-362835704.72	0.0
 8.0	false	NULL	10c4qt584m5y6uWT	8	-183000142	1969-12-31 16:00:15.892	NULL	183000115.72	NULL	NULL	1.0	NULL	0.0	NULL	-8	NULL	-1.8300016E8	-7	-183000115.72	0.0

http://git-wip-us.apache.org/repos/asf/hive/blob/9bfdcd76/ql/src/test/results/clientpositive/vectorization_15.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vectorization_15.q.out b/ql/src/test/results/clientpositive/vectorization_15.q.out
index e020bb8..3e3ceba 100644
--- a/ql/src/test/results/clientpositive/vectorization_15.q.out
+++ b/ql/src/test/results/clientpositive/vectorization_15.q.out
@@ -288,15 +288,15 @@ POSTHOOK: Input: default@alltypesorc
 -51.0	true	NULL	QiOcvR0kt6r7f0R7fiPxQTCU	-51	266531954	1969-12-31 16:00:08.451	NULL	-266531980.28	NULL	NULL	33.0	NULL	0.0	NULL	51	NULL	2.66532E8	-23	266531980.28	0.0
 -51.0	true	NULL	Ybpj38RTTYl7CnJXPNx1g4C	-51	-370919370	1969-12-31 16:00:08.451	NULL	370919343.72	NULL	NULL	33.0	NULL	0.0	NULL	51	NULL	-3.70919296E8	-23	-370919343.72	0.0
 -6.0	NULL	-200.0	NULL	-6	NULL	1969-12-31 15:59:56.094	NULL	NULL	-200.0	-15910.599999999999	3.0	NULL	0.0	-23.0	6	NULL	NULL	-5	NULL	NULL
--62.0	NULL	15601.0	NULL	-62	NULL	1969-12-31 15:59:56.527	NULL	NULL	15601.0	1241106.353	33.0	NULL	0.0	-23.0	62	NULL	NULL	-23	NULL	NULL
+-62.0	NULL	15601.0	NULL	-62	NULL	1969-12-31 16:00:09.889	NULL	NULL	15601.0	1241106.353	33.0	NULL	0.0	-23.0	62	NULL	NULL	-23	NULL	NULL
 11.0	false	NULL	10pO8p1LNx4Y	11	271296824	1969-12-31 16:00:02.351	NULL	-271296850.28	NULL	NULL	0.0	NULL	0.0	NULL	-11	NULL	2.71296832E8	-1	271296850.28	0.0
 11.0	false	NULL	1H6wGP	11	-560827082	1969-12-31 16:00:02.351	NULL	560827055.72	NULL	NULL	0.0	NULL	0.0	NULL	-11	NULL	-5.6082707E8	-1	-560827055.72	0.0
 11.0	false	NULL	2a7V63IL7jK3o	11	-325931647	1969-12-31 16:00:02.351	NULL	325931620.72	NULL	NULL	0.0	NULL	0.0	NULL	-11	NULL	-3.25931648E8	-1	-325931620.72	0.0
 11.0	true	NULL	10	11	92365813	1969-12-31 16:00:02.351	NULL	-92365839.28	NULL	NULL	0.0	NULL	0.0	NULL	-11	NULL	9.2365808E7	-1	92365839.28	0.0
-21.0	NULL	15601.0	NULL	21	NULL	1969-12-31 15:59:56.527	NULL	NULL	15601.0	1241106.353	12.0	NULL	0.0	-23.0	-21	NULL	NULL	-2	NULL	NULL
+21.0	NULL	15601.0	NULL	21	NULL	1969-12-31 16:00:14.256	NULL	NULL	15601.0	1241106.353	12.0	NULL	0.0	-23.0	-21	NULL	NULL	-2	NULL	NULL
 32.0	NULL	-200.0	NULL	32	NULL	1969-12-31 16:00:02.445	NULL	NULL	-200.0	-15910.599999999999	1.0	NULL	0.0	-23.0	-32	NULL	NULL	-23	NULL	NULL
 36.0	NULL	-200.0	NULL	36	NULL	1969-12-31 16:00:00.554	NULL	NULL	-200.0	-15910.599999999999	33.0	NULL	0.0	-23.0	-36	NULL	NULL	-23	NULL	NULL
-5.0	NULL	15601.0	NULL	5	NULL	1969-12-31 15:59:56.527	NULL	NULL	15601.0	1241106.353	3.0	NULL	0.0	-23.0	-5	NULL	NULL	-3	NULL	NULL
+5.0	NULL	15601.0	NULL	5	NULL	1969-12-31 16:00:00.959	NULL	NULL	15601.0	1241106.353	3.0	NULL	0.0	-23.0	-5	NULL	NULL	-3	NULL	NULL
 58.0	NULL	15601.0	NULL	58	NULL	1969-12-31 15:59:56.527	NULL	NULL	15601.0	1241106.353	33.0	NULL	0.0	-23.0	-58	NULL	NULL	-23	NULL	NULL
 8.0	false	NULL	10V3pN5r5lI2qWl2lG103	8	-362835731	1969-12-31 16:00:15.892	NULL	362835704.72	NULL	NULL	1.0	NULL	0.0	NULL	-8	NULL	-3.62835744E8	-7	-362835704.72	0.0
 8.0	false	NULL	10c4qt584m5y6uWT	8	-183000142	1969-12-31 16:00:15.892	NULL	183000115.72	NULL	NULL	1.0	NULL	0.0	NULL	-8	NULL	-1.8300016E8	-7	-183000115.72	0.0

http://git-wip-us.apache.org/repos/asf/hive/blob/9bfdcd76/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedRowBatch.java
----------------------------------------------------------------------
diff --git a/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedRowBatch.java b/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedRowBatch.java
index ea13c24..bebf769 100644
--- a/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedRowBatch.java
+++ b/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedRowBatch.java
@@ -180,12 +180,12 @@ public class VectorizedRowBatch implements Writable {
     return b.toString();
   }
 
-  @Override
-  public String toString() {
+  public String stringify(String prefix) {
     if (size == 0) {
       return "";
     }
     StringBuilder b = new StringBuilder();
+    b.append(prefix);
     b.append("Column vector types: ");
     for (int k = 0; k < projectionSize; k++) {
       int projIndex = projectedColumns[k];
@@ -233,11 +233,18 @@ public class VectorizedRowBatch implements Writable {
           if (k > 0) {
             b.append(", ");
           }
-          cv.stringifyValue(b, i);
+          if (cv != null) {
+            try {
+              cv.stringifyValue(b, i);
+            } catch (Exception ex) {
+              b.append("<invalid>");
+            }
+          }
         }
         b.append(']');
         if (j < size - 1) {
           b.append('\n');
+          b.append(prefix);
         }
       }
     } else {
@@ -260,6 +267,7 @@ public class VectorizedRowBatch implements Writable {
         b.append(']');
         if (i < size - 1) {
           b.append('\n');
+          b.append(prefix);
         }
       }
     }
@@ -267,6 +275,11 @@ public class VectorizedRowBatch implements Writable {
   }
 
   @Override
+  public String toString() {
+    return stringify("");
+  }
+
+  @Override
   public void readFields(DataInput arg0) throws IOException {
     throw new UnsupportedOperationException("Do you really need me?");
   }