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:20 UTC

[08/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/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();
-  }
-}