You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by se...@apache.org on 2018/07/25 18:19:19 UTC

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

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();