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/25 15:05:57 UTC

[1/8] hive git commit: HIVE-20207: Vectorization: Fix NULL / Wrong Results issues in Filter / Compare (Matt McCline, reviewed by Teddy Choi)

Repository: hive
Updated Branches:
  refs/heads/master ca0d70697 -> 71c49878c


http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/vector-code-gen/src/org/apache/hadoop/hive/tools/GenVectorCode.java
----------------------------------------------------------------------
diff --git a/vector-code-gen/src/org/apache/hadoop/hive/tools/GenVectorCode.java b/vector-code-gen/src/org/apache/hadoop/hive/tools/GenVectorCode.java
index 8b36371..666572a 100644
--- a/vector-code-gen/src/org/apache/hadoop/hive/tools/GenVectorCode.java
+++ b/vector-code-gen/src/org/apache/hadoop/hive/tools/GenVectorCode.java
@@ -316,16 +316,22 @@ public class GenVectorCode extends Task {
       {"Decimal64ColumnArithmeticDecimal64Column", "Add", "+"},
       {"Decimal64ColumnArithmeticDecimal64Column", "Subtract", "-"},
 
+      {"ColumnCompareScalar", "Equal", "long", "long", "=="},
       {"ColumnCompareScalar", "Equal", "long", "double", "=="},
       {"ColumnCompareScalar", "Equal", "double", "double", "=="},
+      {"ColumnCompareScalar", "NotEqual", "long", "long", "!="},
       {"ColumnCompareScalar", "NotEqual", "long", "double", "!="},
       {"ColumnCompareScalar", "NotEqual", "double", "double", "!="},
+      {"ColumnCompareScalar", "Less", "long", "long", "<"},
       {"ColumnCompareScalar", "Less", "long", "double", "<"},
       {"ColumnCompareScalar", "Less", "double", "double", "<"},
+      {"ColumnCompareScalar", "LessEqual", "long", "long", "<="},
       {"ColumnCompareScalar", "LessEqual", "long", "double", "<="},
       {"ColumnCompareScalar", "LessEqual", "double", "double", "<="},
+      {"ColumnCompareScalar", "Greater", "long", "long", ">"},
       {"ColumnCompareScalar", "Greater", "long", "double", ">"},
       {"ColumnCompareScalar", "Greater", "double", "double", ">"},
+      {"ColumnCompareScalar", "GreaterEqual", "long", "long", ">="},
       {"ColumnCompareScalar", "GreaterEqual", "long", "double", ">="},
       {"ColumnCompareScalar", "GreaterEqual", "double", "double", ">="},
 
@@ -336,16 +342,22 @@ public class GenVectorCode extends Task {
       {"ColumnCompareScalar", "Greater", "double", "long", ">"},
       {"ColumnCompareScalar", "GreaterEqual", "double", "long", ">="},
 
+      {"ScalarCompareColumn", "Equal", "long", "long", "=="},
       {"ScalarCompareColumn", "Equal", "long", "double", "=="},
       {"ScalarCompareColumn", "Equal", "double", "double", "=="},
+      {"ScalarCompareColumn", "NotEqual", "long", "long", "!="},
       {"ScalarCompareColumn", "NotEqual", "long", "double", "!="},
       {"ScalarCompareColumn", "NotEqual", "double", "double", "!="},
+      {"ScalarCompareColumn", "Less", "long", "long", "<"},
       {"ScalarCompareColumn", "Less", "long", "double", "<"},
       {"ScalarCompareColumn", "Less", "double", "double", "<"},
+      {"ScalarCompareColumn", "LessEqual", "long", "long", "<="},
       {"ScalarCompareColumn", "LessEqual", "long", "double", "<="},
       {"ScalarCompareColumn", "LessEqual", "double", "double", "<="},
+      {"ScalarCompareColumn", "Greater", "long", "long", ">"},
       {"ScalarCompareColumn", "Greater", "long", "double", ">"},
       {"ScalarCompareColumn", "Greater", "double", "double", ">"},
+      {"ScalarCompareColumn", "GreaterEqual", "long", "long", ">="},
       {"ScalarCompareColumn", "GreaterEqual", "long", "double", ">="},
       {"ScalarCompareColumn", "GreaterEqual", "double", "double", ">="},
 
@@ -356,6 +368,28 @@ public class GenVectorCode extends Task {
       {"ScalarCompareColumn", "Greater", "double", "long", ">"},
       {"ScalarCompareColumn", "GreaterEqual", "double", "long", ">="},
 
+      // Compare decimal to decimal.
+      {"DecimalCompareDecimal", "Equal", "==", "Col", "Column"},
+      {"DecimalCompareDecimal", "NotEqual", "!=", "Col", "Column"},
+      {"DecimalCompareDecimal", "Less", "<", "Col", "Column"},
+      {"DecimalCompareDecimal", "LessEqual", "<=", "Col", "Column"},
+      {"DecimalCompareDecimal", "Greater", ">", "Col", "Column"},
+      {"DecimalCompareDecimal", "GreaterEqual", ">=", "Col", "Column"},
+
+      {"DecimalCompareDecimal", "Equal", "==", "Col", "Scalar"},
+      {"DecimalCompareDecimal", "NotEqual", "!=", "Col", "Scalar"},
+      {"DecimalCompareDecimal", "Less", "<", "Col", "Scalar"},
+      {"DecimalCompareDecimal", "LessEqual", "<=", "Col", "Scalar"},
+      {"DecimalCompareDecimal", "Greater", ">", "Col", "Scalar"},
+      {"DecimalCompareDecimal", "GreaterEqual", ">=", "Col", "Scalar"},
+
+      {"DecimalCompareDecimal", "Equal", "==", "Scalar", "Column"},
+      {"DecimalCompareDecimal", "NotEqual", "!=", "Scalar", "Column"},
+      {"DecimalCompareDecimal", "Less", "<", "Scalar", "Column"},
+      {"DecimalCompareDecimal", "LessEqual", "<=", "Scalar", "Column"},
+      {"DecimalCompareDecimal", "Greater", ">", "Scalar", "Column"},
+      {"DecimalCompareDecimal", "GreaterEqual", ">=", "Scalar", "Column"},
+
       // Compare timestamp to timestamp.
       {"TimestampCompareTimestamp", "Equal", "==", "timestamp", "Col", "Column"},
       {"TimestampCompareTimestamp", "NotEqual", "!=", "timestamp", "Col", "Column"},
@@ -478,6 +512,28 @@ public class GenVectorCode extends Task {
       {"LongDoubleCompareTimestamp", "GreaterEqual", "long", ">=", "Scalar", "Column"},
       {"LongDoubleCompareTimestamp", "GreaterEqual", "double", ">=", "Scalar", "Column"},
 
+      // Decimal64
+      {"Decimal64ColumnCompareDecimal64Scalar", "Equal"},
+      {"Decimal64ColumnCompareDecimal64Scalar", "NotEqual"},
+      {"Decimal64ColumnCompareDecimal64Scalar", "Less"},
+      {"Decimal64ColumnCompareDecimal64Scalar", "LessEqual"},
+      {"Decimal64ColumnCompareDecimal64Scalar", "Greater"},
+      {"Decimal64ColumnCompareDecimal64Scalar", "GreaterEqual"},
+
+      {"Decimal64ScalarCompareDecimal64Column", "Equal"},
+      {"Decimal64ScalarCompareDecimal64Column", "NotEqual"},
+      {"Decimal64ScalarCompareDecimal64Column", "Less"},
+      {"Decimal64ScalarCompareDecimal64Column", "LessEqual"},
+      {"Decimal64ScalarCompareDecimal64Column", "Greater"},
+      {"Decimal64ScalarCompareDecimal64Column", "GreaterEqual"},
+
+      {"Decimal64ColumnCompareDecimal64Column", "Equal"},
+      {"Decimal64ColumnCompareDecimal64Column", "NotEqual"},
+      {"Decimal64ColumnCompareDecimal64Column", "Less"},
+      {"Decimal64ColumnCompareDecimal64Column", "LessEqual"},
+      {"Decimal64ColumnCompareDecimal64Column", "Greater"},
+      {"Decimal64ColumnCompareDecimal64Column", "GreaterEqual"},
+
       // Filter long/double.
       {"FilterColumnCompareScalar", "Equal", "long", "double", "=="},
       {"FilterColumnCompareScalar", "Equal", "double", "double", "=="},
@@ -884,16 +940,22 @@ public class GenVectorCode extends Task {
       {"FilterColumnBetweenDynamicValue", "date", ""},
       {"FilterColumnBetweenDynamicValue", "timestamp", ""},
 
+      {"ColumnCompareColumn", "Equal", "long", "long", "=="},
       {"ColumnCompareColumn", "Equal", "long", "double", "=="},
       {"ColumnCompareColumn", "Equal", "double", "double", "=="},
+      {"ColumnCompareColumn", "NotEqual", "long", "long", "!="},
       {"ColumnCompareColumn", "NotEqual", "long", "double", "!="},
       {"ColumnCompareColumn", "NotEqual", "double", "double", "!="},
+      {"ColumnCompareColumn", "Less", "long", "long", "<"},
       {"ColumnCompareColumn", "Less", "long", "double", "<"},
       {"ColumnCompareColumn", "Less", "double", "double", "<"},
+      {"ColumnCompareColumn", "LessEqual", "long", "long", "<="},
       {"ColumnCompareColumn", "LessEqual", "long", "double", "<="},
       {"ColumnCompareColumn", "LessEqual", "double", "double", "<="},
+      {"ColumnCompareColumn", "Greater", "long", "long", ">"},
       {"ColumnCompareColumn", "Greater", "long", "double", ">"},
       {"ColumnCompareColumn", "Greater", "double", "double", ">"},
+      {"ColumnCompareColumn", "GreaterEqual", "long", "long", ">="},
       {"ColumnCompareColumn", "GreaterEqual", "long", "double", ">="},
       {"ColumnCompareColumn", "GreaterEqual", "double", "double", ">="},
 
@@ -1063,6 +1125,11 @@ public class GenVectorCode extends Task {
       {"IfExprScalarScalar", "long", "double"},
       {"IfExprScalarScalar", "double", "double"},
 
+      {"IfExprObjectColumnColumn", "decimal"},
+      {"IfExprObjectColumnScalar", "decimal"},
+      {"IfExprObjectScalarColumn", "decimal"},
+      {"IfExprObjectScalarScalar", "decimal"},
+
       {"IfExprObjectColumnColumn", "timestamp"},
       {"IfExprObjectColumnColumn", "interval_day_time"},
       {"IfExprObjectColumnScalar", "timestamp"},
@@ -1271,6 +1338,9 @@ public class GenVectorCode extends Task {
       } else if (tdesc[0].equals("ScalarCompareColumn")) {
         generateScalarCompareColumn(tdesc);
 
+      } else if (tdesc[0].equals("DecimalCompareDecimal")) {
+        generateDecimalCompareDecimal(tdesc);
+
       } else if (tdesc[0].equals("TimestampCompareTimestamp")) {
         generateTimestampCompareTimestamp(tdesc);
 
@@ -1388,6 +1458,12 @@ public class GenVectorCode extends Task {
         generateFilterStringGroupColumnCompareStringGroupColumn(tdesc);
       } else if (tdesc[0].equals("StringGroupColumnCompareStringGroupColumn")) {
         generateStringGroupColumnCompareStringGroupColumn(tdesc);
+      } else if (tdesc[0].equals("Decimal64ColumnCompareDecimal64Scalar")) {
+        generateDecimal64ColumnCompareDecimal64Scalar(tdesc);
+      } else if (tdesc[0].equals("Decimal64ScalarCompareDecimal64Column")) {
+        generateDecimal64ScalarCompareDecimal64Column(tdesc);
+      } else if (tdesc[0].equals("Decimal64ColumnCompareDecimal64Column")) {
+        generateDecimal64ColumnCompareDecimal64Column(tdesc);
       } else if (tdesc[0].equals("IfExprColumnScalar")) {
         generateIfExprColumnScalar(tdesc);
       } else if (tdesc[0].equals("IfExprScalarColumn")) {
@@ -2253,6 +2329,7 @@ public class GenVectorCode extends Task {
     String objectName;
     String scalarType;
     String scalarImport;
+    String ifDefined = "";
     if (typeName.equals("timestamp")) {
       objectName = "Timestamp";
       scalarType = "Timestamp";
@@ -2261,6 +2338,11 @@ public class GenVectorCode extends Task {
       objectName = "IntervalDayTime";
       scalarType = "HiveIntervalDayTime";
       scalarImport = "org.apache.hadoop.hive.common.type.HiveIntervalDayTime";
+    } else if (typeName.equals("decimal")) {
+      ifDefined = "DECIMAL";
+      objectName = "Decimal";
+      scalarType = "HiveDecimal";
+      scalarImport = "org.apache.hadoop.hive.common.type.HiveDecimal";
     } else {
       objectName = "unknown";
       scalarType = "unknown";
@@ -2276,6 +2358,8 @@ public class GenVectorCode extends Task {
     File templateFile = new File(joinPath(this.expressionTemplateDirectory, tdesc[0] + ".txt"));
     String templateString = readFile(templateFile);
 
+    templateString = evaluateIfDefined(templateString, ifDefined);
+
     templateString = templateString.replaceAll("<ClassName>", className);
     templateString = templateString.replaceAll("<ScalarType>", scalarType);
     templateString = templateString.replaceAll("<ScalarImport>", scalarImport);
@@ -2472,6 +2556,23 @@ public class GenVectorCode extends Task {
     }
   }
 
+  private void generateDecimalCompareDecimal(String[] tdesc) throws Exception {
+    String operatorName = tdesc[1];
+    String operatorSymbol = tdesc[2];
+    String className = "Decimal" + tdesc[3] + operatorName + "Decimal" + tdesc[4];
+
+    //Read the template into a string;
+    String fileName = "Decimal" + (tdesc[3].equals("Col") ? "Column" : tdesc[3]) + "CompareDecimal" +
+        (tdesc[4].equals("Col") ? "Column" : tdesc[4]);
+    File templateFile = new File(joinPath(this.expressionTemplateDirectory, fileName + ".txt"));
+    String templateString = readFile(templateFile);
+    templateString = templateString.replaceAll("<ClassName>", className);
+    templateString = templateString.replaceAll("<OperatorSymbol>", operatorSymbol);
+
+    writeFile(templateFile.lastModified(), expressionOutputDirectory, expressionClassesDirectory,
+        className, templateString);
+  }
+
   // -----------------------------------------------------------------------------------------------
   //
   // Filter timestamp against timestamp, long (seconds), and double (seconds with fractional
@@ -3038,6 +3139,27 @@ public class GenVectorCode extends Task {
        className, templateString);
   }
 
+  private void generateDecimal64ColumnCompareDecimal64Scalar(String[] tdesc) throws IOException {
+    String operatorName = tdesc[1];
+    String className = "Decimal64Col" + operatorName + "Decimal64Scalar";
+    String baseClassName = "LongCol" + operatorName + "LongScalar";
+    generateDecimal64ColumnCompare(tdesc, className, baseClassName);
+  }
+
+  private void generateDecimal64ScalarCompareDecimal64Column(String[] tdesc) throws IOException {
+    String operatorName = tdesc[1];
+    String className = "Decimal64Scalar" + operatorName + "Decimal64Column";
+    String baseClassName = "LongScalar" + operatorName + "LongColumn";
+    generateDecimal64ColumnCompare(tdesc, className, baseClassName);
+  }
+
+  private void generateDecimal64ColumnCompareDecimal64Column(String[] tdesc) throws IOException {
+    String operatorName = tdesc[1];
+    String className = "Decimal64Col" + operatorName + "Decimal64Column";
+    String baseClassName = "LongCol" + operatorName + "LongColumn";
+    generateDecimal64ColumnCompare(tdesc, className, baseClassName);
+  }
+
   private void generateScalarArithmeticColumn(String[] tdesc) throws Exception {
     String operatorName = tdesc[1];
     String operandType1 = tdesc[2];
@@ -3124,7 +3246,7 @@ public class GenVectorCode extends Task {
     String operandType = tdesc[2];
     String className = getCamelCaseType(operandType) + "Scalar" + operatorName
         + getCamelCaseType(operandType) + "Column";
-    String baseClassName = "org.apache.hadoop.hive.ql.exec.vector.expressions.LongScalar" + operatorName + "LongColumn";
+    String baseClassName = "org.apache.hadoop.hive.ql.exec.vector.expressions.gen.LongScalar" + operatorName + "LongColumn";
     //Read the template into a string;
     File templateFile = new File(joinPath(this.expressionTemplateDirectory, tdesc[0] + ".txt"));
     String templateString = readFile(templateFile);
@@ -3183,7 +3305,7 @@ public class GenVectorCode extends Task {
     String operandType = tdesc[2];
     String className = getCamelCaseType(operandType) + "Col" + operatorName
         + getCamelCaseType(operandType) + "Scalar";
-    String baseClassName = "org.apache.hadoop.hive.ql.exec.vector.expressions.LongCol" + operatorName + "LongScalar";
+    String baseClassName = "org.apache.hadoop.hive.ql.exec.vector.expressions.gen.LongCol" + operatorName + "LongScalar";
     //Read the template into a string;
     File templateFile = new File(joinPath(this.expressionTemplateDirectory, tdesc[0] + ".txt"));
     String templateString = readFile(templateFile);


[4/8] hive git commit: HIVE-20207: Vectorization: Fix NULL / Wrong Results issues in Filter / Compare (Matt McCline, reviewed by Teddy Choi)

Posted by mm...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/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
new file mode 100644
index 0000000..1ff11ec
--- /dev/null
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorFilterCompare.java
@@ -0,0 +1,795 @@
+/*
+ * 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 java.lang.reflect.Constructor;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Arrays;
+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;
+import org.apache.hadoop.hive.ql.exec.FunctionInfo;
+import org.apache.hadoop.hive.ql.exec.FunctionRegistry;
+import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.ql.exec.vector.VectorExtractRow;
+import org.apache.hadoop.hive.ql.exec.vector.VectorRandomBatchSource;
+import org.apache.hadoop.hive.ql.exec.vector.VectorRandomRowSource;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizationContext;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatchCtx;
+import org.apache.hadoop.hive.ql.exec.vector.VectorRandomRowSource.GenerationSpec;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.exec.vector.udf.VectorUDFAdaptor;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc;
+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.GenericUDFOPGreaterThan;
+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;
+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 org.apache.hadoop.io.BooleanWritable;
+
+import junit.framework.Assert;
+
+import org.junit.Ignore;
+import org.junit.Test;
+
+public class TestVectorFilterCompare {
+
+  public TestVectorFilterCompare() {
+    // Arithmetic operations rely on getting conf from SessionState, need to initialize here.
+    SessionState ss = new SessionState(new HiveConf());
+    ss.getConf().setVar(HiveConf.ConfVars.HIVE_COMPAT, "latest");
+    SessionState.setCurrentSessionState(ss);
+  }
+
+  @Test
+  public void testIntegers() throws Exception {
+    Random random = new Random(7743);
+
+    doIntegerTests(random);
+  }
+
+  @Test
+  public void testIntegerFloating() throws Exception {
+    Random random = new Random(7743);
+
+    doIntegerFloatingTests(random);
+  }
+
+  @Test
+  public void testFloating() throws Exception {
+    Random random = new Random(7743);
+
+    doFloatingTests(random);
+  }
+
+  @Test
+  public void testDecimal() throws Exception {
+    Random random = new Random(7743);
+
+    doDecimalTests(random, /* tryDecimal64 */ false);
+  }
+
+  @Test
+  public void testDecimal64() throws Exception {
+    Random random = new Random(7743);
+
+    doDecimalTests(random, /* tryDecimal64 */ true);
+  }
+
+  @Test
+  public void testTimestamp() throws Exception {
+    Random random = new Random(7743);
+
+    doTests(random, TypeInfoFactory.timestampTypeInfo, TypeInfoFactory.timestampTypeInfo);
+
+    doTests(random, TypeInfoFactory.timestampTypeInfo, TypeInfoFactory.longTypeInfo);
+    doTests(random, TypeInfoFactory.timestampTypeInfo, TypeInfoFactory.doubleTypeInfo);
+
+    doTests(random, TypeInfoFactory.longTypeInfo, TypeInfoFactory.timestampTypeInfo);
+    doTests(random, TypeInfoFactory.doubleTypeInfo, TypeInfoFactory.timestampTypeInfo);
+  }
+
+  @Test
+  public void testDate() throws Exception {
+    Random random = new Random(7743);
+
+    doTests(random, TypeInfoFactory.dateTypeInfo, TypeInfoFactory.dateTypeInfo);
+  }
+
+  @Test
+  public void testInterval() throws Exception {
+    Random random = new Random(7743);
+
+    doTests(random, TypeInfoFactory.intervalYearMonthTypeInfo, TypeInfoFactory.intervalYearMonthTypeInfo);
+    doTests(random, TypeInfoFactory.intervalDayTimeTypeInfo, TypeInfoFactory.intervalDayTimeTypeInfo);
+  }
+
+  @Test
+  public void testStringFamily() throws Exception {
+    Random random = new Random(7743);
+
+    doTests(random, TypeInfoFactory.stringTypeInfo, TypeInfoFactory.stringTypeInfo);
+
+    doTests(random, new CharTypeInfo(10), new CharTypeInfo(10));
+    doTests(random, new VarcharTypeInfo(10), new VarcharTypeInfo(10));
+  }
+
+  public enum FilterCompareTestMode {
+    ROW_MODE,
+    ADAPTOR,
+    FILTER_VECTOR_EXPRESSION,
+    COMPARE_VECTOR_EXPRESSION;
+
+    static final int count = values().length;
+  }
+
+  public enum ColumnScalarMode {
+    COLUMN_COLUMN,
+    COLUMN_SCALAR,
+    SCALAR_COLUMN;
+
+    static final int count = values().length;
+  }
+
+  private static TypeInfo[] integerTypeInfos = new TypeInfo[] {
+    TypeInfoFactory.byteTypeInfo,
+    TypeInfoFactory.shortTypeInfo,
+    TypeInfoFactory.intTypeInfo,
+    TypeInfoFactory.longTypeInfo
+  };
+
+  // We have test failures with FLOAT.  Ignoring this issue for now.
+  private static TypeInfo[] floatingTypeInfos = new TypeInfo[] {
+    // TypeInfoFactory.floatTypeInfo,
+    TypeInfoFactory.doubleTypeInfo
+  };
+
+  private void doIntegerTests(Random random)
+          throws Exception {
+    for (TypeInfo typeInfo : integerTypeInfos) {
+      doTests(random, typeInfo, typeInfo);
+    }
+  }
+
+  private void doIntegerFloatingTests(Random random)
+      throws Exception {
+    for (TypeInfo typeInfo1 : integerTypeInfos) {
+      for (TypeInfo typeInfo2 : floatingTypeInfos) {
+        doTests(random, typeInfo1, typeInfo2);
+      }
+    }
+    for (TypeInfo typeInfo1 : floatingTypeInfos) {
+      for (TypeInfo typeInfo2 : integerTypeInfos) {
+        doTests(random, typeInfo1, typeInfo2);
+      }
+    }
+  }
+
+  private void doFloatingTests(Random random)
+      throws Exception {
+    for (TypeInfo typeInfo1 : floatingTypeInfos) {
+      for (TypeInfo typeInfo2 : floatingTypeInfos) {
+        doTests(random, typeInfo1, typeInfo2);
+      }
+    }
+  }
+
+  private static TypeInfo[] decimalTypeInfos = new TypeInfo[] {
+    new DecimalTypeInfo(38, 18),
+    new DecimalTypeInfo(25, 2),
+    new DecimalTypeInfo(19, 4),
+    new DecimalTypeInfo(18, 10),
+    new DecimalTypeInfo(17, 3),
+    new DecimalTypeInfo(12, 2),
+    new DecimalTypeInfo(7, 1)
+  };
+
+  private void doDecimalTests(Random random, boolean tryDecimal64)
+      throws Exception {
+    for (TypeInfo typeInfo : decimalTypeInfos) {
+      doTests(random, typeInfo, typeInfo, tryDecimal64);
+    }
+  }
+
+  private TypeInfo getOutputTypeInfo(GenericUDF genericUdfClone,
+      List<ObjectInspector> objectInspectorList)
+    throws HiveException {
+
+    ObjectInspector[] array =
+        objectInspectorList.toArray(new ObjectInspector[objectInspectorList.size()]);
+    ObjectInspector outputObjectInspector = genericUdfClone.initialize(array);
+    return TypeInfoUtils.getTypeInfoFromObjectInspector(outputObjectInspector);
+  }
+
+  public enum Comparison {
+    EQUALS,
+    LESS_THAN,
+    LESS_THAN_EQUAL,
+    GREATER_THAN,
+    GREATER_THAN_EQUAL,
+    NOT_EQUALS;
+  }
+
+  private TypeInfo getDecimalScalarTypeInfo(Object scalarObject) {
+    HiveDecimal dec = (HiveDecimal) scalarObject;
+    int precision = dec.precision();
+    int scale = dec.scale();
+    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 doTests(Random random, TypeInfo typeInfo1, TypeInfo typeInfo2, boolean tryDecimal64)
+      throws Exception {
+    for (ColumnScalarMode columnScalarMode : ColumnScalarMode.values()) {
+      doTestsWithDiffColumnScalar(
+          random, typeInfo1, typeInfo2, columnScalarMode, tryDecimal64);
+    }
+  }
+
+  private void doTests(Random random, TypeInfo typeInfo1, TypeInfo typeInfo2)
+      throws Exception {
+    for (ColumnScalarMode columnScalarMode : ColumnScalarMode.values()) {
+      doTestsWithDiffColumnScalar(
+          random, typeInfo1, typeInfo2, columnScalarMode);
+    }
+  }
+
+  private void doTestsWithDiffColumnScalar(Random random, TypeInfo typeInfo1, TypeInfo typeInfo2,
+      ColumnScalarMode columnScalarMode)
+          throws Exception {
+    doTestsWithDiffColumnScalar(random, typeInfo1, typeInfo2, columnScalarMode, false);
+  }
+
+  private void doTestsWithDiffColumnScalar(Random random, TypeInfo typeInfo1, TypeInfo typeInfo2,
+      ColumnScalarMode columnScalarMode, boolean tryDecimal64)
+          throws Exception {
+    for (Comparison comparison : Comparison.values()) {
+      doTestsWithDiffColumnScalar(
+          random, typeInfo1, typeInfo2, columnScalarMode, comparison, tryDecimal64);
+    }
+  }
+
+  private void doTestsWithDiffColumnScalar(Random random, TypeInfo typeInfo1, TypeInfo typeInfo2,
+      ColumnScalarMode columnScalarMode, Comparison comparison, boolean tryDecimal64)
+          throws Exception {
+
+    String typeName1 = typeInfo1.getTypeName();
+    PrimitiveCategory primitiveCategory1 =
+        ((PrimitiveTypeInfo) typeInfo1).getPrimitiveCategory();
+
+    String typeName2 = typeInfo2.getTypeName();
+    PrimitiveCategory primitiveCategory2 =
+        ((PrimitiveTypeInfo) typeInfo2).getPrimitiveCategory();
+
+    List<GenerationSpec> generationSpecList = new ArrayList<GenerationSpec>();
+    List<DataTypePhysicalVariation> explicitDataTypePhysicalVariationList =
+        new ArrayList<DataTypePhysicalVariation>();
+
+    List<String> columns = new ArrayList<String>();
+    int columnNum = 0;
+
+    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(
+          decimal64Enable1 ?
+              DataTypePhysicalVariation.DECIMAL_64 :
+              DataTypePhysicalVariation.NONE);
+
+      String columnName = "col" + (columnNum++);
+      col1Expr = new ExprNodeColumnDesc(typeInfo1, columnName, "table", false);
+      columns.add(columnName);
+    } else {
+      scalar1Object =
+          VectorRandomRowSource.randomPrimitiveObject(
+              random, (PrimitiveTypeInfo) typeInfo1);
+
+      // Adjust the decimal type to the scalar's type...
+      if (typeInfo1 instanceof DecimalTypeInfo) {
+        typeInfo1 = getDecimalScalarTypeInfo(scalar1Object);
+      }
+
+      col1Expr = new ExprNodeConstantDesc(typeInfo1, scalar1Object);
+    }
+    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(
+          decimal64Enable2 ?
+              DataTypePhysicalVariation.DECIMAL_64 :
+              DataTypePhysicalVariation.NONE);
+
+      String columnName = "col" + (columnNum++);
+      col2Expr = new ExprNodeColumnDesc(typeInfo2, columnName, "table", false);
+      columns.add(columnName);
+    } else {
+      scalar2Object =
+          VectorRandomRowSource.randomPrimitiveObject(
+              random, (PrimitiveTypeInfo) typeInfo2);
+
+      // Adjust the decimal type to the scalar's type...
+      if (typeInfo2 instanceof DecimalTypeInfo) {
+        typeInfo2 = getDecimalScalarTypeInfo(scalar2Object);
+      }
+
+      col2Expr = new ExprNodeConstantDesc(typeInfo2, scalar2Object);
+    }
+
+    List<ObjectInspector> objectInspectorList = new ArrayList<ObjectInspector>();
+    objectInspectorList.add(VectorRandomRowSource.getObjectInspector(typeInfo1));
+    objectInspectorList.add(VectorRandomRowSource.getObjectInspector(typeInfo2));
+
+    List<ExprNodeDesc> children = new ArrayList<ExprNodeDesc>();
+    children.add(col1Expr);
+    children.add(col2Expr);
+
+    //----------------------------------------------------------------------------------------------
+
+    String[] columnNames = columns.toArray(new String[0]);
+
+    VectorRandomRowSource rowSource = new VectorRandomRowSource();
+
+    rowSource.initGenerationSpecSchema(
+        random, generationSpecList, /* maxComplexDepth */ 0, /* allowNull */ true,
+        explicitDataTypePhysicalVariationList);
+
+    Object[][] randomRows = rowSource.randomRows(100000);
+
+    VectorRandomBatchSource batchSource =
+        VectorRandomBatchSource.createInterestingBatches(
+            random,
+            rowSource,
+            randomRows,
+            null);
+
+    GenericUDF genericUdf;
+    switch (comparison) {
+    case EQUALS:
+      genericUdf = new GenericUDFOPEqual();
+      break;
+    case LESS_THAN:
+      genericUdf = new GenericUDFOPLessThan();
+      break;
+    case LESS_THAN_EQUAL:
+      genericUdf = new GenericUDFOPEqualOrLessThan();
+      break;
+    case GREATER_THAN:
+      genericUdf = new GenericUDFOPGreaterThan();
+      break;
+    case GREATER_THAN_EQUAL:
+      genericUdf = new GenericUDFOPEqualOrGreaterThan();
+      break;
+    case NOT_EQUALS:
+      genericUdf = new GenericUDFOPNotEqual();
+      break;
+    default:
+      throw new RuntimeException("Unexpected arithmetic " + comparison);
+    }
+
+    ObjectInspector[] objectInspectors =
+        objectInspectorList.toArray(new ObjectInspector[objectInspectorList.size()]);
+    ObjectInspector outputObjectInspector = null;
+    try {
+      outputObjectInspector = genericUdf.initialize(objectInspectors);
+    } catch (Exception e) {
+      Assert.fail(e.toString());
+    }
+
+    TypeInfo outputTypeInfo = TypeInfoUtils.getTypeInfoFromObjectInspector(outputObjectInspector);
+
+    ExprNodeGenericFuncDesc exprDesc =
+        new ExprNodeGenericFuncDesc(outputTypeInfo, genericUdf, children);
+
+    final int rowCount = randomRows.length;
+    Object[][] resultObjectsArray = new Object[FilterCompareTestMode.count][];
+    for (int i = 0; i < FilterCompareTestMode.count; i++) {
+
+      Object[] resultObjects = new Object[rowCount];
+      resultObjectsArray[i] = resultObjects;
+
+      FilterCompareTestMode filterCompareTestMode = FilterCompareTestMode.values()[i];
+      switch (filterCompareTestMode) {
+      case ROW_MODE:
+        doRowFilterCompareTest(
+            typeInfo1,
+            typeInfo2,
+            columns,
+            children,
+            exprDesc,
+            comparison,
+            randomRows,
+            columnScalarMode,
+            rowSource.rowStructObjectInspector(),
+            outputTypeInfo,
+            resultObjects);
+        break;
+      case ADAPTOR:
+      case FILTER_VECTOR_EXPRESSION:
+      case COMPARE_VECTOR_EXPRESSION:
+        doVectorFilterCompareTest(
+            typeInfo1,
+            typeInfo2,
+            columns,
+            columnNames,
+            rowSource.typeInfos(),
+            rowSource.dataTypePhysicalVariations(),
+            children,
+            exprDesc,
+            comparison,
+            filterCompareTestMode,
+            columnScalarMode,
+            batchSource,
+            exprDesc.getWritableObjectInspector(),
+            outputTypeInfo,
+            resultObjects);
+        break;
+      default:
+        throw new RuntimeException("Unexpected IF statement test mode " + filterCompareTestMode);
+      }
+    }
+
+    for (int i = 0; i < rowCount; i++) {
+      // Row-mode is the expected value.
+      Object expectedResult = resultObjectsArray[0][i];
+
+      for (int v = 1; v < FilterCompareTestMode.count; v++) {
+        FilterCompareTestMode filterCompareTestMode = FilterCompareTestMode.values()[v];
+        Object vectorResult = resultObjectsArray[v][i];
+        if (filterCompareTestMode == FilterCompareTestMode.FILTER_VECTOR_EXPRESSION &&
+            expectedResult == null &&
+            vectorResult != null) {
+          // This is OK.
+          boolean vectorBoolean = ((BooleanWritable) vectorResult).get();
+          if (vectorBoolean) {
+            Assert.fail(
+                "Row " + i +
+                " typeName1 " + typeName1 +
+                " typeName2 " + typeName2 +
+                " outputTypeName " + outputTypeInfo.getTypeName() +
+                " " + comparison +
+                " " + filterCompareTestMode +
+                " " + columnScalarMode +
+                " result is NOT NULL and true" +
+                " does not match row-mode expected result is NULL which means false here" +
+                (columnScalarMode == ColumnScalarMode.SCALAR_COLUMN ?
+                    " scalar1 " + scalar1Object.toString() : "") +
+                " row values " + Arrays.toString(randomRows[i]) +
+                (columnScalarMode == ColumnScalarMode.COLUMN_SCALAR ?
+                    " scalar2 " + scalar2Object.toString() : ""));
+          }
+        } else if (expectedResult == null || vectorResult == null) {
+          if (expectedResult != null || vectorResult != null) {
+            Assert.fail(
+                "Row " + i +
+                " typeName1 " + typeName1 +
+                " typeName2 " + typeName2 +
+                " outputTypeName " + outputTypeInfo.getTypeName() +
+                " " + comparison +
+                " " + filterCompareTestMode +
+                " " + columnScalarMode +
+                " result is NULL " + (vectorResult == null) +
+                " does not match row-mode expected result is NULL " + (expectedResult == null) +
+                (columnScalarMode == ColumnScalarMode.SCALAR_COLUMN ?
+                    " scalar1 " + scalar1Object.toString() : "") +
+                " row values " + Arrays.toString(randomRows[i]) +
+                (columnScalarMode == ColumnScalarMode.COLUMN_SCALAR ?
+                    " scalar2 " + scalar2Object.toString() : ""));
+          }
+        } else {
+
+          if (!expectedResult.equals(vectorResult)) {
+            Assert.fail(
+                "Row " + i +
+                " typeName1 " + typeName1 +
+                " typeName2 " + typeName2 +
+                " outputTypeName " + outputTypeInfo.getTypeName() +
+                " " + comparison +
+                " " + filterCompareTestMode +
+                " " + columnScalarMode +
+                " result " + vectorResult.toString() +
+                " (" + vectorResult.getClass().getSimpleName() + ")" +
+                " does not match row-mode expected result " + expectedResult.toString() +
+                " (" + expectedResult.getClass().getSimpleName() + ")" +
+                (columnScalarMode == ColumnScalarMode.SCALAR_COLUMN ?
+                    " scalar1 " + scalar1Object.toString() : "") +
+                " row values " + Arrays.toString(randomRows[i]) +
+                (columnScalarMode == ColumnScalarMode.COLUMN_SCALAR ?
+                    " scalar2 " + scalar2Object.toString() : ""));
+          }
+        }
+      }
+    }
+  }
+
+  private void doRowFilterCompareTest(TypeInfo typeInfo1,
+      TypeInfo typeInfo2,
+      List<String> columns, List<ExprNodeDesc> children,
+      ExprNodeGenericFuncDesc exprDesc,
+      Comparison comparison,
+      Object[][] randomRows, ColumnScalarMode columnScalarMode,
+      ObjectInspector rowInspector,
+      TypeInfo outputTypeInfo, Object[] resultObjects) throws Exception {
+
+    /*
+    System.out.println(
+        "*DEBUG* typeInfo " + typeInfo1.toString() +
+        " typeInfo2 " + typeInfo2 +
+        " filterCompareTestMode ROW_MODE" +
+        " columnScalarMode " + columnScalarMode +
+        " exprDesc " + exprDesc.toString());
+    */
+
+    HiveConf hiveConf = new HiveConf();
+    ExprNodeEvaluator evaluator =
+        ExprNodeEvaluatorFactory.get(exprDesc, hiveConf);
+    evaluator.initialize(rowInspector);
+
+    ObjectInspector objectInspector =
+        TypeInfoUtils.getStandardWritableObjectInspectorFromTypeInfo(
+            outputTypeInfo);
+
+    final int rowCount = randomRows.length;
+    for (int i = 0; i < rowCount; i++) {
+      Object[] row = randomRows[i];
+      Object result = evaluator.evaluate(row);
+      Object copyResult = null;
+      try {
+        copyResult =
+            ObjectInspectorUtils.copyToStandardObject(
+                result, objectInspector, ObjectInspectorCopyOption.WRITABLE);
+      } catch (Exception e) {
+        Assert.fail(e.toString());
+      }
+      resultObjects[i] = copyResult;
+    }
+  }
+
+  private void extractResultObjects(VectorizedRowBatch batch, int rowIndex,
+      VectorExtractRow resultVectorExtractRow, Object[] scrqtchRow,
+      ObjectInspector objectInspector, Object[] resultObjects) {
+
+    boolean selectedInUse = batch.selectedInUse;
+    int[] selected = batch.selected;
+    for (int logicalIndex = 0; logicalIndex < batch.size; logicalIndex++) {
+      final int batchIndex = (selectedInUse ? selected[logicalIndex] : logicalIndex);
+      resultVectorExtractRow.extractRow(batch, batchIndex, scrqtchRow);
+
+      Object copyResult =
+          ObjectInspectorUtils.copyToStandardObject(
+              scrqtchRow[0], objectInspector, ObjectInspectorCopyOption.WRITABLE);
+      resultObjects[rowIndex++] = copyResult;
+    }
+  }
+
+  private void doVectorFilterCompareTest(TypeInfo typeInfo1,
+      TypeInfo typeInfo2,
+      List<String> columns,
+      String[] columnNames,
+      TypeInfo[] typeInfos, DataTypePhysicalVariation[] dataTypePhysicalVariations,
+      List<ExprNodeDesc> children,
+      ExprNodeGenericFuncDesc exprDesc,
+      Comparison comparison,
+      FilterCompareTestMode filterCompareTestMode, ColumnScalarMode columnScalarMode,
+      VectorRandomBatchSource batchSource,
+      ObjectInspector objectInspector,
+      TypeInfo outputTypeInfo, Object[] resultObjects)
+          throws Exception {
+
+    HiveConf hiveConf = new HiveConf();
+    if (filterCompareTestMode == FilterCompareTestMode.ADAPTOR) {
+      hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_TEST_VECTOR_ADAPTOR_OVERRIDE, true);
+
+      // Don't use DECIMAL_64 with the VectorUDFAdaptor.
+      dataTypePhysicalVariations = null;
+    }
+
+    VectorizationContext vectorizationContext =
+        new VectorizationContext(
+            "name",
+            columns,
+            Arrays.asList(typeInfos),
+            dataTypePhysicalVariations == null ? null : Arrays.asList(dataTypePhysicalVariations),
+            hiveConf);
+    final VectorExpressionDescriptor.Mode mode;
+    switch (filterCompareTestMode) {
+    case ADAPTOR:
+    case COMPARE_VECTOR_EXPRESSION:
+      mode = VectorExpressionDescriptor.Mode.PROJECTION;
+      break;
+    case FILTER_VECTOR_EXPRESSION:
+      mode = VectorExpressionDescriptor.Mode.FILTER;
+      break;
+    default:
+      throw new RuntimeException("Unexpected filter compare mode " + filterCompareTestMode);
+    }
+    VectorExpression vectorExpression =
+        vectorizationContext.getVectorExpression(
+            exprDesc, mode);
+    vectorExpression.transientInit();
+
+    if (filterCompareTestMode == FilterCompareTestMode.COMPARE_VECTOR_EXPRESSION &&
+        vectorExpression instanceof VectorUDFAdaptor) {
+      System.out.println(
+          "*NO NATIVE VECTOR EXPRESSION* typeInfo1 " + typeInfo1.toString() +
+          " typeInfo2 " + typeInfo2.toString() +
+          " " + comparison + " " +
+          " filterCompareTestMode " + filterCompareTestMode +
+          " columnScalarMode " + columnScalarMode +
+          " vectorExpression " + vectorExpression.toString());
+    }
+
+    String[] outputScratchTypeNames= vectorizationContext.getScratchColumnTypeNames();
+    DataTypePhysicalVariation[] outputDataTypePhysicalVariations =
+        vectorizationContext.getScratchDataTypePhysicalVariations();
+
+    VectorizedRowBatchCtx batchContext =
+        new VectorizedRowBatchCtx(
+            columnNames,
+            typeInfos,
+            dataTypePhysicalVariations,
+            /* dataColumnNums */ null,
+            /* partitionColumnCount */ 0,
+            /* virtualColumnCount */ 0,
+            /* neededVirtualColumns */ null,
+            outputScratchTypeNames,
+            outputDataTypePhysicalVariations);
+
+    VectorizedRowBatch batch = batchContext.createVectorizedRowBatch();
+
+    VectorExtractRow resultVectorExtractRow = new VectorExtractRow();
+    final int outputColumnNum = vectorExpression.getOutputColumnNum();
+    resultVectorExtractRow.init(
+        new TypeInfo[] { outputTypeInfo }, new int[] { outputColumnNum });
+    Object[] scrqtchRow = new Object[1];
+
+    // System.out.println("*VECTOR EXPRESSION* " + vectorExpression.getClass().getSimpleName());
+
+    /*
+    System.out.println(
+        "*DEBUG* typeInfo1 " + typeInfo1.toString() +
+        " typeInfo2 " + typeInfo2.toString() +
+        " " + comparison + " " +
+        " filterCompareTestMode " + filterCompareTestMode +
+        " columnScalarMode " + columnScalarMode +
+        " vectorExpression " + vectorExpression.toString());
+    */
+
+    final boolean isFilter = (mode == VectorExpressionDescriptor.Mode.FILTER);
+    boolean copySelectedInUse = false;
+    int[] copySelected = new int[VectorizedRowBatch.DEFAULT_SIZE];
+
+    batchSource.resetBatchIteration();
+    int rowIndex = 0;
+    while (true) {
+      if (!batchSource.fillNextBatch(batch)) {
+        break;
+      }
+      final int originalBatchSize = batch.size;
+      if (isFilter) {
+        copySelectedInUse = batch.selectedInUse;
+        if (batch.selectedInUse) {
+          System.arraycopy(batch.selected, 0, copySelected, 0, originalBatchSize);
+        }
+      }
+
+      // In filter mode, the batch size can be made smaller.
+      vectorExpression.evaluate(batch);
+
+      if (!isFilter) {
+        extractResultObjects(batch, rowIndex, resultVectorExtractRow, scrqtchRow,
+            objectInspector, resultObjects);
+      } else {
+        final int currentBatchSize = batch.size;
+        if (copySelectedInUse && batch.selectedInUse) {
+          int selectIndex = 0;
+          for (int i = 0; i < originalBatchSize; i++) {
+            final int originalBatchIndex = copySelected[i];
+            final boolean booleanResult;
+            if (selectIndex < currentBatchSize && batch.selected[selectIndex] == originalBatchIndex) {
+              booleanResult = true;
+              selectIndex++;
+            } else {
+              booleanResult = false;
+            }
+            resultObjects[rowIndex + i] = new BooleanWritable(booleanResult);
+          }
+        } else if (batch.selectedInUse) {
+          int selectIndex = 0;
+          for (int i = 0; i < originalBatchSize; i++) {
+            final boolean booleanResult;
+            if (selectIndex < currentBatchSize && batch.selected[selectIndex] == i) {
+              booleanResult = true;
+              selectIndex++;
+            } else {
+              booleanResult = false;
+            }
+            resultObjects[rowIndex + i] = new BooleanWritable(booleanResult);
+          }
+        } else if (currentBatchSize == 0) {
+          // Whole batch got zapped.
+          for (int i = 0; i < originalBatchSize; i++) {
+            resultObjects[rowIndex + i] = new BooleanWritable(false);
+          }
+        } else {
+          // Every row kept.
+          for (int i = 0; i < originalBatchSize; i++) {
+            resultObjects[rowIndex + i] = new BooleanWritable(true);
+          }
+        }
+      }
+      rowIndex += originalBatchSize;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/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 9d78ca6..58e32ca 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
@@ -34,6 +34,7 @@ import org.apache.hadoop.hive.ql.exec.vector.VectorizationContext;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatchCtx;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.exec.vector.udf.VectorUDFAdaptor;
 import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
@@ -155,6 +156,20 @@ public class TestVectorIfStatement {
     doIfTests(random, "decimal(10,4)");
   }
 
+  @Test
+  public void testDecimal64() throws Exception {
+    Random random = new Random(12882);
+
+    doIfTestsWithDiffColumnScalar(
+        random, "decimal(10,4)", ColumnScalarMode.COLUMN_COLUMN, DataTypePhysicalVariation.DECIMAL_64, false, false);
+    doIfTestsWithDiffColumnScalar(
+        random, "decimal(10,4)", ColumnScalarMode.COLUMN_SCALAR, DataTypePhysicalVariation.DECIMAL_64, false, false);
+    doIfTestsWithDiffColumnScalar(
+        random, "decimal(10,4)", ColumnScalarMode.SCALAR_COLUMN, DataTypePhysicalVariation.DECIMAL_64, false, false);
+    doIfTestsWithDiffColumnScalar(
+        random, "decimal(10,4)", ColumnScalarMode.SCALAR_SCALAR, DataTypePhysicalVariation.DECIMAL_64, false, false);
+  }
+
   public enum IfStmtTestMode {
     ROW_MODE,
     ADAPTOR_WHEN,
@@ -280,22 +295,6 @@ public class TestVectorIfStatement {
 
     String[] columnNames = columns.toArray(new String[0]);
 
-    String[] outputScratchTypeNames = new String[] { typeName };
-    DataTypePhysicalVariation[] outputDataTypePhysicalVariations =
-        new DataTypePhysicalVariation[] { dataTypePhysicalVariation };
-
-    VectorizedRowBatchCtx batchContext =
-        new VectorizedRowBatchCtx(
-            columnNames,
-            rowSource.typeInfos(),
-            rowSource.dataTypePhysicalVariations(),
-            /* dataColumnNums */ null,
-            /* partitionColumnCount */ 0,
-            /* virtualColumnCount */ 0,
-            /* neededVirtualColumns */ null,
-            outputScratchTypeNames,
-            outputDataTypePhysicalVariations);
-
     Object[][] randomRows = rowSource.randomRows(100000);
 
     VectorRandomBatchSource batchSource =
@@ -324,13 +323,13 @@ public class TestVectorIfStatement {
         doVectorIfTest(
             typeInfo,
             columns,
+            columnNames,
             rowSource.typeInfos(),
             rowSource.dataTypePhysicalVariations(),
             children,
             ifStmtTestMode,
             columnScalarMode,
             batchSource,
-            batchContext,
             resultObjects);
         break;
       default:
@@ -354,14 +353,6 @@ public class TestVectorIfStatement {
           }
         } else {
 
-          if (isDecimal64 && expectedResult instanceof LongWritable) {
-
-            HiveDecimalWritable expectedHiveDecimalWritable = new HiveDecimalWritable(0);
-            expectedHiveDecimalWritable.deserialize64(
-                ((LongWritable) expectedResult).get(), decimal64Scale);
-            expectedResult = expectedHiveDecimalWritable;
-          }
-
           if (!expectedResult.equals(vectorResult)) {
             Assert.fail(
                 "Row " + i + " " + IfStmtTestMode.values()[v] +
@@ -418,10 +409,11 @@ public class TestVectorIfStatement {
 
   private void doVectorIfTest(TypeInfo typeInfo,
       List<String> columns,
+      String[] columnNames,
       TypeInfo[] typeInfos, DataTypePhysicalVariation[] dataTypePhysicalVariations,
       List<ExprNodeDesc> children,
       IfStmtTestMode ifStmtTestMode, ColumnScalarMode columnScalarMode,
-      VectorRandomBatchSource batchSource, VectorizedRowBatchCtx batchContext,
+      VectorRandomBatchSource batchSource,
       Object[] resultObjects)
           throws Exception {
 
@@ -453,10 +445,36 @@ public class TestVectorIfStatement {
             hiveConf);
     VectorExpression vectorExpression = vectorizationContext.getVectorExpression(exprDesc);
 
+    if (ifStmtTestMode == IfStmtTestMode.VECTOR_EXPRESSION &&
+        vectorExpression instanceof VectorUDFAdaptor) {
+      System.out.println(
+          "*NO NATIVE VECTOR EXPRESSION* typeInfo " + typeInfo.toString() +
+          " ifStmtTestMode " + ifStmtTestMode +
+          " columnScalarMode " + columnScalarMode +
+          " vectorExpression " + vectorExpression.toString());
+    }
+
+    String[] outputScratchTypeNames= vectorizationContext.getScratchColumnTypeNames();
+    DataTypePhysicalVariation[] outputDataTypePhysicalVariations =
+        vectorizationContext.getScratchDataTypePhysicalVariations();
+
+    VectorizedRowBatchCtx batchContext =
+        new VectorizedRowBatchCtx(
+            columnNames,
+            typeInfos,
+            dataTypePhysicalVariations,
+            /* dataColumnNums */ null,
+            /* partitionColumnCount */ 0,
+            /* virtualColumnCount */ 0,
+            /* neededVirtualColumns */ null,
+            outputScratchTypeNames,
+            outputDataTypePhysicalVariations);
+
     VectorizedRowBatch batch = batchContext.createVectorizedRowBatch();
 
     VectorExtractRow resultVectorExtractRow = new VectorExtractRow();
-    resultVectorExtractRow.init(new TypeInfo[] { typeInfo }, new int[] { columns.size() });
+    resultVectorExtractRow.init(
+        new TypeInfo[] { typeInfo }, new int[] { vectorExpression.getOutputColumnNum() });
     Object[] scrqtchRow = new Object[1];
 
     // System.out.println("*VECTOR EXPRESSION* " + vectorExpression.getClass().getSimpleName());
@@ -466,7 +484,7 @@ public class TestVectorIfStatement {
         "*DEBUG* typeInfo " + typeInfo.toString() +
         " ifStmtTestMode " + ifStmtTestMode +
         " columnScalarMode " + columnScalarMode +
-        " vectorExpression " + vectorExpression.getClass().getSimpleName());
+        " vectorExpression " + vectorExpression.toString());
     */
 
     batchSource.resetBatchIteration();

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/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 289efb9..54c085b 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
@@ -42,6 +42,7 @@ import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatchCtx;
 import org.apache.hadoop.hive.ql.exec.vector.VectorRandomRowSource.GenerationSpec;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.exec.vector.udf.VectorUDFAdaptor;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc;
@@ -406,6 +407,14 @@ public class TestVectorNegative {
     VectorExpression vectorExpression = vectorizationContext.getVectorExpression(exprDesc);
     vectorExpression.transientInit();
 
+    if (negativeTestMode == NegativeTestMode.VECTOR_EXPRESSION &&
+        vectorExpression instanceof VectorUDFAdaptor) {
+      System.out.println(
+          "*NO NATIVE VECTOR EXPRESSION* typeInfo " + typeInfo.toString() +
+          " negativeTestMode " + negativeTestMode +
+          " vectorExpression " + vectorExpression.toString());
+    }
+
     String[] outputScratchTypeNames= vectorizationContext.getScratchColumnTypeNames();
 
     VectorizedRowBatchCtx batchContext =

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/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 3d030e6..69fd70c 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
@@ -40,6 +40,7 @@ import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatchCtx;
 import org.apache.hadoop.hive.ql.exec.vector.VectorRandomRowSource.GenerationSpec;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.exec.vector.udf.VectorUDFAdaptor;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc;
@@ -400,6 +401,16 @@ public class TestVectorStringConcat {
     VectorExpression vectorExpression = vectorizationContext.getVectorExpression(exprDesc);
     vectorExpression.transientInit();
 
+    if (stringConcatTestMode == StringConcatTestMode.VECTOR_EXPRESSION &&
+        vectorExpression instanceof VectorUDFAdaptor) {
+      System.out.println(
+          "*NO NATIVE VECTOR EXPRESSION* stringTypeInfo1 " + stringTypeInfo1.toString() +
+          " stringTypeInfo2 " + stringTypeInfo2.toString() +
+          " stringConcatTestMode " + stringConcatTestMode +
+          " columnScalarMode " + columnScalarMode +
+          " vectorExpression " + vectorExpression.toString());
+    }
+
     VectorizedRowBatch batch = batchContext.createVectorizedRowBatch();
 
     VectorExtractRow resultVectorExtractRow = new VectorExtractRow();

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/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 4398554..f029358 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
@@ -38,7 +38,7 @@ import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatchCtx;
 import org.apache.hadoop.hive.ql.exec.vector.VectorRandomRowSource.GenerationSpec;
 import org.apache.hadoop.hive.ql.exec.vector.VectorRandomRowSource.StringGenerationOption;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.TestVectorTimestampExtract.TimestampExtractTestMode;
+import org.apache.hadoop.hive.ql.exec.vector.udf.VectorUDFAdaptor;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc;
@@ -104,8 +104,6 @@ public class TestVectorStringUnary {
       doTests(random, typeName, "trim");
 
       doTests(random, typeName, "initcap");
-
-      doTests(random, typeName, "hex");
     }
 
     doTests(random, typeName, "lower");
@@ -341,6 +339,14 @@ public class TestVectorStringUnary {
             hiveConf);
     VectorExpression vectorExpression = vectorizationContext.getVectorExpression(exprDesc);
 
+    if (stringUnaryTestMode == StringUnaryTestMode.VECTOR_EXPRESSION &&
+        vectorExpression instanceof VectorUDFAdaptor) {
+      System.out.println(
+          "*NO NATIVE VECTOR EXPRESSION* typeInfo " + typeInfo.toString() +
+          " stringUnaryTestMode " + stringUnaryTestMode +
+          " vectorExpression " + vectorExpression.toString());
+    }
+
     VectorizedRowBatch batch = batchContext.createVectorizedRowBatch();
 
     VectorExtractRow resultVectorExtractRow = new VectorExtractRow();

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/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 62d296d..694f6f7 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
@@ -38,6 +38,7 @@ import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatchCtx;
 import org.apache.hadoop.hive.ql.exec.vector.VectorRandomRowSource.GenerationSpec;
 import org.apache.hadoop.hive.ql.exec.vector.VectorRandomRowSource.StringGenerationOption;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.exec.vector.udf.VectorUDFAdaptor;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc;
@@ -320,6 +321,14 @@ public class TestVectorSubStr {
             hiveConf);
     VectorExpression vectorExpression = vectorizationContext.getVectorExpression(exprDesc);
 
+    if (subStrTestMode == SubStrTestMode.VECTOR_EXPRESSION &&
+        vectorExpression instanceof VectorUDFAdaptor) {
+      System.out.println(
+          "*NO NATIVE VECTOR EXPRESSION* typeInfo " + typeInfo.toString() +
+          " subStrTestMode " + subStrTestMode +
+          " vectorExpression " + vectorExpression.toString());
+    }
+
     VectorizedRowBatch batch = batchContext.createVectorizedRowBatch();
 
     VectorExtractRow resultVectorExtractRow = new VectorExtractRow();

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/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 3220f4c..5d5e4c9 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
@@ -28,6 +28,7 @@ import org.apache.hadoop.hive.ql.exec.vector.VectorExtractRow;
 import org.apache.hadoop.hive.ql.exec.vector.VectorRandomBatchSource;
 import org.apache.hadoop.hive.ql.exec.vector.VectorRandomRowSource;
 import org.apache.hadoop.hive.ql.exec.vector.VectorRandomRowSource.GenerationSpec;
+import org.apache.hadoop.hive.ql.exec.vector.udf.VectorUDFAdaptor;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizationContext;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatchCtx;
@@ -395,6 +396,14 @@ public class TestVectorTimestampExtract {
     VectorExpression vectorExpression = vectorizationContext.getVectorExpression(exprDesc);
     vectorExpression.transientInit();
 
+    if (timestampExtractTestMode == TimestampExtractTestMode.VECTOR_EXPRESSION &&
+        vectorExpression instanceof VectorUDFAdaptor) {
+      System.out.println(
+          "*NO NATIVE VECTOR EXPRESSION* dateTimeStringTypeInfo " + dateTimeStringTypeInfo.toString() +
+          " timestampExtractTestMode " + timestampExtractTestMode +
+          " vectorExpression " + vectorExpression.toString());
+    }
+
     // System.out.println("*VECTOR EXPRESSION* " + vectorExpression.getClass().getSimpleName());
 
     /*

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/test/results/clientpositive/convert_decimal64_to_decimal.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/convert_decimal64_to_decimal.q.out b/ql/src/test/results/clientpositive/convert_decimal64_to_decimal.q.out
index 8e538d2..5189957 100644
--- a/ql/src/test/results/clientpositive/convert_decimal64_to_decimal.q.out
+++ b/ql/src/test/results/clientpositive/convert_decimal64_to_decimal.q.out
@@ -181,7 +181,7 @@ STAGE PLANS:
               Filter Vectorization:
                   className: VectorFilterOperator
                   native: true
-                  predicateExpression: SelectColumnIsNotNull(col 33:decimal(4,2))(children: ConvertDecimal64ToDecimal(col 19:decimal(4,2)/DECIMAL_64) -> 33:decimal(4,2))
+                  predicateExpression: SelectColumnIsNotNull(col 19:decimal(4,2)/DECIMAL_64)
               predicate: decimal0402_col_20 is not null (type: boolean)
               Statistics: Num rows: 1000 Data size: 2087309 Basic stats: COMPLETE Column stats: NONE
               Select Operator
@@ -199,7 +199,7 @@ STAGE PLANS:
                     0 _col0 (type: decimal(9,2))
                     1 _col1 (type: decimal(9,2))
                   Map Join Vectorization:
-                      bigTableKeyExpressions: ConvertDecimal64ToDecimal(col 19:decimal(9,2)/DECIMAL_64) -> 34:decimal(9,2)
+                      bigTableKeyExpressions: ConvertDecimal64ToDecimal(col 19:decimal(9,2)/DECIMAL_64) -> 33:decimal(9,2)
                       bigTableValueExpressions: col 14:smallint
                       className: VectorMapJoinOperator
                       native: false
@@ -244,7 +244,7 @@ STAGE PLANS:
               includeColumns: [14, 19]
               dataColumns: tinyint_col_1:tinyint, float_col_2:float, bigint_col_3:bigint, boolean_col_4:boolean, decimal0202_col_5:decimal(2,2)/DECIMAL_64, decimal1612_col_6:decimal(16,12)/DECIMAL_64, double_col_7:double, char0205_col_8:char(205), bigint_col_9:bigint, decimal1202_col_10:decimal(12,2)/DECIMAL_64, boolean_col_11:boolean, double_col_12:double, decimal2208_col_13:decimal(22,8), decimal3722_col_14:decimal(37,22), smallint_col_15:smallint, decimal2824_col_16:decimal(28,24), boolean_col_17:boolean, float_col_18:float, timestamp_col_19:timestamp, decimal0402_col_20:decimal(4,2)/DECIMAL_64, char0208_col_21:char(208), char0077_col_22:char(77), decimal2915_col_23:decimal(29,15), char0234_col_24:char(234), timestamp_col_25:timestamp, tinyint_col_26:tinyint, decimal3635_col_27:decimal(36,35), boolean_col_28:boolean, float_col_29:float, smallint_col_30:smallint, varchar0200_col_31:varchar(200), boolean_col_32:boolean
               partitionColumnCount: 0
-              scratchColumnTypeNames: [decimal(4,2), decimal(9,2)]
+              scratchColumnTypeNames: [decimal(9,2)]
       Local Work:
         Map Reduce Local Work
       Reduce Vectorization:

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/test/results/clientpositive/llap/convert_decimal64_to_decimal.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/convert_decimal64_to_decimal.q.out b/ql/src/test/results/clientpositive/llap/convert_decimal64_to_decimal.q.out
index cbc6b25..fddd2cb 100644
--- a/ql/src/test/results/clientpositive/llap/convert_decimal64_to_decimal.q.out
+++ b/ql/src/test/results/clientpositive/llap/convert_decimal64_to_decimal.q.out
@@ -163,7 +163,7 @@ STAGE PLANS:
                     Filter Vectorization:
                         className: VectorFilterOperator
                         native: true
-                        predicateExpression: SelectColumnIsNotNull(col 20:decimal(8,1))(children: ConvertDecimal64ToDecimal(col 3:decimal(8,1)/DECIMAL_64) -> 20:decimal(8,1))
+                        predicateExpression: SelectColumnIsNotNull(col 3:decimal(8,1)/DECIMAL_64)
                     predicate: decimal0801_col_4 is not null (type: boolean)
                     Statistics: Num rows: 1026 Data size: 109272 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
@@ -181,7 +181,7 @@ STAGE PLANS:
                           0 _col0 (type: decimal(9,2))
                           1 _col1 (type: decimal(9,2))
                         Map Join Vectorization:
-                            bigTableKeyExpressions: ConvertDecimal64ToDecimal(col 3:decimal(9,2)/DECIMAL_64) -> 21:decimal(9,2)
+                            bigTableKeyExpressions: ConvertDecimal64ToDecimal(col 3:decimal(9,2)/DECIMAL_64) -> 20:decimal(9,2)
                             className: VectorMapJoinOperator
                             native: false
                             nativeConditionsMet: hive.mapjoin.optimized.hashtable IS true, hive.vectorized.execution.mapjoin.native.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, One MapJoin Condition IS true, No nullsafe IS true, Small table vectorizes IS true
@@ -229,7 +229,7 @@ STAGE PLANS:
                     includeColumns: [3]
                     dataColumns: float_col_1:float, varchar0037_col_2:varchar(37), decimal2912_col_3:decimal(29,12), decimal0801_col_4:decimal(8,1)/DECIMAL_64, timestamp_col_5:timestamp, boolean_col_6:boolean, string_col_7:string, tinyint_col_8:tinyint, boolean_col_9:boolean, decimal1614_col_10:decimal(16,14)/DECIMAL_64, boolean_col_11:boolean, float_col_12:float, char0116_col_13:char(116), boolean_col_14:boolean, string_col_15:string, double_col_16:double, string_col_17:string, bigint_col_18:bigint, int_col_19:int
                     partitionColumnCount: 0
-                    scratchColumnTypeNames: [decimal(8,1), decimal(9,2), bigint]
+                    scratchColumnTypeNames: [decimal(9,2), bigint]
         Map 3 
             Map Operator Tree:
                 TableScan
@@ -243,7 +243,7 @@ STAGE PLANS:
                     Filter Vectorization:
                         className: VectorFilterOperator
                         native: true
-                        predicateExpression: SelectColumnIsNotNull(col 33:decimal(4,2))(children: ConvertDecimal64ToDecimal(col 19:decimal(4,2)/DECIMAL_64) -> 33:decimal(4,2))
+                        predicateExpression: SelectColumnIsNotNull(col 19:decimal(4,2)/DECIMAL_64)
                     predicate: decimal0402_col_20 is not null (type: boolean)
                     Statistics: Num rows: 950 Data size: 104800 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
@@ -282,7 +282,7 @@ STAGE PLANS:
                     includeColumns: [14, 19]
                     dataColumns: tinyint_col_1:tinyint, float_col_2:float, bigint_col_3:bigint, boolean_col_4:boolean, decimal0202_col_5:decimal(2,2)/DECIMAL_64, decimal1612_col_6:decimal(16,12)/DECIMAL_64, double_col_7:double, char0205_col_8:char(205), bigint_col_9:bigint, decimal1202_col_10:decimal(12,2)/DECIMAL_64, boolean_col_11:boolean, double_col_12:double, decimal2208_col_13:decimal(22,8), decimal3722_col_14:decimal(37,22), smallint_col_15:smallint, decimal2824_col_16:decimal(28,24), boolean_col_17:boolean, float_col_18:float, timestamp_col_19:timestamp, decimal0402_col_20:decimal(4,2)/DECIMAL_64, char0208_col_21:char(208), char0077_col_22:char(77), decimal2915_col_23:decimal(29,15), char0234_col_24:char(234), timestamp_col_25:timestamp, tinyint_col_26:tinyint, decimal3635_col_27:decimal(36,35), boolean_col_28:boolean, float_col_29:float, smallint_col_30:smallint, varchar0200_col_31:varchar(200), boolean_col_32:boolean
                     partitionColumnCount: 0
-                    scratchColumnTypeNames: [decimal(4,2)]
+                    scratchColumnTypeNames: []
         Reducer 2 
             Execution mode: vectorized, llap
             Reduce Vectorization:

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/test/results/clientpositive/llap/vector_case_when_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/vector_case_when_1.q.out b/ql/src/test/results/clientpositive/llap/vector_case_when_1.q.out
index b531d79..3ab6547 100644
--- a/ql/src/test/results/clientpositive/llap/vector_case_when_1.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_case_when_1.q.out
@@ -523,8 +523,8 @@ STAGE PLANS:
                     Select Vectorization:
                         className: VectorSelectOperator
                         native: true
-                        projectedOutputColumnNums: [4, 22, 24, 25, 26, 27, 28, 30, 31, 32, 33, 34, 35, 38, 40, 43, 44]
-                        selectExpressions: IfExprStringScalarStringGroupColumn(col 17:boolean, val Singlecol 21:string)(children: LongColEqualLongScalar(col 4:int, val 1) -> 17:boolean, IfExprStringScalarStringGroupColumn(col 18:boolean, val Twocol 22:string)(children: LongColEqualLongScalar(col 4:int, val 2) -> 18:boolean, IfExprStringScalarStringGroupColumn(col 19:boolean, val Somecol 21:string)(children: LongColLessLongScalar(col 4:int, val 10) -> 19:boolean, IfExprStringScalarStringScalar(col 20:boolean, val Many, val Huge number)(children: LongColLessLongScalar(col 4:int, val 100) -> 20:boolean) -> 21:string) -> 22:string) -> 21:string) -> 22:string, IfExprStringScalarStringGroupColumn(col 17:boolean, val Singlecol 23:string)(children: LongColEqualLongScalar(col 4:int, val 1) -> 17:boolean, IfExprStringScalarStringGroupColumn(col 18:boolean, val Twocol 24:string)(children: LongColEqualLongScalar(col 4:int, val 2) -> 18:boolean, IfExprStringScalarStringGroupColumn(col 19:boolean
 , val Somecol 23:string)(children: LongColLessLongScalar(col 4:int, val 10) -> 19:boolean, IfExprColumnNull(col 20:boolean, col 21:string, null)(children: LongColLessLongScalar(col 4:int, val 100) -> 20:boolean, ConstantVectorExpression(val Many) -> 21:string) -> 23:string) -> 24:string) -> 23:string) -> 24:string, IfExprStringScalarStringGroupColumn(col 17:boolean, val Singlecol 23:string)(children: LongColEqualLongScalar(col 4:int, val 1) -> 17:boolean, IfExprStringScalarStringGroupColumn(col 18:boolean, val Twocol 25:string)(children: LongColEqualLongScalar(col 4:int, val 2) -> 18:boolean, IfExprStringScalarStringGroupColumn(col 19:boolean, val Somecol 23:string)(children: LongColLessLongScalar(col 4:int, val 10) -> 19:boolean, IfExprNullNull(null, null) -> 23:string) -> 25:string) -> 23:string) -> 25:string, IfExprLongColumnLongColumn(col 17:boolean, col 18:date, col 19:date)(children: StringGroupColEqualCharScalar(col 14:char(10), val SHIP) -> 17:boolean, VectorUDFDateAddColSca
 lar(col 10:date, val 10) -> 18:date, VectorUDFDateAddColScalar(col 10:date, val 5) -> 19:date) -> 26:date, IfExprDoubleColumnLongScalar(col 17:boolean, col 28:double, val 0)(children: StringGroupColEqualCharScalar(col 8:char(1), val N) -> 17:boolean, DoubleColMultiplyDoubleColumn(col 5:double, col 27:double)(children: DoubleScalarSubtractDoubleColumn(val 1.0, col 6:double) -> 27:double) -> 28:double) -> 27:double, IfExprDoubleColumnDoubleScalar(col 17:boolean, col 29:double, val 0.0)(children: StringGroupColEqualCharScalar(col 8:char(1), val N) -> 17:boolean, DoubleColMultiplyDoubleColumn(col 5:double, col 28:double)(children: DoubleScalarSubtractDoubleColumn(val 1.0, col 6:double) -> 28:double) -> 29:double) -> 28:double, IfExprNullColumn(col 17:boolean, null, col 46)(children: StringGroupColEqualStringScalar(col 23:string, val DELIVER IN PERSON)(children: CastStringGroupToString(col 13:varchar(20)) -> 23:string) -> 17:boolean, ConvertDecimal64ToDecimal(col 7:decimal(10,2)/DECIMAL_
 64) -> 46:decimal(10,2)) -> 30:decimal(10,2), IfExprColumnNull(col 18:boolean, col 47:decimal(10,2), null)(children: StringGroupColEqualStringScalar(col 23:string, val TAKE BACK RETURN)(children: CastStringGroupToString(col 13:varchar(20)) -> 23:string) -> 18:boolean, ConvertDecimal64ToDecimal(col 7:decimal(10,2)/DECIMAL_64) -> 47:decimal(10,2)) -> 31:decimal(10,2), VectorUDFAdaptor(if((CAST( l_shipinstruct AS STRING) = 'DELIVER IN PERSON'), 0, l_tax))(children: StringGroupColEqualStringScalar(col 23:string, val DELIVER IN PERSON)(children: CastStringGroupToString(col 13:varchar(20)) -> 23:string) -> 19:boolean) -> 32:decimal(12,2), VectorUDFAdaptor(if((CAST( l_shipinstruct AS STRING) = 'TAKE BACK RETURN'), l_tax, 0))(children: StringGroupColEqualStringScalar(col 23:string, val TAKE BACK RETURN)(children: CastStringGroupToString(col 13:varchar(20)) -> 23:string) -> 19:boolean) -> 33:decimal(12,2), VectorUDFAdaptor(if((CAST( l_shipinstruct AS STRING) = 'DELIVER IN PERSON'), 0, l_tax)
 )(children: StringGroupColEqualStringScalar(col 23:string, val DELIVER IN PERSON)(children: CastStringGroupToString(col 13:varchar(20)) -> 23:string) -> 19:boolean) -> 34:decimal(10,2), VectorUDFAdaptor(if((CAST( l_shipinstruct AS STRING) = 'TAKE BACK RETURN'), l_tax, 0))(children: StringGroupColEqualStringScalar(col 23:string, val TAKE BACK RETURN)(children: CastStringGroupToString(col 13:varchar(20)) -> 23:string) -> 19:boolean) -> 35:decimal(10,2), IfExprTimestampColumnColumn(col 19:boolean, col 36:timestampcol 37:timestamp)(children: LongColGreaterLongScalar(col 1:int, val 30) -> 19:boolean, CastDateToTimestamp(col 12:date) -> 36:timestamp, CastDateToTimestamp(col 11:date) -> 37:timestamp) -> 38:timestamp, IfExprColumnNull(col 19:boolean, col 39:int, null)(children: LongColGreaterLongScalar(col 2:int, val 10000) -> 19:boolean, VectorUDFDateDiffColCol(col 12:date, col 11:date) -> 39:int) -> 40:int, IfExprNullColumn(col 41:boolean, null, col 42)(children: LongColGreaterLongScalar(
 col 2:int, val 10000) -> 41:boolean, VectorUDFDateDiffColCol(col 12:date, col 11:date) -> 42:int) -> 43:int, IfExprLongScalarLongScalar(col 45:boolean, val 14245, val 14609)(children: LongColGreaterLongScalar(col 44:int, val 100)(children: LongColModuloLongScalar(col 2:int, val 500) -> 44:int) -> 45:boolean) -> 44:date
+                        projectedOutputColumnNums: [4, 22, 24, 25, 26, 27, 28, 30, 31, 32, 33, 34, 36, 40, 42, 45, 46]
+                        selectExpressions: IfExprStringScalarStringGroupColumn(col 17:boolean, val Singlecol 21:string)(children: LongColEqualLongScalar(col 4:int, val 1) -> 17:boolean, IfExprStringScalarStringGroupColumn(col 18:boolean, val Twocol 22:string)(children: LongColEqualLongScalar(col 4:int, val 2) -> 18:boolean, IfExprStringScalarStringGroupColumn(col 19:boolean, val Somecol 21:string)(children: LongColLessLongScalar(col 4:int, val 10) -> 19:boolean, IfExprStringScalarStringScalar(col 20:boolean, val Many, val Huge number)(children: LongColLessLongScalar(col 4:int, val 100) -> 20:boolean) -> 21:string) -> 22:string) -> 21:string) -> 22:string, IfExprStringScalarStringGroupColumn(col 17:boolean, val Singlecol 23:string)(children: LongColEqualLongScalar(col 4:int, val 1) -> 17:boolean, IfExprStringScalarStringGroupColumn(col 18:boolean, val Twocol 24:string)(children: LongColEqualLongScalar(col 4:int, val 2) -> 18:boolean, IfExprStringScalarStringGroupColumn(col 19:boolean
 , val Somecol 23:string)(children: LongColLessLongScalar(col 4:int, val 10) -> 19:boolean, IfExprColumnNull(col 20:boolean, col 21:string, null)(children: LongColLessLongScalar(col 4:int, val 100) -> 20:boolean, ConstantVectorExpression(val Many) -> 21:string) -> 23:string) -> 24:string) -> 23:string) -> 24:string, IfExprStringScalarStringGroupColumn(col 17:boolean, val Singlecol 23:string)(children: LongColEqualLongScalar(col 4:int, val 1) -> 17:boolean, IfExprStringScalarStringGroupColumn(col 18:boolean, val Twocol 25:string)(children: LongColEqualLongScalar(col 4:int, val 2) -> 18:boolean, IfExprStringScalarStringGroupColumn(col 19:boolean, val Somecol 23:string)(children: LongColLessLongScalar(col 4:int, val 10) -> 19:boolean, IfExprNullNull(null, null) -> 23:string) -> 25:string) -> 23:string) -> 25:string, IfExprLongColumnLongColumn(col 17:boolean, col 18:date, col 19:date)(children: StringGroupColEqualCharScalar(col 14:char(10), val SHIP) -> 17:boolean, VectorUDFDateAddColSca
 lar(col 10:date, val 10) -> 18:date, VectorUDFDateAddColScalar(col 10:date, val 5) -> 19:date) -> 26:date, IfExprDoubleColumnLongScalar(col 17:boolean, col 28:double, val 0)(children: StringGroupColEqualCharScalar(col 8:char(1), val N) -> 17:boolean, DoubleColMultiplyDoubleColumn(col 5:double, col 27:double)(children: DoubleScalarSubtractDoubleColumn(val 1.0, col 6:double) -> 27:double) -> 28:double) -> 27:double, IfExprDoubleColumnDoubleScalar(col 17:boolean, col 29:double, val 0.0)(children: StringGroupColEqualCharScalar(col 8:char(1), val N) -> 17:boolean, DoubleColMultiplyDoubleColumn(col 5:double, col 28:double)(children: DoubleScalarSubtractDoubleColumn(val 1.0, col 6:double) -> 28:double) -> 29:double) -> 28:double, IfExprNullColumn(col 17:boolean, null, col 48)(children: StringGroupColEqualStringScalar(col 23:string, val DELIVER IN PERSON)(children: CastStringGroupToString(col 13:varchar(20)) -> 23:string) -> 17:boolean, ConvertDecimal64ToDecimal(col 7:decimal(10,2)/DECIMAL_
 64) -> 48:decimal(10,2)) -> 30:decimal(10,2), IfExprColumnNull(col 18:boolean, col 49:decimal(10,2), null)(children: StringGroupColEqualStringScalar(col 23:string, val TAKE BACK RETURN)(children: CastStringGroupToString(col 13:varchar(20)) -> 23:string) -> 18:boolean, ConvertDecimal64ToDecimal(col 7:decimal(10,2)/DECIMAL_64) -> 49:decimal(10,2)) -> 31:decimal(10,2), VectorUDFAdaptor(if((CAST( l_shipinstruct AS STRING) = 'DELIVER IN PERSON'), 0, l_tax))(children: StringGroupColEqualStringScalar(col 23:string, val DELIVER IN PERSON)(children: CastStringGroupToString(col 13:varchar(20)) -> 23:string) -> 19:boolean) -> 32:decimal(12,2), VectorUDFAdaptor(if((CAST( l_shipinstruct AS STRING) = 'TAKE BACK RETURN'), l_tax, 0))(children: StringGroupColEqualStringScalar(col 23:string, val TAKE BACK RETURN)(children: CastStringGroupToString(col 13:varchar(20)) -> 23:string) -> 19:boolean) -> 33:decimal(12,2), IfExprDecimal64ScalarDecimal64Column(col 19:boolean, decimal64Val 0, decimalVal 0, col
  7:decimal(1,0)/DECIMAL_64)(children: StringGroupColEqualStringScalar(col 23:string, val DELIVER IN PERSON)(children: CastStringGroupToString(col 13:varchar(20)) -> 23:string) -> 19:boolean) -> 34:decimal(10,2)/DECIMAL_64, IfExprDecimal64ColumnDecimal64Scalar(col 35:boolean, col 7:decimal(10,2)/DECIMAL_64, decimal64Val 0, decimalVal 0)(children: StringGroupColEqualStringScalar(col 23:string, val TAKE BACK RETURN)(children: CastStringGroupToString(col 13:varchar(20)) -> 23:string) -> 35:boolean) -> 36:decimal(10,2)/DECIMAL_64, IfExprTimestampColumnColumn(col 37:boolean, col 38:timestampcol 39:timestamp)(children: LongColGreaterLongScalar(col 1:int, val 30) -> 37:boolean, CastDateToTimestamp(col 12:date) -> 38:timestamp, CastDateToTimestamp(col 11:date) -> 39:timestamp) -> 40:timestamp, IfExprColumnNull(col 37:boolean, col 41:int, null)(children: LongColGreaterLongScalar(col 2:int, val 10000) -> 37:boolean, VectorUDFDateDiffColCol(col 12:date, col 11:date) -> 41:int) -> 42:int, IfExpr
 NullColumn(col 43:boolean, null, col 44)(children: LongColGreaterLongScalar(col 2:int, val 10000) -> 43:boolean, VectorUDFDateDiffColCol(col 12:date, col 11:date) -> 44:int) -> 45:int, IfExprLongScalarLongScalar(col 47:boolean, val 14245, val 14609)(children: LongColGreaterLongScalar(col 46:int, val 100)(children: LongColModuloLongScalar(col 2:int, val 500) -> 46:int) -> 47:boolean) -> 46:date
                     Statistics: Num rows: 101 Data size: 57327 Basic stats: COMPLETE Column stats: NONE
                     File Output Operator
                       compressed: false
@@ -552,7 +552,7 @@ STAGE PLANS:
                     includeColumns: [1, 2, 4, 5, 6, 7, 8, 10, 11, 12, 13, 14]
                     dataColumns: l_orderkey:int, l_partkey:int, l_suppkey:int, l_linenumber:int, l_quantity:int, l_extendedprice:double, l_discount:double, l_tax:decimal(10,2)/DECIMAL_64, l_returnflag:char(1), l_linestatus:char(1), l_shipdate:date, l_commitdate:date, l_receiptdate:date, l_shipinstruct:varchar(20), l_shipmode:char(10), l_comment:string
                     partitionColumnCount: 0
-                    scratchColumnTypeNames: [bigint, bigint, bigint, bigint, string, string, string, string, string, bigint, double, double, double, decimal(10,2), decimal(10,2), decimal(12,2), decimal(12,2), decimal(10,2), decimal(10,2), timestamp, timestamp, timestamp, bigint, bigint, bigint, bigint, bigint, bigint, bigint, decimal(10,2), decimal(10,2)]
+                    scratchColumnTypeNames: [bigint, bigint, bigint, bigint, string, string, string, string, string, bigint, double, double, double, decimal(10,2), decimal(10,2), decimal(12,2), decimal(12,2), decimal(10,2)/DECIMAL_64, bigint, decimal(10,2)/DECIMAL_64, bigint, timestamp, timestamp, timestamp, bigint, bigint, bigint, bigint, bigint, bigint, bigint, decimal(10,2), decimal(10,2)]
 
   Stage: Stage-0
     Fetch Operator
@@ -856,8 +856,8 @@ STAGE PLANS:
                     Select Vectorization:
                         className: VectorSelectOperator
                         native: true
-                        projectedOutputColumnNums: [4, 27, 38, 48, 52, 54, 60, 63, 65, 67, 68, 69, 70, 73, 76, 79, 80]
-                        selectExpressions: IfExprColumnCondExpr(col 17:boolean, col 18:stringcol 26:string)(children: LongColEqualLongScalar(col 4:int, val 1) -> 17:boolean, ConstantVectorExpression(val Single) -> 18:string, IfExprColumnCondExpr(col 19:boolean, col 20:stringcol 25:string)(children: LongColEqualLongScalar(col 4:int, val 2) -> 19:boolean, ConstantVectorExpression(val Two) -> 20:string, IfExprColumnCondExpr(col 21:boolean, col 22:stringcol 24:string)(children: LongColLessLongScalar(col 4:int, val 10) -> 21:boolean, ConstantVectorExpression(val Some) -> 22:string, IfExprStringScalarStringScalar(col 23:boolean, val Many, val Huge number)(children: LongColLessLongScalar(col 4:int, val 100) -> 23:boolean) -> 24:string) -> 25:string) -> 26:string) -> 27:string, IfExprColumnCondExpr(col 23:boolean, col 28:stringcol 37:string)(children: LongColEqualLongScalar(col 4:int, val 1) -> 23:boolean, ConstantVectorExpression(val Single) -> 28:string, IfExprColumnCondExpr(col 29:boolea
 n, col 30:stringcol 36:string)(children: LongColEqualLongScalar(col 4:int, val 2) -> 29:boolean, ConstantVectorExpression(val Two) -> 30:string, IfExprColumnCondExpr(col 31:boolean, col 32:stringcol 35:string)(children: LongColLessLongScalar(col 4:int, val 10) -> 31:boolean, ConstantVectorExpression(val Some) -> 32:string, IfExprColumnNull(col 33:boolean, col 34:string, null)(children: LongColLessLongScalar(col 4:int, val 100) -> 33:boolean, ConstantVectorExpression(val Many) -> 34:string) -> 35:string) -> 36:string) -> 37:string) -> 38:string, IfExprColumnCondExpr(col 39:boolean, col 40:stringcol 47:string)(children: LongColEqualLongScalar(col 4:int, val 1) -> 39:boolean, ConstantVectorExpression(val Single) -> 40:string, IfExprColumnCondExpr(col 41:boolean, col 42:stringcol 46:string)(children: LongColEqualLongScalar(col 4:int, val 2) -> 41:boolean, ConstantVectorExpression(val Two) -> 42:string, IfExprColumnCondExpr(col 43:boolean, col 44:stringcol 45:string)(children: LongColLes
 sLongScalar(col 4:int, val 10) -> 43:boolean, ConstantVectorExpression(val Some) -> 44:string, IfExprNullNull(null, null) -> 45:string) -> 46:string) -> 47:string) -> 48:string, IfExprCondExprCondExpr(col 49:boolean, col 50:datecol 51:date)(children: StringGroupColEqualCharScalar(col 14:char(10), val SHIP) -> 49:boolean, VectorUDFDateAddColScalar(col 10:date, val 10) -> 50:date, VectorUDFDateAddColScalar(col 10:date, val 5) -> 51:date) -> 52:date, IfExprDoubleColumnLongScalar(col 57:boolean, col 58:double, val 0)(children: StringGroupColEqualCharScalar(col 8:char(1), val N) -> 57:boolean, DoubleColMultiplyDoubleColumn(col 5:double, col 54:double)(children: DoubleScalarSubtractDoubleColumn(val 1.0, col 6:double) -> 54:double) -> 58:double) -> 54:double, IfExprCondExprColumn(col 57:boolean, col 59:double, col 58:double)(children: StringGroupColEqualCharScalar(col 8:char(1), val N) -> 57:boolean, DoubleColMultiplyDoubleColumn(col 5:double, col 58:double)(children: DoubleScalarSubtractD
 oubleColumn(val 1.0, col 6:double) -> 58:double) -> 59:double, ConstantVectorExpression(val 0.0) -> 58:double) -> 60:double, IfExprNullColumn(col 62:boolean, null, col 82)(children: StringGroupColEqualStringScalar(col 61:string, val DELIVER IN PERSON)(children: CastStringGroupToString(col 13:varchar(20)) -> 61:string) -> 62:boolean, ConvertDecimal64ToDecimal(col 7:decimal(10,2)/DECIMAL_64) -> 82:decimal(10,2)) -> 63:decimal(10,2), IfExprColumnNull(col 64:boolean, col 83:decimal(10,2), null)(children: StringGroupColEqualStringScalar(col 61:string, val TAKE BACK RETURN)(children: CastStringGroupToString(col 13:varchar(20)) -> 61:string) -> 64:boolean, ConvertDecimal64ToDecimal(col 7:decimal(10,2)/DECIMAL_64) -> 83:decimal(10,2)) -> 65:decimal(10,2), VectorUDFAdaptor(if((CAST( l_shipinstruct AS STRING) = 'DELIVER IN PERSON'), 0, l_tax))(children: StringGroupColEqualStringScalar(col 61:string, val DELIVER IN PERSON)(children: CastStringGroupToString(col 13:varchar(20)) -> 61:string) -> 
 66:boolean) -> 67:decimal(12,2), VectorUDFAdaptor(if((CAST( l_shipinstruct AS STRING) = 'TAKE BACK RETURN'), l_tax, 0))(children: StringGroupColEqualStringScalar(col 61:string, val TAKE BACK RETURN)(children: CastStringGroupToString(col 13:varchar(20)) -> 61:string) -> 66:boolean) -> 68:decimal(12,2), VectorUDFAdaptor(if((CAST( l_shipinstruct AS STRING) = 'DELIVER IN PERSON'), 0, l_tax))(children: StringGroupColEqualStringScalar(col 61:string, val DELIVER IN PERSON)(children: CastStringGroupToString(col 13:varchar(20)) -> 61:string) -> 66:boolean) -> 69:decimal(10,2), VectorUDFAdaptor(if((CAST( l_shipinstruct AS STRING) = 'TAKE BACK RETURN'), l_tax, 0))(children: StringGroupColEqualStringScalar(col 61:string, val TAKE BACK RETURN)(children: CastStringGroupToString(col 13:varchar(20)) -> 61:string) -> 66:boolean) -> 70:decimal(10,2), IfExprCondExprCondExpr(col 66:boolean, col 71:timestampcol 72:timestamp)(children: LongColGreaterLongScalar(col 1:int, val 30) -> 66:boolean, CastDateTo
 Timestamp(col 12:date) -> 71:timestamp, CastDateToTimestamp(col 11:date) -> 72:timestamp) -> 73:timestamp, IfExprCondExprNull(col 74:boolean, col 75:int, null)(children: LongColGreaterLongScalar(col 2:int, val 10000) -> 74:boolean, VectorUDFDateDiffColCol(col 12:date, col 11:date) -> 75:int) -> 76:int, IfExprNullCondExpr(col 77:boolean, null, col 78:int)(children: LongColGreaterLongScalar(col 2:int, val 10000) -> 77:boolean, VectorUDFDateDiffColCol(col 12:date, col 11:date) -> 78:int) -> 79:int, IfExprLongScalarLongScalar(col 81:boolean, val 14245, val 14609)(children: LongColGreaterLongScalar(col 80:int, val 100)(children: LongColModuloLongScalar(col 2:int, val 500) -> 80:int) -> 81:boolean) -> 80:date
+                        projectedOutputColumnNums: [4, 27, 38, 48, 52, 54, 60, 63, 65, 67, 68, 69, 71, 75, 78, 81, 82]
+                        selectExpressions: IfExprColumnCondExpr(col 17:boolean, col 18:stringcol 26:string)(children: LongColEqualLongScalar(col 4:int, val 1) -> 17:boolean, ConstantVectorExpression(val Single) -> 18:string, IfExprColumnCondExpr(col 19:boolean, col 20:stringcol 25:string)(children: LongColEqualLongScalar(col 4:int, val 2) -> 19:boolean, ConstantVectorExpression(val Two) -> 20:string, IfExprColumnCondExpr(col 21:boolean, col 22:stringcol 24:string)(children: LongColLessLongScalar(col 4:int, val 10) -> 21:boolean, ConstantVectorExpression(val Some) -> 22:string, IfExprStringScalarStringScalar(col 23:boolean, val Many, val Huge number)(children: LongColLessLongScalar(col 4:int, val 100) -> 23:boolean) -> 24:string) -> 25:string) -> 26:string) -> 27:string, IfExprColumnCondExpr(col 23:boolean, col 28:stringcol 37:string)(children: LongColEqualLongScalar(col 4:int, val 1) -> 23:boolean, ConstantVectorExpression(val Single) -> 28:string, IfExprColumnCondExpr(col 29:boolea
 n, col 30:stringcol 36:string)(children: LongColEqualLongScalar(col 4:int, val 2) -> 29:boolean, ConstantVectorExpression(val Two) -> 30:string, IfExprColumnCondExpr(col 31:boolean, col 32:stringcol 35:string)(children: LongColLessLongScalar(col 4:int, val 10) -> 31:boolean, ConstantVectorExpression(val Some) -> 32:string, IfExprColumnNull(col 33:boolean, col 34:string, null)(children: LongColLessLongScalar(col 4:int, val 100) -> 33:boolean, ConstantVectorExpression(val Many) -> 34:string) -> 35:string) -> 36:string) -> 37:string) -> 38:string, IfExprColumnCondExpr(col 39:boolean, col 40:stringcol 47:string)(children: LongColEqualLongScalar(col 4:int, val 1) -> 39:boolean, ConstantVectorExpression(val Single) -> 40:string, IfExprColumnCondExpr(col 41:boolean, col 42:stringcol 46:string)(children: LongColEqualLongScalar(col 4:int, val 2) -> 41:boolean, ConstantVectorExpression(val Two) -> 42:string, IfExprColumnCondExpr(col 43:boolean, col 44:stringcol 45:string)(children: LongColLes
 sLongScalar(col 4:int, val 10) -> 43:boolean, ConstantVectorExpression(val Some) -> 44:string, IfExprNullNull(null, null) -> 45:string) -> 46:string) -> 47:string) -> 48:string, IfExprCondExprCondExpr(col 49:boolean, col 50:datecol 51:date)(children: StringGroupColEqualCharScalar(col 14:char(10), val SHIP) -> 49:boolean, VectorUDFDateAddColScalar(col 10:date, val 10) -> 50:date, VectorUDFDateAddColScalar(col 10:date, val 5) -> 51:date) -> 52:date, IfExprDoubleColumnLongScalar(col 57:boolean, col 58:double, val 0)(children: StringGroupColEqualCharScalar(col 8:char(1), val N) -> 57:boolean, DoubleColMultiplyDoubleColumn(col 5:double, col 54:double)(children: DoubleScalarSubtractDoubleColumn(val 1.0, col 6:double) -> 54:double) -> 58:double) -> 54:double, IfExprCondExprColumn(col 57:boolean, col 59:double, col 58:double)(children: StringGroupColEqualCharScalar(col 8:char(1), val N) -> 57:boolean, DoubleColMultiplyDoubleColumn(col 5:double, col 58:double)(children: DoubleScalarSubtractD
 oubleColumn(val 1.0, col 6:double) -> 58:double) -> 59:double, ConstantVectorExpression(val 0.0) -> 58:double) -> 60:double, IfExprNullColumn(col 62:boolean, null, col 84)(children: StringGroupColEqualStringScalar(col 61:string, val DELIVER IN PERSON)(children: CastStringGroupToString(col 13:varchar(20)) -> 61:string) -> 62:boolean, ConvertDecimal64ToDecimal(col 7:decimal(10,2)/DECIMAL_64) -> 84:decimal(10,2)) -> 63:decimal(10,2), IfExprColumnNull(col 64:boolean, col 85:decimal(10,2), null)(children: StringGroupColEqualStringScalar(col 61:string, val TAKE BACK RETURN)(children: CastStringGroupToString(col 13:varchar(20)) -> 61:string) -> 64:boolean, ConvertDecimal64ToDecimal(col 7:decimal(10,2)/DECIMAL_64) -> 85:decimal(10,2)) -> 65:decimal(10,2), VectorUDFAdaptor(if((CAST( l_shipinstruct AS STRING) = 'DELIVER IN PERSON'), 0, l_tax))(children: StringGroupColEqualStringScalar(col 61:string, val DELIVER IN PERSON)(children: CastStringGroupToString(col 13:varchar(20)) -> 61:string) -> 
 66:boolean) -> 67:decimal(12,2), VectorUDFAdaptor(if((CAST( l_shipinstruct AS STRING) = 'TAKE BACK RETURN'), l_tax, 0))(children: StringGroupColEqualStringScalar(col 61:string, val TAKE BACK RETURN)(children: CastStringGroupToString(col 13:varchar(20)) -> 61:string) -> 66:boolean) -> 68:decimal(12,2), IfExprDecimal64ScalarDecimal64Column(col 66:boolean, decimal64Val 0, decimalVal 0, col 7:decimal(1,0)/DECIMAL_64)(children: StringGroupColEqualStringScalar(col 61:string, val DELIVER IN PERSON)(children: CastStringGroupToString(col 13:varchar(20)) -> 61:string) -> 66:boolean) -> 69:decimal(10,2)/DECIMAL_64, IfExprDecimal64ColumnDecimal64Scalar(col 70:boolean, col 7:decimal(10,2)/DECIMAL_64, decimal64Val 0, decimalVal 0)(children: StringGroupColEqualStringScalar(col 61:string, val TAKE BACK RETURN)(children: CastStringGroupToString(col 13:varchar(20)) -> 61:string) -> 70:boolean) -> 71:decimal(10,2)/DECIMAL_64, IfExprCondExprCondExpr(col 72:boolean, col 73:timestampcol 74:timestamp)(chi
 ldren: LongColGreaterLongScalar(col 1:int, val 30) -> 72:boolean, CastDateToTimestamp(col 12:date) -> 73:timestamp, CastDateToTimestamp(col 11:date) -> 74:timestamp) -> 75:timestamp, IfExprCondExprNull(col 76:boolean, col 77:int, null)(children: LongColGreaterLongScalar(col 2:int, val 10000) -> 76:boolean, VectorUDFDateDiffColCol(col 12:date, col 11:date) -> 77:int) -> 78:int, IfExprNullCondExpr(col 79:boolean, null, col 80:int)(children: LongColGreaterLongScalar(col 2:int, val 10000) -> 79:boolean, VectorUDFDateDiffColCol(col 12:date, col 11:date) -> 80:int) -> 81:int, IfExprLongScalarLongScalar(col 83:boolean, val 14245, val 14609)(children: LongColGreaterLongScalar(col 82:int, val 100)(children: LongColModuloLongScalar(col 2:int, val 500) -> 82:int) -> 83:boolean) -> 82:date
                     Statistics: Num rows: 101 Data size: 57327 Basic stats: COMPLETE Column stats: NONE
                     File Output Operator
                       compressed: false
@@ -885,7 +885,7 @@ STAGE PLANS:
                     includeColumns: [1, 2, 4, 5, 6, 7, 8, 10, 11, 12, 13, 14]
                     dataColumns: l_orderkey:int, l_partkey:int, l_suppkey:int, l_linenumber:int, l_quantity:int, l_extendedprice:double, l_discount:double, l_tax:decimal(10,2)/DECIMAL_64, l_returnflag:char(1), l_linestatus:char(1), l_shipdate:date, l_commitdate:date, l_receiptdate:date, l_shipinstruct:varchar(20), l_shipmode:char(10), l_comment:string
                     partitionColumnCount: 0
-                    scratchColumnTypeNames: [bigint, string, bigint, string, bigint, string, bigint, string, string, string, string, string, bigint, string, bigint, string, bigint, string, string, string, string, string, bigint, string, bigint, string, bigint, string, string, string, string, string, bigint, bigint, bigint, bigint, bigint, double, double, bigint, bigint, double, double, double, string, bigint, decimal(10,2), bigint, decimal(10,2), bigint, decimal(12,2), decimal(12,2), decimal(10,2), decimal(10,2), timestamp, timestamp, timestamp, bigint, bigint, bigint, bigint, bigint, bigint, bigint, bigint, decimal(10,2), decimal(10,2)]
+                    scratchColumnTypeNames: [bigint, string, bigint, string, bigint, string, bigint, string, string, string, string, string, bigint, string, bigint, string, bigint, string, string, string, string, string, bigint, string, bigint, string, bigint, string, string, string, string, string, bigint, bigint, bigint, bigint, bigint, double, double, bigint, bigint, double, double, double, string, bigint, decimal(10,2), bigint, decimal(10,2), bigint, decimal(12,2), decimal(12,2), decimal(10,2)/DECIMAL_64, bigint, decimal(10,2)/DECIMAL_64, bigint, timestamp, timestamp, timestamp, bigint, bigint, bigint, bigint, bigint, bigint, bigint, bigint, decimal(10,2), decimal(10,2)]
 
   Stage: Stage-0
     Fetch Operator


[8/8] hive git commit: HIVE-20207: Vectorization: Fix NULL / Wrong Results issues in Filter / Compare (Matt McCline, reviewed by Teddy Choi)

Posted by mm...@apache.org.
HIVE-20207: Vectorization: Fix NULL / Wrong Results issues in Filter / Compare (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/71c49878
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/71c49878
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/71c49878

Branch: refs/heads/master
Commit: 71c49878c3669f19f900ec1ffe7652c91ce15d38
Parents: ca0d706
Author: Matt McCline <mm...@hortonworks.com>
Authored: Wed Jul 25 09:58:51 2018 -0500
Committer: Matt McCline <mm...@hortonworks.com>
Committed: Wed Jul 25 09:59:35 2018 -0500

----------------------------------------------------------------------
 .../VectorizedComparisonBench.java              |   19 +-
 .../ExpressionTemplates/ColumnCompareColumn.txt |    4 +-
 .../ExpressionTemplates/ColumnCompareScalar.txt |    4 +-
 .../Decimal64ColumnCompareDecimal64Column.txt   |   54 +
 .../Decimal64ColumnCompareDecimal64Scalar.txt   |   66 +
 .../Decimal64ScalarCompareDecimal64Column.txt   |   66 +
 .../DecimalColumnCompareDecimalColumn.txt       |  153 +
 .../DecimalColumnCompareDecimalScalar.txt       |  177 +
 .../DecimalScalarCompareDecimalColumn.txt       |  180 +
 .../ExpressionTemplates/IfExprColumnScalar.txt  |    6 +-
 .../IfExprObjectColumnColumn.txt                |   41 +
 .../IfExprObjectColumnScalar.txt                |   22 +
 .../IfExprObjectScalarColumn.txt                |   22 +
 .../ExpressionTemplates/IfExprScalarColumn.txt  |    6 +-
 .../ExpressionTemplates/IfExprScalarScalar.txt  |    6 +-
 .../ExpressionTemplates/ScalarCompareColumn.txt |    4 +-
 .../exec/vector/VectorExpressionDescriptor.java |    2 +
 .../ql/exec/vector/VectorizationContext.java    |  222 +-
 .../exec/vector/VectorizationContext.java.orig  | 3771 ++++++++++++++++++
 .../vector/expressions/CastCharToBinary.java    |   55 +
 .../expressions/CastStringToTimestamp.java      |  177 +
 .../IfExprCharScalarStringScalar.java           |    2 +-
 .../IfExprDecimal64ColumnDecimal64Column.java   |   55 +
 .../IfExprDecimal64ColumnDecimal64Scalar.java   |   70 +
 .../IfExprDecimal64ScalarDecimal64Column.java   |   71 +
 .../IfExprDecimal64ScalarDecimal64Scalar.java   |   75 +
 .../expressions/IfExprLongColumnLongColumn.java |    6 +-
 ...fExprStringGroupColumnStringGroupColumn.java |    4 +-
 .../IfExprStringGroupColumnStringScalar.java    |    4 +-
 .../IfExprStringScalarStringGroupColumn.java    |    4 +-
 .../IfExprStringScalarStringScalar.java         |    4 +-
 .../IfExprVarCharScalarStringScalar.java        |    2 +-
 .../expressions/LongColEqualLongColumn.java     |  146 -
 .../expressions/LongColEqualLongScalar.java     |  157 -
 .../LongColGreaterEqualLongColumn.java          |  146 -
 .../LongColGreaterEqualLongScalar.java          |  158 -
 .../expressions/LongColGreaterLongColumn.java   |  146 -
 .../expressions/LongColGreaterLongScalar.java   |  157 -
 .../expressions/LongColLessEqualLongColumn.java |  146 -
 .../expressions/LongColLessEqualLongScalar.java |  158 -
 .../expressions/LongColLessLongColumn.java      |  146 -
 .../expressions/LongColLessLongScalar.java      |  158 -
 .../expressions/LongColNotEqualLongColumn.java  |  146 -
 .../expressions/LongColNotEqualLongScalar.java  |  158 -
 .../expressions/LongScalarEqualLongColumn.java  |  157 -
 .../LongScalarGreaterEqualLongColumn.java       |  160 -
 .../LongScalarGreaterLongColumn.java            |  161 -
 .../LongScalarLessEqualLongColumn.java          |  160 -
 .../expressions/LongScalarLessLongColumn.java   |  161 -
 .../LongScalarNotEqualLongColumn.java           |  161 -
 .../hive/ql/udf/generic/GenericUDFIf.java       |   16 +
 .../hive/ql/udf/generic/GenericUDFOPEqual.java  |    8 +-
 .../generic/GenericUDFOPEqualOrGreaterThan.java |    8 +-
 .../generic/GenericUDFOPEqualOrLessThan.java    |    8 +-
 .../ql/udf/generic/GenericUDFOPGreaterThan.java |    8 +-
 .../ql/udf/generic/GenericUDFOPLessThan.java    |    8 +-
 .../ql/udf/generic/GenericUDFOPNotEqual.java    |    8 +-
 .../ql/udf/generic/GenericUDFTimestamp.java     |    3 +-
 .../exec/vector/TestVectorizationContext.java   |    4 +-
 .../expressions/TestVectorArithmetic.java       |   11 +
 .../expressions/TestVectorCastStatement.java    |   42 +-
 .../expressions/TestVectorDateAddSub.java       |   10 +
 .../vector/expressions/TestVectorDateDiff.java  |   11 +
 .../expressions/TestVectorFilterCompare.java    |  795 ++++
 .../expressions/TestVectorIfStatement.java      |   74 +-
 .../vector/expressions/TestVectorNegative.java  |    9 +
 .../expressions/TestVectorStringConcat.java     |   11 +
 .../expressions/TestVectorStringUnary.java      |   12 +-
 .../vector/expressions/TestVectorSubStr.java    |    9 +
 .../expressions/TestVectorTimestampExtract.java |    9 +
 .../convert_decimal64_to_decimal.q.out          |    6 +-
 .../llap/convert_decimal64_to_decimal.q.out     |   10 +-
 .../llap/vector_case_when_1.q.out               |   12 +-
 .../llap/vector_decimal_mapjoin.q.out           |   24 +-
 .../llap/vector_outer_reference_windowed.q.out  |   48 +-
 .../llap/vector_udf_adaptor_1.q.out             |   32 +-
 .../clientpositive/llap/vectorized_case.q.out   |   16 +-
 .../clientpositive/llap/vectorized_casts.q.out  |    4 +-
 ...vectorized_dynamic_semijoin_reduction2.q.out |    4 +-
 .../llap/vectorized_mapjoin3.q.out              |    6 +-
 .../llap/vectorized_timestamp_funcs.q.out       |   12 +-
 .../llap/vectorized_timestamp_ints_casts.q.out  |    8 +-
 .../spark/vector_decimal_mapjoin.q.out          |   24 +-
 .../clientpositive/spark/vectorized_case.q.out  |   16 +-
 .../spark/vectorized_timestamp_funcs.q.out      |   12 +-
 .../clientpositive/vector_case_when_1.q.out     |   12 +-
 .../clientpositive/vector_decimal_mapjoin.q.out |   16 +-
 .../clientpositive/vectorized_case.q.out        |   16 +-
 .../clientpositive/vectorized_casts.q.out       |    4 +-
 .../vectorized_timestamp_funcs.q.out            |   12 +-
 .../vectorized_timestamp_ints_casts.q.out       |    8 +-
 .../ql/exec/vector/DecimalColumnVector.java     |   21 +-
 .../ql/exec/vector/expressions/StringExpr.java  |   17 +
 .../apache/hadoop/hive/tools/GenVectorCode.java |  126 +-
 94 files changed, 6520 insertions(+), 3146 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/vectorization/VectorizedComparisonBench.java
----------------------------------------------------------------------
diff --git a/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/vectorization/VectorizedComparisonBench.java b/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/vectorization/VectorizedComparisonBench.java
index 4c616f6..d54d1fa 100644
--- a/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/vectorization/VectorizedComparisonBench.java
+++ b/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/vectorization/VectorizedComparisonBench.java
@@ -14,24 +14,7 @@
 package org.apache.hive.benchmark.vectorization;
 
 import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.LongColEqualLongColumn;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.LongColEqualLongScalar;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.LongColGreaterEqualLongColumn;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.LongColGreaterEqualLongScalar;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.LongColGreaterLongColumn;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.LongColGreaterLongScalar;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.LongColLessEqualLongColumn;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.LongColLessEqualLongScalar;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.LongColLessLongColumn;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.LongColLessLongScalar;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.LongColNotEqualLongColumn;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.LongColNotEqualLongScalar;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.LongScalarEqualLongColumn;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.LongScalarGreaterEqualLongColumn;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.LongScalarGreaterLongColumn;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.LongScalarLessEqualLongColumn;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.LongScalarLessLongColumn;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.LongScalarNotEqualLongColumn;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.*;
 import org.openjdk.jmh.annotations.Scope;
 import org.openjdk.jmh.annotations.State;
 import org.openjdk.jmh.runner.Runner;

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/gen/vectorization/ExpressionTemplates/ColumnCompareColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/ColumnCompareColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/ColumnCompareColumn.txt
index 56d96b8..84b2869 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/ColumnCompareColumn.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/ColumnCompareColumn.txt
@@ -33,8 +33,8 @@ public class <ClassName> extends VectorExpression {
 
   private static final long serialVersionUID = 1L;
 
-  private final int colNum1;
-  private final int colNum2;
+  protected final int colNum1;
+  protected final int colNum2;
 
   public <ClassName>(int colNum1, int colNum2, int outputColumnNum) {
     super(outputColumnNum);

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/gen/vectorization/ExpressionTemplates/ColumnCompareScalar.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/ColumnCompareScalar.txt b/ql/src/gen/vectorization/ExpressionTemplates/ColumnCompareScalar.txt
index 9556066..640feb3 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/ColumnCompareScalar.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/ColumnCompareScalar.txt
@@ -36,8 +36,8 @@ public class <ClassName> extends VectorExpression {
 
   private static final long serialVersionUID = 1L;
 
-  private final int colNum;
-  private final <OperandType2> value;
+  protected final int colNum;
+  protected final <OperandType2> value;
 
   public <ClassName>(int colNum, <OperandType2> value, int outputColumnNum) {
     super(outputColumnNum);

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/gen/vectorization/ExpressionTemplates/Decimal64ColumnCompareDecimal64Column.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/Decimal64ColumnCompareDecimal64Column.txt b/ql/src/gen/vectorization/ExpressionTemplates/Decimal64ColumnCompareDecimal64Column.txt
new file mode 100644
index 0000000..d260716
--- /dev/null
+++ b/ql/src/gen/vectorization/ExpressionTemplates/Decimal64ColumnCompareDecimal64Column.txt
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.exec.vector.expressions.gen;
+
+import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+
+/**
+ * Generated from template Decimal64ColumnCompareDecimal64Column.txt, which covers
+ * decimal64 comparison expressions between two columns, however output is not produced in
+ * a separate column. The selected vector of the input {@link VectorizedRowBatch} is updated
+ * for in-place filtering.
+ */
+public class <ClassName> extends <BaseClassName> {
+
+  private static final long serialVersionUID = 1L;
+
+  public <ClassName>(int colNum1, int colNum2, int outputColumnNum) {
+    super(colNum1, colNum2, outputColumnNum);
+  }
+
+  public <ClassName>() {
+    super();
+  }
+
+  @Override
+  public VectorExpressionDescriptor.Descriptor getDescriptor() {
+    return (new VectorExpressionDescriptor.Builder())
+        .setMode(
+            VectorExpressionDescriptor.Mode.PROJECTION)
+        .setNumArguments(2)
+        .setArgumentTypes(
+            VectorExpressionDescriptor.ArgumentType.DECIMAL_64,
+            VectorExpressionDescriptor.ArgumentType.DECIMAL_64)
+        .setInputExpressionTypes(
+            VectorExpressionDescriptor.InputExpressionType.COLUMN,
+            VectorExpressionDescriptor.InputExpressionType.COLUMN).build();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/gen/vectorization/ExpressionTemplates/Decimal64ColumnCompareDecimal64Scalar.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/Decimal64ColumnCompareDecimal64Scalar.txt b/ql/src/gen/vectorization/ExpressionTemplates/Decimal64ColumnCompareDecimal64Scalar.txt
new file mode 100644
index 0000000..802b9a6
--- /dev/null
+++ b/ql/src/gen/vectorization/ExpressionTemplates/Decimal64ColumnCompareDecimal64Scalar.txt
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.exec.vector.expressions.gen;
+
+import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo;
+
+/**
+ * Generated from template Decimal64ColumnCompareDecimal64Scalar.txt, which covers decimal64
+ * comparison  expressions between a column and a scalar, however output is not produced in a
+ * separate column. The selected vector of the input {@link VectorizedRowBatch} is updated for
+ * in-place filtering.
+ */
+public class <ClassName> extends <BaseClassName> {
+
+  private static final long serialVersionUID = 1L;
+
+  public <ClassName>(int colNum, long value, int outputColumnNum) {
+    super(colNum, value, outputColumnNum);
+  }
+
+  public <ClassName>() {
+    super();
+  }
+
+  @Override
+  public String vectorExpressionParameters() {
+    DecimalTypeInfo decimalTypeInfo = (DecimalTypeInfo) inputTypeInfos[0];
+    HiveDecimalWritable writable = new HiveDecimalWritable();
+    writable.deserialize64(value, decimalTypeInfo.scale());
+    return getColumnParamString(0, colNum) + ", decimal64Val " + value +
+        ", decimalVal " + writable.toString();
+  }
+
+  @Override
+  public VectorExpressionDescriptor.Descriptor getDescriptor() {
+    return (new VectorExpressionDescriptor.Builder())
+        .setMode(
+            VectorExpressionDescriptor.Mode.PROJECTION)
+        .setNumArguments(2)
+        .setArgumentTypes(
+            VectorExpressionDescriptor.ArgumentType.DECIMAL_64,
+            VectorExpressionDescriptor.ArgumentType.DECIMAL_64)
+        .setInputExpressionTypes(
+            VectorExpressionDescriptor.InputExpressionType.COLUMN,
+            VectorExpressionDescriptor.InputExpressionType.SCALAR).build();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/gen/vectorization/ExpressionTemplates/Decimal64ScalarCompareDecimal64Column.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/Decimal64ScalarCompareDecimal64Column.txt b/ql/src/gen/vectorization/ExpressionTemplates/Decimal64ScalarCompareDecimal64Column.txt
new file mode 100644
index 0000000..c8b10b6
--- /dev/null
+++ b/ql/src/gen/vectorization/ExpressionTemplates/Decimal64ScalarCompareDecimal64Column.txt
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.exec.vector.expressions.gen;
+
+import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo;
+
+/**
+ * Generated from template Decimal64ScalarCompareDecimal64Column.txt, which covers decimal64
+ * comparison expressions between a scalar and a column, however output is not produced in a
+ * separate column. The selected vector of the input {@link VectorizedRowBatch} is updated for
+ * in-place filtering.
+ */
+public class <ClassName> extends <BaseClassName> {
+
+  private static final long serialVersionUID = 1L;
+
+  public <ClassName>(long value, int colNum, int outputColumnNum) {
+    super(value, colNum, outputColumnNum);
+  }
+
+  public <ClassName>() {
+    super();
+  }
+
+  @Override
+  public String vectorExpressionParameters() {
+    DecimalTypeInfo decimalTypeInfo = (DecimalTypeInfo) inputTypeInfos[1];
+    HiveDecimalWritable writable = new HiveDecimalWritable();
+    writable.deserialize64(value, decimalTypeInfo.scale());
+    return "decimal64Val " + value + ", decimalVal " + writable.toString() +
+        ", " + getColumnParamString(1, colNum);
+  }
+
+  @Override
+  public VectorExpressionDescriptor.Descriptor getDescriptor() {
+    return (new VectorExpressionDescriptor.Builder())
+        .setMode(
+            VectorExpressionDescriptor.Mode.PROJECTION)
+        .setNumArguments(2)
+        .setArgumentTypes(
+            VectorExpressionDescriptor.ArgumentType.DECIMAL_64,
+            VectorExpressionDescriptor.ArgumentType.DECIMAL_64)
+        .setInputExpressionTypes(
+            VectorExpressionDescriptor.InputExpressionType.SCALAR,
+            VectorExpressionDescriptor.InputExpressionType.COLUMN).build();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/gen/vectorization/ExpressionTemplates/DecimalColumnCompareDecimalColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/DecimalColumnCompareDecimalColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/DecimalColumnCompareDecimalColumn.txt
new file mode 100644
index 0000000..1a68de9
--- /dev/null
+++ b/ql/src/gen/vectorization/ExpressionTemplates/DecimalColumnCompareDecimalColumn.txt
@@ -0,0 +1,153 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.exec.vector.expressions.gen;
+
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.NullUtil;
+import org.apache.hadoop.hive.ql.exec.vector.*;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+
+/**
+ * Generated from template DecimalColumnCompareDecimalColumn.txt, which covers comparision
+ * expressions between Decimal columns.
+ */
+public class <ClassName> extends VectorExpression {
+
+  private static final long serialVersionUID = 1L;
+
+  private final int colNum1;
+  private final int colNum2;
+
+  public <ClassName>(int colNum1, int colNum2, int outputColumnNum) {
+    super(outputColumnNum);
+    this.colNum1 = colNum1;
+    this.colNum2 = colNum2;
+  }
+
+  public <ClassName>() {
+     super();
+
+    // Dummy final assignments.
+    colNum1 = -1;
+    colNum2 = -1;
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
+
+    // return immediately if batch is empty
+    final int n = batch.size;
+    if (n == 0) {
+      return;
+    }
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    DecimalColumnVector inputColVector1 = (DecimalColumnVector) batch.cols[colNum1];
+    HiveDecimalWritable[] vector1 = inputColVector1.vector;
+
+    DecimalColumnVector inputColVector2 = (DecimalColumnVector) batch.cols[colNum2];
+    HiveDecimalWritable[] vector2 = inputColVector2.vector;
+
+    LongColumnVector outputColVector = (LongColumnVector) batch.cols[outputColumnNum];
+    int[] sel = batch.selected;
+    long[] outputVector = outputColVector.vector;
+
+    /*
+     * Propagate null values for a two-input operator and set isRepeating and noNulls appropriately.
+     */
+    NullUtil.propagateNullsColCol(
+      inputColVector1, inputColVector2, outputColVector, sel, n, batch.selectedInUse);
+
+    /* Disregard nulls for processing. In other words,
+     * the arithmetic operation is performed even if one or
+     * more inputs are null. This is to improve speed by avoiding
+     * conditional checks in the inner loop.
+     */
+    if (inputColVector1.isRepeating && inputColVector2.isRepeating) {
+      outputVector[0] = vector1[0].compareTo(vector2[0]) <OperatorSymbol> 0 ? 1 : 0;
+    } else if (inputColVector1.isRepeating) {
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = vector1[0].compareTo(vector2[i]) <OperatorSymbol> 0 ? 1 : 0;
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = vector1[0].compareTo(vector2[i]) <OperatorSymbol> 0 ? 1 : 0;
+        }
+      }
+    } else if (inputColVector2.isRepeating) {
+      HiveDecimalWritable value2 = vector2[0];
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = vector1[i].compareTo(value2) <OperatorSymbol> 0 ? 1 : 0;
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = vector1[i].compareTo(value2) <OperatorSymbol> 0 ? 1 : 0;
+        }
+      }
+    } else {
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = vector1[i].compareTo(vector2[i]) <OperatorSymbol> 0 ? 1 : 0;
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = vector1[i].compareTo(vector2[i]) <OperatorSymbol> 0 ? 1 : 0;
+        }
+      }
+    }
+
+    /* For the case when the output can have null values, follow
+     * the convention that the data values must be 1 for long and
+     * NaN for double. This is to prevent possible later zero-divide errors
+     * in complex arithmetic expressions like col2 / (col1 - 1)
+     * in the case when some col1 entries are null.
+     */
+    NullUtil.setNullDataEntriesLong(outputColVector, batch.selectedInUse, sel, n);
+  }
+
+  @Override
+  public String vectorExpressionParameters() {
+    return getColumnParamString(0, colNum1) + ", " + getColumnParamString(1, colNum2);
+  }
+
+  @Override
+  public VectorExpressionDescriptor.Descriptor getDescriptor() {
+    return (new VectorExpressionDescriptor.Builder())
+        .setMode(
+            VectorExpressionDescriptor.Mode.PROJECTION)
+        .setNumArguments(2)
+        .setArgumentTypes(
+            VectorExpressionDescriptor.ArgumentType.DECIMAL,
+            VectorExpressionDescriptor.ArgumentType.DECIMAL)
+        .setInputExpressionTypes(
+            VectorExpressionDescriptor.InputExpressionType.COLUMN,
+            VectorExpressionDescriptor.InputExpressionType.COLUMN).build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/gen/vectorization/ExpressionTemplates/DecimalColumnCompareDecimalScalar.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/DecimalColumnCompareDecimalScalar.txt b/ql/src/gen/vectorization/ExpressionTemplates/DecimalColumnCompareDecimalScalar.txt
new file mode 100644
index 0000000..8d09137
--- /dev/null
+++ b/ql/src/gen/vectorization/ExpressionTemplates/DecimalColumnCompareDecimalScalar.txt
@@ -0,0 +1,177 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.exec.vector.expressions.gen;
+
+import java.util.Arrays;
+
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.hadoop.hive.common.type.HiveDecimal;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.NullUtil;
+import org.apache.hadoop.hive.ql.exec.vector.*;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+
+/**
+ * Generated from template DecimalColumnCompareDecimalScalar.txt, which covers binary comparison
+ * expressions between a column and a scalar. The boolean output is stored in a
+ * separate boolean column.
+ */
+public class <ClassName> extends VectorExpression {
+
+  private static final long serialVersionUID = 1L;
+
+  private final int colNum;
+  private final HiveDecimalWritable value;
+
+  public <ClassName>(int colNum, HiveDecimal value, int outputColumnNum) {
+    super(outputColumnNum);
+    this.colNum = colNum;
+    this.value = new HiveDecimalWritable(value);
+  }
+
+  public <ClassName>() {
+    super();
+
+    // Dummy final assignments.
+    colNum = -1;
+    value = null;
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
+
+    // return immediately if batch is empty
+    final int n = batch.size;
+    if (n == 0) {
+      return;
+    }
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    DecimalColumnVector inputColVector1 = (DecimalColumnVector) batch.cols[colNum];
+    HiveDecimalWritable[] vector1 = inputColVector1.vector;
+
+    LongColumnVector outputColVector = (LongColumnVector) batch.cols[outputColumnNum];
+
+    int[] sel = batch.selected;
+    boolean[] inputIsNull = inputColVector1.isNull;
+    boolean[] outputIsNull = outputColVector.isNull;
+    long[] outputVector = outputColVector.vector;
+
+    // We do not need to do a column reset since we are carefully changing the output.
+    outputColVector.isRepeating = false;
+
+    if (inputColVector1.isRepeating) {
+      if (inputColVector1.noNulls || !inputIsNull[0]) {
+        outputIsNull[0] = false;
+        outputVector[0] = vector1[0].compareTo(value) <OperatorSymbol> 0 ? 1 : 0;
+      } else {
+        outputIsNull[0] = true;
+        outputColVector.noNulls = false;
+      }
+      outputColVector.isRepeating = true;
+      NullUtil.setNullOutputEntriesColScalar(outputColVector, batch.selectedInUse, sel, n);
+      return;
+    }
+
+    if (inputColVector1.noNulls) {
+      if (batch.selectedInUse) {
+
+         // CONSIDER: For large n, fill n or all of isNull array and use the tighter ELSE loop.
+
+         if (!outputColVector.noNulls) {
+           for(int j = 0; j != n; j++) {
+             final int i = sel[j];
+             outputIsNull[i] = false;
+             outputVector[i] = vector1[i].compareTo(value) <OperatorSymbol> 0 ? 1 : 0;
+          }
+         } else {
+           for(int j = 0; j != n; j++) {
+             final int i = sel[j];
+             outputVector[i] = vector1[i].compareTo(value) <OperatorSymbol> 0 ? 1 : 0;
+           }
+         }
+      } else {
+        if (!outputColVector.noNulls) {
+
+          // Assume it is almost always a performance win to fill all of isNull so we can
+          // safely reset noNulls.
+          Arrays.fill(outputIsNull, false);
+          outputColVector.noNulls = true;
+        }
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = vector1[i].compareTo(value) <OperatorSymbol> 0 ? 1 : 0;
+        }
+      }
+    } else /* there are NULLs in the inputColVector */ {
+
+      /*
+       * Do careful maintenance of the outputColVector.noNulls flag.
+       */
+
+      if (batch.selectedInUse) {
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          if (!inputIsNull[i]) {
+            outputIsNull[i] = false;
+            outputVector[i] = vector1[i].compareTo(value) <OperatorSymbol> 0 ? 1 : 0;
+          } else {
+            // Comparison with NULL is NULL.
+            outputIsNull[i] = true;
+            outputColVector.noNulls = false;
+          }
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+         if (!inputIsNull[i]) {
+            outputIsNull[i] = false;
+            outputVector[i] = vector1[i].compareTo(value) <OperatorSymbol> 0 ? 1 : 0;
+          } else {
+            // Comparison with NULL is NULL.
+            outputIsNull[i] = true;
+            outputColVector.noNulls = false;
+          }
+        }
+      }
+    }
+  }
+
+  @Override
+  public String vectorExpressionParameters() {
+    return getColumnParamString(0, colNum) + ", val " + value.toString();
+  }
+
+  @Override
+  public VectorExpressionDescriptor.Descriptor getDescriptor() {
+    return (new VectorExpressionDescriptor.Builder())
+        .setMode(
+            VectorExpressionDescriptor.Mode.PROJECTION)
+        .setNumArguments(2)
+        .setArgumentTypes(
+            VectorExpressionDescriptor.ArgumentType.DECIMAL,
+            VectorExpressionDescriptor.ArgumentType.DECIMAL)
+        .setInputExpressionTypes(
+            VectorExpressionDescriptor.InputExpressionType.COLUMN,
+            VectorExpressionDescriptor.InputExpressionType.SCALAR).build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/gen/vectorization/ExpressionTemplates/DecimalScalarCompareDecimalColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/DecimalScalarCompareDecimalColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/DecimalScalarCompareDecimalColumn.txt
new file mode 100644
index 0000000..1275260
--- /dev/null
+++ b/ql/src/gen/vectorization/ExpressionTemplates/DecimalScalarCompareDecimalColumn.txt
@@ -0,0 +1,180 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.exec.vector.expressions.gen;
+
+import java.util.Arrays;
+
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.hadoop.hive.common.type.HiveDecimal;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.NullUtil;
+import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.*;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+
+/**
+ * Generated from template DecimalScalarCompareDecimalColumn.txt, which covers comparison
+ * expressions between a long/double scalar and a column. The boolean output is stored in a
+ * separate boolean column.
+ */
+public class <ClassName> extends VectorExpression {
+
+  private static final long serialVersionUID = 1L;
+
+  private final HiveDecimalWritable value;
+  private final int colNum;
+
+  public <ClassName>(HiveDecimal value, int colNum, int outputColumnNum) {
+    super(outputColumnNum);
+    this.value = new HiveDecimalWritable(value);
+    this.colNum = colNum;
+  }
+
+  public <ClassName>() {
+     super();
+
+    // Dummy final assignments.
+    value = null;
+    colNum = -1;
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
+
+    // return immediately if batch is empty
+    final int n = batch.size;
+    if (n == 0) {
+      return;
+    }
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    DecimalColumnVector inputColVector2 = (DecimalColumnVector) batch.cols[colNum];
+    HiveDecimalWritable[] vector2 = inputColVector2.vector;
+
+    LongColumnVector outputColVector = (LongColumnVector) batch.cols[outputColumnNum];
+
+    int[] sel = batch.selected;
+    boolean[] inputIsNull = inputColVector2.isNull;
+    boolean[] outputIsNull = outputColVector.isNull;
+
+    long[] outputVector = outputColVector.vector;
+
+    // We do not need to do a column reset since we are carefully changing the output.
+    outputColVector.isRepeating = false;
+
+    if (inputColVector2.isRepeating) {
+      if (inputColVector2.noNulls || !inputIsNull[0]) {
+        outputIsNull[0] = false;
+        outputVector[0] = value.compareTo(vector2[0]) <OperatorSymbol> 0 ? 1 : 0;
+      } else {
+        outputIsNull[0] = true;
+        outputColVector.noNulls = false;
+      }
+      outputColVector.isRepeating = true;
+      NullUtil.setNullOutputEntriesColScalar(outputColVector, batch.selectedInUse, sel, n);
+      return;
+    }
+
+    if (inputColVector2.noNulls) {
+      if (batch.selectedInUse) {
+
+         // CONSIDER: For large n, fill n or all of isNull array and use the tighter ELSE loop.
+
+         if (!outputColVector.noNulls) {
+           for(int j = 0; j != n; j++) {
+             final int i = sel[j];
+             outputIsNull[i] = false;
+             outputVector[i] = value.compareTo(vector2[i]) <OperatorSymbol> 0 ? 1 : 0;
+          }
+         } else {
+           for(int j = 0; j != n; j++) {
+             final int i = sel[j];
+             outputVector[i] = value.compareTo(vector2[i]) <OperatorSymbol> 0 ? 1 : 0;
+           }
+         }
+      } else {
+        if (!outputColVector.noNulls) {
+
+          // Assume it is almost always a performance win to fill all of isNull so we can
+          // safely reset noNulls.
+          Arrays.fill(outputIsNull, false);
+          outputColVector.noNulls = true;
+        }
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = value.compareTo(vector2[i]) <OperatorSymbol> 0 ? 1 : 0;
+        }
+      }
+    } else /* there are NULLs in the inputColVector */ {
+
+      /*
+       * Do careful maintenance of the outputColVector.noNulls flag.
+       */
+
+      if (batch.selectedInUse) {
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          if (!inputIsNull[i]) {
+            outputIsNull[i] = false;
+            outputVector[i] = value.compareTo(vector2[i]) <OperatorSymbol> 0 ? 1 : 0;
+          } else {
+            // Comparison with NULL is NULL.
+            outputIsNull[i] = true;
+            outputColVector.noNulls = false;
+          }
+        }
+      } else {
+        System.arraycopy(inputIsNull, 0, outputIsNull, 0, n);
+        for(int i = 0; i != n; i++) {
+          if (!inputIsNull[i]) {
+            outputIsNull[i] = false;
+            outputVector[i] = value.compareTo(vector2[i]) <OperatorSymbol> 0 ? 1 : 0;
+          } else {
+            // Comparison with NULL is NULL.
+            outputIsNull[i] = true;
+            outputColVector.noNulls = false;
+          }
+        }
+      }
+    }
+  }
+
+  @Override
+  public String vectorExpressionParameters() {
+    return "val " + value.toString() + ", " + getColumnParamString(1, colNum);
+  }
+
+  @Override
+  public VectorExpressionDescriptor.Descriptor getDescriptor() {
+    return (new VectorExpressionDescriptor.Builder())
+        .setMode(
+            VectorExpressionDescriptor.Mode.PROJECTION)
+        .setNumArguments(2)
+        .setArgumentTypes(
+            VectorExpressionDescriptor.ArgumentType.DECIMAL,
+            VectorExpressionDescriptor.ArgumentType.DECIMAL)
+        .setInputExpressionTypes(
+            VectorExpressionDescriptor.InputExpressionType.SCALAR,
+            VectorExpressionDescriptor.InputExpressionType.COLUMN).build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/gen/vectorization/ExpressionTemplates/IfExprColumnScalar.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/IfExprColumnScalar.txt b/ql/src/gen/vectorization/ExpressionTemplates/IfExprColumnScalar.txt
index d50a6f6..3eec95e 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/IfExprColumnScalar.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/IfExprColumnScalar.txt
@@ -35,9 +35,9 @@ public class <ClassName> extends VectorExpression {
 
   private static final long serialVersionUID = 1L;
 
-  private final int arg1Column;
-  private final int arg2Column;
-  private final <OperandType3> arg3Scalar;
+  protected final int arg1Column;
+  protected final int arg2Column;
+  protected final <OperandType3> arg3Scalar;
 
   public <ClassName>(int arg1Column, int arg2Column, <OperandType3> arg3Scalar,
       int outputColumnNum) {

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/gen/vectorization/ExpressionTemplates/IfExprObjectColumnColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/IfExprObjectColumnColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/IfExprObjectColumnColumn.txt
index e8ef279..a8ca31b 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/IfExprObjectColumnColumn.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/IfExprObjectColumnColumn.txt
@@ -23,6 +23,9 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
+#IF DECIMAL
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+#ENDIF DECIMAL
 
 /**
  * Compute IF(expr1, expr2, expr3) for 3 input column expressions.
@@ -64,8 +67,14 @@ public class <ClassName> extends VectorExpression {
     LongColumnVector arg1ColVector = (LongColumnVector) batch.cols[arg1Column];
     <ColumnVectorType> arg2ColVector = (<ColumnVectorType>) batch.cols[arg2Column];
     boolean[] arg2IsNull = arg2ColVector.isNull;
+#IF DECIMAL
+    HiveDecimalWritable[] vector2 = arg2ColVector.vector;
+#ENDIF DECIMAL
     <ColumnVectorType> arg3ColVector = (<ColumnVectorType>) batch.cols[arg3Column];
     boolean[] arg3IsNull = arg3ColVector.isNull;
+#IF DECIMAL
+    HiveDecimalWritable[] vector3 = arg3ColVector.vector;
+#ENDIF DECIMAL
     <ColumnVectorType> outputColVector = (<ColumnVectorType>) batch.cols[outputColumnNum];
     int[] sel = batch.selected;
     boolean[] outputIsNull = outputColVector.isNull;
@@ -107,7 +116,11 @@ public class <ClassName> extends VectorExpression {
           if (vector1[i] == 1) {
             if (!arg2IsNull[i]) {
               outputIsNull[i] = false;
+#IF DECIMAL
+              outputColVector.set(i, vector2[i]);
+#ELSE
               outputColVector.set(i, arg2ColVector.asScratch<ObjectName>(i));
+#ENDIF DECIMAL
             } else {
               outputIsNull[i] = true;
               outputColVector.noNulls = false;
@@ -115,7 +128,11 @@ public class <ClassName> extends VectorExpression {
           } else {
             if (!arg3IsNull[i]) {
               outputIsNull[i] = false;
+#IF DECIMAL
+              outputColVector.set(i, vector3[i]);
+#ELSE
               outputColVector.set(i, arg3ColVector.asScratch<ObjectName>(i));
+#ENDIF DECIMAL
             } else {
               outputIsNull[i] = true;
               outputColVector.noNulls = false;
@@ -127,7 +144,11 @@ public class <ClassName> extends VectorExpression {
           if (vector1[i] == 1) {
             if (!arg2IsNull[i]) {
               outputIsNull[i] = false;
+#IF DECIMAL
+              outputColVector.set(i, vector2[i]);
+#ELSE
               outputColVector.set(i, arg2ColVector.asScratch<ObjectName>(i));
+#ENDIF DECIMAL
             } else {
               outputIsNull[i] = true;
               outputColVector.noNulls = false;
@@ -135,7 +156,11 @@ public class <ClassName> extends VectorExpression {
           } else {
             if (!arg3IsNull[i]) {
               outputIsNull[i] = false;
+#IF DECIMAL
+              outputColVector.set(i, vector3[i]);
+#ELSE
               outputColVector.set(i, arg3ColVector.asScratch<ObjectName>(i));
+#ENDIF DECIMAL
             } else {
               outputIsNull[i] = true;
               outputColVector.noNulls = false;
@@ -150,7 +175,11 @@ public class <ClassName> extends VectorExpression {
           if (!arg1ColVector.isNull[i] && vector1[i] == 1) {
             if (!arg2IsNull[i]) {
               outputIsNull[i] = false;
+#IF DECIMAL
+              outputColVector.set(i, vector2[i]);
+#ELSE
               outputColVector.set(i, arg2ColVector.asScratch<ObjectName>(i));
+#ENDIF DECIMAL
             } else {
               outputIsNull[i] = true;
               outputColVector.noNulls = false;
@@ -158,7 +187,11 @@ public class <ClassName> extends VectorExpression {
           } else {
             if (!arg3IsNull[i]) {
               outputIsNull[i] = false;
+#IF DECIMAL
+              outputColVector.set(i, vector3[i]);
+#ELSE
               outputColVector.set(i, arg3ColVector.asScratch<ObjectName>(i));
+#ENDIF DECIMAL
             } else {
               outputIsNull[i] = true;
               outputColVector.noNulls = false;
@@ -170,7 +203,11 @@ public class <ClassName> extends VectorExpression {
           if (!arg1ColVector.isNull[i] && vector1[i] == 1) {
             if (!arg2IsNull[i]) {
               outputIsNull[i] = false;
+#IF DECIMAL
+              outputColVector.set(i, vector2[i]);
+#ELSE
               outputColVector.set(i, arg2ColVector.asScratch<ObjectName>(i));
+#ENDIF DECIMAL
             } else {
               outputIsNull[i] = true;
               outputColVector.noNulls = false;
@@ -178,7 +215,11 @@ public class <ClassName> extends VectorExpression {
           } else {
             if (!arg3IsNull[i]) {
               outputIsNull[i] = false;
+#IF DECIMAL
+              outputColVector.set(i, vector3[i]);
+#ELSE
               outputColVector.set(i, arg3ColVector.asScratch<ObjectName>(i));
+#ENDIF DECIMAL
             } else {
               outputIsNull[i] = true;
               outputColVector.noNulls = false;

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/gen/vectorization/ExpressionTemplates/IfExprObjectColumnScalar.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/IfExprObjectColumnScalar.txt b/ql/src/gen/vectorization/ExpressionTemplates/IfExprObjectColumnScalar.txt
index 56ae2ca..f0d2908 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/IfExprObjectColumnScalar.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/IfExprObjectColumnScalar.txt
@@ -25,6 +25,9 @@ import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import <ScalarImport>;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
+#IF DECIMAL
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+#ENDIF DECIMAL
 
 /**
  * Compute IF(expr1, expr2, expr3) for 3 input column expressions.
@@ -67,6 +70,9 @@ public class <ClassName> extends VectorExpression {
     LongColumnVector arg1ColVector = (LongColumnVector) batch.cols[arg1Column];
     <ColumnVectorType> arg2ColVector = (<ColumnVectorType>) batch.cols[arg2Column];
     boolean[] arg2IsNull = arg2ColVector.isNull;
+#IF DECIMAL
+    HiveDecimalWritable[] vector2 = arg2ColVector.vector;
+#ENDIF DECIMAL
     <ColumnVectorType> outputColVector = (<ColumnVectorType>) batch.cols[outputColumnNum];
     int[] sel = batch.selected;
     boolean[] outputIsNull = outputColVector.isNull;
@@ -104,7 +110,11 @@ public class <ClassName> extends VectorExpression {
           if (vector1[i] == 1) {
             if (!arg2IsNull[i]) {
               outputIsNull[i] = false;
+#IF DECIMAL
+              outputColVector.set(i, vector2[i]);
+#ELSE
               outputColVector.set(i, arg2ColVector.asScratch<ObjectName>(i));
+#ENDIF DECIMAL
             } else {
               outputIsNull[i] = true;
               outputColVector.noNulls = false;
@@ -119,7 +129,11 @@ public class <ClassName> extends VectorExpression {
           if (vector1[i] == 1) {
             if (!arg2IsNull[i]) {
               outputIsNull[i] = false;
+#IF DECIMAL
+              outputColVector.set(i, vector2[i]);
+#ELSE
               outputColVector.set(i, arg2ColVector.asScratch<ObjectName>(i));
+#ENDIF DECIMAL
             } else {
               outputIsNull[i] = true;
               outputColVector.noNulls = false;
@@ -138,7 +152,11 @@ public class <ClassName> extends VectorExpression {
           if (!arg1ColVector.isNull[i] && vector1[i] == 1) {
             if (!arg2IsNull[i]) {
               outputIsNull[i] = false;
+#IF DECIMAL
+              outputColVector.set(i, vector2[i]);
+#ELSE
               outputColVector.set(i, arg2ColVector.asScratch<ObjectName>(i));
+#ENDIF DECIMAL
             } else {
               outputIsNull[i] = true;
               outputColVector.noNulls = false;
@@ -153,7 +171,11 @@ public class <ClassName> extends VectorExpression {
           if (!arg1ColVector.isNull[i] && vector1[i] == 1) {
             if (!arg2IsNull[i]) {
               outputIsNull[i] = false;
+#IF DECIMAL
+              outputColVector.set(i, vector2[i]);
+#ELSE
               outputColVector.set(i, arg2ColVector.asScratch<ObjectName>(i));
+#ENDIF DECIMAL
             } else {
               outputIsNull[i] = true;
               outputColVector.noNulls = false;

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/gen/vectorization/ExpressionTemplates/IfExprObjectScalarColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/IfExprObjectScalarColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/IfExprObjectScalarColumn.txt
index 271b589..980f506 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/IfExprObjectScalarColumn.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/IfExprObjectScalarColumn.txt
@@ -25,6 +25,9 @@ import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import <ScalarImport>;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
+#IF DECIMAL
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+#ENDIF DECIMAL
 
 /**
  * Compute IF(expr1, expr2, expr3) for 3 input column expressions.
@@ -67,6 +70,9 @@ public class <ClassName> extends VectorExpression {
     LongColumnVector arg1ColVector = (LongColumnVector) batch.cols[arg1Column];
     <ColumnVectorType> arg3ColVector = (<ColumnVectorType>) batch.cols[arg3Column];
     boolean[] arg3IsNull = arg3ColVector.isNull;
+#IF DECIMAL
+    HiveDecimalWritable[] vector3 = arg3ColVector.vector;
+#ENDIF DECIMAL
     <ColumnVectorType> outputColVector = (<ColumnVectorType>) batch.cols[outputColumnNum];
     int[] sel = batch.selected;
     boolean[] outputIsNull = outputColVector.isNull;
@@ -109,7 +115,11 @@ public class <ClassName> extends VectorExpression {
           } else {
             if (!arg3IsNull[i]) {
               outputIsNull[i] = false;
+#IF DECIMAL
+              outputColVector.set(i, vector3[i]);
+#ELSE
               outputColVector.set(i, arg3ColVector.asScratch<ObjectName>(i));
+#ENDIF DECIMAL
             } else {
               outputIsNull[i] = true;
               outputColVector.noNulls = false;
@@ -124,7 +134,11 @@ public class <ClassName> extends VectorExpression {
           } else {
             if (!arg3IsNull[i]) {
               outputIsNull[i] = false;
+#IF DECIMAL
+              outputColVector.set(i, vector3[i]);
+#ELSE
               outputColVector.set(i, arg3ColVector.asScratch<ObjectName>(i));
+#ENDIF DECIMAL
             } else {
               outputIsNull[i] = true;
               outputColVector.noNulls = false;
@@ -143,7 +157,11 @@ public class <ClassName> extends VectorExpression {
           } else {
             if (!arg3IsNull[i]) {
               outputIsNull[i] = false;
+#IF DECIMAL
+              outputColVector.set(i, vector3[i]);
+#ELSE
               outputColVector.set(i, arg3ColVector.asScratch<ObjectName>(i));
+#ENDIF DECIMAL
             } else {
               outputIsNull[i] = true;
               outputColVector.noNulls = false;
@@ -158,7 +176,11 @@ public class <ClassName> extends VectorExpression {
           } else {
             if (!arg3IsNull[i]) {
               outputIsNull[i] = false;
+#IF DECIMAL
+              outputColVector.set(i, vector3[i]);
+#ELSE
               outputColVector.set(i, arg3ColVector.asScratch<ObjectName>(i));
+#ENDIF DECIMAL
             } else {
               outputIsNull[i] = true;
               outputColVector.noNulls = false;

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/gen/vectorization/ExpressionTemplates/IfExprScalarColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/IfExprScalarColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/IfExprScalarColumn.txt
index 3658129..a5a2957 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/IfExprScalarColumn.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/IfExprScalarColumn.txt
@@ -35,9 +35,9 @@ public class <ClassName> extends VectorExpression {
 
   private static final long serialVersionUID = 1L;
 
-  private final int arg1Column;
-  private final <OperandType2> arg2Scalar;
-  private final int arg3Column;
+  protected final int arg1Column;
+  protected final <OperandType2> arg2Scalar;
+  protected final int arg3Column;
 
   public <ClassName>(int arg1Column, <OperandType2> arg2Scalar, int arg3Column,
       int outputColumnNum) {

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/gen/vectorization/ExpressionTemplates/IfExprScalarScalar.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/IfExprScalarScalar.txt b/ql/src/gen/vectorization/ExpressionTemplates/IfExprScalarScalar.txt
index d11459a..20ce883 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/IfExprScalarScalar.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/IfExprScalarScalar.txt
@@ -36,9 +36,9 @@ public class <ClassName> extends VectorExpression {
 
   private static final long serialVersionUID = 1L;
 
-  private final int arg1Column;
-  private final <OperandType2> arg2Scalar;
-  private final <OperandType3> arg3Scalar;
+  protected final int arg1Column;
+  protected final <OperandType2> arg2Scalar;
+  protected final <OperandType3> arg3Scalar;
 
   public <ClassName>(int arg1Column, <OperandType2> arg2Scalar, <OperandType3> arg3Scalar,
       int outputColumnNum) {

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/gen/vectorization/ExpressionTemplates/ScalarCompareColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/ScalarCompareColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/ScalarCompareColumn.txt
index 753f061..60dc725 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/ScalarCompareColumn.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/ScalarCompareColumn.txt
@@ -37,8 +37,8 @@ public class <ClassName> extends VectorExpression {
 
   private static final long serialVersionUID = 1L;
 
-  private final <OperandType1> value;
-  private final int colNum;
+  protected final <OperandType1> value;
+  protected final int colNum;
 
   public <ClassName>(<OperandType1> value, int colNum, int outputColumnNum) {
     super(outputColumnNum);

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorExpressionDescriptor.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorExpressionDescriptor.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorExpressionDescriptor.java
index 2f8a419..fb40f5e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorExpressionDescriptor.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorExpressionDescriptor.java
@@ -89,6 +89,8 @@ public class VectorExpressionDescriptor {
     INT_INTERVAL_YEAR_MONTH     (INT_FAMILY.value | INTERVAL_YEAR_MONTH.value),
     INT_DATE_INTERVAL_YEAR_MONTH  (INT_FAMILY.value | DATE.value | INTERVAL_YEAR_MONTH.value),
     STRING_DATETIME_FAMILY  (STRING_FAMILY.value | DATETIME_FAMILY.value),
+    STRING_FAMILY_BINARY    (STRING_FAMILY.value | BINARY.value),
+    STRING_BINARY           (STRING.value | BINARY.value),
     ALL_FAMILY              (0xFFFFFFL);
 
     private final long value;

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java
index 20cc894..e541217 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java
@@ -1533,88 +1533,88 @@ public class VectorizationContext {
     return false;
   }
 
-  public boolean haveCandidateForDecimal64VectorExpression(int numChildren,
-      List<ExprNodeDesc> childExpr, TypeInfo returnType) throws HiveException {
-
-    // For now, just 2 Decimal64 inputs and a Decimal64 or boolean output.
-    return (numChildren == 2 &&
-        checkExprNodeDescForDecimal64(childExpr.get(0)) &&
-        checkExprNodeDescForDecimal64(childExpr.get(1)) &&
-        (checkTypeInfoForDecimal64(returnType) ||
-            returnType.equals(TypeInfoFactory.booleanTypeInfo)));
-  }
-
   private VectorExpression getDecimal64VectorExpressionForUdf(GenericUDF genericUdf,
-      Class<?> udfClass, List<ExprNodeDesc> childExpr, int numChildren,
-      VectorExpressionDescriptor.Mode mode, TypeInfo returnType) throws HiveException {
-
-    ExprNodeDesc child1 = childExpr.get(0);
-    ExprNodeDesc child2 = childExpr.get(1);
-
-    DecimalTypeInfo decimalTypeInfo1 = (DecimalTypeInfo) child1.getTypeInfo();
-    DecimalTypeInfo decimalTypeInfo2 = (DecimalTypeInfo) child2.getTypeInfo();
-
-    DataTypePhysicalVariation dataTypePhysicalVariation1 = DataTypePhysicalVariation.DECIMAL_64;
-    DataTypePhysicalVariation dataTypePhysicalVariation2 = DataTypePhysicalVariation.DECIMAL_64;
-
-    final int scale1 = decimalTypeInfo1.scale();
-    final int scale2 = decimalTypeInfo2.scale();
+      Class<?> udfClass, List<ExprNodeDesc> childExprs, int numChildren,
+      VectorExpressionDescriptor.Mode mode, TypeInfo returnTypeInfo) throws HiveException {
 
     VectorExpressionDescriptor.Builder builder = new VectorExpressionDescriptor.Builder();
     builder.setNumArguments(numChildren);
     builder.setMode(mode);
 
-    boolean isColumnScaleEstablished = false;
-    int columnScale = 0;
-    boolean hasScalar = false;
-    builder.setArgumentType(0, ArgumentType.DECIMAL_64);
-    if (child1 instanceof ExprNodeGenericFuncDesc ||
-        child1 instanceof ExprNodeColumnDesc) {
-      builder.setInputExpressionType(0, InputExpressionType.COLUMN);
-      isColumnScaleEstablished = true;
-      columnScale = scale1;
-    } else if (child1 instanceof ExprNodeConstantDesc) {
-      if (isNullConst(child1)) {
-
-        // Cannot handle NULL scalar parameter.
+    // DECIMAL_64 decimals must have same scale.
+    boolean anyDecimal64Expr = false;
+    boolean isDecimal64ScaleEstablished = false;
+    int decimal64ColumnScale = 0;
+
+    for (int i = 0; i < numChildren; i++) {
+      ExprNodeDesc childExpr = childExprs.get(i);
+
+      /*
+       * For columns, we check decimal columns for DECIMAL_64 DataTypePhysicalVariation.
+       * For UDFs, we check for @VectorizedExpressionsSupportDecimal64 annotation, etc.
+       */
+      final boolean isExprDecimal64 = checkExprNodeDescForDecimal64(childExpr);
+      if (isExprDecimal64) {
+        anyDecimal64Expr = true;
+      }
+
+      TypeInfo typeInfo = childExpr.getTypeInfo();
+      if (childExpr instanceof ExprNodeGenericFuncDesc ||
+          childExpr instanceof ExprNodeColumnDesc) {
+        if (isExprDecimal64) {
+          DecimalTypeInfo decimalTypeInfo = (DecimalTypeInfo) typeInfo;
+          if (!isDecimal64ScaleEstablished) {
+            decimal64ColumnScale = decimalTypeInfo.getScale();
+            isDecimal64ScaleEstablished = true;
+          } else if (decimalTypeInfo.getScale() != decimal64ColumnScale) {
+            return null;
+          }
+        }
+        builder.setInputExpressionType(i, InputExpressionType.COLUMN);
+      } else if (childExpr instanceof ExprNodeConstantDesc) {
+        if (isNullConst(childExpr)) {
+          // Cannot handle NULL scalar parameter.
+          return null;
+        }
+        builder.setInputExpressionType(i, InputExpressionType.SCALAR);
+      } else {
         return null;
       }
-      hasScalar = true;
-      builder.setInputExpressionType(0, InputExpressionType.SCALAR);
-    } else {
 
-      // Currently, only functions, columns, and scalars supported.
-      return null;
+      if (isExprDecimal64) {
+        builder.setArgumentType(i, ArgumentType.DECIMAL_64);
+      } else {
+        String undecoratedTypeName = getUndecoratedName(childExpr.getTypeString());
+        if (undecoratedTypeName == null) {
+          return null;
+        }
+        builder.setArgumentType(i, undecoratedTypeName);
+      }
     }
 
-    builder.setArgumentType(1, ArgumentType.DECIMAL_64);
-    if (child2 instanceof ExprNodeGenericFuncDesc ||
-        child2 instanceof ExprNodeColumnDesc) {
-      builder.setInputExpressionType(1, InputExpressionType.COLUMN);
-      if (!isColumnScaleEstablished) {
-        isColumnScaleEstablished = true;
-        columnScale = scale2;
-      } else if (columnScale != scale2) {
-
-        // We only support Decimal64 on 2 columns when the have the same scale.
-        return null;
-      }
-    } else if (child2 instanceof ExprNodeConstantDesc) {
-      // Cannot have SCALAR, SCALAR.
-      if (!isColumnScaleEstablished) {
-        return null;
-      }
-      if (isNullConst(child2)) {
+    if (!anyDecimal64Expr) {
+      return null;
+    }
 
-        // Cannot handle NULL scalar parameter.
+    final boolean isReturnDecimal64 = checkTypeInfoForDecimal64(returnTypeInfo);
+    final DataTypePhysicalVariation returnDataTypePhysicalVariation;
+    if (isReturnDecimal64) {
+      DecimalTypeInfo returnDecimalTypeInfo = (DecimalTypeInfo) returnTypeInfo;
+      if (!isDecimal64ScaleEstablished) {
+        decimal64ColumnScale = returnDecimalTypeInfo.getScale();
+        isDecimal64ScaleEstablished = true;
+      } else if (returnDecimalTypeInfo.getScale() != decimal64ColumnScale) {
         return null;
       }
-      hasScalar = true;
-      builder.setInputExpressionType(1, InputExpressionType.SCALAR);
-    } else {
+      returnDataTypePhysicalVariation = DataTypePhysicalVariation.DECIMAL_64;
+    } else if (returnTypeInfo instanceof DecimalTypeInfo){
 
-      // Currently, only functions, columns, and scalars supported.
+      // Currently, we don't have any vectorized expressions that take DECIMAL_64 inputs
+      // and produce a regular decimal.  Or, currently, a way to express that in the
+      // descriptor.
       return null;
+    } else {
+      returnDataTypePhysicalVariation = DataTypePhysicalVariation.NONE;
     }
 
     VectorExpressionDescriptor.Descriptor descriptor = builder.build();
@@ -1632,19 +1632,26 @@ public class VectorizationContext {
 
     List<VectorExpression> children = new ArrayList<VectorExpression>();
     Object[] arguments = new Object[numChildren];
+    TypeInfo[] typeInfos = new TypeInfo[numChildren];
+    DataTypePhysicalVariation[] dataTypePhysicalVariations = new DataTypePhysicalVariation[numChildren];
 
     for (int i = 0; i < numChildren; i++) {
-      ExprNodeDesc child = childExpr.get(i);
-      if (child instanceof ExprNodeGenericFuncDesc) {
-        VectorExpression vChild = getVectorExpression(child, childrenMode);
+      ExprNodeDesc childExpr = childExprs.get(i);
+      TypeInfo typeInfo = childExpr.getTypeInfo();
+      typeInfos[i] = typeInfo;
+      dataTypePhysicalVariations[i] =
+          (checkTypeInfoForDecimal64(typeInfo) ?
+              DataTypePhysicalVariation.DECIMAL_64 : DataTypePhysicalVariation.NONE);
+      if (childExpr instanceof ExprNodeGenericFuncDesc) {
+        VectorExpression vChild = getVectorExpression(childExpr, childrenMode);
         children.add(vChild);
         arguments[i] = vChild.getOutputColumnNum();
-      } else if (child instanceof ExprNodeColumnDesc) {
-        int colIndex = getInputColumnIndex((ExprNodeColumnDesc) child);
+      } else if (childExpr instanceof ExprNodeColumnDesc) {
+        int colIndex = getInputColumnIndex((ExprNodeColumnDesc) childExpr);
         if (childrenMode == VectorExpressionDescriptor.Mode.FILTER) {
 
           VectorExpression filterExpr =
-              getFilterOnBooleanColumnExpression((ExprNodeColumnDesc) child, colIndex);
+              getFilterOnBooleanColumnExpression((ExprNodeColumnDesc) childExpr, colIndex);
           if (filterExpr == null) {
             return null;
           }
@@ -1653,16 +1660,27 @@ public class VectorizationContext {
         }
         arguments[i] = colIndex;
       } else {
-        Preconditions.checkState(child instanceof ExprNodeConstantDesc);
-        ExprNodeConstantDesc constDesc = (ExprNodeConstantDesc) child;
-        HiveDecimal hiveDecimal = (HiveDecimal) constDesc.getValue();
-        if (hiveDecimal.scale() > columnScale) {
+        Preconditions.checkState(childExpr instanceof ExprNodeConstantDesc);
+        ExprNodeConstantDesc constDesc = (ExprNodeConstantDesc) childExpr;
+        if (typeInfo instanceof DecimalTypeInfo) {
+          if (!isDecimal64ScaleEstablished) {
+            return null;
+          }
+          HiveDecimal hiveDecimal = (HiveDecimal) constDesc.getValue();
+          if (hiveDecimal.scale() > decimal64ColumnScale) {
 
-          // For now, bail out on decimal constants with larger scale than column scale.
-          return null;
+            // For now, bail out on decimal constants with larger scale than column scale.
+            return null;
+          }
+          final long decimal64Scalar =
+              new HiveDecimalWritable(hiveDecimal).serialize64(decimal64ColumnScale);
+          arguments[i] = decimal64Scalar;
+        } else {
+          Object scalarValue = getVectorTypeScalarValue(constDesc);
+          arguments[i] =
+              (scalarValue == null) ?
+                  getConstantVectorExpression(null, typeInfo, childrenMode) : scalarValue;
         }
-        final long decimal64Scalar = new HiveDecimalWritable(hiveDecimal).serialize64(columnScale);
-        arguments[i] = decimal64Scalar;
       }
     }
 
@@ -1672,13 +1690,13 @@ public class VectorizationContext {
      * The instantiateExpression method sets the output column and type information.
      */
     VectorExpression  vectorExpression =
-        instantiateExpression(vectorClass, returnType, DataTypePhysicalVariation.DECIMAL_64, arguments);
+        instantiateExpression(vectorClass, returnTypeInfo, returnDataTypePhysicalVariation, arguments);
     if (vectorExpression == null) {
-      handleCouldNotInstantiateVectorExpression(vectorClass, returnType, DataTypePhysicalVariation.DECIMAL_64, arguments);
+      handleCouldNotInstantiateVectorExpression(vectorClass, returnTypeInfo, returnDataTypePhysicalVariation, arguments);
     }
 
-    vectorExpression.setInputTypeInfos(decimalTypeInfo1, decimalTypeInfo2);
-    vectorExpression.setInputDataTypePhysicalVariations(dataTypePhysicalVariation1, dataTypePhysicalVariation2);
+    vectorExpression.setInputTypeInfos(typeInfos);
+    vectorExpression.setInputDataTypePhysicalVariations(dataTypePhysicalVariations);
 
     if ((vectorExpression != null) && !children.isEmpty()) {
       vectorExpression.setChildExpressions(children.toArray(new VectorExpression[0]));
@@ -1728,16 +1746,15 @@ public class VectorizationContext {
       return null;
     }
 
-    // Should we intercept here for a possible Decimal64 vector expression class?
-    if (haveCandidateForDecimal64VectorExpression(numChildren, childExpr, returnType)) {
-      VectorExpression result = getDecimal64VectorExpressionForUdf(genericUdf, udfClass,
-          childExpr, numChildren, mode, returnType);
-      if (result != null) {
-        return result;
-      }
-      // Otherwise, fall through and proceed with non-Decimal64 vector expression classes...
+    // Intercept here for a possible Decimal64 vector expression class.
+    VectorExpression result = getDecimal64VectorExpressionForUdf(genericUdf, udfClass,
+        childExpr, numChildren, mode, returnType);
+    if (result != null) {
+      return result;
     }
 
+    // Otherwise, fall through and proceed with non-Decimal64 vector expression classes...
+
     VectorExpressionDescriptor.Builder builder = new VectorExpressionDescriptor.Builder();
     builder.setNumArguments(numChildren);
     builder.setMode(mode);
@@ -2062,6 +2079,8 @@ public class VectorizationContext {
       ve = getCastToChar(childExpr, returnType);
     } else if (udf instanceof GenericUDFToVarchar) {
       ve = getCastToVarChar(childExpr, returnType);
+    } else if (udf instanceof GenericUDFToBinary) {
+      ve = getCastToBinary(childExpr, returnType);
     } else if (udf instanceof GenericUDFTimestamp) {
       ve = getCastToTimestamp((GenericUDFTimestamp)udf, childExpr, mode, returnType);
     }
@@ -2864,6 +2883,25 @@ public class VectorizationContext {
     return null;
   }
 
+  private VectorExpression getCastToBinary(List<ExprNodeDesc> childExpr, TypeInfo returnType)
+      throws HiveException {
+    ExprNodeDesc child = childExpr.get(0);
+    String inputType = childExpr.get(0).getTypeString();
+    if (child instanceof ExprNodeConstantDesc) {
+      // Don't do constant folding here.  Wait until the optimizer is changed to do it.
+      // Family of related JIRAs: HIVE-7421, HIVE-7422, and HIVE-7424.
+      return null;
+    }
+    if (inputType.equalsIgnoreCase("string") || varcharTypePattern.matcher(inputType).matches()) {
+
+      // STRING and VARCHAR types require no conversion, so use a no-op.
+      return getIdentityExpression(childExpr);
+    } else if (charTypePattern.matcher(inputType).matches()) {
+      return createVectorExpression(CastCharToBinary.class, childExpr, VectorExpressionDescriptor.Mode.PROJECTION, returnType);
+    }
+    return null;
+  }
+
   private VectorExpression getCastToDoubleExpression(Class<?> udf, List<ExprNodeDesc> childExpr,
       TypeInfo returnType) throws HiveException {
     ExprNodeDesc child = childExpr.get(0);


[7/8] hive git commit: HIVE-20207: Vectorization: Fix NULL / Wrong Results issues in Filter / Compare (Matt McCline, reviewed by Teddy Choi)

Posted by mm...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java.orig
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java.orig b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java.orig
new file mode 100644
index 0000000..20cc894
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java.orig
@@ -0,0 +1,3771 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.exec.vector;
+
+import java.lang.reflect.Constructor;
+import java.nio.charset.StandardCharsets;
+import java.sql.Timestamp;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.TreeSet;
+import java.util.regex.Pattern;
+
+import org.apache.commons.lang.ArrayUtils;
+import org.apache.hadoop.hive.common.type.Date;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+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.HiveIntervalDayTime;
+import org.apache.hadoop.hive.common.type.HiveIntervalYearMonth;
+import org.apache.hadoop.hive.common.type.HiveVarchar;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluator;
+import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluatorFactory;
+import org.apache.hadoop.hive.ql.exec.FunctionInfo;
+import org.apache.hadoop.hive.ql.exec.FunctionRegistry;
+import org.apache.hadoop.hive.ql.exec.UDF;
+import org.apache.hadoop.hive.ql.exec.vector.ColumnVector.Type;
+import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor.ArgumentType;
+import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor.InputExpressionType;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.*;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.*;
+import org.apache.hadoop.hive.ql.exec.vector.udf.VectorUDFAdaptor;
+import org.apache.hadoop.hive.ql.exec.vector.udf.VectorUDFArgDesc;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeDynamicValueDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeFieldDesc;
+import org.apache.hadoop.hive.ql.udf.*;
+import org.apache.hadoop.hive.ql.udf.generic.*;
+import org.apache.hadoop.hive.serde2.ByteStream.Output;
+import org.apache.hadoop.hive.serde2.binarysortable.fast.BinarySortableSerializeWrite;
+import org.apache.hadoop.hive.serde2.io.DateWritableV2;
+import org.apache.hadoop.hive.serde2.io.DoubleWritable;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.hadoop.hive.serde2.objectinspector.ConstantObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory;
+import org.apache.hadoop.hive.serde2.typeinfo.BaseCharTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.HiveDecimalUtils;
+import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.StructTypeInfo;
+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 org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hive.common.util.AnnotationUtils;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+
+/**
+ * Context class for vectorization execution.
+ * Main role is to map column names to column indices and serves as a
+ * factory class for building vectorized expressions out of descriptors.
+ *
+ */
+public class VectorizationContext {
+
+  private static final Logger LOG = LoggerFactory.getLogger(
+      VectorizationContext.class.getName());
+
+  private final String contextName;
+  private final int level;
+
+  VectorExpressionDescriptor vMap;
+
+  private final List<String> initialColumnNames;
+  private List<TypeInfo> initialTypeInfos;
+  private List<DataTypePhysicalVariation> initialDataTypePhysicalVariations;
+
+  private List<Integer> projectedColumns;
+  private List<String> projectionColumnNames;
+  private Map<String, Integer> projectionColumnMap;
+
+  //columnName to column position map
+  // private final Map<String, Integer> columnMap;
+  private int firstOutputColumnIndex;
+
+  public enum HiveVectorAdaptorUsageMode {
+    NONE,
+    CHOSEN,
+    ALL;
+
+    public static HiveVectorAdaptorUsageMode getHiveConfValue(HiveConf hiveConf) {
+      String string = HiveConf.getVar(hiveConf,
+          HiveConf.ConfVars.HIVE_VECTOR_ADAPTOR_USAGE_MODE);
+      return valueOf(string.toUpperCase());
+    }
+  }
+
+  private HiveVectorAdaptorUsageMode hiveVectorAdaptorUsageMode;
+  private boolean testVectorAdaptorOverride;
+
+  public enum HiveVectorIfStmtMode {
+    ADAPTOR,
+    GOOD,
+    BETTER;
+
+    public static HiveVectorIfStmtMode getHiveConfValue(HiveConf hiveConf) {
+      String string = HiveConf.getVar(hiveConf,
+          HiveConf.ConfVars.HIVE_VECTORIZED_IF_EXPR_MODE);
+      return valueOf(string.toUpperCase());
+    }
+  }
+
+  private HiveVectorIfStmtMode hiveVectorIfStmtMode;
+
+  //when set to true use the overflow checked vector expressions
+  private boolean useCheckedVectorExpressions;
+
+  private boolean reuseScratchColumns =
+      HiveConf.ConfVars.HIVE_VECTORIZATION_TESTING_REUSE_SCRATCH_COLUMNS.defaultBoolVal;
+
+  private boolean adaptorSuppressEvaluateExceptions;
+
+  private void setHiveConfVars(HiveConf hiveConf) {
+    hiveVectorAdaptorUsageMode = HiveVectorAdaptorUsageMode.getHiveConfValue(hiveConf);
+    testVectorAdaptorOverride =
+        HiveConf.getBoolVar(hiveConf, ConfVars.HIVE_TEST_VECTOR_ADAPTOR_OVERRIDE);
+    hiveVectorIfStmtMode = HiveVectorIfStmtMode.getHiveConfValue(hiveConf);
+    this.reuseScratchColumns =
+        HiveConf.getBoolVar(hiveConf, ConfVars.HIVE_VECTORIZATION_TESTING_REUSE_SCRATCH_COLUMNS);
+    this.ocm.setReuseColumns(reuseScratchColumns);
+    useCheckedVectorExpressions =
+        HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVE_VECTORIZATION_USE_CHECKED_EXPRESSIONS);
+    adaptorSuppressEvaluateExceptions =
+        HiveConf.getBoolVar(
+            hiveConf, HiveConf.ConfVars.HIVE_VECTORIZED_ADAPTOR_SUPPRESS_EVALUATE_EXCEPTIONS);
+  }
+
+  private void copyHiveConfVars(VectorizationContext vContextEnvironment) {
+    hiveVectorAdaptorUsageMode = vContextEnvironment.hiveVectorAdaptorUsageMode;
+    testVectorAdaptorOverride = vContextEnvironment.testVectorAdaptorOverride;
+    hiveVectorIfStmtMode = vContextEnvironment.hiveVectorIfStmtMode;
+    this.reuseScratchColumns = vContextEnvironment.reuseScratchColumns;
+    useCheckedVectorExpressions = vContextEnvironment.useCheckedVectorExpressions;
+    adaptorSuppressEvaluateExceptions = vContextEnvironment.adaptorSuppressEvaluateExceptions;
+    this.ocm.setReuseColumns(reuseScratchColumns);
+  }
+
+  // Convenient constructor for initial batch creation takes
+  // a list of columns names and maps them to 0..n-1 indices.
+  public VectorizationContext(
+      String contextName,
+      List<String> initialColumnNames,
+      List<TypeInfo> initialTypeInfos,
+      List<DataTypePhysicalVariation> initialDataTypePhysicalVariations,
+      HiveConf hiveConf) {
+    this.contextName = contextName;
+    level = 0;
+    this.initialColumnNames = initialColumnNames;
+    this.initialTypeInfos = initialTypeInfos;
+    this.initialDataTypePhysicalVariations = initialDataTypePhysicalVariations;
+    this.projectionColumnNames = initialColumnNames;
+
+    projectedColumns = new ArrayList<Integer>();
+    projectionColumnMap = new HashMap<String, Integer>();
+    for (int i = 0; i < this.projectionColumnNames.size(); i++) {
+      projectedColumns.add(i);
+      projectionColumnMap.put(projectionColumnNames.get(i), i);
+    }
+
+    int firstOutputColumnIndex = projectedColumns.size();
+    this.ocm = new OutputColumnManager(firstOutputColumnIndex);
+    this.firstOutputColumnIndex = firstOutputColumnIndex;
+    vMap = new VectorExpressionDescriptor();
+
+    if (hiveConf != null) {
+      setHiveConfVars(hiveConf);
+    }
+  }
+
+  // Convenient constructor for initial batch creation takes
+  // a list of columns names and maps them to 0..n-1 indices.
+  public VectorizationContext(String contextName, List<String> initialColumnNames,
+      HiveConf hiveConf) {
+    this.contextName = contextName;
+    level = 0;
+    this.initialColumnNames = initialColumnNames;
+    this.projectionColumnNames = initialColumnNames;
+
+    projectedColumns = new ArrayList<Integer>();
+    projectionColumnMap = new HashMap<String, Integer>();
+    for (int i = 0; i < this.projectionColumnNames.size(); i++) {
+      projectedColumns.add(i);
+      projectionColumnMap.put(projectionColumnNames.get(i), i);
+    }
+
+    int firstOutputColumnIndex = projectedColumns.size();
+    this.ocm = new OutputColumnManager(firstOutputColumnIndex);
+    this.firstOutputColumnIndex = firstOutputColumnIndex;
+    vMap = new VectorExpressionDescriptor();
+
+    if (hiveConf != null) {
+      setHiveConfVars(hiveConf);
+    }
+  }
+
+  public VectorizationContext(String contextName, List<String> initialColumnNames,
+      VectorizationContext vContextEnvironment) {
+    this(contextName, initialColumnNames, (HiveConf) null);
+    copyHiveConfVars(vContextEnvironment);
+  }
+
+  @VisibleForTesting
+  public VectorizationContext(String contextName, List<String> initialColumnNames) {
+    this(contextName, initialColumnNames, (HiveConf) null);
+  }
+
+  // Constructor to with the individual addInitialColumn method
+  // followed by a call to finishedAddingInitialColumns.
+  public VectorizationContext(String contextName, HiveConf hiveConf) {
+    this.contextName = contextName;
+    level = 0;
+    initialColumnNames = new ArrayList<String>();
+    projectedColumns = new ArrayList<Integer>();
+    projectionColumnNames = new ArrayList<String>();
+    projectionColumnMap = new HashMap<String, Integer>();
+    this.ocm = new OutputColumnManager(0);
+    this.firstOutputColumnIndex = 0;
+    vMap = new VectorExpressionDescriptor();
+
+    if (hiveConf != null) {
+      setHiveConfVars(hiveConf);
+    }
+
+  }
+
+  @VisibleForTesting
+  public VectorizationContext(String contextName) {
+    this(contextName, (HiveConf) null);
+  }
+
+  // Constructor useful making a projection vectorization context.  E.g. VectorSelectOperator.
+  // Use with resetProjectionColumns and addProjectionColumn.
+  // Keeps existing output column map, etc.
+  public VectorizationContext(String contextName, VectorizationContext vContext) {
+    this.contextName = contextName;
+    level = vContext.level + 1;
+    this.initialColumnNames = vContext.initialColumnNames;
+    this.initialTypeInfos = vContext.initialTypeInfos;
+    this.initialDataTypePhysicalVariations = vContext.initialDataTypePhysicalVariations;
+    this.projectedColumns = new ArrayList<Integer>();
+    this.projectionColumnNames = new ArrayList<String>();
+    this.projectionColumnMap = new HashMap<String, Integer>();
+
+    this.ocm = vContext.ocm;
+    this.firstOutputColumnIndex = vContext.firstOutputColumnIndex;
+    vMap = new VectorExpressionDescriptor();
+
+    copyHiveConfVars(vContext);
+  }
+
+  // Add an initial column to a vectorization context when
+  // a vectorized row batch is being created.
+  public void addInitialColumn(String columnName) {
+    initialColumnNames.add(columnName);
+    int index = projectedColumns.size();
+    projectedColumns.add(index);
+    projectionColumnNames.add(columnName);
+    projectionColumnMap.put(columnName, index);
+  }
+
+  // Finishes the vectorization context after all the initial
+  // columns have been added.
+  @VisibleForTesting
+  public void finishedAddingInitialColumns() {
+    int firstOutputColumnIndex = projectedColumns.size();
+    this.ocm = new OutputColumnManager(firstOutputColumnIndex);
+    this.ocm.setReuseColumns(this.reuseScratchColumns);
+    this.firstOutputColumnIndex = firstOutputColumnIndex;
+  }
+
+  // Empties the projection columns.
+  public void resetProjectionColumns() {
+    projectedColumns = new ArrayList<Integer>();
+    projectionColumnNames = new ArrayList<String>();
+    projectionColumnMap = new HashMap<String, Integer>();
+  }
+
+  // Add a projection column to a projection vectorization context.
+  public void addProjectionColumn(String columnName, int vectorBatchColIndex) {
+    if (vectorBatchColIndex < 0) {
+      throw new RuntimeException("Negative projected column number");
+    }
+    projectedColumns.add(vectorBatchColIndex);
+    projectionColumnNames.add(columnName);
+    projectionColumnMap.put(columnName, vectorBatchColIndex);
+  }
+
+  public void setInitialTypeInfos(List<TypeInfo> initialTypeInfos) {
+    this.initialTypeInfos = initialTypeInfos;
+    final int size = initialTypeInfos.size();
+    initialDataTypePhysicalVariations = new ArrayList<DataTypePhysicalVariation>(size);
+    for (int i = 0; i < size; i++) {
+      initialDataTypePhysicalVariations.add(DataTypePhysicalVariation.NONE);
+    }
+  }
+
+  public void setInitialDataTypePhysicalVariations(
+      List<DataTypePhysicalVariation> initialDataTypePhysicalVariations) {
+    this.initialDataTypePhysicalVariations = initialDataTypePhysicalVariations;
+  }
+
+  public List<String> getInitialColumnNames() {
+    return initialColumnNames;
+  }
+
+  public List<Integer> getProjectedColumns() {
+    return projectedColumns;
+  }
+
+  public List<String> getProjectionColumnNames() {
+    return projectionColumnNames;
+  }
+
+  public Map<String, Integer> getProjectionColumnMap() {
+    return projectionColumnMap;
+  }
+
+  public TypeInfo[] getInitialTypeInfos() {
+    return initialTypeInfos.toArray(new TypeInfo[0]);
+  }
+
+  public TypeInfo getTypeInfo(int columnNum) throws HiveException {
+    if (initialTypeInfos == null) {
+      throw new HiveException("initialTypeInfos array is null in contextName " + contextName);
+    }
+    final int initialSize = initialTypeInfos.size();
+    if (columnNum < initialSize) {
+      return initialTypeInfos.get(columnNum);
+    } else {
+      String typeName = ocm.getScratchTypeName(columnNum);
+
+      // Replace unparsable synonyms.
+      typeName = VectorizationContext.mapTypeNameSynonyms(typeName);
+
+      // Make CHAR and VARCHAR type info parsable.
+      if (typeName.equals("char")) {
+        typeName = "char(" + HiveChar.MAX_CHAR_LENGTH + ")";
+      } else if (typeName.equals("varchar")) {
+        typeName = "varchar(" + HiveVarchar.MAX_VARCHAR_LENGTH + ")";
+      }
+
+      TypeInfo typeInfo =
+          TypeInfoUtils.getTypeInfoFromTypeString(typeName);
+      return typeInfo;
+    }
+  }
+
+  public DataTypePhysicalVariation getDataTypePhysicalVariation(int columnNum) throws HiveException {
+    if (initialDataTypePhysicalVariations == null) {
+      return null;
+    }
+    if (columnNum < initialDataTypePhysicalVariations.size()) {
+      return initialDataTypePhysicalVariations.get(columnNum);
+    }
+    return ocm.getDataTypePhysicalVariation(columnNum);
+  }
+
+  public TypeInfo[] getAllTypeInfos() throws HiveException {
+    final int size = initialTypeInfos.size() + ocm.outputColCount;
+
+    TypeInfo[] result = new TypeInfo[size];
+    for (int i = 0; i < size; i++) {
+      result[i] = getTypeInfo(i);
+    }
+    return result;
+  }
+
+  public static final Pattern decimalTypePattern = Pattern.compile("decimal.*",
+      Pattern.CASE_INSENSITIVE);
+
+  public static final Pattern charTypePattern = Pattern.compile("char.*",
+	      Pattern.CASE_INSENSITIVE);
+
+  public static final Pattern varcharTypePattern = Pattern.compile("varchar.*",
+	      Pattern.CASE_INSENSITIVE);
+
+  public static final Pattern charVarcharTypePattern = Pattern.compile("char.*|varchar.*",
+      Pattern.CASE_INSENSITIVE);
+
+  public static final Pattern structTypePattern = Pattern.compile("struct.*",
+      Pattern.CASE_INSENSITIVE);
+
+  public static final Pattern listTypePattern = Pattern.compile("array.*",
+      Pattern.CASE_INSENSITIVE);
+
+  public static final Pattern mapTypePattern = Pattern.compile("map.*",
+      Pattern.CASE_INSENSITIVE);
+
+  //Map column number to type (this is always non-null for a useful vec context)
+  private OutputColumnManager ocm;
+
+  // Set of UDF classes for type casting data types in row-mode.
+  private static Set<Class<?>> castExpressionUdfs = new HashSet<Class<?>>();
+  static {
+    castExpressionUdfs.add(GenericUDFToString.class);
+    castExpressionUdfs.add(GenericUDFToDecimal.class);
+    castExpressionUdfs.add(GenericUDFToBinary.class);
+    castExpressionUdfs.add(GenericUDFToDate.class);
+    castExpressionUdfs.add(GenericUDFToUnixTimeStamp.class);
+    castExpressionUdfs.add(GenericUDFToUtcTimestamp.class);
+    castExpressionUdfs.add(GenericUDFToChar.class);
+    castExpressionUdfs.add(GenericUDFToVarchar.class);
+    castExpressionUdfs.add(GenericUDFTimestamp.class);
+    castExpressionUdfs.add(GenericUDFToIntervalYearMonth.class);
+    castExpressionUdfs.add(GenericUDFToIntervalDayTime.class);
+    castExpressionUdfs.add(UDFToByte.class);
+    castExpressionUdfs.add(UDFToBoolean.class);
+    castExpressionUdfs.add(UDFToDouble.class);
+    castExpressionUdfs.add(UDFToFloat.class);
+    castExpressionUdfs.add(UDFToInteger.class);
+    castExpressionUdfs.add(UDFToLong.class);
+    castExpressionUdfs.add(UDFToShort.class);
+  }
+
+  // Set of GenericUDFs which require need implicit type casting of decimal parameters.
+  // Vectorization for mathmatical functions currently depends on decimal params automatically
+  // being converted to the return type (see getImplicitCastExpression()), which is not correct
+  // in the general case. This set restricts automatic type conversion to just these functions.
+  private static Set<Class<?>> udfsNeedingImplicitDecimalCast = new HashSet<Class<?>>();
+  static {
+    udfsNeedingImplicitDecimalCast.add(GenericUDFOPPlus.class);
+    udfsNeedingImplicitDecimalCast.add(GenericUDFOPMinus.class);
+    udfsNeedingImplicitDecimalCast.add(GenericUDFOPMultiply.class);
+    udfsNeedingImplicitDecimalCast.add(GenericUDFOPDivide.class);
+    udfsNeedingImplicitDecimalCast.add(GenericUDFOPMod.class);
+    udfsNeedingImplicitDecimalCast.add(GenericUDFRound.class);
+    udfsNeedingImplicitDecimalCast.add(GenericUDFBRound.class);
+    udfsNeedingImplicitDecimalCast.add(GenericUDFFloor.class);
+    udfsNeedingImplicitDecimalCast.add(GenericUDFCbrt.class);
+    udfsNeedingImplicitDecimalCast.add(GenericUDFCeil.class);
+    udfsNeedingImplicitDecimalCast.add(GenericUDFAbs.class);
+    udfsNeedingImplicitDecimalCast.add(GenericUDFPosMod.class);
+    udfsNeedingImplicitDecimalCast.add(GenericUDFPower.class);
+    udfsNeedingImplicitDecimalCast.add(GenericUDFFactorial.class);
+    udfsNeedingImplicitDecimalCast.add(GenericUDFOPPositive.class);
+    udfsNeedingImplicitDecimalCast.add(GenericUDFOPNegative.class);
+    udfsNeedingImplicitDecimalCast.add(GenericUDFCoalesce.class);
+    udfsNeedingImplicitDecimalCast.add(GenericUDFElt.class);
+    udfsNeedingImplicitDecimalCast.add(GenericUDFGreatest.class);
+    udfsNeedingImplicitDecimalCast.add(GenericUDFLeast.class);
+    udfsNeedingImplicitDecimalCast.add(GenericUDFIn.class);
+    udfsNeedingImplicitDecimalCast.add(GenericUDFOPEqual.class);
+    udfsNeedingImplicitDecimalCast.add(GenericUDFOPEqualNS.class);
+    udfsNeedingImplicitDecimalCast.add(GenericUDFOPNotEqual.class);
+    udfsNeedingImplicitDecimalCast.add(GenericUDFOPLessThan.class);
+    udfsNeedingImplicitDecimalCast.add(GenericUDFOPEqualOrLessThan.class);
+    udfsNeedingImplicitDecimalCast.add(GenericUDFOPGreaterThan.class);
+    udfsNeedingImplicitDecimalCast.add(GenericUDFOPEqualOrGreaterThan.class);
+    udfsNeedingImplicitDecimalCast.add(GenericUDFBetween.class);
+    udfsNeedingImplicitDecimalCast.add(UDFSqrt.class);
+    udfsNeedingImplicitDecimalCast.add(UDFRand.class);
+    udfsNeedingImplicitDecimalCast.add(UDFLn.class);
+    udfsNeedingImplicitDecimalCast.add(UDFLog2.class);
+    udfsNeedingImplicitDecimalCast.add(UDFSin.class);
+    udfsNeedingImplicitDecimalCast.add(UDFAsin.class);
+    udfsNeedingImplicitDecimalCast.add(UDFCos.class);
+    udfsNeedingImplicitDecimalCast.add(UDFAcos.class);
+    udfsNeedingImplicitDecimalCast.add(UDFLog10.class);
+    udfsNeedingImplicitDecimalCast.add(UDFLog.class);
+    udfsNeedingImplicitDecimalCast.add(UDFExp.class);
+    udfsNeedingImplicitDecimalCast.add(UDFDegrees.class);
+    udfsNeedingImplicitDecimalCast.add(UDFRadians.class);
+    udfsNeedingImplicitDecimalCast.add(UDFAtan.class);
+    udfsNeedingImplicitDecimalCast.add(UDFTan.class);
+    udfsNeedingImplicitDecimalCast.add(UDFOPLongDivide.class);
+  }
+
+  protected boolean needsImplicitCastForDecimal(GenericUDF udf) {
+    Class<?> udfClass = udf.getClass();
+    if (udf instanceof GenericUDFBridge) {
+      udfClass = ((GenericUDFBridge) udf).getUdfClass();
+    }
+    return udfsNeedingImplicitDecimalCast.contains(udfClass);
+  }
+
+  public int getInputColumnIndex(String name) throws HiveException {
+    if (name == null) {
+      throw new HiveException("Null column name");
+    }
+    if (!projectionColumnMap.containsKey(name)) {
+      throw new HiveException(String.format("The column %s is not in the vectorization context column map %s.",
+                 name, projectionColumnMap.toString()));
+    }
+    final int projectedColumnNum = projectionColumnMap.get(name);
+    if (projectedColumnNum < 0) {
+      throw new HiveException("Negative projected column number");
+    }
+    return projectedColumnNum;
+  }
+
+  protected int getInputColumnIndex(ExprNodeColumnDesc colExpr) throws HiveException {
+    // Call the regular method since it does error checking.
+    return getInputColumnIndex(colExpr.getColumn());
+  }
+
+  private static class OutputColumnManager {
+    private final int initialOutputCol;
+    private int outputColCount = 0;
+    private boolean reuseScratchColumns = true;
+
+    protected OutputColumnManager(int initialOutputCol) {
+      this.initialOutputCol = initialOutputCol;
+    }
+
+    //The complete list of output columns. These should be added to the
+    //Vectorized row batch for processing. The index in the row batch is
+    //equal to the index in this array plus initialOutputCol.
+    //Start with size 100 and double when needed.
+    private String[] scratchVectorTypeNames = new String[100];
+    private DataTypePhysicalVariation[] scratchDataTypePhysicalVariations =
+        new DataTypePhysicalVariation[100];
+
+    private final Set<Integer> usedOutputColumns = new HashSet<Integer>();
+
+    int allocateOutputColumn(TypeInfo typeInfo) throws HiveException {
+      return allocateOutputColumn(typeInfo, DataTypePhysicalVariation.NONE);
+    }
+
+    int allocateOutputColumn(TypeInfo typeInfo,
+        DataTypePhysicalVariation dataTypePhysicalVariation) throws HiveException {
+
+        if (initialOutputCol < 0) {
+          // This is a test calling.
+          return 0;
+        }
+
+        // CONCERN: We currently differentiate DECIMAL columns by their precision and scale...,
+        // which could lead to a lot of extra unnecessary scratch columns.
+        String vectorTypeName = getScratchName(typeInfo);
+        int relativeCol = allocateOutputColumnInternal(vectorTypeName, dataTypePhysicalVariation);
+        return initialOutputCol + relativeCol;
+      }
+
+    private int allocateOutputColumnInternal(String columnType, DataTypePhysicalVariation dataTypePhysicalVariation) {
+      for (int i = 0; i < outputColCount; i++) {
+
+        // Re-use an existing, available column of the same required type.
+        if (usedOutputColumns.contains(i) ||
+            !(scratchVectorTypeNames[i].equalsIgnoreCase(columnType) &&
+              scratchDataTypePhysicalVariations[i] == dataTypePhysicalVariation)) {
+          continue;
+        }
+        //Use i
+        usedOutputColumns.add(i);
+        return i;
+      }
+      //Out of allocated columns
+      if (outputColCount < scratchVectorTypeNames.length) {
+        int newIndex = outputColCount;
+        scratchVectorTypeNames[outputColCount] = columnType;
+        scratchDataTypePhysicalVariations[outputColCount++] = dataTypePhysicalVariation;
+        usedOutputColumns.add(newIndex);
+        return newIndex;
+      } else {
+        //Expand the array
+        scratchVectorTypeNames = Arrays.copyOf(scratchVectorTypeNames, 2*outputColCount);
+        scratchDataTypePhysicalVariations = Arrays.copyOf(scratchDataTypePhysicalVariations, 2*outputColCount);
+        int newIndex = outputColCount;
+        scratchVectorTypeNames[outputColCount] = columnType;
+        scratchDataTypePhysicalVariations[outputColCount++] = dataTypePhysicalVariation;
+        usedOutputColumns.add(newIndex);
+        return newIndex;
+      }
+    }
+
+    void freeOutputColumn(int index) {
+      if (initialOutputCol < 0 || reuseScratchColumns == false) {
+        // This is a test
+        return;
+      }
+      int colIndex = index-initialOutputCol;
+      if (colIndex >= 0) {
+        usedOutputColumns.remove(index-initialOutputCol);
+      }
+    }
+
+    public int[] currentScratchColumns() {
+      TreeSet<Integer> treeSet = new TreeSet<Integer>();
+      for (Integer col : usedOutputColumns) {
+        treeSet.add(initialOutputCol + col);
+      }
+      return ArrayUtils.toPrimitive(treeSet.toArray(new Integer[0]));
+    }
+
+    public String getScratchTypeName(int columnNum) {
+      return scratchVectorTypeNames[columnNum - initialOutputCol];
+    }
+
+    public DataTypePhysicalVariation getDataTypePhysicalVariation(int columnNum) {
+      if (scratchDataTypePhysicalVariations == null) {
+        return null;
+      }
+      return scratchDataTypePhysicalVariations[columnNum - initialOutputCol];
+    }
+
+    // Allow debugging by disabling column reuse (input cols are never reused by design, only
+    // scratch cols are)
+    public void setReuseColumns(boolean reuseColumns) {
+      this.reuseScratchColumns = reuseColumns;
+    }
+  }
+
+  public int allocateScratchColumn(TypeInfo typeInfo) throws HiveException {
+    return ocm.allocateOutputColumn(typeInfo);
+  }
+
+  public int[] currentScratchColumns() {
+    return ocm.currentScratchColumns();
+  }
+
+  private VectorExpression getFilterOnBooleanColumnExpression(ExprNodeColumnDesc exprDesc,
+      int columnNum) throws HiveException {
+    VectorExpression expr = null;
+
+    // Evaluate the column as a boolean, converting if necessary.
+    TypeInfo typeInfo = exprDesc.getTypeInfo();
+    if (typeInfo.getCategory() == Category.PRIMITIVE &&
+        ((PrimitiveTypeInfo) typeInfo).getPrimitiveCategory() == PrimitiveCategory.BOOLEAN) {
+      expr = new SelectColumnIsTrue(columnNum);
+
+      expr.setInputTypeInfos(typeInfo);
+      expr.setInputDataTypePhysicalVariations(DataTypePhysicalVariation.NONE);
+
+    } else {
+      // Ok, we need to convert.
+      ArrayList<ExprNodeDesc> exprAsList = new ArrayList<ExprNodeDesc>(1);
+      exprAsList.add(exprDesc);
+
+      // First try our cast method that will handle a few special cases.
+      VectorExpression castToBooleanExpr = getCastToBoolean(exprAsList);
+      if (castToBooleanExpr == null) {
+
+        // Ok, try the UDF.
+        castToBooleanExpr = getVectorExpressionForUdf(null, UDFToBoolean.class, exprAsList,
+            VectorExpressionDescriptor.Mode.PROJECTION, TypeInfoFactory.booleanTypeInfo);
+        if (castToBooleanExpr == null) {
+          throw new HiveException("Cannot vectorize converting expression " +
+              exprDesc.getExprString() + " to boolean");
+        }
+      }
+
+      final int outputColumnNum = castToBooleanExpr.getOutputColumnNum();
+
+      expr = new SelectColumnIsTrue(outputColumnNum);
+
+      expr.setChildExpressions(new VectorExpression[] {castToBooleanExpr});
+
+      expr.setInputTypeInfos(castToBooleanExpr.getOutputTypeInfo());
+      expr.setInputDataTypePhysicalVariations(DataTypePhysicalVariation.NONE);
+    }
+    return expr;
+  }
+
+  private VectorExpression getColumnVectorExpression(ExprNodeColumnDesc exprDesc,
+      VectorExpressionDescriptor.Mode mode) throws HiveException {
+    int columnNum = getInputColumnIndex(exprDesc.getColumn());
+    VectorExpression expr = null;
+    switch (mode) {
+    case FILTER:
+      expr = getFilterOnBooleanColumnExpression(exprDesc, columnNum);
+      break;
+    case PROJECTION:
+      {
+        expr = new IdentityExpression(columnNum);
+
+        TypeInfo identityTypeInfo = exprDesc.getTypeInfo();
+        DataTypePhysicalVariation identityDataTypePhysicalVariation =
+            getDataTypePhysicalVariation(columnNum);
+
+        expr.setInputTypeInfos(identityTypeInfo);
+        expr.setInputDataTypePhysicalVariations(identityDataTypePhysicalVariation);
+
+        expr.setOutputTypeInfo(identityTypeInfo);
+        expr.setOutputDataTypePhysicalVariation(identityDataTypePhysicalVariation);
+      }
+      break;
+    default:
+      throw new RuntimeException("Unexpected mode " + mode);
+    }
+    return expr;
+  }
+
+  public VectorExpression[] getVectorExpressionsUpConvertDecimal64(List<ExprNodeDesc> exprNodes)
+      throws HiveException {
+    VectorExpression[] vecExprs =
+        getVectorExpressions(exprNodes, VectorExpressionDescriptor.Mode.PROJECTION);
+    final int size = vecExprs.length;
+    for (int i = 0; i < size; i++) {
+      VectorExpression vecExpr = vecExprs[i];
+      if (vecExpr.getOutputColumnVectorType() == ColumnVector.Type.DECIMAL_64) {
+        vecExprs[i] = wrapWithDecimal64ToDecimalConversion(vecExpr);
+      }
+    }
+    return vecExprs;
+  }
+
+  public VectorExpression[] getVectorExpressions(List<ExprNodeDesc> exprNodes) throws HiveException {
+    return getVectorExpressions(exprNodes, VectorExpressionDescriptor.Mode.PROJECTION);
+  }
+
+  public VectorExpression[] getVectorExpressions(List<ExprNodeDesc> exprNodes, VectorExpressionDescriptor.Mode mode)
+    throws HiveException {
+
+    int i = 0;
+    if (null == exprNodes) {
+      return new VectorExpression[0];
+    }
+    VectorExpression[] ret = new VectorExpression[exprNodes.size()];
+    for (ExprNodeDesc e : exprNodes) {
+      ret[i++] = getVectorExpression(e, mode);
+    }
+    return ret;
+  }
+
+  public VectorExpression getVectorExpression(ExprNodeDesc exprDesc) throws HiveException {
+    return getVectorExpression(exprDesc, VectorExpressionDescriptor.Mode.PROJECTION);
+  }
+
+  /**
+   * Returns a vector expression for a given expression
+   * description.
+   * @param exprDesc, Expression description
+   * @param mode
+   * @return {@link VectorExpression}
+   * @throws HiveException
+   */
+  public VectorExpression getVectorExpression(ExprNodeDesc exprDesc, VectorExpressionDescriptor.Mode mode) throws HiveException {
+    VectorExpression ve = null;
+    if (exprDesc instanceof ExprNodeColumnDesc) {
+      ve = getColumnVectorExpression((ExprNodeColumnDesc) exprDesc, mode);
+    } else if (exprDesc instanceof ExprNodeGenericFuncDesc) {
+      ExprNodeGenericFuncDesc expr = (ExprNodeGenericFuncDesc) exprDesc;
+      // push not through between...
+      if ("not".equals(expr.getFuncText())) {
+        if (expr.getChildren() != null && expr.getChildren().size() == 1) {
+          ExprNodeDesc child = expr.getChildren().get(0);
+          if (child instanceof ExprNodeGenericFuncDesc) {
+            ExprNodeGenericFuncDesc childExpr = (ExprNodeGenericFuncDesc) child;
+            if ("between".equals(childExpr.getFuncText())) {
+              ExprNodeConstantDesc flag = (ExprNodeConstantDesc) childExpr.getChildren().get(0);
+              List<ExprNodeDesc> newChildren = new ArrayList<>();
+              if (Boolean.TRUE.equals(flag.getValue())) {
+                newChildren.add(new ExprNodeConstantDesc(Boolean.FALSE));
+              } else {
+                newChildren.add(new ExprNodeConstantDesc(Boolean.TRUE));
+              }
+              newChildren
+                  .addAll(childExpr.getChildren().subList(1, childExpr.getChildren().size()));
+              expr.setTypeInfo(childExpr.getTypeInfo());
+              expr.setGenericUDF(childExpr.getGenericUDF());
+              expr.setChildren(newChildren);
+            }
+          }
+        }
+      }
+      // Add cast expression if needed. Child expressions of a udf may return different data types
+      // and that would require converting their data types to evaluate the udf.
+      // For example decimal column added to an integer column would require integer column to be
+      // cast to decimal.
+      // Note: this is a no-op for custom UDFs
+      List<ExprNodeDesc> childExpressions = getChildExpressionsWithImplicitCast(expr.getGenericUDF(),
+          exprDesc.getChildren(), exprDesc.getTypeInfo());
+
+      // Are we forcing the usage of VectorUDFAdaptor for test purposes?
+      if (!testVectorAdaptorOverride) {
+        ve = getGenericUdfVectorExpression(expr.getGenericUDF(),
+            childExpressions, mode, exprDesc.getTypeInfo());
+      }
+      if (ve == null) {
+        // Ok, no vectorized class available.  No problem -- try to use the VectorUDFAdaptor
+        // when configured.
+        //
+        // NOTE: We assume if hiveVectorAdaptorUsageMode has not been set it because we are
+        // executing a test that didn't create a HiveConf, etc.  No usage of VectorUDFAdaptor in
+        // that case.
+        if (hiveVectorAdaptorUsageMode != null) {
+          switch (hiveVectorAdaptorUsageMode) {
+          case NONE:
+            // No VectorUDFAdaptor usage.
+            throw new HiveException(
+                "Could not vectorize expression (mode = " + mode.name() + "): " + exprDesc.toString()
+                  + " because hive.vectorized.adaptor.usage.mode=none");
+          case CHOSEN:
+            if (isNonVectorizedPathUDF(expr, mode)) {
+              ve = getCustomUDFExpression(expr, mode);
+            } else {
+              throw new HiveException(
+                  "Could not vectorize expression (mode = " + mode.name() + "): " + exprDesc.toString()
+                    + " because hive.vectorized.adaptor.usage.mode=chosen"
+                    + " and the UDF wasn't one of the chosen ones");
+            }
+            break;
+          case ALL:
+            if (LOG.isDebugEnabled()) {
+              LOG.debug("We will try to use the VectorUDFAdaptor for " + exprDesc.toString()
+                  + " because hive.vectorized.adaptor.usage.mode=all");
+            }
+            ve = getCustomUDFExpression(expr, mode);
+            break;
+          default:
+            throw new RuntimeException("Unknown hive vector adaptor usage mode " +
+              hiveVectorAdaptorUsageMode.name());
+          }
+          if (ve == null) {
+            throw new HiveException(
+                "Unable vectorize expression (mode = " + mode.name() + "): " + exprDesc.toString()
+                  + " even for the VectorUDFAdaptor");
+          }
+        }
+      }
+    } else if (exprDesc instanceof ExprNodeConstantDesc) {
+      ve = getConstantVectorExpression(((ExprNodeConstantDesc) exprDesc).getValue(), exprDesc.getTypeInfo(),
+          mode);
+    } else if (exprDesc instanceof ExprNodeDynamicValueDesc) {
+      ve = getDynamicValueVectorExpression((ExprNodeDynamicValueDesc) exprDesc, mode);
+    } else if (exprDesc instanceof ExprNodeFieldDesc) {
+      // Get the GenericUDFStructField to process the field of Struct type
+      ve = getGenericUDFStructField((ExprNodeFieldDesc)exprDesc,
+          mode, exprDesc.getTypeInfo());
+    }
+    if (ve == null) {
+      throw new HiveException(
+          "Could not vectorize expression (mode = " + mode.name() + "): " + exprDesc.toString());
+    }
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Input Expression = " + exprDesc.toString()
+          + ", Vectorized Expression = " + ve.toString());
+    }
+
+    return ve;
+  }
+
+  private VectorExpression getGenericUDFStructField(ExprNodeFieldDesc exprNodeFieldDesc,
+      VectorExpressionDescriptor.Mode mode, TypeInfo returnType) throws HiveException {
+    // set the arguments for GenericUDFStructField
+    List<ExprNodeDesc> children = new ArrayList<>(2);
+    children.add(exprNodeFieldDesc.getDesc());
+    children.add(new ExprNodeConstantDesc(getStructFieldIndex(exprNodeFieldDesc)));
+
+    return getVectorExpressionForUdf(null, GenericUDFStructField.class, children, mode, returnType);
+  }
+
+  /**
+   * The field of Struct is stored in StructColumnVector.fields[index].
+   * Check the StructTypeInfo.getAllStructFieldNames() and compare to the field name, get the index.
+   */
+  private int getStructFieldIndex(ExprNodeFieldDesc exprNodeFieldDesc) throws HiveException {
+    ExprNodeDesc structNodeDesc = exprNodeFieldDesc.getDesc();
+    String fieldName = exprNodeFieldDesc.getFieldName();
+    StructTypeInfo structTypeInfo = (StructTypeInfo) structNodeDesc.getTypeInfo();
+    int index = 0;
+    boolean isFieldExist = false;
+    for (String fn : structTypeInfo.getAllStructFieldNames()) {
+      if (fieldName.equals(fn)) {
+        isFieldExist = true;
+        break;
+      }
+      index++;
+    }
+    if (isFieldExist) {
+      return index;
+    } else {
+      throw new HiveException("Could not vectorize expression:" + exprNodeFieldDesc.toString()
+          + ", the field " + fieldName + " doesn't exist.");
+    }
+  }
+
+  /**
+   * Given a udf and its children, return the common type to which the children's type should be
+   * cast.
+   */
+  private TypeInfo getCommonTypeForChildExpressions(GenericUDF genericUdf,
+      List<ExprNodeDesc> children, TypeInfo returnType) throws HiveException {
+    TypeInfo commonType;
+    if (genericUdf instanceof GenericUDFBaseCompare) {
+
+      // Apply comparison rules
+      TypeInfo tLeft = children.get(0).getTypeInfo();
+      TypeInfo tRight = children.get(1).getTypeInfo();
+      commonType = FunctionRegistry.getCommonClassForComparison(tLeft, tRight);
+      if (commonType == null) {
+        commonType = returnType;
+      }
+    } else if (genericUdf instanceof GenericUDFIn) {
+      TypeInfo colTi = children.get(0).getTypeInfo();
+      if (colTi.getCategory() != Category.PRIMITIVE) {
+        return colTi; // Handled later, only struct will be supported.
+      }
+      TypeInfo opTi = GenericUDFUtils.deriveInType(children);
+      if (opTi == null || opTi.getCategory() != Category.PRIMITIVE) {
+        throw new HiveException("Cannot vectorize IN() - common type is " + opTi);
+      }
+      if (((PrimitiveTypeInfo)colTi).getPrimitiveCategory() !=
+          ((PrimitiveTypeInfo)opTi).getPrimitiveCategory()) {
+        throw new HiveException("Cannot vectorize IN() - casting a column is not supported. "
+            + "Column type is " + colTi + " but the common type is " + opTi);
+      }
+      return colTi;
+    } else {
+      // The children type should be converted to return type
+      commonType = returnType;
+    }
+    return commonType;
+  }
+
+  /**
+   * Add a cast expression to the expression tree if needed. The output of child expressions of a given UDF might
+   * need a cast if their return type is different from the return type of the UDF.
+   *
+   * @param genericUDF The given UDF
+   * @param children Child expressions of the UDF that might require a cast.
+   * @param returnType The return type of the UDF.
+   * @return List of child expressions added with cast.
+   */
+  private List<ExprNodeDesc> getChildExpressionsWithImplicitCast(GenericUDF genericUDF,
+      List<ExprNodeDesc> children, TypeInfo returnType) throws HiveException {
+
+    if (isCustomUDF(genericUDF.getUdfName())) {
+      // no implicit casts possible
+      return children;
+    }
+
+    if (isExcludedFromCast(genericUDF)) {
+      // No implicit cast needed
+      return children;
+    }
+    if (children == null) {
+      return null;
+    }
+
+    TypeInfo commonType = getCommonTypeForChildExpressions(genericUDF, children, returnType);
+
+    if (commonType == null) {
+
+      // Couldn't determine common type, don't cast
+      return children;
+    }
+
+    List<ExprNodeDesc> childrenWithCasts = new ArrayList<ExprNodeDesc>();
+    boolean atleastOneCastNeeded = false;
+    if (genericUDF instanceof GenericUDFElt) {
+      int i = 0;
+      for (ExprNodeDesc child : children) {
+        TypeInfo castType = commonType;
+        if (i++ == 0) {
+          castType = isIntFamily(child.getTypeString()) ? child.getTypeInfo() : TypeInfoFactory.intTypeInfo;
+        }
+        ExprNodeDesc castExpression = getImplicitCastExpression(genericUDF, child, castType);
+        if (castExpression != null) {
+          atleastOneCastNeeded = true;
+          childrenWithCasts.add(castExpression);
+        } else {
+          childrenWithCasts.add(child);
+        }
+      }
+    } else {
+      for (ExprNodeDesc child : children) {
+        ExprNodeDesc castExpression = getImplicitCastExpression(genericUDF, child, commonType);
+        if (castExpression != null) {
+          atleastOneCastNeeded = true;
+          childrenWithCasts.add(castExpression);
+        } else {
+          childrenWithCasts.add(child);
+        }
+      }
+    }
+    if (atleastOneCastNeeded) {
+      return childrenWithCasts;
+    } else {
+      return children;
+    }
+  }
+
+  private boolean isExcludedFromCast(GenericUDF genericUDF) {
+    boolean ret = castExpressionUdfs.contains(genericUDF.getClass())
+        || (genericUDF instanceof GenericUDFRound) || (genericUDF instanceof GenericUDFBetween);
+
+    if (ret) {
+      return ret;
+    }
+
+    if (genericUDF instanceof GenericUDFBridge) {
+      Class<?> udfClass = ((GenericUDFBridge) genericUDF).getUdfClass();
+      return castExpressionUdfs.contains(udfClass)
+          || UDFSign.class.isAssignableFrom(udfClass);
+    }
+    return false;
+  }
+
+  /**
+   * Creates a DecimalTypeInfo object with appropriate precision and scale for the given
+   * inputTypeInfo.
+   */
+  private TypeInfo updatePrecision(TypeInfo inputTypeInfo, DecimalTypeInfo returnType) {
+    if (!(inputTypeInfo instanceof PrimitiveTypeInfo)) {
+      return returnType;
+    }
+    PrimitiveTypeInfo ptinfo = (PrimitiveTypeInfo) inputTypeInfo;
+    int precision = getPrecisionForType(ptinfo);
+    // TODO: precision and scale would be practically invalid for string conversion (38,38)
+    int scale = HiveDecimalUtils.getScaleForType(ptinfo);
+    return new DecimalTypeInfo(precision, scale);
+  }
+
+  /**
+   * The GenericUDFs might need their children output to be cast to the given castType.
+   * This method returns a cast expression that would achieve the required casting.
+   */
+  private ExprNodeDesc getImplicitCastExpression(GenericUDF udf, ExprNodeDesc child, TypeInfo castType)
+      throws HiveException {
+    TypeInfo inputTypeInfo = child.getTypeInfo();
+    String inputTypeString = inputTypeInfo.getTypeName();
+    String castTypeString = castType.getTypeName();
+
+    if (inputTypeString.equals(castTypeString)) {
+      // Nothing to be done
+      return null;
+    }
+    boolean inputTypeDecimal = false;
+    boolean castTypeDecimal = false;
+    if (decimalTypePattern.matcher(inputTypeString).matches()) {
+      inputTypeDecimal = true;
+    }
+    if (decimalTypePattern.matcher(castTypeString).matches()) {
+      castTypeDecimal = true;
+    }
+
+    if (castTypeDecimal && !inputTypeDecimal) {
+      if (needsImplicitCastForDecimal(udf)) {
+        // Cast the input to decimal
+        // If castType is decimal, try not to lose precision for numeric types.
+        castType = updatePrecision(inputTypeInfo, (DecimalTypeInfo) castType);
+        GenericUDFToDecimal castToDecimalUDF = new GenericUDFToDecimal();
+        castToDecimalUDF.setTypeInfo(castType);
+        List<ExprNodeDesc> children = new ArrayList<ExprNodeDesc>();
+        children.add(child);
+        ExprNodeDesc desc = new ExprNodeGenericFuncDesc(castType, castToDecimalUDF, children);
+        return desc;
+      }
+    } else if (!castTypeDecimal && inputTypeDecimal) {
+      if (needsImplicitCastForDecimal(udf)) {
+        // Cast decimal input to returnType
+        GenericUDF genericUdf = getGenericUDFForCast(castType);
+        List<ExprNodeDesc> children = new ArrayList<ExprNodeDesc>();
+        children.add(child);
+        ExprNodeDesc desc = new ExprNodeGenericFuncDesc(castType, genericUdf, children);
+        return desc;
+      }
+    } else {
+
+      // Casts to exact types including long to double etc. are needed in some special cases.
+      if (udf instanceof GenericUDFCoalesce || udf instanceof GenericUDFNvl
+          || udf instanceof GenericUDFElt) {
+        GenericUDF genericUdf = getGenericUDFForCast(castType);
+        List<ExprNodeDesc> children = new ArrayList<ExprNodeDesc>();
+        children.add(child);
+        ExprNodeDesc desc = new ExprNodeGenericFuncDesc(castType, genericUdf, children);
+        return desc;
+      }
+    }
+    return null;
+  }
+
+  private int getPrecisionForType(PrimitiveTypeInfo typeInfo) {
+    if (isFloatFamily(typeInfo.getTypeName())) {
+      return HiveDecimal.MAX_PRECISION;
+    }
+    return HiveDecimalUtils.getPrecisionForType(typeInfo);
+  }
+
+  public static GenericUDF getGenericUDFForCast(TypeInfo castType) throws HiveException {
+    UDF udfClass = null;
+    GenericUDF genericUdf = null;
+    switch (((PrimitiveTypeInfo) castType).getPrimitiveCategory()) {
+      case BYTE:
+        udfClass = new UDFToByte();
+        break;
+      case SHORT:
+        udfClass = new UDFToShort();
+        break;
+      case INT:
+        udfClass = new UDFToInteger();
+        break;
+      case LONG:
+        udfClass = new UDFToLong();
+        break;
+      case FLOAT:
+        udfClass = new UDFToFloat();
+        break;
+      case DOUBLE:
+        udfClass = new UDFToDouble();
+        break;
+      case STRING:
+        genericUdf = new GenericUDFToString();
+        break;
+      case CHAR:
+        genericUdf = new GenericUDFToChar();
+        break;
+      case VARCHAR:
+        genericUdf = new GenericUDFToVarchar();
+        break;
+      case BOOLEAN:
+        udfClass = new UDFToBoolean();
+        break;
+      case DATE:
+        genericUdf = new GenericUDFToDate();
+        break;
+      case TIMESTAMP:
+        genericUdf = new GenericUDFTimestamp();
+        break;
+      case INTERVAL_YEAR_MONTH:
+        genericUdf = new GenericUDFToIntervalYearMonth();
+        break;
+      case INTERVAL_DAY_TIME:
+        genericUdf = new GenericUDFToIntervalDayTime();
+        break;
+      case BINARY:
+        genericUdf = new GenericUDFToBinary();
+        break;
+      case DECIMAL:
+        genericUdf = new GenericUDFToDecimal();
+        break;
+      case VOID:
+      case UNKNOWN:
+        // fall-through to throw exception, its not expected for execution to reach here.
+        break;
+    }
+    if (genericUdf == null) {
+      if (udfClass == null) {
+        throw new HiveException("Could not add implicit cast for type "+castType.getTypeName());
+      }
+      GenericUDFBridge genericUDFBridge = new GenericUDFBridge();
+      genericUDFBridge.setUdfClassName(udfClass.getClass().getName());
+      genericUDFBridge.setUdfName(udfClass.getClass().getSimpleName());
+      genericUdf = genericUDFBridge;
+    }
+    if (genericUdf instanceof SettableUDF) {
+      ((SettableUDF) genericUdf).setTypeInfo(castType);
+    }
+    return genericUdf;
+  }
+
+  /* Return true if this is one of a small set of functions for which
+   * it is significantly easier to use the old code path in vectorized
+   * mode instead of implementing a new, optimized VectorExpression.
+   *
+   * Depending on performance requirements and frequency of use, these
+   * may be implemented in the future with an optimized VectorExpression.
+   */
+  public static boolean isNonVectorizedPathUDF(ExprNodeGenericFuncDesc expr,
+      VectorExpressionDescriptor.Mode mode) {
+    GenericUDF gudf = expr.getGenericUDF();
+    if (gudf instanceof GenericUDFBridge) {
+      GenericUDFBridge bridge = (GenericUDFBridge) gudf;
+      Class<? extends UDF> udfClass = bridge.getUdfClass();
+      if (udfClass.equals(UDFHex.class)
+          || udfClass.equals(UDFRegExpExtract.class)
+          || udfClass.equals(UDFRegExpReplace.class)
+          || udfClass.equals(UDFConv.class)
+          || udfClass.equals(UDFFromUnixTime.class) && isIntFamily(arg0Type(expr))
+          || isCastToIntFamily(udfClass) && isStringFamily(arg0Type(expr))
+          || isCastToFloatFamily(udfClass) && isStringFamily(arg0Type(expr))) {
+        return true;
+      }
+    } else if ((gudf instanceof GenericUDFTimestamp && isStringFamily(arg0Type(expr)))
+
+            /* GenericUDFCase and GenericUDFWhen are implemented with the UDF Adaptor because
+             * of their complexity and generality. In the future, variations of these
+             * can be optimized to run faster for the vectorized code path. For example,
+             * CASE col WHEN 1 then "one" WHEN 2 THEN "two" ELSE "other" END
+             * is an example of a GenericUDFCase that has all constant arguments
+             * except for the first argument. This is probably a common case and a
+             * good candidate for a fast, special-purpose VectorExpression. Then
+             * the UDF Adaptor code path could be used as a catch-all for
+             * non-optimized general cases.
+             */
+            || gudf instanceof GenericUDFCase
+            || gudf instanceof GenericUDFWhen) {
+      return true;
+    } else if ((gudf instanceof GenericUDFToString
+                   || gudf instanceof GenericUDFToChar
+                   || gudf instanceof GenericUDFToVarchar) &&
+               (arg0Type(expr).equals("timestamp")
+                   || arg0Type(expr).equals("double")
+                   || arg0Type(expr).equals("float"))) {
+      return true;
+    } else if (gudf instanceof GenericUDFBetween && (mode == VectorExpressionDescriptor.Mode.PROJECTION)) {
+      // between has 4 args here, but can be vectorized like this
+      return true;
+    }
+    return false;
+  }
+
+  public static boolean isCastToIntFamily(Class<? extends UDF> udfClass) {
+    return udfClass.equals(UDFToByte.class)
+        || udfClass.equals(UDFToShort.class)
+        || udfClass.equals(UDFToInteger.class)
+        || udfClass.equals(UDFToLong.class);
+
+    // Boolean is purposely excluded.
+  }
+
+  public static boolean isCastToFloatFamily(Class<? extends UDF> udfClass) {
+    return udfClass.equals(UDFToDouble.class)
+        || udfClass.equals(UDFToFloat.class);
+  }
+
+  // Return the type string of the first argument (argument 0).
+  public static String arg0Type(ExprNodeGenericFuncDesc expr) {
+    String type = expr.getChildren().get(0).getTypeString();
+    return type;
+  }
+
+  // Return true if this is a custom UDF or custom GenericUDF.
+  // This two functions are for use only in the planner. It will fail in a task.
+  public static boolean isCustomUDF(ExprNodeGenericFuncDesc expr) {
+    return isCustomUDF(expr.getFuncText());
+  }
+
+  private static boolean isCustomUDF(String udfName) {
+    if (udfName == null) {
+      return false;
+    }
+    FunctionInfo funcInfo;
+    try {
+      funcInfo = FunctionRegistry.getFunctionInfo(udfName);
+    } catch (SemanticException e) {
+      LOG.warn("Failed to load " + udfName, e);
+      funcInfo = null;
+    }
+    if (funcInfo == null) {
+      return false;
+    }
+    boolean isNativeFunc = funcInfo.isNative();
+    return !isNativeFunc;
+  }
+
+  /**
+   * Handles only the special cases of cast/+ve/-ve operator on a constant.
+   * @param exprDesc
+   * @return The same expression if no evaluation done, else return the constant
+   *         expression.
+   * @throws HiveException
+   */
+  ExprNodeDesc evaluateCastOnConstants(ExprNodeDesc exprDesc) throws HiveException {
+    if (!(exprDesc instanceof ExprNodeGenericFuncDesc)) {
+      return exprDesc;
+    }
+
+    if (exprDesc.getChildren() == null || (exprDesc.getChildren().size() != 1) ) {
+      return exprDesc;
+    }
+
+    ExprNodeConstantDesc foldedChild = null;
+    if (!( exprDesc.getChildren().get(0) instanceof ExprNodeConstantDesc)) {
+
+      // try recursive folding
+      ExprNodeDesc expr = evaluateCastOnConstants(exprDesc.getChildren().get(0));
+      if (expr instanceof ExprNodeConstantDesc) {
+        foldedChild = (ExprNodeConstantDesc) expr;
+      }
+    } else {
+      foldedChild = (ExprNodeConstantDesc) exprDesc.getChildren().get(0);
+    }
+
+    if (foldedChild == null) {
+      return exprDesc;
+    }
+
+    ObjectInspector childoi = foldedChild.getWritableObjectInspector();
+    GenericUDF gudf = ((ExprNodeGenericFuncDesc) exprDesc).getGenericUDF();
+
+    // Only evaluate +ve/-ve or cast on constant or recursive casting.
+    if (gudf instanceof GenericUDFOPNegative || gudf instanceof GenericUDFOPPositive ||
+        castExpressionUdfs.contains(gudf.getClass())
+        || ((gudf instanceof GenericUDFBridge)
+            && castExpressionUdfs.contains(((GenericUDFBridge) gudf).getUdfClass()))) {
+      ExprNodeEvaluator<?> evaluator = ExprNodeEvaluatorFactory.get(exprDesc);
+      ObjectInspector output = evaluator.initialize(childoi);
+      Object constant = evaluator.evaluate(null);
+      Object java = ObjectInspectorUtils.copyToStandardJavaObject(constant, output);
+      return new ExprNodeConstantDesc(exprDesc.getTypeInfo(), java);
+    }
+
+    return exprDesc;
+  }
+
+  /* For cast on constant operator in all members of the input list and return new list
+   * containing results.
+   */
+  private List<ExprNodeDesc> evaluateCastOnConstants(List<ExprNodeDesc> childExpr)
+      throws HiveException {
+    List<ExprNodeDesc> evaluatedChildren = new ArrayList<ExprNodeDesc>();
+    if (childExpr != null) {
+        for (ExprNodeDesc expr : childExpr) {
+          expr = this.evaluateCastOnConstants(expr);
+          evaluatedChildren.add(expr);
+        }
+    }
+    return evaluatedChildren;
+  }
+
+  private VectorExpression getConstantVectorExpression(Object constantValue, TypeInfo typeInfo,
+      VectorExpressionDescriptor.Mode mode) throws HiveException {
+    String typeName =  typeInfo.getTypeName();
+    VectorExpressionDescriptor.ArgumentType vectorArgType =
+        VectorExpressionDescriptor.ArgumentType.fromHiveTypeName(typeName);
+    if (vectorArgType == VectorExpressionDescriptor.ArgumentType.NONE) {
+      throw new HiveException("No vector argument type for type name " + typeName);
+    }
+    int outCol = -1;
+    if (mode == VectorExpressionDescriptor.Mode.PROJECTION) {
+      outCol = ocm.allocateOutputColumn(typeInfo);
+    }
+    if (constantValue == null) {
+      return new ConstantVectorExpression(outCol, typeInfo, true);
+    }
+
+    // Boolean is special case.
+    if (typeName.equalsIgnoreCase("boolean")) {
+      if (mode == VectorExpressionDescriptor.Mode.FILTER) {
+        if (((Boolean) constantValue).booleanValue()) {
+          return new FilterConstantBooleanVectorExpression(1);
+        } else {
+          return new FilterConstantBooleanVectorExpression(0);
+        }
+      } else {
+        if (((Boolean) constantValue).booleanValue()) {
+          return new ConstantVectorExpression(outCol, 1, typeInfo);
+        } else {
+          return new ConstantVectorExpression(outCol, 0, typeInfo);
+        }
+      }
+    }
+
+    switch (vectorArgType) {
+    case INT_FAMILY:
+      return new ConstantVectorExpression(outCol, ((Number) constantValue).longValue(), typeInfo);
+    case DATE:
+      return new ConstantVectorExpression(outCol, DateWritableV2.dateToDays((Date) constantValue), typeInfo);
+    case TIMESTAMP:
+      return new ConstantVectorExpression(outCol,
+          ((org.apache.hadoop.hive.common.type.Timestamp) constantValue).toSqlTimestamp(), typeInfo);
+    case INTERVAL_YEAR_MONTH:
+      return new ConstantVectorExpression(outCol,
+          ((HiveIntervalYearMonth) constantValue).getTotalMonths(), typeInfo);
+    case INTERVAL_DAY_TIME:
+      return new ConstantVectorExpression(outCol, (HiveIntervalDayTime) constantValue, typeInfo);
+    case FLOAT_FAMILY:
+      return new ConstantVectorExpression(outCol, ((Number) constantValue).doubleValue(), typeInfo);
+    case DECIMAL:
+      return new ConstantVectorExpression(outCol, (HiveDecimal) constantValue, typeInfo);
+    case STRING:
+      return new ConstantVectorExpression(outCol, ((String) constantValue).getBytes(), typeInfo);
+    case CHAR:
+      return new ConstantVectorExpression(outCol, ((HiveChar) constantValue), typeInfo);
+    case VARCHAR:
+      return new ConstantVectorExpression(outCol, ((HiveVarchar) constantValue), typeInfo);
+    default:
+      throw new HiveException("Unsupported constant type: " + typeName + ", object class " + constantValue.getClass().getSimpleName());
+    }
+  }
+
+  private VectorExpression getDynamicValueVectorExpression(ExprNodeDynamicValueDesc dynamicValueExpr,
+      VectorExpressionDescriptor.Mode mode) throws HiveException {
+    String typeName =  dynamicValueExpr.getTypeInfo().getTypeName();
+    VectorExpressionDescriptor.ArgumentType vectorArgType = VectorExpressionDescriptor.ArgumentType.fromHiveTypeName(typeName);
+    if (vectorArgType == VectorExpressionDescriptor.ArgumentType.NONE) {
+      throw new HiveException("No vector argument type for type name " + typeName);
+    }
+    int outCol = -1;
+    if (mode == VectorExpressionDescriptor.Mode.PROJECTION) {
+      outCol = ocm.allocateOutputColumn(dynamicValueExpr.getTypeInfo());
+    }
+
+    return new DynamicValueVectorExpression(outCol, dynamicValueExpr.getTypeInfo(), dynamicValueExpr.getDynamicValue());
+  }
+
+  /**
+   * Used as a fast path for operations that don't modify their input, like unary +
+   * and casting boolean to long. IdentityExpression and its children are always
+   * projections.
+   */
+  private VectorExpression getIdentityExpression(List<ExprNodeDesc> childExprList)
+      throws HiveException {
+    ExprNodeDesc childExpr = childExprList.get(0);
+    int identityCol;
+    TypeInfo identityTypeInfo;
+    DataTypePhysicalVariation identityDataTypePhysicalVariation;
+    VectorExpression v1 = null;
+    if (childExpr instanceof ExprNodeGenericFuncDesc) {
+      v1 = getVectorExpression(childExpr);
+      identityCol = v1.getOutputColumnNum();
+      identityTypeInfo = v1.getOutputTypeInfo();
+      identityDataTypePhysicalVariation = v1.getOutputDataTypePhysicalVariation();
+    } else if (childExpr instanceof ExprNodeColumnDesc) {
+      ExprNodeColumnDesc colDesc = (ExprNodeColumnDesc) childExpr;
+      identityCol = getInputColumnIndex(colDesc.getColumn());
+      identityTypeInfo = colDesc.getTypeInfo();
+
+      // CONSIDER: Validation of type information
+
+      identityDataTypePhysicalVariation = getDataTypePhysicalVariation(identityCol);
+    } else {
+      throw new HiveException("Expression not supported: "+childExpr);
+    }
+
+    VectorExpression ve = new IdentityExpression(identityCol);
+
+    if (v1 != null) {
+      ve.setChildExpressions(new VectorExpression [] {v1});
+    }
+
+    ve.setInputTypeInfos(identityTypeInfo);
+    ve.setInputDataTypePhysicalVariations(identityDataTypePhysicalVariation);
+
+    ve.setOutputTypeInfo(identityTypeInfo);
+    ve.setOutputDataTypePhysicalVariation(identityDataTypePhysicalVariation);
+
+    return ve;
+  }
+
+
+  private boolean checkExprNodeDescForDecimal64(ExprNodeDesc exprNodeDesc) throws HiveException {
+    if (exprNodeDesc instanceof ExprNodeColumnDesc) {
+      int colIndex = getInputColumnIndex((ExprNodeColumnDesc) exprNodeDesc);
+      DataTypePhysicalVariation dataTypePhysicalVariation = getDataTypePhysicalVariation(colIndex);
+      return (dataTypePhysicalVariation == DataTypePhysicalVariation.DECIMAL_64);
+    } else if (exprNodeDesc instanceof ExprNodeGenericFuncDesc) {
+
+      // Is the result Decimal64 precision?
+      TypeInfo returnType = exprNodeDesc.getTypeInfo();
+      if (!checkTypeInfoForDecimal64(returnType)) {
+        return false;
+      }
+      DecimalTypeInfo returnDecimalType = (DecimalTypeInfo) returnType;
+
+      GenericUDF udf = ((ExprNodeGenericFuncDesc) exprNodeDesc).getGenericUDF();
+      Class<?> udfClass = udf.getClass();
+
+      // We have a class-level annotation that says whether the UDF's vectorization expressions
+      // support Decimal64.
+      VectorizedExpressionsSupportDecimal64 annotation =
+          AnnotationUtils.getAnnotation(udfClass, VectorizedExpressionsSupportDecimal64.class);
+      if (annotation == null) {
+        return false;
+      }
+
+      // Carefully check the children to make sure they are Decimal64.
+      List<ExprNodeDesc> children = exprNodeDesc.getChildren();
+      for (ExprNodeDesc childExprNodeDesc : children) {
+
+        // Some cases were converted before calling getVectorExpressionForUdf.
+        // So, emulate those cases first.
+
+        if (childExprNodeDesc instanceof ExprNodeConstantDesc) {
+          DecimalTypeInfo childDecimalTypeInfo =
+              decimalTypeFromCastToDecimal(childExprNodeDesc, returnDecimalType);
+          if (childDecimalTypeInfo == null) {
+            return false;
+          }
+          if (!checkTypeInfoForDecimal64(childDecimalTypeInfo)) {
+            return false;
+          }
+          continue;
+        }
+
+        // Otherwise, recurse.
+        if (!checkExprNodeDescForDecimal64(childExprNodeDesc)) {
+          return false;
+        }
+      }
+      return true;
+    } else if (exprNodeDesc instanceof ExprNodeConstantDesc) {
+      return checkTypeInfoForDecimal64(exprNodeDesc.getTypeInfo());
+    }
+    return false;
+  }
+
+  private boolean checkTypeInfoForDecimal64(TypeInfo typeInfo) {
+    if (typeInfo instanceof DecimalTypeInfo) {
+      DecimalTypeInfo decimalTypeInfo = (DecimalTypeInfo) typeInfo;
+      return HiveDecimalWritable.isPrecisionDecimal64(decimalTypeInfo.precision());
+    }
+    return false;
+  }
+
+  public boolean haveCandidateForDecimal64VectorExpression(int numChildren,
+      List<ExprNodeDesc> childExpr, TypeInfo returnType) throws HiveException {
+
+    // For now, just 2 Decimal64 inputs and a Decimal64 or boolean output.
+    return (numChildren == 2 &&
+        checkExprNodeDescForDecimal64(childExpr.get(0)) &&
+        checkExprNodeDescForDecimal64(childExpr.get(1)) &&
+        (checkTypeInfoForDecimal64(returnType) ||
+            returnType.equals(TypeInfoFactory.booleanTypeInfo)));
+  }
+
+  private VectorExpression getDecimal64VectorExpressionForUdf(GenericUDF genericUdf,
+      Class<?> udfClass, List<ExprNodeDesc> childExpr, int numChildren,
+      VectorExpressionDescriptor.Mode mode, TypeInfo returnType) throws HiveException {
+
+    ExprNodeDesc child1 = childExpr.get(0);
+    ExprNodeDesc child2 = childExpr.get(1);
+
+    DecimalTypeInfo decimalTypeInfo1 = (DecimalTypeInfo) child1.getTypeInfo();
+    DecimalTypeInfo decimalTypeInfo2 = (DecimalTypeInfo) child2.getTypeInfo();
+
+    DataTypePhysicalVariation dataTypePhysicalVariation1 = DataTypePhysicalVariation.DECIMAL_64;
+    DataTypePhysicalVariation dataTypePhysicalVariation2 = DataTypePhysicalVariation.DECIMAL_64;
+
+    final int scale1 = decimalTypeInfo1.scale();
+    final int scale2 = decimalTypeInfo2.scale();
+
+    VectorExpressionDescriptor.Builder builder = new VectorExpressionDescriptor.Builder();
+    builder.setNumArguments(numChildren);
+    builder.setMode(mode);
+
+    boolean isColumnScaleEstablished = false;
+    int columnScale = 0;
+    boolean hasScalar = false;
+    builder.setArgumentType(0, ArgumentType.DECIMAL_64);
+    if (child1 instanceof ExprNodeGenericFuncDesc ||
+        child1 instanceof ExprNodeColumnDesc) {
+      builder.setInputExpressionType(0, InputExpressionType.COLUMN);
+      isColumnScaleEstablished = true;
+      columnScale = scale1;
+    } else if (child1 instanceof ExprNodeConstantDesc) {
+      if (isNullConst(child1)) {
+
+        // Cannot handle NULL scalar parameter.
+        return null;
+      }
+      hasScalar = true;
+      builder.setInputExpressionType(0, InputExpressionType.SCALAR);
+    } else {
+
+      // Currently, only functions, columns, and scalars supported.
+      return null;
+    }
+
+    builder.setArgumentType(1, ArgumentType.DECIMAL_64);
+    if (child2 instanceof ExprNodeGenericFuncDesc ||
+        child2 instanceof ExprNodeColumnDesc) {
+      builder.setInputExpressionType(1, InputExpressionType.COLUMN);
+      if (!isColumnScaleEstablished) {
+        isColumnScaleEstablished = true;
+        columnScale = scale2;
+      } else if (columnScale != scale2) {
+
+        // We only support Decimal64 on 2 columns when the have the same scale.
+        return null;
+      }
+    } else if (child2 instanceof ExprNodeConstantDesc) {
+      // Cannot have SCALAR, SCALAR.
+      if (!isColumnScaleEstablished) {
+        return null;
+      }
+      if (isNullConst(child2)) {
+
+        // Cannot handle NULL scalar parameter.
+        return null;
+      }
+      hasScalar = true;
+      builder.setInputExpressionType(1, InputExpressionType.SCALAR);
+    } else {
+
+      // Currently, only functions, columns, and scalars supported.
+      return null;
+    }
+
+    VectorExpressionDescriptor.Descriptor descriptor = builder.build();
+    Class<?> vectorClass =
+        this.vMap.getVectorExpressionClass(udfClass, descriptor, useCheckedVectorExpressions);
+    if (vectorClass == null) {
+      return null;
+    }
+
+    VectorExpressionDescriptor.Mode childrenMode = getChildrenMode(mode, udfClass);
+
+    /*
+     * Custom build arguments.
+     */
+
+    List<VectorExpression> children = new ArrayList<VectorExpression>();
+    Object[] arguments = new Object[numChildren];
+
+    for (int i = 0; i < numChildren; i++) {
+      ExprNodeDesc child = childExpr.get(i);
+      if (child instanceof ExprNodeGenericFuncDesc) {
+        VectorExpression vChild = getVectorExpression(child, childrenMode);
+        children.add(vChild);
+        arguments[i] = vChild.getOutputColumnNum();
+      } else if (child instanceof ExprNodeColumnDesc) {
+        int colIndex = getInputColumnIndex((ExprNodeColumnDesc) child);
+        if (childrenMode == VectorExpressionDescriptor.Mode.FILTER) {
+
+          VectorExpression filterExpr =
+              getFilterOnBooleanColumnExpression((ExprNodeColumnDesc) child, colIndex);
+          if (filterExpr == null) {
+            return null;
+          }
+
+          children.add(filterExpr);
+        }
+        arguments[i] = colIndex;
+      } else {
+        Preconditions.checkState(child instanceof ExprNodeConstantDesc);
+        ExprNodeConstantDesc constDesc = (ExprNodeConstantDesc) child;
+        HiveDecimal hiveDecimal = (HiveDecimal) constDesc.getValue();
+        if (hiveDecimal.scale() > columnScale) {
+
+          // For now, bail out on decimal constants with larger scale than column scale.
+          return null;
+        }
+        final long decimal64Scalar = new HiveDecimalWritable(hiveDecimal).serialize64(columnScale);
+        arguments[i] = decimal64Scalar;
+      }
+    }
+
+    /*
+     * Instantiate Decimal64 vector expression.
+     *
+     * The instantiateExpression method sets the output column and type information.
+     */
+    VectorExpression  vectorExpression =
+        instantiateExpression(vectorClass, returnType, DataTypePhysicalVariation.DECIMAL_64, arguments);
+    if (vectorExpression == null) {
+      handleCouldNotInstantiateVectorExpression(vectorClass, returnType, DataTypePhysicalVariation.DECIMAL_64, arguments);
+    }
+
+    vectorExpression.setInputTypeInfos(decimalTypeInfo1, decimalTypeInfo2);
+    vectorExpression.setInputDataTypePhysicalVariations(dataTypePhysicalVariation1, dataTypePhysicalVariation2);
+
+    if ((vectorExpression != null) && !children.isEmpty()) {
+      vectorExpression.setChildExpressions(children.toArray(new VectorExpression[0]));
+    }
+
+    return vectorExpression;
+  }
+
+  private VectorExpression getVectorExpressionForUdf(GenericUDF genericUdf,
+      Class<?> udfClass, List<ExprNodeDesc> childExpr, VectorExpressionDescriptor.Mode mode,
+      TypeInfo returnType) throws HiveException {
+
+    int numChildren = (childExpr == null) ? 0 : childExpr.size();
+
+    if (numChildren > 2 && genericUdf != null && mode == VectorExpressionDescriptor.Mode.FILTER &&
+        ((genericUdf instanceof GenericUDFOPOr) || (genericUdf instanceof GenericUDFOPAnd))) {
+
+      // Special case handling for Multi-OR and Multi-AND.
+
+      for (int i = 0; i < numChildren; i++) {
+        ExprNodeDesc child = childExpr.get(i);
+        String childTypeString = child.getTypeString();
+        if (childTypeString == null) {
+          throw new HiveException("Null child type name string");
+        }
+        TypeInfo typeInfo = TypeInfoUtils.getTypeInfoFromTypeString(childTypeString);
+        Type columnVectorType = VectorizationContext.getColumnVectorTypeFromTypeInfo(typeInfo);
+        if (columnVectorType != ColumnVector.Type.LONG){
+          return null;
+        }
+        if (!(child instanceof ExprNodeGenericFuncDesc) && !(child instanceof ExprNodeColumnDesc)) {
+          return null;
+        }
+      }
+      Class<?> vclass;
+      if (genericUdf instanceof GenericUDFOPOr) {
+        vclass = FilterExprOrExpr.class;
+      } else if (genericUdf instanceof GenericUDFOPAnd) {
+        vclass = FilterExprAndExpr.class;
+      } else {
+        throw new RuntimeException("Unexpected multi-child UDF");
+      }
+      VectorExpressionDescriptor.Mode childrenMode = getChildrenMode(mode, udfClass);
+      return createVectorExpression(vclass, childExpr, childrenMode, returnType);
+    }
+    if (numChildren > VectorExpressionDescriptor.MAX_NUM_ARGUMENTS) {
+      return null;
+    }
+
+    // Should we intercept here for a possible Decimal64 vector expression class?
+    if (haveCandidateForDecimal64VectorExpression(numChildren, childExpr, returnType)) {
+      VectorExpression result = getDecimal64VectorExpressionForUdf(genericUdf, udfClass,
+          childExpr, numChildren, mode, returnType);
+      if (result != null) {
+        return result;
+      }
+      // Otherwise, fall through and proceed with non-Decimal64 vector expression classes...
+    }
+
+    VectorExpressionDescriptor.Builder builder = new VectorExpressionDescriptor.Builder();
+    builder.setNumArguments(numChildren);
+    builder.setMode(mode);
+    for (int i = 0; i < numChildren; i++) {
+      ExprNodeDesc child = childExpr.get(i);
+      TypeInfo childTypeInfo = child.getTypeInfo();
+      String childTypeString = childTypeInfo.toString();
+      if (childTypeString == null) {
+        throw new HiveException("Null child type name string");
+      }
+      String undecoratedTypeName = getUndecoratedName(childTypeString);
+      if (undecoratedTypeName == null) {
+        throw new HiveException("No match for type string " + childTypeString + " from undecorated type name method");
+      }
+      builder.setArgumentType(i, undecoratedTypeName);
+      if ((child instanceof ExprNodeGenericFuncDesc) || (child instanceof ExprNodeColumnDesc)
+          || (child instanceof ExprNodeFieldDesc)) {
+        builder.setInputExpressionType(i, InputExpressionType.COLUMN);
+      } else if (child instanceof ExprNodeConstantDesc) {
+        if (isNullConst(child)) {
+          // Cannot handle NULL scalar parameter.
+          return null;
+        }
+        builder.setInputExpressionType(i, InputExpressionType.SCALAR);
+      } else if (child instanceof ExprNodeDynamicValueDesc) {
+        builder.setInputExpressionType(i, InputExpressionType.DYNAMICVALUE);
+      } else {
+        throw new HiveException("Cannot handle expression type: " + child.getClass().getSimpleName());
+      }
+    }
+    VectorExpressionDescriptor.Descriptor descriptor = builder.build();
+    Class<?> vclass =
+        this.vMap.getVectorExpressionClass(udfClass, descriptor, useCheckedVectorExpressions);
+    if (vclass == null) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("No vector udf found for "+udfClass.getSimpleName() + ", descriptor: "+descriptor);
+      }
+      return null;
+    }
+    VectorExpressionDescriptor.Mode childrenMode = getChildrenMode(mode, udfClass);
+    return createVectorExpression(vclass, childExpr, childrenMode, returnType);
+  }
+
+  private VectorExpression createDecimal64ToDecimalConversion(int colIndex, TypeInfo resultTypeInfo)
+      throws HiveException {
+    Object [] conversionArgs = new Object[1];
+    conversionArgs[0] = colIndex;
+    VectorExpression vectorExpression =
+        instantiateExpression(
+            ConvertDecimal64ToDecimal.class,
+            resultTypeInfo,
+            DataTypePhysicalVariation.NONE,
+            conversionArgs);
+    if (vectorExpression == null) {
+      handleCouldNotInstantiateVectorExpression(
+          ConvertDecimal64ToDecimal.class, resultTypeInfo, DataTypePhysicalVariation.NONE,
+          conversionArgs);
+    }
+
+    vectorExpression.setInputTypeInfos(resultTypeInfo);
+    vectorExpression.setInputDataTypePhysicalVariations(DataTypePhysicalVariation.DECIMAL_64);
+
+    return vectorExpression;
+  }
+
+  public VectorExpression wrapWithDecimal64ToDecimalConversion(VectorExpression inputExpression)
+      throws HiveException {
+
+    VectorExpression wrapExpression = createDecimal64ToDecimalConversion(
+        inputExpression.getOutputColumnNum(), inputExpression.getOutputTypeInfo());
+    if (inputExpression instanceof IdentityExpression) {
+      return wrapExpression;
+    }
+
+    // CONCERN: Leaking scratch column?
+    VectorExpression[] child = new VectorExpression[1];
+    child[0] = inputExpression;
+    wrapExpression.setChildExpressions(child);
+
+    return wrapExpression;
+  }
+
+  private VectorExpression createVectorExpression(Class<?> vectorClass,
+      List<ExprNodeDesc> childExpr, VectorExpressionDescriptor.Mode childrenMode, TypeInfo returnType) throws HiveException {
+    int numChildren = childExpr == null ? 0: childExpr.size();
+
+    TypeInfo[] inputTypeInfos = new TypeInfo[numChildren];
+    DataTypePhysicalVariation[] inputDataTypePhysicalVariations = new DataTypePhysicalVariation[numChildren];
+
+    List<VectorExpression> children = new ArrayList<VectorExpression>();
+    Object[] arguments = new Object[numChildren];
+
+    for (int i = 0; i < numChildren; i++) {
+      ExprNodeDesc child = childExpr.get(i);
+      TypeInfo childTypeInfo = child.getTypeInfo();
+
+      inputTypeInfos[i] = childTypeInfo;
+      inputDataTypePhysicalVariations[i] = DataTypePhysicalVariation.NONE;   // Assume.
+
+      if ((child instanceof ExprNodeGenericFuncDesc) || (child instanceof ExprNodeFieldDesc)) {
+        VectorExpression vChild = getVectorExpression(child, childrenMode);
+          children.add(vChild);
+          arguments[i] = vChild.getOutputColumnNum();
+
+          // Update.
+          inputDataTypePhysicalVariations[i] = vChild.getOutputDataTypePhysicalVariation();
+      } else if (child instanceof ExprNodeColumnDesc) {
+        int colIndex = getInputColumnIndex((ExprNodeColumnDesc) child);
+
+        // CONSIDER: Validate type information
+
+        if (childTypeInfo instanceof DecimalTypeInfo) {
+
+          // In this method, we must only process non-Decimal64 column vectors.
+          // Convert Decimal64 columns to regular decimal.
+          DataTypePhysicalVariation dataTypePhysicalVariation = getDataTypePhysicalVariation(colIndex);
+          if (dataTypePhysicalVariation != null && dataTypePhysicalVariation == DataTypePhysicalVariation.DECIMAL_64) {
+
+            // FUTURE: Can we reuse this conversion?
+            VectorExpression vChild = createDecimal64ToDecimalConversion(colIndex, childTypeInfo);
+            children.add(vChild);
+            arguments[i] = vChild.getOutputColumnNum();
+
+            // Update.
+            inputDataTypePhysicalVariations[i] = vChild.getOutputDataTypePhysicalVariation();
+            continue;
+          }
+        }
+        if (childrenMode == VectorExpressionDescriptor.Mode.FILTER) {
+
+          // In filter mode, the column must be a boolean
+          SelectColumnIsTrue selectColumnIsTrue = new SelectColumnIsTrue(colIndex);
+
+          selectColumnIsTrue.setInputTypeInfos(childTypeInfo);
+          selectColumnIsTrue.setInputDataTypePhysicalVariations(DataTypePhysicalVariation.NONE);
+
+          children.add(selectColumnIsTrue);
+        }
+        arguments[i] = colIndex;
+      } else if (child instanceof ExprNodeConstantDesc) {
+        Object scalarValue = getVectorTypeScalarValue((ExprNodeConstantDesc) child);
+        arguments[i] = (null == scalarValue) ? getConstantVectorExpression(null, child.getTypeInfo(), childrenMode) : scalarValue;
+      } else if (child instanceof ExprNodeDynamicValueDesc) {
+        arguments[i] = ((ExprNodeDynamicValueDesc) child).getDynamicValue();
+      } else {
+        throw new HiveException("Cannot handle expression type: " + child.getClass().getSimpleName());
+      }
+    }
+    VectorExpression vectorExpression = instantiateExpression(vectorClass, returnType, DataTypePhysicalVariation.NONE, arguments);
+    if (vectorExpression == null) {
+      handleCouldNotInstantiateVectorExpression(vectorClass, returnType, DataTypePhysicalVariation.NONE, arguments);
+    }
+
+    vectorExpression.setInputTypeInfos(inputTypeInfos);
+    vectorExpression.setInputDataTypePhysicalVariations(inputDataTypePhysicalVariations);
+
+    if ((vectorExpression != null) && !children.isEmpty()) {
+      vectorExpression.setChildExpressions(children.toArray(new VectorExpression[0]));
+    }
+
+    for (VectorExpression ve : children) {
+      ocm.freeOutputColumn(ve.getOutputColumnNum());
+    }
+
+    return vectorExpression;
+  }
+
+  private void handleCouldNotInstantiateVectorExpression(Class<?> vectorClass, TypeInfo returnType,
+      DataTypePhysicalVariation dataTypePhysicalVariation, Object[] arguments) throws HiveException {
+    String displayString = "Could not instantiate vector expression class " + vectorClass.getName() +
+        " for arguments " + Arrays.toString(arguments) + " return type " +
+        VectorExpression.getTypeName(returnType, dataTypePhysicalVariation);
+    throw new HiveException(displayString);
+  }
+
+  private VectorExpressionDescriptor.Mode getChildrenMode(VectorExpressionDescriptor.Mode mode, Class<?> udf) {
+    if (mode.equals(VectorExpressionDescriptor.Mode.FILTER) && (udf.equals(GenericUDFOPAnd.class) || udf.equals(GenericUDFOPOr.class))) {
+      return VectorExpressionDescriptor.Mode.FILTER;
+    }
+    return VectorExpressionDescriptor.Mode.PROJECTION;
+  }
+
+  private String getNewInstanceArgumentString(Object [] args) {
+    if (args == null) {
+      return "arguments: NULL";
+    }
+    ArrayList<String> argClasses = new ArrayList<String>();
+    for (Object obj : args) {
+      argClasses.add(obj.getClass().getSimpleName());
+    }
+    return "arguments: " + Arrays.toString(args) + ", argument classes: " + argClasses.toString();
+  }
+
+  private static final int STACK_LENGTH_LIMIT = 15;
+
+  public static String getStackTraceAsSingleLine(Throwable e) {
+    StringBuilder sb = new StringBuilder();
+    sb.append(e);
+    sb.append(" stack trace: ");
+    StackTraceElement[] stackTrace = e.getStackTrace();
+    int length = stackTrace.length;
+    boolean isTruncated = false;
+    if (length > STACK_LENGTH_LIMIT) {
+      length = STACK_LENGTH_LIMIT;
+      isTruncated = true;
+    }
+    for (int i = 0; i < length; i++) {
+      if (i > 0) {
+        sb.append(", ");
+      }
+      sb.append(stackTrace[i]);
+    }
+    if (isTruncated) {
+      sb.append(", ...");
+    }
+
+    // Attempt to cleanup stack trace elements that vary by VM.
+    String cleaned = sb.toString().replaceAll("GeneratedConstructorAccessor[0-9]*", "GeneratedConstructorAccessor<omitted>");
+
+    return cleaned;
+  }
+
+  public VectorExpression instantiateExpression(Class<?> vclass, TypeInfo returnTypeInfo,
+      DataTypePhysicalVariation returnDataTypePhysicalVariation, Object...args)
+      throws HiveException {
+    VectorExpression ve = null;
+    Constructor<?> ctor = getConstructor(vclass);
+    int numParams = ctor.getParameterTypes().length;
+    int argsLength = (args == null) ? 0 : args.length;
+    if (numParams == 0) {
+      try {
+        ve = (VectorExpression) ctor.newInstance();
+      } catch (Exception ex) {
+        throw new HiveException("Could not instantiate " + vclass.getSimpleName() + " with 0 arguments, exception: " +
+            getStackTraceAsSingleLine(ex));
+      }
+    } else if (numParams == argsLength) {
+      try {
+        ve = (VectorExpression) ctor.newInstance(args);
+      } catch (Exception ex) {
+          throw new HiveException("Could not instantiate " + vclass.getSimpleName() + " with " + getNewInstanceArgumentString(args) + ", exception: " +
+              getStackTraceAsSingleLine(ex));
+      }
+    } else if (numParams == argsLength + 1) {
+      // Additional argument is needed, which is the outputcolumn.
+      Object [] newArgs = null;
+      try {
+        if (returnTypeInfo == null) {
+          throw new HiveException("Missing output type information");
+        }
+        String returnTypeName = returnTypeInfo.getTypeName();
+        returnTypeName = VectorizationContext.mapTypeNameSynonyms(returnTypeName);
+
+        // Special handling for decimal because decimal types need scale and precision parameter.
+        // This special handling should be avoided by using returnType uniformly for all cases.
+        final int outputColumnNum =
+            ocm.allocateOutputColumn(returnTypeInfo, returnDataTypePhysicalVariation);
+
+        newArgs = Arrays.copyOf(args, numParams);
+        newArgs[numParams-1] = outputColumnNum;
+
+        ve = (VectorExpression) ctor.newInstance(newArgs);
+
+        /*
+         * Caller is responsible for setting children and input type information.
+         */
+        ve.setOutputTypeInfo(returnTypeInfo);
+        ve.setOutputDataTypePhysicalVariation(returnDataTypePhysicalVariation);
+
+      } catch (Exception ex) {
+          throw new HiveException("Could not instantiate " + vclass.getSimpleName() + " with arguments " + getNewInstanceArgumentString(newArgs) + ", exception: " +
+              getStackTraceAsSingleLine(ex));
+      }
+    }
+    // Add maxLength parameter to UDFs that have CHAR or VARCHAR output.
+    if (ve instanceof TruncStringOutput) {
+      TruncStringOutput truncStringOutput = (TruncStringOutput) ve;
+      if (returnTypeInfo instanceof BaseCharTypeInfo) {
+        BaseCharTypeInfo baseCharTypeInfo = (BaseCharTypeInfo) returnTypeInfo;
+        truncStringOutput.setMaxLength(baseCharTypeInfo.getLength());
+      }
+    }
+    return ve;
+  }
+
+  private VectorExpression getGenericUdfVectorExpression(GenericUDF udf,
+      List<ExprNodeDesc> childExpr, VectorExpressionDescriptor.Mode mode, TypeInfo returnType) throws HiveException {
+
+    List<ExprNodeDesc> castedChildren = evaluateCastOnConstants(childExpr);
+    childExpr = castedChildren;
+
+    //First handle special cases.  If one of the special case methods cannot handle it,
+    // it returns null.
+    VectorExpression ve = null;
+    if (udf instanceof GenericUDFBetween && mode == VectorExpressionDescriptor.Mode.FILTER) {
+      ve = getBetweenFilterExpression(childExpr, mode, returnType);
+    } else if (udf instanceof GenericUDFIn) {
+      ve = getInExpression(childExpr, mode, returnType);
+    } else if (udf instanceof GenericUDFIf) {
+      ve = getIfExpression((GenericUDFIf) udf, childExpr, mode, returnType);
+    } else if (udf instanceof GenericUDFWhen) {
+      ve = getWhenExpression(childExpr, mode, returnType);
+    } else if (udf instanceof GenericUDFOPPositive) {
+      ve = getIdentityExpression(childExpr);
+    } else if (udf instanceof GenericUDFCoalesce || udf instanceof GenericUDFNvl) {
+
+      // Coalesce is a special case because it can take variable number of arguments.
+      // Nvl is a specialization of the Coalesce.
+      ve = getCoalesceExpression(childExpr, returnType);
+    } else if (udf instanceof GenericUDFElt) {
+
+      // Elt is a special case because it can take variable number of arguments.
+      ve = getEltExpression(childExpr, returnType);
+    } else if (udf instanceof GenericUDFBridge) {
+      ve = getGenericUDFBridgeVectorExpression((GenericUDFBridge) udf, childExpr, mode,
+          returnType);
+    } else if (udf instanceof GenericUDFToString) {
+      ve = getCastToString(childExpr, returnType);
+    } else if (udf instanceof GenericUDFToDecimal) {
+      ve = getCastToDecimal(childExpr, returnType);
+    } else if (udf instanceof GenericUDFToChar) {
+      ve = getCastToChar(childExpr, returnType);
+    } else if (udf instanceof GenericUDFToVarchar) {
+      ve = getCastToVarChar(childExpr, returnType);
+    } else if (udf instanceof GenericUDFTimestamp) {
+      ve = getCastToTimestamp((GenericUDFTimestamp)udf, childExpr, mode, returnType);
+    }
+    if (ve != null) {
+      return ve;
+    }
+    // Now do a general lookup
+    Class<?> udfClass = udf.getClass();
+    boolean isSubstituted = false;
+    if (udf instanceof GenericUDFBridge) {
+      udfClass = ((GenericUDFBridge) udf).getUdfClass();
+      isSubstituted = true;
+    }
+
+    ve = getVectorExpressionForUdf((!isSubstituted ? udf : null),
+        udfClass, castedChildren, mode, returnType);
+
+    return ve;
+  }
+
+  private VectorExpression getCastToTimestamp(GenericUDFTimestamp udf,
+      List<ExprNodeDesc> childExpr, VectorExpressionDescriptor.Mode mode, TypeInfo returnType) throws HiveException {
+    VectorExpression ve = getVectorExpressionForUdf(udf, udf.getClass(), childExpr, mode, returnType);
+
+    // Replace with the milliseconds conversion
+    if (!udf.isIntToTimestampInSeconds() && ve instanceof CastLongToTimestamp) {
+      ve = createVectorExpression(CastMillisecondsLongToTimestamp.class,
+          childExpr, VectorExpressionDescriptor.Mode.PROJECTION, returnType);
+    }
+
+    return ve;
+  }
+
+  private void freeNonColumns(VectorExpression[] vectorChildren) {
+    if (vectorChildren == null) {
+      return;
+    }
+    for (VectorExpression v : vectorChildren) {
+      if (!(v instanceof IdentityExpression)) {
+        ocm.freeOutputColumn(v.getOutputColumnNum());
+      }
+    }
+  }
+
+  private VectorExpression getCoalesceExpression(List<ExprNodeDesc> childExpr, TypeInfo returnType)
+      throws HiveException {
+    int[] inputColumns = new int[childExpr.size()];
+    VectorExpression[] vectorChildren =
+        getVectorExpressions(childExpr, VectorExpressionDescriptor.Mode.PROJECTION);
+
+    final int size = vectorChildren.length;
+    TypeInfo[] inputTypeInfos = new TypeInfo[size];
+    DataTypePhysicalVariation[] inputDataTypePhysicalVariations = new DataTypePhysicalVariation[size];
+    int i = 0;
+    for (VectorExpression ve : vectorChildren) {
+      inputColumns[i] = ve.getOutputColumn

<TRUNCATED>

[2/8] hive git commit: HIVE-20207: Vectorization: Fix NULL / Wrong Results issues in Filter / Compare (Matt McCline, reviewed by Teddy Choi)

Posted by mm...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/test/results/clientpositive/spark/vectorized_timestamp_funcs.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/vectorized_timestamp_funcs.q.out b/ql/src/test/results/clientpositive/spark/vectorized_timestamp_funcs.q.out
index 1791c89..f094fba 100644
--- a/ql/src/test/results/clientpositive/spark/vectorized_timestamp_funcs.q.out
+++ b/ql/src/test/results/clientpositive/spark/vectorized_timestamp_funcs.q.out
@@ -476,7 +476,7 @@ STAGE PLANS:
                         className: VectorSelectOperator
                         native: true
                         projectedOutputColumnNums: [5, 7, 8, 9, 6, 11, 10, 13, 14]
-                        selectExpressions: VectorUDFUnixTimeStampString(col 2:string) -> 5:bigint, VectorUDFYearDate(col 6, field YEAR)(children: CastStringToDate(col 2:string) -> 6:date) -> 7:int, VectorUDFMonthDate(col 6, field MONTH)(children: CastStringToDate(col 2:string) -> 6:date) -> 8:int, VectorUDFDayOfMonthDate(col 6, field DAY_OF_MONTH)(children: CastStringToDate(col 2:string) -> 6:date) -> 9:int, VectorUDFDayOfMonthString(col 2:string, fieldStart 8, fieldLength 2) -> 6:int, VectorUDFWeekOfYearDate(col 10, field WEEK_OF_YEAR)(children: CastStringToDate(col 2:string) -> 10:date) -> 11:int, VectorUDFHourTimestamp(col 12:timestamp, field HOUR_OF_DAY)(children: VectorUDFAdaptor(CAST( stimestamp1 AS TIMESTAMP)) -> 12:timestamp) -> 10:int, VectorUDFMinuteTimestamp(col 12:timestamp, field MINUTE)(children: VectorUDFAdaptor(CAST( stimestamp1 AS TIMESTAMP)) -> 12:timestamp) -> 13:int, VectorUDFSecondTimestamp(col 12:timestamp, field SECOND)(children: VectorUDFAdaptor(CAST( stimest
 amp1 AS TIMESTAMP)) -> 12:timestamp) -> 14:int
+                        selectExpressions: VectorUDFUnixTimeStampString(col 2:string) -> 5:bigint, VectorUDFYearDate(col 6, field YEAR)(children: CastStringToDate(col 2:string) -> 6:date) -> 7:int, VectorUDFMonthDate(col 6, field MONTH)(children: CastStringToDate(col 2:string) -> 6:date) -> 8:int, VectorUDFDayOfMonthDate(col 6, field DAY_OF_MONTH)(children: CastStringToDate(col 2:string) -> 6:date) -> 9:int, VectorUDFDayOfMonthString(col 2:string, fieldStart 8, fieldLength 2) -> 6:int, VectorUDFWeekOfYearDate(col 10, field WEEK_OF_YEAR)(children: CastStringToDate(col 2:string) -> 10:date) -> 11:int, VectorUDFHourTimestamp(col 12:timestamp, field HOUR_OF_DAY)(children: CastStringToTimestamp(col 2:string) -> 12:timestamp) -> 10:int, VectorUDFMinuteTimestamp(col 12:timestamp, field MINUTE)(children: CastStringToTimestamp(col 2:string) -> 12:timestamp) -> 13:int, VectorUDFSecondTimestamp(col 12:timestamp, field SECOND)(children: CastStringToTimestamp(col 2:string) -> 12:timestamp) -> 14
 :int
                     Statistics: Num rows: 52 Data size: 3219 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col0 (type: bigint)
@@ -495,7 +495,7 @@ STAGE PLANS:
                 featureSupportInUse: [DECIMAL_64]
                 inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
                 allNative: true
-                usesVectorUDFAdaptor: true
+                usesVectorUDFAdaptor: false
                 vectorized: true
         Reducer 2 
             Execution mode: vectorized
@@ -668,7 +668,7 @@ STAGE PLANS:
                         className: VectorSelectOperator
                         native: true
                         projectedOutputColumnNums: [7, 6, 8, 9, 11, 10, 14, 15, 16]
-                        selectExpressions: LongColEqualLongColumn(col 5:bigint, col 6:bigint)(children: VectorUDFUnixTimeStampTimestamp(col 1:timestamp) -> 5:bigint, VectorUDFUnixTimeStampString(col 2:string) -> 6:bigint) -> 7:boolean, LongColEqualLongColumn(col 5:int, col 8:int)(children: VectorUDFYearTimestamp(col 1:timestamp, field YEAR) -> 5:int, VectorUDFYearDate(col 6, field YEAR)(children: CastStringToDate(col 2:string) -> 6:date) -> 8:int) -> 6:boolean, LongColEqualLongColumn(col 5:int, col 9:int)(children: VectorUDFMonthTimestamp(col 1:timestamp, field MONTH) -> 5:int, VectorUDFMonthDate(col 8, field MONTH)(children: CastStringToDate(col 2:string) -> 8:date) -> 9:int) -> 8:boolean, LongColEqualLongColumn(col 5:int, col 10:int)(children: VectorUDFDayOfMonthTimestamp(col 1:timestamp, field DAY_OF_MONTH) -> 5:int, VectorUDFDayOfMonthDate(col 9, field DAY_OF_MONTH)(children: CastStringToDate(col 2:string) -> 9:date) -> 10:int) -> 9:boolean, LongColEqualLongColumn(col 5:int, col
  10:int)(children: VectorUDFDayOfMonthTimestamp(col 1:timestamp, field DAY_OF_MONTH) -> 5:int, VectorUDFDayOfMonthString(col 2:string, fieldStart 8, fieldLength 2) -> 10:int) -> 11:boolean, LongColEqualLongColumn(col 5:int, col 12:int)(children: VectorUDFWeekOfYearTimestamp(col 1:timestamp, field WEEK_OF_YEAR) -> 5:int, VectorUDFWeekOfYearDate(col 10, field WEEK_OF_YEAR)(children: CastStringToDate(col 2:string) -> 10:date) -> 12:int) -> 10:boolean, LongColEqualLongColumn(col 5:int, col 12:int)(children: VectorUDFHourTimestamp(col 1:timestamp, field HOUR_OF_DAY) -> 5:int, VectorUDFHourTimestamp(col 13:timestamp, field HOUR_OF_DAY)(children: VectorUDFAdaptor(CAST( stimestamp1 AS TIMESTAMP)) -> 13:timestamp) -> 12:int) -> 14:boolean, LongColEqualLongColumn(col 5:int, col 12:int)(children: VectorUDFMinuteTimestamp(col 1:timestamp, field MINUTE) -> 5:int, VectorUDFMinuteTimestamp(col 13:timestamp, field MINUTE)(children: VectorUDFAdaptor(CAST( stimestamp1 AS TIMESTAMP)) -> 13:timestamp) 
 -> 12:int) -> 15:boolean, LongColEqualLongColumn(col 5:int, col 12:int)(children: VectorUDFSecondTimestamp(col 1:timestamp, field SECOND) -> 5:int, VectorUDFSecondTimestamp(col 13:timestamp, field SECOND)(children: VectorUDFAdaptor(CAST( stimestamp1 AS TIMESTAMP)) -> 13:timestamp) -> 12:int) -> 16:boolean
+                        selectExpressions: LongColEqualLongColumn(col 5:bigint, col 6:bigint)(children: VectorUDFUnixTimeStampTimestamp(col 1:timestamp) -> 5:bigint, VectorUDFUnixTimeStampString(col 2:string) -> 6:bigint) -> 7:boolean, LongColEqualLongColumn(col 5:int, col 8:int)(children: VectorUDFYearTimestamp(col 1:timestamp, field YEAR) -> 5:int, VectorUDFYearDate(col 6, field YEAR)(children: CastStringToDate(col 2:string) -> 6:date) -> 8:int) -> 6:boolean, LongColEqualLongColumn(col 5:int, col 9:int)(children: VectorUDFMonthTimestamp(col 1:timestamp, field MONTH) -> 5:int, VectorUDFMonthDate(col 8, field MONTH)(children: CastStringToDate(col 2:string) -> 8:date) -> 9:int) -> 8:boolean, LongColEqualLongColumn(col 5:int, col 10:int)(children: VectorUDFDayOfMonthTimestamp(col 1:timestamp, field DAY_OF_MONTH) -> 5:int, VectorUDFDayOfMonthDate(col 9, field DAY_OF_MONTH)(children: CastStringToDate(col 2:string) -> 9:date) -> 10:int) -> 9:boolean, LongColEqualLongColumn(col 5:int, col
  10:int)(children: VectorUDFDayOfMonthTimestamp(col 1:timestamp, field DAY_OF_MONTH) -> 5:int, VectorUDFDayOfMonthString(col 2:string, fieldStart 8, fieldLength 2) -> 10:int) -> 11:boolean, LongColEqualLongColumn(col 5:int, col 12:int)(children: VectorUDFWeekOfYearTimestamp(col 1:timestamp, field WEEK_OF_YEAR) -> 5:int, VectorUDFWeekOfYearDate(col 10, field WEEK_OF_YEAR)(children: CastStringToDate(col 2:string) -> 10:date) -> 12:int) -> 10:boolean, LongColEqualLongColumn(col 5:int, col 12:int)(children: VectorUDFHourTimestamp(col 1:timestamp, field HOUR_OF_DAY) -> 5:int, VectorUDFHourTimestamp(col 13:timestamp, field HOUR_OF_DAY)(children: CastStringToTimestamp(col 2:string) -> 13:timestamp) -> 12:int) -> 14:boolean, LongColEqualLongColumn(col 5:int, col 12:int)(children: VectorUDFMinuteTimestamp(col 1:timestamp, field MINUTE) -> 5:int, VectorUDFMinuteTimestamp(col 13:timestamp, field MINUTE)(children: CastStringToTimestamp(col 2:string) -> 13:timestamp) -> 12:int) -> 15:boolean, Lo
 ngColEqualLongColumn(col 5:int, col 12:int)(children: VectorUDFSecondTimestamp(col 1:timestamp, field SECOND) -> 5:int, VectorUDFSecondTimestamp(col 13:timestamp, field SECOND)(children: CastStringToTimestamp(col 2:string) -> 13:timestamp) -> 12:int) -> 16:boolean
                     Statistics: Num rows: 52 Data size: 3219 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col0 (type: boolean)
@@ -687,7 +687,7 @@ STAGE PLANS:
                 featureSupportInUse: [DECIMAL_64]
                 inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
                 allNative: true
-                usesVectorUDFAdaptor: true
+                usesVectorUDFAdaptor: false
                 vectorized: true
         Reducer 2 
             Execution mode: vectorized
@@ -860,7 +860,7 @@ STAGE PLANS:
                         className: VectorSelectOperator
                         native: true
                         projectedOutputColumnNums: [2, 4, 5, 6, 3, 8, 7, 10, 11]
-                        selectExpressions: VectorUDFUnixTimeStampString(col 0:string) -> 2:bigint, VectorUDFYearDate(col 3, field YEAR)(children: CastStringToDate(col 0:string) -> 3:date) -> 4:int, VectorUDFMonthDate(col 3, field MONTH)(children: CastStringToDate(col 0:string) -> 3:date) -> 5:int, VectorUDFDayOfMonthDate(col 3, field DAY_OF_MONTH)(children: CastStringToDate(col 0:string) -> 3:date) -> 6:int, VectorUDFDayOfMonthString(col 0:string, fieldStart 8, fieldLength 2) -> 3:int, VectorUDFWeekOfYearDate(col 7, field WEEK_OF_YEAR)(children: CastStringToDate(col 0:string) -> 7:date) -> 8:int, VectorUDFHourTimestamp(col 9:timestamp, field HOUR_OF_DAY)(children: VectorUDFAdaptor(CAST( stimestamp1 AS TIMESTAMP)) -> 9:timestamp) -> 7:int, VectorUDFMinuteTimestamp(col 9:timestamp, field MINUTE)(children: VectorUDFAdaptor(CAST( stimestamp1 AS TIMESTAMP)) -> 9:timestamp) -> 10:int, VectorUDFSecondTimestamp(col 9:timestamp, field SECOND)(children: VectorUDFAdaptor(CAST( stimestamp1 AS T
 IMESTAMP)) -> 9:timestamp) -> 11:int
+                        selectExpressions: VectorUDFUnixTimeStampString(col 0:string) -> 2:bigint, VectorUDFYearDate(col 3, field YEAR)(children: CastStringToDate(col 0:string) -> 3:date) -> 4:int, VectorUDFMonthDate(col 3, field MONTH)(children: CastStringToDate(col 0:string) -> 3:date) -> 5:int, VectorUDFDayOfMonthDate(col 3, field DAY_OF_MONTH)(children: CastStringToDate(col 0:string) -> 3:date) -> 6:int, VectorUDFDayOfMonthString(col 0:string, fieldStart 8, fieldLength 2) -> 3:int, VectorUDFWeekOfYearDate(col 7, field WEEK_OF_YEAR)(children: CastStringToDate(col 0:string) -> 7:date) -> 8:int, VectorUDFHourTimestamp(col 9:timestamp, field HOUR_OF_DAY)(children: CastStringToTimestamp(col 0:string) -> 9:timestamp) -> 7:int, VectorUDFMinuteTimestamp(col 9:timestamp, field MINUTE)(children: CastStringToTimestamp(col 0:string) -> 9:timestamp) -> 10:int, VectorUDFSecondTimestamp(col 9:timestamp, field SECOND)(children: CastStringToTimestamp(col 0:string) -> 9:timestamp) -> 11:int
                     Statistics: Num rows: 3 Data size: 294 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col0 (type: bigint)
@@ -879,7 +879,7 @@ STAGE PLANS:
                 featureSupportInUse: [DECIMAL_64]
                 inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
                 allNative: true
-                usesVectorUDFAdaptor: true
+                usesVectorUDFAdaptor: false
                 vectorized: true
         Reducer 2 
             Execution mode: vectorized

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/test/results/clientpositive/vector_case_when_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vector_case_when_1.q.out b/ql/src/test/results/clientpositive/vector_case_when_1.q.out
index 4151082..01fc3ce 100644
--- a/ql/src/test/results/clientpositive/vector_case_when_1.q.out
+++ b/ql/src/test/results/clientpositive/vector_case_when_1.q.out
@@ -515,8 +515,8 @@ STAGE PLANS:
               Select Vectorization:
                   className: VectorSelectOperator
                   native: true
-                  projectedOutputColumnNums: [4, 22, 24, 25, 26, 27, 28, 30, 31, 32, 33, 34, 35, 38, 40, 43, 44]
-                  selectExpressions: IfExprStringScalarStringGroupColumn(col 17:boolean, val Singlecol 21:string)(children: LongColEqualLongScalar(col 4:int, val 1) -> 17:boolean, IfExprStringScalarStringGroupColumn(col 18:boolean, val Twocol 22:string)(children: LongColEqualLongScalar(col 4:int, val 2) -> 18:boolean, IfExprStringScalarStringGroupColumn(col 19:boolean, val Somecol 21:string)(children: LongColLessLongScalar(col 4:int, val 10) -> 19:boolean, IfExprStringScalarStringScalar(col 20:boolean, val Many, val Huge number)(children: LongColLessLongScalar(col 4:int, val 100) -> 20:boolean) -> 21:string) -> 22:string) -> 21:string) -> 22:string, IfExprStringScalarStringGroupColumn(col 17:boolean, val Singlecol 23:string)(children: LongColEqualLongScalar(col 4:int, val 1) -> 17:boolean, IfExprStringScalarStringGroupColumn(col 18:boolean, val Twocol 24:string)(children: LongColEqualLongScalar(col 4:int, val 2) -> 18:boolean, IfExprStringScalarStringGroupColumn(col 19:boolean, val 
 Somecol 23:string)(children: LongColLessLongScalar(col 4:int, val 10) -> 19:boolean, IfExprColumnNull(col 20:boolean, col 21:string, null)(children: LongColLessLongScalar(col 4:int, val 100) -> 20:boolean, ConstantVectorExpression(val Many) -> 21:string) -> 23:string) -> 24:string) -> 23:string) -> 24:string, IfExprStringScalarStringGroupColumn(col 17:boolean, val Singlecol 23:string)(children: LongColEqualLongScalar(col 4:int, val 1) -> 17:boolean, IfExprStringScalarStringGroupColumn(col 18:boolean, val Twocol 25:string)(children: LongColEqualLongScalar(col 4:int, val 2) -> 18:boolean, IfExprStringScalarStringGroupColumn(col 19:boolean, val Somecol 23:string)(children: LongColLessLongScalar(col 4:int, val 10) -> 19:boolean, IfExprNullNull(null, null) -> 23:string) -> 25:string) -> 23:string) -> 25:string, IfExprLongColumnLongColumn(col 17:boolean, col 18:date, col 19:date)(children: StringGroupColEqualCharScalar(col 14:char(10), val SHIP) -> 17:boolean, VectorUDFDateAddColScalar(co
 l 10:date, val 10) -> 18:date, VectorUDFDateAddColScalar(col 10:date, val 5) -> 19:date) -> 26:date, IfExprDoubleColumnLongScalar(col 17:boolean, col 28:double, val 0)(children: StringGroupColEqualCharScalar(col 8:char(1), val N) -> 17:boolean, DoubleColMultiplyDoubleColumn(col 5:double, col 27:double)(children: DoubleScalarSubtractDoubleColumn(val 1.0, col 6:double) -> 27:double) -> 28:double) -> 27:double, IfExprDoubleColumnDoubleScalar(col 17:boolean, col 29:double, val 0.0)(children: StringGroupColEqualCharScalar(col 8:char(1), val N) -> 17:boolean, DoubleColMultiplyDoubleColumn(col 5:double, col 28:double)(children: DoubleScalarSubtractDoubleColumn(val 1.0, col 6:double) -> 28:double) -> 29:double) -> 28:double, IfExprNullColumn(col 17:boolean, null, col 46)(children: StringGroupColEqualStringScalar(col 23:string, val DELIVER IN PERSON)(children: CastStringGroupToString(col 13:varchar(20)) -> 23:string) -> 17:boolean, ConvertDecimal64ToDecimal(col 7:decimal(10,2)/DECIMAL_64) ->
  46:decimal(10,2)) -> 30:decimal(10,2), IfExprColumnNull(col 18:boolean, col 47:decimal(10,2), null)(children: StringGroupColEqualStringScalar(col 23:string, val TAKE BACK RETURN)(children: CastStringGroupToString(col 13:varchar(20)) -> 23:string) -> 18:boolean, ConvertDecimal64ToDecimal(col 7:decimal(10,2)/DECIMAL_64) -> 47:decimal(10,2)) -> 31:decimal(10,2), VectorUDFAdaptor(if((CAST( l_shipinstruct AS STRING) = 'DELIVER IN PERSON'), 0, l_tax))(children: StringGroupColEqualStringScalar(col 23:string, val DELIVER IN PERSON)(children: CastStringGroupToString(col 13:varchar(20)) -> 23:string) -> 19:boolean) -> 32:decimal(12,2), VectorUDFAdaptor(if((CAST( l_shipinstruct AS STRING) = 'TAKE BACK RETURN'), l_tax, 0))(children: StringGroupColEqualStringScalar(col 23:string, val TAKE BACK RETURN)(children: CastStringGroupToString(col 13:varchar(20)) -> 23:string) -> 19:boolean) -> 33:decimal(12,2), VectorUDFAdaptor(if((CAST( l_shipinstruct AS STRING) = 'DELIVER IN PERSON'), 0, l_tax))(chil
 dren: StringGroupColEqualStringScalar(col 23:string, val DELIVER IN PERSON)(children: CastStringGroupToString(col 13:varchar(20)) -> 23:string) -> 19:boolean) -> 34:decimal(10,2), VectorUDFAdaptor(if((CAST( l_shipinstruct AS STRING) = 'TAKE BACK RETURN'), l_tax, 0))(children: StringGroupColEqualStringScalar(col 23:string, val TAKE BACK RETURN)(children: CastStringGroupToString(col 13:varchar(20)) -> 23:string) -> 19:boolean) -> 35:decimal(10,2), IfExprTimestampColumnColumn(col 19:boolean, col 36:timestampcol 37:timestamp)(children: LongColGreaterLongScalar(col 1:int, val 30) -> 19:boolean, CastDateToTimestamp(col 12:date) -> 36:timestamp, CastDateToTimestamp(col 11:date) -> 37:timestamp) -> 38:timestamp, IfExprColumnNull(col 19:boolean, col 39:int, null)(children: LongColGreaterLongScalar(col 2:int, val 10000) -> 19:boolean, VectorUDFDateDiffColCol(col 12:date, col 11:date) -> 39:int) -> 40:int, IfExprNullColumn(col 41:boolean, null, col 42)(children: LongColGreaterLongScalar(col 2:
 int, val 10000) -> 41:boolean, VectorUDFDateDiffColCol(col 12:date, col 11:date) -> 42:int) -> 43:int, IfExprLongScalarLongScalar(col 45:boolean, val 14245, val 14609)(children: LongColGreaterLongScalar(col 44:int, val 100)(children: LongColModuloLongScalar(col 2:int, val 500) -> 44:int) -> 45:boolean) -> 44:date
+                  projectedOutputColumnNums: [4, 22, 24, 25, 26, 27, 28, 30, 31, 32, 33, 34, 36, 40, 42, 45, 46]
+                  selectExpressions: IfExprStringScalarStringGroupColumn(col 17:boolean, val Singlecol 21:string)(children: LongColEqualLongScalar(col 4:int, val 1) -> 17:boolean, IfExprStringScalarStringGroupColumn(col 18:boolean, val Twocol 22:string)(children: LongColEqualLongScalar(col 4:int, val 2) -> 18:boolean, IfExprStringScalarStringGroupColumn(col 19:boolean, val Somecol 21:string)(children: LongColLessLongScalar(col 4:int, val 10) -> 19:boolean, IfExprStringScalarStringScalar(col 20:boolean, val Many, val Huge number)(children: LongColLessLongScalar(col 4:int, val 100) -> 20:boolean) -> 21:string) -> 22:string) -> 21:string) -> 22:string, IfExprStringScalarStringGroupColumn(col 17:boolean, val Singlecol 23:string)(children: LongColEqualLongScalar(col 4:int, val 1) -> 17:boolean, IfExprStringScalarStringGroupColumn(col 18:boolean, val Twocol 24:string)(children: LongColEqualLongScalar(col 4:int, val 2) -> 18:boolean, IfExprStringScalarStringGroupColumn(col 19:boolean, val 
 Somecol 23:string)(children: LongColLessLongScalar(col 4:int, val 10) -> 19:boolean, IfExprColumnNull(col 20:boolean, col 21:string, null)(children: LongColLessLongScalar(col 4:int, val 100) -> 20:boolean, ConstantVectorExpression(val Many) -> 21:string) -> 23:string) -> 24:string) -> 23:string) -> 24:string, IfExprStringScalarStringGroupColumn(col 17:boolean, val Singlecol 23:string)(children: LongColEqualLongScalar(col 4:int, val 1) -> 17:boolean, IfExprStringScalarStringGroupColumn(col 18:boolean, val Twocol 25:string)(children: LongColEqualLongScalar(col 4:int, val 2) -> 18:boolean, IfExprStringScalarStringGroupColumn(col 19:boolean, val Somecol 23:string)(children: LongColLessLongScalar(col 4:int, val 10) -> 19:boolean, IfExprNullNull(null, null) -> 23:string) -> 25:string) -> 23:string) -> 25:string, IfExprLongColumnLongColumn(col 17:boolean, col 18:date, col 19:date)(children: StringGroupColEqualCharScalar(col 14:char(10), val SHIP) -> 17:boolean, VectorUDFDateAddColScalar(co
 l 10:date, val 10) -> 18:date, VectorUDFDateAddColScalar(col 10:date, val 5) -> 19:date) -> 26:date, IfExprDoubleColumnLongScalar(col 17:boolean, col 28:double, val 0)(children: StringGroupColEqualCharScalar(col 8:char(1), val N) -> 17:boolean, DoubleColMultiplyDoubleColumn(col 5:double, col 27:double)(children: DoubleScalarSubtractDoubleColumn(val 1.0, col 6:double) -> 27:double) -> 28:double) -> 27:double, IfExprDoubleColumnDoubleScalar(col 17:boolean, col 29:double, val 0.0)(children: StringGroupColEqualCharScalar(col 8:char(1), val N) -> 17:boolean, DoubleColMultiplyDoubleColumn(col 5:double, col 28:double)(children: DoubleScalarSubtractDoubleColumn(val 1.0, col 6:double) -> 28:double) -> 29:double) -> 28:double, IfExprNullColumn(col 17:boolean, null, col 48)(children: StringGroupColEqualStringScalar(col 23:string, val DELIVER IN PERSON)(children: CastStringGroupToString(col 13:varchar(20)) -> 23:string) -> 17:boolean, ConvertDecimal64ToDecimal(col 7:decimal(10,2)/DECIMAL_64) ->
  48:decimal(10,2)) -> 30:decimal(10,2), IfExprColumnNull(col 18:boolean, col 49:decimal(10,2), null)(children: StringGroupColEqualStringScalar(col 23:string, val TAKE BACK RETURN)(children: CastStringGroupToString(col 13:varchar(20)) -> 23:string) -> 18:boolean, ConvertDecimal64ToDecimal(col 7:decimal(10,2)/DECIMAL_64) -> 49:decimal(10,2)) -> 31:decimal(10,2), VectorUDFAdaptor(if((CAST( l_shipinstruct AS STRING) = 'DELIVER IN PERSON'), 0, l_tax))(children: StringGroupColEqualStringScalar(col 23:string, val DELIVER IN PERSON)(children: CastStringGroupToString(col 13:varchar(20)) -> 23:string) -> 19:boolean) -> 32:decimal(12,2), VectorUDFAdaptor(if((CAST( l_shipinstruct AS STRING) = 'TAKE BACK RETURN'), l_tax, 0))(children: StringGroupColEqualStringScalar(col 23:string, val TAKE BACK RETURN)(children: CastStringGroupToString(col 13:varchar(20)) -> 23:string) -> 19:boolean) -> 33:decimal(12,2), IfExprDecimal64ScalarDecimal64Column(col 19:boolean, decimal64Val 0, decimalVal 0, col 7:dec
 imal(1,0)/DECIMAL_64)(children: StringGroupColEqualStringScalar(col 23:string, val DELIVER IN PERSON)(children: CastStringGroupToString(col 13:varchar(20)) -> 23:string) -> 19:boolean) -> 34:decimal(10,2)/DECIMAL_64, IfExprDecimal64ColumnDecimal64Scalar(col 35:boolean, col 7:decimal(10,2)/DECIMAL_64, decimal64Val 0, decimalVal 0)(children: StringGroupColEqualStringScalar(col 23:string, val TAKE BACK RETURN)(children: CastStringGroupToString(col 13:varchar(20)) -> 23:string) -> 35:boolean) -> 36:decimal(10,2)/DECIMAL_64, IfExprTimestampColumnColumn(col 37:boolean, col 38:timestampcol 39:timestamp)(children: LongColGreaterLongScalar(col 1:int, val 30) -> 37:boolean, CastDateToTimestamp(col 12:date) -> 38:timestamp, CastDateToTimestamp(col 11:date) -> 39:timestamp) -> 40:timestamp, IfExprColumnNull(col 37:boolean, col 41:int, null)(children: LongColGreaterLongScalar(col 2:int, val 10000) -> 37:boolean, VectorUDFDateDiffColCol(col 12:date, col 11:date) -> 41:int) -> 42:int, IfExprNullCo
 lumn(col 43:boolean, null, col 44)(children: LongColGreaterLongScalar(col 2:int, val 10000) -> 43:boolean, VectorUDFDateDiffColCol(col 12:date, col 11:date) -> 44:int) -> 45:int, IfExprLongScalarLongScalar(col 47:boolean, val 14245, val 14609)(children: LongColGreaterLongScalar(col 46:int, val 100)(children: LongColModuloLongScalar(col 2:int, val 500) -> 46:int) -> 47:boolean) -> 46:date
               Statistics: Num rows: 101 Data size: 78500 Basic stats: COMPLETE Column stats: NONE
               File Output Operator
                 compressed: false
@@ -543,7 +543,7 @@ STAGE PLANS:
               includeColumns: [1, 2, 4, 5, 6, 7, 8, 10, 11, 12, 13, 14]
               dataColumns: l_orderkey:int, l_partkey:int, l_suppkey:int, l_linenumber:int, l_quantity:int, l_extendedprice:double, l_discount:double, l_tax:decimal(10,2)/DECIMAL_64, l_returnflag:char(1), l_linestatus:char(1), l_shipdate:date, l_commitdate:date, l_receiptdate:date, l_shipinstruct:varchar(20), l_shipmode:char(10), l_comment:string
               partitionColumnCount: 0
-              scratchColumnTypeNames: [bigint, bigint, bigint, bigint, string, string, string, string, string, bigint, double, double, double, decimal(10,2), decimal(10,2), decimal(12,2), decimal(12,2), decimal(10,2), decimal(10,2), timestamp, timestamp, timestamp, bigint, bigint, bigint, bigint, bigint, bigint, bigint, decimal(10,2), decimal(10,2)]
+              scratchColumnTypeNames: [bigint, bigint, bigint, bigint, string, string, string, string, string, bigint, double, double, double, decimal(10,2), decimal(10,2), decimal(12,2), decimal(12,2), decimal(10,2)/DECIMAL_64, bigint, decimal(10,2)/DECIMAL_64, bigint, timestamp, timestamp, timestamp, bigint, bigint, bigint, bigint, bigint, bigint, bigint, decimal(10,2), decimal(10,2)]
 
   Stage: Stage-0
     Fetch Operator
@@ -844,8 +844,8 @@ STAGE PLANS:
               Select Vectorization:
                   className: VectorSelectOperator
                   native: true
-                  projectedOutputColumnNums: [4, 27, 38, 48, 52, 54, 60, 63, 65, 67, 68, 69, 70, 73, 76, 79, 80]
-                  selectExpressions: IfExprColumnCondExpr(col 17:boolean, col 18:stringcol 26:string)(children: LongColEqualLongScalar(col 4:int, val 1) -> 17:boolean, ConstantVectorExpression(val Single) -> 18:string, IfExprColumnCondExpr(col 19:boolean, col 20:stringcol 25:string)(children: LongColEqualLongScalar(col 4:int, val 2) -> 19:boolean, ConstantVectorExpression(val Two) -> 20:string, IfExprColumnCondExpr(col 21:boolean, col 22:stringcol 24:string)(children: LongColLessLongScalar(col 4:int, val 10) -> 21:boolean, ConstantVectorExpression(val Some) -> 22:string, IfExprStringScalarStringScalar(col 23:boolean, val Many, val Huge number)(children: LongColLessLongScalar(col 4:int, val 100) -> 23:boolean) -> 24:string) -> 25:string) -> 26:string) -> 27:string, IfExprColumnCondExpr(col 23:boolean, col 28:stringcol 37:string)(children: LongColEqualLongScalar(col 4:int, val 1) -> 23:boolean, ConstantVectorExpression(val Single) -> 28:string, IfExprColumnCondExpr(col 29:boolean, col
  30:stringcol 36:string)(children: LongColEqualLongScalar(col 4:int, val 2) -> 29:boolean, ConstantVectorExpression(val Two) -> 30:string, IfExprColumnCondExpr(col 31:boolean, col 32:stringcol 35:string)(children: LongColLessLongScalar(col 4:int, val 10) -> 31:boolean, ConstantVectorExpression(val Some) -> 32:string, IfExprColumnNull(col 33:boolean, col 34:string, null)(children: LongColLessLongScalar(col 4:int, val 100) -> 33:boolean, ConstantVectorExpression(val Many) -> 34:string) -> 35:string) -> 36:string) -> 37:string) -> 38:string, IfExprColumnCondExpr(col 39:boolean, col 40:stringcol 47:string)(children: LongColEqualLongScalar(col 4:int, val 1) -> 39:boolean, ConstantVectorExpression(val Single) -> 40:string, IfExprColumnCondExpr(col 41:boolean, col 42:stringcol 46:string)(children: LongColEqualLongScalar(col 4:int, val 2) -> 41:boolean, ConstantVectorExpression(val Two) -> 42:string, IfExprColumnCondExpr(col 43:boolean, col 44:stringcol 45:string)(children: LongColLessLongS
 calar(col 4:int, val 10) -> 43:boolean, ConstantVectorExpression(val Some) -> 44:string, IfExprNullNull(null, null) -> 45:string) -> 46:string) -> 47:string) -> 48:string, IfExprCondExprCondExpr(col 49:boolean, col 50:datecol 51:date)(children: StringGroupColEqualCharScalar(col 14:char(10), val SHIP) -> 49:boolean, VectorUDFDateAddColScalar(col 10:date, val 10) -> 50:date, VectorUDFDateAddColScalar(col 10:date, val 5) -> 51:date) -> 52:date, IfExprDoubleColumnLongScalar(col 57:boolean, col 58:double, val 0)(children: StringGroupColEqualCharScalar(col 8:char(1), val N) -> 57:boolean, DoubleColMultiplyDoubleColumn(col 5:double, col 54:double)(children: DoubleScalarSubtractDoubleColumn(val 1.0, col 6:double) -> 54:double) -> 58:double) -> 54:double, IfExprCondExprColumn(col 57:boolean, col 59:double, col 58:double)(children: StringGroupColEqualCharScalar(col 8:char(1), val N) -> 57:boolean, DoubleColMultiplyDoubleColumn(col 5:double, col 58:double)(children: DoubleScalarSubtractDoubleC
 olumn(val 1.0, col 6:double) -> 58:double) -> 59:double, ConstantVectorExpression(val 0.0) -> 58:double) -> 60:double, IfExprNullColumn(col 62:boolean, null, col 82)(children: StringGroupColEqualStringScalar(col 61:string, val DELIVER IN PERSON)(children: CastStringGroupToString(col 13:varchar(20)) -> 61:string) -> 62:boolean, ConvertDecimal64ToDecimal(col 7:decimal(10,2)/DECIMAL_64) -> 82:decimal(10,2)) -> 63:decimal(10,2), IfExprColumnNull(col 64:boolean, col 83:decimal(10,2), null)(children: StringGroupColEqualStringScalar(col 61:string, val TAKE BACK RETURN)(children: CastStringGroupToString(col 13:varchar(20)) -> 61:string) -> 64:boolean, ConvertDecimal64ToDecimal(col 7:decimal(10,2)/DECIMAL_64) -> 83:decimal(10,2)) -> 65:decimal(10,2), VectorUDFAdaptor(if((CAST( l_shipinstruct AS STRING) = 'DELIVER IN PERSON'), 0, l_tax))(children: StringGroupColEqualStringScalar(col 61:string, val DELIVER IN PERSON)(children: CastStringGroupToString(col 13:varchar(20)) -> 61:string) -> 66:boo
 lean) -> 67:decimal(12,2), VectorUDFAdaptor(if((CAST( l_shipinstruct AS STRING) = 'TAKE BACK RETURN'), l_tax, 0))(children: StringGroupColEqualStringScalar(col 61:string, val TAKE BACK RETURN)(children: CastStringGroupToString(col 13:varchar(20)) -> 61:string) -> 66:boolean) -> 68:decimal(12,2), VectorUDFAdaptor(if((CAST( l_shipinstruct AS STRING) = 'DELIVER IN PERSON'), 0, l_tax))(children: StringGroupColEqualStringScalar(col 61:string, val DELIVER IN PERSON)(children: CastStringGroupToString(col 13:varchar(20)) -> 61:string) -> 66:boolean) -> 69:decimal(10,2), VectorUDFAdaptor(if((CAST( l_shipinstruct AS STRING) = 'TAKE BACK RETURN'), l_tax, 0))(children: StringGroupColEqualStringScalar(col 61:string, val TAKE BACK RETURN)(children: CastStringGroupToString(col 13:varchar(20)) -> 61:string) -> 66:boolean) -> 70:decimal(10,2), IfExprCondExprCondExpr(col 66:boolean, col 71:timestampcol 72:timestamp)(children: LongColGreaterLongScalar(col 1:int, val 30) -> 66:boolean, CastDateToTimest
 amp(col 12:date) -> 71:timestamp, CastDateToTimestamp(col 11:date) -> 72:timestamp) -> 73:timestamp, IfExprCondExprNull(col 74:boolean, col 75:int, null)(children: LongColGreaterLongScalar(col 2:int, val 10000) -> 74:boolean, VectorUDFDateDiffColCol(col 12:date, col 11:date) -> 75:int) -> 76:int, IfExprNullCondExpr(col 77:boolean, null, col 78:int)(children: LongColGreaterLongScalar(col 2:int, val 10000) -> 77:boolean, VectorUDFDateDiffColCol(col 12:date, col 11:date) -> 78:int) -> 79:int, IfExprLongScalarLongScalar(col 81:boolean, val 14245, val 14609)(children: LongColGreaterLongScalar(col 80:int, val 100)(children: LongColModuloLongScalar(col 2:int, val 500) -> 80:int) -> 81:boolean) -> 80:date
+                  projectedOutputColumnNums: [4, 27, 38, 48, 52, 54, 60, 63, 65, 67, 68, 69, 71, 75, 78, 81, 82]
+                  selectExpressions: IfExprColumnCondExpr(col 17:boolean, col 18:stringcol 26:string)(children: LongColEqualLongScalar(col 4:int, val 1) -> 17:boolean, ConstantVectorExpression(val Single) -> 18:string, IfExprColumnCondExpr(col 19:boolean, col 20:stringcol 25:string)(children: LongColEqualLongScalar(col 4:int, val 2) -> 19:boolean, ConstantVectorExpression(val Two) -> 20:string, IfExprColumnCondExpr(col 21:boolean, col 22:stringcol 24:string)(children: LongColLessLongScalar(col 4:int, val 10) -> 21:boolean, ConstantVectorExpression(val Some) -> 22:string, IfExprStringScalarStringScalar(col 23:boolean, val Many, val Huge number)(children: LongColLessLongScalar(col 4:int, val 100) -> 23:boolean) -> 24:string) -> 25:string) -> 26:string) -> 27:string, IfExprColumnCondExpr(col 23:boolean, col 28:stringcol 37:string)(children: LongColEqualLongScalar(col 4:int, val 1) -> 23:boolean, ConstantVectorExpression(val Single) -> 28:string, IfExprColumnCondExpr(col 29:boolean, col
  30:stringcol 36:string)(children: LongColEqualLongScalar(col 4:int, val 2) -> 29:boolean, ConstantVectorExpression(val Two) -> 30:string, IfExprColumnCondExpr(col 31:boolean, col 32:stringcol 35:string)(children: LongColLessLongScalar(col 4:int, val 10) -> 31:boolean, ConstantVectorExpression(val Some) -> 32:string, IfExprColumnNull(col 33:boolean, col 34:string, null)(children: LongColLessLongScalar(col 4:int, val 100) -> 33:boolean, ConstantVectorExpression(val Many) -> 34:string) -> 35:string) -> 36:string) -> 37:string) -> 38:string, IfExprColumnCondExpr(col 39:boolean, col 40:stringcol 47:string)(children: LongColEqualLongScalar(col 4:int, val 1) -> 39:boolean, ConstantVectorExpression(val Single) -> 40:string, IfExprColumnCondExpr(col 41:boolean, col 42:stringcol 46:string)(children: LongColEqualLongScalar(col 4:int, val 2) -> 41:boolean, ConstantVectorExpression(val Two) -> 42:string, IfExprColumnCondExpr(col 43:boolean, col 44:stringcol 45:string)(children: LongColLessLongS
 calar(col 4:int, val 10) -> 43:boolean, ConstantVectorExpression(val Some) -> 44:string, IfExprNullNull(null, null) -> 45:string) -> 46:string) -> 47:string) -> 48:string, IfExprCondExprCondExpr(col 49:boolean, col 50:datecol 51:date)(children: StringGroupColEqualCharScalar(col 14:char(10), val SHIP) -> 49:boolean, VectorUDFDateAddColScalar(col 10:date, val 10) -> 50:date, VectorUDFDateAddColScalar(col 10:date, val 5) -> 51:date) -> 52:date, IfExprDoubleColumnLongScalar(col 57:boolean, col 58:double, val 0)(children: StringGroupColEqualCharScalar(col 8:char(1), val N) -> 57:boolean, DoubleColMultiplyDoubleColumn(col 5:double, col 54:double)(children: DoubleScalarSubtractDoubleColumn(val 1.0, col 6:double) -> 54:double) -> 58:double) -> 54:double, IfExprCondExprColumn(col 57:boolean, col 59:double, col 58:double)(children: StringGroupColEqualCharScalar(col 8:char(1), val N) -> 57:boolean, DoubleColMultiplyDoubleColumn(col 5:double, col 58:double)(children: DoubleScalarSubtractDoubleC
 olumn(val 1.0, col 6:double) -> 58:double) -> 59:double, ConstantVectorExpression(val 0.0) -> 58:double) -> 60:double, IfExprNullColumn(col 62:boolean, null, col 84)(children: StringGroupColEqualStringScalar(col 61:string, val DELIVER IN PERSON)(children: CastStringGroupToString(col 13:varchar(20)) -> 61:string) -> 62:boolean, ConvertDecimal64ToDecimal(col 7:decimal(10,2)/DECIMAL_64) -> 84:decimal(10,2)) -> 63:decimal(10,2), IfExprColumnNull(col 64:boolean, col 85:decimal(10,2), null)(children: StringGroupColEqualStringScalar(col 61:string, val TAKE BACK RETURN)(children: CastStringGroupToString(col 13:varchar(20)) -> 61:string) -> 64:boolean, ConvertDecimal64ToDecimal(col 7:decimal(10,2)/DECIMAL_64) -> 85:decimal(10,2)) -> 65:decimal(10,2), VectorUDFAdaptor(if((CAST( l_shipinstruct AS STRING) = 'DELIVER IN PERSON'), 0, l_tax))(children: StringGroupColEqualStringScalar(col 61:string, val DELIVER IN PERSON)(children: CastStringGroupToString(col 13:varchar(20)) -> 61:string) -> 66:boo
 lean) -> 67:decimal(12,2), VectorUDFAdaptor(if((CAST( l_shipinstruct AS STRING) = 'TAKE BACK RETURN'), l_tax, 0))(children: StringGroupColEqualStringScalar(col 61:string, val TAKE BACK RETURN)(children: CastStringGroupToString(col 13:varchar(20)) -> 61:string) -> 66:boolean) -> 68:decimal(12,2), IfExprDecimal64ScalarDecimal64Column(col 66:boolean, decimal64Val 0, decimalVal 0, col 7:decimal(1,0)/DECIMAL_64)(children: StringGroupColEqualStringScalar(col 61:string, val DELIVER IN PERSON)(children: CastStringGroupToString(col 13:varchar(20)) -> 61:string) -> 66:boolean) -> 69:decimal(10,2)/DECIMAL_64, IfExprDecimal64ColumnDecimal64Scalar(col 70:boolean, col 7:decimal(10,2)/DECIMAL_64, decimal64Val 0, decimalVal 0)(children: StringGroupColEqualStringScalar(col 61:string, val TAKE BACK RETURN)(children: CastStringGroupToString(col 13:varchar(20)) -> 61:string) -> 70:boolean) -> 71:decimal(10,2)/DECIMAL_64, IfExprCondExprCondExpr(col 72:boolean, col 73:timestampcol 74:timestamp)(children:
  LongColGreaterLongScalar(col 1:int, val 30) -> 72:boolean, CastDateToTimestamp(col 12:date) -> 73:timestamp, CastDateToTimestamp(col 11:date) -> 74:timestamp) -> 75:timestamp, IfExprCondExprNull(col 76:boolean, col 77:int, null)(children: LongColGreaterLongScalar(col 2:int, val 10000) -> 76:boolean, VectorUDFDateDiffColCol(col 12:date, col 11:date) -> 77:int) -> 78:int, IfExprNullCondExpr(col 79:boolean, null, col 80:int)(children: LongColGreaterLongScalar(col 2:int, val 10000) -> 79:boolean, VectorUDFDateDiffColCol(col 12:date, col 11:date) -> 80:int) -> 81:int, IfExprLongScalarLongScalar(col 83:boolean, val 14245, val 14609)(children: LongColGreaterLongScalar(col 82:int, val 100)(children: LongColModuloLongScalar(col 2:int, val 500) -> 82:int) -> 83:boolean) -> 82:date
               Statistics: Num rows: 101 Data size: 78500 Basic stats: COMPLETE Column stats: NONE
               File Output Operator
                 compressed: false
@@ -872,7 +872,7 @@ STAGE PLANS:
               includeColumns: [1, 2, 4, 5, 6, 7, 8, 10, 11, 12, 13, 14]
               dataColumns: l_orderkey:int, l_partkey:int, l_suppkey:int, l_linenumber:int, l_quantity:int, l_extendedprice:double, l_discount:double, l_tax:decimal(10,2)/DECIMAL_64, l_returnflag:char(1), l_linestatus:char(1), l_shipdate:date, l_commitdate:date, l_receiptdate:date, l_shipinstruct:varchar(20), l_shipmode:char(10), l_comment:string
               partitionColumnCount: 0
-              scratchColumnTypeNames: [bigint, string, bigint, string, bigint, string, bigint, string, string, string, string, string, bigint, string, bigint, string, bigint, string, string, string, string, string, bigint, string, bigint, string, bigint, string, string, string, string, string, bigint, bigint, bigint, bigint, bigint, double, double, bigint, bigint, double, double, double, string, bigint, decimal(10,2), bigint, decimal(10,2), bigint, decimal(12,2), decimal(12,2), decimal(10,2), decimal(10,2), timestamp, timestamp, timestamp, bigint, bigint, bigint, bigint, bigint, bigint, bigint, bigint, decimal(10,2), decimal(10,2)]
+              scratchColumnTypeNames: [bigint, string, bigint, string, bigint, string, bigint, string, string, string, string, string, bigint, string, bigint, string, bigint, string, string, string, string, string, bigint, string, bigint, string, bigint, string, string, string, string, string, bigint, bigint, bigint, bigint, bigint, double, double, bigint, bigint, double, double, double, string, bigint, decimal(10,2), bigint, decimal(10,2), bigint, decimal(12,2), decimal(12,2), decimal(10,2)/DECIMAL_64, bigint, decimal(10,2)/DECIMAL_64, bigint, timestamp, timestamp, timestamp, bigint, bigint, bigint, bigint, bigint, bigint, bigint, bigint, decimal(10,2), decimal(10,2)]
 
   Stage: Stage-0
     Fetch Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/test/results/clientpositive/vector_decimal_mapjoin.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vector_decimal_mapjoin.q.out b/ql/src/test/results/clientpositive/vector_decimal_mapjoin.q.out
index 0f090bf..aa9f54b 100644
--- a/ql/src/test/results/clientpositive/vector_decimal_mapjoin.q.out
+++ b/ql/src/test/results/clientpositive/vector_decimal_mapjoin.q.out
@@ -690,7 +690,7 @@ STAGE PLANS:
               Filter Vectorization:
                   className: VectorFilterOperator
                   native: true
-                  predicateExpression: SelectColumnIsNotNull(col 3:decimal(14,2))(children: ConvertDecimal64ToDecimal(col 0:decimal(14,2)/DECIMAL_64) -> 3:decimal(14,2))
+                  predicateExpression: SelectColumnIsNotNull(col 0:decimal(14,2)/DECIMAL_64)
               predicate: dec is not null (type: boolean)
               Statistics: Num rows: 1049 Data size: 11234 Basic stats: COMPLETE Column stats: NONE
               Select Operator
@@ -708,8 +708,8 @@ STAGE PLANS:
                     0 _col0 (type: decimal(16,2))
                     1 _col0 (type: decimal(16,2))
                   Map Join Vectorization:
-                      bigTableKeyExpressions: ConvertDecimal64ToDecimal(col 0:decimal(16,2)/DECIMAL_64) -> 4:decimal(16,2)
-                      bigTableValueExpressions: ConvertDecimal64ToDecimal(col 0:decimal(16,2)/DECIMAL_64) -> 5:decimal(16,2)
+                      bigTableKeyExpressions: ConvertDecimal64ToDecimal(col 0:decimal(16,2)/DECIMAL_64) -> 3:decimal(16,2)
+                      bigTableValueExpressions: ConvertDecimal64ToDecimal(col 0:decimal(16,2)/DECIMAL_64) -> 4:decimal(16,2)
                       className: VectorMapJoinOperator
                       native: false
                       nativeConditionsMet: hive.mapjoin.optimized.hashtable IS true, hive.vectorized.execution.mapjoin.native.enabled IS true, One MapJoin Condition IS true, No nullsafe IS true, Small table vectorizes IS true
@@ -742,7 +742,7 @@ STAGE PLANS:
               includeColumns: [0]
               dataColumns: dec:decimal(14,2)/DECIMAL_64, value_dec:decimal(14,2)/DECIMAL_64
               partitionColumnCount: 0
-              scratchColumnTypeNames: [decimal(14,2), decimal(16,2), decimal(16,2)]
+              scratchColumnTypeNames: [decimal(16,2), decimal(16,2)]
       Local Work:
         Map Reduce Local Work
 
@@ -933,7 +933,7 @@ STAGE PLANS:
               Filter Vectorization:
                   className: VectorFilterOperator
                   native: true
-                  predicateExpression: SelectColumnIsNotNull(col 3:decimal(14,2))(children: ConvertDecimal64ToDecimal(col 0:decimal(14,2)/DECIMAL_64) -> 3:decimal(14,2))
+                  predicateExpression: SelectColumnIsNotNull(col 0:decimal(14,2)/DECIMAL_64)
               predicate: dec is not null (type: boolean)
               Statistics: Num rows: 1049 Data size: 11234 Basic stats: COMPLETE Column stats: NONE
               Select Operator
@@ -951,8 +951,8 @@ STAGE PLANS:
                     0 _col0 (type: decimal(16,2))
                     1 _col0 (type: decimal(16,2))
                   Map Join Vectorization:
-                      bigTableKeyExpressions: ConvertDecimal64ToDecimal(col 0:decimal(16,2)/DECIMAL_64) -> 4:decimal(16,2)
-                      bigTableValueExpressions: ConvertDecimal64ToDecimal(col 0:decimal(16,2)/DECIMAL_64) -> 5:decimal(16,2), ConvertDecimal64ToDecimal(col 1:decimal(14,2)/DECIMAL_64) -> 3:decimal(14,2)
+                      bigTableKeyExpressions: ConvertDecimal64ToDecimal(col 0:decimal(16,2)/DECIMAL_64) -> 3:decimal(16,2)
+                      bigTableValueExpressions: ConvertDecimal64ToDecimal(col 0:decimal(16,2)/DECIMAL_64) -> 4:decimal(16,2), ConvertDecimal64ToDecimal(col 1:decimal(14,2)/DECIMAL_64) -> 5:decimal(14,2)
                       className: VectorMapJoinOperator
                       native: false
                       nativeConditionsMet: hive.mapjoin.optimized.hashtable IS true, hive.vectorized.execution.mapjoin.native.enabled IS true, One MapJoin Condition IS true, No nullsafe IS true, Small table vectorizes IS true
@@ -985,7 +985,7 @@ STAGE PLANS:
               includeColumns: [0, 1]
               dataColumns: dec:decimal(14,2)/DECIMAL_64, value_dec:decimal(14,2)/DECIMAL_64
               partitionColumnCount: 0
-              scratchColumnTypeNames: [decimal(14,2), decimal(16,2), decimal(16,2), decimal(14,0)]
+              scratchColumnTypeNames: [decimal(16,2), decimal(16,2), decimal(14,2), decimal(14,0)]
       Local Work:
         Map Reduce Local Work
 

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/test/results/clientpositive/vectorized_case.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vectorized_case.q.out b/ql/src/test/results/clientpositive/vectorized_case.q.out
index 625162e..828131f 100644
--- a/ql/src/test/results/clientpositive/vectorized_case.q.out
+++ b/ql/src/test/results/clientpositive/vectorized_case.q.out
@@ -527,7 +527,7 @@ STAGE PLANS:
                   className: VectorSelectOperator
                   native: true
                   projectedOutputColumnNums: [6]
-                  selectExpressions: IfExprCondExprCondExpr(col 3:boolean, col 7:decimal(11,0)col 8:decimal(11,0))(children: VectorUDFAdaptor((member = 1)) -> 3:boolean, ConvertDecimal64ToDecimal(col 4:decimal(11,0)/DECIMAL_64)(children: Decimal64ColAddDecimal64Scalar(col 1:decimal(10,0)/DECIMAL_64, decimal64Val 1, decimalVal 1) -> 4:decimal(11,0)/DECIMAL_64) -> 7:decimal(11,0), ConvertDecimal64ToDecimal(col 5:decimal(11,0)/DECIMAL_64)(children: Decimal64ColAddDecimal64Scalar(col 1:decimal(10,0)/DECIMAL_64, decimal64Val 2, decimalVal 2) -> 5:decimal(11,0)/DECIMAL_64) -> 8:decimal(11,0)) -> 6:decimal(11,0)
+                  selectExpressions: IfExprCondExprCondExpr(col 3:boolean, col 7:decimal(11,0)col 8:decimal(11,0))(children: Decimal64ColEqualDecimal64Scalar(col 0:decimal(10,0)/DECIMAL_64, decimal64Val 1, decimalVal 1) -> 3:boolean, ConvertDecimal64ToDecimal(col 4:decimal(11,0)/DECIMAL_64)(children: Decimal64ColAddDecimal64Scalar(col 1:decimal(10,0)/DECIMAL_64, decimal64Val 1, decimalVal 1) -> 4:decimal(11,0)/DECIMAL_64) -> 7:decimal(11,0), ConvertDecimal64ToDecimal(col 5:decimal(11,0)/DECIMAL_64)(children: Decimal64ColAddDecimal64Scalar(col 1:decimal(10,0)/DECIMAL_64, decimal64Val 2, decimalVal 2) -> 5:decimal(11,0)/DECIMAL_64) -> 8:decimal(11,0)) -> 6:decimal(11,0)
               Statistics: Num rows: 3 Data size: 672 Basic stats: COMPLETE Column stats: NONE
               File Output Operator
                 compressed: false
@@ -547,7 +547,7 @@ STAGE PLANS:
           featureSupportInUse: [DECIMAL_64]
           inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
           allNative: false
-          usesVectorUDFAdaptor: true
+          usesVectorUDFAdaptor: false
           vectorized: true
           rowBatchContext:
               dataColumnCount: 2
@@ -604,7 +604,7 @@ STAGE PLANS:
                   className: VectorSelectOperator
                   native: true
                   projectedOutputColumnNums: [8]
-                  selectExpressions: VectorUDFAdaptor(CASE WHEN ((member = 1)) THEN (1) ELSE ((attr + 2)) END)(children: VectorUDFAdaptor((member = 1)) -> 6:boolean, ConvertDecimal64ToDecimal(col 7:decimal(11,0)/DECIMAL_64)(children: Decimal64ColAddDecimal64Scalar(col 1:decimal(10,0)/DECIMAL_64, decimal64Val 2, decimalVal 2) -> 7:decimal(11,0)/DECIMAL_64) -> 9:decimal(11,0)) -> 8:decimal(11,0)
+                  selectExpressions: IfExprDecimal64ScalarDecimal64Column(col 6:boolean, decimal64Val 1, decimalVal 1, col 7:decimal(1,0)/DECIMAL_64)(children: Decimal64ColEqualDecimal64Scalar(col 0:decimal(10,0)/DECIMAL_64, decimal64Val 1, decimalVal 1) -> 6:boolean, Decimal64ColAddDecimal64Scalar(col 1:decimal(10,0)/DECIMAL_64, decimal64Val 2, decimalVal 2) -> 7:decimal(11,0)/DECIMAL_64) -> 8:decimal(11,0)/DECIMAL_64
               Statistics: Num rows: 3 Data size: 672 Basic stats: COMPLETE Column stats: NONE
               File Output Operator
                 compressed: false
@@ -624,14 +624,14 @@ STAGE PLANS:
           featureSupportInUse: [DECIMAL_64]
           inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
           allNative: false
-          usesVectorUDFAdaptor: true
+          usesVectorUDFAdaptor: false
           vectorized: true
           rowBatchContext:
               dataColumnCount: 2
               includeColumns: [0, 1]
               dataColumns: member:decimal(10,0)/DECIMAL_64, attr:decimal(10,0)/DECIMAL_64
               partitionColumnCount: 0
-              scratchColumnTypeNames: [bigint, decimal(1,0), decimal(11,0)/DECIMAL_64, bigint, decimal(11,0)/DECIMAL_64, decimal(11,0), decimal(11,0)]
+              scratchColumnTypeNames: [bigint, decimal(1,0), decimal(11,0)/DECIMAL_64, bigint, decimal(11,0)/DECIMAL_64, decimal(11,0)/DECIMAL_64]
 
   Stage: Stage-0
     Fetch Operator
@@ -681,7 +681,7 @@ STAGE PLANS:
                   className: VectorSelectOperator
                   native: true
                   projectedOutputColumnNums: [8]
-                  selectExpressions: VectorUDFAdaptor(CASE WHEN ((member = 1)) THEN ((attr + 1)) ELSE (2) END)(children: VectorUDFAdaptor((member = 1)) -> 6:boolean, ConvertDecimal64ToDecimal(col 7:decimal(11,0)/DECIMAL_64)(children: Decimal64ColAddDecimal64Scalar(col 1:decimal(10,0)/DECIMAL_64, decimal64Val 1, decimalVal 1) -> 7:decimal(11,0)/DECIMAL_64) -> 9:decimal(11,0)) -> 8:decimal(11,0)
+                  selectExpressions: IfExprDecimal64ColumnDecimal64Scalar(col 6:boolean, col 7:decimal(11,0)/DECIMAL_64, decimal64Val 2, decimalVal 2)(children: Decimal64ColEqualDecimal64Scalar(col 0:decimal(10,0)/DECIMAL_64, decimal64Val 1, decimalVal 1) -> 6:boolean, Decimal64ColAddDecimal64Scalar(col 1:decimal(10,0)/DECIMAL_64, decimal64Val 1, decimalVal 1) -> 7:decimal(11,0)/DECIMAL_64) -> 8:decimal(11,0)/DECIMAL_64
               Statistics: Num rows: 3 Data size: 672 Basic stats: COMPLETE Column stats: NONE
               File Output Operator
                 compressed: false
@@ -701,14 +701,14 @@ STAGE PLANS:
           featureSupportInUse: [DECIMAL_64]
           inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
           allNative: false
-          usesVectorUDFAdaptor: true
+          usesVectorUDFAdaptor: false
           vectorized: true
           rowBatchContext:
               dataColumnCount: 2
               includeColumns: [0, 1]
               dataColumns: member:decimal(10,0)/DECIMAL_64, attr:decimal(10,0)/DECIMAL_64
               partitionColumnCount: 0
-              scratchColumnTypeNames: [bigint, decimal(11,0)/DECIMAL_64, decimal(1,0), bigint, decimal(11,0)/DECIMAL_64, decimal(11,0), decimal(11,0)]
+              scratchColumnTypeNames: [bigint, decimal(11,0)/DECIMAL_64, decimal(1,0), bigint, decimal(11,0)/DECIMAL_64, decimal(11,0)/DECIMAL_64]
 
   Stage: Stage-0
     Fetch Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/test/results/clientpositive/vectorized_casts.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vectorized_casts.q.out b/ql/src/test/results/clientpositive/vectorized_casts.q.out
index 4c82e17..99694ad 100644
--- a/ql/src/test/results/clientpositive/vectorized_casts.q.out
+++ b/ql/src/test/results/clientpositive/vectorized_casts.q.out
@@ -181,7 +181,7 @@ STAGE PLANS:
                     className: VectorSelectOperator
                     native: true
                     projectedOutputColumnNums: [13, 14, 15, 16, 17, 18, 10, 20, 19, 21, 0, 1, 2, 3, 22, 23, 10, 24, 25, 27, 28, 29, 30, 31, 32, 33, 34, 4, 5, 35, 36, 37, 38, 39, 5, 41, 43, 45, 47, 48, 49, 51, 54, 55, 8, 56, 57, 26, 58, 59, 60, 61, 62, 63, 64, 65, 6, 67, 68, 69, 70, 66, 73]
-                    selectExpressions: CastLongToBooleanViaLongToLong(col 0:tinyint) -> 13:boolean, CastLongToBooleanViaLongToLong(col 1:smallint) -> 14:boolean, CastLongToBooleanViaLongToLong(col 2:int) -> 15:boolean, CastLongToBooleanViaLongToLong(col 3:bigint) -> 16:boolean, CastDoubleToBooleanViaDoubleToLong(col 4:float) -> 17:boolean, CastDoubleToBooleanViaDoubleToLong(col 5:double) -> 18:boolean, CastLongToBooleanViaLongToLong(col 19:bigint)(children: LongColMultiplyLongScalar(col 3:bigint, val 0) -> 19:bigint) -> 20:boolean, CastTimestampToBoolean(col 8:timestamp) -> 19:boolean, CastStringToBoolean(col 6) -> 21:boolean, CastDoubleToLong(col 4:float) -> 22:int, CastDoubleToLong(col 5:double) -> 23:int, CastTimestampToLong(col 8:timestamp) -> 24:int, CastStringToLong(col 6:string) -> 25:int, CastStringToLong(col 26:string)(children: StringSubstrColStartLen(col 6:string, start 0, length 1) -> 26:string) -> 27:int, CastDoubleToLong(col 4:float) -> 28:tinyint, CastDoubleToLong(col
  4:float) -> 29:smallint, CastDoubleToLong(col 4:float) -> 30:bigint, CastLongToDouble(col 0:tinyint) -> 31:double, CastLongToDouble(col 1:smallint) -> 32:double, CastLongToDouble(col 2:int) -> 33:double, CastLongToDouble(col 3:bigint) -> 34:double, CastLongToDouble(col 10:boolean) -> 35:double, CastTimestampToDouble(col 8:timestamp) -> 36:double, CastStringToDouble(col 6:string) -> 37:double, CastStringToDouble(col 26:string)(children: StringSubstrColStartLen(col 6:string, start 0, length 1) -> 26:string) -> 38:double, CastLongToFloatViaLongToDouble(col 2:int) -> 39:float, CastMillisecondsLongToTimestamp(col 0:tinyint) -> 41:timestamp, CastMillisecondsLongToTimestamp(col 1:smallint) -> 43:timestamp, CastMillisecondsLongToTimestamp(col 2:int) -> 45:timestamp, CastMillisecondsLongToTimestamp(col 3:bigint) -> 47:timestamp, CastDoubleToTimestamp(col 4:float) -> 48:timestamp, CastDoubleToTimestamp(col 5:double) -> 49:timestamp, CastMillisecondsLongToTimestamp(col 10:boolean) -> 51:times
 tamp, CastMillisecondsLongToTimestamp(col 52:bigint)(children: LongColMultiplyLongScalar(col 3:bigint, val 0) -> 52:bigint) -> 54:timestamp, CastDateToTimestamp(col 52:date)(children: CastTimestampToDate(col 8:timestamp) -> 52:date) -> 55:timestamp, VectorUDFAdaptor(CAST( cstring1 AS TIMESTAMP)) -> 56:timestamp, VectorUDFAdaptor(CAST( substr(cstring1, 1, 1) AS TIMESTAMP))(children: StringSubstrColStartLen(col 6:string, start 0, length 1) -> 26:string) -> 57:timestamp, CastLongToString(col 0:tinyint) -> 26:string, CastLongToString(col 1:smallint) -> 58:string, CastLongToString(col 2:int) -> 59:string, CastLongToString(col 3:bigint) -> 60:string, CastFloatToString(col 4:float) -> 61:string, CastDoubleToString(col 5:double) -> 62:string, CastBooleanToStringViaLongToString(col 10:boolean) -> 63:string, CastLongToString(col 52:bigint)(children: LongColMultiplyLongScalar(col 3:bigint, val 0) -> 52:bigint) -> 64:string, CastTimestampToString(col 8:timestamp) -> 65:string, CastStringGroupTo
 String(col 66:char(10))(children: CastStringGroupToChar(col 6:string, maxLength 10) -> 66:char(10)) -> 67:string, CastStringGroupToString(col 66:varchar(10))(children: CastStringGroupToVarChar(col 6:string, maxLength 10) -> 66:varchar(10)) -> 68:string, CastLongToFloatViaLongToDouble(col 52:int)(children: CastDoubleToLong(col 4:float) -> 52:int) -> 69:float, CastLongToDouble(col 52:int)(children: LongColMultiplyLongScalar(col 2:int, val 2) -> 52:int) -> 70:double, CastDoubleToString(col 71:double)(children: FuncSinDoubleToDouble(col 4:float) -> 71:double) -> 66:string, DoubleColAddDoubleColumn(col 71:double, col 72:double)(children: col 71:float, CastLongToDouble(col 10:boolean) -> 72:double) -> 73:double
+                    selectExpressions: CastLongToBooleanViaLongToLong(col 0:tinyint) -> 13:boolean, CastLongToBooleanViaLongToLong(col 1:smallint) -> 14:boolean, CastLongToBooleanViaLongToLong(col 2:int) -> 15:boolean, CastLongToBooleanViaLongToLong(col 3:bigint) -> 16:boolean, CastDoubleToBooleanViaDoubleToLong(col 4:float) -> 17:boolean, CastDoubleToBooleanViaDoubleToLong(col 5:double) -> 18:boolean, CastLongToBooleanViaLongToLong(col 19:bigint)(children: LongColMultiplyLongScalar(col 3:bigint, val 0) -> 19:bigint) -> 20:boolean, CastTimestampToBoolean(col 8:timestamp) -> 19:boolean, CastStringToBoolean(col 6) -> 21:boolean, CastDoubleToLong(col 4:float) -> 22:int, CastDoubleToLong(col 5:double) -> 23:int, CastTimestampToLong(col 8:timestamp) -> 24:int, CastStringToLong(col 6:string) -> 25:int, CastStringToLong(col 26:string)(children: StringSubstrColStartLen(col 6:string, start 0, length 1) -> 26:string) -> 27:int, CastDoubleToLong(col 4:float) -> 28:tinyint, CastDoubleToLong(col
  4:float) -> 29:smallint, CastDoubleToLong(col 4:float) -> 30:bigint, CastLongToDouble(col 0:tinyint) -> 31:double, CastLongToDouble(col 1:smallint) -> 32:double, CastLongToDouble(col 2:int) -> 33:double, CastLongToDouble(col 3:bigint) -> 34:double, CastLongToDouble(col 10:boolean) -> 35:double, CastTimestampToDouble(col 8:timestamp) -> 36:double, CastStringToDouble(col 6:string) -> 37:double, CastStringToDouble(col 26:string)(children: StringSubstrColStartLen(col 6:string, start 0, length 1) -> 26:string) -> 38:double, CastLongToFloatViaLongToDouble(col 2:int) -> 39:float, CastMillisecondsLongToTimestamp(col 0:tinyint) -> 41:timestamp, CastMillisecondsLongToTimestamp(col 1:smallint) -> 43:timestamp, CastMillisecondsLongToTimestamp(col 2:int) -> 45:timestamp, CastMillisecondsLongToTimestamp(col 3:bigint) -> 47:timestamp, CastDoubleToTimestamp(col 4:float) -> 48:timestamp, CastDoubleToTimestamp(col 5:double) -> 49:timestamp, CastMillisecondsLongToTimestamp(col 10:boolean) -> 51:times
 tamp, CastMillisecondsLongToTimestamp(col 52:bigint)(children: LongColMultiplyLongScalar(col 3:bigint, val 0) -> 52:bigint) -> 54:timestamp, CastDateToTimestamp(col 52:date)(children: CastTimestampToDate(col 8:timestamp) -> 52:date) -> 55:timestamp, CastStringToTimestamp(col 6:string) -> 56:timestamp, CastStringToTimestamp(col 26:string)(children: StringSubstrColStartLen(col 6:string, start 0, length 1) -> 26:string) -> 57:timestamp, CastLongToString(col 0:tinyint) -> 26:string, CastLongToString(col 1:smallint) -> 58:string, CastLongToString(col 2:int) -> 59:string, CastLongToString(col 3:bigint) -> 60:string, CastFloatToString(col 4:float) -> 61:string, CastDoubleToString(col 5:double) -> 62:string, CastBooleanToStringViaLongToString(col 10:boolean) -> 63:string, CastLongToString(col 52:bigint)(children: LongColMultiplyLongScalar(col 3:bigint, val 0) -> 52:bigint) -> 64:string, CastTimestampToString(col 8:timestamp) -> 65:string, CastStringGroupToString(col 66:char(10))(children: C
 astStringGroupToChar(col 6:string, maxLength 10) -> 66:char(10)) -> 67:string, CastStringGroupToString(col 66:varchar(10))(children: CastStringGroupToVarChar(col 6:string, maxLength 10) -> 66:varchar(10)) -> 68:string, CastLongToFloatViaLongToDouble(col 52:int)(children: CastDoubleToLong(col 4:float) -> 52:int) -> 69:float, CastLongToDouble(col 52:int)(children: LongColMultiplyLongScalar(col 2:int, val 2) -> 52:int) -> 70:double, CastDoubleToString(col 71:double)(children: FuncSinDoubleToDouble(col 4:float) -> 71:double) -> 66:string, DoubleColAddDoubleColumn(col 71:double, col 72:double)(children: col 71:float, CastLongToDouble(col 10:boolean) -> 72:double) -> 73:double
                 Statistics: Num rows: 6144 Data size: 1453997 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
@@ -201,7 +201,7 @@ STAGE PLANS:
           featureSupportInUse: [DECIMAL_64]
           inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
           allNative: false
-          usesVectorUDFAdaptor: true
+          usesVectorUDFAdaptor: false
           vectorized: true
           rowBatchContext:
               dataColumnCount: 12

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/test/results/clientpositive/vectorized_timestamp_funcs.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vectorized_timestamp_funcs.q.out b/ql/src/test/results/clientpositive/vectorized_timestamp_funcs.q.out
index 8bd77f9..4da63b1 100644
--- a/ql/src/test/results/clientpositive/vectorized_timestamp_funcs.q.out
+++ b/ql/src/test/results/clientpositive/vectorized_timestamp_funcs.q.out
@@ -456,7 +456,7 @@ STAGE PLANS:
                   className: VectorSelectOperator
                   native: true
                   projectedOutputColumnNums: [5, 7, 8, 9, 6, 11, 10, 13, 14]
-                  selectExpressions: VectorUDFUnixTimeStampString(col 2:string) -> 5:bigint, VectorUDFYearDate(col 6, field YEAR)(children: CastStringToDate(col 2:string) -> 6:date) -> 7:int, VectorUDFMonthDate(col 6, field MONTH)(children: CastStringToDate(col 2:string) -> 6:date) -> 8:int, VectorUDFDayOfMonthDate(col 6, field DAY_OF_MONTH)(children: CastStringToDate(col 2:string) -> 6:date) -> 9:int, VectorUDFDayOfMonthString(col 2:string, fieldStart 8, fieldLength 2) -> 6:int, VectorUDFWeekOfYearDate(col 10, field WEEK_OF_YEAR)(children: CastStringToDate(col 2:string) -> 10:date) -> 11:int, VectorUDFHourTimestamp(col 12:timestamp, field HOUR_OF_DAY)(children: VectorUDFAdaptor(CAST( stimestamp1 AS TIMESTAMP)) -> 12:timestamp) -> 10:int, VectorUDFMinuteTimestamp(col 12:timestamp, field MINUTE)(children: VectorUDFAdaptor(CAST( stimestamp1 AS TIMESTAMP)) -> 12:timestamp) -> 13:int, VectorUDFSecondTimestamp(col 12:timestamp, field SECOND)(children: VectorUDFAdaptor(CAST( stimestamp1 A
 S TIMESTAMP)) -> 12:timestamp) -> 14:int
+                  selectExpressions: VectorUDFUnixTimeStampString(col 2:string) -> 5:bigint, VectorUDFYearDate(col 6, field YEAR)(children: CastStringToDate(col 2:string) -> 6:date) -> 7:int, VectorUDFMonthDate(col 6, field MONTH)(children: CastStringToDate(col 2:string) -> 6:date) -> 8:int, VectorUDFDayOfMonthDate(col 6, field DAY_OF_MONTH)(children: CastStringToDate(col 2:string) -> 6:date) -> 9:int, VectorUDFDayOfMonthString(col 2:string, fieldStart 8, fieldLength 2) -> 6:int, VectorUDFWeekOfYearDate(col 10, field WEEK_OF_YEAR)(children: CastStringToDate(col 2:string) -> 10:date) -> 11:int, VectorUDFHourTimestamp(col 12:timestamp, field HOUR_OF_DAY)(children: CastStringToTimestamp(col 2:string) -> 12:timestamp) -> 10:int, VectorUDFMinuteTimestamp(col 12:timestamp, field MINUTE)(children: CastStringToTimestamp(col 2:string) -> 12:timestamp) -> 13:int, VectorUDFSecondTimestamp(col 12:timestamp, field SECOND)(children: CastStringToTimestamp(col 2:string) -> 12:timestamp) -> 14:int
               Statistics: Num rows: 52 Data size: 3179 Basic stats: COMPLETE Column stats: NONE
               Reduce Output Operator
                 key expressions: _col0 (type: bigint)
@@ -476,7 +476,7 @@ STAGE PLANS:
           featureSupportInUse: [DECIMAL_64]
           inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
           allNative: false
-          usesVectorUDFAdaptor: true
+          usesVectorUDFAdaptor: false
           vectorized: true
       Reduce Vectorization:
           enabled: false
@@ -633,7 +633,7 @@ STAGE PLANS:
                   className: VectorSelectOperator
                   native: true
                   projectedOutputColumnNums: [7, 6, 8, 9, 11, 10, 14, 15, 16]
-                  selectExpressions: LongColEqualLongColumn(col 5:bigint, col 6:bigint)(children: VectorUDFUnixTimeStampTimestamp(col 1:timestamp) -> 5:bigint, VectorUDFUnixTimeStampString(col 2:string) -> 6:bigint) -> 7:boolean, LongColEqualLongColumn(col 5:int, col 8:int)(children: VectorUDFYearTimestamp(col 1:timestamp, field YEAR) -> 5:int, VectorUDFYearDate(col 6, field YEAR)(children: CastStringToDate(col 2:string) -> 6:date) -> 8:int) -> 6:boolean, LongColEqualLongColumn(col 5:int, col 9:int)(children: VectorUDFMonthTimestamp(col 1:timestamp, field MONTH) -> 5:int, VectorUDFMonthDate(col 8, field MONTH)(children: CastStringToDate(col 2:string) -> 8:date) -> 9:int) -> 8:boolean, LongColEqualLongColumn(col 5:int, col 10:int)(children: VectorUDFDayOfMonthTimestamp(col 1:timestamp, field DAY_OF_MONTH) -> 5:int, VectorUDFDayOfMonthDate(col 9, field DAY_OF_MONTH)(children: CastStringToDate(col 2:string) -> 9:date) -> 10:int) -> 9:boolean, LongColEqualLongColumn(col 5:int, col 10:in
 t)(children: VectorUDFDayOfMonthTimestamp(col 1:timestamp, field DAY_OF_MONTH) -> 5:int, VectorUDFDayOfMonthString(col 2:string, fieldStart 8, fieldLength 2) -> 10:int) -> 11:boolean, LongColEqualLongColumn(col 5:int, col 12:int)(children: VectorUDFWeekOfYearTimestamp(col 1:timestamp, field WEEK_OF_YEAR) -> 5:int, VectorUDFWeekOfYearDate(col 10, field WEEK_OF_YEAR)(children: CastStringToDate(col 2:string) -> 10:date) -> 12:int) -> 10:boolean, LongColEqualLongColumn(col 5:int, col 12:int)(children: VectorUDFHourTimestamp(col 1:timestamp, field HOUR_OF_DAY) -> 5:int, VectorUDFHourTimestamp(col 13:timestamp, field HOUR_OF_DAY)(children: VectorUDFAdaptor(CAST( stimestamp1 AS TIMESTAMP)) -> 13:timestamp) -> 12:int) -> 14:boolean, LongColEqualLongColumn(col 5:int, col 12:int)(children: VectorUDFMinuteTimestamp(col 1:timestamp, field MINUTE) -> 5:int, VectorUDFMinuteTimestamp(col 13:timestamp, field MINUTE)(children: VectorUDFAdaptor(CAST( stimestamp1 AS TIMESTAMP)) -> 13:timestamp) -> 12:
 int) -> 15:boolean, LongColEqualLongColumn(col 5:int, col 12:int)(children: VectorUDFSecondTimestamp(col 1:timestamp, field SECOND) -> 5:int, VectorUDFSecondTimestamp(col 13:timestamp, field SECOND)(children: VectorUDFAdaptor(CAST( stimestamp1 AS TIMESTAMP)) -> 13:timestamp) -> 12:int) -> 16:boolean
+                  selectExpressions: LongColEqualLongColumn(col 5:bigint, col 6:bigint)(children: VectorUDFUnixTimeStampTimestamp(col 1:timestamp) -> 5:bigint, VectorUDFUnixTimeStampString(col 2:string) -> 6:bigint) -> 7:boolean, LongColEqualLongColumn(col 5:int, col 8:int)(children: VectorUDFYearTimestamp(col 1:timestamp, field YEAR) -> 5:int, VectorUDFYearDate(col 6, field YEAR)(children: CastStringToDate(col 2:string) -> 6:date) -> 8:int) -> 6:boolean, LongColEqualLongColumn(col 5:int, col 9:int)(children: VectorUDFMonthTimestamp(col 1:timestamp, field MONTH) -> 5:int, VectorUDFMonthDate(col 8, field MONTH)(children: CastStringToDate(col 2:string) -> 8:date) -> 9:int) -> 8:boolean, LongColEqualLongColumn(col 5:int, col 10:int)(children: VectorUDFDayOfMonthTimestamp(col 1:timestamp, field DAY_OF_MONTH) -> 5:int, VectorUDFDayOfMonthDate(col 9, field DAY_OF_MONTH)(children: CastStringToDate(col 2:string) -> 9:date) -> 10:int) -> 9:boolean, LongColEqualLongColumn(col 5:int, col 10:in
 t)(children: VectorUDFDayOfMonthTimestamp(col 1:timestamp, field DAY_OF_MONTH) -> 5:int, VectorUDFDayOfMonthString(col 2:string, fieldStart 8, fieldLength 2) -> 10:int) -> 11:boolean, LongColEqualLongColumn(col 5:int, col 12:int)(children: VectorUDFWeekOfYearTimestamp(col 1:timestamp, field WEEK_OF_YEAR) -> 5:int, VectorUDFWeekOfYearDate(col 10, field WEEK_OF_YEAR)(children: CastStringToDate(col 2:string) -> 10:date) -> 12:int) -> 10:boolean, LongColEqualLongColumn(col 5:int, col 12:int)(children: VectorUDFHourTimestamp(col 1:timestamp, field HOUR_OF_DAY) -> 5:int, VectorUDFHourTimestamp(col 13:timestamp, field HOUR_OF_DAY)(children: CastStringToTimestamp(col 2:string) -> 13:timestamp) -> 12:int) -> 14:boolean, LongColEqualLongColumn(col 5:int, col 12:int)(children: VectorUDFMinuteTimestamp(col 1:timestamp, field MINUTE) -> 5:int, VectorUDFMinuteTimestamp(col 13:timestamp, field MINUTE)(children: CastStringToTimestamp(col 2:string) -> 13:timestamp) -> 12:int) -> 15:boolean, LongColE
 qualLongColumn(col 5:int, col 12:int)(children: VectorUDFSecondTimestamp(col 1:timestamp, field SECOND) -> 5:int, VectorUDFSecondTimestamp(col 13:timestamp, field SECOND)(children: CastStringToTimestamp(col 2:string) -> 13:timestamp) -> 12:int) -> 16:boolean
               Statistics: Num rows: 52 Data size: 3179 Basic stats: COMPLETE Column stats: NONE
               Reduce Output Operator
                 key expressions: _col0 (type: boolean)
@@ -653,7 +653,7 @@ STAGE PLANS:
           featureSupportInUse: [DECIMAL_64]
           inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
           allNative: false
-          usesVectorUDFAdaptor: true
+          usesVectorUDFAdaptor: false
           vectorized: true
       Reduce Vectorization:
           enabled: false
@@ -810,7 +810,7 @@ STAGE PLANS:
                   className: VectorSelectOperator
                   native: true
                   projectedOutputColumnNums: [2, 4, 5, 6, 3, 8, 7, 10, 11]
-                  selectExpressions: VectorUDFUnixTimeStampString(col 0:string) -> 2:bigint, VectorUDFYearDate(col 3, field YEAR)(children: CastStringToDate(col 0:string) -> 3:date) -> 4:int, VectorUDFMonthDate(col 3, field MONTH)(children: CastStringToDate(col 0:string) -> 3:date) -> 5:int, VectorUDFDayOfMonthDate(col 3, field DAY_OF_MONTH)(children: CastStringToDate(col 0:string) -> 3:date) -> 6:int, VectorUDFDayOfMonthString(col 0:string, fieldStart 8, fieldLength 2) -> 3:int, VectorUDFWeekOfYearDate(col 7, field WEEK_OF_YEAR)(children: CastStringToDate(col 0:string) -> 7:date) -> 8:int, VectorUDFHourTimestamp(col 9:timestamp, field HOUR_OF_DAY)(children: VectorUDFAdaptor(CAST( stimestamp1 AS TIMESTAMP)) -> 9:timestamp) -> 7:int, VectorUDFMinuteTimestamp(col 9:timestamp, field MINUTE)(children: VectorUDFAdaptor(CAST( stimestamp1 AS TIMESTAMP)) -> 9:timestamp) -> 10:int, VectorUDFSecondTimestamp(col 9:timestamp, field SECOND)(children: VectorUDFAdaptor(CAST( stimestamp1 AS TIMESTA
 MP)) -> 9:timestamp) -> 11:int
+                  selectExpressions: VectorUDFUnixTimeStampString(col 0:string) -> 2:bigint, VectorUDFYearDate(col 3, field YEAR)(children: CastStringToDate(col 0:string) -> 3:date) -> 4:int, VectorUDFMonthDate(col 3, field MONTH)(children: CastStringToDate(col 0:string) -> 3:date) -> 5:int, VectorUDFDayOfMonthDate(col 3, field DAY_OF_MONTH)(children: CastStringToDate(col 0:string) -> 3:date) -> 6:int, VectorUDFDayOfMonthString(col 0:string, fieldStart 8, fieldLength 2) -> 3:int, VectorUDFWeekOfYearDate(col 7, field WEEK_OF_YEAR)(children: CastStringToDate(col 0:string) -> 7:date) -> 8:int, VectorUDFHourTimestamp(col 9:timestamp, field HOUR_OF_DAY)(children: CastStringToTimestamp(col 0:string) -> 9:timestamp) -> 7:int, VectorUDFMinuteTimestamp(col 9:timestamp, field MINUTE)(children: CastStringToTimestamp(col 0:string) -> 9:timestamp) -> 10:int, VectorUDFSecondTimestamp(col 9:timestamp, field SECOND)(children: CastStringToTimestamp(col 0:string) -> 9:timestamp) -> 11:int
               Statistics: Num rows: 3 Data size: 294 Basic stats: COMPLETE Column stats: NONE
               Reduce Output Operator
                 key expressions: _col0 (type: bigint)
@@ -830,7 +830,7 @@ STAGE PLANS:
           featureSupportInUse: [DECIMAL_64]
           inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
           allNative: false
-          usesVectorUDFAdaptor: true
+          usesVectorUDFAdaptor: false
           vectorized: true
       Reduce Vectorization:
           enabled: false

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/test/results/clientpositive/vectorized_timestamp_ints_casts.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vectorized_timestamp_ints_casts.q.out b/ql/src/test/results/clientpositive/vectorized_timestamp_ints_casts.q.out
index 0d2c284..e67855c 100644
--- a/ql/src/test/results/clientpositive/vectorized_timestamp_ints_casts.q.out
+++ b/ql/src/test/results/clientpositive/vectorized_timestamp_ints_casts.q.out
@@ -66,7 +66,7 @@ STAGE PLANS:
                     className: VectorSelectOperator
                     native: true
                     projectedOutputColumnNums: [15, 17, 19, 21, 22, 23, 25, 27, 8, 28, 30]
-                    selectExpressions: CastMillisecondsLongToTimestamp(col 0:tinyint) -> 15:timestamp, CastMillisecondsLongToTimestamp(col 1:smallint) -> 17:timestamp, CastMillisecondsLongToTimestamp(col 2:int) -> 19:timestamp, CastMillisecondsLongToTimestamp(col 3:bigint) -> 21:timestamp, CastDoubleToTimestamp(col 4:float) -> 22:timestamp, CastDoubleToTimestamp(col 5:double) -> 23:timestamp, CastMillisecondsLongToTimestamp(col 10:boolean) -> 25:timestamp, CastMillisecondsLongToTimestamp(col 13:bigint)(children: LongColMultiplyLongScalar(col 3:bigint, val 0) -> 13:bigint) -> 27:timestamp, VectorUDFAdaptor(CAST( cstring1 AS TIMESTAMP)) -> 28:timestamp, VectorUDFAdaptor(CAST( substr(cstring1, 1, 1) AS TIMESTAMP))(children: StringSubstrColStartLen(col 6:string, start 0, length 1) -> 29:string) -> 30:timestamp
+                    selectExpressions: CastMillisecondsLongToTimestamp(col 0:tinyint) -> 15:timestamp, CastMillisecondsLongToTimestamp(col 1:smallint) -> 17:timestamp, CastMillisecondsLongToTimestamp(col 2:int) -> 19:timestamp, CastMillisecondsLongToTimestamp(col 3:bigint) -> 21:timestamp, CastDoubleToTimestamp(col 4:float) -> 22:timestamp, CastDoubleToTimestamp(col 5:double) -> 23:timestamp, CastMillisecondsLongToTimestamp(col 10:boolean) -> 25:timestamp, CastMillisecondsLongToTimestamp(col 13:bigint)(children: LongColMultiplyLongScalar(col 3:bigint, val 0) -> 13:bigint) -> 27:timestamp, CastStringToTimestamp(col 6:string) -> 28:timestamp, CastStringToTimestamp(col 29:string)(children: StringSubstrColStartLen(col 6:string, start 0, length 1) -> 29:string) -> 30:timestamp
                 Statistics: Num rows: 6144 Data size: 1453997 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
@@ -86,7 +86,7 @@ STAGE PLANS:
           featureSupportInUse: [DECIMAL_64]
           inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
           allNative: false
-          usesVectorUDFAdaptor: true
+          usesVectorUDFAdaptor: false
           vectorized: true
 
   Stage: Stage-0
@@ -227,7 +227,7 @@ STAGE PLANS:
                     className: VectorSelectOperator
                     native: true
                     projectedOutputColumnNums: [14, 15, 16, 17, 18, 19, 20, 21, 8, 22, 24]
-                    selectExpressions: CastLongToTimestamp(col 0:tinyint) -> 14:timestamp, CastLongToTimestamp(col 1:smallint) -> 15:timestamp, CastLongToTimestamp(col 2:int) -> 16:timestamp, CastLongToTimestamp(col 3:bigint) -> 17:timestamp, CastDoubleToTimestamp(col 4:float) -> 18:timestamp, CastDoubleToTimestamp(col 5:double) -> 19:timestamp, CastLongToTimestamp(col 10:boolean) -> 20:timestamp, CastLongToTimestamp(col 13:bigint)(children: LongColMultiplyLongScalar(col 3:bigint, val 0) -> 13:bigint) -> 21:timestamp, VectorUDFAdaptor(CAST( cstring1 AS TIMESTAMP)) -> 22:timestamp, VectorUDFAdaptor(CAST( substr(cstring1, 1, 1) AS TIMESTAMP))(children: StringSubstrColStartLen(col 6:string, start 0, length 1) -> 23:string) -> 24:timestamp
+                    selectExpressions: CastLongToTimestamp(col 0:tinyint) -> 14:timestamp, CastLongToTimestamp(col 1:smallint) -> 15:timestamp, CastLongToTimestamp(col 2:int) -> 16:timestamp, CastLongToTimestamp(col 3:bigint) -> 17:timestamp, CastDoubleToTimestamp(col 4:float) -> 18:timestamp, CastDoubleToTimestamp(col 5:double) -> 19:timestamp, CastLongToTimestamp(col 10:boolean) -> 20:timestamp, CastLongToTimestamp(col 13:bigint)(children: LongColMultiplyLongScalar(col 3:bigint, val 0) -> 13:bigint) -> 21:timestamp, CastStringToTimestamp(col 6:string) -> 22:timestamp, CastStringToTimestamp(col 23:string)(children: StringSubstrColStartLen(col 6:string, start 0, length 1) -> 23:string) -> 24:timestamp
                 Statistics: Num rows: 6144 Data size: 1453997 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
@@ -247,7 +247,7 @@ STAGE PLANS:
           featureSupportInUse: [DECIMAL_64]
           inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
           allNative: false
-          usesVectorUDFAdaptor: true
+          usesVectorUDFAdaptor: false
           vectorized: true
 
   Stage: Stage-0

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/DecimalColumnVector.java
----------------------------------------------------------------------
diff --git a/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/DecimalColumnVector.java b/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/DecimalColumnVector.java
index c1d6a3a..e074fb9 100644
--- a/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/DecimalColumnVector.java
+++ b/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/DecimalColumnVector.java
@@ -62,8 +62,27 @@ public class DecimalColumnVector extends ColumnVector {
   }
 
   @Override
+  // Simplify vector by brute-force flattening noNulls and isRepeating
+  // This can be used to reduce combinatorial explosion of code paths in VectorExpressions
+  // with many arguments.
   public void flatten(boolean selectedInUse, int[] sel, int size) {
-    throw new RuntimeException("Not implemented");
+    flattenPush();
+    if (isRepeating) {
+      isRepeating = false;
+      HiveDecimalWritable repeat = vector[0];
+      if (selectedInUse) {
+        for (int j = 0; j < size; j++) {
+          int i = sel[j];
+          vector[i].set(repeat);
+        }
+      } else {
+        for (int i = 0; i < size; i++) {
+          vector[i].set(repeat);
+        }
+      }
+      flattenRepeatingNulls(selectedInUse, sel, size);
+    }
+    flattenNoNulls(selectedInUse, sel, size);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringExpr.java
----------------------------------------------------------------------
diff --git a/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringExpr.java b/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringExpr.java
index 162e8e6..bcbad4b 100644
--- a/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringExpr.java
+++ b/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringExpr.java
@@ -129,6 +129,23 @@ public class StringExpr {
     return charCount;
   }
 
+  public static void padRight(BytesColumnVector outV, int i, byte[] bytes, int start, int length,
+      int maxCharacterLength) {
+
+    final int characterLength = StringExpr.characterCount(bytes, start, length);
+    final int blankPadLength = Math.max(maxCharacterLength - characterLength, 0);
+    final int resultLength = length + blankPadLength;
+    outV.ensureValPreallocated(resultLength);
+    byte[] resultBytes = outV.getValPreallocatedBytes();
+    final int resultStart = outV.getValPreallocatedStart();
+    System.arraycopy(bytes, start, resultBytes, resultStart, length);
+    final int padEnd = resultStart + resultLength;
+    for (int p = resultStart + length; p < padEnd; p++) {
+      resultBytes[p] = ' ';
+    }
+    outV.setValPreallocated(i, resultLength);
+  }
+
   // A setVal with the same function signature as rightTrim, leftTrim, truncate, etc, below.
   // Useful for class generation via templates.
   public static void assign(BytesColumnVector outV, int i, byte[] bytes, int start, int length) {


[6/8] hive git commit: HIVE-20207: Vectorization: Fix NULL / Wrong Results issues in Filter / Compare (Matt McCline, reviewed by Teddy Choi)

Posted by mm...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastCharToBinary.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastCharToBinary.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastCharToBinary.java
new file mode 100644
index 0000000..f8f60a8
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastCharToBinary.java
@@ -0,0 +1,55 @@
+/*
+ * 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;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.serde2.typeinfo.CharTypeInfo;
+
+public class CastCharToBinary extends StringUnaryUDFDirect {
+
+  private static final long serialVersionUID = 1L;
+  private int maxLength;
+
+  public CastCharToBinary(int inputColumn, int outputColumnNum) {
+    super(inputColumn, outputColumnNum);
+  }
+
+  public CastCharToBinary() {
+    super();
+  }
+
+  @Override
+  public void transientInit() throws HiveException {
+    super.transientInit();
+
+    maxLength = ((CharTypeInfo) inputTypeInfos[0]).getLength();
+  }
+
+  /**
+   * Do pad out the CHAR type into the BINARY result, taking into account Unicode...
+   */
+  protected void func(BytesColumnVector outV, byte[][] vector, int[] start, int[] length, int i) {
+    StringExpr.padRight(outV, i, vector[i], start[i], length[i], maxLength);
+  }
+
+  public String vectorExpressionParameters() {
+    return getColumnParamString(0, inputColumn) + ", maxLength " + maxLength;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringToTimestamp.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringToTimestamp.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringToTimestamp.java
new file mode 100644
index 0000000..b48b013
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastStringToTimestamp.java
@@ -0,0 +1,177 @@
+/*
+ * 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 java.util.Arrays;
+import java.sql.Timestamp;
+
+import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils;
+
+
+/**
+ * Casts a string vector to a Timestamp vector.
+ */
+public class CastStringToTimestamp extends VectorExpression {
+  private static final long serialVersionUID = 1L;
+
+  private final int inputColumn;
+
+  public CastStringToTimestamp() {
+    super();
+
+    // Dummy final assignments.
+    inputColumn = -1;
+  }
+
+  public CastStringToTimestamp(int inputColumn, int outputColumnNum) {
+    super(outputColumnNum);
+    this.inputColumn = inputColumn;
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) throws HiveException {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    BytesColumnVector inputColVector = (BytesColumnVector) batch.cols[inputColumn];
+    int[] sel = batch.selected;
+    int n = batch.size;
+    TimestampColumnVector outputColVector = (TimestampColumnVector) batch.cols[outputColumnNum];
+
+    boolean[] inputIsNull = inputColVector.isNull;
+    boolean[] outputIsNull = outputColVector.isNull;
+
+    if (n == 0) {
+
+      // Nothing to do
+      return;
+    }
+
+    // We do not need to do a column reset since we are carefully changing the output.
+    outputColVector.isRepeating = false;
+
+    if (inputColVector.isRepeating) {
+      if (inputColVector.noNulls || !inputIsNull[0]) {
+        // Set isNull before call in case it changes it mind.
+        outputIsNull[0] = false;
+        evaluate(outputColVector, inputColVector, 0);
+      } else {
+        outputIsNull[0] = true;
+        outputColVector.noNulls = false;
+      }
+      outputColVector.isRepeating = true;
+      return;
+    }
+
+    if (inputColVector.noNulls) {
+      if (batch.selectedInUse) {
+
+        // CONSIDER: For large n, fill n or all of isNull array and use the tighter ELSE loop.
+
+        if (!outputColVector.noNulls) {
+          for(int j = 0; j != n; j++) {
+           final int i = sel[j];
+           // Set isNull before call in case it changes it mind.
+           outputIsNull[i] = false;
+           evaluate(outputColVector, inputColVector, i);
+         }
+        } else {
+          for(int j = 0; j != n; j++) {
+            final int i = sel[j];
+            evaluate(outputColVector, inputColVector, i);
+          }
+        }
+      } else {
+        if (!outputColVector.noNulls) {
+
+          // Assume it is almost always a performance win to fill all of isNull so we can
+          // safely reset noNulls.
+          Arrays.fill(outputIsNull, false);
+          outputColVector.noNulls = true;
+        }
+        for(int i = 0; i != n; i++) {
+          evaluate(outputColVector, inputColVector, i);
+        }
+      }
+    } else /* there are NULLs in the inputColVector */ {
+
+      // Carefully handle NULLs...
+
+      outputColVector.noNulls = false;
+
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          // Set isNull before call in case it changes it mind.
+          outputColVector.isNull[i] = inputColVector.isNull[i];
+          if (!inputColVector.isNull[i]) {
+            evaluate(outputColVector, inputColVector, i);
+          }
+        }
+      } else {
+        // Set isNull before calls in case they change their mind.
+        System.arraycopy(inputColVector.isNull, 0, outputColVector.isNull, 0, n);
+        for(int i = 0; i != n; i++) {
+          if (!inputColVector.isNull[i]) {
+            evaluate(outputColVector, inputColVector, i);
+          }
+        }
+      }
+    }
+  }
+
+  private void evaluate(TimestampColumnVector outputColVector, BytesColumnVector inputColVector, int i) {
+    try {
+      org.apache.hadoop.hive.common.type.Timestamp timestamp =
+          PrimitiveObjectInspectorUtils.getTimestampFromString(
+              new String(
+                  inputColVector.vector[i], inputColVector.start[i], inputColVector.length[i],
+                  "UTF-8"));
+      outputColVector.set(i, timestamp.toSqlTimestamp());
+    } catch (Exception e) {
+      outputColVector.setNullValue(i);
+      outputColVector.isNull[i] = true;
+      outputColVector.noNulls = false;
+    }
+  }
+
+  @Override
+  public String vectorExpressionParameters() {
+    return getColumnParamString(0, inputColumn);
+  }
+
+  @Override
+  public VectorExpressionDescriptor.Descriptor getDescriptor() {
+    VectorExpressionDescriptor.Builder b = new VectorExpressionDescriptor.Builder();
+    b.setMode(VectorExpressionDescriptor.Mode.PROJECTION)
+        .setNumArguments(1)
+        .setArgumentTypes(
+            VectorExpressionDescriptor.ArgumentType.STRING_FAMILY)
+        .setInputExpressionTypes(
+            VectorExpressionDescriptor.InputExpressionType.COLUMN);
+    return b.build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprCharScalarStringScalar.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprCharScalarStringScalar.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprCharScalarStringScalar.java
index 0b0c532..256cb8d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprCharScalarStringScalar.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprCharScalarStringScalar.java
@@ -48,7 +48,7 @@ public class IfExprCharScalarStringScalar extends IfExprStringScalarStringScalar
         .setArgumentTypes(
             VectorExpressionDescriptor.ArgumentType.INT_FAMILY,
             VectorExpressionDescriptor.ArgumentType.CHAR,
-            VectorExpressionDescriptor.ArgumentType.STRING)
+            VectorExpressionDescriptor.ArgumentType.CHAR)
         .setInputExpressionTypes(
             VectorExpressionDescriptor.InputExpressionType.COLUMN,
             VectorExpressionDescriptor.InputExpressionType.SCALAR,

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprDecimal64ColumnDecimal64Column.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprDecimal64ColumnDecimal64Column.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprDecimal64ColumnDecimal64Column.java
new file mode 100644
index 0000000..0cf2ffe
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprDecimal64ColumnDecimal64Column.java
@@ -0,0 +1,55 @@
+/*
+ * 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.VectorExpressionDescriptor;
+
+/**
+ * Compute IF(expr1, expr2, expr3) for 3 input column expressions.
+ * The first is always a boolean (LongColumnVector).
+ * The second and third are long columns or long expression results.
+ */
+public class IfExprDecimal64ColumnDecimal64Column extends IfExprLongColumnLongColumn {
+
+  private static final long serialVersionUID = 1L;
+
+  public IfExprDecimal64ColumnDecimal64Column(int arg1Column, int arg2Column, int arg3Column,
+      int outputColumnNum) {
+    super(arg1Column, arg2Column, arg3Column, outputColumnNum);
+  }
+
+  public IfExprDecimal64ColumnDecimal64Column() {
+    super();
+  }
+
+  @Override
+  public VectorExpressionDescriptor.Descriptor getDescriptor() {
+    return (new VectorExpressionDescriptor.Builder())
+        .setMode(
+            VectorExpressionDescriptor.Mode.PROJECTION)
+        .setNumArguments(3)
+        .setArgumentTypes(
+            VectorExpressionDescriptor.ArgumentType.getType("long"),
+            VectorExpressionDescriptor.ArgumentType.DECIMAL_64,
+            VectorExpressionDescriptor.ArgumentType.DECIMAL_64)
+        .setInputExpressionTypes(
+            VectorExpressionDescriptor.InputExpressionType.COLUMN,
+            VectorExpressionDescriptor.InputExpressionType.COLUMN,
+            VectorExpressionDescriptor.InputExpressionType.COLUMN).build();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprDecimal64ColumnDecimal64Scalar.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprDecimal64ColumnDecimal64Scalar.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprDecimal64ColumnDecimal64Scalar.java
new file mode 100644
index 0000000..f5f9dc9
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprDecimal64ColumnDecimal64Scalar.java
@@ -0,0 +1,70 @@
+/*
+ * 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.expressions.gen.IfExprLongColumnLongScalar;
+import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo;
+
+/**
+ * Compute IF(expr1, expr2, expr3) for 3 input column expressions.
+ * The first is always a boolean (LongColumnVector).
+ * The second is a column or non-constant expression result.
+ * The third is a constant value.
+ */
+public class IfExprDecimal64ColumnDecimal64Scalar extends IfExprLongColumnLongScalar {
+
+  private static final long serialVersionUID = 1L;
+
+  public IfExprDecimal64ColumnDecimal64Scalar(int arg1Column, int arg2Column, long arg3Scalar,
+      int outputColumnNum) {
+    super(arg1Column, arg2Column, arg3Scalar, outputColumnNum);
+  }
+
+  public IfExprDecimal64ColumnDecimal64Scalar() {
+    super();
+  }
+
+  @Override
+  public String vectorExpressionParameters() {
+    DecimalTypeInfo decimalTypeInfo = (DecimalTypeInfo) inputTypeInfos[2];
+    HiveDecimalWritable writable = new HiveDecimalWritable();
+    writable.deserialize64(arg3Scalar, decimalTypeInfo.scale());
+    return getColumnParamString(0, arg1Column) + ", " + getColumnParamString(1, arg2Column) +
+        ", decimal64Val " + arg3Scalar +
+        ", decimalVal " + writable.toString();
+  }
+
+  @Override
+  public VectorExpressionDescriptor.Descriptor getDescriptor() {
+    return (new VectorExpressionDescriptor.Builder())
+        .setMode(
+            VectorExpressionDescriptor.Mode.PROJECTION)
+        .setNumArguments(3)
+        .setArgumentTypes(
+            VectorExpressionDescriptor.ArgumentType.getType("long"),
+            VectorExpressionDescriptor.ArgumentType.DECIMAL_64,
+            VectorExpressionDescriptor.ArgumentType.DECIMAL_64)
+        .setInputExpressionTypes(
+            VectorExpressionDescriptor.InputExpressionType.COLUMN,
+            VectorExpressionDescriptor.InputExpressionType.COLUMN,
+            VectorExpressionDescriptor.InputExpressionType.SCALAR).build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprDecimal64ScalarDecimal64Column.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprDecimal64ScalarDecimal64Column.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprDecimal64ScalarDecimal64Column.java
new file mode 100644
index 0000000..cf16c8a
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprDecimal64ScalarDecimal64Column.java
@@ -0,0 +1,71 @@
+/*
+ * 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.expressions.gen.IfExprLongScalarLongColumn;
+import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo;
+
+/**
+ * Compute IF(expr1, expr2, expr3) for 3 input column expressions.
+ * The first is always a boolean (LongColumnVector).
+ * The second is a column or non-constant expression result.
+ * The third is a constant value.
+ */
+public class IfExprDecimal64ScalarDecimal64Column extends IfExprLongScalarLongColumn {
+
+  private static final long serialVersionUID = 1L;
+
+  public IfExprDecimal64ScalarDecimal64Column(int arg1Column, long arg2Scalar, int arg3Column,
+      int outputColumnNum) {
+    super(arg1Column, arg2Scalar, arg3Column, outputColumnNum);
+  }
+
+  public IfExprDecimal64ScalarDecimal64Column() {
+    super();
+  }
+
+  @Override
+  public String vectorExpressionParameters() {
+    DecimalTypeInfo decimalTypeInfo = (DecimalTypeInfo) inputTypeInfos[1];
+    HiveDecimalWritable writable = new HiveDecimalWritable();
+    writable.deserialize64(arg2Scalar, decimalTypeInfo.scale());
+    return
+        getColumnParamString(0, arg1Column) +
+        ", decimal64Val " + arg2Scalar + ", decimalVal " + writable.toString() +
+        ", " + getColumnParamString(1, arg3Column);
+  }
+
+  @Override
+  public VectorExpressionDescriptor.Descriptor getDescriptor() {
+    return (new VectorExpressionDescriptor.Builder())
+        .setMode(
+            VectorExpressionDescriptor.Mode.PROJECTION)
+        .setNumArguments(3)
+        .setArgumentTypes(
+            VectorExpressionDescriptor.ArgumentType.getType("long"),
+            VectorExpressionDescriptor.ArgumentType.DECIMAL_64,
+            VectorExpressionDescriptor.ArgumentType.DECIMAL_64)
+        .setInputExpressionTypes(
+            VectorExpressionDescriptor.InputExpressionType.COLUMN,
+            VectorExpressionDescriptor.InputExpressionType.SCALAR,
+            VectorExpressionDescriptor.InputExpressionType.COLUMN).build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprDecimal64ScalarDecimal64Scalar.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprDecimal64ScalarDecimal64Scalar.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprDecimal64ScalarDecimal64Scalar.java
new file mode 100644
index 0000000..8e76a94
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprDecimal64ScalarDecimal64Scalar.java
@@ -0,0 +1,75 @@
+/*
+ * 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.expressions.gen.IfExprLongScalarLongScalar;
+import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo;
+
+/**
+ * Compute IF(expr1, expr2, expr3) for 3 input  expressions.
+ * The first is always a boolean (LongColumnVector).
+ * The second is a constant value.
+ * The third is a constant value.
+ */
+public class IfExprDecimal64ScalarDecimal64Scalar extends IfExprLongScalarLongScalar {
+
+  private static final long serialVersionUID = 1L;
+
+  public IfExprDecimal64ScalarDecimal64Scalar(int arg1Column, long arg2Scalar, long arg3Scalar,
+      int outputColumnNum) {
+    super(arg1Column, arg2Scalar, arg3Scalar, outputColumnNum);
+  }
+
+  public IfExprDecimal64ScalarDecimal64Scalar() {
+    super();
+  }
+
+  @Override
+  public String vectorExpressionParameters() {
+    DecimalTypeInfo decimalTypeInfo1 = (DecimalTypeInfo) inputTypeInfos[1];
+    HiveDecimalWritable writable1 = new HiveDecimalWritable();
+    writable1.deserialize64(arg2Scalar, decimalTypeInfo1.scale());
+
+    DecimalTypeInfo decimalTypeInfo2 = (DecimalTypeInfo) inputTypeInfos[2];
+    HiveDecimalWritable writable2 = new HiveDecimalWritable();
+    writable2.deserialize64(arg3Scalar, decimalTypeInfo2.scale());
+    return
+        getColumnParamString(0, arg1Column) +
+        ", decimal64Val1 " + arg2Scalar + ", decimalVal1 " + writable1.toString() +
+        ", decimal64Val2 " + arg3Scalar + ", decimalVal2 " + writable2.toString();
+  }
+
+  @Override
+  public VectorExpressionDescriptor.Descriptor getDescriptor() {
+    return (new VectorExpressionDescriptor.Builder())
+        .setMode(
+            VectorExpressionDescriptor.Mode.PROJECTION)
+        .setNumArguments(3)
+        .setArgumentTypes(
+            VectorExpressionDescriptor.ArgumentType.getType("long"),
+            VectorExpressionDescriptor.ArgumentType.DECIMAL_64,
+            VectorExpressionDescriptor.ArgumentType.DECIMAL_64)
+        .setInputExpressionTypes(
+            VectorExpressionDescriptor.InputExpressionType.COLUMN,
+            VectorExpressionDescriptor.InputExpressionType.SCALAR,
+            VectorExpressionDescriptor.InputExpressionType.SCALAR).build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprLongColumnLongColumn.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprLongColumnLongColumn.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprLongColumnLongColumn.java
index 2a10e29..e4cb57b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprLongColumnLongColumn.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprLongColumnLongColumn.java
@@ -31,9 +31,9 @@ public class IfExprLongColumnLongColumn extends VectorExpression {
 
   private static final long serialVersionUID = 1L;
 
-  private final int arg1Column;
-  private final int arg2Column;
-  private final int arg3Column;
+  protected final int arg1Column;
+  protected final int arg2Column;
+  protected final int arg3Column;
 
   public IfExprLongColumnLongColumn(int arg1Column, int arg2Column, int arg3Column,
       int outputColumnNum) {

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprStringGroupColumnStringGroupColumn.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprStringGroupColumnStringGroupColumn.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprStringGroupColumnStringGroupColumn.java
index fc244ec..6290858 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprStringGroupColumnStringGroupColumn.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprStringGroupColumnStringGroupColumn.java
@@ -196,8 +196,8 @@ public class IfExprStringGroupColumnStringGroupColumn extends VectorExpression {
         .setNumArguments(3)
         .setArgumentTypes(
             VectorExpressionDescriptor.ArgumentType.INT_FAMILY,
-            VectorExpressionDescriptor.ArgumentType.STRING_FAMILY,
-            VectorExpressionDescriptor.ArgumentType.STRING_FAMILY)
+            VectorExpressionDescriptor.ArgumentType.STRING_FAMILY_BINARY,
+            VectorExpressionDescriptor.ArgumentType.STRING_FAMILY_BINARY)
         .setInputExpressionTypes(
             VectorExpressionDescriptor.InputExpressionType.COLUMN,
             VectorExpressionDescriptor.InputExpressionType.COLUMN,

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprStringGroupColumnStringScalar.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprStringGroupColumnStringScalar.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprStringGroupColumnStringScalar.java
index e6b30d9..728f955 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprStringGroupColumnStringScalar.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprStringGroupColumnStringScalar.java
@@ -185,8 +185,8 @@ public class IfExprStringGroupColumnStringScalar extends VectorExpression {
         .setNumArguments(3)
         .setArgumentTypes(
             VectorExpressionDescriptor.ArgumentType.INT_FAMILY,
-            VectorExpressionDescriptor.ArgumentType.STRING_FAMILY,
-            VectorExpressionDescriptor.ArgumentType.STRING)
+            VectorExpressionDescriptor.ArgumentType.STRING_FAMILY_BINARY,
+            VectorExpressionDescriptor.ArgumentType.STRING_BINARY)
         .setInputExpressionTypes(
             VectorExpressionDescriptor.InputExpressionType.COLUMN,
             VectorExpressionDescriptor.InputExpressionType.COLUMN,

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprStringScalarStringGroupColumn.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprStringScalarStringGroupColumn.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprStringScalarStringGroupColumn.java
index 52ed087..de852e7 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprStringScalarStringGroupColumn.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprStringScalarStringGroupColumn.java
@@ -188,8 +188,8 @@ public class IfExprStringScalarStringGroupColumn extends VectorExpression {
         .setNumArguments(3)
         .setArgumentTypes(
             VectorExpressionDescriptor.ArgumentType.INT_FAMILY,
-            VectorExpressionDescriptor.ArgumentType.STRING,
-            VectorExpressionDescriptor.ArgumentType.STRING_FAMILY)
+            VectorExpressionDescriptor.ArgumentType.STRING_BINARY,
+            VectorExpressionDescriptor.ArgumentType.STRING_FAMILY_BINARY)
         .setInputExpressionTypes(
             VectorExpressionDescriptor.InputExpressionType.COLUMN,
             VectorExpressionDescriptor.InputExpressionType.SCALAR,

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprStringScalarStringScalar.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprStringScalarStringScalar.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprStringScalarStringScalar.java
index 9992743..05773a4 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprStringScalarStringScalar.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprStringScalarStringScalar.java
@@ -152,8 +152,8 @@ public class IfExprStringScalarStringScalar extends VectorExpression {
         .setNumArguments(3)
         .setArgumentTypes(
             VectorExpressionDescriptor.ArgumentType.INT_FAMILY,
-            VectorExpressionDescriptor.ArgumentType.STRING,
-            VectorExpressionDescriptor.ArgumentType.STRING)
+            VectorExpressionDescriptor.ArgumentType.STRING_BINARY,
+            VectorExpressionDescriptor.ArgumentType.STRING_BINARY)
         .setInputExpressionTypes(
             VectorExpressionDescriptor.InputExpressionType.COLUMN,
             VectorExpressionDescriptor.InputExpressionType.SCALAR,

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprVarCharScalarStringScalar.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprVarCharScalarStringScalar.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprVarCharScalarStringScalar.java
index 6ab1ad7..814e3fe 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprVarCharScalarStringScalar.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprVarCharScalarStringScalar.java
@@ -47,7 +47,7 @@ public class IfExprVarCharScalarStringScalar extends IfExprStringScalarStringSca
         .setArgumentTypes(
             VectorExpressionDescriptor.ArgumentType.INT_FAMILY,
             VectorExpressionDescriptor.ArgumentType.VARCHAR,
-            VectorExpressionDescriptor.ArgumentType.STRING)
+            VectorExpressionDescriptor.ArgumentType.VARCHAR)
         .setInputExpressionTypes(
             VectorExpressionDescriptor.InputExpressionType.COLUMN,
             VectorExpressionDescriptor.InputExpressionType.SCALAR,

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColEqualLongColumn.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColEqualLongColumn.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColEqualLongColumn.java
deleted file mode 100644
index 308ddf9..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColEqualLongColumn.java
+++ /dev/null
@@ -1,146 +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.LongColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
-import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
-import org.apache.hadoop.hive.ql.metadata.HiveException;
-
-public class LongColEqualLongColumn extends VectorExpression {
-
-  private static final long serialVersionUID = 1L;
-
-  private final int colNum1;
-  private final int colNum2;
-
-  public LongColEqualLongColumn(int colNum1, int colNum2, int outputColumnNum) {
-    super(outputColumnNum);
-    this.colNum1 = colNum1;
-    this.colNum2 = colNum2;
-  }
-
-  public LongColEqualLongColumn() {
-    super();
-
-    // Dummy final assignments.
-    colNum1 = -1;
-    colNum2 = -1;
-  }
-
-  @Override
-  public void evaluate(VectorizedRowBatch batch) throws HiveException {
-
-    if (childExpressions != null) {
-      super.evaluateChildren(batch);
-    }
-
-    LongColumnVector inputColVector1 = (LongColumnVector) batch.cols[colNum1];
-    LongColumnVector inputColVector2 = (LongColumnVector) batch.cols[colNum2];
-    LongColumnVector outputColVector = (LongColumnVector) batch.cols[outputColumnNum];
-    int[] sel = batch.selected;
-    int n = batch.size;
-    long[] vector1 = inputColVector1.vector;
-    long[] vector2 = inputColVector2.vector;
-    long[] outputVector = outputColVector.vector;
-    long vector1Value = vector1[0];
-    long vector2Value = vector2[0];
-
-    // return immediately if batch is empty
-    if (n == 0) {
-      return;
-    }
-
-    /*
-     * Propagate null values for a two-input operator and set isRepeating and noNulls appropriately.
-     */
-    NullUtil.propagateNullsColCol(
-        inputColVector1, inputColVector2, outputColVector, sel, n, batch.selectedInUse);
-
-    /* Disregard nulls for processing. In other words,
-     * the arithmetic operation is performed even if one or
-     * more inputs are null. This is to improve speed by avoiding
-     * conditional checks in the inner loop.
-     */
-    if (inputColVector1.isRepeating && inputColVector2.isRepeating) {
-      outputVector[0] = vector1Value == vector2Value ? 1 : 0;
-    } else if (inputColVector1.isRepeating) {
-      if (batch.selectedInUse) {
-        for(int j = 0; j != n; j++) {
-          int i = sel[j];
-          outputVector[i] = vector1Value == vector2[i] ? 1 : 0;
-        }
-      } else {
-        for(int i = 0; i != n; i++) {
-          // The SIMD optimized form of "a == b" is "(((a - b) ^ (b - a)) >>> 63) ^ 1"
-          outputVector[i] = (((vector1Value - vector2[i]) ^ (vector2[i] - vector1Value)) >>> 63) ^ 1;
-        }
-      }
-    } else if (inputColVector2.isRepeating) {
-      if (batch.selectedInUse) {
-        for(int j = 0; j != n; j++) {
-          int i = sel[j];
-          outputVector[i] = vector1[i] == vector2Value ? 1 : 0;
-        }
-      } else {
-        for(int i = 0; i != n; i++) {
-          outputVector[i] = (((vector1[i] - vector2Value) ^ (vector2Value - vector1[i])) >>> 63) ^ 1;
-        }
-      }
-    } else {
-      if (batch.selectedInUse) {
-        for(int j = 0; j != n; j++) {
-          int i = sel[j];
-          outputVector[i] = vector1[i] == vector2[i] ? 1 : 0;
-        }
-      } else {
-        for(int i = 0; i != n; i++) {
-          outputVector[i] = (((vector1[i] - vector2[i]) ^ (vector2[i] - vector1[i])) >>> 63) ^ 1;
-        }
-      }
-    }
-
-    /* For the case when the output can have null values, follow
-     * the convention that the data values must be 1 for long and
-     * NaN for double. This is to prevent possible later zero-divide errors
-     * in complex arithmetic expressions like col2 / (col1 - 1)
-     * in the case when some col1 entries are null.
-     */
-    NullUtil.setNullDataEntriesLong(outputColVector, batch.selectedInUse, sel, n);
-  }
-
-  @Override
-  public String vectorExpressionParameters() {
-    return getColumnParamString(0, colNum1) + ", " + getColumnParamString(1, colNum2);
-  }
-
-  @Override
-  public VectorExpressionDescriptor.Descriptor getDescriptor() {
-    return (new VectorExpressionDescriptor.Builder())
-        .setMode(
-            VectorExpressionDescriptor.Mode.PROJECTION)
-        .setNumArguments(2)
-        .setArgumentTypes(
-            VectorExpressionDescriptor.ArgumentType.getType("int_date_interval_year_month"),
-            VectorExpressionDescriptor.ArgumentType.getType("int_date_interval_year_month"))
-        .setInputExpressionTypes(
-            VectorExpressionDescriptor.InputExpressionType.COLUMN,
-            VectorExpressionDescriptor.InputExpressionType.COLUMN).build();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColEqualLongScalar.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColEqualLongScalar.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColEqualLongScalar.java
deleted file mode 100644
index 797dd39..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColEqualLongScalar.java
+++ /dev/null
@@ -1,157 +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 java.util.Arrays;
-
-import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
-import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
-import org.apache.hadoop.hive.ql.metadata.HiveException;
-
-public class LongColEqualLongScalar extends VectorExpression {
-  private static final long serialVersionUID = 1L;
-
-  protected final int colNum;
-  protected final long value;
-
-  public LongColEqualLongScalar(int colNum, long value, int outputColumnNum) {
-    super(outputColumnNum);
-    this.colNum = colNum;
-    this.value = value;
-  }
-
-  public LongColEqualLongScalar() {
-    super();
-
-    // Dummy final assignments.
-    colNum = -1;
-    value = 0;
-  }
-
-  @Override
-  public void evaluate(VectorizedRowBatch batch) throws HiveException {
-
-    // return immediately if batch is empty
-    final int n = batch.size;
-    if (n == 0) {
-      return;
-    }
-
-    if (childExpressions != null) {
-      super.evaluateChildren(batch);
-    }
-
-    LongColumnVector inputColVector = (LongColumnVector) batch.cols[colNum];
-    LongColumnVector outputColVector = (LongColumnVector) batch.cols[outputColumnNum];
-    int[] sel = batch.selected;
-    boolean[] inputIsNull = inputColVector.isNull;
-    boolean[] outputIsNull = outputColVector.isNull;
-
-    long[] vector = inputColVector.vector;
-    long[] outputVector = outputColVector.vector;
-
-    // We do not need to do a column reset since we are carefully changing the output.
-    outputColVector.isRepeating = false;
-
-    if (inputColVector.isRepeating) {
-      if (inputColVector.noNulls || !inputIsNull[0]) {
-        outputIsNull[0] = false;
-        outputVector[0] = vector[0] == value ? 1 : 0;
-      } else {
-        outputIsNull[0] = true;
-        outputColVector.noNulls = false;
-      }
-      outputColVector.isRepeating = true;
-      return;
-    }
-
-    if (inputColVector.noNulls) {
-      if (batch.selectedInUse) {
-
-        // CONSIDER: For large n, fill n or all of isNull array and use the tighter ELSE loop.
-
-        if (!outputColVector.noNulls) {
-          for(int j = 0; j != n; j++) {
-           final int i = sel[j];
-           // Set isNull before call in case it changes it mind.
-           outputIsNull[i] = false;
-           outputVector[i] = (((vector[i] - value) ^ (value - vector[i])) >>> 63) ^ 1;
-         }
-        } else {
-          for(int j = 0; j != n; j++) {
-            final int i = sel[j];
-            outputVector[i] = (((vector[i] - value) ^ (value - vector[i])) >>> 63) ^ 1;
-          }
-        }
-      } else {
-        if (!outputColVector.noNulls) {
-
-          // Assume it is almost always a performance win to fill all of isNull so we can
-          // safely reset noNulls.
-          Arrays.fill(outputIsNull, false);
-          outputColVector.noNulls = true;
-        }
-        for(int i = 0; i != n; i++) {
-          outputVector[i] = (((vector[i] - value) ^ (value - vector[i])) >>> 63) ^ 1;
-        }
-      }
-    } else /* there are nulls in the inputColVector */ {
-
-      // Carefully handle NULLs...
-
-      /*
-       * For better performance on LONG/DOUBLE we don't want the conditional
-       * statements inside the for loop.
-       */
-      outputColVector.noNulls = false;
-
-      if (batch.selectedInUse) {
-        for(int j=0; j != n; j++) {
-          int i = sel[j];
-          outputIsNull[i] = inputIsNull[i];
-          outputVector[i] = (((vector[i] - value) ^ (value - vector[i])) >>> 63) ^ 1;
-        }
-      } else {
-        System.arraycopy(inputIsNull, 0, outputIsNull, 0, n);
-        for(int i = 0; i != n; i++) {
-          outputVector[i] = (((vector[i] - value) ^ (value - vector[i])) >>> 63) ^ 1;
-        }
-      }
-    }
-  }
-
-  public String vectorExpressionParameters() {
-    return getColumnParamString(0, colNum) + ", val " + value;
-  }
-
-  @Override
-  public VectorExpressionDescriptor.Descriptor getDescriptor() {
-    return (new VectorExpressionDescriptor.Builder())
-        .setMode(
-            VectorExpressionDescriptor.Mode.PROJECTION)
-        .setNumArguments(2)
-        .setArgumentTypes(
-            VectorExpressionDescriptor.ArgumentType.getType("long"),
-            VectorExpressionDescriptor.ArgumentType.getType("long"))
-        .setInputExpressionTypes(
-            VectorExpressionDescriptor.InputExpressionType.COLUMN,
-            VectorExpressionDescriptor.InputExpressionType.SCALAR).build();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColGreaterEqualLongColumn.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColGreaterEqualLongColumn.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColGreaterEqualLongColumn.java
deleted file mode 100644
index ad9c7a3..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColGreaterEqualLongColumn.java
+++ /dev/null
@@ -1,146 +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.LongColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
-import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
-import org.apache.hadoop.hive.ql.metadata.HiveException;
-
-public class LongColGreaterEqualLongColumn extends VectorExpression {
-
-  private static final long serialVersionUID = 1L;
-
-  private final int colNum1;
-  private final int colNum2;
-
-  public LongColGreaterEqualLongColumn(int colNum1, int colNum2, int outputColumnNum) {
-    super(outputColumnNum);
-    this.colNum1 = colNum1;
-    this.colNum2 = colNum2;
-  }
-
-  public LongColGreaterEqualLongColumn() {
-    super();
-
-    // Dummy final assignments.
-    colNum1 = -1;
-    colNum2 = -1;
-  }
-
-  @Override
-  public void evaluate(VectorizedRowBatch batch) throws HiveException {
-
-    if (childExpressions != null) {
-      super.evaluateChildren(batch);
-    }
-
-    LongColumnVector inputColVector1 = (LongColumnVector) batch.cols[colNum1];
-    LongColumnVector inputColVector2 = (LongColumnVector) batch.cols[colNum2];
-    LongColumnVector outputColVector = (LongColumnVector) batch.cols[outputColumnNum];
-    int[] sel = batch.selected;
-    int n = batch.size;
-    long[] vector1 = inputColVector1.vector;
-    long[] vector2 = inputColVector2.vector;
-    long[] outputVector = outputColVector.vector;
-    long vector1Value = vector1[0];
-    long vector2Value = vector2[0];
-
-    // return immediately if batch is empty
-    if (n == 0) {
-      return;
-    }
-
-    /*
-     * Propagate null values for a two-input operator and set isRepeating and noNulls appropriately.
-     */
-    NullUtil.propagateNullsColCol(
-        inputColVector1, inputColVector2, outputColVector, sel, n, batch.selectedInUse);
-
-    /* Disregard nulls for processing. In other words,
-     * the arithmetic operation is performed even if one or
-     * more inputs are null. This is to improve speed by avoiding
-     * conditional checks in the inner loop.
-     */
-    if (inputColVector1.isRepeating && inputColVector2.isRepeating) {
-      outputVector[0] = vector1Value >= vector2Value ? 1 : 0;
-    } else if (inputColVector1.isRepeating) {
-      if (batch.selectedInUse) {
-        for(int j = 0; j != n; j++) {
-          int i = sel[j];
-          outputVector[i] = vector1Value >= vector2[i] ? 1 : 0;
-        }
-      } else {
-        for(int i = 0; i != n; i++) {
-          // The SIMD optimized form of "a >= b" is "((a - b) >>> 63) ^ 1"
-          outputVector[i] = ((vector1Value - vector2[i]) >>> 63) ^ 1;
-        }
-      }
-    } else if (inputColVector2.isRepeating) {
-      if (batch.selectedInUse) {
-        for(int j = 0; j != n; j++) {
-          int i = sel[j];
-          outputVector[i] = vector1[i] >= vector2Value ? 1 : 0;
-        }
-      } else {
-        for(int i = 0; i != n; i++) {
-          outputVector[i] = ((vector1[i] - vector2Value) >>> 63) ^ 1;
-        }
-      }
-    } else {
-      if (batch.selectedInUse) {
-        for(int j = 0; j != n; j++) {
-          int i = sel[j];
-          outputVector[i] = vector1[i] >= vector2[i] ? 1 : 0;
-        }
-      } else {
-        for(int i = 0; i != n; i++) {
-          outputVector[i] = ((vector1[i] - vector2[i]) >>> 63) ^ 1;
-        }
-      }
-    }
-
-    /* For the case when the output can have null values, follow
-     * the convention that the data values must be 1 for long and
-     * NaN for double. This is to prevent possible later zero-divide errors
-     * in complex arithmetic expressions like col2 / (col1 - 1)
-     * in the case when some col1 entries are null.
-     */
-    NullUtil.setNullDataEntriesLong(outputColVector, batch.selectedInUse, sel, n);
-  }
-
-  @Override
-  public String vectorExpressionParameters() {
-    return getColumnParamString(0, colNum1) + ", " + getColumnParamString(1, colNum2);
-  }
-
-  @Override
-  public VectorExpressionDescriptor.Descriptor getDescriptor() {
-    return (new VectorExpressionDescriptor.Builder())
-        .setMode(
-            VectorExpressionDescriptor.Mode.PROJECTION)
-        .setNumArguments(2)
-        .setArgumentTypes(
-            VectorExpressionDescriptor.ArgumentType.getType("int_date_interval_year_month"),
-            VectorExpressionDescriptor.ArgumentType.getType("int_date_interval_year_month"))
-        .setInputExpressionTypes(
-            VectorExpressionDescriptor.InputExpressionType.COLUMN,
-            VectorExpressionDescriptor.InputExpressionType.COLUMN).build();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColGreaterEqualLongScalar.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColGreaterEqualLongScalar.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColGreaterEqualLongScalar.java
deleted file mode 100644
index 497826a..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColGreaterEqualLongScalar.java
+++ /dev/null
@@ -1,158 +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 java.util.Arrays;
-
-import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
-import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
-import org.apache.hadoop.hive.ql.metadata.HiveException;
-
-public class LongColGreaterEqualLongScalar extends VectorExpression {
-
-  private static final long serialVersionUID = 1L;
-
-  protected int colNum;
-  protected long value;
-
-  public LongColGreaterEqualLongScalar(int colNum, long value, int outputColumnNum) {
-    super(outputColumnNum);
-    this.colNum = colNum;
-    this.value = value;
-  }
-
-  public LongColGreaterEqualLongScalar() {
-    super();
-
-    // Dummy final assignments.
-    colNum = -1;
-    value = 0;
-  }
-
-  @Override
-  public void evaluate(VectorizedRowBatch batch) throws HiveException {
-
-    if (childExpressions != null) {
-      super.evaluateChildren(batch);
-    }
-
-    LongColumnVector inputColVector = (LongColumnVector) batch.cols[colNum];
-    LongColumnVector outputColVector = (LongColumnVector) batch.cols[outputColumnNum];
-    int[] sel = batch.selected;
-    boolean[] inputIsNull = inputColVector.isNull;
-    boolean[] outputIsNull = outputColVector.isNull;
-    int n = batch.size;
-    long[] vector = inputColVector.vector;
-    long[] outputVector = outputColVector.vector;
-
-    // return immediately if batch is empty
-    if (n == 0) {
-      return;
-    }
-
-    // We do not need to do a column reset since we are carefully changing the output.
-    outputColVector.isRepeating = false;
-
-    if (inputColVector.isRepeating) {
-      if (inputColVector.noNulls || !inputIsNull[0]) {
-        outputIsNull[0] = false;
-        outputVector[0] = vector[0] >= value ? 1 : 0;
-      } else {
-        outputIsNull[0] = true;
-        outputColVector.noNulls = false;
-      }
-      outputColVector.isRepeating = true;
-      return;
-    }
-
-    if (inputColVector.noNulls) {
-      if (batch.selectedInUse) {
-
-        // CONSIDER: For large n, fill n or all of isNull array and use the tighter ELSE loop.
-
-        if (!outputColVector.noNulls) {
-          for(int j = 0; j != n; j++) {
-           final int i = sel[j];
-           // Set isNull before call in case it changes it mind.
-           outputIsNull[i] = false;
-           outputVector[i] = ((vector[i] - value) >>> 63) ^ 1;
-         }
-        } else {
-          for(int j = 0; j != n; j++) {
-            final int i = sel[j];
-            outputVector[i] = ((vector[i] - value) >>> 63) ^ 1;
-          }
-        }
-      } else {
-        if (!outputColVector.noNulls) {
-
-          // Assume it is almost always a performance win to fill all of isNull so we can
-          // safely reset noNulls.
-          Arrays.fill(outputIsNull, false);
-          outputColVector.noNulls = true;
-        }
-        for(int i = 0; i != n; i++) {
-          outputVector[i] = ((vector[i] - value) >>> 63) ^ 1;
-        }
-      }
-    } else /* there are nulls in the inputColVector */ {
-
-      // Carefully handle NULLs...
-
-      /*
-       * For better performance on LONG/DOUBLE we don't want the conditional
-       * statements inside the for loop.
-       */
-      outputColVector.noNulls = false;
-
-      if (batch.selectedInUse) {
-        for(int j=0; j != n; j++) {
-          int i = sel[j];
-          outputIsNull[i] = inputIsNull[i];
-          outputVector[i] = ((vector[i] - value) >>> 63) ^ 1;
-        }
-      } else {
-        System.arraycopy(inputIsNull, 0, outputIsNull, 0, n);
-        for(int i = 0; i != n; i++) {
-          outputVector[i] = ((vector[i] - value) >>> 63) ^ 1;
-        }
-      }
-    }
-  }
-
-  @Override
-  public String vectorExpressionParameters() {
-    return getColumnParamString(0, colNum) + ", val " + value;
-  }
-
-  @Override
-  public VectorExpressionDescriptor.Descriptor getDescriptor() {
-    return (new VectorExpressionDescriptor.Builder())
-        .setMode(
-            VectorExpressionDescriptor.Mode.PROJECTION)
-        .setNumArguments(2)
-        .setArgumentTypes(
-            VectorExpressionDescriptor.ArgumentType.getType("long"),
-            VectorExpressionDescriptor.ArgumentType.getType("long"))
-        .setInputExpressionTypes(
-            VectorExpressionDescriptor.InputExpressionType.COLUMN,
-            VectorExpressionDescriptor.InputExpressionType.SCALAR).build();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColGreaterLongColumn.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColGreaterLongColumn.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColGreaterLongColumn.java
deleted file mode 100644
index 3e1a13e..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColGreaterLongColumn.java
+++ /dev/null
@@ -1,146 +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.LongColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
-import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
-import org.apache.hadoop.hive.ql.metadata.HiveException;
-
-public class LongColGreaterLongColumn extends VectorExpression {
-
-  private static final long serialVersionUID = 1L;
-
-  private final int colNum1;
-  private final int colNum2;
-
-  public LongColGreaterLongColumn(int colNum1, int colNum2, int outputColumnNum) {
-    super(outputColumnNum);
-    this.colNum1 = colNum1;
-    this.colNum2 = colNum2;
-  }
-
-  public LongColGreaterLongColumn() {
-    super();
-
-    // Dummy final assignments.
-    colNum1 = -1;
-    colNum2 = -1;
-  }
-
-  @Override
-  public void evaluate(VectorizedRowBatch batch) throws HiveException {
-
-    if (childExpressions != null) {
-      super.evaluateChildren(batch);
-    }
-
-    LongColumnVector inputColVector1 = (LongColumnVector) batch.cols[colNum1];
-    LongColumnVector inputColVector2 = (LongColumnVector) batch.cols[colNum2];
-    LongColumnVector outputColVector = (LongColumnVector) batch.cols[outputColumnNum];
-    int[] sel = batch.selected;
-    int n = batch.size;
-    long[] vector1 = inputColVector1.vector;
-    long[] vector2 = inputColVector2.vector;
-    long[] outputVector = outputColVector.vector;
-    long vector1Value = vector1[0];
-    long vector2Value = vector2[0];
-
-    // return immediately if batch is empty
-    if (n == 0) {
-      return;
-    }
-
-    /*
-     * Propagate null values for a two-input operator and set isRepeating and noNulls appropriately.
-     */
-    NullUtil.propagateNullsColCol(
-        inputColVector1, inputColVector2, outputColVector, sel, n, batch.selectedInUse);
-
-    /* Disregard nulls for processing. In other words,
-     * the arithmetic operation is performed even if one or
-     * more inputs are null. This is to improve speed by avoiding
-     * conditional checks in the inner loop.
-     */
-    if (inputColVector1.isRepeating && inputColVector2.isRepeating) {
-      outputVector[0] = vector1Value > vector2Value ? 1 : 0;
-    } else if (inputColVector1.isRepeating) {
-      if (batch.selectedInUse) {
-        for(int j = 0; j != n; j++) {
-          int i = sel[j];
-          outputVector[i] = vector1Value > vector2[i] ? 1 : 0;
-        }
-      } else {
-        for(int i = 0; i != n; i++) {
-          // The SIMD optimized form of "a > b" is "(b - a) >>> 63"
-          outputVector[i] = (vector2[i] - vector1Value) >>> 63;
-        }
-      }
-    } else if (inputColVector2.isRepeating) {
-      if (batch.selectedInUse) {
-        for(int j = 0; j != n; j++) {
-          int i = sel[j];
-          outputVector[i] = vector1[i] > vector2Value ? 1 : 0;
-        }
-      } else {
-        for(int i = 0; i != n; i++) {
-          outputVector[i] = (vector2Value - vector1[i]) >>> 63;
-        }
-      }
-    } else {
-      if (batch.selectedInUse) {
-        for(int j = 0; j != n; j++) {
-          int i = sel[j];
-          outputVector[i] = vector1[i] > vector2[i] ? 1 : 0;
-        }
-      } else {
-        for(int i = 0; i != n; i++) {
-          outputVector[i] = (vector2[i] - vector1[i]) >>> 63;
-        }
-      }
-    }
-
-    /* For the case when the output can have null values, follow
-     * the convention that the data values must be 1 for long and
-     * NaN for double. This is to prevent possible later zero-divide errors
-     * in complex arithmetic expressions like col2 / (col1 - 1)
-     * in the case when some col1 entries are null.
-     */
-    NullUtil.setNullDataEntriesLong(outputColVector, batch.selectedInUse, sel, n);
-  }
-
-  @Override
-  public String vectorExpressionParameters() {
-    return getColumnParamString(0, colNum1) + ", " + getColumnParamString(1, colNum2);
-  }
-
-  @Override
-  public VectorExpressionDescriptor.Descriptor getDescriptor() {
-    return (new VectorExpressionDescriptor.Builder())
-        .setMode(
-            VectorExpressionDescriptor.Mode.PROJECTION)
-        .setNumArguments(2)
-        .setArgumentTypes(
-            VectorExpressionDescriptor.ArgumentType.getType("int_date_interval_year_month"),
-            VectorExpressionDescriptor.ArgumentType.getType("int_date_interval_year_month"))
-        .setInputExpressionTypes(
-            VectorExpressionDescriptor.InputExpressionType.COLUMN,
-            VectorExpressionDescriptor.InputExpressionType.COLUMN).build();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColGreaterLongScalar.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColGreaterLongScalar.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColGreaterLongScalar.java
deleted file mode 100644
index a4deedf..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColGreaterLongScalar.java
+++ /dev/null
@@ -1,157 +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 java.util.Arrays;
-
-import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
-import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
-import org.apache.hadoop.hive.ql.metadata.HiveException;
-
-public class LongColGreaterLongScalar extends VectorExpression {
-  private static final long serialVersionUID = 1L;
-
-  protected final int colNum;
-  protected final long value;
-
-  public LongColGreaterLongScalar(int colNum, long value, int outputColumnNum) {
-    super(outputColumnNum);
-    this.colNum = colNum;
-    this.value = value;
-  }
-
-  public LongColGreaterLongScalar() {
-    super();
-
-    // Dummy final assignments.
-    colNum = -1;
-    value = 0;
-  }
-
-  @Override
-  public void evaluate(VectorizedRowBatch batch) throws HiveException {
-
-    if (childExpressions != null) {
-      super.evaluateChildren(batch);
-    }
-
-    LongColumnVector inputColVector = (LongColumnVector) batch.cols[colNum];
-    LongColumnVector outputColVector = (LongColumnVector) batch.cols[outputColumnNum];
-    int[] sel = batch.selected;
-    boolean[] inputIsNull = inputColVector.isNull;
-    boolean[] outputIsNull = outputColVector.isNull;
-    int n = batch.size;
-    long[] vector = inputColVector.vector;
-    long[] outputVector = outputColVector.vector;
-
-    // return immediately if batch is empty
-    if (n == 0) {
-      return;
-    }
-
-    // We do not need to do a column reset since we are carefully changing the output.
-    outputColVector.isRepeating = false;
-
-    if (inputColVector.isRepeating) {
-      if (inputColVector.noNulls || !inputIsNull[0]) {
-        outputIsNull[0] = false;
-        outputVector[0] = vector[0] > value ? 1 : 0;
-      } else {
-        outputIsNull[0] = true;
-        outputColVector.noNulls = false;
-      }
-      outputColVector.isRepeating = true;
-      return;
-    }
-
-    if (inputColVector.noNulls) {
-      if (batch.selectedInUse) {
-
-        // CONSIDER: For large n, fill n or all of isNull array and use the tighter ELSE loop.
-
-        if (!outputColVector.noNulls) {
-          for(int j = 0; j != n; j++) {
-           final int i = sel[j];
-           // Set isNull before call in case it changes it mind.
-           outputIsNull[i] = false;
-           outputVector[i] = (value - vector[i]) >>> 63;
-         }
-        } else {
-          for(int j = 0; j != n; j++) {
-            final int i = sel[j];
-            outputVector[i] = (value - vector[i]) >>> 63;
-          }
-        }
-      } else {
-        if (!outputColVector.noNulls) {
-
-          // Assume it is almost always a performance win to fill all of isNull so we can
-          // safely reset noNulls.
-          Arrays.fill(outputIsNull, false);
-          outputColVector.noNulls = true;
-        }
-        for(int i = 0; i != n; i++) {
-          outputVector[i] = (value - vector[i]) >>> 63;
-        }
-      }
-    } else /* there are nulls in the inputColVector */ {
-
-      // Carefully handle NULLs...
-
-      /*
-       * For better performance on LONG/DOUBLE we don't want the conditional
-       * statements inside the for loop.
-       */
-      outputColVector.noNulls = false;
-
-      if (batch.selectedInUse) {
-        for(int j=0; j != n; j++) {
-          int i = sel[j];
-          outputIsNull[i] = inputIsNull[i];
-          outputVector[i] = (value - vector[i]) >>> 63;
-        }
-      } else {
-        System.arraycopy(inputIsNull, 0, outputIsNull, 0, n);
-        for(int i = 0; i != n; i++) {
-          outputVector[i] = (value - vector[i]) >>> 63;
-        }
-      }
-    }
-  }
-
-  @Override
-  public String vectorExpressionParameters() {
-    return getColumnParamString(0, colNum) + ", val " + value;
-  }
-
-  @Override
-  public VectorExpressionDescriptor.Descriptor getDescriptor() {
-    return (new VectorExpressionDescriptor.Builder())
-        .setMode(
-            VectorExpressionDescriptor.Mode.PROJECTION)
-        .setNumArguments(2)
-        .setArgumentTypes(
-            VectorExpressionDescriptor.ArgumentType.getType("long"),
-            VectorExpressionDescriptor.ArgumentType.getType("long"))
-        .setInputExpressionTypes(
-            VectorExpressionDescriptor.InputExpressionType.COLUMN,
-            VectorExpressionDescriptor.InputExpressionType.SCALAR).build();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColLessEqualLongColumn.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColLessEqualLongColumn.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColLessEqualLongColumn.java
deleted file mode 100644
index 3212269..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColLessEqualLongColumn.java
+++ /dev/null
@@ -1,146 +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.LongColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
-import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
-import org.apache.hadoop.hive.ql.metadata.HiveException;
-
-public class LongColLessEqualLongColumn extends VectorExpression {
-
-  private static final long serialVersionUID = 1L;
-
-  private final int colNum1;
-  private final int colNum2;
-
-  public LongColLessEqualLongColumn(int colNum1, int colNum2, int outputColumnNum) {
-    super(outputColumnNum);
-    this.colNum1 = colNum1;
-    this.colNum2 = colNum2;
-  }
-
-  public LongColLessEqualLongColumn() {
-    super();
-
-    // Dummy final assignments.
-    colNum1 = -1;
-    colNum2 = -1;
-  }
-
-  @Override
-  public void evaluate(VectorizedRowBatch batch) throws HiveException {
-
-    if (childExpressions != null) {
-      super.evaluateChildren(batch);
-    }
-
-    LongColumnVector inputColVector1 = (LongColumnVector) batch.cols[colNum1];
-    LongColumnVector inputColVector2 = (LongColumnVector) batch.cols[colNum2];
-    LongColumnVector outputColVector = (LongColumnVector) batch.cols[outputColumnNum];
-    int[] sel = batch.selected;
-    int n = batch.size;
-    long[] vector1 = inputColVector1.vector;
-    long[] vector2 = inputColVector2.vector;
-    long[] outputVector = outputColVector.vector;
-    long vector1Value = vector1[0];
-    long vector2Value = vector2[0];
-
-    // return immediately if batch is empty
-    if (n == 0) {
-      return;
-    }
-
-    /*
-     * Propagate null values for a two-input operator and set isRepeating and noNulls appropriately.
-     */ 
-    NullUtil.propagateNullsColCol(
-        inputColVector1, inputColVector2, outputColVector, sel, n, batch.selectedInUse);
-
-    /* Disregard nulls for processing. In other words,
-     * the arithmetic operation is performed even if one or
-     * more inputs are null. This is to improve speed by avoiding
-     * conditional checks in the inner loop.
-     */
-    if (inputColVector1.isRepeating && inputColVector2.isRepeating) {
-      outputVector[0] = vector1Value <= vector2Value ? 1 : 0;
-    } else if (inputColVector1.isRepeating) {
-      if (batch.selectedInUse) {
-        for(int j = 0; j != n; j++) {
-          int i = sel[j];
-          outputVector[i] = vector1Value <= vector2[i] ? 1 : 0;
-        }
-      } else {
-        for(int i = 0; i != n; i++) {
-          // The SIMD optimized form of "a <= b" is "((b - a) >>> 63) ^ 1"
-          outputVector[i] = ((vector2[i] - vector1Value) >>> 63) ^ 1;
-        }
-      }
-    } else if (inputColVector2.isRepeating) {
-      if (batch.selectedInUse) {
-        for(int j = 0; j != n; j++) {
-          int i = sel[j];
-          outputVector[i] = vector1[i] <= vector2Value ? 1 : 0;
-        }
-      } else {
-        for(int i = 0; i != n; i++) {
-          outputVector[i] = ((vector2Value - vector1[i]) >>> 63) ^ 1;
-        }
-      }
-    } else {
-      if (batch.selectedInUse) {
-        for(int j = 0; j != n; j++) {
-          int i = sel[j];
-          outputVector[i] = vector1[i] <= vector2[i] ? 1 : 0;
-        }
-      } else {
-        for(int i = 0; i != n; i++) {
-          outputVector[i] = ((vector2[i] - vector1[i]) >>> 63) ^ 1;
-        }
-      }
-    }
-
-    /* For the case when the output can have null values, follow
-     * the convention that the data values must be 1 for long and
-     * NaN for double. This is to prevent possible later zero-divide errors
-     * in complex arithmetic expressions like col2 / (col1 - 1)
-     * in the case when some col1 entries are null.
-     */
-    NullUtil.setNullDataEntriesLong(outputColVector, batch.selectedInUse, sel, n);
-  }
-
-  @Override
-  public String vectorExpressionParameters() {
-    return getColumnParamString(0, colNum1) + ", " + getColumnParamString(1, colNum2);
-  }
-
-  @Override
-  public VectorExpressionDescriptor.Descriptor getDescriptor() {
-    return (new VectorExpressionDescriptor.Builder())
-        .setMode(
-            VectorExpressionDescriptor.Mode.PROJECTION)
-        .setNumArguments(2)
-        .setArgumentTypes(
-            VectorExpressionDescriptor.ArgumentType.getType("int_date_interval_year_month"),
-            VectorExpressionDescriptor.ArgumentType.getType("int_date_interval_year_month"))
-        .setInputExpressionTypes(
-            VectorExpressionDescriptor.InputExpressionType.COLUMN,
-            VectorExpressionDescriptor.InputExpressionType.COLUMN).build();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColLessEqualLongScalar.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColLessEqualLongScalar.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColLessEqualLongScalar.java
deleted file mode 100644
index c2f0de1..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColLessEqualLongScalar.java
+++ /dev/null
@@ -1,158 +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 java.util.Arrays;
-
-import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
-import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
-import org.apache.hadoop.hive.ql.metadata.HiveException;
-
-public class LongColLessEqualLongScalar extends VectorExpression {
-
-  private static final long serialVersionUID = 1L;
-
-  protected int colNum;
-  protected long value;
-
-  public LongColLessEqualLongScalar(int colNum, long value, int outputColumnNum) {
-    super(outputColumnNum);
-    this.colNum = colNum;
-    this.value = value;
-  }
-
-  public LongColLessEqualLongScalar() {
-    super();
-
-    // Dummy final assignments.
-    colNum = -1;
-    value = 0;
-  }
-
-  @Override
-  public void evaluate(VectorizedRowBatch batch) throws HiveException {
-
-    if (childExpressions != null) {
-      super.evaluateChildren(batch);
-    }
-
-    LongColumnVector inputColVector = (LongColumnVector) batch.cols[colNum];
-    LongColumnVector outputColVector = (LongColumnVector) batch.cols[outputColumnNum];
-    int[] sel = batch.selected;
-    boolean[] inputIsNull = inputColVector.isNull;
-    boolean[] outputIsNull = outputColVector.isNull;
-    int n = batch.size;
-    long[] vector = inputColVector.vector;
-    long[] outputVector = outputColVector.vector;
-
-    // return immediately if batch is empty
-    if (n == 0) {
-      return;
-    }
-
-    // We do not need to do a column reset since we are carefully changing the output.
-    outputColVector.isRepeating = false;
-
-    if (inputColVector.isRepeating) {
-      if (inputColVector.noNulls || !inputIsNull[0]) {
-        outputIsNull[0] = false;
-        outputVector[0] = vector[0] <= value ? 1 : 0;
-      } else {
-        outputIsNull[0] = true;
-        outputColVector.noNulls = false;
-      }
-      outputColVector.isRepeating = true;
-      return;
-    }
-
-    if (inputColVector.noNulls) {
-      if (batch.selectedInUse) {
-
-        // CONSIDER: For large n, fill n or all of isNull array and use the tighter ELSE loop.
-
-        if (!outputColVector.noNulls) {
-          for(int j = 0; j != n; j++) {
-           final int i = sel[j];
-           // Set isNull before call in case it changes it mind.
-           outputIsNull[i] = false;
-           outputVector[i] = ((value - vector[i]) >>> 63) ^ 1;
-         }
-        } else {
-          for(int j = 0; j != n; j++) {
-            final int i = sel[j];
-            outputVector[i] = ((value - vector[i]) >>> 63) ^ 1;
-          }
-        }
-      } else {
-        if (!outputColVector.noNulls) {
-
-          // Assume it is almost always a performance win to fill all of isNull so we can
-          // safely reset noNulls.
-          Arrays.fill(outputIsNull, false);
-          outputColVector.noNulls = true;
-        }
-        for(int i = 0; i != n; i++) {
-          outputVector[i] = ((value - vector[i]) >>> 63) ^ 1;
-        }
-      }
-    } else /* there are nulls in the inputColVector */ {
-
-      // Carefully handle NULLs...
-
-      /*
-       * For better performance on LONG/DOUBLE we don't want the conditional
-       * statements inside the for loop.
-       */
-      outputColVector.noNulls = false;
-
-      if (batch.selectedInUse) {
-        for(int j=0; j != n; j++) {
-          int i = sel[j];
-          outputIsNull[i] = inputIsNull[i];
-          outputVector[i] = ((value - vector[i]) >>> 63) ^ 1;
-        }
-      } else {
-        System.arraycopy(inputIsNull, 0, outputIsNull, 0, n);
-        for(int i = 0; i != n; i++) {
-          outputVector[i] = ((value - vector[i]) >>> 63) ^ 1;
-        }
-      }
-    }
-  }
-
-  @Override
-  public String vectorExpressionParameters() {
-    return getColumnParamString(0, colNum) + ", val " + value;
-  }
-
-  @Override
-  public VectorExpressionDescriptor.Descriptor getDescriptor() {
-    return (new VectorExpressionDescriptor.Builder())
-        .setMode(
-            VectorExpressionDescriptor.Mode.PROJECTION)
-        .setNumArguments(2)
-        .setArgumentTypes(
-            VectorExpressionDescriptor.ArgumentType.getType("long"),
-            VectorExpressionDescriptor.ArgumentType.getType("long"))
-        .setInputExpressionTypes(
-            VectorExpressionDescriptor.InputExpressionType.COLUMN,
-            VectorExpressionDescriptor.InputExpressionType.SCALAR).build();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColLessLongColumn.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColLessLongColumn.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColLessLongColumn.java
deleted file mode 100644
index 7c2ee20..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColLessLongColumn.java
+++ /dev/null
@@ -1,146 +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.LongColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
-import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
-import org.apache.hadoop.hive.ql.metadata.HiveException;
-
-public class LongColLessLongColumn extends VectorExpression {
-
-  private static final long serialVersionUID = 1L;
-
-  private final int colNum1;
-  private final int colNum2;
-
-  public LongColLessLongColumn(int colNum1, int colNum2, int outputColumnNum) {
-    super(outputColumnNum);
-    this.colNum1 = colNum1;
-    this.colNum2 = colNum2;
-  }
-
-  public LongColLessLongColumn() {
-    super();
-
-    // Dummy final assignments.
-    colNum1 = -1;
-    colNum2 = -1;
-  }
-
-  @Override
-  public void evaluate(VectorizedRowBatch batch) throws HiveException {
-
-    if (childExpressions != null) {
-      super.evaluateChildren(batch);
-    }
-
-    LongColumnVector inputColVector1 = (LongColumnVector) batch.cols[colNum1];
-    LongColumnVector inputColVector2 = (LongColumnVector) batch.cols[colNum2];
-    LongColumnVector outputColVector = (LongColumnVector) batch.cols[outputColumnNum];
-    int[] sel = batch.selected;
-    int n = batch.size;
-    long[] vector1 = inputColVector1.vector;
-    long[] vector2 = inputColVector2.vector;
-    long[] outputVector = outputColVector.vector;
-    long vector1Value = vector1[0];
-    long vector2Value = vector2[0];
-
-    // return immediately if batch is empty
-    if (n == 0) {
-      return;
-    }
-
-    /*
-     * Propagate null values for a two-input operator and set isRepeating and noNulls appropriately.
-     */
-    NullUtil.propagateNullsColCol(
-        inputColVector1, inputColVector2, outputColVector, sel, n, batch.selectedInUse);
-
-    /* Disregard nulls for processing. In other words,
-     * the arithmetic operation is performed even if one or
-     * more inputs are null. This is to improve speed by avoiding
-     * conditional checks in the inner loop.
-     */
-    if (inputColVector1.isRepeating && inputColVector2.isRepeating) {
-      outputVector[0] = vector1Value < vector2Value ? 1 : 0;
-    } else if (inputColVector1.isRepeating) {
-      if (batch.selectedInUse) {
-        for(int j = 0; j != n; j++) {
-          int i = sel[j];
-          outputVector[i] = vector1Value < vector2[i] ? 1 : 0;
-        }
-      } else {
-        for(int i = 0; i != n; i++) {
-          // The SIMD optimized form of "a < b" is "(a - b) >>> 63"
-          outputVector[i] = (vector1Value - vector2[i]) >>> 63;
-        }
-      }
-    } else if (inputColVector2.isRepeating) {
-      if (batch.selectedInUse) {
-        for(int j = 0; j != n; j++) {
-          int i = sel[j];
-          outputVector[i] = vector1[i] < vector2Value ? 1 : 0;
-        }
-      } else {
-        for(int i = 0; i != n; i++) {
-          outputVector[i] = (vector1[i] - vector2Value) >>> 63;
-        }
-      }
-    } else {
-      if (batch.selectedInUse) {
-        for(int j = 0; j != n; j++) {
-          int i = sel[j];
-          outputVector[i] = vector1[i] < vector2[i] ? 1 : 0;
-        }
-      } else {
-        for(int i = 0; i != n; i++) {
-          outputVector[i] = (vector1[i] - vector2[i]) >>> 63;
-        }
-      }
-    }
-
-    /* For the case when the output can have null values, follow
-     * the convention that the data values must be 1 for long and
-     * NaN for double. This is to prevent possible later zero-divide errors
-     * in complex arithmetic expressions like col2 / (col1 - 1)
-     * in the case when some col1 entries are null.
-     */
-    NullUtil.setNullDataEntriesLong(outputColVector, batch.selectedInUse, sel, n);
-  }
-
-  @Override
-  public String vectorExpressionParameters() {
-    return getColumnParamString(0, colNum1) + ", " + getColumnParamString(1, colNum2);
-  }
-
-  @Override
-  public VectorExpressionDescriptor.Descriptor getDescriptor() {
-    return (new VectorExpressionDescriptor.Builder())
-        .setMode(
-            VectorExpressionDescriptor.Mode.PROJECTION)
-        .setNumArguments(2)
-        .setArgumentTypes(
-            VectorExpressionDescriptor.ArgumentType.getType("int_date_interval_year_month"),
-            VectorExpressionDescriptor.ArgumentType.getType("int_date_interval_year_month"))
-        .setInputExpressionTypes(
-            VectorExpressionDescriptor.InputExpressionType.COLUMN,
-            VectorExpressionDescriptor.InputExpressionType.COLUMN).build();
-  }
-}


[3/8] hive git commit: HIVE-20207: Vectorization: Fix NULL / Wrong Results issues in Filter / Compare (Matt McCline, reviewed by Teddy Choi)

Posted by mm...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/test/results/clientpositive/llap/vector_decimal_mapjoin.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/vector_decimal_mapjoin.q.out b/ql/src/test/results/clientpositive/llap/vector_decimal_mapjoin.q.out
index 27d6c1c..b66fb9f 100644
--- a/ql/src/test/results/clientpositive/llap/vector_decimal_mapjoin.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_decimal_mapjoin.q.out
@@ -740,7 +740,7 @@ STAGE PLANS:
                     Filter Vectorization:
                         className: VectorFilterOperator
                         native: true
-                        predicateExpression: SelectColumnIsNotNull(col 3:decimal(14,2))(children: ConvertDecimal64ToDecimal(col 0:decimal(14,2)/DECIMAL_64) -> 3:decimal(14,2))
+                        predicateExpression: SelectColumnIsNotNull(col 0:decimal(14,2)/DECIMAL_64)
                     predicate: dec is not null (type: boolean)
                     Statistics: Num rows: 997 Data size: 106235 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
@@ -758,8 +758,8 @@ STAGE PLANS:
                           0 _col0 (type: decimal(16,2))
                           1 _col0 (type: decimal(16,2))
                         Map Join Vectorization:
-                            bigTableKeyExpressions: ConvertDecimal64ToDecimal(col 0:decimal(16,2)/DECIMAL_64) -> 4:decimal(16,2)
-                            bigTableValueExpressions: ConvertDecimal64ToDecimal(col 0:decimal(16,2)/DECIMAL_64) -> 5:decimal(16,2)
+                            bigTableKeyExpressions: ConvertDecimal64ToDecimal(col 0:decimal(16,2)/DECIMAL_64) -> 3:decimal(16,2)
+                            bigTableValueExpressions: ConvertDecimal64ToDecimal(col 0:decimal(16,2)/DECIMAL_64) -> 4:decimal(16,2)
                             className: VectorMapJoinOperator
                             native: false
                             nativeConditionsMet: hive.mapjoin.optimized.hashtable IS true, hive.vectorized.execution.mapjoin.native.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, One MapJoin Condition IS true, No nullsafe IS true, Small table vectorizes IS true
@@ -795,7 +795,7 @@ STAGE PLANS:
                     includeColumns: [0]
                     dataColumns: dec:decimal(14,2)/DECIMAL_64, value_dec:decimal(14,2)/DECIMAL_64
                     partitionColumnCount: 0
-                    scratchColumnTypeNames: [decimal(14,2), decimal(16,2), decimal(16,2)]
+                    scratchColumnTypeNames: [decimal(16,2), decimal(16,2)]
         Map 2 
             Map Operator Tree:
                 TableScan
@@ -809,7 +809,7 @@ STAGE PLANS:
                     Filter Vectorization:
                         className: VectorFilterOperator
                         native: true
-                        predicateExpression: SelectColumnIsNotNull(col 3:decimal(14,0))(children: ConvertDecimal64ToDecimal(col 0:decimal(14,0)/DECIMAL_64) -> 3:decimal(14,0))
+                        predicateExpression: SelectColumnIsNotNull(col 0:decimal(14,0)/DECIMAL_64)
                     predicate: dec is not null (type: boolean)
                     Statistics: Num rows: 997 Data size: 106235 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
@@ -847,7 +847,7 @@ STAGE PLANS:
                     includeColumns: [0]
                     dataColumns: dec:decimal(14,0)/DECIMAL_64, value_dec:decimal(14,0)/DECIMAL_64
                     partitionColumnCount: 0
-                    scratchColumnTypeNames: [decimal(14,0)]
+                    scratchColumnTypeNames: []
 
   Stage: Stage-0
     Fetch Operator
@@ -1017,7 +1017,7 @@ STAGE PLANS:
                     Filter Vectorization:
                         className: VectorFilterOperator
                         native: true
-                        predicateExpression: SelectColumnIsNotNull(col 3:decimal(14,2))(children: ConvertDecimal64ToDecimal(col 0:decimal(14,2)/DECIMAL_64) -> 3:decimal(14,2))
+                        predicateExpression: SelectColumnIsNotNull(col 0:decimal(14,2)/DECIMAL_64)
                     predicate: dec is not null (type: boolean)
                     Statistics: Num rows: 997 Data size: 212470 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
@@ -1035,8 +1035,8 @@ STAGE PLANS:
                           0 _col0 (type: decimal(16,2))
                           1 _col0 (type: decimal(16,2))
                         Map Join Vectorization:
-                            bigTableKeyExpressions: ConvertDecimal64ToDecimal(col 0:decimal(16,2)/DECIMAL_64) -> 4:decimal(16,2)
-                            bigTableValueExpressions: ConvertDecimal64ToDecimal(col 0:decimal(16,2)/DECIMAL_64) -> 5:decimal(16,2), ConvertDecimal64ToDecimal(col 1:decimal(14,2)/DECIMAL_64) -> 3:decimal(14,2)
+                            bigTableKeyExpressions: ConvertDecimal64ToDecimal(col 0:decimal(16,2)/DECIMAL_64) -> 3:decimal(16,2)
+                            bigTableValueExpressions: ConvertDecimal64ToDecimal(col 0:decimal(16,2)/DECIMAL_64) -> 4:decimal(16,2), ConvertDecimal64ToDecimal(col 1:decimal(14,2)/DECIMAL_64) -> 5:decimal(14,2)
                             className: VectorMapJoinOperator
                             native: false
                             nativeConditionsMet: hive.mapjoin.optimized.hashtable IS true, hive.vectorized.execution.mapjoin.native.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, One MapJoin Condition IS true, No nullsafe IS true, Small table vectorizes IS true
@@ -1072,7 +1072,7 @@ STAGE PLANS:
                     includeColumns: [0, 1]
                     dataColumns: dec:decimal(14,2)/DECIMAL_64, value_dec:decimal(14,2)/DECIMAL_64
                     partitionColumnCount: 0
-                    scratchColumnTypeNames: [decimal(14,2), decimal(16,2), decimal(16,2), decimal(14,0)]
+                    scratchColumnTypeNames: [decimal(16,2), decimal(16,2), decimal(14,2), decimal(14,0)]
         Map 2 
             Map Operator Tree:
                 TableScan
@@ -1086,7 +1086,7 @@ STAGE PLANS:
                     Filter Vectorization:
                         className: VectorFilterOperator
                         native: true
-                        predicateExpression: SelectColumnIsNotNull(col 3:decimal(14,0))(children: ConvertDecimal64ToDecimal(col 0:decimal(14,0)/DECIMAL_64) -> 3:decimal(14,0))
+                        predicateExpression: SelectColumnIsNotNull(col 0:decimal(14,0)/DECIMAL_64)
                     predicate: dec is not null (type: boolean)
                     Statistics: Num rows: 997 Data size: 212470 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
@@ -1125,7 +1125,7 @@ STAGE PLANS:
                     includeColumns: [0, 1]
                     dataColumns: dec:decimal(14,0)/DECIMAL_64, value_dec:decimal(14,0)/DECIMAL_64
                     partitionColumnCount: 0
-                    scratchColumnTypeNames: [decimal(14,0)]
+                    scratchColumnTypeNames: []
 
   Stage: Stage-0
     Fetch Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/test/results/clientpositive/llap/vector_outer_reference_windowed.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/vector_outer_reference_windowed.q.out b/ql/src/test/results/clientpositive/llap/vector_outer_reference_windowed.q.out
index 897c145..ba2d9df 100644
--- a/ql/src/test/results/clientpositive/llap/vector_outer_reference_windowed.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_outer_reference_windowed.q.out
@@ -730,7 +730,7 @@ STAGE PLANS:
                     Filter Vectorization:
                         className: VectorFilterOperator
                         native: true
-                        predicateExpression: SelectColumnIsNotNull(col 3:decimal(15,2))(children: ConvertDecimal64ToDecimal(col 0:decimal(15,2)/DECIMAL_64) -> 3:decimal(15,2))
+                        predicateExpression: SelectColumnIsNotNull(col 0:decimal(15,2)/DECIMAL_64)
                     predicate: c1 is not null (type: boolean)
                     Statistics: Num rows: 4 Data size: 896 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
@@ -769,7 +769,7 @@ STAGE PLANS:
                     includeColumns: [0, 1]
                     dataColumns: c1:decimal(15,2)/DECIMAL_64, c2:decimal(15,2)/DECIMAL_64
                     partitionColumnCount: 0
-                    scratchColumnTypeNames: [decimal(15,2)]
+                    scratchColumnTypeNames: []
         Map 5 
             Map Operator Tree:
                 TableScan
@@ -783,7 +783,7 @@ STAGE PLANS:
                     Filter Vectorization:
                         className: VectorFilterOperator
                         native: true
-                        predicateExpression: SelectColumnIsNotNull(col 3:decimal(15,2))(children: ConvertDecimal64ToDecimal(col 0:decimal(15,2)/DECIMAL_64) -> 3:decimal(15,2))
+                        predicateExpression: SelectColumnIsNotNull(col 0:decimal(15,2)/DECIMAL_64)
                     predicate: c1 is not null (type: boolean)
                     Statistics: Num rows: 4 Data size: 448 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
@@ -821,7 +821,7 @@ STAGE PLANS:
                     includeColumns: [0]
                     dataColumns: c1:decimal(15,2)/DECIMAL_64, c2:decimal(15,2)/DECIMAL_64
                     partitionColumnCount: 0
-                    scratchColumnTypeNames: [decimal(15,2)]
+                    scratchColumnTypeNames: []
         Reducer 2 
             Execution mode: llap
             Reduce Operator Tree:
@@ -1039,7 +1039,7 @@ STAGE PLANS:
                     Filter Vectorization:
                         className: VectorFilterOperator
                         native: true
-                        predicateExpression: SelectColumnIsNotNull(col 3:decimal(15,2))(children: ConvertDecimal64ToDecimal(col 0:decimal(15,2)/DECIMAL_64) -> 3:decimal(15,2))
+                        predicateExpression: SelectColumnIsNotNull(col 0:decimal(15,2)/DECIMAL_64)
                     predicate: c1 is not null (type: boolean)
                     Statistics: Num rows: 4 Data size: 448 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
@@ -1077,7 +1077,7 @@ STAGE PLANS:
                     includeColumns: [0]
                     dataColumns: c1:decimal(15,2)/DECIMAL_64, c2:decimal(15,2)/DECIMAL_64
                     partitionColumnCount: 0
-                    scratchColumnTypeNames: [decimal(15,2)]
+                    scratchColumnTypeNames: []
         Map 5 
             Map Operator Tree:
                 TableScan
@@ -1091,7 +1091,7 @@ STAGE PLANS:
                     Filter Vectorization:
                         className: VectorFilterOperator
                         native: true
-                        predicateExpression: SelectColumnIsNotNull(col 3:decimal(15,2))(children: ConvertDecimal64ToDecimal(col 0:decimal(15,2)/DECIMAL_64) -> 3:decimal(15,2))
+                        predicateExpression: SelectColumnIsNotNull(col 0:decimal(15,2)/DECIMAL_64)
                     predicate: c1 is not null (type: boolean)
                     Statistics: Num rows: 4 Data size: 896 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
@@ -1130,7 +1130,7 @@ STAGE PLANS:
                     includeColumns: [0, 1]
                     dataColumns: c1:decimal(15,2)/DECIMAL_64, c2:decimal(15,2)/DECIMAL_64
                     partitionColumnCount: 0
-                    scratchColumnTypeNames: [decimal(15,2)]
+                    scratchColumnTypeNames: []
         Reducer 2 
             Execution mode: llap
             Reduce Operator Tree:
@@ -1347,7 +1347,7 @@ STAGE PLANS:
                     Filter Vectorization:
                         className: VectorFilterOperator
                         native: true
-                        predicateExpression: SelectColumnIsNotNull(col 3:decimal(15,2))(children: ConvertDecimal64ToDecimal(col 0:decimal(15,2)/DECIMAL_64) -> 3:decimal(15,2))
+                        predicateExpression: SelectColumnIsNotNull(col 0:decimal(15,2)/DECIMAL_64)
                     predicate: c1 is not null (type: boolean)
                     Statistics: Num rows: 4 Data size: 896 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
@@ -1386,7 +1386,7 @@ STAGE PLANS:
                     includeColumns: [0, 1]
                     dataColumns: c1:decimal(15,2)/DECIMAL_64, c2:decimal(15,2)/DECIMAL_64
                     partitionColumnCount: 0
-                    scratchColumnTypeNames: [decimal(15,2)]
+                    scratchColumnTypeNames: []
         Map 4 
             Map Operator Tree:
                 TableScan
@@ -1400,7 +1400,7 @@ STAGE PLANS:
                     Filter Vectorization:
                         className: VectorFilterOperator
                         native: true
-                        predicateExpression: SelectColumnIsNotNull(col 3:decimal(15,2))(children: ConvertDecimal64ToDecimal(col 0:decimal(15,2)/DECIMAL_64) -> 3:decimal(15,2))
+                        predicateExpression: SelectColumnIsNotNull(col 0:decimal(15,2)/DECIMAL_64)
                     predicate: c1 is not null (type: boolean)
                     Statistics: Num rows: 4 Data size: 896 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
@@ -1439,7 +1439,7 @@ STAGE PLANS:
                     includeColumns: [0, 1]
                     dataColumns: c1:decimal(15,2)/DECIMAL_64, c2:decimal(15,2)/DECIMAL_64
                     partitionColumnCount: 0
-                    scratchColumnTypeNames: [decimal(15,2)]
+                    scratchColumnTypeNames: []
         Reducer 2 
             Execution mode: llap
             Reduce Operator Tree:
@@ -2028,7 +2028,7 @@ STAGE PLANS:
                     Filter Vectorization:
                         className: VectorFilterOperator
                         native: true
-                        predicateExpression: SelectColumnIsNotNull(col 3:decimal(7,2))(children: ConvertDecimal64ToDecimal(col 0:decimal(7,2)/DECIMAL_64) -> 3:decimal(7,2))
+                        predicateExpression: SelectColumnIsNotNull(col 0:decimal(7,2)/DECIMAL_64)
                     predicate: c1 is not null (type: boolean)
                     Statistics: Num rows: 4 Data size: 896 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
@@ -2067,7 +2067,7 @@ STAGE PLANS:
                     includeColumns: [0, 1]
                     dataColumns: c1:decimal(7,2)/DECIMAL_64, c2:decimal(7,2)/DECIMAL_64
                     partitionColumnCount: 0
-                    scratchColumnTypeNames: [decimal(7,2)]
+                    scratchColumnTypeNames: []
         Map 5 
             Map Operator Tree:
                 TableScan
@@ -2081,7 +2081,7 @@ STAGE PLANS:
                     Filter Vectorization:
                         className: VectorFilterOperator
                         native: true
-                        predicateExpression: SelectColumnIsNotNull(col 3:decimal(7,2))(children: ConvertDecimal64ToDecimal(col 0:decimal(7,2)/DECIMAL_64) -> 3:decimal(7,2))
+                        predicateExpression: SelectColumnIsNotNull(col 0:decimal(7,2)/DECIMAL_64)
                     predicate: c1 is not null (type: boolean)
                     Statistics: Num rows: 4 Data size: 448 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
@@ -2119,7 +2119,7 @@ STAGE PLANS:
                     includeColumns: [0]
                     dataColumns: c1:decimal(7,2)/DECIMAL_64, c2:decimal(7,2)/DECIMAL_64
                     partitionColumnCount: 0
-                    scratchColumnTypeNames: [decimal(7,2)]
+                    scratchColumnTypeNames: []
         Reducer 2 
             Execution mode: llap
             Reduce Operator Tree:
@@ -2337,7 +2337,7 @@ STAGE PLANS:
                     Filter Vectorization:
                         className: VectorFilterOperator
                         native: true
-                        predicateExpression: SelectColumnIsNotNull(col 3:decimal(7,2))(children: ConvertDecimal64ToDecimal(col 0:decimal(7,2)/DECIMAL_64) -> 3:decimal(7,2))
+                        predicateExpression: SelectColumnIsNotNull(col 0:decimal(7,2)/DECIMAL_64)
                     predicate: c1 is not null (type: boolean)
                     Statistics: Num rows: 4 Data size: 448 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
@@ -2375,7 +2375,7 @@ STAGE PLANS:
                     includeColumns: [0]
                     dataColumns: c1:decimal(7,2)/DECIMAL_64, c2:decimal(7,2)/DECIMAL_64
                     partitionColumnCount: 0
-                    scratchColumnTypeNames: [decimal(7,2)]
+                    scratchColumnTypeNames: []
         Map 5 
             Map Operator Tree:
                 TableScan
@@ -2389,7 +2389,7 @@ STAGE PLANS:
                     Filter Vectorization:
                         className: VectorFilterOperator
                         native: true
-                        predicateExpression: SelectColumnIsNotNull(col 3:decimal(7,2))(children: ConvertDecimal64ToDecimal(col 0:decimal(7,2)/DECIMAL_64) -> 3:decimal(7,2))
+                        predicateExpression: SelectColumnIsNotNull(col 0:decimal(7,2)/DECIMAL_64)
                     predicate: c1 is not null (type: boolean)
                     Statistics: Num rows: 4 Data size: 896 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
@@ -2428,7 +2428,7 @@ STAGE PLANS:
                     includeColumns: [0, 1]
                     dataColumns: c1:decimal(7,2)/DECIMAL_64, c2:decimal(7,2)/DECIMAL_64
                     partitionColumnCount: 0
-                    scratchColumnTypeNames: [decimal(7,2)]
+                    scratchColumnTypeNames: []
         Reducer 2 
             Execution mode: llap
             Reduce Operator Tree:
@@ -2645,7 +2645,7 @@ STAGE PLANS:
                     Filter Vectorization:
                         className: VectorFilterOperator
                         native: true
-                        predicateExpression: SelectColumnIsNotNull(col 3:decimal(7,2))(children: ConvertDecimal64ToDecimal(col 0:decimal(7,2)/DECIMAL_64) -> 3:decimal(7,2))
+                        predicateExpression: SelectColumnIsNotNull(col 0:decimal(7,2)/DECIMAL_64)
                     predicate: c1 is not null (type: boolean)
                     Statistics: Num rows: 4 Data size: 896 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
@@ -2684,7 +2684,7 @@ STAGE PLANS:
                     includeColumns: [0, 1]
                     dataColumns: c1:decimal(7,2)/DECIMAL_64, c2:decimal(7,2)/DECIMAL_64
                     partitionColumnCount: 0
-                    scratchColumnTypeNames: [decimal(7,2)]
+                    scratchColumnTypeNames: []
         Map 4 
             Map Operator Tree:
                 TableScan
@@ -2698,7 +2698,7 @@ STAGE PLANS:
                     Filter Vectorization:
                         className: VectorFilterOperator
                         native: true
-                        predicateExpression: SelectColumnIsNotNull(col 3:decimal(7,2))(children: ConvertDecimal64ToDecimal(col 0:decimal(7,2)/DECIMAL_64) -> 3:decimal(7,2))
+                        predicateExpression: SelectColumnIsNotNull(col 0:decimal(7,2)/DECIMAL_64)
                     predicate: c1 is not null (type: boolean)
                     Statistics: Num rows: 4 Data size: 896 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
@@ -2737,7 +2737,7 @@ STAGE PLANS:
                     includeColumns: [0, 1]
                     dataColumns: c1:decimal(7,2)/DECIMAL_64, c2:decimal(7,2)/DECIMAL_64
                     partitionColumnCount: 0
-                    scratchColumnTypeNames: [decimal(7,2)]
+                    scratchColumnTypeNames: []
         Reducer 2 
             Execution mode: llap
             Reduce Operator Tree:

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/test/results/clientpositive/llap/vector_udf_adaptor_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/vector_udf_adaptor_1.q.out b/ql/src/test/results/clientpositive/llap/vector_udf_adaptor_1.q.out
index e7a0ffb..1f3558b 100644
--- a/ql/src/test/results/clientpositive/llap/vector_udf_adaptor_1.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_udf_adaptor_1.q.out
@@ -295,8 +295,8 @@ STAGE PLANS:
                     Select Vectorization:
                         className: VectorSelectOperator
                         native: true
-                        projectedOutputColumnNums: [0, 1, 2, 5, 8, 11, 14, 16, 20]
-                        selectExpressions: IfExprColumnNull(col 4:boolean, col 1:int, null)(children: LongColLessLongScalar(col 1:int, val 40) -> 4:boolean, col 1:int) -> 5:int, IfExprColumnNull(col 6:boolean, col 7:timestamp, null)(children: LongColGreaterLongScalar(col 1:int, val 40) -> 6:boolean, ConstantVectorExpression(val 2011-01-01 01:01:01) -> 7:timestamp) -> 8:timestamp, IfExprColumnNull(col 10:boolean, col 0:string, null)(children: LongColGreaterLongScalar(col 9:int, val 8)(children: StringLength(col 0:string) -> 9:int) -> 10:boolean, col 0:string) -> 11:string, IfExprColumnNull(col 12:boolean, col 13:binary, null)(children: LongColLessLongScalar(col 9:int, val 8)(children: StringLength(col 0:string) -> 9:int) -> 12:boolean, VectorUDFAdaptor(CAST( name AS BINARY)) -> 13:binary) -> 14:binary, IfExprColumnNull(col 9:boolean, col 15:int, null)(children: LongColGreaterLongScalar(col 1:int, val 40) -> 9:boolean, StringLength(col 0:string) -> 15:int) -> 16:int, IfExprColumnNull(
 col 18:boolean, col 19:double, null)(children: LongColGreaterLongScalar(col 17:int, val 10)(children: StringLength(col 0:string) -> 17:int) -> 18:boolean, DoubleScalarMultiplyDoubleColumn(val 2.0, col 2:double) -> 19:double) -> 20:double
+                        projectedOutputColumnNums: [0, 1, 2, 5, 8, 11, 13, 15, 19]
+                        selectExpressions: IfExprColumnNull(col 4:boolean, col 1:int, null)(children: LongColLessLongScalar(col 1:int, val 40) -> 4:boolean, col 1:int) -> 5:int, IfExprColumnNull(col 6:boolean, col 7:timestamp, null)(children: LongColGreaterLongScalar(col 1:int, val 40) -> 6:boolean, ConstantVectorExpression(val 2011-01-01 01:01:01) -> 7:timestamp) -> 8:timestamp, IfExprColumnNull(col 10:boolean, col 0:string, null)(children: LongColGreaterLongScalar(col 9:int, val 8)(children: StringLength(col 0:string) -> 9:int) -> 10:boolean, col 0:string) -> 11:string, IfExprColumnNull(col 12:boolean, col 0:string, null)(children: LongColLessLongScalar(col 9:int, val 8)(children: StringLength(col 0:string) -> 9:int) -> 12:boolean, col 0:string) -> 13:binary, IfExprColumnNull(col 9:boolean, col 14:int, null)(children: LongColGreaterLongScalar(col 1:int, val 40) -> 9:boolean, StringLength(col 0:string) -> 14:int) -> 15:int, IfExprColumnNull(col 17:boolean, col 18:double, null)(chil
 dren: LongColGreaterLongScalar(col 16:int, val 10)(children: StringLength(col 0:string) -> 16:int) -> 17:boolean, DoubleScalarMultiplyDoubleColumn(val 2.0, col 2:double) -> 18:double) -> 19:double
                     Statistics: Num rows: 12 Data size: 2352 Basic stats: COMPLETE Column stats: NONE
                     File Output Operator
                       compressed: false
@@ -318,14 +318,14 @@ STAGE PLANS:
                 featureSupportInUse: [DECIMAL_64]
                 inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
                 allNative: false
-                usesVectorUDFAdaptor: true
+                usesVectorUDFAdaptor: false
                 vectorized: true
                 rowBatchContext:
                     dataColumnCount: 3
                     includeColumns: [0, 1, 2]
                     dataColumns: name:string, age:int, gpa:double
                     partitionColumnCount: 0
-                    scratchColumnTypeNames: [bigint, bigint, bigint, timestamp, timestamp, bigint, bigint, string, bigint, string, string, bigint, bigint, bigint, bigint, double, double]
+                    scratchColumnTypeNames: [bigint, bigint, bigint, timestamp, timestamp, bigint, bigint, string, bigint, string, bigint, bigint, bigint, bigint, double, double]
 
   Stage: Stage-2
     Dependency Collection
@@ -471,8 +471,8 @@ STAGE PLANS:
                     Select Vectorization:
                         className: VectorSelectOperator
                         native: true
-                        projectedOutputColumnNums: [0, 1, 2, 5, 8, 11, 14, 16, 20]
-                        selectExpressions: IfExprColumnNull(col 4:boolean, col 1:int, null)(children: LongColLessLongScalar(col 1:int, val 40) -> 4:boolean, col 1:int) -> 5:int, IfExprColumnNull(col 6:boolean, col 7:timestamp, null)(children: LongColGreaterLongScalar(col 1:int, val 40) -> 6:boolean, ConstantVectorExpression(val 2011-01-01 01:01:01) -> 7:timestamp) -> 8:timestamp, IfExprColumnNull(col 10:boolean, col 0:string, null)(children: LongColGreaterLongScalar(col 9:int, val 8)(children: StringLength(col 0:string) -> 9:int) -> 10:boolean, col 0:string) -> 11:string, IfExprCondExprNull(col 12:boolean, col 13:binary, null)(children: LongColLessLongScalar(col 9:int, val 8)(children: StringLength(col 0:string) -> 9:int) -> 12:boolean, VectorUDFAdaptor(CAST( name AS BINARY)) -> 13:binary) -> 14:binary, IfExprCondExprNull(col 9:boolean, col 15:int, null)(children: LongColGreaterLongScalar(col 1:int, val 40) -> 9:boolean, StringLength(col 0:string) -> 15:int) -> 16:int, IfExprCondExp
 rNull(col 18:boolean, col 19:double, null)(children: LongColGreaterLongScalar(col 17:int, val 10)(children: StringLength(col 0:string) -> 17:int) -> 18:boolean, DoubleScalarMultiplyDoubleColumn(val 2.0, col 2:double) -> 19:double) -> 20:double
+                        projectedOutputColumnNums: [0, 1, 2, 5, 8, 11, 13, 15, 19]
+                        selectExpressions: IfExprColumnNull(col 4:boolean, col 1:int, null)(children: LongColLessLongScalar(col 1:int, val 40) -> 4:boolean, col 1:int) -> 5:int, IfExprColumnNull(col 6:boolean, col 7:timestamp, null)(children: LongColGreaterLongScalar(col 1:int, val 40) -> 6:boolean, ConstantVectorExpression(val 2011-01-01 01:01:01) -> 7:timestamp) -> 8:timestamp, IfExprColumnNull(col 10:boolean, col 0:string, null)(children: LongColGreaterLongScalar(col 9:int, val 8)(children: StringLength(col 0:string) -> 9:int) -> 10:boolean, col 0:string) -> 11:string, IfExprCondExprNull(col 12:boolean, col 0:string, null)(children: LongColLessLongScalar(col 9:int, val 8)(children: StringLength(col 0:string) -> 9:int) -> 12:boolean, col 0:string) -> 13:binary, IfExprCondExprNull(col 9:boolean, col 14:int, null)(children: LongColGreaterLongScalar(col 1:int, val 40) -> 9:boolean, StringLength(col 0:string) -> 14:int) -> 15:int, IfExprCondExprNull(col 17:boolean, col 18:double, null
 )(children: LongColGreaterLongScalar(col 16:int, val 10)(children: StringLength(col 0:string) -> 16:int) -> 17:boolean, DoubleScalarMultiplyDoubleColumn(val 2.0, col 2:double) -> 18:double) -> 19:double
                     Statistics: Num rows: 12 Data size: 2352 Basic stats: COMPLETE Column stats: NONE
                     File Output Operator
                       compressed: false
@@ -494,14 +494,14 @@ STAGE PLANS:
                 featureSupportInUse: [DECIMAL_64]
                 inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
                 allNative: false
-                usesVectorUDFAdaptor: true
+                usesVectorUDFAdaptor: false
                 vectorized: true
                 rowBatchContext:
                     dataColumnCount: 3
                     includeColumns: [0, 1, 2]
                     dataColumns: name:string, age:int, gpa:double
                     partitionColumnCount: 0
-                    scratchColumnTypeNames: [bigint, bigint, bigint, timestamp, timestamp, bigint, bigint, string, bigint, string, string, bigint, bigint, bigint, bigint, double, double]
+                    scratchColumnTypeNames: [bigint, bigint, bigint, timestamp, timestamp, bigint, bigint, string, bigint, string, bigint, bigint, bigint, bigint, double, double]
 
   Stage: Stage-2
     Dependency Collection
@@ -803,8 +803,8 @@ STAGE PLANS:
                     Select Vectorization:
                         className: VectorSelectOperator
                         native: true
-                        projectedOutputColumnNums: [0, 1, 2, 5, 8, 11, 14, 16, 20]
-                        selectExpressions: IfExprNullColumn(col 4:boolean, null, col 1)(children: LongColLessLongScalar(col 1:int, val 40) -> 4:boolean, col 1:int) -> 5:int, IfExprNullColumn(col 6:boolean, null, col 7)(children: LongColGreaterLongScalar(col 1:int, val 40) -> 6:boolean, ConstantVectorExpression(val 2011-01-01 01:01:01) -> 7:timestamp) -> 8:timestamp, IfExprNullColumn(col 10:boolean, null, col 0)(children: LongColGreaterLongScalar(col 9:int, val 8)(children: StringLength(col 0:string) -> 9:int) -> 10:boolean, col 0:string) -> 11:string, IfExprNullColumn(col 12:boolean, null, col 13)(children: LongColLessLongScalar(col 9:int, val 8)(children: StringLength(col 0:string) -> 9:int) -> 12:boolean, VectorUDFAdaptor(CAST( name AS BINARY)) -> 13:binary) -> 14:binary, IfExprNullColumn(col 9:boolean, null, col 15)(children: LongColGreaterLongScalar(col 1:int, val 40) -> 9:boolean, StringLength(col 0:string) -> 15:int) -> 16:int, IfExprNullColumn(col 18:boolean, null, col 19)(ch
 ildren: LongColGreaterLongScalar(col 17:int, val 10)(children: StringLength(col 0:string) -> 17:int) -> 18:boolean, DoubleScalarMultiplyDoubleColumn(val 2.0, col 2:double) -> 19:double) -> 20:double
+                        projectedOutputColumnNums: [0, 1, 2, 5, 8, 11, 13, 15, 19]
+                        selectExpressions: IfExprNullColumn(col 4:boolean, null, col 1)(children: LongColLessLongScalar(col 1:int, val 40) -> 4:boolean, col 1:int) -> 5:int, IfExprNullColumn(col 6:boolean, null, col 7)(children: LongColGreaterLongScalar(col 1:int, val 40) -> 6:boolean, ConstantVectorExpression(val 2011-01-01 01:01:01) -> 7:timestamp) -> 8:timestamp, IfExprNullColumn(col 10:boolean, null, col 0)(children: LongColGreaterLongScalar(col 9:int, val 8)(children: StringLength(col 0:string) -> 9:int) -> 10:boolean, col 0:string) -> 11:string, IfExprNullColumn(col 12:boolean, null, col 0)(children: LongColLessLongScalar(col 9:int, val 8)(children: StringLength(col 0:string) -> 9:int) -> 12:boolean, col 0:string) -> 13:binary, IfExprNullColumn(col 9:boolean, null, col 14)(children: LongColGreaterLongScalar(col 1:int, val 40) -> 9:boolean, StringLength(col 0:string) -> 14:int) -> 15:int, IfExprNullColumn(col 17:boolean, null, col 18)(children: LongColGreaterLongScalar(col 16:i
 nt, val 10)(children: StringLength(col 0:string) -> 16:int) -> 17:boolean, DoubleScalarMultiplyDoubleColumn(val 2.0, col 2:double) -> 18:double) -> 19:double
                     Statistics: Num rows: 12 Data size: 2352 Basic stats: COMPLETE Column stats: NONE
                     File Output Operator
                       compressed: false
@@ -826,14 +826,14 @@ STAGE PLANS:
                 featureSupportInUse: [DECIMAL_64]
                 inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
                 allNative: false
-                usesVectorUDFAdaptor: true
+                usesVectorUDFAdaptor: false
                 vectorized: true
                 rowBatchContext:
                     dataColumnCount: 3
                     includeColumns: [0, 1, 2]
                     dataColumns: name:string, age:int, gpa:double
                     partitionColumnCount: 0
-                    scratchColumnTypeNames: [bigint, bigint, bigint, timestamp, timestamp, bigint, bigint, string, bigint, string, string, bigint, bigint, bigint, bigint, double, double]
+                    scratchColumnTypeNames: [bigint, bigint, bigint, timestamp, timestamp, bigint, bigint, string, bigint, string, bigint, bigint, bigint, bigint, double, double]
 
   Stage: Stage-2
     Dependency Collection
@@ -979,8 +979,8 @@ STAGE PLANS:
                     Select Vectorization:
                         className: VectorSelectOperator
                         native: true
-                        projectedOutputColumnNums: [0, 1, 2, 5, 8, 11, 14, 16, 20]
-                        selectExpressions: IfExprNullColumn(col 4:boolean, null, col 1)(children: LongColLessLongScalar(col 1:int, val 40) -> 4:boolean, col 1:int) -> 5:int, IfExprNullColumn(col 6:boolean, null, col 7)(children: LongColGreaterLongScalar(col 1:int, val 40) -> 6:boolean, ConstantVectorExpression(val 2011-01-01 01:01:01) -> 7:timestamp) -> 8:timestamp, IfExprNullColumn(col 10:boolean, null, col 0)(children: LongColGreaterLongScalar(col 9:int, val 8)(children: StringLength(col 0:string) -> 9:int) -> 10:boolean, col 0:string) -> 11:string, IfExprNullCondExpr(col 12:boolean, null, col 13:binary)(children: LongColLessLongScalar(col 9:int, val 8)(children: StringLength(col 0:string) -> 9:int) -> 12:boolean, VectorUDFAdaptor(CAST( name AS BINARY)) -> 13:binary) -> 14:binary, IfExprNullCondExpr(col 9:boolean, null, col 15:int)(children: LongColGreaterLongScalar(col 1:int, val 40) -> 9:boolean, StringLength(col 0:string) -> 15:int) -> 16:int, IfExprNullCondExpr(col 18:boolean,
  null, col 19:double)(children: LongColGreaterLongScalar(col 17:int, val 10)(children: StringLength(col 0:string) -> 17:int) -> 18:boolean, DoubleScalarMultiplyDoubleColumn(val 2.0, col 2:double) -> 19:double) -> 20:double
+                        projectedOutputColumnNums: [0, 1, 2, 5, 8, 11, 13, 15, 19]
+                        selectExpressions: IfExprNullColumn(col 4:boolean, null, col 1)(children: LongColLessLongScalar(col 1:int, val 40) -> 4:boolean, col 1:int) -> 5:int, IfExprNullColumn(col 6:boolean, null, col 7)(children: LongColGreaterLongScalar(col 1:int, val 40) -> 6:boolean, ConstantVectorExpression(val 2011-01-01 01:01:01) -> 7:timestamp) -> 8:timestamp, IfExprNullColumn(col 10:boolean, null, col 0)(children: LongColGreaterLongScalar(col 9:int, val 8)(children: StringLength(col 0:string) -> 9:int) -> 10:boolean, col 0:string) -> 11:string, IfExprNullCondExpr(col 12:boolean, null, col 0:string)(children: LongColLessLongScalar(col 9:int, val 8)(children: StringLength(col 0:string) -> 9:int) -> 12:boolean, col 0:string) -> 13:binary, IfExprNullCondExpr(col 9:boolean, null, col 14:int)(children: LongColGreaterLongScalar(col 1:int, val 40) -> 9:boolean, StringLength(col 0:string) -> 14:int) -> 15:int, IfExprNullCondExpr(col 17:boolean, null, col 18:double)(children: LongColGr
 eaterLongScalar(col 16:int, val 10)(children: StringLength(col 0:string) -> 16:int) -> 17:boolean, DoubleScalarMultiplyDoubleColumn(val 2.0, col 2:double) -> 18:double) -> 19:double
                     Statistics: Num rows: 12 Data size: 2352 Basic stats: COMPLETE Column stats: NONE
                     File Output Operator
                       compressed: false
@@ -1002,14 +1002,14 @@ STAGE PLANS:
                 featureSupportInUse: [DECIMAL_64]
                 inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
                 allNative: false
-                usesVectorUDFAdaptor: true
+                usesVectorUDFAdaptor: false
                 vectorized: true
                 rowBatchContext:
                     dataColumnCount: 3
                     includeColumns: [0, 1, 2]
                     dataColumns: name:string, age:int, gpa:double
                     partitionColumnCount: 0
-                    scratchColumnTypeNames: [bigint, bigint, bigint, timestamp, timestamp, bigint, bigint, string, bigint, string, string, bigint, bigint, bigint, bigint, double, double]
+                    scratchColumnTypeNames: [bigint, bigint, bigint, timestamp, timestamp, bigint, bigint, string, bigint, string, bigint, bigint, bigint, bigint, double, double]
 
   Stage: Stage-2
     Dependency Collection

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/test/results/clientpositive/llap/vectorized_case.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/vectorized_case.q.out b/ql/src/test/results/clientpositive/llap/vectorized_case.q.out
index 78098b4..7e5fee0 100644
--- a/ql/src/test/results/clientpositive/llap/vectorized_case.q.out
+++ b/ql/src/test/results/clientpositive/llap/vectorized_case.q.out
@@ -596,7 +596,7 @@ STAGE PLANS:
                         className: VectorSelectOperator
                         native: true
                         projectedOutputColumnNums: [6]
-                        selectExpressions: IfExprCondExprCondExpr(col 3:boolean, col 7:decimal(11,0)col 8:decimal(11,0))(children: VectorUDFAdaptor((member = 1)) -> 3:boolean, ConvertDecimal64ToDecimal(col 4:decimal(11,0)/DECIMAL_64)(children: Decimal64ColAddDecimal64Scalar(col 1:decimal(10,0)/DECIMAL_64, decimal64Val 1, decimalVal 1) -> 4:decimal(11,0)/DECIMAL_64) -> 7:decimal(11,0), ConvertDecimal64ToDecimal(col 5:decimal(11,0)/DECIMAL_64)(children: Decimal64ColAddDecimal64Scalar(col 1:decimal(10,0)/DECIMAL_64, decimal64Val 2, decimalVal 2) -> 5:decimal(11,0)/DECIMAL_64) -> 8:decimal(11,0)) -> 6:decimal(11,0)
+                        selectExpressions: IfExprCondExprCondExpr(col 3:boolean, col 7:decimal(11,0)col 8:decimal(11,0))(children: Decimal64ColEqualDecimal64Scalar(col 0:decimal(10,0)/DECIMAL_64, decimal64Val 1, decimalVal 1) -> 3:boolean, ConvertDecimal64ToDecimal(col 4:decimal(11,0)/DECIMAL_64)(children: Decimal64ColAddDecimal64Scalar(col 1:decimal(10,0)/DECIMAL_64, decimal64Val 1, decimalVal 1) -> 4:decimal(11,0)/DECIMAL_64) -> 7:decimal(11,0), ConvertDecimal64ToDecimal(col 5:decimal(11,0)/DECIMAL_64)(children: Decimal64ColAddDecimal64Scalar(col 1:decimal(10,0)/DECIMAL_64, decimal64Val 2, decimalVal 2) -> 5:decimal(11,0)/DECIMAL_64) -> 8:decimal(11,0)) -> 6:decimal(11,0)
                     Statistics: Num rows: 3 Data size: 336 Basic stats: COMPLETE Column stats: COMPLETE
                     File Output Operator
                       compressed: false
@@ -617,7 +617,7 @@ STAGE PLANS:
                 featureSupportInUse: [DECIMAL_64]
                 inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
                 allNative: false
-                usesVectorUDFAdaptor: true
+                usesVectorUDFAdaptor: false
                 vectorized: true
                 rowBatchContext:
                     dataColumnCount: 2
@@ -677,7 +677,7 @@ STAGE PLANS:
                         className: VectorSelectOperator
                         native: true
                         projectedOutputColumnNums: [8]
-                        selectExpressions: VectorUDFAdaptor(CASE WHEN ((member = 1)) THEN (1) ELSE ((attr + 2)) END)(children: VectorUDFAdaptor((member = 1)) -> 6:boolean, ConvertDecimal64ToDecimal(col 7:decimal(11,0)/DECIMAL_64)(children: Decimal64ColAddDecimal64Scalar(col 1:decimal(10,0)/DECIMAL_64, decimal64Val 2, decimalVal 2) -> 7:decimal(11,0)/DECIMAL_64) -> 9:decimal(11,0)) -> 8:decimal(11,0)
+                        selectExpressions: IfExprDecimal64ScalarDecimal64Column(col 6:boolean, decimal64Val 1, decimalVal 1, col 7:decimal(1,0)/DECIMAL_64)(children: Decimal64ColEqualDecimal64Scalar(col 0:decimal(10,0)/DECIMAL_64, decimal64Val 1, decimalVal 1) -> 6:boolean, Decimal64ColAddDecimal64Scalar(col 1:decimal(10,0)/DECIMAL_64, decimal64Val 2, decimalVal 2) -> 7:decimal(11,0)/DECIMAL_64) -> 8:decimal(11,0)/DECIMAL_64
                     Statistics: Num rows: 3 Data size: 336 Basic stats: COMPLETE Column stats: COMPLETE
                     File Output Operator
                       compressed: false
@@ -698,14 +698,14 @@ STAGE PLANS:
                 featureSupportInUse: [DECIMAL_64]
                 inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
                 allNative: false
-                usesVectorUDFAdaptor: true
+                usesVectorUDFAdaptor: false
                 vectorized: true
                 rowBatchContext:
                     dataColumnCount: 2
                     includeColumns: [0, 1]
                     dataColumns: member:decimal(10,0)/DECIMAL_64, attr:decimal(10,0)/DECIMAL_64
                     partitionColumnCount: 0
-                    scratchColumnTypeNames: [bigint, decimal(1,0), decimal(11,0)/DECIMAL_64, bigint, decimal(11,0)/DECIMAL_64, decimal(11,0), decimal(11,0)]
+                    scratchColumnTypeNames: [bigint, decimal(1,0), decimal(11,0)/DECIMAL_64, bigint, decimal(11,0)/DECIMAL_64, decimal(11,0)/DECIMAL_64]
 
   Stage: Stage-0
     Fetch Operator
@@ -758,7 +758,7 @@ STAGE PLANS:
                         className: VectorSelectOperator
                         native: true
                         projectedOutputColumnNums: [8]
-                        selectExpressions: VectorUDFAdaptor(CASE WHEN ((member = 1)) THEN ((attr + 1)) ELSE (2) END)(children: VectorUDFAdaptor((member = 1)) -> 6:boolean, ConvertDecimal64ToDecimal(col 7:decimal(11,0)/DECIMAL_64)(children: Decimal64ColAddDecimal64Scalar(col 1:decimal(10,0)/DECIMAL_64, decimal64Val 1, decimalVal 1) -> 7:decimal(11,0)/DECIMAL_64) -> 9:decimal(11,0)) -> 8:decimal(11,0)
+                        selectExpressions: IfExprDecimal64ColumnDecimal64Scalar(col 6:boolean, col 7:decimal(11,0)/DECIMAL_64, decimal64Val 2, decimalVal 2)(children: Decimal64ColEqualDecimal64Scalar(col 0:decimal(10,0)/DECIMAL_64, decimal64Val 1, decimalVal 1) -> 6:boolean, Decimal64ColAddDecimal64Scalar(col 1:decimal(10,0)/DECIMAL_64, decimal64Val 1, decimalVal 1) -> 7:decimal(11,0)/DECIMAL_64) -> 8:decimal(11,0)/DECIMAL_64
                     Statistics: Num rows: 3 Data size: 336 Basic stats: COMPLETE Column stats: COMPLETE
                     File Output Operator
                       compressed: false
@@ -779,14 +779,14 @@ STAGE PLANS:
                 featureSupportInUse: [DECIMAL_64]
                 inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
                 allNative: false
-                usesVectorUDFAdaptor: true
+                usesVectorUDFAdaptor: false
                 vectorized: true
                 rowBatchContext:
                     dataColumnCount: 2
                     includeColumns: [0, 1]
                     dataColumns: member:decimal(10,0)/DECIMAL_64, attr:decimal(10,0)/DECIMAL_64
                     partitionColumnCount: 0
-                    scratchColumnTypeNames: [bigint, decimal(11,0)/DECIMAL_64, decimal(1,0), bigint, decimal(11,0)/DECIMAL_64, decimal(11,0), decimal(11,0)]
+                    scratchColumnTypeNames: [bigint, decimal(11,0)/DECIMAL_64, decimal(1,0), bigint, decimal(11,0)/DECIMAL_64, decimal(11,0)/DECIMAL_64]
 
   Stage: Stage-0
     Fetch Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/test/results/clientpositive/llap/vectorized_casts.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/vectorized_casts.q.out b/ql/src/test/results/clientpositive/llap/vectorized_casts.q.out
index e7f774b..b835c47 100644
--- a/ql/src/test/results/clientpositive/llap/vectorized_casts.q.out
+++ b/ql/src/test/results/clientpositive/llap/vectorized_casts.q.out
@@ -184,7 +184,7 @@ STAGE PLANS:
                           className: VectorSelectOperator
                           native: true
                           projectedOutputColumnNums: [13, 14, 15, 16, 17, 18, 10, 20, 19, 21, 0, 1, 2, 3, 22, 23, 10, 24, 25, 27, 28, 29, 30, 31, 32, 33, 34, 4, 5, 35, 36, 37, 38, 39, 5, 41, 43, 45, 47, 48, 49, 51, 54, 55, 8, 56, 57, 26, 58, 59, 60, 61, 62, 63, 64, 65, 6, 67, 68, 69, 70, 66, 73]
-                          selectExpressions: CastLongToBooleanViaLongToLong(col 0:tinyint) -> 13:boolean, CastLongToBooleanViaLongToLong(col 1:smallint) -> 14:boolean, CastLongToBooleanViaLongToLong(col 2:int) -> 15:boolean, CastLongToBooleanViaLongToLong(col 3:bigint) -> 16:boolean, CastDoubleToBooleanViaDoubleToLong(col 4:float) -> 17:boolean, CastDoubleToBooleanViaDoubleToLong(col 5:double) -> 18:boolean, CastLongToBooleanViaLongToLong(col 19:bigint)(children: LongColMultiplyLongScalar(col 3:bigint, val 0) -> 19:bigint) -> 20:boolean, CastTimestampToBoolean(col 8:timestamp) -> 19:boolean, CastStringToBoolean(col 6) -> 21:boolean, CastDoubleToLong(col 4:float) -> 22:int, CastDoubleToLong(col 5:double) -> 23:int, CastTimestampToLong(col 8:timestamp) -> 24:int, CastStringToLong(col 6:string) -> 25:int, CastStringToLong(col 26:string)(children: StringSubstrColStartLen(col 6:string, start 0, length 1) -> 26:string) -> 27:int, CastDoubleToLong(col 4:float) -> 28:tinyint, CastDoubleToLo
 ng(col 4:float) -> 29:smallint, CastDoubleToLong(col 4:float) -> 30:bigint, CastLongToDouble(col 0:tinyint) -> 31:double, CastLongToDouble(col 1:smallint) -> 32:double, CastLongToDouble(col 2:int) -> 33:double, CastLongToDouble(col 3:bigint) -> 34:double, CastLongToDouble(col 10:boolean) -> 35:double, CastTimestampToDouble(col 8:timestamp) -> 36:double, CastStringToDouble(col 6:string) -> 37:double, CastStringToDouble(col 26:string)(children: StringSubstrColStartLen(col 6:string, start 0, length 1) -> 26:string) -> 38:double, CastLongToFloatViaLongToDouble(col 2:int) -> 39:float, CastMillisecondsLongToTimestamp(col 0:tinyint) -> 41:timestamp, CastMillisecondsLongToTimestamp(col 1:smallint) -> 43:timestamp, CastMillisecondsLongToTimestamp(col 2:int) -> 45:timestamp, CastMillisecondsLongToTimestamp(col 3:bigint) -> 47:timestamp, CastDoubleToTimestamp(col 4:float) -> 48:timestamp, CastDoubleToTimestamp(col 5:double) -> 49:timestamp, CastMillisecondsLongToTimestamp(col 10:boolean) -> 51
 :timestamp, CastMillisecondsLongToTimestamp(col 52:bigint)(children: LongColMultiplyLongScalar(col 3:bigint, val 0) -> 52:bigint) -> 54:timestamp, CastDateToTimestamp(col 52:date)(children: CastTimestampToDate(col 8:timestamp) -> 52:date) -> 55:timestamp, VectorUDFAdaptor(CAST( cstring1 AS TIMESTAMP)) -> 56:timestamp, VectorUDFAdaptor(CAST( substr(cstring1, 1, 1) AS TIMESTAMP))(children: StringSubstrColStartLen(col 6:string, start 0, length 1) -> 26:string) -> 57:timestamp, CastLongToString(col 0:tinyint) -> 26:string, CastLongToString(col 1:smallint) -> 58:string, CastLongToString(col 2:int) -> 59:string, CastLongToString(col 3:bigint) -> 60:string, CastFloatToString(col 4:float) -> 61:string, CastDoubleToString(col 5:double) -> 62:string, CastBooleanToStringViaLongToString(col 10:boolean) -> 63:string, CastLongToString(col 52:bigint)(children: LongColMultiplyLongScalar(col 3:bigint, val 0) -> 52:bigint) -> 64:string, CastTimestampToString(col 8:timestamp) -> 65:string, CastStringG
 roupToString(col 66:char(10))(children: CastStringGroupToChar(col 6:string, maxLength 10) -> 66:char(10)) -> 67:string, CastStringGroupToString(col 66:varchar(10))(children: CastStringGroupToVarChar(col 6:string, maxLength 10) -> 66:varchar(10)) -> 68:string, CastLongToFloatViaLongToDouble(col 52:int)(children: CastDoubleToLong(col 4:float) -> 52:int) -> 69:float, CastLongToDouble(col 52:int)(children: LongColMultiplyLongScalar(col 2:int, val 2) -> 52:int) -> 70:double, CastDoubleToString(col 71:double)(children: FuncSinDoubleToDouble(col 4:float) -> 71:double) -> 66:string, DoubleColAddDoubleColumn(col 71:double, col 72:double)(children: col 71:float, CastLongToDouble(col 10:boolean) -> 72:double) -> 73:double
+                          selectExpressions: CastLongToBooleanViaLongToLong(col 0:tinyint) -> 13:boolean, CastLongToBooleanViaLongToLong(col 1:smallint) -> 14:boolean, CastLongToBooleanViaLongToLong(col 2:int) -> 15:boolean, CastLongToBooleanViaLongToLong(col 3:bigint) -> 16:boolean, CastDoubleToBooleanViaDoubleToLong(col 4:float) -> 17:boolean, CastDoubleToBooleanViaDoubleToLong(col 5:double) -> 18:boolean, CastLongToBooleanViaLongToLong(col 19:bigint)(children: LongColMultiplyLongScalar(col 3:bigint, val 0) -> 19:bigint) -> 20:boolean, CastTimestampToBoolean(col 8:timestamp) -> 19:boolean, CastStringToBoolean(col 6) -> 21:boolean, CastDoubleToLong(col 4:float) -> 22:int, CastDoubleToLong(col 5:double) -> 23:int, CastTimestampToLong(col 8:timestamp) -> 24:int, CastStringToLong(col 6:string) -> 25:int, CastStringToLong(col 26:string)(children: StringSubstrColStartLen(col 6:string, start 0, length 1) -> 26:string) -> 27:int, CastDoubleToLong(col 4:float) -> 28:tinyint, CastDoubleToLo
 ng(col 4:float) -> 29:smallint, CastDoubleToLong(col 4:float) -> 30:bigint, CastLongToDouble(col 0:tinyint) -> 31:double, CastLongToDouble(col 1:smallint) -> 32:double, CastLongToDouble(col 2:int) -> 33:double, CastLongToDouble(col 3:bigint) -> 34:double, CastLongToDouble(col 10:boolean) -> 35:double, CastTimestampToDouble(col 8:timestamp) -> 36:double, CastStringToDouble(col 6:string) -> 37:double, CastStringToDouble(col 26:string)(children: StringSubstrColStartLen(col 6:string, start 0, length 1) -> 26:string) -> 38:double, CastLongToFloatViaLongToDouble(col 2:int) -> 39:float, CastMillisecondsLongToTimestamp(col 0:tinyint) -> 41:timestamp, CastMillisecondsLongToTimestamp(col 1:smallint) -> 43:timestamp, CastMillisecondsLongToTimestamp(col 2:int) -> 45:timestamp, CastMillisecondsLongToTimestamp(col 3:bigint) -> 47:timestamp, CastDoubleToTimestamp(col 4:float) -> 48:timestamp, CastDoubleToTimestamp(col 5:double) -> 49:timestamp, CastMillisecondsLongToTimestamp(col 10:boolean) -> 51
 :timestamp, CastMillisecondsLongToTimestamp(col 52:bigint)(children: LongColMultiplyLongScalar(col 3:bigint, val 0) -> 52:bigint) -> 54:timestamp, CastDateToTimestamp(col 52:date)(children: CastTimestampToDate(col 8:timestamp) -> 52:date) -> 55:timestamp, CastStringToTimestamp(col 6:string) -> 56:timestamp, CastStringToTimestamp(col 26:string)(children: StringSubstrColStartLen(col 6:string, start 0, length 1) -> 26:string) -> 57:timestamp, CastLongToString(col 0:tinyint) -> 26:string, CastLongToString(col 1:smallint) -> 58:string, CastLongToString(col 2:int) -> 59:string, CastLongToString(col 3:bigint) -> 60:string, CastFloatToString(col 4:float) -> 61:string, CastDoubleToString(col 5:double) -> 62:string, CastBooleanToStringViaLongToString(col 10:boolean) -> 63:string, CastLongToString(col 52:bigint)(children: LongColMultiplyLongScalar(col 3:bigint, val 0) -> 52:bigint) -> 64:string, CastTimestampToString(col 8:timestamp) -> 65:string, CastStringGroupToString(col 66:char(10))(child
 ren: CastStringGroupToChar(col 6:string, maxLength 10) -> 66:char(10)) -> 67:string, CastStringGroupToString(col 66:varchar(10))(children: CastStringGroupToVarChar(col 6:string, maxLength 10) -> 66:varchar(10)) -> 68:string, CastLongToFloatViaLongToDouble(col 52:int)(children: CastDoubleToLong(col 4:float) -> 52:int) -> 69:float, CastLongToDouble(col 52:int)(children: LongColMultiplyLongScalar(col 2:int, val 2) -> 52:int) -> 70:double, CastDoubleToString(col 71:double)(children: FuncSinDoubleToDouble(col 4:float) -> 71:double) -> 66:string, DoubleColAddDoubleColumn(col 71:double, col 72:double)(children: col 71:float, CastLongToDouble(col 10:boolean) -> 72:double) -> 73:double
                       Statistics: Num rows: 6144 Data size: 16362860 Basic stats: COMPLETE Column stats: COMPLETE
                       File Output Operator
                         compressed: false
@@ -205,7 +205,7 @@ STAGE PLANS:
                 featureSupportInUse: [DECIMAL_64]
                 inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
                 allNative: false
-                usesVectorUDFAdaptor: true
+                usesVectorUDFAdaptor: false
                 vectorized: true
                 rowBatchContext:
                     dataColumnCount: 12

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/test/results/clientpositive/llap/vectorized_dynamic_semijoin_reduction2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/vectorized_dynamic_semijoin_reduction2.q.out b/ql/src/test/results/clientpositive/llap/vectorized_dynamic_semijoin_reduction2.q.out
index c03b298..e416954 100644
--- a/ql/src/test/results/clientpositive/llap/vectorized_dynamic_semijoin_reduction2.q.out
+++ b/ql/src/test/results/clientpositive/llap/vectorized_dynamic_semijoin_reduction2.q.out
@@ -262,7 +262,7 @@ STAGE PLANS:
                     Filter Vectorization:
                         className: VectorFilterOperator
                         native: true
-                        predicateExpression: FilterExprAndExpr(children: SelectColumnIsNotNull(col 9:decimal(10,1))(children: ConvertDecimal64ToDecimal(col 1:decimal(10,1)/DECIMAL_64) -> 9:decimal(10,1)), FilterExprAndExpr(children: FilterDecimalColumnBetweenDynamicValue(col 9:decimal(10,1), left 0, right 0)(children: ConvertDecimal64ToDecimal(col 1:decimal(10,1)/DECIMAL_64) -> 9:decimal(10,1)), VectorInBloomFilterColDynamicValue(children: ConvertDecimal64ToDecimal(col 1:decimal(10,1)/DECIMAL_64) -> 9:decimal(10,1))))
+                        predicateExpression: FilterExprAndExpr(children: SelectColumnIsNotNull(col 1:decimal(10,1)/DECIMAL_64), FilterExprAndExpr(children: FilterDecimalColumnBetweenDynamicValue(col 9:decimal(10,1), left 0, right 0)(children: ConvertDecimal64ToDecimal(col 1:decimal(10,1)/DECIMAL_64) -> 9:decimal(10,1)), VectorInBloomFilterColDynamicValue(children: ConvertDecimal64ToDecimal(col 1:decimal(10,1)/DECIMAL_64) -> 9:decimal(10,1))))
                     predicate: ((partkey_decimal BETWEEN DynamicValue(RS_7_b_partkey_decimal_min) AND DynamicValue(RS_7_b_partkey_decimal_max) and in_bloom_filter(partkey_decimal, DynamicValue(RS_7_b_partkey_decimal_bloom_filter))) and partkey_decimal is not null) (type: boolean)
                     Statistics: Num rows: 100 Data size: 11200 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
@@ -314,7 +314,7 @@ STAGE PLANS:
                     Filter Vectorization:
                         className: VectorFilterOperator
                         native: true
-                        predicateExpression: SelectColumnIsNotNull(col 9:decimal(10,1))(children: ConvertDecimal64ToDecimal(col 1:decimal(10,1)/DECIMAL_64) -> 9:decimal(10,1))
+                        predicateExpression: SelectColumnIsNotNull(col 1:decimal(10,1)/DECIMAL_64)
                     predicate: partkey_decimal is not null (type: boolean)
                     Statistics: Num rows: 20 Data size: 2240 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/test/results/clientpositive/llap/vectorized_mapjoin3.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/vectorized_mapjoin3.q.out b/ql/src/test/results/clientpositive/llap/vectorized_mapjoin3.q.out
index e9aea17..23120a5 100644
--- a/ql/src/test/results/clientpositive/llap/vectorized_mapjoin3.q.out
+++ b/ql/src/test/results/clientpositive/llap/vectorized_mapjoin3.q.out
@@ -128,7 +128,7 @@ STAGE PLANS:
                     Filter Vectorization:
                         className: VectorFilterOperator
                         native: true
-                        predicateExpression: FilterExprAndExpr(children: SelectColumnIsNotNull(col 3:decimal(8,1))(children: ConvertDecimal64ToDecimal(col 0:decimal(8,1)/DECIMAL_64) -> 3:decimal(8,1)), SelectColumnIsNotNull(col 1:int))
+                        predicateExpression: FilterExprAndExpr(children: SelectColumnIsNotNull(col 0:decimal(8,1)/DECIMAL_64), SelectColumnIsNotNull(col 1:int))
                     predicate: (decimal0801_col is not null and int_col_1 is not null) (type: boolean)
                     Statistics: Num rows: 4 Data size: 464 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
@@ -333,7 +333,7 @@ STAGE PLANS:
                     Filter Vectorization:
                         className: VectorFilterOperator
                         native: true
-                        predicateExpression: FilterExprAndExpr(children: SelectColumnIsNotNull(col 1:int), SelectColumnIsNotNull(col 3:decimal(8,1))(children: ConvertDecimal64ToDecimal(col 0:decimal(8,1)/DECIMAL_64) -> 3:decimal(8,1)))
+                        predicateExpression: FilterExprAndExpr(children: SelectColumnIsNotNull(col 1:int), SelectColumnIsNotNull(col 0:decimal(8,1)/DECIMAL_64))
                     predicate: (decimal0801_col is not null and int_col_1 is not null) (type: boolean)
                     Statistics: Num rows: 4 Data size: 464 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
@@ -538,7 +538,7 @@ STAGE PLANS:
                     Filter Vectorization:
                         className: VectorFilterOperator
                         native: true
-                        predicateExpression: FilterExprAndExpr(children: SelectColumnIsNotNull(col 3:decimal(8,1))(children: ConvertDecimal64ToDecimal(col 0:decimal(8,1)/DECIMAL_64) -> 3:decimal(8,1)), SelectColumnIsNotNull(col 1:int))
+                        predicateExpression: FilterExprAndExpr(children: SelectColumnIsNotNull(col 0:decimal(8,1)/DECIMAL_64), SelectColumnIsNotNull(col 1:int))
                     predicate: (decimal0801_col is not null and int_col_1 is not null) (type: boolean)
                     Statistics: Num rows: 4 Data size: 464 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/test/results/clientpositive/llap/vectorized_timestamp_funcs.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/vectorized_timestamp_funcs.q.out b/ql/src/test/results/clientpositive/llap/vectorized_timestamp_funcs.q.out
index cc7699b..4edd0e4 100644
--- a/ql/src/test/results/clientpositive/llap/vectorized_timestamp_funcs.q.out
+++ b/ql/src/test/results/clientpositive/llap/vectorized_timestamp_funcs.q.out
@@ -479,7 +479,7 @@ STAGE PLANS:
                         className: VectorSelectOperator
                         native: true
                         projectedOutputColumnNums: [5, 7, 8, 9, 6, 11, 10, 13, 14]
-                        selectExpressions: VectorUDFUnixTimeStampString(col 2:string) -> 5:bigint, VectorUDFYearDate(col 6, field YEAR)(children: CastStringToDate(col 2:string) -> 6:date) -> 7:int, VectorUDFMonthDate(col 6, field MONTH)(children: CastStringToDate(col 2:string) -> 6:date) -> 8:int, VectorUDFDayOfMonthDate(col 6, field DAY_OF_MONTH)(children: CastStringToDate(col 2:string) -> 6:date) -> 9:int, VectorUDFDayOfMonthString(col 2:string, fieldStart 8, fieldLength 2) -> 6:int, VectorUDFWeekOfYearDate(col 10, field WEEK_OF_YEAR)(children: CastStringToDate(col 2:string) -> 10:date) -> 11:int, VectorUDFHourTimestamp(col 12:timestamp, field HOUR_OF_DAY)(children: VectorUDFAdaptor(CAST( stimestamp1 AS TIMESTAMP)) -> 12:timestamp) -> 10:int, VectorUDFMinuteTimestamp(col 12:timestamp, field MINUTE)(children: VectorUDFAdaptor(CAST( stimestamp1 AS TIMESTAMP)) -> 12:timestamp) -> 13:int, VectorUDFSecondTimestamp(col 12:timestamp, field SECOND)(children: VectorUDFAdaptor(CAST( stimest
 amp1 AS TIMESTAMP)) -> 12:timestamp) -> 14:int
+                        selectExpressions: VectorUDFUnixTimeStampString(col 2:string) -> 5:bigint, VectorUDFYearDate(col 6, field YEAR)(children: CastStringToDate(col 2:string) -> 6:date) -> 7:int, VectorUDFMonthDate(col 6, field MONTH)(children: CastStringToDate(col 2:string) -> 6:date) -> 8:int, VectorUDFDayOfMonthDate(col 6, field DAY_OF_MONTH)(children: CastStringToDate(col 2:string) -> 6:date) -> 9:int, VectorUDFDayOfMonthString(col 2:string, fieldStart 8, fieldLength 2) -> 6:int, VectorUDFWeekOfYearDate(col 10, field WEEK_OF_YEAR)(children: CastStringToDate(col 2:string) -> 10:date) -> 11:int, VectorUDFHourTimestamp(col 12:timestamp, field HOUR_OF_DAY)(children: CastStringToTimestamp(col 2:string) -> 12:timestamp) -> 10:int, VectorUDFMinuteTimestamp(col 12:timestamp, field MINUTE)(children: CastStringToTimestamp(col 2:string) -> 12:timestamp) -> 13:int, VectorUDFSecondTimestamp(col 12:timestamp, field SECOND)(children: CastStringToTimestamp(col 2:string) -> 12:timestamp) -> 14
 :int
                     Statistics: Num rows: 52 Data size: 2080 Basic stats: COMPLETE Column stats: COMPLETE
                     Reduce Output Operator
                       key expressions: _col0 (type: bigint)
@@ -499,7 +499,7 @@ STAGE PLANS:
                 featureSupportInUse: [DECIMAL_64]
                 inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
                 allNative: true
-                usesVectorUDFAdaptor: true
+                usesVectorUDFAdaptor: false
                 vectorized: true
         Reducer 2 
             Execution mode: vectorized, llap
@@ -673,7 +673,7 @@ STAGE PLANS:
                         className: VectorSelectOperator
                         native: true
                         projectedOutputColumnNums: [7, 6, 8, 9, 11, 10, 14, 15, 16]
-                        selectExpressions: LongColEqualLongColumn(col 5:bigint, col 6:bigint)(children: VectorUDFUnixTimeStampTimestamp(col 1:timestamp) -> 5:bigint, VectorUDFUnixTimeStampString(col 2:string) -> 6:bigint) -> 7:boolean, LongColEqualLongColumn(col 5:int, col 8:int)(children: VectorUDFYearTimestamp(col 1:timestamp, field YEAR) -> 5:int, VectorUDFYearDate(col 6, field YEAR)(children: CastStringToDate(col 2:string) -> 6:date) -> 8:int) -> 6:boolean, LongColEqualLongColumn(col 5:int, col 9:int)(children: VectorUDFMonthTimestamp(col 1:timestamp, field MONTH) -> 5:int, VectorUDFMonthDate(col 8, field MONTH)(children: CastStringToDate(col 2:string) -> 8:date) -> 9:int) -> 8:boolean, LongColEqualLongColumn(col 5:int, col 10:int)(children: VectorUDFDayOfMonthTimestamp(col 1:timestamp, field DAY_OF_MONTH) -> 5:int, VectorUDFDayOfMonthDate(col 9, field DAY_OF_MONTH)(children: CastStringToDate(col 2:string) -> 9:date) -> 10:int) -> 9:boolean, LongColEqualLongColumn(col 5:int, col
  10:int)(children: VectorUDFDayOfMonthTimestamp(col 1:timestamp, field DAY_OF_MONTH) -> 5:int, VectorUDFDayOfMonthString(col 2:string, fieldStart 8, fieldLength 2) -> 10:int) -> 11:boolean, LongColEqualLongColumn(col 5:int, col 12:int)(children: VectorUDFWeekOfYearTimestamp(col 1:timestamp, field WEEK_OF_YEAR) -> 5:int, VectorUDFWeekOfYearDate(col 10, field WEEK_OF_YEAR)(children: CastStringToDate(col 2:string) -> 10:date) -> 12:int) -> 10:boolean, LongColEqualLongColumn(col 5:int, col 12:int)(children: VectorUDFHourTimestamp(col 1:timestamp, field HOUR_OF_DAY) -> 5:int, VectorUDFHourTimestamp(col 13:timestamp, field HOUR_OF_DAY)(children: VectorUDFAdaptor(CAST( stimestamp1 AS TIMESTAMP)) -> 13:timestamp) -> 12:int) -> 14:boolean, LongColEqualLongColumn(col 5:int, col 12:int)(children: VectorUDFMinuteTimestamp(col 1:timestamp, field MINUTE) -> 5:int, VectorUDFMinuteTimestamp(col 13:timestamp, field MINUTE)(children: VectorUDFAdaptor(CAST( stimestamp1 AS TIMESTAMP)) -> 13:timestamp) 
 -> 12:int) -> 15:boolean, LongColEqualLongColumn(col 5:int, col 12:int)(children: VectorUDFSecondTimestamp(col 1:timestamp, field SECOND) -> 5:int, VectorUDFSecondTimestamp(col 13:timestamp, field SECOND)(children: VectorUDFAdaptor(CAST( stimestamp1 AS TIMESTAMP)) -> 13:timestamp) -> 12:int) -> 16:boolean
+                        selectExpressions: LongColEqualLongColumn(col 5:bigint, col 6:bigint)(children: VectorUDFUnixTimeStampTimestamp(col 1:timestamp) -> 5:bigint, VectorUDFUnixTimeStampString(col 2:string) -> 6:bigint) -> 7:boolean, LongColEqualLongColumn(col 5:int, col 8:int)(children: VectorUDFYearTimestamp(col 1:timestamp, field YEAR) -> 5:int, VectorUDFYearDate(col 6, field YEAR)(children: CastStringToDate(col 2:string) -> 6:date) -> 8:int) -> 6:boolean, LongColEqualLongColumn(col 5:int, col 9:int)(children: VectorUDFMonthTimestamp(col 1:timestamp, field MONTH) -> 5:int, VectorUDFMonthDate(col 8, field MONTH)(children: CastStringToDate(col 2:string) -> 8:date) -> 9:int) -> 8:boolean, LongColEqualLongColumn(col 5:int, col 10:int)(children: VectorUDFDayOfMonthTimestamp(col 1:timestamp, field DAY_OF_MONTH) -> 5:int, VectorUDFDayOfMonthDate(col 9, field DAY_OF_MONTH)(children: CastStringToDate(col 2:string) -> 9:date) -> 10:int) -> 9:boolean, LongColEqualLongColumn(col 5:int, col
  10:int)(children: VectorUDFDayOfMonthTimestamp(col 1:timestamp, field DAY_OF_MONTH) -> 5:int, VectorUDFDayOfMonthString(col 2:string, fieldStart 8, fieldLength 2) -> 10:int) -> 11:boolean, LongColEqualLongColumn(col 5:int, col 12:int)(children: VectorUDFWeekOfYearTimestamp(col 1:timestamp, field WEEK_OF_YEAR) -> 5:int, VectorUDFWeekOfYearDate(col 10, field WEEK_OF_YEAR)(children: CastStringToDate(col 2:string) -> 10:date) -> 12:int) -> 10:boolean, LongColEqualLongColumn(col 5:int, col 12:int)(children: VectorUDFHourTimestamp(col 1:timestamp, field HOUR_OF_DAY) -> 5:int, VectorUDFHourTimestamp(col 13:timestamp, field HOUR_OF_DAY)(children: CastStringToTimestamp(col 2:string) -> 13:timestamp) -> 12:int) -> 14:boolean, LongColEqualLongColumn(col 5:int, col 12:int)(children: VectorUDFMinuteTimestamp(col 1:timestamp, field MINUTE) -> 5:int, VectorUDFMinuteTimestamp(col 13:timestamp, field MINUTE)(children: CastStringToTimestamp(col 2:string) -> 13:timestamp) -> 12:int) -> 15:boolean, Lo
 ngColEqualLongColumn(col 5:int, col 12:int)(children: VectorUDFSecondTimestamp(col 1:timestamp, field SECOND) -> 5:int, VectorUDFSecondTimestamp(col 13:timestamp, field SECOND)(children: CastStringToTimestamp(col 2:string) -> 13:timestamp) -> 12:int) -> 16:boolean
                     Statistics: Num rows: 52 Data size: 1872 Basic stats: COMPLETE Column stats: COMPLETE
                     Reduce Output Operator
                       key expressions: _col0 (type: boolean)
@@ -693,7 +693,7 @@ STAGE PLANS:
                 featureSupportInUse: [DECIMAL_64]
                 inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
                 allNative: true
-                usesVectorUDFAdaptor: true
+                usesVectorUDFAdaptor: false
                 vectorized: true
         Reducer 2 
             Execution mode: vectorized, llap
@@ -867,7 +867,7 @@ STAGE PLANS:
                         className: VectorSelectOperator
                         native: true
                         projectedOutputColumnNums: [2, 4, 5, 6, 3, 8, 7, 10, 11]
-                        selectExpressions: VectorUDFUnixTimeStampString(col 0:string) -> 2:bigint, VectorUDFYearDate(col 3, field YEAR)(children: CastStringToDate(col 0:string) -> 3:date) -> 4:int, VectorUDFMonthDate(col 3, field MONTH)(children: CastStringToDate(col 0:string) -> 3:date) -> 5:int, VectorUDFDayOfMonthDate(col 3, field DAY_OF_MONTH)(children: CastStringToDate(col 0:string) -> 3:date) -> 6:int, VectorUDFDayOfMonthString(col 0:string, fieldStart 8, fieldLength 2) -> 3:int, VectorUDFWeekOfYearDate(col 7, field WEEK_OF_YEAR)(children: CastStringToDate(col 0:string) -> 7:date) -> 8:int, VectorUDFHourTimestamp(col 9:timestamp, field HOUR_OF_DAY)(children: VectorUDFAdaptor(CAST( stimestamp1 AS TIMESTAMP)) -> 9:timestamp) -> 7:int, VectorUDFMinuteTimestamp(col 9:timestamp, field MINUTE)(children: VectorUDFAdaptor(CAST( stimestamp1 AS TIMESTAMP)) -> 9:timestamp) -> 10:int, VectorUDFSecondTimestamp(col 9:timestamp, field SECOND)(children: VectorUDFAdaptor(CAST( stimestamp1 AS T
 IMESTAMP)) -> 9:timestamp) -> 11:int
+                        selectExpressions: VectorUDFUnixTimeStampString(col 0:string) -> 2:bigint, VectorUDFYearDate(col 3, field YEAR)(children: CastStringToDate(col 0:string) -> 3:date) -> 4:int, VectorUDFMonthDate(col 3, field MONTH)(children: CastStringToDate(col 0:string) -> 3:date) -> 5:int, VectorUDFDayOfMonthDate(col 3, field DAY_OF_MONTH)(children: CastStringToDate(col 0:string) -> 3:date) -> 6:int, VectorUDFDayOfMonthString(col 0:string, fieldStart 8, fieldLength 2) -> 3:int, VectorUDFWeekOfYearDate(col 7, field WEEK_OF_YEAR)(children: CastStringToDate(col 0:string) -> 7:date) -> 8:int, VectorUDFHourTimestamp(col 9:timestamp, field HOUR_OF_DAY)(children: CastStringToTimestamp(col 0:string) -> 9:timestamp) -> 7:int, VectorUDFMinuteTimestamp(col 9:timestamp, field MINUTE)(children: CastStringToTimestamp(col 0:string) -> 9:timestamp) -> 10:int, VectorUDFSecondTimestamp(col 9:timestamp, field SECOND)(children: CastStringToTimestamp(col 0:string) -> 9:timestamp) -> 11:int
                     Statistics: Num rows: 3 Data size: 120 Basic stats: COMPLETE Column stats: COMPLETE
                     Reduce Output Operator
                       key expressions: _col0 (type: bigint)
@@ -887,7 +887,7 @@ STAGE PLANS:
                 featureSupportInUse: [DECIMAL_64]
                 inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
                 allNative: true
-                usesVectorUDFAdaptor: true
+                usesVectorUDFAdaptor: false
                 vectorized: true
         Reducer 2 
             Execution mode: vectorized, llap

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/test/results/clientpositive/llap/vectorized_timestamp_ints_casts.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/vectorized_timestamp_ints_casts.q.out b/ql/src/test/results/clientpositive/llap/vectorized_timestamp_ints_casts.q.out
index bbf07a8..bdf2600 100644
--- a/ql/src/test/results/clientpositive/llap/vectorized_timestamp_ints_casts.q.out
+++ b/ql/src/test/results/clientpositive/llap/vectorized_timestamp_ints_casts.q.out
@@ -69,7 +69,7 @@ STAGE PLANS:
                           className: VectorSelectOperator
                           native: true
                           projectedOutputColumnNums: [15, 17, 19, 21, 22, 23, 25, 27, 8, 28, 30]
-                          selectExpressions: CastMillisecondsLongToTimestamp(col 0:tinyint) -> 15:timestamp, CastMillisecondsLongToTimestamp(col 1:smallint) -> 17:timestamp, CastMillisecondsLongToTimestamp(col 2:int) -> 19:timestamp, CastMillisecondsLongToTimestamp(col 3:bigint) -> 21:timestamp, CastDoubleToTimestamp(col 4:float) -> 22:timestamp, CastDoubleToTimestamp(col 5:double) -> 23:timestamp, CastMillisecondsLongToTimestamp(col 10:boolean) -> 25:timestamp, CastMillisecondsLongToTimestamp(col 13:bigint)(children: LongColMultiplyLongScalar(col 3:bigint, val 0) -> 13:bigint) -> 27:timestamp, VectorUDFAdaptor(CAST( cstring1 AS TIMESTAMP)) -> 28:timestamp, VectorUDFAdaptor(CAST( substr(cstring1, 1, 1) AS TIMESTAMP))(children: StringSubstrColStartLen(col 6:string, start 0, length 1) -> 29:string) -> 30:timestamp
+                          selectExpressions: CastMillisecondsLongToTimestamp(col 0:tinyint) -> 15:timestamp, CastMillisecondsLongToTimestamp(col 1:smallint) -> 17:timestamp, CastMillisecondsLongToTimestamp(col 2:int) -> 19:timestamp, CastMillisecondsLongToTimestamp(col 3:bigint) -> 21:timestamp, CastDoubleToTimestamp(col 4:float) -> 22:timestamp, CastDoubleToTimestamp(col 5:double) -> 23:timestamp, CastMillisecondsLongToTimestamp(col 10:boolean) -> 25:timestamp, CastMillisecondsLongToTimestamp(col 13:bigint)(children: LongColMultiplyLongScalar(col 3:bigint, val 0) -> 13:bigint) -> 27:timestamp, CastStringToTimestamp(col 6:string) -> 28:timestamp, CastStringToTimestamp(col 29:string)(children: StringSubstrColStartLen(col 6:string, start 0, length 1) -> 29:string) -> 30:timestamp
                       Statistics: Num rows: 6144 Data size: 2703360 Basic stats: COMPLETE Column stats: COMPLETE
                       File Output Operator
                         compressed: false
@@ -90,7 +90,7 @@ STAGE PLANS:
                 featureSupportInUse: [DECIMAL_64]
                 inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
                 allNative: false
-                usesVectorUDFAdaptor: true
+                usesVectorUDFAdaptor: false
                 vectorized: true
 
   Stage: Stage-0
@@ -234,7 +234,7 @@ STAGE PLANS:
                           className: VectorSelectOperator
                           native: true
                           projectedOutputColumnNums: [14, 15, 16, 17, 18, 19, 20, 21, 8, 22, 24]
-                          selectExpressions: CastLongToTimestamp(col 0:tinyint) -> 14:timestamp, CastLongToTimestamp(col 1:smallint) -> 15:timestamp, CastLongToTimestamp(col 2:int) -> 16:timestamp, CastLongToTimestamp(col 3:bigint) -> 17:timestamp, CastDoubleToTimestamp(col 4:float) -> 18:timestamp, CastDoubleToTimestamp(col 5:double) -> 19:timestamp, CastLongToTimestamp(col 10:boolean) -> 20:timestamp, CastLongToTimestamp(col 13:bigint)(children: LongColMultiplyLongScalar(col 3:bigint, val 0) -> 13:bigint) -> 21:timestamp, VectorUDFAdaptor(CAST( cstring1 AS TIMESTAMP)) -> 22:timestamp, VectorUDFAdaptor(CAST( substr(cstring1, 1, 1) AS TIMESTAMP))(children: StringSubstrColStartLen(col 6:string, start 0, length 1) -> 23:string) -> 24:timestamp
+                          selectExpressions: CastLongToTimestamp(col 0:tinyint) -> 14:timestamp, CastLongToTimestamp(col 1:smallint) -> 15:timestamp, CastLongToTimestamp(col 2:int) -> 16:timestamp, CastLongToTimestamp(col 3:bigint) -> 17:timestamp, CastDoubleToTimestamp(col 4:float) -> 18:timestamp, CastDoubleToTimestamp(col 5:double) -> 19:timestamp, CastLongToTimestamp(col 10:boolean) -> 20:timestamp, CastLongToTimestamp(col 13:bigint)(children: LongColMultiplyLongScalar(col 3:bigint, val 0) -> 13:bigint) -> 21:timestamp, CastStringToTimestamp(col 6:string) -> 22:timestamp, CastStringToTimestamp(col 23:string)(children: StringSubstrColStartLen(col 6:string, start 0, length 1) -> 23:string) -> 24:timestamp
                       Statistics: Num rows: 6144 Data size: 2703360 Basic stats: COMPLETE Column stats: COMPLETE
                       File Output Operator
                         compressed: false
@@ -255,7 +255,7 @@ STAGE PLANS:
                 featureSupportInUse: [DECIMAL_64]
                 inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
                 allNative: false
-                usesVectorUDFAdaptor: true
+                usesVectorUDFAdaptor: false
                 vectorized: true
 
   Stage: Stage-0

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/test/results/clientpositive/spark/vector_decimal_mapjoin.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/vector_decimal_mapjoin.q.out b/ql/src/test/results/clientpositive/spark/vector_decimal_mapjoin.q.out
index f711413..c15b3b9 100644
--- a/ql/src/test/results/clientpositive/spark/vector_decimal_mapjoin.q.out
+++ b/ql/src/test/results/clientpositive/spark/vector_decimal_mapjoin.q.out
@@ -739,7 +739,7 @@ STAGE PLANS:
                     Filter Vectorization:
                         className: VectorFilterOperator
                         native: true
-                        predicateExpression: SelectColumnIsNotNull(col 3:decimal(14,0))(children: ConvertDecimal64ToDecimal(col 0:decimal(14,0)/DECIMAL_64) -> 3:decimal(14,0))
+                        predicateExpression: SelectColumnIsNotNull(col 0:decimal(14,0)/DECIMAL_64)
                     predicate: dec is not null (type: boolean)
                     Statistics: Num rows: 1049 Data size: 4966 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
@@ -772,7 +772,7 @@ STAGE PLANS:
                     includeColumns: [0]
                     dataColumns: dec:decimal(14,0)/DECIMAL_64, value_dec:decimal(14,0)/DECIMAL_64
                     partitionColumnCount: 0
-                    scratchColumnTypeNames: [decimal(14,0)]
+                    scratchColumnTypeNames: []
             Local Work:
               Map Reduce Local Work
 
@@ -793,7 +793,7 @@ STAGE PLANS:
                     Filter Vectorization:
                         className: VectorFilterOperator
                         native: true
-                        predicateExpression: SelectColumnIsNotNull(col 3:decimal(14,2))(children: ConvertDecimal64ToDecimal(col 0:decimal(14,2)/DECIMAL_64) -> 3:decimal(14,2))
+                        predicateExpression: SelectColumnIsNotNull(col 0:decimal(14,2)/DECIMAL_64)
                     predicate: dec is not null (type: boolean)
                     Statistics: Num rows: 1049 Data size: 11234 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
@@ -811,8 +811,8 @@ STAGE PLANS:
                           0 _col0 (type: decimal(16,2))
                           1 _col0 (type: decimal(16,2))
                         Map Join Vectorization:
-                            bigTableKeyExpressions: ConvertDecimal64ToDecimal(col 0:decimal(16,2)/DECIMAL_64) -> 4:decimal(16,2)
-                            bigTableValueExpressions: ConvertDecimal64ToDecimal(col 0:decimal(16,2)/DECIMAL_64) -> 5:decimal(16,2)
+                            bigTableKeyExpressions: ConvertDecimal64ToDecimal(col 0:decimal(16,2)/DECIMAL_64) -> 3:decimal(16,2)
+                            bigTableValueExpressions: ConvertDecimal64ToDecimal(col 0:decimal(16,2)/DECIMAL_64) -> 4:decimal(16,2)
                             className: VectorMapJoinOperator
                             native: false
                             nativeConditionsMet: hive.mapjoin.optimized.hashtable IS true, hive.vectorized.execution.mapjoin.native.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, One MapJoin Condition IS true, No nullsafe IS true, Small table vectorizes IS true
@@ -847,7 +847,7 @@ STAGE PLANS:
                     includeColumns: [0]
                     dataColumns: dec:decimal(14,2)/DECIMAL_64, value_dec:decimal(14,2)/DECIMAL_64
                     partitionColumnCount: 0
-                    scratchColumnTypeNames: [decimal(14,2), decimal(16,2), decimal(16,2)]
+                    scratchColumnTypeNames: [decimal(16,2), decimal(16,2)]
             Local Work:
               Map Reduce Local Work
 
@@ -1017,7 +1017,7 @@ STAGE PLANS:
                     Filter Vectorization:
                         className: VectorFilterOperator
                         native: true
-                        predicateExpression: SelectColumnIsNotNull(col 3:decimal(14,0))(children: ConvertDecimal64ToDecimal(col 0:decimal(14,0)/DECIMAL_64) -> 3:decimal(14,0))
+                        predicateExpression: SelectColumnIsNotNull(col 0:decimal(14,0)/DECIMAL_64)
                     predicate: dec is not null (type: boolean)
                     Statistics: Num rows: 1049 Data size: 4966 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
@@ -1050,7 +1050,7 @@ STAGE PLANS:
                     includeColumns: [0, 1]
                     dataColumns: dec:decimal(14,0)/DECIMAL_64, value_dec:decimal(14,0)/DECIMAL_64
                     partitionColumnCount: 0
-                    scratchColumnTypeNames: [decimal(14,0)]
+                    scratchColumnTypeNames: []
             Local Work:
               Map Reduce Local Work
 
@@ -1071,7 +1071,7 @@ STAGE PLANS:
                     Filter Vectorization:
                         className: VectorFilterOperator
                         native: true
-                        predicateExpression: SelectColumnIsNotNull(col 3:decimal(14,2))(children: ConvertDecimal64ToDecimal(col 0:decimal(14,2)/DECIMAL_64) -> 3:decimal(14,2))
+                        predicateExpression: SelectColumnIsNotNull(col 0:decimal(14,2)/DECIMAL_64)
                     predicate: dec is not null (type: boolean)
                     Statistics: Num rows: 1049 Data size: 11234 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
@@ -1089,8 +1089,8 @@ STAGE PLANS:
                           0 _col0 (type: decimal(16,2))
                           1 _col0 (type: decimal(16,2))
                         Map Join Vectorization:
-                            bigTableKeyExpressions: ConvertDecimal64ToDecimal(col 0:decimal(16,2)/DECIMAL_64) -> 4:decimal(16,2)
-                            bigTableValueExpressions: ConvertDecimal64ToDecimal(col 0:decimal(16,2)/DECIMAL_64) -> 5:decimal(16,2), ConvertDecimal64ToDecimal(col 1:decimal(14,2)/DECIMAL_64) -> 3:decimal(14,2)
+                            bigTableKeyExpressions: ConvertDecimal64ToDecimal(col 0:decimal(16,2)/DECIMAL_64) -> 3:decimal(16,2)
+                            bigTableValueExpressions: ConvertDecimal64ToDecimal(col 0:decimal(16,2)/DECIMAL_64) -> 4:decimal(16,2), ConvertDecimal64ToDecimal(col 1:decimal(14,2)/DECIMAL_64) -> 5:decimal(14,2)
                             className: VectorMapJoinOperator
                             native: false
                             nativeConditionsMet: hive.mapjoin.optimized.hashtable IS true, hive.vectorized.execution.mapjoin.native.enabled IS true, hive.execution.engine spark IN [tez, spark] IS true, One MapJoin Condition IS true, No nullsafe IS true, Small table vectorizes IS true
@@ -1125,7 +1125,7 @@ STAGE PLANS:
                     includeColumns: [0, 1]
                     dataColumns: dec:decimal(14,2)/DECIMAL_64, value_dec:decimal(14,2)/DECIMAL_64
                     partitionColumnCount: 0
-                    scratchColumnTypeNames: [decimal(14,2), decimal(16,2), decimal(16,2), decimal(14,0)]
+                    scratchColumnTypeNames: [decimal(16,2), decimal(16,2), decimal(14,2), decimal(14,0)]
             Local Work:
               Map Reduce Local Work
 

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/test/results/clientpositive/spark/vectorized_case.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/vectorized_case.q.out b/ql/src/test/results/clientpositive/spark/vectorized_case.q.out
index 028351b..4d322cf 100644
--- a/ql/src/test/results/clientpositive/spark/vectorized_case.q.out
+++ b/ql/src/test/results/clientpositive/spark/vectorized_case.q.out
@@ -590,7 +590,7 @@ STAGE PLANS:
                         className: VectorSelectOperator
                         native: true
                         projectedOutputColumnNums: [6]
-                        selectExpressions: IfExprCondExprCondExpr(col 3:boolean, col 7:decimal(11,0)col 8:decimal(11,0))(children: VectorUDFAdaptor((member = 1)) -> 3:boolean, ConvertDecimal64ToDecimal(col 4:decimal(11,0)/DECIMAL_64)(children: Decimal64ColAddDecimal64Scalar(col 1:decimal(10,0)/DECIMAL_64, decimal64Val 1, decimalVal 1) -> 4:decimal(11,0)/DECIMAL_64) -> 7:decimal(11,0), ConvertDecimal64ToDecimal(col 5:decimal(11,0)/DECIMAL_64)(children: Decimal64ColAddDecimal64Scalar(col 1:decimal(10,0)/DECIMAL_64, decimal64Val 2, decimalVal 2) -> 5:decimal(11,0)/DECIMAL_64) -> 8:decimal(11,0)) -> 6:decimal(11,0)
+                        selectExpressions: IfExprCondExprCondExpr(col 3:boolean, col 7:decimal(11,0)col 8:decimal(11,0))(children: Decimal64ColEqualDecimal64Scalar(col 0:decimal(10,0)/DECIMAL_64, decimal64Val 1, decimalVal 1) -> 3:boolean, ConvertDecimal64ToDecimal(col 4:decimal(11,0)/DECIMAL_64)(children: Decimal64ColAddDecimal64Scalar(col 1:decimal(10,0)/DECIMAL_64, decimal64Val 1, decimalVal 1) -> 4:decimal(11,0)/DECIMAL_64) -> 7:decimal(11,0), ConvertDecimal64ToDecimal(col 5:decimal(11,0)/DECIMAL_64)(children: Decimal64ColAddDecimal64Scalar(col 1:decimal(10,0)/DECIMAL_64, decimal64Val 2, decimalVal 2) -> 5:decimal(11,0)/DECIMAL_64) -> 8:decimal(11,0)) -> 6:decimal(11,0)
                     Statistics: Num rows: 3 Data size: 672 Basic stats: COMPLETE Column stats: NONE
                     File Output Operator
                       compressed: false
@@ -610,7 +610,7 @@ STAGE PLANS:
                 featureSupportInUse: [DECIMAL_64]
                 inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
                 allNative: false
-                usesVectorUDFAdaptor: true
+                usesVectorUDFAdaptor: false
                 vectorized: true
                 rowBatchContext:
                     dataColumnCount: 2
@@ -670,7 +670,7 @@ STAGE PLANS:
                         className: VectorSelectOperator
                         native: true
                         projectedOutputColumnNums: [8]
-                        selectExpressions: VectorUDFAdaptor(CASE WHEN ((member = 1)) THEN (1) ELSE ((attr + 2)) END)(children: VectorUDFAdaptor((member = 1)) -> 6:boolean, ConvertDecimal64ToDecimal(col 7:decimal(11,0)/DECIMAL_64)(children: Decimal64ColAddDecimal64Scalar(col 1:decimal(10,0)/DECIMAL_64, decimal64Val 2, decimalVal 2) -> 7:decimal(11,0)/DECIMAL_64) -> 9:decimal(11,0)) -> 8:decimal(11,0)
+                        selectExpressions: IfExprDecimal64ScalarDecimal64Column(col 6:boolean, decimal64Val 1, decimalVal 1, col 7:decimal(1,0)/DECIMAL_64)(children: Decimal64ColEqualDecimal64Scalar(col 0:decimal(10,0)/DECIMAL_64, decimal64Val 1, decimalVal 1) -> 6:boolean, Decimal64ColAddDecimal64Scalar(col 1:decimal(10,0)/DECIMAL_64, decimal64Val 2, decimalVal 2) -> 7:decimal(11,0)/DECIMAL_64) -> 8:decimal(11,0)/DECIMAL_64
                     Statistics: Num rows: 3 Data size: 672 Basic stats: COMPLETE Column stats: NONE
                     File Output Operator
                       compressed: false
@@ -690,14 +690,14 @@ STAGE PLANS:
                 featureSupportInUse: [DECIMAL_64]
                 inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
                 allNative: false
-                usesVectorUDFAdaptor: true
+                usesVectorUDFAdaptor: false
                 vectorized: true
                 rowBatchContext:
                     dataColumnCount: 2
                     includeColumns: [0, 1]
                     dataColumns: member:decimal(10,0)/DECIMAL_64, attr:decimal(10,0)/DECIMAL_64
                     partitionColumnCount: 0
-                    scratchColumnTypeNames: [bigint, decimal(1,0), decimal(11,0)/DECIMAL_64, bigint, decimal(11,0)/DECIMAL_64, decimal(11,0), decimal(11,0)]
+                    scratchColumnTypeNames: [bigint, decimal(1,0), decimal(11,0)/DECIMAL_64, bigint, decimal(11,0)/DECIMAL_64, decimal(11,0)/DECIMAL_64]
 
   Stage: Stage-0
     Fetch Operator
@@ -750,7 +750,7 @@ STAGE PLANS:
                         className: VectorSelectOperator
                         native: true
                         projectedOutputColumnNums: [8]
-                        selectExpressions: VectorUDFAdaptor(CASE WHEN ((member = 1)) THEN ((attr + 1)) ELSE (2) END)(children: VectorUDFAdaptor((member = 1)) -> 6:boolean, ConvertDecimal64ToDecimal(col 7:decimal(11,0)/DECIMAL_64)(children: Decimal64ColAddDecimal64Scalar(col 1:decimal(10,0)/DECIMAL_64, decimal64Val 1, decimalVal 1) -> 7:decimal(11,0)/DECIMAL_64) -> 9:decimal(11,0)) -> 8:decimal(11,0)
+                        selectExpressions: IfExprDecimal64ColumnDecimal64Scalar(col 6:boolean, col 7:decimal(11,0)/DECIMAL_64, decimal64Val 2, decimalVal 2)(children: Decimal64ColEqualDecimal64Scalar(col 0:decimal(10,0)/DECIMAL_64, decimal64Val 1, decimalVal 1) -> 6:boolean, Decimal64ColAddDecimal64Scalar(col 1:decimal(10,0)/DECIMAL_64, decimal64Val 1, decimalVal 1) -> 7:decimal(11,0)/DECIMAL_64) -> 8:decimal(11,0)/DECIMAL_64
                     Statistics: Num rows: 3 Data size: 672 Basic stats: COMPLETE Column stats: NONE
                     File Output Operator
                       compressed: false
@@ -770,14 +770,14 @@ STAGE PLANS:
                 featureSupportInUse: [DECIMAL_64]
                 inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
                 allNative: false
-                usesVectorUDFAdaptor: true
+                usesVectorUDFAdaptor: false
                 vectorized: true
                 rowBatchContext:
                     dataColumnCount: 2
                     includeColumns: [0, 1]
                     dataColumns: member:decimal(10,0)/DECIMAL_64, attr:decimal(10,0)/DECIMAL_64
                     partitionColumnCount: 0
-                    scratchColumnTypeNames: [bigint, decimal(11,0)/DECIMAL_64, decimal(1,0), bigint, decimal(11,0)/DECIMAL_64, decimal(11,0), decimal(11,0)]
+                    scratchColumnTypeNames: [bigint, decimal(11,0)/DECIMAL_64, decimal(1,0), bigint, decimal(11,0)/DECIMAL_64, decimal(11,0)/DECIMAL_64]
 
   Stage: Stage-0
     Fetch Operator


[5/8] hive git commit: HIVE-20207: Vectorization: Fix NULL / Wrong Results issues in Filter / Compare (Matt McCline, reviewed by Teddy Choi)

Posted by mm...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColLessLongScalar.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColLessLongScalar.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColLessLongScalar.java
deleted file mode 100644
index c7efe84..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColLessLongScalar.java
+++ /dev/null
@@ -1,158 +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 java.util.Arrays;
-
-import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
-import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
-import org.apache.hadoop.hive.ql.metadata.HiveException;
-
-public class LongColLessLongScalar extends VectorExpression {
-
-  private static final long serialVersionUID = 1L;
-
-  protected final int colNum;
-  protected final long value;
-
-  public LongColLessLongScalar(int colNum, long value, int outputColumnNum) {
-    super(outputColumnNum);
-    this.colNum = colNum;
-    this.value = value;
-  }
-
-  public LongColLessLongScalar() {
-    super();
-
-    // Dummy final assignments.
-    colNum = -1;
-    value = 0;
-  }
-
-  @Override
-  public void evaluate(VectorizedRowBatch batch) throws HiveException {
-
-    if (childExpressions != null) {
-      super.evaluateChildren(batch);
-    }
-
-    LongColumnVector inputColVector = (LongColumnVector) batch.cols[colNum];
-    LongColumnVector outputColVector = (LongColumnVector) batch.cols[outputColumnNum];
-    int[] sel = batch.selected;
-    boolean[] inputIsNull = inputColVector.isNull;
-    boolean[] outputIsNull = outputColVector.isNull;
-    int n = batch.size;
-    long[] vector = inputColVector.vector;
-    long[] outputVector = outputColVector.vector;
-
-    // return immediately if batch is empty
-    if (n == 0) {
-      return;
-    }
-
-    // We do not need to do a column reset since we are carefully changing the output.
-    outputColVector.isRepeating = false;
-
-    if (inputColVector.isRepeating) {
-      if (inputColVector.noNulls || !inputIsNull[0]) {
-        outputIsNull[0] = false;
-        outputVector[0] = vector[0] < value ? 1 : 0;
-      } else {
-        outputIsNull[0] = true;
-        outputColVector.noNulls = false;
-      }
-      outputColVector.isRepeating = true;
-      return;
-    }
-
-    if (inputColVector.noNulls) {
-      if (batch.selectedInUse) {
-
-        // CONSIDER: For large n, fill n or all of isNull array and use the tighter ELSE loop.
-
-        if (!outputColVector.noNulls) {
-          for(int j = 0; j != n; j++) {
-           final int i = sel[j];
-           // Set isNull before call in case it changes it mind.
-           outputIsNull[i] = false;
-           outputVector[i] = (vector[i] - value) >>> 63;
-         }
-        } else {
-          for(int j = 0; j != n; j++) {
-            final int i = sel[j];
-            outputVector[i] = (vector[i] - value) >>> 63;
-          }
-        }
-      } else {
-        if (!outputColVector.noNulls) {
-
-          // Assume it is almost always a performance win to fill all of isNull so we can
-          // safely reset noNulls.
-          Arrays.fill(outputIsNull, false);
-          outputColVector.noNulls = true;
-        }
-        for(int i = 0; i != n; i++) {
-          outputVector[i] = (vector[i] - value) >>> 63;
-        }
-      }
-    } else /* there are nulls in the inputColVector */ {
-
-      // Carefully handle NULLs...
-
-      /*
-       * For better performance on LONG/DOUBLE we don't want the conditional
-       * statements inside the for loop.
-       */
-      outputColVector.noNulls = false;
-
-      if (batch.selectedInUse) {
-        for(int j=0; j != n; j++) {
-          int i = sel[j];
-          outputIsNull[i] = inputIsNull[i];
-          outputVector[i] = (vector[i] - value) >>> 63;
-        }
-      } else {
-        System.arraycopy(inputIsNull, 0, outputIsNull, 0, n);
-        for(int i = 0; i != n; i++) {
-          outputVector[i] = (vector[i] - value) >>> 63;
-        }
-      }
-    }
-  }
-
-  @Override
-  public String vectorExpressionParameters() {
-    return getColumnParamString(0, colNum) + ", val " + value;
-  }
-
-  @Override
-  public VectorExpressionDescriptor.Descriptor getDescriptor() {
-    return (new VectorExpressionDescriptor.Builder())
-        .setMode(
-            VectorExpressionDescriptor.Mode.PROJECTION)
-        .setNumArguments(2)
-        .setArgumentTypes(
-            VectorExpressionDescriptor.ArgumentType.getType("long"),
-            VectorExpressionDescriptor.ArgumentType.getType("long"))
-        .setInputExpressionTypes(
-            VectorExpressionDescriptor.InputExpressionType.COLUMN,
-            VectorExpressionDescriptor.InputExpressionType.SCALAR).build();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColNotEqualLongColumn.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColNotEqualLongColumn.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColNotEqualLongColumn.java
deleted file mode 100644
index 213b876..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColNotEqualLongColumn.java
+++ /dev/null
@@ -1,146 +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.LongColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
-import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
-import org.apache.hadoop.hive.ql.metadata.HiveException;
-
-public class LongColNotEqualLongColumn extends VectorExpression {
-
-  private static final long serialVersionUID = 1L;
-
-  private final int colNum1;
-  private final int colNum2;
-
-  public LongColNotEqualLongColumn(int colNum1, int colNum2, int outputColumnNum) {
-    super(outputColumnNum);
-    this.colNum1 = colNum1;
-    this.colNum2 = colNum2;
-  }
-
-  public LongColNotEqualLongColumn() {
-    super();
-
-    // Dummy final assignments.
-    colNum1 = -1;
-    colNum2 = -1;
-  }
-
-  @Override
-  public void evaluate(VectorizedRowBatch batch) throws HiveException {
-
-    if (childExpressions != null) {
-      super.evaluateChildren(batch);
-    }
-
-    LongColumnVector inputColVector1 = (LongColumnVector) batch.cols[colNum1];
-    LongColumnVector inputColVector2 = (LongColumnVector) batch.cols[colNum2];
-    LongColumnVector outputColVector = (LongColumnVector) batch.cols[outputColumnNum];
-    int[] sel = batch.selected;
-    int n = batch.size;
-    long[] vector1 = inputColVector1.vector;
-    long[] vector2 = inputColVector2.vector;
-    long[] outputVector = outputColVector.vector;
-    long vector1Value = vector1[0];
-    long vector2Value = vector2[0];
-
-    // return immediately if batch is empty
-    if (n == 0) {
-      return;
-    }
-
-    /*
-     * Propagate null values for a two-input operator and set isRepeating and noNulls appropriately.
-     */ 
-    NullUtil.propagateNullsColCol(
-        inputColVector1, inputColVector2, outputColVector, sel, n, batch.selectedInUse);
-
-    /* Disregard nulls for processing. In other words,
-     * the arithmetic operation is performed even if one or
-     * more inputs are null. This is to improve speed by avoiding
-     * conditional checks in the inner loop.
-     */
-    if (inputColVector1.isRepeating && inputColVector2.isRepeating) {
-      outputVector[0] = vector1Value != vector2Value ? 1 : 0;
-    } else if (inputColVector1.isRepeating) {
-      if (batch.selectedInUse) {
-        for(int j = 0; j != n; j++) {
-          int i = sel[j];
-          outputVector[i] = vector1Value != vector2[i] ? 1 : 0;
-        }
-      } else {
-        for(int i = 0; i != n; i++) {
-          // The SIMD optimized form of "a != b" is "((a - b) ^ (b - a)) >>> 63"
-          outputVector[i] = ((vector1Value - vector2[i]) ^ (vector2[i] - vector1Value)) >>> 63;
-        }
-      }
-    } else if (inputColVector2.isRepeating) {
-      if (batch.selectedInUse) {
-        for(int j = 0; j != n; j++) {
-          int i = sel[j];
-          outputVector[i] = vector1[i] != vector2Value ? 1 : 0;
-        }
-      } else {
-        for(int i = 0; i != n; i++) {
-          outputVector[i] = ((vector1[i] - vector2Value) ^ (vector2Value - vector1[i])) >>> 63;
-        }
-      }
-    } else {
-      if (batch.selectedInUse) {
-        for(int j = 0; j != n; j++) {
-          int i = sel[j];
-          outputVector[i] = vector1[i] != vector2[i] ? 1 : 0;
-        }
-      } else {
-        for(int i = 0; i != n; i++) {
-          outputVector[i] = ((vector1[i] - vector2[i]) ^ (vector2[i] - vector1[i])) >>> 63;
-        }
-      }
-    }
-
-    /* For the case when the output can have null values, follow
-     * the convention that the data values must be 1 for long and
-     * NaN for double. This is to prevent possible later zero-divide errors
-     * in complex arithmetic expressions like col2 / (col1 - 1)
-     * in the case when some col1 entries are null.
-     */
-    NullUtil.setNullDataEntriesLong(outputColVector, batch.selectedInUse, sel, n);
-  }
-
-  @Override
-  public String vectorExpressionParameters() {
-    return getColumnParamString(0, colNum1) + ", " + getColumnParamString(1, colNum2);
-  }
-
-  @Override
-  public VectorExpressionDescriptor.Descriptor getDescriptor() {
-    return (new VectorExpressionDescriptor.Builder())
-        .setMode(
-            VectorExpressionDescriptor.Mode.PROJECTION)
-        .setNumArguments(2)
-        .setArgumentTypes(
-            VectorExpressionDescriptor.ArgumentType.getType("int_date_interval_year_month"),
-            VectorExpressionDescriptor.ArgumentType.getType("int_date_interval_year_month"))
-        .setInputExpressionTypes(
-            VectorExpressionDescriptor.InputExpressionType.COLUMN,
-            VectorExpressionDescriptor.InputExpressionType.COLUMN).build();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColNotEqualLongScalar.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColNotEqualLongScalar.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColNotEqualLongScalar.java
deleted file mode 100644
index c2b52b8..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColNotEqualLongScalar.java
+++ /dev/null
@@ -1,158 +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 java.util.Arrays;
-
-import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
-import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
-import org.apache.hadoop.hive.ql.metadata.HiveException;
-
-public class LongColNotEqualLongScalar extends VectorExpression {
-
-  private static final long serialVersionUID = 1L;
-
-  protected final int colNum;
-  protected final long value;
-
-  public LongColNotEqualLongScalar(int colNum, long value, int outputColumnNum) {
-    super(outputColumnNum);
-    this.colNum = colNum;
-    this.value = value;
-  }
-
-  public LongColNotEqualLongScalar() {
-    super();
-
-    // Dummy final assignments.
-    colNum = -1;
-    value = 0;
-  }
-
-  @Override
-  public void evaluate(VectorizedRowBatch batch) throws HiveException {
-
-    if (childExpressions != null) {
-      super.evaluateChildren(batch);
-    }
-
-    LongColumnVector inputColVector = (LongColumnVector) batch.cols[colNum];
-    LongColumnVector outputColVector = (LongColumnVector) batch.cols[outputColumnNum];
-    int[] sel = batch.selected;
-    boolean[] inputIsNull = inputColVector.isNull;
-    boolean[] outputIsNull = outputColVector.isNull;
-    int n = batch.size;
-    long[] vector = inputColVector.vector;
-    long[] outputVector = outputColVector.vector;
-
-    // return immediately if batch is empty
-    if (n == 0) {
-      return;
-    }
-
-    // We do not need to do a column reset since we are carefully changing the output.
-    outputColVector.isRepeating = false;
-
-    if (inputColVector.isRepeating) {
-      if (inputColVector.noNulls || !inputIsNull[0]) {
-        outputIsNull[0] = false;
-        outputVector[0] = vector[0] != value ? 1 : 0;
-      } else {
-        outputIsNull[0] = true;
-        outputColVector.noNulls = false;
-      }
-      outputColVector.isRepeating = true;
-      return;
-    }
-
-    if (inputColVector.noNulls) {
-      if (batch.selectedInUse) {
-
-        // CONSIDER: For large n, fill n or all of isNull array and use the tighter ELSE loop.
-
-        if (!outputColVector.noNulls) {
-          for(int j = 0; j != n; j++) {
-           final int i = sel[j];
-           // Set isNull before call in case it changes it mind.
-           outputIsNull[i] = false;
-           outputVector[i] = ((vector[i] - value) ^ (value - vector[i])) >>> 63;
-         }
-        } else {
-          for(int j = 0; j != n; j++) {
-            final int i = sel[j];
-            outputVector[i] = ((vector[i] - value) ^ (value - vector[i])) >>> 63;
-          }
-        }
-      } else {
-        if (!outputColVector.noNulls) {
-
-          // Assume it is almost always a performance win to fill all of isNull so we can
-          // safely reset noNulls.
-          Arrays.fill(outputIsNull, false);
-          outputColVector.noNulls = true;
-        }
-        for(int i = 0; i != n; i++) {
-          outputVector[i] = ((vector[i] - value) ^ (value - vector[i])) >>> 63;
-        }
-      }
-    } else /* there are nulls in the inputColVector */ {
-
-      // Carefully handle NULLs...
-
-      /*
-       * For better performance on LONG/DOUBLE we don't want the conditional
-       * statements inside the for loop.
-       */
-      outputColVector.noNulls = false;
-
-      if (batch.selectedInUse) {
-        for(int j=0; j != n; j++) {
-          int i = sel[j];
-          outputIsNull[i] = inputIsNull[i];
-          outputVector[i] = ((vector[i] - value) ^ (value - vector[i])) >>> 63;
-        }
-      } else {
-        System.arraycopy(inputIsNull, 0, outputIsNull, 0, n);
-        for(int i = 0; i != n; i++) {
-          outputVector[i] = ((vector[i] - value) ^ (value - vector[i])) >>> 63;
-        }
-      }
-    }
-  }
-
-  @Override
-  public String vectorExpressionParameters() {
-    return getColumnParamString(0, colNum) + ", val " + value;
-  }
-
-  @Override
-  public VectorExpressionDescriptor.Descriptor getDescriptor() {
-    return (new VectorExpressionDescriptor.Builder())
-        .setMode(
-            VectorExpressionDescriptor.Mode.PROJECTION)
-        .setNumArguments(2)
-        .setArgumentTypes(
-            VectorExpressionDescriptor.ArgumentType.getType("long"),
-            VectorExpressionDescriptor.ArgumentType.getType("long"))
-        .setInputExpressionTypes(
-            VectorExpressionDescriptor.InputExpressionType.COLUMN,
-            VectorExpressionDescriptor.InputExpressionType.SCALAR).build();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongScalarEqualLongColumn.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongScalarEqualLongColumn.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongScalarEqualLongColumn.java
deleted file mode 100644
index 7b28f71..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongScalarEqualLongColumn.java
+++ /dev/null
@@ -1,157 +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 java.util.Arrays;
-
-import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
-import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
-import org.apache.hadoop.hive.ql.metadata.HiveException;
-
-public class LongScalarEqualLongColumn extends VectorExpression {
-
-  private static final long serialVersionUID = 1L;
-
-  protected final int colNum;
-  protected final long value;
-
-  public LongScalarEqualLongColumn(long value, int colNum, int outputColumnNum) {
-    super(outputColumnNum);
-    this.colNum = colNum;
-    this.value = value;
-  }
-
-  public LongScalarEqualLongColumn() {
-    super();
-
-    // Dummy final assignments.
-    colNum = -1;
-    value = 0;
-  }
-
-  @Override
-  public void evaluate(VectorizedRowBatch batch) throws HiveException {
-
-    if (childExpressions != null) {
-      super.evaluateChildren(batch);
-    }
-
-    LongColumnVector inputColVector = (LongColumnVector) batch.cols[colNum];
-    LongColumnVector outputColVector = (LongColumnVector) batch.cols[outputColumnNum];
-    int[] sel = batch.selected;
-    boolean[] inputIsNull = inputColVector.isNull;
-    boolean[] outputIsNull = outputColVector.isNull;
-    int n = batch.size;
-    long[] vector = inputColVector.vector;
-    long[] outputVector = outputColVector.vector;
-
-    // return immediately if batch is empty
-    if (n == 0) {
-      return;
-    }
-
-    // We do not need to do a column reset since we are carefully changing the output.
-    outputColVector.isRepeating = false;
-
-    if (inputColVector.isRepeating) {
-      if (inputColVector.noNulls || !inputIsNull[0]) {
-        outputIsNull[0] = false;
-        outputVector[0] = vector[0] == value ? 1 : 0;
-      } else {
-        outputIsNull[0] = true;
-        outputColVector.noNulls = false;
-      }
-      outputColVector.isRepeating = true;
-      return;
-    }
-
-    if (inputColVector.noNulls) {
-      if (batch.selectedInUse) {
-
-        // CONSIDER: For large n, fill n or all of isNull array and use the tighter ELSE loop.
-
-        if (!outputColVector.noNulls) {
-          for(int j = 0; j != n; j++) {
-           final int i = sel[j];
-           // Set isNull before call in case it changes it mind.
-           outputIsNull[i] = false;
-           outputVector[i] = (((value - vector[i]) ^ (vector[i] - value)) >>> 63) ^ 1;
-         }
-        } else {
-          for(int j = 0; j != n; j++) {
-            final int i = sel[j];
-            outputVector[i] = (((value - vector[i]) ^ (vector[i] - value)) >>> 63) ^ 1;
-          }
-        }
-      } else {
-        if (!outputColVector.noNulls) {
-
-          // Assume it is almost always a performance win to fill all of isNull so we can
-          // safely reset noNulls.
-          Arrays.fill(outputIsNull, false);
-          outputColVector.noNulls = true;
-        }
-        for(int i = 0; i != n; i++) {
-          outputVector[i] = (((value - vector[i]) ^ (vector[i] - value)) >>> 63) ^ 1;
-        }
-      }
-    } else /* there are nulls in the inputColVector */ {
-
-      // Carefully handle NULLs...
-
-      /*
-       * For better performance on LONG/DOUBLE we don't want the conditional
-       * statements inside the for loop.
-       */
-      outputColVector.noNulls = false;
-
-      if (batch.selectedInUse) {
-        for(int j=0; j != n; j++) {
-          int i = sel[j];
-          outputIsNull[i] = inputIsNull[i];
-          outputVector[i] = (((value - vector[i]) ^ (vector[i] - value)) >>> 63) ^ 1;
-        }
-      } else {
-        System.arraycopy(inputIsNull, 0, outputIsNull, 0, n);
-        for(int i = 0; i != n; i++) {
-          outputVector[i] = (((value - vector[i]) ^ (vector[i] - value)) >>> 63) ^ 1;
-        }
-      }
-    }
-  }
-
-  public String vectorExpressionParameters() {
-    return "val " + value + ", " + getColumnParamString(1, colNum);
-  }
-
-  @Override
-  public VectorExpressionDescriptor.Descriptor getDescriptor() {
-    return (new VectorExpressionDescriptor.Builder())
-        .setMode(
-            VectorExpressionDescriptor.Mode.PROJECTION)
-        .setNumArguments(2)
-        .setArgumentTypes(
-            VectorExpressionDescriptor.ArgumentType.getType("long"),
-            VectorExpressionDescriptor.ArgumentType.getType("long"))
-        .setInputExpressionTypes(
-            VectorExpressionDescriptor.InputExpressionType.SCALAR,
-            VectorExpressionDescriptor.InputExpressionType.COLUMN).build();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongScalarGreaterEqualLongColumn.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongScalarGreaterEqualLongColumn.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongScalarGreaterEqualLongColumn.java
deleted file mode 100644
index 10fd423..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongScalarGreaterEqualLongColumn.java
+++ /dev/null
@@ -1,160 +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 java.util.Arrays;
-
-import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
-import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
-import org.apache.hadoop.hive.ql.metadata.HiveException;
-
-public class LongScalarGreaterEqualLongColumn extends VectorExpression {
-  private static final long serialVersionUID = 1L;
-
-  protected final int colNum;
-  protected final long value;
-
-  public LongScalarGreaterEqualLongColumn(long value, int colNum, int outputColumnNum) {
-    super(outputColumnNum);
-    this.colNum = colNum;
-    this.value = value;
-  }
-
-  public LongScalarGreaterEqualLongColumn() {
-    super();
-
-    // Dummy final assignments.
-    colNum = -1;
-    value = 0;
-  }
-
-  @Override
-  public void evaluate(VectorizedRowBatch batch) throws HiveException {
-
-    if (childExpressions != null) {
-      super.evaluateChildren(batch);
-    }
-
-    LongColumnVector inputColVector = (LongColumnVector) batch.cols[colNum];
-    LongColumnVector outputColVector = (LongColumnVector) batch.cols[outputColumnNum];
-    int[] sel = batch.selected;
-    boolean[] inputIsNull = inputColVector.isNull;
-    boolean[] outputIsNull = outputColVector.isNull;
-    int n = batch.size;
-    long[] vector = inputColVector.vector;
-    long[] outputVector = outputColVector.vector;
-
-    // return immediately if batch is empty
-    if (n == 0) {
-      return;
-    }
-
-    // We do not need to do a column reset since we are carefully changing the output.
-    outputColVector.isRepeating = false;
-
-    if (inputColVector.isRepeating) {
-      if (inputColVector.noNulls || !inputIsNull[0]) {
-        outputIsNull[0] = false;
-        outputVector[0] = value >= vector[0] ? 1 : 0;
-      } else {
-        outputIsNull[0] = true;
-        outputColVector.noNulls = false;
-      }
-      outputColVector.isRepeating = true;
-      return;
-    }
-
-    if (inputColVector.noNulls) {
-      if (batch.selectedInUse) {
-
-        // CONSIDER: For large n, fill n or all of isNull array and use the tighter ELSE loop.
-
-        if (!outputColVector.noNulls) {
-          for(int j = 0; j != n; j++) {
-           final int i = sel[j];
-           // Set isNull before call in case it changes it mind.
-           outputIsNull[i] = false;
-           // The SIMD optimized form of "a >= b" is "((a - b) >>> 63) ^ 1"
-           outputVector[i] = ((value - vector[i]) >>> 63) ^ 1;
-         }
-        } else {
-          for(int j = 0; j != n; j++) {
-            final int i = sel[j];
-            // The SIMD optimized form of "a >= b" is "((a - b) >>> 63) ^ 1"
-            outputVector[i] = ((value - vector[i]) >>> 63) ^ 1;
-          }
-        }
-      } else {
-        if (!outputColVector.noNulls) {
-
-          // Assume it is almost always a performance win to fill all of isNull so we can
-          // safely reset noNulls.
-          Arrays.fill(outputIsNull, false);
-          outputColVector.noNulls = true;
-        }
-        for(int i = 0; i != n; i++) {
-          // The SIMD optimized form of "a >= b" is "((a - b) >>> 63) ^ 1"
-          outputVector[i] = ((value - vector[i]) >>> 63) ^ 1;
-        }
-      }
-    } else /* there are nulls in the inputColVector */ {
-
-      // Carefully handle NULLs...
-
-      /*
-       * For better performance on LONG/DOUBLE we don't want the conditional
-       * statements inside the for loop.
-       */
-      outputColVector.noNulls = false;
-
-      if (batch.selectedInUse) {
-        for(int j=0; j != n; j++) {
-          int i = sel[j];
-          outputIsNull[i] = inputIsNull[i];
-          outputVector[i] = ((value - vector[i]) >>> 63) ^ 1;
-        }
-      } else {
-        System.arraycopy(inputIsNull, 0, outputIsNull, 0, n);
-        for(int i = 0; i != n; i++) {
-          outputVector[i] = ((value - vector[i]) >>> 63) ^ 1;
-        }
-      }
-    }
-  }
-
-  @Override
-  public String vectorExpressionParameters() {
-    return "val " + value + ", " + getColumnParamString(1, colNum);
-  }
-
-  @Override
-  public VectorExpressionDescriptor.Descriptor getDescriptor() {
-    return (new VectorExpressionDescriptor.Builder())
-        .setMode(
-            VectorExpressionDescriptor.Mode.PROJECTION)
-        .setNumArguments(2)
-        .setArgumentTypes(
-            VectorExpressionDescriptor.ArgumentType.getType("long"),
-            VectorExpressionDescriptor.ArgumentType.getType("long"))
-        .setInputExpressionTypes(
-            VectorExpressionDescriptor.InputExpressionType.SCALAR,
-            VectorExpressionDescriptor.InputExpressionType.COLUMN).build();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongScalarGreaterLongColumn.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongScalarGreaterLongColumn.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongScalarGreaterLongColumn.java
deleted file mode 100644
index ad9ccf5..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongScalarGreaterLongColumn.java
+++ /dev/null
@@ -1,161 +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 java.util.Arrays;
-
-import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
-import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
-import org.apache.hadoop.hive.ql.metadata.HiveException;
-
-public class LongScalarGreaterLongColumn extends VectorExpression {
-
-  private static final long serialVersionUID = 1L;
-
-  protected int colNum;
-  protected long value;
-
-  public LongScalarGreaterLongColumn(long value, int colNum, int outputColumnNum) {
-    super(outputColumnNum);
-    this.colNum = colNum;
-    this.value = value;
-  }
-
-  public LongScalarGreaterLongColumn() {
-    super();
-
-    // Dummy final assignments.
-    colNum = -1;
-    value = -1;
-  }
-
-  @Override
-  public void evaluate(VectorizedRowBatch batch) throws HiveException {
-
-    if (childExpressions != null) {
-      super.evaluateChildren(batch);
-    }
-
-    LongColumnVector inputColVector = (LongColumnVector) batch.cols[colNum];
-    LongColumnVector outputColVector = (LongColumnVector) batch.cols[outputColumnNum];
-    int[] sel = batch.selected;
-    boolean[] inputIsNull = inputColVector.isNull;
-    boolean[] outputIsNull = outputColVector.isNull;
-    int n = batch.size;
-    long[] vector = inputColVector.vector;
-    long[] outputVector = outputColVector.vector;
-
-    // return immediately if batch is empty
-    if (n == 0) {
-      return;
-    }
-
-    // We do not need to do a column reset since we are carefully changing the output.
-    outputColVector.isRepeating = false;
-
-    if (inputColVector.isRepeating) {
-      if (inputColVector.noNulls || !inputIsNull[0]) {
-        outputIsNull[0] = false;
-        outputVector[0] = value > vector[0] ? 1 : 0;
-      } else {
-        outputIsNull[0] = true;
-        outputColVector.noNulls = false;
-      }
-      outputColVector.isRepeating = true;
-      return;
-    }
-
-    if (inputColVector.noNulls) {
-      if (batch.selectedInUse) {
-
-        // CONSIDER: For large n, fill n or all of isNull array and use the tighter ELSE loop.
-
-        if (!outputColVector.noNulls) {
-          for(int j = 0; j != n; j++) {
-           final int i = sel[j];
-           // Set isNull before call in case it changes it mind.
-           outputIsNull[i] = false;
-           // The SIMD optimized form of "a > b" is "(b - a) >>> 63"
-           outputVector[i] = (vector[i] - value) >>> 63;
-         }
-        } else {
-          for(int j = 0; j != n; j++) {
-            final int i = sel[j];
-            // The SIMD optimized form of "a > b" is "(b - a) >>> 63"
-            outputVector[i] = (vector[i] - value) >>> 63;
-          }
-        }
-      } else {
-        if (!outputColVector.noNulls) {
-
-          // Assume it is almost always a performance win to fill all of isNull so we can
-          // safely reset noNulls.
-          Arrays.fill(outputIsNull, false);
-          outputColVector.noNulls = true;
-        }
-        for(int i = 0; i != n; i++) {
-          // The SIMD optimized form of "a > b" is "(b - a) >>> 63"
-          outputVector[i] = (vector[i] - value) >>> 63;
-        }
-      }
-    } else /* there are nulls in the inputColVector */ {
-
-      // Carefully handle NULLs...
-
-      /*
-       * For better performance on LONG/DOUBLE we don't want the conditional
-       * statements inside the for loop.
-       */
-      outputColVector.noNulls = false;
-
-      if (batch.selectedInUse) {
-        for(int j=0; j != n; j++) {
-          int i = sel[j];
-          outputIsNull[i] = inputIsNull[i];
-          outputVector[i] = (vector[i] - value) >>> 63;
-        }
-      } else {
-        System.arraycopy(inputIsNull, 0, outputIsNull, 0, n);
-        for(int i = 0; i != n; i++) {
-          outputVector[i] = (vector[i] - value) >>> 63;
-        }
-      }
-    }
-  }
-
-  @Override
-  public String vectorExpressionParameters() {
-    return "val " + value + ", " + getColumnParamString(0, colNum);
-  }
-
-  @Override
-  public VectorExpressionDescriptor.Descriptor getDescriptor() {
-    return (new VectorExpressionDescriptor.Builder())
-        .setMode(
-            VectorExpressionDescriptor.Mode.PROJECTION)
-        .setNumArguments(2)
-        .setArgumentTypes(
-            VectorExpressionDescriptor.ArgumentType.getType("long"),
-            VectorExpressionDescriptor.ArgumentType.getType("long"))
-        .setInputExpressionTypes(
-            VectorExpressionDescriptor.InputExpressionType.SCALAR,
-            VectorExpressionDescriptor.InputExpressionType.COLUMN).build();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongScalarLessEqualLongColumn.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongScalarLessEqualLongColumn.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongScalarLessEqualLongColumn.java
deleted file mode 100644
index abe5d08..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongScalarLessEqualLongColumn.java
+++ /dev/null
@@ -1,160 +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 java.util.Arrays;
-
-import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
-import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
-import org.apache.hadoop.hive.ql.metadata.HiveException;
-
-public class LongScalarLessEqualLongColumn extends VectorExpression {
-
-  private static final long serialVersionUID = 1L;
-
-  protected final int colNum;
-  protected final long value;
-
-  public LongScalarLessEqualLongColumn(long value, int colNum, int outputColumnNum) {
-    super(outputColumnNum);
-    this.colNum = colNum;
-    this.value = value;
-  }
-
-  public LongScalarLessEqualLongColumn() {
-    super();
-
-    // Dummy final assignments.
-    colNum = -1;
-    value = 0;
-  }
-
-  @Override
-  public void evaluate(VectorizedRowBatch batch) throws HiveException {
-
-    if (childExpressions != null) {
-      super.evaluateChildren(batch);
-    }
-
-    LongColumnVector inputColVector = (LongColumnVector) batch.cols[colNum];
-    LongColumnVector outputColVector = (LongColumnVector) batch.cols[outputColumnNum];
-    int[] sel = batch.selected;
-    boolean[] inputIsNull = inputColVector.isNull;
-    boolean[] outputIsNull = outputColVector.isNull;
-    int n = batch.size;
-    long[] vector = inputColVector.vector;
-    long[] outputVector = outputColVector.vector;
-
-    // return immediately if batch is empty
-    if (n == 0) {
-      return;
-    }
-
-    // We do not need to do a column reset since we are carefully changing the output.
-    outputColVector.isRepeating = false;
-
-    if (inputColVector.isRepeating) {
-      if (inputColVector.noNulls || !inputIsNull[0]) {
-        outputIsNull[0] = false;
-        outputVector[0] = value <= vector[0] ? 1 : 0;
-      } else {
-        outputIsNull[0] = true;
-        outputColVector.noNulls = false;
-      }
-      outputColVector.isRepeating = true;
-      return;
-    }
-
-    if (inputColVector.noNulls) {
-      if (batch.selectedInUse) {
-
-        // CONSIDER: For large n, fill n or all of isNull array and use the tighter ELSE loop.
-
-        if (!outputColVector.noNulls) {
-          for(int j = 0; j != n; j++) {
-           final int i = sel[j];
-           // Set isNull before call in case it changes it mind.
-           outputIsNull[i] = false;
-           // The SIMD optimized form of "a <= b" is "((b - a) >>> 63) ^ 1"
-           outputVector[i] = ((vector[i] - value) >>> 63) ^ 1;
-         }
-        } else {
-          for(int j = 0; j != n; j++) {
-            final int i = sel[j];
-            // The SIMD optimized form of "a <= b" is "((b - a) >>> 63) ^ 1"
-            outputVector[i] = ((vector[i] - value) >>> 63) ^ 1;
-          }
-        }
-      } else {
-        if (!outputColVector.noNulls) {
-
-          // Assume it is almost always a performance win to fill all of isNull so we can
-          // safely reset noNulls.
-          Arrays.fill(outputIsNull, false);
-          outputColVector.noNulls = true;
-        }
-        for(int i = 0; i != n; i++) {
-          // The SIMD optimized form of "a <= b" is "((b - a) >>> 63) ^ 1"
-          outputVector[i] = ((vector[i] - value) >>> 63) ^ 1;
-        }
-      }
-    } else /* there are nulls in the inputColVector */ {
-
-      // Carefully handle NULLs...
-
-      /*
-       * For better performance on LONG/DOUBLE we don't want the conditional
-       * statements inside the for loop.
-       */
-      outputColVector.noNulls = false;
-
-      if (batch.selectedInUse) {
-        for(int j=0; j != n; j++) {
-          int i = sel[j];
-          outputIsNull[i] = inputIsNull[i];
-          outputVector[i] = ((vector[i] - value) >>> 63) ^ 1;
-        }
-      } else {
-        System.arraycopy(inputIsNull, 0, outputIsNull, 0, n);
-        for(int i = 0; i != n; i++) {
-          outputVector[i] = ((vector[i] - value) >>> 63) ^ 1;
-        }
-      }
-    }
-  }
-
-  public String vectorExpressionParameters() {
-    return "val " + value + ", " + getColumnParamString(1, colNum);
-  }
-
-  @Override
-  public VectorExpressionDescriptor.Descriptor getDescriptor() {
-    return (new VectorExpressionDescriptor.Builder())
-        .setMode(
-            VectorExpressionDescriptor.Mode.PROJECTION)
-        .setNumArguments(2)
-        .setArgumentTypes(
-            VectorExpressionDescriptor.ArgumentType.getType("long"),
-            VectorExpressionDescriptor.ArgumentType.getType("long"))
-        .setInputExpressionTypes(
-            VectorExpressionDescriptor.InputExpressionType.SCALAR,
-            VectorExpressionDescriptor.InputExpressionType.COLUMN).build();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongScalarLessLongColumn.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongScalarLessLongColumn.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongScalarLessLongColumn.java
deleted file mode 100644
index 87ab939..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongScalarLessLongColumn.java
+++ /dev/null
@@ -1,161 +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 java.util.Arrays;
-
-import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
-import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
-import org.apache.hadoop.hive.ql.metadata.HiveException;
-
-public class LongScalarLessLongColumn extends VectorExpression {
-
-  private static final long serialVersionUID = 1L;
-
-  protected final int colNum;
-  protected final long value;
-
-  public LongScalarLessLongColumn(long value, int colNum, int outputColumnNum) {
-    super(outputColumnNum);
-    this.colNum = colNum;
-    this.value = value;
-  }
-
-  public LongScalarLessLongColumn() {
-    super();
-
-    // Dummy final assignments.
-    colNum = -1;
-    value = 0;
-  }
-
-  @Override
-  public void evaluate(VectorizedRowBatch batch) throws HiveException {
-
-    if (childExpressions != null) {
-      super.evaluateChildren(batch);
-    }
-
-    LongColumnVector inputColVector = (LongColumnVector) batch.cols[colNum];
-    LongColumnVector outputColVector = (LongColumnVector) batch.cols[outputColumnNum];
-    int[] sel = batch.selected;
-    boolean[] inputIsNull = inputColVector.isNull;
-    boolean[] outputIsNull = outputColVector.isNull;
-    int n = batch.size;
-    long[] vector = inputColVector.vector;
-    long[] outputVector = outputColVector.vector;
-
-    // return immediately if batch is empty
-    if (n == 0) {
-      return;
-    }
-
-    // We do not need to do a column reset since we are carefully changing the output.
-    outputColVector.isRepeating = false;
-
-    if (inputColVector.isRepeating) {
-      if (inputColVector.noNulls || !inputIsNull[0]) {
-        outputIsNull[0] = false;
-        outputVector[0] = value < vector[0] ? 1 : 0;
-      } else {
-        outputIsNull[0] = true;
-        outputColVector.noNulls = false;
-      }
-      outputColVector.isRepeating = true;
-      return;
-    }
-
-    if (inputColVector.noNulls) {
-      if (batch.selectedInUse) {
-
-        // CONSIDER: For large n, fill n or all of isNull array and use the tighter ELSE loop.
-
-        if (!outputColVector.noNulls) {
-          for(int j = 0; j != n; j++) {
-           final int i = sel[j];
-           // Set isNull before call in case it changes it mind.
-           outputIsNull[i] = false;
-           // The SIMD optimized form of "a < b" is "(a - b) >>> 63"
-           outputVector[i] = (value - vector[i]) >>> 63;
-         }
-        } else {
-          for(int j = 0; j != n; j++) {
-            final int i = sel[j];
-            // The SIMD optimized form of "a < b" is "(a - b) >>> 63"
-            outputVector[i] = (value - vector[i]) >>> 63;
-          }
-        }
-      } else {
-        if (!outputColVector.noNulls) {
-
-          // Assume it is almost always a performance win to fill all of isNull so we can
-          // safely reset noNulls.
-          Arrays.fill(outputIsNull, false);
-          outputColVector.noNulls = true;
-        }
-        for(int i = 0; i != n; i++) {
-          // The SIMD optimized form of "a < b" is "(a - b) >>> 63"
-          outputVector[i] = (value - vector[i]) >>> 63;
-        }
-      }
-    } else /* there are nulls in the inputColVector */ {
-
-      // Carefully handle NULLs...
-
-      /*
-       * For better performance on LONG/DOUBLE we don't want the conditional
-       * statements inside the for loop.
-       */
-      outputColVector.noNulls = false;
-
-      if (batch.selectedInUse) {
-        for(int j=0; j != n; j++) {
-          int i = sel[j];
-          outputIsNull[i] = inputIsNull[i];
-          outputVector[i] = (value - vector[i]) >>> 63;
-        }
-      } else {
-        System.arraycopy(inputIsNull, 0, outputIsNull, 0, n);
-        for(int i = 0; i != n; i++) {
-          outputVector[i] = (value - vector[i]) >>> 63;
-        }
-      }
-    }
-  }
-
-  @Override
-  public String vectorExpressionParameters() {
-    return "val " + value + ", " + getColumnParamString(1, colNum);
-  }
-
-  @Override
-  public VectorExpressionDescriptor.Descriptor getDescriptor() {
-    return (new VectorExpressionDescriptor.Builder())
-        .setMode(
-            VectorExpressionDescriptor.Mode.PROJECTION)
-        .setNumArguments(2)
-        .setArgumentTypes(
-            VectorExpressionDescriptor.ArgumentType.getType("long"),
-            VectorExpressionDescriptor.ArgumentType.getType("long"))
-        .setInputExpressionTypes(
-            VectorExpressionDescriptor.InputExpressionType.SCALAR,
-            VectorExpressionDescriptor.InputExpressionType.COLUMN).build();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongScalarNotEqualLongColumn.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongScalarNotEqualLongColumn.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongScalarNotEqualLongColumn.java
deleted file mode 100644
index d936dee..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongScalarNotEqualLongColumn.java
+++ /dev/null
@@ -1,161 +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 java.util.Arrays;
-
-import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
-import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
-import org.apache.hadoop.hive.ql.metadata.HiveException;
-
-public class LongScalarNotEqualLongColumn extends VectorExpression {
-
-  private static final long serialVersionUID = 1L;
-
-  protected final int colNum;
-  protected final long value;
-
-  public LongScalarNotEqualLongColumn(long value, int colNum, int outputColumnNum) {
-    super(outputColumnNum);
-    this.colNum = colNum;
-    this.value = value;
-  }
-
-  public LongScalarNotEqualLongColumn() {
-    super();
-
-    // Dummy final assignments.
-    colNum = -1;
-    value = 0;
-  }
-
-  @Override
-  public void evaluate(VectorizedRowBatch batch) throws HiveException {
-
-    if (childExpressions != null) {
-      super.evaluateChildren(batch);
-    }
-
-    LongColumnVector inputColVector = (LongColumnVector) batch.cols[colNum];
-    LongColumnVector outputColVector = (LongColumnVector) batch.cols[outputColumnNum];
-    int[] sel = batch.selected;
-    boolean[] inputIsNull = inputColVector.isNull;
-    boolean[] outputIsNull = outputColVector.isNull;
-    int n = batch.size;
-    long[] vector = inputColVector.vector;
-    long[] outputVector = outputColVector.vector;
-
-    // return immediately if batch is empty
-    if (n == 0) {
-      return;
-    }
-
-    // We do not need to do a column reset since we are carefully changing the output.
-    outputColVector.isRepeating = false;
-
-    if (inputColVector.isRepeating) {
-      if (inputColVector.noNulls || !inputIsNull[0]) {
-        outputIsNull[0] = false;
-        outputVector[0] = value != vector[0] ? 1 : 0;
-      } else {
-        outputIsNull[0] = true;
-        outputColVector.noNulls = false;
-      }
-      outputColVector.isRepeating = true;
-      return;
-    }
-
-    if (inputColVector.noNulls) {
-      if (batch.selectedInUse) {
-
-        // CONSIDER: For large n, fill n or all of isNull array and use the tighter ELSE loop.
-
-        if (!outputColVector.noNulls) {
-          for(int j = 0; j != n; j++) {
-           final int i = sel[j];
-           // Set isNull before call in case it changes it mind.
-           outputIsNull[i] = false;
-           // The SIMD optimized form of "a != b" is "((a - b) ^ (b - a)) >>> 63"
-           outputVector[i] = ((vector[i] - value) ^ (value - vector[i])) >>> 63;
-         }
-        } else {
-          for(int j = 0; j != n; j++) {
-            final int i = sel[j];
-            // The SIMD optimized form of "a != b" is "((a - b) ^ (b - a)) >>> 63"
-            outputVector[i] = ((vector[i] - value) ^ (value - vector[i])) >>> 63;
-          }
-        }
-      } else {
-        if (!outputColVector.noNulls) {
-
-          // Assume it is almost always a performance win to fill all of isNull so we can
-          // safely reset noNulls.
-          Arrays.fill(outputIsNull, false);
-          outputColVector.noNulls = true;
-        }
-        for(int i = 0; i != n; i++) {
-          // The SIMD optimized form of "a != b" is "((a - b) ^ (b - a)) >>> 63"
-          outputVector[i] = ((vector[i] - value) ^ (value - vector[i])) >>> 63;
-        }
-      }
-    } else /* there are nulls in the inputColVector */ {
-
-      // Carefully handle NULLs...
-
-      /*
-       * For better performance on LONG/DOUBLE we don't want the conditional
-       * statements inside the for loop.
-       */
-      outputColVector.noNulls = false;
-
-      if (batch.selectedInUse) {
-        for(int j=0; j != n; j++) {
-          int i = sel[j];
-          outputIsNull[i] = inputIsNull[i];
-          outputVector[i] = ((vector[i] - value) ^ (value - vector[i])) >>> 63;
-        }
-      } else {
-        System.arraycopy(inputIsNull, 0, outputIsNull, 0, n);
-        for(int i = 0; i != n; i++) {
-          outputVector[i] = ((vector[i] - value) ^ (value - vector[i])) >>> 63;
-        }
-      }
-    }
-  }
-
-  @Override
-  public String vectorExpressionParameters() {
-    return "val " + value + ", " + getColumnParamString(1, colNum);
-  }
-
-  @Override
-  public VectorExpressionDescriptor.Descriptor getDescriptor() {
-    return (new VectorExpressionDescriptor.Builder())
-        .setMode(
-            VectorExpressionDescriptor.Mode.PROJECTION)
-        .setNumArguments(2)
-        .setArgumentTypes(
-            VectorExpressionDescriptor.ArgumentType.getType("long"),
-            VectorExpressionDescriptor.ArgumentType.getType("long"))
-        .setInputExpressionTypes(
-            VectorExpressionDescriptor.InputExpressionType.SCALAR,
-            VectorExpressionDescriptor.InputExpressionType.COLUMN).build();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFIf.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFIf.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFIf.java
index 142dd1b..25c54e9 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFIf.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFIf.java
@@ -23,11 +23,16 @@ import org.apache.hadoop.hive.ql.exec.UDFArgumentException;
 import org.apache.hadoop.hive.ql.exec.UDFArgumentLengthException;
 import org.apache.hadoop.hive.ql.exec.UDFArgumentTypeException;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedExpressions;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedExpressionsSupportDecimal64;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.serde.serdeConstants;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.BooleanObjectInspector;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.IfExprDecimalColumnColumn;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.IfExprDecimalColumnScalar;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.IfExprDecimalScalarColumn;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.IfExprDecimalScalarScalar;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.IfExprLongColumnLongScalar;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.IfExprDoubleColumnDoubleScalar;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.IfExprDoubleColumnLongScalar;
@@ -45,6 +50,10 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.IfExprIntervalDayTi
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.IfExprIntervalDayTimeColumnScalar;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.IfExprIntervalDayTimeScalarColumn;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.IfExprIntervalDayTimeScalarScalar;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.IfExprDecimal64ColumnDecimal64Column;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.IfExprDecimal64ColumnDecimal64Scalar;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.IfExprDecimal64ScalarDecimal64Column;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.IfExprDecimal64ScalarDecimal64Scalar;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.IfExprLongColumnLongColumn;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.IfExprStringGroupColumnStringGroupColumn;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.IfExprStringGroupColumnStringScalar;
@@ -85,6 +94,9 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.IfExprVarCharScalarStri
   IfExprLongScalarLongScalar.class, IfExprDoubleScalarDoubleScalar.class,
   IfExprLongScalarDoubleScalar.class, IfExprDoubleScalarLongScalar.class,
 
+  IfExprDecimal64ColumnDecimal64Column.class, IfExprDecimal64ColumnDecimal64Scalar.class,
+  IfExprDecimal64ScalarDecimal64Column.class, IfExprDecimal64ScalarDecimal64Scalar.class,
+
   IfExprStringGroupColumnStringGroupColumn.class,
   IfExprStringGroupColumnStringScalar.class,
   IfExprStringGroupColumnCharScalar.class, IfExprStringGroupColumnVarCharScalar.class,
@@ -94,11 +106,15 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.IfExprVarCharScalarStri
   IfExprStringScalarCharScalar.class, IfExprStringScalarVarCharScalar.class,
   IfExprCharScalarStringScalar.class, IfExprVarCharScalarStringScalar.class,
 
+  IfExprDecimalColumnColumn.class, IfExprDecimalColumnScalar.class,
+  IfExprDecimalScalarColumn.class, IfExprDecimalScalarScalar.class,
+
   IfExprIntervalDayTimeColumnColumn.class, IfExprIntervalDayTimeColumnScalar.class,
   IfExprIntervalDayTimeScalarColumn.class, IfExprIntervalDayTimeScalarScalar.class,
   IfExprTimestampColumnColumn.class, IfExprTimestampColumnScalar.class,
   IfExprTimestampScalarColumn.class, IfExprTimestampScalarScalar.class,
 })
+@VectorizedExpressionsSupportDecimal64()
 public class GenericUDFIf extends GenericUDF {
   private transient ObjectInspector[] argumentOIs;
   private transient GenericUDFUtils.ReturnObjectInspectorResolver returnOIResolver;

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPEqual.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPEqual.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPEqual.java
index 75bc478..6dcd3e4 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPEqual.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPEqual.java
@@ -21,9 +21,6 @@ package org.apache.hadoop.hive.ql.udf.generic;
 import org.apache.hadoop.hive.ql.exec.Description;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedExpressions;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedExpressionsSupportDecimal64;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.LongColEqualLongColumn;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.LongColEqualLongScalar;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.LongScalarEqualLongColumn;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.*;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils;
@@ -39,6 +36,11 @@ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils;
   LongScalarEqualLongColumn.class, LongScalarEqualDoubleColumn.class,
   DoubleScalarEqualLongColumn.class, DoubleScalarEqualDoubleColumn.class,
 
+  DecimalColEqualDecimalColumn.class, DecimalColEqualDecimalScalar.class,
+  DecimalScalarEqualDecimalColumn.class,
+  Decimal64ColEqualDecimal64Column.class, Decimal64ColEqualDecimal64Scalar.class,
+  Decimal64ScalarEqualDecimal64Column.class,
+
   StringGroupColEqualStringGroupColumn.class, FilterStringGroupColEqualStringGroupColumn.class,
   StringGroupColEqualStringScalar.class,
   StringGroupColEqualVarCharScalar.class, StringGroupColEqualCharScalar.class,

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPEqualOrGreaterThan.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPEqualOrGreaterThan.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPEqualOrGreaterThan.java
index 90594b3..4383b3d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPEqualOrGreaterThan.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPEqualOrGreaterThan.java
@@ -21,9 +21,6 @@ package org.apache.hadoop.hive.ql.udf.generic;
 import org.apache.hadoop.hive.ql.exec.Description;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedExpressions;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedExpressionsSupportDecimal64;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.LongColGreaterEqualLongColumn;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.LongColGreaterEqualLongScalar;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.LongScalarGreaterEqualLongColumn;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.*;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils;
@@ -40,6 +37,11 @@ import org.apache.hadoop.io.Text;
   LongScalarGreaterEqualLongColumn.class, LongScalarGreaterEqualDoubleColumn.class,
   DoubleScalarGreaterEqualLongColumn.class, DoubleScalarGreaterEqualDoubleColumn.class,
 
+  DecimalColGreaterEqualDecimalColumn.class, DecimalColGreaterEqualDecimalScalar.class,
+  DecimalScalarGreaterEqualDecimalColumn.class,
+  Decimal64ColGreaterEqualDecimal64Column.class, Decimal64ColGreaterEqualDecimal64Scalar.class,
+  Decimal64ScalarGreaterEqualDecimal64Column.class,
+
   StringGroupColGreaterEqualStringGroupColumn.class, FilterStringGroupColGreaterEqualStringGroupColumn.class,
   StringGroupColGreaterEqualStringScalar.class,
   StringGroupColGreaterEqualVarCharScalar.class, StringGroupColGreaterEqualCharScalar.class,

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPEqualOrLessThan.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPEqualOrLessThan.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPEqualOrLessThan.java
index 5176803..c24ae43 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPEqualOrLessThan.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPEqualOrLessThan.java
@@ -21,9 +21,6 @@ package org.apache.hadoop.hive.ql.udf.generic;
 import org.apache.hadoop.hive.ql.exec.Description;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedExpressions;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedExpressionsSupportDecimal64;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.LongColLessEqualLongColumn;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.LongColLessEqualLongScalar;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.LongScalarLessEqualLongColumn;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.*;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils;
@@ -40,6 +37,11 @@ import org.apache.hadoop.io.Text;
   LongScalarLessEqualLongColumn.class, LongScalarLessEqualDoubleColumn.class,
   DoubleScalarLessEqualLongColumn.class, DoubleScalarLessEqualDoubleColumn.class,
 
+  DecimalColLessEqualDecimalColumn.class, DecimalColLessEqualDecimalScalar.class,
+  DecimalScalarLessEqualDecimalColumn.class,
+  Decimal64ColLessEqualDecimal64Column.class, Decimal64ColLessEqualDecimal64Scalar.class,
+  Decimal64ScalarLessEqualDecimal64Column.class,
+
   StringGroupColLessEqualStringGroupColumn.class, FilterStringGroupColLessEqualStringGroupColumn.class,
   StringGroupColLessEqualStringScalar.class,
   StringGroupColLessEqualVarCharScalar.class, StringGroupColLessEqualCharScalar.class,

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPGreaterThan.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPGreaterThan.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPGreaterThan.java
index 938e64f..e86551d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPGreaterThan.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPGreaterThan.java
@@ -21,9 +21,6 @@ package org.apache.hadoop.hive.ql.udf.generic;
 import org.apache.hadoop.hive.ql.exec.Description;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedExpressions;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedExpressionsSupportDecimal64;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.LongColGreaterLongColumn;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.LongColGreaterLongScalar;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.LongScalarGreaterLongColumn;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.*;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils;
@@ -40,6 +37,11 @@ import org.apache.hadoop.io.Text;
   LongScalarGreaterLongColumn.class, LongScalarGreaterDoubleColumn.class,
   DoubleScalarGreaterLongColumn.class, DoubleScalarGreaterDoubleColumn.class,
 
+  DecimalColGreaterDecimalColumn.class, DecimalColGreaterDecimalScalar.class,
+  DecimalScalarGreaterDecimalColumn.class,
+  Decimal64ColGreaterDecimal64Column.class, Decimal64ColGreaterDecimal64Scalar.class,
+  Decimal64ScalarGreaterDecimal64Column.class,
+
   StringGroupColGreaterStringGroupColumn.class, FilterStringGroupColGreaterStringGroupColumn.class,
   StringGroupColGreaterStringScalar.class,
   StringGroupColGreaterVarCharScalar.class, StringGroupColGreaterCharScalar.class,

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPLessThan.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPLessThan.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPLessThan.java
index 734d394..709ac72 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPLessThan.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPLessThan.java
@@ -21,9 +21,6 @@ package org.apache.hadoop.hive.ql.udf.generic;
 import org.apache.hadoop.hive.ql.exec.Description;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedExpressions;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedExpressionsSupportDecimal64;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.LongColLessLongColumn;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.LongColLessLongScalar;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.LongScalarLessLongColumn;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.*;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils;
@@ -40,6 +37,11 @@ import org.apache.hadoop.io.Text;
     LongScalarLessLongColumn.class, LongScalarLessDoubleColumn.class,
     DoubleScalarLessLongColumn.class, DoubleScalarLessDoubleColumn.class,
 
+    DecimalColLessDecimalColumn.class, DecimalColLessDecimalScalar.class,
+    DecimalScalarLessDecimalColumn.class,
+    Decimal64ColLessDecimal64Column.class, Decimal64ColLessDecimal64Scalar.class,
+    Decimal64ScalarLessDecimal64Column.class,
+
     StringGroupColLessStringGroupColumn.class, FilterStringGroupColLessStringGroupColumn.class,
     StringGroupColLessStringScalar.class,
     StringGroupColLessVarCharScalar.class, StringGroupColLessCharScalar.class,

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPNotEqual.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPNotEqual.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPNotEqual.java
index 00f8b7d..849b21d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPNotEqual.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPNotEqual.java
@@ -21,9 +21,6 @@ package org.apache.hadoop.hive.ql.udf.generic;
 import org.apache.hadoop.hive.ql.exec.Description;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedExpressions;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedExpressionsSupportDecimal64;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.LongColNotEqualLongColumn;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.LongColNotEqualLongScalar;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.LongScalarNotEqualLongColumn;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.*;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils;
@@ -39,6 +36,11 @@ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils;
   LongScalarNotEqualLongColumn.class, LongScalarNotEqualDoubleColumn.class,
   DoubleScalarNotEqualLongColumn.class, DoubleScalarNotEqualDoubleColumn.class,
 
+  DecimalColNotEqualDecimalColumn.class, DecimalColNotEqualDecimalScalar.class,
+  DecimalScalarNotEqualDecimalColumn.class,
+  Decimal64ColNotEqualDecimal64Column.class, Decimal64ColNotEqualDecimal64Scalar.class,
+  Decimal64ScalarNotEqualDecimal64Column.class,
+
   StringGroupColNotEqualStringGroupColumn.class, FilterStringGroupColNotEqualStringGroupColumn.class,
   StringGroupColNotEqualStringScalar.class,
   StringGroupColNotEqualVarCharScalar.class, StringGroupColNotEqualCharScalar.class,

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFTimestamp.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFTimestamp.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFTimestamp.java
index 4d0e85d..ee869db 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFTimestamp.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFTimestamp.java
@@ -28,6 +28,7 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.CastDateToTimestamp;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.CastDecimalToTimestamp;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.CastDoubleToTimestamp;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.CastLongToTimestamp;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.CastStringToTimestamp;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hadoop.hive.ql.session.SessionState.LogHelper;
@@ -49,7 +50,7 @@ import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectIn
 @Description(name = "timestamp",
 value = "cast(date as timestamp) - Returns timestamp")
 @VectorizedExpressions({CastLongToTimestamp.class, CastDateToTimestamp.class,
-  CastDoubleToTimestamp.class, CastDecimalToTimestamp.class})
+  CastDoubleToTimestamp.class, CastDecimalToTimestamp.class, CastStringToTimestamp.class})
 public class GenericUDFTimestamp extends GenericUDF {
 
   private transient PrimitiveObjectInspector argumentOI;

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorizationContext.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorizationContext.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorizationContext.java
index 70a481d..2039ee0 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorizationContext.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorizationContext.java
@@ -56,8 +56,8 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.IsNotNull;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.IsNull;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.LongColModuloLongColumn;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.LongColumnInList;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.LongColEqualLongScalar;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.LongColGreaterLongScalar;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.LongColEqualLongScalar;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.LongColGreaterLongScalar;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.NotCol;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.RoundWithNumDigitsDoubleToDouble;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.SelectColumnIsFalse;

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/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 eb85823..a09daf3 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
@@ -42,6 +42,7 @@ import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatchCtx;
 import org.apache.hadoop.hive.ql.exec.vector.VectorRandomRowSource.GenerationSpec;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.exec.vector.udf.VectorUDFAdaptor;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc;
@@ -672,6 +673,16 @@ public class TestVectorArithmetic {
     VectorExpression vectorExpression = vectorizationContext.getVectorExpression(exprDesc);
     vectorExpression.transientInit();
 
+    if (arithmeticTestMode == ArithmeticTestMode.VECTOR_EXPRESSION &&
+        vectorExpression instanceof VectorUDFAdaptor) {
+      System.out.println(
+          "*NO NATIVE VECTOR EXPRESSION* typeInfo1 " + typeInfo1.toString() +
+          " typeInfo2 " + typeInfo2.toString() +
+          " arithmeticTestMode " + arithmeticTestMode +
+          " columnScalarMode " + columnScalarMode +
+          " vectorExpression " + vectorExpression.toString());
+    }
+
     String[] outputScratchTypeNames= vectorizationContext.getScratchColumnTypeNames();
     DataTypePhysicalVariation[] outputDataTypePhysicalVariations =
         vectorizationContext.getScratchDataTypePhysicalVariations();

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/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 2c82c4e..d4d8ef7 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
@@ -38,6 +38,7 @@ import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatchCtx;
 import org.apache.hadoop.hive.ql.exec.vector.VectorRandomRowSource.GenerationSpec;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.IdentityExpression;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.exec.vector.udf.VectorUDFAdaptor;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.metadata.VirtualColumn;
 import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc;
@@ -157,13 +158,6 @@ public class TestVectorCastStatement {
   }
 
   @Test
-  public void testBinary() throws Exception {
-    Random random = new Random(12882);
-
-    doIfTests(random, "binary");
-  }
-
-  @Test
   public void testDecimal() throws Exception {
     Random random = new Random(9300);
 
@@ -203,17 +197,15 @@ public class TestVectorCastStatement {
         continue;
       }
 
-      // BINARY conversions supported by GenericUDFDecimal, GenericUDFTimestamp.
-      if (primitiveCategory == PrimitiveCategory.BINARY) {
-        if (targetPrimitiveCategory == PrimitiveCategory.DECIMAL ||
-            targetPrimitiveCategory == PrimitiveCategory.TIMESTAMP) {
-          continue;
-        }
-      }
-
-      // DATE conversions supported by GenericUDFDecimal.
+      // DATE conversions NOT supported by integers, floating point, and GenericUDFDecimal.
       if (primitiveCategory == PrimitiveCategory.DATE) {
-        if (targetPrimitiveCategory == PrimitiveCategory.DECIMAL) {
+        if (targetPrimitiveCategory == PrimitiveCategory.BYTE ||
+            targetPrimitiveCategory == PrimitiveCategory.SHORT ||
+            targetPrimitiveCategory == PrimitiveCategory.INT ||
+            targetPrimitiveCategory == PrimitiveCategory.LONG ||
+            targetPrimitiveCategory == PrimitiveCategory.FLOAT ||
+            targetPrimitiveCategory == PrimitiveCategory.DOUBLE ||
+            targetPrimitiveCategory == PrimitiveCategory.DECIMAL) {
           continue;
         }
       }
@@ -364,9 +356,10 @@ public class TestVectorCastStatement {
                 " sourceTypeName " + typeName +
                 " targetTypeName " + targetTypeName +
                 " " + CastStmtTestMode.values()[v] +
-                " result is NULL " + (vectorResult == null ? "YES" : "NO") +
+                " result is NULL " + (vectorResult == null ? "YES" : "NO result " + vectorResult.toString()) +
                 " does not match row-mode expected result is NULL " +
-                (expectedResult == null ? "YES" : "NO"));
+                (expectedResult == null ? "YES" : "NO result " + expectedResult.toString()) +
+                " row values " + Arrays.toString(randomRows[i]));
           }
         } else {
 
@@ -387,7 +380,8 @@ public class TestVectorCastStatement {
                 " result " + vectorResult.toString() +
                 " (" + vectorResult.getClass().getSimpleName() + ")" +
                 " does not match row-mode expected result " + expectedResult.toString() +
-                " (" + expectedResult.getClass().getSimpleName() + ")");
+                " (" + expectedResult.getClass().getSimpleName() + ")" +
+                " row values " + Arrays.toString(randomRows[i]));
           }
         }
       }
@@ -490,6 +484,14 @@ public class TestVectorCastStatement {
     VectorExpression vectorExpression = vectorizationContext.getVectorExpression(exprDesc);
     vectorExpression.transientInit();
 
+    if (castStmtTestMode == CastStmtTestMode.VECTOR_EXPRESSION &&
+        vectorExpression instanceof VectorUDFAdaptor) {
+      System.out.println(
+          "*NO NATIVE VECTOR EXPRESSION* typeInfo " + typeInfo.toString() +
+          " castStmtTestMode " + castStmtTestMode +
+          " vectorExpression " + vectorExpression.toString());
+    }
+
     // System.out.println("*VECTOR EXPRESSION* " + vectorExpression.getClass().getSimpleName());
 
     /*

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/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 da21aba..4dc01be 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
@@ -38,6 +38,7 @@ import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatchCtx;
 import org.apache.hadoop.hive.ql.exec.vector.VectorRandomRowSource.GenerationSpec;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.exec.vector.udf.VectorUDFAdaptor;
 import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
@@ -448,6 +449,15 @@ public class TestVectorDateAddSub {
     VectorExpression vectorExpression = vectorizationContext.getVectorExpression(exprDesc);
     vectorExpression.transientInit();
 
+    if (dateAddSubTestMode == DateAddSubTestMode.VECTOR_EXPRESSION &&
+        vectorExpression instanceof VectorUDFAdaptor) {
+      System.out.println(
+          "*NO NATIVE VECTOR EXPRESSION* dateTimeStringTypeInfo " + dateTimeStringTypeInfo.toString() +
+          " dateAddSubTestMode " + dateAddSubTestMode +
+          " columnScalarMode " + columnScalarMode +
+          " vectorExpression " + vectorExpression.toString());
+    }
+
     VectorizedRowBatch batch = batchContext.createVectorizedRowBatch();
 
     VectorExtractRow resultVectorExtractRow = new VectorExtractRow();

http://git-wip-us.apache.org/repos/asf/hive/blob/71c49878/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 d5343ad..c5c5c72 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
@@ -38,6 +38,7 @@ import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatchCtx;
 import org.apache.hadoop.hive.ql.exec.vector.VectorRandomRowSource.GenerationSpec;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.exec.vector.udf.VectorUDFAdaptor;
 import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
@@ -434,6 +435,16 @@ public class TestVectorDateDiff {
     VectorExpression vectorExpression = vectorizationContext.getVectorExpression(exprDesc);
     vectorExpression.transientInit();
 
+    if (dateDiffTestMode == DateDiffTestMode.VECTOR_EXPRESSION &&
+        vectorExpression instanceof VectorUDFAdaptor) {
+      System.out.println(
+          "*NO NATIVE VECTOR EXPRESSION* dateTimeStringTypeInfo1 " + dateTimeStringTypeInfo1.toString() +
+          " dateTimeStringTypeInfo2 " + dateTimeStringTypeInfo2.toString() +
+          " dateDiffTestMode " + dateDiffTestMode +
+          " columnScalarMode " + columnScalarMode +
+          " vectorExpression " + vectorExpression.toString());
+    }
+
     VectorizedRowBatch batch = batchContext.createVectorizedRowBatch();
 
     VectorExtractRow resultVectorExtractRow = new VectorExtractRow();