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/07/19 07:12:46 UTC

hive git commit: HIVE-20197: Vectorization: Add DECIMAL_64 testing, add Date/Interval/Timestamp arithmetic, and add more GROUP BY Aggregation tests (Matt McCline, reviewed by Teddy Choi)

Repository: hive
Updated Branches:
  refs/heads/master d5c589bd9 -> aa1f0de85


HIVE-20197: Vectorization: Add DECIMAL_64 testing, add Date/Interval/Timestamp arithmetic, and add more GROUP BY Aggregation tests (Matt McCline, reviewed by Teddy Choi)


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

Branch: refs/heads/master
Commit: aa1f0de85dd63a0951508b912bbae19288944e5b
Parents: d5c589b
Author: Matt McCline <mm...@hortonworks.com>
Authored: Thu Jul 19 02:12:40 2018 -0500
Committer: Matt McCline <mm...@hortonworks.com>
Committed: Thu Jul 19 02:12:40 2018 -0500

----------------------------------------------------------------------
 ...eColumnArithmeticIntervalYearMonthColumn.txt |   3 +-
 ...YearMonthColumnArithmeticTimestampColumn.txt |   4 +-
 .../hive/ql/exec/vector/VectorAssignRow.java    |  20 ++-
 .../hive/ql/udf/generic/GenericUDAFCount.java   |   6 +-
 .../ql/exec/vector/VectorRandomRowSource.java   |  35 ++---
 .../vector/aggregation/AggregationBase.java     |  22 ++-
 .../aggregation/TestVectorAggregation.java      | 133 ++++++++++++++---
 .../expressions/TestVectorArithmetic.java       | 143 +++++++++++++++++--
 .../expressions/TestVectorCastStatement.java    |   2 +
 .../expressions/TestVectorDateAddSub.java       |   4 +
 .../vector/expressions/TestVectorDateDiff.java  |   4 +
 .../expressions/TestVectorIfStatement.java      |   2 +
 .../vector/expressions/TestVectorNegative.java  |   2 +
 .../expressions/TestVectorStringConcat.java     |   2 +
 .../expressions/TestVectorStringUnary.java      |   2 +
 .../vector/expressions/TestVectorSubStr.java    |   2 +
 .../expressions/TestVectorTimestampExtract.java |   3 +
 17 files changed, 321 insertions(+), 68 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/aa1f0de8/ql/src/gen/vectorization/ExpressionTemplates/DateColumnArithmeticIntervalYearMonthColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/DateColumnArithmeticIntervalYearMonthColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/DateColumnArithmeticIntervalYearMonthColumn.txt
index c3982ed..68d0a04 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/DateColumnArithmeticIntervalYearMonthColumn.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/DateColumnArithmeticIntervalYearMonthColumn.txt
@@ -123,11 +123,11 @@ public class <ClassName> extends VectorExpression {
         }
       }
     } else if (inputColVector2.isRepeating) {
+      scratchIntervalYearMonth2.set((int) vector2[0]);
       if (batch.selectedInUse) {
         for(int j = 0; j != n; j++) {
           int i = sel[j];
           scratchDate1.setTime(DateWritableV2.daysToMillis((int) vector1[i]));
-          scratchIntervalYearMonth2.set((int) vector2[i]);
           dtm.<OperatorMethod>(
               scratchDate1, scratchIntervalYearMonth2,  outputDate);
           outputVector[i] = DateWritableV2.dateToDays(outputDate);
@@ -135,7 +135,6 @@ public class <ClassName> extends VectorExpression {
       } else {
         for(int i = 0; i != n; i++) {
           scratchDate1.setTime(DateWritableV2.daysToMillis((int) vector1[i]));
-          scratchIntervalYearMonth2.set((int) vector2[i]);
           dtm.<OperatorMethod>(
               scratchDate1, scratchIntervalYearMonth2,  outputDate);
           outputVector[i] = DateWritableV2.dateToDays(outputDate);

http://git-wip-us.apache.org/repos/asf/hive/blob/aa1f0de8/ql/src/gen/vectorization/ExpressionTemplates/IntervalYearMonthColumnArithmeticTimestampColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/IntervalYearMonthColumnArithmeticTimestampColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/IntervalYearMonthColumnArithmeticTimestampColumn.txt
index 08e033a..a112e8f 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/IntervalYearMonthColumnArithmeticTimestampColumn.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/IntervalYearMonthColumnArithmeticTimestampColumn.txt
@@ -122,14 +122,14 @@ public class <ClassName> extends VectorExpression {
           int i = sel[j];
           scratchIntervalYearMonth1.set((int) vector1[i]);
           dtm.<OperatorMethod>(
-              scratchIntervalYearMonth1, inputColVector2.asScratchTimestamp(i), outputColVector.getScratchTimestamp());
+              scratchIntervalYearMonth1, value2, outputColVector.getScratchTimestamp());
           outputColVector.setFromScratchTimestamp(i);
         }
       } else {
         for(int i = 0; i != n; i++) {
           scratchIntervalYearMonth1.set((int) vector1[i]);
           dtm.<OperatorMethod>(
-              scratchIntervalYearMonth1, inputColVector2.asScratchTimestamp(i), outputColVector.getScratchTimestamp());
+              scratchIntervalYearMonth1, value2, outputColVector.getScratchTimestamp());
           outputColVector.setFromScratchTimestamp(i);
         }
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/aa1f0de8/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorAssignRow.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorAssignRow.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorAssignRow.java
index d1dcad9..fc675c5 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorAssignRow.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorAssignRow.java
@@ -514,12 +514,22 @@ public class VectorAssignRow {
           }
           break;
         case DECIMAL:
-          if (object instanceof HiveDecimal) {
-            ((DecimalColumnVector) columnVector).set(
-                batchIndex, (HiveDecimal) object);
+          if (columnVector instanceof DecimalColumnVector) {
+            if (object instanceof HiveDecimal) {
+              ((DecimalColumnVector) columnVector).set(
+                  batchIndex, (HiveDecimal) object);
+            } else {
+              ((DecimalColumnVector) columnVector).set(
+                  batchIndex, (HiveDecimalWritable) object);
+            }
           } else {
-            ((DecimalColumnVector) columnVector).set(
-                batchIndex, (HiveDecimalWritable) object);
+            if (object instanceof HiveDecimal) {
+              ((Decimal64ColumnVector) columnVector).set(
+                  batchIndex, (HiveDecimal) object);
+            } else {
+              ((Decimal64ColumnVector) columnVector).set(
+                  batchIndex, (HiveDecimalWritable) object);
+            }
           }
           break;
         case INTERVAL_YEAR_MONTH:

http://git-wip-us.apache.org/repos/asf/hive/blob/aa1f0de8/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCount.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCount.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCount.java
index b025122..2d7cc8d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCount.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCount.java
@@ -118,10 +118,14 @@ public class GenericUDAFCount implements GenericUDAFResolver2 {
       this.isWindowing = isWindowing;
     }
 
-    private void setCountAllColumns(boolean countAllCols) {
+    public void setCountAllColumns(boolean countAllCols) {
       countAllColumns = countAllCols;
     }
 
+    public boolean getCountAllColumns() {
+      return countAllColumns;
+    }
+
     private void setCountDistinct(boolean countDistinct) {
       this.countDistinct = countDistinct;
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/aa1f0de8/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 a1cefaa..b6ae7d2 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
@@ -654,7 +654,11 @@ public class VectorRandomRowSource {
       dataTypePhysicalVariations[c] = dataTypePhysicalVariation;
       final Category category = typeInfo.getCategory();
       categories[c] = category;
-      ObjectInspector objectInspector = getObjectInspector(typeInfo, dataTypePhysicalVariation);
+
+      // Do not represent DECIMAL_64 to make ROW mode tests easier --
+      // make the VECTOR mode tests convert into the VectorizedRowBatch.
+      ObjectInspector objectInspector = getObjectInspector(typeInfo, DataTypePhysicalVariation.NONE);
+
       switch (category) {
       case PRIMITIVE:
         {
@@ -1161,24 +1165,14 @@ public class VectorRandomRowSource {
       }
     case DECIMAL:
       {
-        if (dataTypePhysicalVariation == dataTypePhysicalVariation.DECIMAL_64) {
-          final long value;
-          if (object instanceof HiveDecimal) {
-            DecimalTypeInfo decimalTypeInfo = (DecimalTypeInfo) primitiveTypeInfo;
-            value = new HiveDecimalWritable((HiveDecimal) object).serialize64(
-                decimalTypeInfo.getScale());
-          } else {
-            value = (long) object;
-          }
-          return ((WritableLongObjectInspector) objectInspector).create(value);
+        // Do not represent DECIMAL_64 to make ROW mode tests easier --
+        // make the VECTOR mode tests convert into the VectorizedRowBatch.
+        WritableHiveDecimalObjectInspector writableOI =
+            new WritableHiveDecimalObjectInspector((DecimalTypeInfo) primitiveTypeInfo);
+        if (object instanceof HiveDecimal) {
+          return writableOI.create((HiveDecimal) object);
         } else {
-          WritableHiveDecimalObjectInspector writableOI =
-              new WritableHiveDecimalObjectInspector((DecimalTypeInfo) primitiveTypeInfo);
-          if (object instanceof HiveDecimal) {
-            return writableOI.create((HiveDecimal) object);
-          } else {
-            return writableOI.copyObject(object);
-          }
+          return writableOI.copyObject(object);
         }
       }
     default:
@@ -1501,11 +1495,10 @@ public class VectorRandomRowSource {
       return getRandIntervalDayTime(r);
     case DECIMAL:
       {
+        // Do not represent DECIMAL_64 to make ROW mode tests easier --
+        // make the VECTOR mode tests convert into the VectorizedRowBatch.
         DecimalTypeInfo decimalTypeInfo = (DecimalTypeInfo) primitiveTypeInfo;
         HiveDecimal hiveDecimal = getRandHiveDecimal(r, decimalTypeInfo);
-        if (dataTypePhysicalVariation == DataTypePhysicalVariation.DECIMAL_64) {
-          return new HiveDecimalWritable(hiveDecimal).serialize64(decimalTypeInfo.getScale());
-        }
         return hiveDecimal;
       }
     default:

http://git-wip-us.apache.org/repos/asf/hive/blob/aa1f0de8/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/aggregation/AggregationBase.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/aggregation/AggregationBase.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/aggregation/AggregationBase.java
index 583241c..f75ab37 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/aggregation/AggregationBase.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/aggregation/AggregationBase.java
@@ -41,6 +41,8 @@ import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.optimizer.physical.Vectorizer;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFCount;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFCount.GenericUDAFCountEvaluator;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFResolver;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator.AggregationBuffer;
@@ -82,6 +84,8 @@ public class AggregationBase {
       Object[] results)
           throws Exception {
 
+    // System.out.println("*ROW AGGREGATION EXPRESSION* " + evaluator.getClass().getSimpleName());
+
     /*
     System.out.println(
         "*DEBUG* typeInfo " + typeInfo.toString() +
@@ -95,7 +99,14 @@ public class AggregationBase {
     ObjectInspector objectInspector = TypeInfoUtils
         .getStandardWritableObjectInspectorFromTypeInfo(outputTypeInfo);
 
-    Object[] parameterArray = new Object[1];
+    final boolean isCountStar;
+    if (evaluator instanceof GenericUDAFCountEvaluator) {
+      GenericUDAFCountEvaluator countEvaluator = (GenericUDAFCountEvaluator) evaluator;
+      isCountStar = countEvaluator.getCountAllColumns();
+    } else {
+      isCountStar = false;
+    }
+    final Object[] parameterArray = isCountStar ? new Object[0] : new Object[1];
     final int rowCount = randomRows.length;
     for (int i = 0; i < rowCount; i++) {
       Object[] row = randomRows[i];
@@ -112,7 +123,9 @@ public class AggregationBase {
         aggregationBuffer = evaluator.getNewAggregationBuffer();
         aggregationBuffers[key] = aggregationBuffer;
       }
-      parameterArray[0] = row[1];
+      if (!isCountStar) {
+        parameterArray[0] = row[1];
+      }
       evaluator.aggregate(aggregationBuffer, parameterArray);
     }
 
@@ -231,6 +244,8 @@ public class AggregationBase {
     }
     VectorExpression.doTransientInit(vecAggrExpr.getInputExpression());
 
+    // System.out.println("*VECTOR AGGREGATION EXPRESSION* " + vecAggrExpr.getClass().getSimpleName());
+
     /*
     System.out.println(
         "*DEBUG* typeInfo " + typeInfo.toString() +
@@ -316,7 +331,7 @@ public class AggregationBase {
         new VectorizedRowBatchCtx(
             outputColumnNames,
             outputTypeInfos,
-            null,
+            new DataTypePhysicalVariation[] { vecAggrExpr.getOutputDataTypePhysicalVariation() },
             /* dataColumnNums */ null,
             /* partitionColumnCount */ 0,
             /* virtualColumnCount */ 0,
@@ -374,6 +389,7 @@ public class AggregationBase {
       int maxKeyCount, List<String> columns, String[] columnNames,
       List<ExprNodeDesc> parameters, Object[][] randomRows,
       VectorRandomRowSource rowSource, VectorRandomBatchSource batchSource,
+      boolean tryDecimal64,
       Object[] resultsArray)
           throws Exception {
 

http://git-wip-us.apache.org/repos/asf/hive/blob/aa1f0de8/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 c5f0483..d4ed6b5 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
@@ -22,6 +22,7 @@ import java.lang.reflect.Constructor;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Random;
@@ -35,6 +36,7 @@ import org.apache.hadoop.hive.ql.exec.vector.VectorRandomRowSource.GenerationSpe
 import org.apache.hadoop.hive.ql.exec.vector.expressions.aggregates.VectorAggregateExpression;
 import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFCount.GenericUDAFCountEvaluator;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFVariance;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator.AggregationBuffer;
@@ -46,6 +48,7 @@ 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.HiveDecimalWritable;
 import org.apache.hadoop.hive.serde2.io.ShortWritable;
 
 import junit.framework.Assert;
@@ -73,7 +76,14 @@ public class TestVectorAggregation extends AggregationBase {
   public void testAvgDecimal() throws Exception {
     Random random = new Random(7743);
 
-    doDecimalTests("avg", random);
+    doDecimalTests("avg", random, /* tryDecimal64 */ false);
+  }
+
+  @Test
+  public void testAvgDecimal64() throws Exception {
+    Random random = new Random(7743);
+
+    doDecimalTests("avg", random, /* tryDecimal64 */ true);
   }
 
   @Test
@@ -101,12 +111,29 @@ public class TestVectorAggregation extends AggregationBase {
   }
 
   @Test
+  public void testCountStar() throws Exception {
+    Random random = new Random(7743);
+
+    doTests(
+        random, "count", TypeInfoFactory.shortTypeInfo, true, false);
+    doTests(
+        random, "count", TypeInfoFactory.longTypeInfo, true, false);
+    doTests(
+        random, "count", TypeInfoFactory.doubleTypeInfo, true, false);
+    doTests(
+        random, "count", new DecimalTypeInfo(18, 10), true, false);
+    doTests(
+        random, "count", TypeInfoFactory.stringTypeInfo, true, false);
+  }
+
+  @Test
   public void testMax() throws Exception {
     Random random = new Random(7743);
 
     doIntegerTests("max", random);
     doFloatingTests("max", random);
-    doDecimalTests("max", random);
+    doDecimalTests("max", random, /* tryDecimal64 */ false);
+    doDecimalTests("max", random, /* tryDecimal64 */ true);
 
     doTests(
         random, "max", TypeInfoFactory.timestampTypeInfo);
@@ -122,7 +149,8 @@ public class TestVectorAggregation extends AggregationBase {
 
     doIntegerTests("min", random);
     doFloatingTests("min", random);
-    doDecimalTests("min", random);
+    doDecimalTests("min", random, /* tryDecimal64 */ false);
+    doDecimalTests("min", random, /* tryDecimal64 */ true);
 
     doTests(
         random, "min", TypeInfoFactory.timestampTypeInfo);
@@ -143,11 +171,36 @@ public class TestVectorAggregation extends AggregationBase {
     doTests(
         random, "sum", TypeInfoFactory.doubleTypeInfo);
 
-    doDecimalTests("sum", random);
+    doDecimalTests("sum", random, /* tryDecimal64 */ false);
+    doDecimalTests("sum", random, /* tryDecimal64 */ true);
+
+    doTests(
+        random, "sum", TypeInfoFactory.timestampTypeInfo);
   }
 
-  private final static Set<String> varianceNames =
-      GenericUDAFVariance.VarianceKind.nameMap.keySet();
+  @Ignore
+  @Test
+  public void testBloomFilter() throws Exception {
+    Random random = new Random(7743);
+
+    doIntegerTests("bloom_filter", random);
+    doFloatingTests("bloom_filter", random);
+    doDecimalTests("bloom_filter", random, /* tryDecimal64 */ false);
+
+    doTests(
+        random, "bloom_filter", TypeInfoFactory.timestampTypeInfo);
+
+    doStringFamilyTests("bloom_filter", random);
+  }
+
+  private final static Set<String> varianceNames = new HashSet<String>();
+  static {
+    // Don't include synonyms.
+    varianceNames.add("variance");
+    varianceNames.add("var_samp");
+    varianceNames.add("std");
+    varianceNames.add("stddev_samp");
+  }
 
   @Test
   public void testVarianceIntegers() throws Exception {
@@ -172,7 +225,17 @@ public class TestVectorAggregation extends AggregationBase {
     Random random = new Random(7743);
 
     for (String aggregationName : varianceNames) {
-      doDecimalTests(aggregationName, random);
+      doDecimalTests(aggregationName, random, /* tryDecimal64 */ false);
+    }
+  }
+
+  @Test
+  public void testVarianceTimestamp() throws Exception {
+    Random random = new Random(7743);
+
+    for (String aggregationName : varianceNames) {
+      doTests(
+          random, aggregationName, TypeInfoFactory.timestampTypeInfo);
     }
   }
 
@@ -215,11 +278,12 @@ public class TestVectorAggregation extends AggregationBase {
     new DecimalTypeInfo(7, 1)
   };
 
-  private void doDecimalTests(String aggregationName, Random random)
-      throws Exception {
+  private void doDecimalTests(String aggregationName, Random random,
+      boolean tryDecimal64)
+          throws Exception {
     for (TypeInfo typeInfo : decimalTypeInfos) {
       doTests(
-          random, aggregationName, typeInfo);
+          random, aggregationName, typeInfo, /* isCountStar */ false, tryDecimal64);
     }
   }
 
@@ -239,6 +303,15 @@ public class TestVectorAggregation extends AggregationBase {
     }
   }
 
+  private boolean checkDecimal64(boolean tryDecimal64, TypeInfo typeInfo) {
+    if (!tryDecimal64 || !(typeInfo instanceof DecimalTypeInfo)) {
+      return false;
+    }
+    DecimalTypeInfo decimalTypeInfo = (DecimalTypeInfo) typeInfo;
+    boolean result = HiveDecimalWritable.isPrecisionDecimal64(decimalTypeInfo.getPrecision());
+    return result;
+  }
+
   public static int getLinearRandomNumber(Random random, int maxSize) {
     //Get a linearly multiplied random number
     int randomMultiplier = maxSize * (maxSize + 1) / 2;
@@ -366,6 +439,7 @@ public class TestVectorAggregation extends AggregationBase {
         mergeRandomRows,
         mergeRowSource,
         mergeBatchSource,
+        /* tryDecimal64 */ false,
         mergeResultsArray);
 
     verifyAggregationResults(
@@ -378,6 +452,12 @@ public class TestVectorAggregation extends AggregationBase {
 
   private void doTests(Random random, String aggregationName, TypeInfo typeInfo)
       throws Exception {
+    doTests(random, aggregationName, typeInfo, false, false);
+  }
+
+  private void doTests(Random random, String aggregationName, TypeInfo typeInfo,
+      boolean isCountStar, boolean tryDecimal64)
+          throws Exception {
 
     List<GenerationSpec> dataAggrGenerationSpecList = new ArrayList<GenerationSpec>();
     List<DataTypePhysicalVariation> explicitDataTypePhysicalVariationList =
@@ -388,9 +468,13 @@ public class TestVectorAggregation extends AggregationBase {
     dataAggrGenerationSpecList.add(keyGenerationSpec);
     explicitDataTypePhysicalVariationList.add(DataTypePhysicalVariation.NONE);
 
+    final boolean decimal64Enable = checkDecimal64(tryDecimal64, typeInfo);
     GenerationSpec generationSpec = GenerationSpec.createSameType(typeInfo);
     dataAggrGenerationSpecList.add(generationSpec);
-    explicitDataTypePhysicalVariationList.add(DataTypePhysicalVariation.NONE);
+    explicitDataTypePhysicalVariationList.add(
+        decimal64Enable ?
+            DataTypePhysicalVariation.DECIMAL_64 :
+            DataTypePhysicalVariation.NONE);
 
     List<String> columns = new ArrayList<String>();
     columns.add("col0");
@@ -398,7 +482,9 @@ public class TestVectorAggregation extends AggregationBase {
 
     ExprNodeColumnDesc dataAggrCol1Expr = new ExprNodeColumnDesc(typeInfo, "col1", "table", false);
     List<ExprNodeDesc> dataAggrParameters = new ArrayList<ExprNodeDesc>();
-    dataAggrParameters.add(dataAggrCol1Expr);
+    if (!isCountStar) {
+      dataAggrParameters.add(dataAggrCol1Expr);
+    }
     final int dataAggrParameterCount = dataAggrParameters.size();
     ObjectInspector[] dataAggrParameterObjectInspectors = new ObjectInspector[dataAggrParameterCount];
     for (int i = 0; i < dataAggrParameterCount; i++) {
@@ -420,8 +506,9 @@ public class TestVectorAggregation extends AggregationBase {
 
     VectorRandomRowSource partial1RowSource = new VectorRandomRowSource();
 
+    boolean allowNull = !aggregationName.equals("bloom_filter");
     partial1RowSource.initGenerationSpecSchema(
-        random, dataAggrGenerationSpecList, /* maxComplexDepth */ 0, /* allowNull */ true,
+        random, dataAggrGenerationSpecList, /* maxComplexDepth */ 0, allowNull,
         explicitDataTypePhysicalVariationList);
 
     Object[][] partial1RandomRows = partial1RowSource.randomRows(TEST_ROW_COUNT);
@@ -441,6 +528,11 @@ public class TestVectorAggregation extends AggregationBase {
             null);
 
     GenericUDAFEvaluator partial1Evaluator = getEvaluator(aggregationName, typeInfo);
+    if (isCountStar) {
+      Assert.assertTrue(partial1Evaluator instanceof GenericUDAFCountEvaluator);
+      GenericUDAFCountEvaluator countEvaluator = (GenericUDAFCountEvaluator) partial1Evaluator;
+      countEvaluator.setCountAllColumns(true);
+    }
 
     /*
     System.out.println(
@@ -473,6 +565,7 @@ public class TestVectorAggregation extends AggregationBase {
         partial1RandomRows,
         partial1RowSource,
         partial1BatchSource,
+        tryDecimal64,
         partial1ResultsArray);
 
     verifyAggregationResults(
@@ -488,16 +581,9 @@ public class TestVectorAggregation extends AggregationBase {
     } else {
       switch (aggregationName) {
       case "avg":
-        /*
-        if (typeInfo instanceof DecimalTypeInfo) {
-          // UNDONE: Row-mode GenericUDAFAverage does not call enforcePrecisionScale...
-          hasDifferentCompleteExpr = false;
-        } else {
-          hasDifferentCompleteExpr = true;
-        }
-        */
         hasDifferentCompleteExpr = true;
         break;
+      case "bloom_filter":
       case "count":
       case "max":
       case "min":
@@ -570,6 +656,7 @@ public class TestVectorAggregation extends AggregationBase {
           completeRandomRows,
           completeRowSource,
           completeBatchSource,
+          tryDecimal64,
           completeResultsArray);
 
       verifyAggregationResults(
@@ -588,6 +675,7 @@ public class TestVectorAggregation extends AggregationBase {
       case "avg":
         hasDifferentPartial2Expr = true;
         break;
+      case "bloom_filter":
       case "count":
       case "max":
       case "min":
@@ -599,7 +687,7 @@ public class TestVectorAggregation extends AggregationBase {
       }
     }
 
-    if (hasDifferentPartial2Expr && false) {
+    if (hasDifferentPartial2Expr) {
 
       /*
        * PARTIAL2.
@@ -628,6 +716,7 @@ public class TestVectorAggregation extends AggregationBase {
       case "avg":
         hasDifferentFinalExpr = true;
         break;
+      case "bloom_filter":
       case "count":
         hasDifferentFinalExpr = true;
         break;

http://git-wip-us.apache.org/repos/asf/hive/blob/aa1f0de8/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 36a8652..eb85823 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
@@ -114,10 +114,58 @@ public class TestVectorArithmetic {
   }
 
   @Test
-  public void testDecimals() throws Exception {
+  public void testDecimal() throws Exception {
     Random random = new Random(7743);
 
-    doDecimalTests(random);
+    doDecimalTests(random, /* tryDecimal64 */ false);
+  }
+
+  @Test
+  public void testDecimal64() throws Exception {
+    Random random = new Random(7743);
+
+    doDecimalTests(random, /* tryDecimal64 */ true);
+  }
+
+  @Test
+  public void testInterval() throws Exception {
+    Random random = new Random(7743);
+
+    doIntervalTests(random);
+  }
+
+  @Test
+  public void testTimestampInterval() throws Exception {
+    Random random = new Random(7743);
+
+    doAddSubTests(random, TypeInfoFactory.timestampTypeInfo, TypeInfoFactory.intervalYearMonthTypeInfo);
+    doAddSubTests(random, TypeInfoFactory.timestampTypeInfo, TypeInfoFactory.intervalDayTimeTypeInfo);
+
+    doSubTests(random, TypeInfoFactory.timestampTypeInfo, TypeInfoFactory.timestampTypeInfo);
+
+    doAddTests(random, TypeInfoFactory.intervalYearMonthTypeInfo, TypeInfoFactory.timestampTypeInfo);
+    doAddTests(random, TypeInfoFactory.intervalDayTimeTypeInfo, TypeInfoFactory.timestampTypeInfo);
+  }
+
+  @Test
+  public void testTimestampDate() throws Exception {
+    Random random = new Random(7743);
+
+    doSubTests(random, TypeInfoFactory.dateTypeInfo, TypeInfoFactory.timestampTypeInfo);
+    doSubTests(random, TypeInfoFactory.timestampTypeInfo, TypeInfoFactory.dateTypeInfo);
+  }
+
+  @Test
+  public void testDateInterval() throws Exception {
+    Random random = new Random(7743);
+
+    doAddSubTests(random, TypeInfoFactory.dateTypeInfo, TypeInfoFactory.intervalYearMonthTypeInfo);
+    doAddSubTests(random, TypeInfoFactory.dateTypeInfo, TypeInfoFactory.intervalDayTimeTypeInfo);
+
+    doSubTests(random, TypeInfoFactory.dateTypeInfo, TypeInfoFactory.dateTypeInfo);
+
+    doAddTests(random, TypeInfoFactory.intervalYearMonthTypeInfo, TypeInfoFactory.dateTypeInfo);
+    doAddTests(random, TypeInfoFactory.intervalDayTimeTypeInfo, TypeInfoFactory.dateTypeInfo);
   }
 
   public enum ArithmeticTestMode {
@@ -201,16 +249,29 @@ public class TestVectorArithmetic {
     new DecimalTypeInfo(7, 1)
   };
 
-  private void doDecimalTests(Random random)
+  private void doDecimalTests(Random random, boolean tryDecimal64)
       throws Exception {
     for (TypeInfo typeInfo : decimalTypeInfos) {
       for (ColumnScalarMode columnScalarMode : ColumnScalarMode.values()) {
         doTestsWithDiffColumnScalar(
-            random, typeInfo, typeInfo, columnScalarMode);
+            random, typeInfo, typeInfo, columnScalarMode, tryDecimal64);
       }
     }
   }
 
+  private static TypeInfo[] intervalTypeInfos = new TypeInfo[] {
+    TypeInfoFactory.intervalYearMonthTypeInfo,
+    TypeInfoFactory.intervalDayTimeTypeInfo
+  };
+
+  private void doIntervalTests(Random random)
+      throws Exception {
+    for (TypeInfo typeInfo : intervalTypeInfos) {
+      doAddSubTests(
+          random, typeInfo, typeInfo);
+    }
+  }
+
   private TypeInfo getOutputTypeInfo(GenericUDF genericUdfClone,
       List<ObjectInspector> objectInspectorList)
     throws HiveException {
@@ -236,16 +297,57 @@ public class TestVectorArithmetic {
     return new DecimalTypeInfo(precision, scale);
   }
 
+  private boolean checkDecimal64(boolean tryDecimal64, TypeInfo typeInfo) {
+    if (!tryDecimal64 || !(typeInfo instanceof DecimalTypeInfo)) {
+      return false;
+    }
+    DecimalTypeInfo decimalTypeInfo = (DecimalTypeInfo) typeInfo;
+    boolean result = HiveDecimalWritable.isPrecisionDecimal64(decimalTypeInfo.getPrecision());
+    return result;
+  }
+
+  private void doAddTests(Random random, TypeInfo typeInfo1, TypeInfo typeInfo2)
+      throws Exception {
+    for (ColumnScalarMode columnScalarMode : ColumnScalarMode.values()) {
+      doTestsWithDiffColumnScalar(
+          random, typeInfo1, typeInfo2, columnScalarMode, Arithmetic.ADD, false);
+    }
+  }
+
+  private void doSubTests(Random random, TypeInfo typeInfo1, TypeInfo typeInfo2)
+      throws Exception {
+    for (ColumnScalarMode columnScalarMode : ColumnScalarMode.values()) {
+      doTestsWithDiffColumnScalar(
+          random, typeInfo1, typeInfo2, columnScalarMode, Arithmetic.SUBTRACT, false);
+    }
+  }
+
+  private void doAddSubTests(Random random, TypeInfo typeInfo1, TypeInfo typeInfo2)
+          throws Exception {
+    doAddTests(random, typeInfo1, typeInfo2);
+    doSubTests(random, typeInfo1, typeInfo2);
+  }
+
   private void doTestsWithDiffColumnScalar(Random random, TypeInfo typeInfo1, TypeInfo typeInfo2,
       ColumnScalarMode columnScalarMode)
           throws Exception {
     for (Arithmetic arithmetic : Arithmetic.values()) {
-      doTestsWithDiffColumnScalar(random, typeInfo1, typeInfo2, columnScalarMode, arithmetic);
+      doTestsWithDiffColumnScalar(
+          random, typeInfo1, typeInfo2, columnScalarMode, arithmetic, false);
+    }
+  }
+
+  private void doTestsWithDiffColumnScalar(Random random, TypeInfo typeInfo1, TypeInfo typeInfo2,
+      ColumnScalarMode columnScalarMode, boolean tryDecimal64)
+          throws Exception {
+    for (Arithmetic arithmetic : Arithmetic.values()) {
+      doTestsWithDiffColumnScalar(
+          random, typeInfo1, typeInfo2, columnScalarMode, arithmetic, tryDecimal64);
     }
   }
 
   private void doTestsWithDiffColumnScalar(Random random, TypeInfo typeInfo1, TypeInfo typeInfo2,
-      ColumnScalarMode columnScalarMode, Arithmetic arithmetic)
+      ColumnScalarMode columnScalarMode, Arithmetic arithmetic, boolean tryDecimal64)
           throws Exception {
 
     String typeName1 = typeInfo1.getTypeName();
@@ -265,11 +367,15 @@ public class TestVectorArithmetic {
 
     ExprNodeDesc col1Expr;
     Object scalar1Object = null;
+    final boolean decimal64Enable1 = checkDecimal64(tryDecimal64, typeInfo1);
     if (columnScalarMode == ColumnScalarMode.COLUMN_COLUMN ||
         columnScalarMode == ColumnScalarMode.COLUMN_SCALAR) {
       generationSpecList.add(
           GenerationSpec.createSameType(typeInfo1));
-      explicitDataTypePhysicalVariationList.add(DataTypePhysicalVariation.NONE);
+      explicitDataTypePhysicalVariationList.add(
+          decimal64Enable1 ?
+              DataTypePhysicalVariation.DECIMAL_64 :
+              DataTypePhysicalVariation.NONE);
 
       String columnName = "col" + (columnNum++);
       col1Expr = new ExprNodeColumnDesc(typeInfo1, columnName, "table", false);
@@ -288,12 +394,16 @@ public class TestVectorArithmetic {
     }
     ExprNodeDesc col2Expr;
     Object scalar2Object = null;
+    final boolean decimal64Enable2 = checkDecimal64(tryDecimal64, typeInfo2);
     if (columnScalarMode == ColumnScalarMode.COLUMN_COLUMN ||
         columnScalarMode == ColumnScalarMode.SCALAR_COLUMN) {
       generationSpecList.add(
           GenerationSpec.createSameType(typeInfo2));
 
-      explicitDataTypePhysicalVariationList.add(DataTypePhysicalVariation.NONE);
+      explicitDataTypePhysicalVariationList.add(
+          decimal64Enable2 ?
+              DataTypePhysicalVariation.DECIMAL_64 :
+              DataTypePhysicalVariation.NONE);
 
       String columnName = "col" + (columnNum++);
       col2Expr = new ExprNodeColumnDesc(typeInfo2, columnName, "table", false);
@@ -430,7 +540,8 @@ public class TestVectorArithmetic {
           if (expectedResult != null || vectorResult != null) {
             Assert.fail(
                 "Row " + i +
-                " typeName " + typeName1 +
+                " typeName1 " + typeName1 +
+                " typeName2 " + typeName2 +
                 " outputTypeName " + outputTypeInfo.getTypeName() +
                 " " + arithmetic +
                 " " + ArithmeticTestMode.values()[v] +
@@ -448,7 +559,8 @@ public class TestVectorArithmetic {
           if (!expectedResult.equals(vectorResult)) {
             Assert.fail(
                 "Row " + i +
-                " typeName " + typeName1 +
+                " typeName1 " + typeName1 +
+                " typeName2 " + typeName2 +
                 " outputTypeName " + outputTypeInfo.getTypeName() +
                 " " + arithmetic +
                 " " + ArithmeticTestMode.values()[v] +
@@ -545,6 +657,9 @@ public class TestVectorArithmetic {
     HiveConf hiveConf = new HiveConf();
     if (arithmeticTestMode == ArithmeticTestMode.ADAPTOR) {
       hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_TEST_VECTOR_ADAPTOR_OVERRIDE, true);
+
+      // Don't use DECIMAL_64 with the VectorUDFAdaptor.
+      dataTypePhysicalVariations = null;
     }
 
     VectorizationContext vectorizationContext =
@@ -552,12 +667,14 @@ public class TestVectorArithmetic {
             "name",
             columns,
             Arrays.asList(typeInfos),
-            Arrays.asList(dataTypePhysicalVariations),
+            dataTypePhysicalVariations == null ? null : Arrays.asList(dataTypePhysicalVariations),
             hiveConf);
     VectorExpression vectorExpression = vectorizationContext.getVectorExpression(exprDesc);
     vectorExpression.transientInit();
 
     String[] outputScratchTypeNames= vectorizationContext.getScratchColumnTypeNames();
+    DataTypePhysicalVariation[] outputDataTypePhysicalVariations =
+        vectorizationContext.getScratchDataTypePhysicalVariations();
 
     VectorizedRowBatchCtx batchContext =
         new VectorizedRowBatchCtx(
@@ -569,7 +686,7 @@ public class TestVectorArithmetic {
             /* virtualColumnCount */ 0,
             /* neededVirtualColumns */ null,
             outputScratchTypeNames,
-            null);
+            outputDataTypePhysicalVariations);
 
     VectorizedRowBatch batch = batchContext.createVectorizedRowBatch();
 
@@ -578,6 +695,8 @@ public class TestVectorArithmetic {
         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() +

http://git-wip-us.apache.org/repos/asf/hive/blob/aa1f0de8/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 9e61fcd..2c82c4e 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
@@ -490,6 +490,8 @@ public class TestVectorCastStatement {
     VectorExpression vectorExpression = vectorizationContext.getVectorExpression(exprDesc);
     vectorExpression.transientInit();
 
+    // System.out.println("*VECTOR EXPRESSION* " + vectorExpression.getClass().getSimpleName());
+
     /*
     System.out.println(
         "*DEBUG* typeInfo " + typeInfo.toString() +

http://git-wip-us.apache.org/repos/asf/hive/blob/aa1f0de8/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 c4146be..da21aba 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
@@ -454,6 +454,9 @@ public class TestVectorDateAddSub {
     resultVectorExtractRow.init(new TypeInfo[] { TypeInfoFactory.dateTypeInfo }, new int[] { columns.size() });
     Object[] scrqtchRow = new Object[1];
 
+    // System.out.println("*VECTOR EXPRESSION* " + vectorExpression.getClass().getSimpleName());
+
+    /*
     System.out.println(
         "*DEBUG* dateTimeStringTypeInfo " + dateTimeStringTypeInfo.toString() +
         " integerTypeInfo " + integerTypeInfo +
@@ -461,6 +464,7 @@ public class TestVectorDateAddSub {
         " dateAddSubTestMode " + dateAddSubTestMode +
         " columnScalarMode " + columnScalarMode +
         " vectorExpression " + vectorExpression.toString());
+    */
 
     batchSource.resetBatchIteration();
     int rowIndex = 0;

http://git-wip-us.apache.org/repos/asf/hive/blob/aa1f0de8/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 b382c2a..d5343ad 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
@@ -440,12 +440,16 @@ public class TestVectorDateDiff {
     resultVectorExtractRow.init(new TypeInfo[] { TypeInfoFactory.intTypeInfo }, new int[] { columns.size() });
     Object[] scrqtchRow = new Object[1];
 
+    // System.out.println("*VECTOR EXPRESSION* " + vectorExpression.getClass().getSimpleName());
+
+    /*
     System.out.println(
         "*DEBUG* dateTimeStringTypeInfo1 " + dateTimeStringTypeInfo1.toString() +
         " dateTimeStringTypeInfo2 " + dateTimeStringTypeInfo2.toString() +
         " dateDiffTestMode " + dateDiffTestMode +
         " columnScalarMode " + columnScalarMode +
         " vectorExpression " + vectorExpression.toString());
+    */
 
     batchSource.resetBatchIteration();
     int rowIndex = 0;

http://git-wip-us.apache.org/repos/asf/hive/blob/aa1f0de8/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 9020016..9d78ca6 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
@@ -459,6 +459,8 @@ public class TestVectorIfStatement {
     resultVectorExtractRow.init(new TypeInfo[] { typeInfo }, new int[] { columns.size() });
     Object[] scrqtchRow = new Object[1];
 
+    // System.out.println("*VECTOR EXPRESSION* " + vectorExpression.getClass().getSimpleName());
+
     /*
     System.out.println(
         "*DEBUG* typeInfo " + typeInfo.toString() +

http://git-wip-us.apache.org/repos/asf/hive/blob/aa1f0de8/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 d43249e..289efb9 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
@@ -427,6 +427,8 @@ public class TestVectorNegative {
         new TypeInfo[] { outputTypeInfo }, new int[] { vectorExpression.getOutputColumnNum() });
     Object[] scrqtchRow = new Object[1];
 
+    // System.out.println("*VECTOR EXPRESSION* " + vectorExpression.getClass().getSimpleName());
+
     /*
     System.out.println(
         "*DEBUG* typeInfo " + typeInfo.toString() +

http://git-wip-us.apache.org/repos/asf/hive/blob/aa1f0de8/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 f3050c2..3d030e6 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
@@ -407,6 +407,8 @@ public class TestVectorStringConcat {
         new TypeInfo[] { outputTypeInfo }, new int[] { columns.size() });
     Object[] scrqtchRow = new Object[1];
 
+    // System.out.println("*VECTOR EXPRESSION* " + vectorExpression.getClass().getSimpleName());
+
     /*
     System.out.println(
         "*DEBUG* stringTypeInfo1 " + stringTypeInfo1.toString() +

http://git-wip-us.apache.org/repos/asf/hive/blob/aa1f0de8/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 8df5595..4398554 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
@@ -347,6 +347,8 @@ public class TestVectorStringUnary {
     resultVectorExtractRow.init(new TypeInfo[] { targetTypeInfo }, new int[] { columns.size() });
     Object[] scrqtchRow = new Object[1];
 
+    // System.out.println("*VECTOR EXPRESSION* " + vectorExpression.getClass().getSimpleName());
+
     /*
     System.out.println(
         "*DEBUG* typeInfo " + typeInfo.toString() +

http://git-wip-us.apache.org/repos/asf/hive/blob/aa1f0de8/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorSubStr.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorSubStr.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorSubStr.java
index b1344ab..62d296d 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorSubStr.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorSubStr.java
@@ -326,6 +326,8 @@ public class TestVectorSubStr {
     resultVectorExtractRow.init(new TypeInfo[] { targetTypeInfo }, new int[] { columns.size() });
     Object[] scrqtchRow = new Object[1];
 
+    // System.out.println("*VECTOR EXPRESSION* " + vectorExpression.getClass().getSimpleName());
+
     /*
     System.out.println(
         "*DEBUG* typeInfo " + typeInfo.toString() +

http://git-wip-us.apache.org/repos/asf/hive/blob/aa1f0de8/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorTimestampExtract.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorTimestampExtract.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorTimestampExtract.java
index e56a6c3..3220f4c 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorTimestampExtract.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorTimestampExtract.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.hive.ql.exec.vector.expressions;
 
 import junit.framework.Assert;
+
 import org.apache.hadoop.hive.common.type.DataTypePhysicalVariation;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluator;
@@ -394,6 +395,8 @@ public class TestVectorTimestampExtract {
     VectorExpression vectorExpression = vectorizationContext.getVectorExpression(exprDesc);
     vectorExpression.transientInit();
 
+    // System.out.println("*VECTOR EXPRESSION* " + vectorExpression.getClass().getSimpleName());
+
     /*
     System.out.println(
         "*DEBUG* dateTimeStringTypeInfo " + dateTimeStringTypeInfo.toString() +