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/08/25 16:21:40 UTC

[3/5] hive git commit: HIVE-20315: Vectorization: Fix more NULL / Wrong Results issues and avoid unnecessary casts/conversions (Matt McCline, reviewed by Teddy Choi)

http://git-wip-us.apache.org/repos/asf/hive/blob/fa36381f/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringGroupToString.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringGroupToString.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringGroupToString.java
deleted file mode 100644
index 8232e67..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringGroupToString.java
+++ /dev/null
@@ -1,40 +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;
-
-import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
-
-// cast string group to string (varchar to string, etc.)
-public class CastStringGroupToString extends StringUnaryUDFDirect {
-
-  private static final long serialVersionUID = 1L;
-
-  public CastStringGroupToString() {
-    super();
-  }
-
-  public CastStringGroupToString(int inputColumn, int outputColumnNum) {
-    super(inputColumn, outputColumnNum);
-  }
-
-  @Override
-  protected void func(BytesColumnVector outV, byte[][] vector, int[] start, int[] length, int i) {
-    outV.setVal(i, vector[i], start[i], length[i]);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/fa36381f/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorElt.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorElt.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorElt.java
index 00e529d..75e60eb 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorElt.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorElt.java
@@ -66,53 +66,157 @@ public class VectorElt extends VectorExpression {
 
     outputVector.init();
 
-    outputVector.noNulls = false;
     outputVector.isRepeating = false;
 
+    final int limit = inputColumns.length;
     LongColumnVector inputIndexVector = (LongColumnVector) batch.cols[inputColumns[0]];
+    boolean[] inputIndexIsNull = inputIndexVector.isNull;
     long[] indexVector = inputIndexVector.vector;
     if (inputIndexVector.isRepeating) {
-      int index = (int)indexVector[0];
-      if (index > 0 && index < inputColumns.length) {
-        BytesColumnVector cv = (BytesColumnVector) batch.cols[inputColumns[index]];
-        if (cv.isRepeating) {
-          outputVector.setElement(0, 0, cv);
-          outputVector.isRepeating = true;
-        } else if (batch.selectedInUse) {
-          for (int j = 0; j != n; j++) {
-            int i = sel[j];
-            outputVector.setVal(i, cv.vector[0], cv.start[0], cv.length[0]);
+      if (inputIndexVector.noNulls || !inputIndexIsNull[0]) {
+        int repeatedIndex = (int) indexVector[0];
+        if (repeatedIndex > 0 && repeatedIndex < limit) {
+          BytesColumnVector cv = (BytesColumnVector) batch.cols[inputColumns[repeatedIndex]];
+          if (cv.isRepeating) {
+            outputVector.isNull[0] = false;
+            outputVector.setElement(0, 0, cv);
+            outputVector.isRepeating = true;
+          } else if (cv.noNulls) {
+            if (batch.selectedInUse) {
+              for (int j = 0; j != n; j++) {
+                int i = sel[j];
+                outputVector.isNull[i] = false;
+                outputVector.setVal(i, cv.vector[i], cv.start[i], cv.length[i]);
+              }
+            } else {
+              for (int i = 0; i != n; i++) {
+                outputVector.isNull[i] = false;
+                outputVector.setVal(i, cv.vector[i], cv.start[i], cv.length[i]);
+              }
+            }
+          } else {
+            if (batch.selectedInUse) {
+              for (int j = 0; j != n; j++) {
+                int i = sel[j];
+                if (!cv.isNull[i]) {
+                  outputVector.isNull[i] = false;
+                  outputVector.setVal(i, cv.vector[i], cv.start[i], cv.length[i]);
+                } else {
+                  outputVector.isNull[i] = true;
+                  outputVector.noNulls = false;
+                }
+              }
+            } else {
+              for (int i = 0; i != n; i++) {
+                if (!cv.isNull[i]) {
+                  outputVector.isNull[i] = false;
+                  outputVector.setVal(i, cv.vector[i], cv.start[i], cv.length[i]);
+                } else {
+                  outputVector.isNull[i] = true;
+                  outputVector.noNulls = false;
+                }
+              }
+            }
           }
         } else {
-          for (int i = 0; i != n; i++) {
-            outputVector.setVal(i, cv.vector[0], cv.start[0], cv.length[0]);
-          }
+          outputVector.isNull[0] = true;
+          outputVector.noNulls = false;
+          outputVector.isRepeating = true;
         }
       } else {
         outputVector.isNull[0] = true;
+        outputVector.noNulls = false;
         outputVector.isRepeating = true;
       }
-    } else if (batch.selectedInUse) {
-      for (int j = 0; j != n; j++) {
-        int i = sel[j];
-        int index = (int)indexVector[i];
-        if (index > 0 && index < inputColumns.length) {
-          BytesColumnVector cv = (BytesColumnVector) batch.cols[inputColumns[index]];
-          int cvi = cv.isRepeating ? 0 : i;
-          outputVector.setVal(i, cv.vector[cvi], cv.start[cvi], cv.length[cvi]);
-        } else {
-          outputVector.isNull[i] = true;
+      return;
+    }
+
+    if (inputIndexVector.noNulls) {
+      if (batch.selectedInUse) {
+        for (int j = 0; j != n; j++) {
+          int i = sel[j];
+          int index = (int) indexVector[i];
+          if (index > 0 && index < limit) {
+            BytesColumnVector cv = (BytesColumnVector) batch.cols[inputColumns[index]];
+            int adjusted = cv.isRepeating ? 0 : i;
+            if (!cv.isNull[adjusted]) {
+              outputVector.isNull[i] = false;
+              outputVector.setVal(i, cv.vector[adjusted], cv.start[adjusted], cv.length[adjusted]);
+            } else {
+              outputVector.isNull[i] = true;
+              outputVector.noNulls = false;
+            }
+          } else {
+            outputVector.isNull[i] = true;
+            outputVector.noNulls = false;
+          }
+        }
+      } else {
+        for (int i = 0; i != n; i++) {
+          int index = (int) indexVector[i];
+          if (index > 0 && index < limit) {
+            BytesColumnVector cv = (BytesColumnVector) batch.cols[inputColumns[index]];
+            int adjusted = cv.isRepeating ? 0 : i;
+            if (!cv.isNull[adjusted]) {
+              outputVector.isNull[i] = false;
+              outputVector.setVal(i, cv.vector[adjusted], cv.start[adjusted], cv.length[adjusted]);
+            } else {
+              outputVector.isNull[i] = true;
+              outputVector.noNulls = false;
+            }
+          } else {
+            outputVector.isNull[i] = true;
+            outputVector.noNulls = false;
+          }
         }
       }
     } else {
-      for (int i = 0; i != n; i++) {
-        int index = (int)indexVector[i];
-        if (index > 0 && index < inputColumns.length) {
-          BytesColumnVector cv = (BytesColumnVector) batch.cols[inputColumns[index]];
-          int cvi = cv.isRepeating ? 0 : i;
-          outputVector.setVal(i, cv.vector[cvi], cv.start[cvi], cv.length[cvi]);
-        } else {
-          outputVector.isNull[i] = true;
+      if (batch.selectedInUse) {
+        for (int j = 0; j != n; j++) {
+          int i = sel[j];
+          if (!inputIndexVector.isNull[i]) {
+            int index = (int) indexVector[i];
+            if (index > 0 && index < limit) {
+              BytesColumnVector cv = (BytesColumnVector) batch.cols[inputColumns[index]];
+              int adjusted = cv.isRepeating ? 0 : i;
+              if (cv.noNulls || !cv.isNull[adjusted]) {
+                outputVector.isNull[i] = false;
+                outputVector.setVal(i, cv.vector[adjusted], cv.start[adjusted], cv.length[adjusted]);
+              } else {
+                outputVector.isNull[i] = true;
+                outputVector.noNulls = false;
+              }
+            } else {
+              outputVector.isNull[i] = true;
+              outputVector.noNulls = false;
+            }
+          } else {
+            outputVector.isNull[i] = true;
+            outputVector.noNulls = false;
+          }
+        }
+      } else {
+        for (int i = 0; i != n; i++) {
+          if (!inputIndexVector.isNull[i]) {
+            int index = (int) indexVector[i];
+            if (index > 0 && index < limit) {
+              BytesColumnVector cv = (BytesColumnVector) batch.cols[inputColumns[index]];
+              int adjusted = cv.isRepeating ? 0 : i;
+              if (cv.noNulls || !cv.isNull[adjusted]) {
+                outputVector.isNull[i] = false;
+                outputVector.setVal(i, cv.vector[adjusted], cv.start[adjusted], cv.length[adjusted]);
+              } else {
+                outputVector.isNull[i] = true;
+                outputVector.noNulls = false;
+              }
+            } else {
+              outputVector.isNull[i] = true;
+              outputVector.noNulls = false;
+            }
+          } else {
+            outputVector.isNull[i] = true;
+            outputVector.noNulls = false;
+          }
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/fa36381f/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorExpressionWriterFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorExpressionWriterFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorExpressionWriterFactory.java
index 6a87927..7829b22 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorExpressionWriterFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorExpressionWriterFactory.java
@@ -67,6 +67,7 @@ import org.apache.hadoop.hive.serde2.typeinfo.MapTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.StructTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.ListTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
 import org.apache.hadoop.hive.serde2.typeinfo.UnionTypeInfo;
 import org.apache.hadoop.io.Text;
@@ -601,6 +602,18 @@ public final class VectorExpressionWriterFactory {
     }
 
     /**
+     * Compiles the appropriate vector expression writer based on an expression info (ExprNodeDesc)
+     */
+    public static VectorExpressionWriter genVectorExpressionWritable(VectorExpression vecExpr)
+      throws HiveException {
+      TypeInfo outputTypeInfo = vecExpr.getOutputTypeInfo();
+      ObjectInspector objectInspector =
+          TypeInfoUtils.getStandardWritableObjectInspectorFromTypeInfo(
+              outputTypeInfo);
+      return genVectorExpressionWritable(objectInspector);
+    }
+
+    /**
      * Specialized writer for ListColumnVector. Will throw cast exception
      * if the wrong vector column is used.
      */
@@ -1746,6 +1759,19 @@ public final class VectorExpressionWriterFactory {
   }
 
   /**
+   * Helper function to create an array of writers from a list of expression descriptors.
+   */
+  public static VectorExpressionWriter[] getExpressionWriters(VectorExpression[] vecExprs)
+      throws HiveException {
+    VectorExpressionWriter[] writers = new VectorExpressionWriter[vecExprs.length];
+    for(int i=0; i<writers.length; ++i) {
+      VectorExpression vecExpr = vecExprs[i];
+      writers[i] = genVectorExpressionWritable(vecExpr);
+    }
+    return writers;
+  }
+
+  /**
    * A poor man Java closure. Works around the problem of having to return multiple objects
    * from one function call.
    */

http://git-wip-us.apache.org/repos/asf/hive/blob/fa36381f/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorRowObject.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorRowObject.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorRowObject.java
index d1efaec..1f2b171 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorRowObject.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorRowObject.java
@@ -58,7 +58,8 @@ public class TestVectorRowObject extends TestCase {
 
     VectorRandomRowSource source = new VectorRandomRowSource();
 
-    source.init(r, VectorRandomRowSource.SupportedTypes.ALL, 4);
+    source.init(r, VectorRandomRowSource.SupportedTypes.ALL, 4,
+        /* allowNulls */ true, /* isUnicodeOk */ true);
 
     VectorizedRowBatchCtx batchContext = new VectorizedRowBatchCtx();
     batchContext.init(source.rowStructObjectInspector(), emptyScratchTypeNames);

http://git-wip-us.apache.org/repos/asf/hive/blob/fa36381f/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorSerDeRow.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorSerDeRow.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorSerDeRow.java
index 83cdb2d..8b1b612 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorSerDeRow.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorSerDeRow.java
@@ -21,12 +21,15 @@ package org.apache.hadoop.hive.ql.exec.vector;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.List;
 import java.util.Properties;
 import java.util.Random;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.serde.serdeConstants;
 import org.apache.hadoop.hive.serde2.SerDeException;
+import org.apache.hadoop.hive.common.type.HiveChar;
+import org.apache.hadoop.hive.common.type.HiveVarchar;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.serde2.ByteStream.Output;
 import org.apache.hadoop.hive.serde2.binarysortable.BinarySortableSerDe;
@@ -47,6 +50,9 @@ import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.UnionObject;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 import org.apache.hadoop.hive.serde2.fast.SerializeWrite;
+import org.apache.hadoop.hive.serde2.io.HiveCharWritable;
+import org.apache.hadoop.hive.serde2.io.HiveVarcharWritable;
+import org.apache.hadoop.io.Text;
 
 import junit.framework.TestCase;
 
@@ -147,7 +153,10 @@ public class TestVectorSerDeRow extends TestCase {
 
     VectorRandomRowSource source = new VectorRandomRowSource();
 
-    source.init(r, VectorRandomRowSource.SupportedTypes.ALL, 4, false);
+    // FUTURE: try NULLs and UNICODE.
+    source.init(
+        r, VectorRandomRowSource.SupportedTypes.ALL, 4,
+        /* allowNulls */ false, /* isUnicodeOk */ false);
 
     VectorizedRowBatchCtx batchContext = new VectorizedRowBatchCtx();
     batchContext.init(source.rowStructObjectInspector(), emptyScratchTypeNames);
@@ -206,8 +215,100 @@ public class TestVectorSerDeRow extends TestCase {
     }
   }
 
+  private String getDifferenceInfo(Object actualRow, Object expectedRow) {
+    if (actualRow instanceof List && expectedRow instanceof List) {
+      List<Object> actualList = (List) actualRow;
+      final int actualSize = actualList.size();
+      List<Object> expectedList = (List) expectedRow;
+      final int expectedSize = expectedList.size();
+      if (actualSize != expectedSize) {
+        return "Actual size " + actualSize + ", expected size " + expectedSize;
+      }
+      for (int i = 0; i < actualSize; i++) {
+        Object actualObject = actualList.get(i);
+        Object expecedObject = expectedList.get(i);
+        if (!actualObject.equals(expecedObject)) {
+          return "Column " + i + " is different";
+        }
+      }
+    } else {
+      if (!actualRow.equals(expectedRow)) {
+        return "Object is different";
+      }
+    }
+    return "Actual and expected row are the same";
+  }
+
+  private String getObjectDisplayString(Object object) {
+    StringBuilder sb = new StringBuilder();
+
+    if (object == null) {
+      sb.append("NULL");
+    } else if (object instanceof Text ||
+               object instanceof HiveChar || object instanceof HiveCharWritable ||
+               object instanceof HiveVarchar || object instanceof HiveVarcharWritable) {
+        final String string;
+        if (object instanceof Text) {
+          Text text = (Text) object;
+          string = text.toString();
+        } else if (object instanceof HiveChar) {
+          HiveChar hiveChar = (HiveChar) object;
+          string = hiveChar.getStrippedValue();
+        } else if (object instanceof HiveCharWritable) {
+          HiveChar hiveChar = ((HiveCharWritable) object).getHiveChar();
+          string = hiveChar.getStrippedValue();
+        } else if (object instanceof HiveVarchar) {
+          HiveVarchar hiveVarchar = (HiveVarchar) object;
+          string = hiveVarchar.getValue();
+        } else if (object instanceof HiveVarcharWritable) {
+          HiveVarchar hiveVarchar = ((HiveVarcharWritable) object).getHiveVarchar();
+          string = hiveVarchar.getValue();
+        } else {
+          throw new RuntimeException("Unexpected");
+        }
+
+        byte[] bytes = string.getBytes();
+        final int byteLength = bytes.length;
+
+        sb.append("'");
+        sb.append(string);
+        sb.append("' (byte length ");
+        sb.append(bytes.length);
+        sb.append(", string length ");
+        sb.append(string.length());
+        sb.append(", bytes ");
+        sb.append(VectorizedBatchUtil.displayBytes(bytes, 0, byteLength));
+        sb.append(")");
+    } else {
+      sb.append(object.toString());
+    }
+    return sb.toString();
+  }
+
+  private String getRowDisplayString(Object row) {
+    StringBuilder sb = new StringBuilder();
+    if (row instanceof List) {
+      List<Object> list = (List) row;
+      final int size = list.size();
+      boolean isFirst = true;
+      for (int i = 0; i < size; i++) {
+        if (isFirst) {
+          isFirst = false;
+        } else {
+          sb.append(", ");
+        }
+        Object object = list.get(i);
+        sb.append(getObjectDisplayString(object));
+      }
+    } else {
+      sb.append(getObjectDisplayString(row));
+    }
+    return sb.toString();
+  }
+
   void examineBatch(VectorizedRowBatch batch, VectorExtractRow vectorExtractRow,
-      TypeInfo[] typeInfos, Object[][] randomRows, int firstRandomRowIndex ) {
+      TypeInfo[] typeInfos, Object[][] randomRows, int firstRandomRowIndex,
+      String title) {
 
     int rowSize = vectorExtractRow.getCount();
     Object[] row = new Object[rowSize];
@@ -228,9 +329,15 @@ public class TestVectorSerDeRow extends TestCase {
               " batch index " + i + " firstRandomRowIndex " + firstRandomRowIndex);
         }
         if (!rowObj.equals(expectedObj)) {
+          String actualValueString = getRowDisplayString(rowObj);
+          String expectedValueString = getRowDisplayString(expectedObj);
+          String differentInfoString = getDifferenceInfo(row, expectedObj);
           fail("Row " + (firstRandomRowIndex + i) + " and column " + c + " mismatch (" +
-              typeInfos[c].getCategory() + " actual value " + rowObj +
-              " and expected value " + expectedObj + ")");
+              typeInfos[c].getCategory() + " actual value '" + actualValueString + "'" +
+              " and expected value '" + expectedValueString + "')" +
+              " difference info " + differentInfoString +
+              " typeInfos " + Arrays.toString(typeInfos) +
+              " title " + title);
         }
       }
     }
@@ -283,19 +390,27 @@ public class TestVectorSerDeRow extends TestCase {
       throws HiveException, IOException, SerDeException {
 
     for (int i = 0; i < 20; i++) {
-      innerTestVectorDeserializeRow(r, serializationType, alternate1, alternate2, useExternalBuffer);
+      innerTestVectorDeserializeRow(
+          r, i,serializationType, alternate1, alternate2, useExternalBuffer);
     }
   }
 
   void innerTestVectorDeserializeRow(
-      Random r, SerializationType serializationType,
+      Random r, int iteration,
+      SerializationType serializationType,
       boolean alternate1, boolean alternate2, boolean useExternalBuffer)
       throws HiveException, IOException, SerDeException {
 
+    String title = "serializationType: " + serializationType + ", iteration " + iteration;
+
     String[] emptyScratchTypeNames = new String[0];
 
     VectorRandomRowSource source = new VectorRandomRowSource();
-    source.init(r, VectorRandomRowSource.SupportedTypes.ALL, 4, false);
+
+    // FUTURE: try NULLs and UNICODE.
+    source.init(
+        r, VectorRandomRowSource.SupportedTypes.ALL, 4,
+        /* allowNulls */ false, /* isUnicodeOk */ false);
 
     VectorizedRowBatchCtx batchContext = new VectorizedRowBatchCtx();
     batchContext.init(source.rowStructObjectInspector(), emptyScratchTypeNames);
@@ -426,13 +541,17 @@ public class TestVectorSerDeRow extends TestCase {
       }
       batch.size++;
       if (batch.size == batch.DEFAULT_SIZE) {
-        examineBatch(batch, vectorExtractRow, typeInfos, randomRows, firstRandomRowIndex);
+        examineBatch(
+            batch, vectorExtractRow, typeInfos, randomRows, firstRandomRowIndex,
+            title);
         firstRandomRowIndex = i + 1;
         batch.reset();
       }
     }
     if (batch.size > 0) {
-      examineBatch(batch, vectorExtractRow, typeInfos, randomRows, firstRandomRowIndex);
+      examineBatch(
+          batch, vectorExtractRow, typeInfos, randomRows, firstRandomRowIndex,
+          title);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/fa36381f/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/VectorRandomRowSource.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/VectorRandomRowSource.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/VectorRandomRowSource.java
index af73ee6..b84273a 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/VectorRandomRowSource.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/VectorRandomRowSource.java
@@ -129,6 +129,7 @@ public class VectorRandomRowSource {
   private String[] alphabets;
 
   private boolean allowNull;
+  private boolean isUnicodeOk;
 
   private boolean addEscapables;
   private String needsEscapeStr;
@@ -289,26 +290,28 @@ public class VectorRandomRowSource {
     ALL, PRIMITIVES, ALL_EXCEPT_MAP
   }
 
-  public void init(Random r, SupportedTypes supportedTypes, int maxComplexDepth) {
-    init(r, supportedTypes, maxComplexDepth, true);
-  }
-
-  public void init(Random r, SupportedTypes supportedTypes, int maxComplexDepth, boolean allowNull) {
+  public void init(Random r, SupportedTypes supportedTypes, int maxComplexDepth, boolean allowNull,
+      boolean isUnicodeOk) {
     this.r = r;
     this.allowNull = allowNull;
+    this.isUnicodeOk = isUnicodeOk;
     chooseSchema(supportedTypes, null, null, null, maxComplexDepth);
   }
 
-  public void init(Random r, Set<String> allowedTypeNameSet, int maxComplexDepth, boolean allowNull) {
+  public void init(Random r, Set<String> allowedTypeNameSet, int maxComplexDepth, boolean allowNull,
+      boolean isUnicodeOk) {
     this.r = r;
     this.allowNull = allowNull;
+    this.isUnicodeOk = isUnicodeOk;
     chooseSchema(SupportedTypes.ALL, allowedTypeNameSet, null, null, maxComplexDepth);
   }
 
   public void initExplicitSchema(Random r, List<String> explicitTypeNameList, int maxComplexDepth,
-      boolean allowNull, List<DataTypePhysicalVariation> explicitDataTypePhysicalVariationList) {
+      boolean allowNull, boolean isUnicodeOk,
+      List<DataTypePhysicalVariation> explicitDataTypePhysicalVariationList) {
     this.r = r;
     this.allowNull = allowNull;
+    this.isUnicodeOk = isUnicodeOk;
 
     List<GenerationSpec> generationSpecList = new ArrayList<GenerationSpec>();
     for (String explicitTypeName : explicitTypeNameList) {
@@ -324,9 +327,11 @@ public class VectorRandomRowSource {
   }
 
   public void initGenerationSpecSchema(Random r, List<GenerationSpec> generationSpecList, int maxComplexDepth,
-      boolean allowNull, List<DataTypePhysicalVariation> explicitDataTypePhysicalVariationList) {
+      boolean allowNull, boolean isUnicodeOk,
+      List<DataTypePhysicalVariation> explicitDataTypePhysicalVariationList) {
     this.r = r;
     this.allowNull = allowNull;
+    this.isUnicodeOk = isUnicodeOk;
     chooseSchema(
         SupportedTypes.ALL, null, generationSpecList, explicitDataTypePhysicalVariationList,
         maxComplexDepth);
@@ -1009,9 +1014,19 @@ public class VectorRandomRowSource {
       PrimitiveTypeInfo[] primitiveTypeInfos,
       DataTypePhysicalVariation[] dataTypePhysicalVariations) {
 
+    return randomPrimitiveRow(
+        columnCount, r, primitiveTypeInfos, dataTypePhysicalVariations, false);
+  }
+
+  public static Object[] randomPrimitiveRow(int columnCount, Random r,
+      PrimitiveTypeInfo[] primitiveTypeInfos,
+      DataTypePhysicalVariation[] dataTypePhysicalVariations, boolean isUnicodeOk) {
+
     final Object row[] = new Object[columnCount];
     for (int c = 0; c < columnCount; c++) {
-      row[c] = randomPrimitiveObject(r, primitiveTypeInfos[c], dataTypePhysicalVariations[c]);
+      row[c] =
+          randomPrimitiveObject(
+              r, primitiveTypeInfos[c], dataTypePhysicalVariations[c], isUnicodeOk);
     }
     return row;
   }
@@ -1624,11 +1639,11 @@ public class VectorRandomRowSource {
   }
 
   public static Object randomPrimitiveObject(Random r, PrimitiveTypeInfo primitiveTypeInfo) {
-    return randomPrimitiveObject(r, primitiveTypeInfo, DataTypePhysicalVariation.NONE);
+    return randomPrimitiveObject(r, primitiveTypeInfo, DataTypePhysicalVariation.NONE, false);
   }
 
   public static Object randomPrimitiveObject(Random r, PrimitiveTypeInfo primitiveTypeInfo,
-      DataTypePhysicalVariation dataTypePhysicalVariation) {
+      DataTypePhysicalVariation dataTypePhysicalVariation, boolean isUnicodeOk) {
 
     switch (primitiveTypeInfo.getPrimitiveCategory()) {
     case BOOLEAN:
@@ -1648,11 +1663,11 @@ public class VectorRandomRowSource {
     case DOUBLE:
       return Double.valueOf(r.nextDouble() * 10 - 5);
     case STRING:
-      return RandomTypeUtil.getRandString(r);
+      return getRandString(r, isUnicodeOk);
     case CHAR:
-      return getRandHiveChar(r, (CharTypeInfo) primitiveTypeInfo);
+      return getRandHiveChar(r, (CharTypeInfo) primitiveTypeInfo, isUnicodeOk);
     case VARCHAR:
-      return getRandHiveVarchar(r, (VarcharTypeInfo) primitiveTypeInfo);
+      return getRandHiveVarchar(r, (VarcharTypeInfo) primitiveTypeInfo, isUnicodeOk);
     case BINARY:
       return getRandBinary(r, 1 + r.nextInt(100));
     case TIMESTAMP:
@@ -1682,22 +1697,30 @@ public class VectorRandomRowSource {
     return RandomTypeUtil.getRandTimestamp(r).toString();
   }
 
-  public static HiveChar getRandHiveChar(Random r, CharTypeInfo charTypeInfo) {
+  public static String getRandString(Random r, boolean isUnicodeOk) {
+    return getRandString(r, r.nextInt(10), isUnicodeOk);
+  }
+
+  public static String getRandString(Random r, int length, boolean isUnicodeOk) {
+    return
+        !isUnicodeOk || r.nextBoolean() ?
+            RandomTypeUtil.getRandString(r, "abcdefghijklmnopqrstuvwxyz", length) :
+            RandomTypeUtil.getRandUnicodeString(r, length);
+  }
+
+  public static HiveChar getRandHiveChar(Random r, CharTypeInfo charTypeInfo, boolean isUnicodeOk) {
     final int maxLength = 1 + r.nextInt(charTypeInfo.getLength());
-    final String randomString = RandomTypeUtil.getRandString(r, "abcdefghijklmnopqrstuvwxyz", 100);
+    final String randomString = getRandString(r, 100, isUnicodeOk);
     return new HiveChar(randomString, maxLength);
   }
 
-  public static HiveVarchar getRandHiveVarchar(Random r, VarcharTypeInfo varcharTypeInfo, String alphabet) {
+  public static HiveVarchar getRandHiveVarchar(Random r, VarcharTypeInfo varcharTypeInfo,
+      boolean isUnicodeOk) {
     final int maxLength = 1 + r.nextInt(varcharTypeInfo.getLength());
-    final String randomString = RandomTypeUtil.getRandString(r, alphabet, 100);
+    final String randomString = getRandString(r, 100, isUnicodeOk);
     return new HiveVarchar(randomString, maxLength);
   }
 
-  public static HiveVarchar getRandHiveVarchar(Random r, VarcharTypeInfo varcharTypeInfo) {
-    return getRandHiveVarchar(r, varcharTypeInfo, "abcdefghijklmnopqrstuvwxyz");
-  }
-
   public static byte[] getRandBinary(Random r, int len){
     final byte[] bytes = new byte[len];
     for (int j = 0; j < len; j++){

http://git-wip-us.apache.org/repos/asf/hive/blob/fa36381f/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/VectorVerifyFast.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/VectorVerifyFast.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/VectorVerifyFast.java
index 458aae8..a0ba0e1 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/VectorVerifyFast.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/VectorVerifyFast.java
@@ -364,9 +364,9 @@ public class VectorVerifyFast {
       case STRING:
       {
         Text value = (Text) object;
-        byte[] stringBytes = value.getBytes();
-        int stringLength = stringBytes.length;
-        serializeWrite.writeString(stringBytes, 0, stringLength);
+        byte[] bytes = value.getBytes();
+        int byteLength = value.getLength();
+        serializeWrite.writeString(bytes, 0, byteLength);
       }
       break;
       case CHAR:

http://git-wip-us.apache.org/repos/asf/hive/blob/fa36381f/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/aggregation/TestVectorAggregation.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/aggregation/TestVectorAggregation.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/aggregation/TestVectorAggregation.java
index d4ed6b5..211eaa2 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/aggregation/TestVectorAggregation.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/aggregation/TestVectorAggregation.java
@@ -369,7 +369,8 @@ public class TestVectorAggregation extends AggregationBase {
     VectorRandomRowSource mergeRowSource = new VectorRandomRowSource();
 
     mergeRowSource.initGenerationSpecSchema(
-        random, mergeAggrGenerationSpecList, /* maxComplexDepth */ 0, /* allowNull */ false,
+        random, mergeAggrGenerationSpecList, /* maxComplexDepth */ 0,
+        /* allowNull */ false, /* isUnicodeOk */ true,
         mergeDataTypePhysicalVariationList);
 
     Object[][] mergeRandomRows = mergeRowSource.randomRows(TEST_ROW_COUNT);
@@ -508,7 +509,8 @@ public class TestVectorAggregation extends AggregationBase {
 
     boolean allowNull = !aggregationName.equals("bloom_filter");
     partial1RowSource.initGenerationSpecSchema(
-        random, dataAggrGenerationSpecList, /* maxComplexDepth */ 0, allowNull,
+        random, dataAggrGenerationSpecList, /* maxComplexDepth */ 0,
+        allowNull,  /* isUnicodeOk */ true,
         explicitDataTypePhysicalVariationList);
 
     Object[][] partial1RandomRows = partial1RowSource.randomRows(TEST_ROW_COUNT);
@@ -604,7 +606,8 @@ public class TestVectorAggregation extends AggregationBase {
       VectorRandomRowSource completeRowSource = new VectorRandomRowSource();
 
       completeRowSource.initGenerationSpecSchema(
-          random, dataAggrGenerationSpecList, /* maxComplexDepth */ 0, /* allowNull */ true,
+          random, dataAggrGenerationSpecList, /* maxComplexDepth */ 0,
+          /* allowNull */ true, /* isUnicodeOk */ true,
           explicitDataTypePhysicalVariationList);
 
       Object[][] completeRandomRows = completeRowSource.randomRows(TEST_ROW_COUNT);

http://git-wip-us.apache.org/repos/asf/hive/blob/fa36381f/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorArithmetic.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorArithmetic.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorArithmetic.java
index 1b61071..1329d79 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorArithmetic.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorArithmetic.java
@@ -26,9 +26,7 @@ import java.util.List;
 import java.util.Random;
 
 import org.apache.hadoop.hive.common.type.DataTypePhysicalVariation;
-import org.apache.hadoop.hive.common.type.HiveChar;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
-import org.apache.hadoop.hive.common.type.HiveVarchar;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluator;
 import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluatorFactory;
@@ -59,25 +57,16 @@ import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPMinus;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPMod;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPMultiply;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPPlus;
-import org.apache.hadoop.hive.ql.udf.generic.GenericUDF.DeferredJavaObject;
-import org.apache.hadoop.hive.ql.udf.generic.GenericUDF.DeferredObject;
-import org.apache.hadoop.hive.serde2.io.HiveCharWritable;
 import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
-import org.apache.hadoop.hive.serde2.io.HiveVarcharWritable;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption;
 import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory;
-import org.apache.hadoop.hive.serde2.typeinfo.CharTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
-import org.apache.hadoop.hive.serde2.typeinfo.VarcharTypeInfo;
-import org.apache.hadoop.hive.serde2.io.ShortWritable;
-import org.apache.hadoop.io.IntWritable;
-import org.apache.hadoop.io.LongWritable;
 
 import junit.framework.Assert;
 
@@ -437,7 +426,8 @@ public class TestVectorArithmetic {
     VectorRandomRowSource rowSource = new VectorRandomRowSource();
 
     rowSource.initGenerationSpecSchema(
-        random, generationSpecList, /* maxComplexDepth */ 0, /* allowNull */ true,
+        random, generationSpecList, /* maxComplexDepth */ 0,
+        /* allowNull */ true, /* isUnicodeOk */ true,
         explicitDataTypePhysicalVariationList);
 
     Object[][] randomRows = rowSource.randomRows(100000);

http://git-wip-us.apache.org/repos/asf/hive/blob/fa36381f/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorBetweenIn.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorBetweenIn.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorBetweenIn.java
index 5b69bdf..16bb445 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorBetweenIn.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorBetweenIn.java
@@ -453,7 +453,8 @@ public class TestVectorBetweenIn {
     VectorRandomRowSource rowSource = new VectorRandomRowSource();
 
     rowSource.initGenerationSpecSchema(
-        random, generationSpecList, /* maxComplexDepth */ 0, /* allowNull */ true,
+        random, generationSpecList, /* maxComplexDepth */ 0,
+        /* allowNull */ true, /* isUnicodeOk */ true,
         explicitDataTypePhysicalVariationList);
 
     List<String> columns = new ArrayList<String>();
@@ -575,7 +576,8 @@ public class TestVectorBetweenIn {
     VectorRandomRowSource structRowSource = new VectorRandomRowSource();
 
     structRowSource.initGenerationSpecSchema(
-        random, structGenerationSpecList, /* maxComplexDepth */ 0, /* allowNull */ true,
+        random, structGenerationSpecList, /* maxComplexDepth */ 0,
+        /* allowNull */ true, /* isUnicodeOk */ true,
         structExplicitDataTypePhysicalVariationList);
 
     Object[][] structRandomRows = structRowSource.randomRows(100000);
@@ -597,7 +599,8 @@ public class TestVectorBetweenIn {
     VectorRandomRowSource rowSource = new VectorRandomRowSource();
 
     rowSource.initGenerationSpecSchema(
-        random, generationSpecList, /* maxComplexDepth */ 0, /* allowNull */ true,
+        random, generationSpecList, /* maxComplexDepth */ 0,
+        /* allowNull */ true, /* isUnicodeOk */ true,
         explicitDataTypePhysicalVariationList);
 
     Object[][] randomRows = rowSource.randomRows(100000);
@@ -729,7 +732,7 @@ public class TestVectorBetweenIn {
            continue;
          }
       case VECTOR_EXPRESSION:
-        if (!doVectorCastTest(
+        if (!doVectorBetweenInTest(
               typeInfo,
               betweenInVariation,
               compareList,
@@ -866,7 +869,7 @@ public class TestVectorBetweenIn {
     }
   }
 
-  private boolean doVectorCastTest(TypeInfo typeInfo,
+  private boolean doVectorBetweenInTest(TypeInfo typeInfo,
       BetweenInVariation betweenInVariation, List<Object> compareList,
       List<String> columns, String[] columnNames,
       TypeInfo[] typeInfos, DataTypePhysicalVariation[] dataTypePhysicalVariations,
@@ -899,13 +902,24 @@ public class TestVectorBetweenIn {
                 VectorExpressionDescriptor.Mode.PROJECTION));
     vectorExpression.transientInit();
 
-    if (betweenInTestMode == BetweenInTestMode.VECTOR_EXPRESSION &&
-        vectorExpression instanceof VectorUDFAdaptor) {
-      System.out.println(
-          "*NO NATIVE VECTOR EXPRESSION* typeInfo " + typeInfo.toString() +
-          " betweenInTestMode " + betweenInTestMode +
-          " betweenInVariation " + betweenInVariation +
-          " vectorExpression " + vectorExpression.toString());
+    if (betweenInTestMode == BetweenInTestMode.VECTOR_EXPRESSION) {
+      String vecExprString = vectorExpression.toString();
+      if (vectorExpression instanceof VectorUDFAdaptor) {
+        System.out.println(
+            "*NO NATIVE VECTOR EXPRESSION* typeInfo " + typeInfo.toString() +
+            " betweenInTestMode " + betweenInTestMode +
+            " betweenInVariation " + betweenInVariation +
+            " vectorExpression " + vecExprString);
+      } else if (dataTypePhysicalVariations[0] == DataTypePhysicalVariation.DECIMAL_64) {
+        final String nameToCheck = vectorExpression.getClass().getSimpleName();
+        if (!nameToCheck.contains("Decimal64")) {
+          System.out.println(
+              "*EXPECTED DECIMAL_64 VECTOR EXPRESSION* typeInfo " + typeInfo.toString() +
+              " betweenInTestMode " + betweenInTestMode +
+              " betweenInVariation " + betweenInVariation +
+              " vectorExpression " + vecExprString);
+        }
+      }
     }
 
     // System.out.println("*VECTOR EXPRESSION* " + vectorExpression.getClass().getSimpleName());

http://git-wip-us.apache.org/repos/asf/hive/blob/fa36381f/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorCastStatement.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorCastStatement.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorCastStatement.java
index cc1415a..8a68506 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorCastStatement.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorCastStatement.java
@@ -24,8 +24,6 @@ import java.util.List;
 import java.util.Random;
 
 import org.apache.hadoop.hive.common.type.DataTypePhysicalVariation;
-import org.apache.hadoop.hive.common.type.HiveChar;
-import org.apache.hadoop.hive.common.type.HiveVarchar;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluator;
 import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluatorFactory;
@@ -48,23 +46,17 @@ import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDF;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFIf;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFWhen;
-import org.apache.hadoop.hive.serde2.io.DoubleWritable;
-import org.apache.hadoop.hive.serde2.io.HiveCharWritable;
 import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
-import org.apache.hadoop.hive.serde2.io.HiveVarcharWritable;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption;
 import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory;
-import org.apache.hadoop.hive.serde2.typeinfo.CharTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
-import org.apache.hadoop.hive.serde2.typeinfo.VarcharTypeInfo;
 import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.Text;
 
 import junit.framework.Assert;
 
@@ -286,7 +278,8 @@ public class TestVectorCastStatement {
     VectorRandomRowSource rowSource = new VectorRandomRowSource();
 
     rowSource.initGenerationSpecSchema(
-        random, generationSpecList, /* maxComplexDepth */ 0, /* allowNull */ true,
+        random, generationSpecList, /* maxComplexDepth */ 0,
+        /* allowNull */ true, /* isUnicodeOk */ true,
         explicitDataTypePhysicalVariationList);
 
     List<String> columns = new ArrayList<String>();

http://git-wip-us.apache.org/repos/asf/hive/blob/fa36381f/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorCoalesceElt.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorCoalesceElt.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorCoalesceElt.java
index 0bca490..d367fb9 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorCoalesceElt.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorCoalesceElt.java
@@ -54,6 +54,7 @@ import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
+import org.apache.hadoop.io.IntWritable;
 
 import junit.framework.Assert;
 
@@ -66,7 +67,11 @@ public class TestVectorCoalesceElt {
   public void testCoalesce() throws Exception {
     Random random = new Random(5371);
 
-    doCoalesceElt(random, /* isCoalesce */ true, false);
+    // Grind through a few more index values...
+    int iteration = 0;
+    for (int i = 0; i < 10; i++) {
+      iteration =  doCoalesceElt(random, iteration, /* isCoalesce */ true, false);
+    }
   }
 
   @Test
@@ -74,9 +79,10 @@ public class TestVectorCoalesceElt {
     Random random = new Random(5371);
 
     // Grind through a few more index values...
-    for (int i = 0; i < 4; i++) {
-      doCoalesceElt(random, /* isCoalesce */ false, false);
-      doCoalesceElt(random, /* isCoalesce */ false, true);
+    int iteration = 0;
+    for (int i = 0; i < 10; i++) {
+      iteration = doCoalesceElt(random, iteration, /* isCoalesce */ false, false);
+      iteration = doCoalesceElt(random, iteration, /* isCoalesce */ false, true);
     }
   }
 
@@ -88,39 +94,41 @@ public class TestVectorCoalesceElt {
     static final int count = values().length;
   }
 
-  private void doCoalesceElt(Random random, boolean isCoalesce, boolean isEltIndexConst)
-      throws Exception {
+  private int doCoalesceElt(Random random, int iteration, boolean isCoalesce,
+      boolean isEltIndexConst)
+          throws Exception {
 
-    doCoalesceOnRandomDataType(random, isCoalesce, isEltIndexConst, /* columnCount */ 2,
+    doCoalesceOnRandomDataType(random, iteration++, isCoalesce, isEltIndexConst, /* columnCount */ 2,
         /* constantColumns */ null, /* nullConstantColumns */ null, /* allowNulls */ true);
-    doCoalesceOnRandomDataType(random, isCoalesce, isEltIndexConst, /* columnCount */ 2,
+    doCoalesceOnRandomDataType(random, iteration++, isCoalesce, isEltIndexConst, /* columnCount */ 2,
         /* constantColumns */ null, /* nullConstantColumns */ null, /* allowNulls */ false);
 
-    doCoalesceOnRandomDataType(random, isCoalesce, isEltIndexConst, /* columnCount */ 3,
+    doCoalesceOnRandomDataType(random, iteration++, isCoalesce, isEltIndexConst, /* columnCount */ 3,
         /* constantColumns */ null, /* nullConstantColumns */ null, /* allowNulls */ true);
-    doCoalesceOnRandomDataType(random, isCoalesce, isEltIndexConst, /* columnCount */ 3,
+    doCoalesceOnRandomDataType(random, iteration++, isCoalesce, isEltIndexConst, /* columnCount */ 3,
         new int[] { 0 }, /* nullConstantColumns */ null, /* allowNulls */ true);
-    doCoalesceOnRandomDataType(random, isCoalesce, isEltIndexConst, /* columnCount */ 3,
+    doCoalesceOnRandomDataType(random, iteration++, isCoalesce, isEltIndexConst, /* columnCount */ 3,
         new int[] { 0 }, /* nullConstantColumns */ new int[] { 0 }, /* allowNulls */ true);
-    doCoalesceOnRandomDataType(random, isCoalesce, isEltIndexConst, /* columnCount */ 3,
+    doCoalesceOnRandomDataType(random, iteration++, isCoalesce, isEltIndexConst, /* columnCount */ 3,
         new int[] { 1 }, /* nullConstantColumns */ null, /* allowNulls */ true);
-    doCoalesceOnRandomDataType(random, isCoalesce, isEltIndexConst, /* columnCount */ 3,
+    doCoalesceOnRandomDataType(random, iteration++, isCoalesce, isEltIndexConst, /* columnCount */ 3,
         new int[] { 1 }, /* nullConstantColumns */ new int[] { 1 }, /* allowNulls */ true);
-    doCoalesceOnRandomDataType(random, isCoalesce, isEltIndexConst, /* columnCount */ 3,
+    doCoalesceOnRandomDataType(random, iteration++, isCoalesce, isEltIndexConst, /* columnCount */ 3,
         new int[] { 0, 2 }, /* nullConstantColumns */ null, /* allowNulls */ true);
-    doCoalesceOnRandomDataType(random, isCoalesce, isEltIndexConst, /* columnCount */ 3,
+    doCoalesceOnRandomDataType(random, iteration++, isCoalesce, isEltIndexConst, /* columnCount */ 3,
         new int[] { 0, 2 }, /* nullConstantColumns */ new int[] { 0 }, /* allowNulls */ true);
-    doCoalesceOnRandomDataType(random, isCoalesce, isEltIndexConst, /* columnCount */ 3,
+    doCoalesceOnRandomDataType(random, iteration++, isCoalesce, isEltIndexConst, /* columnCount */ 3,
         new int[] { 0, 2 }, /* nullConstantColumns */ new int[] { 0, 2 }, /* allowNulls */ false);
 
-    doCoalesceOnRandomDataType(random, isCoalesce, isEltIndexConst, /* columnCount */ 4,
+    doCoalesceOnRandomDataType(random, iteration++, isCoalesce, isEltIndexConst, /* columnCount */ 4,
         /* constantColumns */ null, /* nullConstantColumns */ null, /* allowNulls */ true);
-    doCoalesceOnRandomDataType(random, isCoalesce, isEltIndexConst, /* columnCount */ 4,
+    doCoalesceOnRandomDataType(random, iteration++, isCoalesce, isEltIndexConst, /* columnCount */ 4,
         /* constantColumns */ null, /* nullConstantColumns */ null, /* allowNulls */ false);
-    doCoalesceOnRandomDataType(random, isCoalesce, isEltIndexConst, /* columnCount */ 4,
+    doCoalesceOnRandomDataType(random, iteration++, isCoalesce, isEltIndexConst, /* columnCount */ 4,
         new int[] { 0, 1, 2 }, /* nullConstantColumns */ new int[] { 0, 1, 2 }, /* allowNulls */ true);
-    doCoalesceOnRandomDataType(random, isCoalesce, isEltIndexConst, /* columnCount */ 4,
+    doCoalesceOnRandomDataType(random, iteration++, isCoalesce, isEltIndexConst, /* columnCount */ 4,
         new int[] { 0, 1, 2 }, /* nullConstantColumns */ new int[] { 0, 1, 2 }, /* allowNulls */ false);
+    return iteration;
   }
 
   private boolean contains(int[] columns, int column) {
@@ -135,7 +143,7 @@ public class TestVectorCoalesceElt {
     return false;
   }
 
-  private boolean doCoalesceOnRandomDataType(Random random,
+  private boolean doCoalesceOnRandomDataType(Random random, int iteration,
       boolean isCoalesce, boolean isEltIndexConst, int columnCount,
       int[] constantColumns, int[] nullConstantColumns, boolean allowNulls)
       throws Exception {
@@ -187,17 +195,18 @@ public class TestVectorCoalesceElt {
 
       List<Object> intValueList = new ArrayList<Object>();
       for (int i = -1; i < columnCount + 2; i++) {
-        intValueList.add(i);
+        intValueList.add(
+            new IntWritable(i));
       }
       final int intValueListCount = intValueList.size();
-      ExprNodeDesc colExpr;
+      ExprNodeDesc intColExpr;
       if (!isEltIndexConst) {
         generationSpecList.add(
             GenerationSpec.createValueList(intTypeInfo, intValueList));
         explicitDataTypePhysicalVariationList.add(DataTypePhysicalVariation.NONE);
         String columnName = "col" + columnNum++;
         columns.add(columnName);
-        colExpr = new ExprNodeColumnDesc(intTypeInfo, columnName, "table", false);
+        intColExpr = new ExprNodeColumnDesc(intTypeInfo, columnName, "table", false);
       } else {
         final Object scalarObject;
         if (random.nextInt(10) != 0) {
@@ -205,8 +214,9 @@ public class TestVectorCoalesceElt {
         } else {
           scalarObject = null;
         }
-        colExpr = new ExprNodeConstantDesc(typeInfo, scalarObject);
+        intColExpr = new ExprNodeConstantDesc(typeInfo, scalarObject);
       }
+      children.add(intColExpr);
     }
     for (int c = 0; c < columnCount; c++) {
       ExprNodeDesc colExpr;
@@ -235,7 +245,8 @@ public class TestVectorCoalesceElt {
     VectorRandomRowSource rowSource = new VectorRandomRowSource();
 
     rowSource.initGenerationSpecSchema(
-        random, generationSpecList, /* maxComplexDepth */ 0, /* allowNull */ allowNulls,
+        random, generationSpecList, /* maxComplexDepth */ 0,
+        /* allowNull */ allowNulls,  /* isUnicodeOk */ true,
         explicitDataTypePhysicalVariationList);
 
     String[] columnNames = columns.toArray(new String[0]);
@@ -295,6 +306,7 @@ public class TestVectorCoalesceElt {
       case VECTOR_EXPRESSION:
         if (!doVectorCastTest(
               typeInfo,
+              iteration,
               columns,
               columnNames,
               rowSource.typeInfos(),
@@ -327,9 +339,10 @@ public class TestVectorCoalesceElt {
                 "Row " + i +
                 " sourceTypeName " + typeName +
                 " " + coalesceEltTestMode +
+                " iteration " + iteration +
                 " result is NULL " + (vectorResult == null ? "YES" : "NO result " + vectorResult.toString()) +
                 " does not match row-mode expected result is NULL " +
-                (expectedResult == null ? "YES" : "NO result " + expectedResult.toString()) +
+                (expectedResult == null ? "YES" : "NO result '" + expectedResult.toString()) + "'" +
                 " row values " + Arrays.toString(randomRows[i]) +
                 " exprDesc " + exprDesc.toString());
           }
@@ -340,9 +353,10 @@ public class TestVectorCoalesceElt {
                 "Row " + i +
                 " sourceTypeName " + typeName +
                 " " + coalesceEltTestMode +
-                " result " + vectorResult.toString() +
+                " iteration " + iteration +
+                " result '" + vectorResult.toString() + "'" +
                 " (" + vectorResult.getClass().getSimpleName() + ")" +
-                " does not match row-mode expected result " + expectedResult.toString() +
+                " does not match row-mode expected result '" + expectedResult.toString() + "'" +
                 " (" + expectedResult.getClass().getSimpleName() + ")" +
                 " row values " + Arrays.toString(randomRows[i]) +
                 " exprDesc " + exprDesc.toString());
@@ -410,7 +424,7 @@ public class TestVectorCoalesceElt {
     }
   }
 
-  private boolean doVectorCastTest(TypeInfo typeInfo,
+  private boolean doVectorCastTest(TypeInfo typeInfo, int iteration,
       List<String> columns, String[] columnNames,
       TypeInfo[] typeInfos, DataTypePhysicalVariation[] dataTypePhysicalVariations,
       List<ExprNodeDesc> children,
@@ -445,7 +459,7 @@ public class TestVectorCoalesceElt {
           " vectorExpression " + vectorExpression.toString());
     }
 
-    System.out.println("*VECTOR EXPRESSION* " + vectorExpression.getClass().getSimpleName());
+    // System.out.println("*VECTOR EXPRESSION* " + vectorExpression.getClass().getSimpleName());
 
     /*
     System.out.println(
@@ -474,17 +488,6 @@ public class TestVectorCoalesceElt {
         new TypeInfo[] { outputTypeInfo }, new int[] { vectorExpression.getOutputColumnNum() });
     Object[] scrqtchRow = new Object[1];
 
-    // System.out.println("*VECTOR EXPRESSION* " + vectorExpression.getClass().getSimpleName());
-
-    /*
-    System.out.println(
-        "*DEBUG* typeInfo1 " + typeInfo1.toString() +
-        " typeInfo2 " + typeInfo2.toString() +
-        " arithmeticTestMode " + arithmeticTestMode +
-        " columnScalarMode " + columnScalarMode +
-        " vectorExpression " + vectorExpression.toString());
-    */
-
     batchSource.resetBatchIteration();
     int rowIndex = 0;
     while (true) {

http://git-wip-us.apache.org/repos/asf/hive/blob/fa36381f/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorDateAddSub.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorDateAddSub.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorDateAddSub.java
index 68c14c8..b0e4b26 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorDateAddSub.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorDateAddSub.java
@@ -25,8 +25,6 @@ import java.util.List;
 import java.util.Random;
 
 import org.apache.hadoop.hive.common.type.DataTypePhysicalVariation;
-import org.apache.hadoop.hive.common.type.HiveChar;
-import org.apache.hadoop.hive.common.type.HiveVarchar;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluator;
 import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluatorFactory;
@@ -46,23 +44,18 @@ import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDF;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFDateAdd;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFDateSub;
-import org.apache.hadoop.hive.serde2.io.HiveCharWritable;
 import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
-import org.apache.hadoop.hive.serde2.io.HiveVarcharWritable;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption;
 import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory;
-import org.apache.hadoop.hive.serde2.typeinfo.CharTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
-import org.apache.hadoop.hive.serde2.typeinfo.VarcharTypeInfo;
 import org.apache.hadoop.hive.serde2.io.ShortWritable;
 import org.apache.hadoop.io.IntWritable;
-import org.apache.hadoop.io.LongWritable;
 
 import junit.framework.Assert;
 
@@ -242,7 +235,8 @@ public class TestVectorDateAddSub {
     VectorRandomRowSource rowSource = new VectorRandomRowSource();
 
     rowSource.initGenerationSpecSchema(
-        random, generationSpecList, /* maxComplexDepth */ 0, /* allowNull */ true,
+        random, generationSpecList, /* maxComplexDepth */ 0,
+        /* allowNull */ true, /* isUnicodeOk */ true,
         explicitDataTypePhysicalVariationList);
 
     Object[][] randomRows = rowSource.randomRows(100000);

http://git-wip-us.apache.org/repos/asf/hive/blob/fa36381f/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorDateDiff.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorDateDiff.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorDateDiff.java
index 0da9d8c..d89299c 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorDateDiff.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorDateDiff.java
@@ -25,8 +25,6 @@ import java.util.List;
 import java.util.Random;
 
 import org.apache.hadoop.hive.common.type.DataTypePhysicalVariation;
-import org.apache.hadoop.hive.common.type.HiveChar;
-import org.apache.hadoop.hive.common.type.HiveVarchar;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluator;
 import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluatorFactory;
@@ -54,16 +52,12 @@ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption;
 import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory;
-import org.apache.hadoop.hive.serde2.typeinfo.CharTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
-import org.apache.hadoop.hive.serde2.typeinfo.VarcharTypeInfo;
 import org.apache.hadoop.hive.serde2.io.ShortWritable;
-import org.apache.hadoop.io.IntWritable;
-import org.apache.hadoop.io.LongWritable;
 
 import junit.framework.Assert;
 
@@ -251,7 +245,8 @@ public class TestVectorDateDiff {
     VectorRandomRowSource rowSource = new VectorRandomRowSource();
 
     rowSource.initGenerationSpecSchema(
-        random, generationSpecList, /* maxComplexDepth */ 0, /* allowNull */ true,
+        random, generationSpecList, /* maxComplexDepth */ 0,
+        /* allowNull */ true, /* isUnicodeOk */ true,
         explicitDataTypePhysicalVariationList);
 
     Object[][] randomRows = rowSource.randomRows(100000);

http://git-wip-us.apache.org/repos/asf/hive/blob/fa36381f/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorFilterCompare.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorFilterCompare.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorFilterCompare.java
index ba9eaca..abdbc1c 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorFilterCompare.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorFilterCompare.java
@@ -26,9 +26,7 @@ import java.util.List;
 import java.util.Random;
 
 import org.apache.hadoop.hive.common.type.DataTypePhysicalVariation;
-import org.apache.hadoop.hive.common.type.HiveChar;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
-import org.apache.hadoop.hive.common.type.HiveVarchar;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluator;
 import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluatorFactory;
@@ -60,12 +58,8 @@ import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPLessThan;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqualOrGreaterThan;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqualOrLessThan;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqual;
-import org.apache.hadoop.hive.ql.udf.generic.GenericUDF.DeferredJavaObject;
-import org.apache.hadoop.hive.ql.udf.generic.GenericUDF.DeferredObject;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPNotEqual;
-import org.apache.hadoop.hive.serde2.io.HiveCharWritable;
 import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
-import org.apache.hadoop.hive.serde2.io.HiveVarcharWritable;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption;
@@ -77,9 +71,6 @@ import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
 import org.apache.hadoop.hive.serde2.typeinfo.VarcharTypeInfo;
-import org.apache.hadoop.hive.serde2.io.ShortWritable;
-import org.apache.hadoop.io.IntWritable;
-import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.BooleanWritable;
 
 import junit.framework.Assert;
@@ -402,7 +393,8 @@ public class TestVectorFilterCompare {
     VectorRandomRowSource rowSource = new VectorRandomRowSource();
 
     rowSource.initGenerationSpecSchema(
-        random, generationSpecList, /* maxComplexDepth */ 0, /* allowNull */ true,
+        random, generationSpecList, /* maxComplexDepth */ 0,
+        /* allowNull */ true, /* isUnicodeOk */ true,
         explicitDataTypePhysicalVariationList);
 
     Object[][] randomRows = rowSource.randomRows(100000);

http://git-wip-us.apache.org/repos/asf/hive/blob/fa36381f/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorIfStatement.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorIfStatement.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorIfStatement.java
index d8ae175..ce66e2b 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorIfStatement.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorIfStatement.java
@@ -273,7 +273,8 @@ public class TestVectorIfStatement {
     VectorRandomRowSource rowSource = new VectorRandomRowSource();
 
     rowSource.initExplicitSchema(
-        random, explicitTypeNameList, /* maxComplexDepth */ 0, /* allowNull */ true,
+        random, explicitTypeNameList, /* maxComplexDepth */ 0,
+        /* allowNull */ true, /* isUnicodeOk */ true,
         explicitDataTypePhysicalVariationList);
 
     List<String> columns = new ArrayList<String>();

http://git-wip-us.apache.org/repos/asf/hive/blob/fa36381f/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorIndex.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorIndex.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorIndex.java
index 648feb0..9a49088 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorIndex.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorIndex.java
@@ -28,8 +28,6 @@ import java.util.TreeSet;
 import java.util.stream.IntStream;
 
 import org.apache.hadoop.hive.common.type.DataTypePhysicalVariation;
-import org.apache.hadoop.hive.common.type.HiveChar;
-import org.apache.hadoop.hive.common.type.HiveVarchar;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluator;
 import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluatorFactory;
@@ -306,7 +304,8 @@ public class TestVectorIndex {
     VectorRandomRowSource rowSource = new VectorRandomRowSource();
 
     rowSource.initGenerationSpecSchema(
-        random, generationSpecList, /* maxComplexDepth */ 0, /* allowNull */ allowNulls,
+        random, generationSpecList, /* maxComplexDepth */ 0,
+        /* allowNull */ allowNulls,  /* isUnicodeOk */ true,
         explicitDataTypePhysicalVariationList);
 
     String[] columnNames = columns.toArray(new String[0]);

http://git-wip-us.apache.org/repos/asf/hive/blob/fa36381f/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorNegative.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorNegative.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorNegative.java
index ea39848..a3f665b 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorNegative.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorNegative.java
@@ -51,34 +51,16 @@ import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDF;
-import org.apache.hadoop.hive.ql.udf.generic.GenericUDFDateAdd;
-import org.apache.hadoop.hive.ql.udf.generic.GenericUDFDateDiff;
-import org.apache.hadoop.hive.ql.udf.generic.GenericUDFDateSub;
-import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPDivide;
-import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPMinus;
-import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPMod;
-import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPMultiply;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPNegative;
-import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPPlus;
-import org.apache.hadoop.hive.ql.udf.generic.GenericUDF.DeferredJavaObject;
-import org.apache.hadoop.hive.ql.udf.generic.GenericUDF.DeferredObject;
-import org.apache.hadoop.hive.serde2.io.HiveCharWritable;
-import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
-import org.apache.hadoop.hive.serde2.io.HiveVarcharWritable;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption;
 import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory;
-import org.apache.hadoop.hive.serde2.typeinfo.CharTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
-import org.apache.hadoop.hive.serde2.typeinfo.VarcharTypeInfo;
-import org.apache.hadoop.hive.serde2.io.ShortWritable;
-import org.apache.hadoop.io.IntWritable;
-import org.apache.hadoop.io.LongWritable;
 
 import junit.framework.Assert;
 
@@ -219,7 +201,8 @@ public class TestVectorNegative {
     VectorRandomRowSource rowSource = new VectorRandomRowSource();
 
     rowSource.initGenerationSpecSchema(
-        random, generationSpecList, /* maxComplexDepth */ 0, /* allowNull */ true,
+        random, generationSpecList, /* maxComplexDepth */ 0,
+        /* allowNull */ true, /* isUnicodeOk */ true,
         explicitDataTypePhysicalVariationList);
 
     Object[][] randomRows = rowSource.randomRows(100000);

http://git-wip-us.apache.org/repos/asf/hive/blob/fa36381f/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorNull.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorNull.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorNull.java
index 9b0a2ae..06d8fc8 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorNull.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorNull.java
@@ -24,8 +24,6 @@ import java.util.List;
 import java.util.Random;
 
 import org.apache.hadoop.hive.common.type.DataTypePhysicalVariation;
-import org.apache.hadoop.hive.common.type.HiveChar;
-import org.apache.hadoop.hive.common.type.HiveVarchar;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluator;
 import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluatorFactory;
@@ -52,26 +50,17 @@ import org.apache.hadoop.hive.ql.udf.generic.GenericUDF;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPNot;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPNotNull;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPNull;
-import org.apache.hadoop.hive.serde2.io.DoubleWritable;
-import org.apache.hadoop.hive.serde2.io.HiveCharWritable;
-import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
-import org.apache.hadoop.hive.serde2.io.HiveVarcharWritable;
 import org.apache.hadoop.hive.serde2.objectinspector.ConstantObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption;
 import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
-import org.apache.hadoop.hive.serde2.typeinfo.CharTypeInfo;
-import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
-import org.apache.hadoop.hive.serde2.typeinfo.VarcharTypeInfo;
 import org.apache.hadoop.io.BooleanWritable;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.Text;
 
 import junit.framework.Assert;
 
@@ -159,7 +148,8 @@ public class TestVectorNull {
     VectorRandomRowSource rowSource = new VectorRandomRowSource();
 
     rowSource.initGenerationSpecSchema(
-        random, generationSpecList, /* maxComplexDepth */ 0, /* allowNull */ true,
+        random, generationSpecList, /* maxComplexDepth */ 0,
+        /* allowNull */ true, /* isUnicodeOk */ true,
         explicitDataTypePhysicalVariationList);
 
     List<String> columns = new ArrayList<String>();

http://git-wip-us.apache.org/repos/asf/hive/blob/fa36381f/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorStringConcat.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorStringConcat.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorStringConcat.java
index 8877b06..bb41c4f 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorStringConcat.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorStringConcat.java
@@ -191,7 +191,8 @@ public class TestVectorStringConcat {
     VectorRandomRowSource rowSource = new VectorRandomRowSource();
 
     rowSource.initGenerationSpecSchema(
-        random, generationSpecList, /* maxComplexDepth */ 0, /* allowNull */ true,
+        random, generationSpecList, /* maxComplexDepth */ 0,
+        /* allowNull */ true, /* isUnicodeOk */ true,
         explicitDataTypePhysicalVariationList);
 
     Object[][] randomRows = rowSource.randomRows(100000);

http://git-wip-us.apache.org/repos/asf/hive/blob/fa36381f/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorStringUnary.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorStringUnary.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorStringUnary.java
index dd53157..4099b45 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorStringUnary.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorStringUnary.java
@@ -149,7 +149,8 @@ public class TestVectorStringUnary {
     VectorRandomRowSource rowSource = new VectorRandomRowSource();
 
     rowSource.initGenerationSpecSchema(
-        random, generationSpecList, /* maxComplexDepth */ 0, /* allowNull */ true,
+        random, generationSpecList, /* maxComplexDepth */ 0,
+        /* allowNull */ true, /* isUnicodeOk */ true,
         explicitDataTypePhysicalVariationList);
 
     List<ExprNodeDesc> children = new ArrayList<ExprNodeDesc>();