You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ch...@apache.org on 2015/10/13 11:44:11 UTC

[1/2] hive git commit: HIVE-11533 : Loop optimization for SIMD in integer comparisons. (Teddy Choi, reviewed by ChengXiang Li)

Repository: hive
Updated Branches:
  refs/heads/master 7524ad200 -> 2529e9bb8


http://git-wip-us.apache.org/repos/asf/hive/blob/2529e9bb/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
new file mode 100644
index 0000000..5352186
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColNotEqualLongScalar.java
@@ -0,0 +1,151 @@
+/**
+ * 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.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+
+public class LongColNotEqualLongScalar extends VectorExpression {
+
+  private static final long serialVersionUID = 1L;
+
+  private int colNum;
+  private long value;
+  private int outputColumn;
+
+  public LongColNotEqualLongScalar(int colNum, long value, int outputColumn) {
+    this.colNum = colNum;
+    this.value = value;
+    this.outputColumn = outputColumn;
+  }
+
+  public LongColNotEqualLongScalar() {
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    LongColumnVector inputColVector = (LongColumnVector) batch.cols[colNum];
+    LongColumnVector outputColVector = (LongColumnVector) batch.cols[outputColumn];
+    int[] sel = batch.selected;
+    boolean[] nullPos = inputColVector.isNull;
+    boolean[] outNulls = outputColVector.isNull;
+    int n = batch.size;
+    long[] vector = inputColVector.vector;
+    long[] outputVector = outputColVector.vector;
+
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+
+    outputColVector.isRepeating = false;
+    outputColVector.noNulls = inputColVector.noNulls;
+    if (inputColVector.noNulls) {
+      if (inputColVector.isRepeating) {
+        //All must be selected otherwise size would be zero
+        //Repeating property will not change.
+        outputVector[0] = vector[0] != value ? 1 : 0;
+        outputColVector.isRepeating = true;
+      } else if (batch.selectedInUse) {
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = vector[i] != value ? 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] = ((vector[i] - value) ^ (value - vector[i])) >>> 63;
+        }
+      }
+    } else {
+      if (inputColVector.isRepeating) {
+        //All must be selected otherwise size would be zero
+        //Repeating property will not change.
+        if (!nullPos[0]) {
+          outputVector[0] = vector[0] != value ? 1 : 0;
+          outNulls[0] = false;
+        } else {
+          outNulls[0] = true;
+        }
+        outputColVector.isRepeating = true;
+      } else if (batch.selectedInUse) {
+        System.arraycopy(nullPos, 0, outNulls, 0, n);
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = vector[i] != value ? 1 : 0;
+        }
+      } else {
+        System.arraycopy(nullPos, 0, outNulls, 0, n);
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = ((vector[i] - value) ^ (value - vector[i])) >>> 63;
+        }
+      }
+    }
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return outputColumn;
+  }
+
+  @Override
+  public String getOutputType() {
+    return "long";
+  }
+
+  public int getColNum() {
+    return colNum;
+  }
+
+  public void setColNum(int colNum) {
+    this.colNum = colNum;
+  }
+
+  public long getValue() {
+    return value;
+  }
+
+  public void setValue(long value) {
+    this.value = value;
+  }
+
+  public void setOutputColumn(int outputColumn) {
+    this.outputColumn = outputColumn;
+  }
+
+  @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/2529e9bb/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
new file mode 100644
index 0000000..98fd0e8
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongScalarEqualLongColumn.java
@@ -0,0 +1,151 @@
+/**
+ * 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.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+
+public class LongScalarEqualLongColumn extends VectorExpression {
+
+  private static final long serialVersionUID = 1L;
+
+  private int colNum;
+  private long value;
+  private int outputColumn;
+
+  public LongScalarEqualLongColumn(long value, int colNum, int outputColumn) {
+    this.colNum = colNum;
+    this.value = value;
+    this.outputColumn = outputColumn;
+  }
+
+  public LongScalarEqualLongColumn() {
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    LongColumnVector inputColVector = (LongColumnVector) batch.cols[colNum];
+    LongColumnVector outputColVector = (LongColumnVector) batch.cols[outputColumn];
+    int[] sel = batch.selected;
+    boolean[] nullPos = inputColVector.isNull;
+    boolean[] outNulls = outputColVector.isNull;
+    int n = batch.size;
+    long[] vector = inputColVector.vector;
+    long[] outputVector = outputColVector.vector;
+
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+
+    outputColVector.isRepeating = false;
+    outputColVector.noNulls = inputColVector.noNulls;
+    if (inputColVector.noNulls) {
+      if (inputColVector.isRepeating) {
+        //All must be selected otherwise size would be zero
+        //Repeating property will not change.
+        outputVector[0] = value == vector[0] ? 1 : 0;
+        outputColVector.isRepeating = true;
+      } else if (batch.selectedInUse) {
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = value == vector[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] = (((value - vector[i]) ^ (vector[i] - value)) >>> 63) ^ 1;
+        }
+      }
+    } else {
+      if (inputColVector.isRepeating) {
+        //All must be selected otherwise size would be zero
+        //Repeating property will not change.
+        if (!nullPos[0]) {
+          outputVector[0] = value == vector[0] ? 1 : 0;
+          outNulls[0] = false;
+        } else {
+          outNulls[0] = true;
+        }
+        outputColVector.isRepeating = true;
+      } else if (batch.selectedInUse) {
+        System.arraycopy(nullPos, 0, outNulls, 0, n);
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = value == vector[i] ? 1 : 0;
+        }
+      } else {
+        System.arraycopy(nullPos, 0, outNulls, 0, n);
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = (((value - vector[i]) ^ (vector[i] - value)) >>> 63) ^ 1;
+        }
+      }
+    }
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return outputColumn;
+  }
+
+  @Override
+  public String getOutputType() {
+    return "long";
+  }
+
+  public int getColNum() {
+    return colNum;
+  }
+
+  public void setColNum(int colNum) {
+    this.colNum = colNum;
+  }
+
+  public long getValue() {
+    return value;
+  }
+
+  public void setValue(long value) {
+    this.value = value;
+  }
+
+  public void setOutputColumn(int outputColumn) {
+    this.outputColumn = outputColumn;
+  }
+
+  @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/2529e9bb/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
new file mode 100644
index 0000000..12a1c7f
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongScalarGreaterEqualLongColumn.java
@@ -0,0 +1,151 @@
+/**
+ * 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.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+
+public class LongScalarGreaterEqualLongColumn extends VectorExpression {
+
+  private static final long serialVersionUID = 1L;
+
+  private int colNum;
+  private long value;
+  private int outputColumn;
+
+  public LongScalarGreaterEqualLongColumn(long value, int colNum, int outputColumn) {
+    this.colNum = colNum;
+    this.value = value;
+    this.outputColumn = outputColumn;
+  }
+
+  public LongScalarGreaterEqualLongColumn() {
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    LongColumnVector inputColVector = (LongColumnVector) batch.cols[colNum];
+    LongColumnVector outputColVector = (LongColumnVector) batch.cols[outputColumn];
+    int[] sel = batch.selected;
+    boolean[] nullPos = inputColVector.isNull;
+    boolean[] outNulls = outputColVector.isNull;
+    int n = batch.size;
+    long[] vector = inputColVector.vector;
+    long[] outputVector = outputColVector.vector;
+
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+
+    outputColVector.isRepeating = false;
+    outputColVector.noNulls = inputColVector.noNulls;
+    if (inputColVector.noNulls) {
+      if (inputColVector.isRepeating) {
+        //All must be selected otherwise size would be zero
+        //Repeating property will not change.
+        outputVector[0] = value >= vector[0] ? 1 : 0;
+        outputColVector.isRepeating = true;
+      } else if (batch.selectedInUse) {
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = value >= vector[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] = ((value - vector[i]) >>> 63) ^ 1;
+        }
+      }
+    } else {
+      if (inputColVector.isRepeating) {
+        //All must be selected otherwise size would be zero
+        //Repeating property will not change.
+        if (!nullPos[0]) {
+          outputVector[0] = value >= vector[0] ? 1 : 0;
+          outNulls[0] = false;
+        } else {
+          outNulls[0] = true;
+        }
+        outputColVector.isRepeating = true;
+      } else if (batch.selectedInUse) {
+        System.arraycopy(nullPos, 0, outNulls, 0, n);
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = value >= vector[i] ? 1 : 0;
+        }
+      } else {
+        System.arraycopy(nullPos, 0, outNulls, 0, n);
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = ((value - vector[i]) >>> 63) ^ 1;
+        }
+      }
+    }
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return outputColumn;
+  }
+
+  @Override
+  public String getOutputType() {
+    return "long";
+  }
+
+  public int getColNum() {
+    return colNum;
+  }
+
+  public void setColNum(int colNum) {
+    this.colNum = colNum;
+  }
+
+  public long getValue() {
+    return value;
+  }
+
+  public void setValue(long value) {
+    this.value = value;
+  }
+
+  public void setOutputColumn(int outputColumn) {
+    this.outputColumn = outputColumn;
+  }
+
+  @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/2529e9bb/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
new file mode 100644
index 0000000..1fe2ec3
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongScalarGreaterLongColumn.java
@@ -0,0 +1,151 @@
+/**
+ * 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.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+
+public class LongScalarGreaterLongColumn extends VectorExpression {
+
+  private static final long serialVersionUID = 1L;
+
+  private int colNum;
+  private long value;
+  private int outputColumn;
+
+  public LongScalarGreaterLongColumn(long value, int colNum, int outputColumn) {
+    this.colNum = colNum;
+    this.value = value;
+    this.outputColumn = outputColumn;
+  }
+
+  public LongScalarGreaterLongColumn() {
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    LongColumnVector inputColVector = (LongColumnVector) batch.cols[colNum];
+    LongColumnVector outputColVector = (LongColumnVector) batch.cols[outputColumn];
+    int[] sel = batch.selected;
+    boolean[] nullPos = inputColVector.isNull;
+    boolean[] outNulls = outputColVector.isNull;
+    int n = batch.size;
+    long[] vector = inputColVector.vector;
+    long[] outputVector = outputColVector.vector;
+
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+
+    outputColVector.isRepeating = false;
+    outputColVector.noNulls = inputColVector.noNulls;
+    if (inputColVector.noNulls) {
+      if (inputColVector.isRepeating) {
+        //All must be selected otherwise size would be zero
+        //Repeating property will not change.
+        outputVector[0] = value > vector[0] ? 1 : 0;
+        outputColVector.isRepeating = true;
+      } else if (batch.selectedInUse) {
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = value > vector[i] ? 1 : 0;
+        }
+      } else {
+        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 {
+      if (inputColVector.isRepeating) {
+        //All must be selected otherwise size would be zero
+        //Repeating property will not change.
+        if (!nullPos[0]) {
+          outputVector[0] = value > vector[0] ? 1 : 0;
+          outNulls[0] = false;
+        } else {
+          outNulls[0] = true;
+        }
+        outputColVector.isRepeating = true;
+      } else if (batch.selectedInUse) {
+        System.arraycopy(nullPos, 0, outNulls, 0, n);
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = value > vector[i] ? 1 : 0;
+        }
+      } else {
+        System.arraycopy(nullPos, 0, outNulls, 0, n);
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = (vector[i] - value) >>> 63;
+        }
+      }
+    }
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return outputColumn;
+  }
+
+  @Override
+  public String getOutputType() {
+    return "long";
+  }
+
+  public int getColNum() {
+    return colNum;
+  }
+
+  public void setColNum(int colNum) {
+    this.colNum = colNum;
+  }
+
+  public long getValue() {
+    return value;
+  }
+
+  public void setValue(long value) {
+    this.value = value;
+  }
+
+  public void setOutputColumn(int outputColumn) {
+    this.outputColumn = outputColumn;
+  }
+
+  @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/2529e9bb/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
new file mode 100644
index 0000000..af3a4ab
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongScalarLessEqualLongColumn.java
@@ -0,0 +1,151 @@
+/**
+ * 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.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+
+public class LongScalarLessEqualLongColumn extends VectorExpression {
+
+  private static final long serialVersionUID = 1L;
+
+  private int colNum;
+  private long value;
+  private int outputColumn;
+
+  public LongScalarLessEqualLongColumn(long value, int colNum, int outputColumn) {
+    this.colNum = colNum;
+    this.value = value;
+    this.outputColumn = outputColumn;
+  }
+
+  public LongScalarLessEqualLongColumn() {
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    LongColumnVector inputColVector = (LongColumnVector) batch.cols[colNum];
+    LongColumnVector outputColVector = (LongColumnVector) batch.cols[outputColumn];
+    int[] sel = batch.selected;
+    boolean[] nullPos = inputColVector.isNull;
+    boolean[] outNulls = outputColVector.isNull;
+    int n = batch.size;
+    long[] vector = inputColVector.vector;
+    long[] outputVector = outputColVector.vector;
+
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+
+    outputColVector.isRepeating = false;
+    outputColVector.noNulls = inputColVector.noNulls;
+    if (inputColVector.noNulls) {
+      if (inputColVector.isRepeating) {
+        //All must be selected otherwise size would be zero
+        //Repeating property will not change.
+        outputVector[0] = value <= vector[0] ? 1 : 0;
+        outputColVector.isRepeating = true;
+      } else if (batch.selectedInUse) {
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = value <= vector[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] = ((vector[i] - value) >>> 63) ^ 1;
+        }
+      }
+    } else {
+      if (inputColVector.isRepeating) {
+        //All must be selected otherwise size would be zero
+        //Repeating property will not change.
+        if (!nullPos[0]) {
+          outputVector[0] = value <= vector[0] ? 1 : 0;
+          outNulls[0] = false;
+        } else {
+          outNulls[0] = true;
+        }
+        outputColVector.isRepeating = true;
+      } else if (batch.selectedInUse) {
+        System.arraycopy(nullPos, 0, outNulls, 0, n);
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = value <= vector[i] ? 1 : 0;
+        }
+      } else {
+        System.arraycopy(nullPos, 0, outNulls, 0, n);
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = ((vector[i] - value) >>> 63) ^ 1;
+        }
+      }
+    }
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return outputColumn;
+  }
+
+  @Override
+  public String getOutputType() {
+    return "long";
+  }
+
+  public int getColNum() {
+    return colNum;
+  }
+
+  public void setColNum(int colNum) {
+    this.colNum = colNum;
+  }
+
+  public long getValue() {
+    return value;
+  }
+
+  public void setValue(long value) {
+    this.value = value;
+  }
+
+  public void setOutputColumn(int outputColumn) {
+    this.outputColumn = outputColumn;
+  }
+
+  @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/2529e9bb/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
new file mode 100644
index 0000000..0963367
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongScalarLessLongColumn.java
@@ -0,0 +1,151 @@
+/**
+ * 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.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+
+public class LongScalarLessLongColumn extends VectorExpression {
+
+  private static final long serialVersionUID = 1L;
+
+  private int colNum;
+  private long value;
+  private int outputColumn;
+
+  public LongScalarLessLongColumn(long value, int colNum, int outputColumn) {
+    this.colNum = colNum;
+    this.value = value;
+    this.outputColumn = outputColumn;
+  }
+
+  public LongScalarLessLongColumn() {
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    LongColumnVector inputColVector = (LongColumnVector) batch.cols[colNum];
+    LongColumnVector outputColVector = (LongColumnVector) batch.cols[outputColumn];
+    int[] sel = batch.selected;
+    boolean[] nullPos = inputColVector.isNull;
+    boolean[] outNulls = outputColVector.isNull;
+    int n = batch.size;
+    long[] vector = inputColVector.vector;
+    long[] outputVector = outputColVector.vector;
+
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+
+    outputColVector.isRepeating = false;
+    outputColVector.noNulls = inputColVector.noNulls;
+    if (inputColVector.noNulls) {
+      if (inputColVector.isRepeating) {
+        //All must be selected otherwise size would be zero
+        //Repeating property will not change.
+        outputVector[0] = value < vector[0] ? 1 : 0;
+        outputColVector.isRepeating = true;
+      } else if (batch.selectedInUse) {
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = value < vector[i] ? 1 : 0;
+        }
+      } else {
+        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 {
+      if (inputColVector.isRepeating) {
+        //All must be selected otherwise size would be zero
+        //Repeating property will not change.
+        if (!nullPos[0]) {
+          outputVector[0] = value < vector[0] ? 1 : 0;
+          outNulls[0] = false;
+        } else {
+          outNulls[0] = true;
+        }
+        outputColVector.isRepeating = true;
+      } else if (batch.selectedInUse) {
+        System.arraycopy(nullPos, 0, outNulls, 0, n);
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = value < vector[i] ? 1 : 0;
+        }
+      } else {
+        System.arraycopy(nullPos, 0, outNulls, 0, n);
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = (value - vector[i]) >>> 63;
+        }
+      }
+    }
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return outputColumn;
+  }
+
+  @Override
+  public String getOutputType() {
+    return "long";
+  }
+
+  public int getColNum() {
+    return colNum;
+  }
+
+  public void setColNum(int colNum) {
+    this.colNum = colNum;
+  }
+
+  public long getValue() {
+    return value;
+  }
+
+  public void setValue(long value) {
+    this.value = value;
+  }
+
+  public void setOutputColumn(int outputColumn) {
+    this.outputColumn = outputColumn;
+  }
+
+  @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/2529e9bb/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
new file mode 100644
index 0000000..33ad3c2
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongScalarNotEqualLongColumn.java
@@ -0,0 +1,151 @@
+/**
+ * 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.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+
+public class LongScalarNotEqualLongColumn extends VectorExpression {
+
+  private static final long serialVersionUID = 1L;
+
+  private int colNum;
+  private long value;
+  private int outputColumn;
+
+  public LongScalarNotEqualLongColumn(long value, int colNum, int outputColumn) {
+    this.colNum = colNum;
+    this.value = value;
+    this.outputColumn = outputColumn;
+  }
+
+  public LongScalarNotEqualLongColumn() {
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    LongColumnVector inputColVector = (LongColumnVector) batch.cols[colNum];
+    LongColumnVector outputColVector = (LongColumnVector) batch.cols[outputColumn];
+    int[] sel = batch.selected;
+    boolean[] nullPos = inputColVector.isNull;
+    boolean[] outNulls = outputColVector.isNull;
+    int n = batch.size;
+    long[] vector = inputColVector.vector;
+    long[] outputVector = outputColVector.vector;
+
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+
+    outputColVector.isRepeating = false;
+    outputColVector.noNulls = inputColVector.noNulls;
+    if (inputColVector.noNulls) {
+      if (inputColVector.isRepeating) {
+        //All must be selected otherwise size would be zero
+        //Repeating property will not change.
+        outputVector[0] = value != vector[0] ? 1 : 0;
+        outputColVector.isRepeating = true;
+      } else if (batch.selectedInUse) {
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = value != vector[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] = ((vector[i] - value) ^ (value - vector[i])) >>> 63;
+        }
+      }
+    } else {
+      if (inputColVector.isRepeating) {
+        //All must be selected otherwise size would be zero
+        //Repeating property will not change.
+        if (!nullPos[0]) {
+          outputVector[0] = value != vector[0] ? 1 : 0;
+          outNulls[0] = false;
+        } else {
+          outNulls[0] = true;
+        }
+        outputColVector.isRepeating = true;
+      } else if (batch.selectedInUse) {
+        System.arraycopy(nullPos, 0, outNulls, 0, n);
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = value != vector[i] ? 1 : 0;
+        }
+      } else {
+        System.arraycopy(nullPos, 0, outNulls, 0, n);
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = ((vector[i] - value) ^ (value - vector[i])) >>> 63;
+        }
+      }
+    }
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return outputColumn;
+  }
+
+  @Override
+  public String getOutputType() {
+    return "long";
+  }
+
+  public int getColNum() {
+    return colNum;
+  }
+
+  public void setColNum(int colNum) {
+    this.colNum = colNum;
+  }
+
+  public long getValue() {
+    return value;
+  }
+
+  public void setValue(long value) {
+    this.value = value;
+  }
+
+  public void setOutputColumn(int outputColumn) {
+    this.outputColumn = outputColumn;
+  }
+
+  @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/2529e9bb/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 23f7f1f..27537bf 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
@@ -20,6 +20,9 @@ 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.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;

http://git-wip-us.apache.org/repos/asf/hive/blob/2529e9bb/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 bd5b61a..90d98bb 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
@@ -20,6 +20,9 @@ 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.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;

http://git-wip-us.apache.org/repos/asf/hive/blob/2529e9bb/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 ad971cd..35133d4 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
@@ -20,6 +20,9 @@ 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.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;

http://git-wip-us.apache.org/repos/asf/hive/blob/2529e9bb/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 85f0601..be05b4e 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
@@ -20,6 +20,9 @@ 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.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;

http://git-wip-us.apache.org/repos/asf/hive/blob/2529e9bb/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 b445ae6..9d72f9e 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
@@ -20,6 +20,9 @@ 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.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;

http://git-wip-us.apache.org/repos/asf/hive/blob/2529e9bb/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 ac8ec2f..7023225 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
@@ -20,6 +20,9 @@ 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.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;

http://git-wip-us.apache.org/repos/asf/hive/blob/2529e9bb/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 704c654..2fed9a7 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
@@ -54,6 +54,8 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.IfExprVarCharScalarStri
 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.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.NotCol;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.RoundWithNumDigitsDoubleToDouble;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.SelectColumnIsFalse;
@@ -100,8 +102,6 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.FuncLnDoubleToDoubl
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.FuncRoundDoubleToDouble;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.FuncSinDoubleToDouble;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.LongColAddLongColumn;
-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.gen.LongColModuloLongColumn;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.LongColMultiplyLongColumn;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.LongColSubtractLongColumn;
@@ -1491,4 +1491,96 @@ public class TestVectorizationContext {
     ve = vc.getVectorExpression(exprDesc);
     assertTrue(ve instanceof IfExprVarCharScalarStringGroupColumn);
   }
-}
+
+  @Test
+  public void testSIMDEqual() {
+    long a;
+    long b;
+
+    a = 0; b = 0; assertEquals(a == b ? 1 : 0, (((a - b) ^ (b - a)) >>> 63) ^ 1);
+    a = 1; b = 0; assertEquals(a == b ? 1 : 0, (((a - b) ^ (b - a)) >>> 63) ^ 1);
+    a = 0; b = 1; assertEquals(a == b ? 1 : 0, (((a - b) ^ (b - a)) >>> 63) ^ 1);
+  }
+
+  @Test
+  public void testSIMDGreaterThan() {
+    long a;
+    long b;
+
+    a = 0; b = 0; assertEquals(a > b ? 1 : 0, (b - a) >>> 63);
+    a = 1; b = 0; assertEquals(a > b ? 1 : 0, (b - a) >>> 63);
+    a = 0; b = 1; assertEquals(a > b ? 1 : 0, (b - a) >>> 63);
+  }
+
+  @Test
+  public void testSIMDGreaterEqual() {
+    long a;
+    long b;
+
+    a = 0;
+    b = 0;
+    assertEquals(a >= b ? 1 : 0, ((a - b) >>> 63) ^ 1);
+
+    a = 1;
+    b = 0;
+    assertEquals(a >= b ? 1 : 0, ((a - b) >>> 63) ^ 1);
+
+    a = 0;
+    b = 1;
+    assertEquals(a >= b ? 1 : 0, ((a - b) >>> 63) ^ 1);
+  }
+
+  @Test
+  public void testSIMDLessEqual() {
+    long a;
+    long b;
+
+    a = 0;
+    b = 0;
+    assertEquals(a <= b ? 1 : 0, ((b - a) >>> 63) ^ 1);
+
+    a = 1;
+    b = 0;
+    assertEquals(a <= b ? 1 : 0, ((b - a) >>> 63) ^ 1);
+
+    a = 0;
+    b = 1;
+    assertEquals(a <= b ? 1 : 0, ((b - a) >>> 63) ^ 1);
+  }
+
+  @Test
+  public void testSIMDLessThan() {
+    long a;
+    long b;
+
+    a = 0;
+    b = 0;
+    assertEquals(a < b ? 1 : 0, (a - b) >>> 63);
+
+    a = 1;
+    b = 0;
+    assertEquals(a < b ? 1 : 0, (a - b) >>> 63);
+
+    a = 0;
+    b = 1;
+    assertEquals(a < b ? 1 : 0, (a - b) >>> 63);
+  }
+
+  @Test
+  public void testSIMDNotEqual() {
+    long a;
+    long b;
+
+    a = 0;
+    b = 0;
+    assertEquals(a != b ? 1 : 0, ((a - b) ^ (b - a)) >>> 63);
+
+    a = 1;
+    b = 0;
+    assertEquals(a != b ? 1 : 0, ((a - b) ^ (b - a)) >>> 63);
+
+    a = 0;
+    b = 1;
+    assertEquals(a != b ? 1 : 0, ((a - b) ^ (b - a)) >>> 63);
+  }
+}
\ No newline at end of file


[2/2] hive git commit: HIVE-11533 : Loop optimization for SIMD in integer comparisons. (Teddy Choi, reviewed by ChengXiang Li)

Posted by ch...@apache.org.
HIVE-11533 : Loop optimization for SIMD in integer comparisons. (Teddy Choi, reviewed by ChengXiang Li)


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

Branch: refs/heads/master
Commit: 2529e9bb838af2805dcd2e44ae65e56d32bb12ef
Parents: 7524ad2
Author: chengxiang <ch...@apache.com>
Authored: Tue Oct 13 17:49:26 2015 +0800
Committer: chengxiang <ch...@apache.com>
Committed: Tue Oct 13 17:49:26 2015 +0800

----------------------------------------------------------------------
 .../apache/hadoop/hive/ant/GenVectorCode.java   |  30 +---
 .../vectorization/VectorizationBench.java       | 166 ++++++++++++++++++
 .../expressions/LongColEqualLongColumn.java     | 169 +++++++++++++++++++
 .../expressions/LongColEqualLongScalar.java     | 151 +++++++++++++++++
 .../LongColGreaterEqualLongColumn.java          | 169 +++++++++++++++++++
 .../LongColGreaterEqualLongScalar.java          | 151 +++++++++++++++++
 .../expressions/LongColGreaterLongColumn.java   | 169 +++++++++++++++++++
 .../expressions/LongColGreaterLongScalar.java   | 151 +++++++++++++++++
 .../expressions/LongColLessEqualLongColumn.java | 169 +++++++++++++++++++
 .../expressions/LongColLessEqualLongScalar.java | 151 +++++++++++++++++
 .../expressions/LongColLessLongColumn.java      | 169 +++++++++++++++++++
 .../expressions/LongColLessLongScalar.java      | 151 +++++++++++++++++
 .../expressions/LongColNotEqualLongColumn.java  | 169 +++++++++++++++++++
 .../expressions/LongColNotEqualLongScalar.java  | 151 +++++++++++++++++
 .../expressions/LongScalarEqualLongColumn.java  | 151 +++++++++++++++++
 .../LongScalarGreaterEqualLongColumn.java       | 151 +++++++++++++++++
 .../LongScalarGreaterLongColumn.java            | 151 +++++++++++++++++
 .../LongScalarLessEqualLongColumn.java          | 151 +++++++++++++++++
 .../expressions/LongScalarLessLongColumn.java   | 151 +++++++++++++++++
 .../LongScalarNotEqualLongColumn.java           | 151 +++++++++++++++++
 .../hive/ql/udf/generic/GenericUDFOPEqual.java  |   3 +
 .../generic/GenericUDFOPEqualOrGreaterThan.java |   3 +
 .../generic/GenericUDFOPEqualOrLessThan.java    |   3 +
 .../ql/udf/generic/GenericUDFOPGreaterThan.java |   3 +
 .../ql/udf/generic/GenericUDFOPLessThan.java    |   3 +
 .../ql/udf/generic/GenericUDFOPNotEqual.java    |   3 +
 .../exec/vector/TestVectorizationContext.java   |  98 ++++++++++-
 27 files changed, 3111 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/2529e9bb/ant/src/org/apache/hadoop/hive/ant/GenVectorCode.java
----------------------------------------------------------------------
diff --git a/ant/src/org/apache/hadoop/hive/ant/GenVectorCode.java b/ant/src/org/apache/hadoop/hive/ant/GenVectorCode.java
index ba7648c..6c57da2 100644
--- a/ant/src/org/apache/hadoop/hive/ant/GenVectorCode.java
+++ b/ant/src/org/apache/hadoop/hive/ant/GenVectorCode.java
@@ -224,17 +224,11 @@ public class GenVectorCode extends Task {
       {"ColumnCompareScalar", "GreaterEqual", "long", "double", ">="},
       {"ColumnCompareScalar", "GreaterEqual", "double", "double", ">="},
 
-      {"ColumnCompareScalar", "Equal", "long", "long", "=="},
       {"ColumnCompareScalar", "Equal", "double", "long", "=="},
-      {"ColumnCompareScalar", "NotEqual", "long", "long", "!="},
       {"ColumnCompareScalar", "NotEqual", "double", "long", "!="},
-      {"ColumnCompareScalar", "Less", "long", "long", "<"},
       {"ColumnCompareScalar", "Less", "double", "long", "<"},
-      {"ColumnCompareScalar", "LessEqual", "long", "long", "<="},
       {"ColumnCompareScalar", "LessEqual", "double", "long", "<="},
-      {"ColumnCompareScalar", "Greater", "long", "long", ">"},
       {"ColumnCompareScalar", "Greater", "double", "long", ">"},
-      {"ColumnCompareScalar", "GreaterEqual", "long", "long", ">="},
       {"ColumnCompareScalar", "GreaterEqual", "double", "long", ">="},
 
       {"ScalarCompareColumn", "Equal", "long", "double", "=="},
@@ -250,17 +244,11 @@ public class GenVectorCode extends Task {
       {"ScalarCompareColumn", "GreaterEqual", "long", "double", ">="},
       {"ScalarCompareColumn", "GreaterEqual", "double", "double", ">="},
 
-      {"ScalarCompareColumn", "Equal", "long", "long", "=="},
       {"ScalarCompareColumn", "Equal", "double", "long", "=="},
-      {"ScalarCompareColumn", "NotEqual", "long", "long", "!="},
       {"ScalarCompareColumn", "NotEqual", "double", "long", "!="},
-      {"ScalarCompareColumn", "Less", "long", "long", "<"},
       {"ScalarCompareColumn", "Less", "double", "long", "<"},
-      {"ScalarCompareColumn", "LessEqual", "long", "long", "<="},
       {"ScalarCompareColumn", "LessEqual", "double", "long", "<="},
-      {"ScalarCompareColumn", "Greater", "long", "long", ">"},
       {"ScalarCompareColumn", "Greater", "double", "long", ">"},
-      {"ScalarCompareColumn", "GreaterEqual", "long", "long", ">="},
       {"ScalarCompareColumn", "GreaterEqual", "double", "long", ">="},
 
       {"TimestampColumnCompareTimestampScalar", "Equal"},
@@ -598,17 +586,11 @@ public class GenVectorCode extends Task {
       {"ColumnCompareColumn", "GreaterEqual", "long", "double", ">="},
       {"ColumnCompareColumn", "GreaterEqual", "double", "double", ">="},
 
-      {"ColumnCompareColumn", "Equal", "long", "long", "=="},
       {"ColumnCompareColumn", "Equal", "double", "long", "=="},
-      {"ColumnCompareColumn", "NotEqual", "long", "long", "!="},
       {"ColumnCompareColumn", "NotEqual", "double", "long", "!="},
-      {"ColumnCompareColumn", "Less", "long", "long", "<"},
       {"ColumnCompareColumn", "Less", "double", "long", "<"},
-      {"ColumnCompareColumn", "LessEqual", "long", "long", "<="},
       {"ColumnCompareColumn", "LessEqual", "double", "long", "<="},
-      {"ColumnCompareColumn", "Greater", "long", "long", ">"},
       {"ColumnCompareColumn", "Greater", "double", "long", ">"},
-      {"ColumnCompareColumn", "GreaterEqual", "long", "long", ">="},
       {"ColumnCompareColumn", "GreaterEqual", "double", "long", ">="},
 
       // Interval comparisons
@@ -1891,7 +1873,7 @@ public class GenVectorCode extends Task {
   private void generateTimestampScalarCompareTimestampColumn(String[] tdesc) throws Exception {
     String operatorName = tdesc[1];
     String className = "TimestampScalar" + operatorName + "TimestampColumn";
-    String baseClassName = "LongScalar" + operatorName + "LongColumn";
+    String baseClassName = "org.apache.hadoop.hive.ql.exec.vector.expressions.LongScalar" + operatorName + "LongColumn";
     //Read the template into a string;
     File templateFile = new File(joinPath(this.expressionTemplateDirectory, tdesc[0] + ".txt"));
     String templateString = readFile(templateFile);
@@ -1904,7 +1886,7 @@ public class GenVectorCode extends Task {
   private void generateTimestampColumnCompareTimestampScalar(String[] tdesc) throws Exception {
     String operatorName = tdesc[1];
     String className = "TimestampCol" + operatorName + "TimestampScalar";
-    String baseClassName = "LongCol" + operatorName + "LongScalar";
+    String baseClassName = "org.apache.hadoop.hive.ql.exec.vector.expressions.LongCol" + operatorName + "LongScalar";
     //Read the template into a string;
     File templateFile = new File(joinPath(this.expressionTemplateDirectory, tdesc[0] + ".txt"));
     String templateString = readFile(templateFile);
@@ -1954,7 +1936,7 @@ public class GenVectorCode extends Task {
     String operatorName = tdesc[1];
     String operandType = tdesc[2];
     String className = getCamelCaseType(operandType) + "Scalar" + operatorName + "TimestampColumn";
-    String baseClassName = "LongScalar" + operatorName + "LongColumn";
+    String baseClassName = "org.apache.hadoop.hive.ql.exec.vector.expressions.LongScalar" + operatorName + "LongColumn";
     //Read the template into a string;
     File templateFile = new File(joinPath(this.expressionTemplateDirectory, tdesc[0] + ".txt"));
     String templateString = readFile(templateFile);
@@ -1970,7 +1952,7 @@ public class GenVectorCode extends Task {
     String operatorName = tdesc[1];
     String operandType = tdesc[2];
     String className = "TimestampCol" + operatorName + getCamelCaseType(operandType) + "Scalar";
-    String baseClassName = "LongCol" + operatorName + "LongScalar";
+    String baseClassName = "org.apache.hadoop.hive.ql.exec.vector.expressions.LongCol" + operatorName + "LongScalar";
     //Read the template into a string;
     File templateFile = new File(joinPath(this.expressionTemplateDirectory, tdesc[0] + ".txt"));
     String templateString = readFile(templateFile);
@@ -2354,7 +2336,7 @@ public class GenVectorCode extends Task {
     String operandType = tdesc[2];
     String className = getCamelCaseType(operandType) + "Scalar" + operatorName
         + getCamelCaseType(operandType) + "Column";
-    String baseClassName = "LongScalar" + operatorName + "LongColumn";
+    String baseClassName = "org.apache.hadoop.hive.ql.exec.vector.expressions.LongScalar" + operatorName + "LongColumn";
     //Read the template into a string;
     File templateFile = new File(joinPath(this.expressionTemplateDirectory, tdesc[0] + ".txt"));
     String templateString = readFile(templateFile);
@@ -2386,7 +2368,7 @@ public class GenVectorCode extends Task {
     String operandType = tdesc[2];
     String className = getCamelCaseType(operandType) + "Col" + operatorName
         + getCamelCaseType(operandType) + "Scalar";
-    String baseClassName = "LongCol" + operatorName + "LongScalar";
+    String baseClassName = "org.apache.hadoop.hive.ql.exec.vector.expressions.LongCol" + operatorName + "LongScalar";
     //Read the template into a string;
     File templateFile = new File(joinPath(this.expressionTemplateDirectory, tdesc[0] + ".txt"));
     String templateString = readFile(templateFile);

http://git-wip-us.apache.org/repos/asf/hive/blob/2529e9bb/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/vectorization/VectorizationBench.java
----------------------------------------------------------------------
diff --git a/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/vectorization/VectorizationBench.java b/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/vectorization/VectorizationBench.java
index dcd9501..642c5e1 100644
--- a/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/vectorization/VectorizationBench.java
+++ b/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/vectorization/VectorizationBench.java
@@ -24,6 +24,25 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.ColOrCol;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.LongColDivideLongColumn;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.NotCol;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.LongColEqualLongColumn;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.LongColGreaterEqualLongColumn;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.LongColGreaterLongColumn;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.LongColLessEqualLongColumn;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.LongColLessLongColumn;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.LongColNotEqualLongColumn;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.LongColEqualLongScalar;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.LongColGreaterEqualLongScalar;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.LongColGreaterLongScalar;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.LongColLessEqualLongScalar;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.LongColLessLongScalar;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.LongColNotEqualLongScalar;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.LongScalarEqualLongColumn;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.LongScalarGreaterEqualLongColumn;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.LongScalarGreaterLongColumn;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.LongScalarLessEqualLongColumn;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.LongScalarLessLongColumn;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.LongScalarNotEqualLongColumn;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.LongColDivideLongColumn;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.DoubleColAddDoubleColumn;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.DoubleColDivideDoubleColumn;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.LongColAddLongColumn;
@@ -42,6 +61,7 @@ import org.openjdk.jmh.runner.RunnerException;
 import org.openjdk.jmh.runner.options.Options;
 import org.openjdk.jmh.runner.options.OptionsBuilder;
 
+import java.util.Arrays;
 import java.lang.Override;
 import java.util.Random;
 import java.util.concurrent.TimeUnit;
@@ -87,6 +107,7 @@ public class VectorizationBench {
         rowBatch.cols[i] = cols[i];
       }
       rowBatch.cols[colNum] = output;
+//      rowBatch.selectedInUse = true;
       return rowBatch;
     }
 
@@ -332,6 +353,151 @@ public class VectorizationBench {
     }
   }
 
+  public static class LongColEqualLongColumnBench extends AbstractExpression {
+    @Override
+    public void setup() {
+      rowBatch = buildRowBatch(new LongColumnVector(), 2, getLongColumnVector(), getLongColumnVector());
+      expression = new LongColEqualLongColumn(0, 1, 2);
+    }
+  }
+
+  public static class LongColGreaterEqualLongColumnBench extends AbstractExpression {
+    @Override
+    public void setup() {
+      rowBatch = buildRowBatch(new LongColumnVector(), 2, getLongColumnVector(), getLongColumnVector());
+      expression = new LongColGreaterEqualLongColumn(0, 1, 2);
+    }
+  }
+
+  public static class LongColGreaterLongColumnBench extends AbstractExpression {
+    @Override
+    public void setup() {
+      rowBatch = buildRowBatch(new LongColumnVector(), 2, getLongColumnVector(), getLongColumnVector());
+      expression = new LongColGreaterLongColumn(0, 1, 2);
+    }
+  }
+
+  public static class LongColLessEqualLongColumnBench extends AbstractExpression {
+    @Override
+    public void setup() {
+      rowBatch = buildRowBatch(new LongColumnVector(), 2, getLongColumnVector(), getLongColumnVector());
+      expression = new LongColLessEqualLongColumn(0, 1, 2);
+    }
+  }
+
+  public static class LongColLessLongColumnBench extends AbstractExpression {
+    @Override
+    public void setup() {
+      rowBatch = buildRowBatch(new LongColumnVector(), 2, getLongColumnVector(), getLongColumnVector());
+      expression = new LongColLessLongColumn(0, 1, 2);
+    }
+  }
+
+  public static class LongColNotEqualLongColumnBench extends AbstractExpression {
+    @Override
+    public void setup() {
+      rowBatch = buildRowBatch(new LongColumnVector(), 2, getLongColumnVector(), getLongColumnVector());
+      expression = new LongColNotEqualLongColumn(0, 1, 2);
+    }
+  }
+
+  public static class LongColEqualLongScalarBench extends AbstractExpression {
+    @Override
+    public void setup() {
+      rowBatch = buildRowBatch(new LongColumnVector(), 1, getLongColumnVector());
+      expression = new LongColEqualLongScalar(0, 0, 1);
+    }
+  }
+
+  public static class LongColGreaterEqualLongScalarBench extends AbstractExpression {
+    @Override
+    public void setup() {
+      rowBatch = buildRowBatch(new LongColumnVector(), 1, getLongColumnVector());
+      expression = new LongColGreaterEqualLongScalar(0, 0, 1);
+    }
+  }
+
+  public static class LongColGreaterLongScalarBench extends AbstractExpression {
+    @Override
+    public void setup() {
+      rowBatch = buildRowBatch(new LongColumnVector(), 1, getLongColumnVector());
+      expression = new LongColGreaterLongScalar(0, 0, 1);
+    }
+  }
+
+  public static class LongColLessEqualLongScalarBench extends AbstractExpression {
+    @Override
+    public void setup() {
+      rowBatch = buildRowBatch(new LongColumnVector(), 1, getLongColumnVector());
+      expression = new LongColLessEqualLongScalar(0, 0, 1);
+    }
+  }
+
+  public static class LongColLessLongScalarBench extends AbstractExpression {
+    @Override
+    public void setup() {
+      rowBatch = buildRowBatch(new LongColumnVector(), 1, getLongColumnVector());
+      expression = new LongColLessLongScalar(0, 0, 1);
+    }
+  }
+
+  public static class LongColNotEqualLongScalarBench extends AbstractExpression {
+    @Override
+    public void setup() {
+      rowBatch = buildRowBatch(new LongColumnVector(), 1, getLongColumnVector());
+      expression = new LongColNotEqualLongScalar(0, 0, 1);
+    }
+  }
+
+
+  public static class LongScalarEqualLongColumnBench extends AbstractExpression {
+    @Override
+    public void setup() {
+      rowBatch = buildRowBatch(new LongColumnVector(), 1, getLongColumnVector());
+      expression = new LongScalarEqualLongColumn(0, 0, 1);
+    }
+  }
+
+  public static class LongScalarGreaterEqualLongColumnBench extends AbstractExpression {
+    @Override
+    public void setup() {
+      rowBatch = buildRowBatch(new LongColumnVector(), 1, getLongColumnVector());
+      expression = new LongScalarGreaterEqualLongColumn(0, 0, 1);
+    }
+  }
+
+  public static class LongScalarGreaterLongColumnBench extends AbstractExpression {
+    @Override
+    public void setup() {
+      rowBatch = buildRowBatch(new LongColumnVector(), 1, getLongColumnVector());
+      expression = new LongScalarGreaterLongColumn(0, 0, 1);
+    }
+  }
+
+  public static class LongScalarLessEqualLongColumnBench extends AbstractExpression {
+    @Override
+    public void setup() {
+      rowBatch = buildRowBatch(new LongColumnVector(), 1, getLongColumnVector());
+      expression = new LongScalarLessEqualLongColumn(0, 0, 1);
+    }
+  }
+
+  public static class LongScalarLessLongColumnBench extends AbstractExpression {
+    @Override
+    public void setup() {
+      rowBatch = buildRowBatch(new LongColumnVector(), 1, getLongColumnVector());
+      expression = new LongScalarLessLongColumn(0, 0, 1);
+    }
+  }
+
+  public static class LongScalarNotEqualLongColumnBench extends AbstractExpression {
+    @Override
+    public void setup() {
+      rowBatch = buildRowBatch(new LongColumnVector(), 1, getLongColumnVector());
+      expression = new LongScalarNotEqualLongColumn(0, 0, 1);
+    }
+  }
+
   public static void main(String[] args) throws RunnerException {
     Options opt = new OptionsBuilder().include(".*" + VectorizationBench.class.getSimpleName() +
       ".*").build();

http://git-wip-us.apache.org/repos/asf/hive/blob/2529e9bb/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
new file mode 100644
index 0000000..9b9f15e
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColEqualLongColumn.java
@@ -0,0 +1,169 @@
+/**
+ * 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;
+
+public class LongColEqualLongColumn extends VectorExpression {
+
+  private static final long serialVersionUID = 1L;
+
+  private int colNum1;
+  private int colNum2;
+  private int outputColumn;
+
+  public LongColEqualLongColumn(int colNum1, int colNum2, int outputColumn) {
+    this.colNum1 = colNum1;
+    this.colNum2 = colNum2;
+    this.outputColumn = outputColumn;
+  }
+
+  public LongColEqualLongColumn() {
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    LongColumnVector inputColVector1 = (LongColumnVector) batch.cols[colNum1];
+    LongColumnVector inputColVector2 = (LongColumnVector) batch.cols[colNum2];
+    LongColumnVector outputColVector = (LongColumnVector) batch.cols[outputColumn];
+    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;
+    }
+
+    outputColVector.isRepeating =
+        inputColVector1.isRepeating && inputColVector2.isRepeating
+            || inputColVector1.isRepeating && !inputColVector1.noNulls && inputColVector1.isNull[0]
+            || inputColVector2.isRepeating && !inputColVector2.noNulls && inputColVector2.isNull[0];
+
+    // Handle nulls first  
+    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 int getOutputColumn() {
+    return outputColumn;
+  }
+
+  @Override
+  public String getOutputType() {
+    return "long";
+  }
+
+  public int getColNum1() {
+    return colNum1;
+  }
+
+  public void setColNum1(int colNum1) {
+    this.colNum1 = colNum1;
+  }
+
+  public int getColNum2() {
+    return colNum2;
+  }
+
+  public void setColNum2(int colNum2) {
+    this.colNum2 = colNum2;
+  }
+
+  public void setOutputColumn(int outputColumn) {
+    this.outputColumn = outputColumn;
+  }
+
+  @Override
+  public VectorExpressionDescriptor.Descriptor getDescriptor() {
+    return (new VectorExpressionDescriptor.Builder())
+        .setMode(
+            VectorExpressionDescriptor.Mode.PROJECTION)
+        .setNumArguments(2)
+        .setArgumentTypes(
+            VectorExpressionDescriptor.ArgumentType.getType("int_datetime_interval_family"),
+            VectorExpressionDescriptor.ArgumentType.getType("int_datetime_interval_family"))
+        .setInputExpressionTypes(
+            VectorExpressionDescriptor.InputExpressionType.COLUMN,
+            VectorExpressionDescriptor.InputExpressionType.COLUMN).build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/2529e9bb/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
new file mode 100644
index 0000000..caacf3d
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColEqualLongScalar.java
@@ -0,0 +1,151 @@
+/**
+ * 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.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+
+public class LongColEqualLongScalar extends VectorExpression {
+
+  private static final long serialVersionUID = 1L;
+
+  private int colNum;
+  private long value;
+  private int outputColumn;
+
+  public LongColEqualLongScalar(int colNum, long value, int outputColumn) {
+    this.colNum = colNum;
+    this.value = value;
+    this.outputColumn = outputColumn;
+  }
+
+  public LongColEqualLongScalar() {
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    LongColumnVector inputColVector = (LongColumnVector) batch.cols[colNum];
+    LongColumnVector outputColVector = (LongColumnVector) batch.cols[outputColumn];
+    int[] sel = batch.selected;
+    boolean[] nullPos = inputColVector.isNull;
+    boolean[] outNulls = outputColVector.isNull;
+    int n = batch.size;
+    long[] vector = inputColVector.vector;
+    long[] outputVector = outputColVector.vector;
+
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+
+    outputColVector.isRepeating = false;
+    outputColVector.noNulls = inputColVector.noNulls;
+    if (inputColVector.noNulls) {
+      if (inputColVector.isRepeating) {
+        //All must be selected otherwise size would be zero
+        //Repeating property will not change.
+        outputVector[0] = vector[0] == value ? 1 : 0;
+        outputColVector.isRepeating = true;
+      } else if (batch.selectedInUse) {
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = vector[i] == value ? 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] = (((vector[i] - value) ^ (value - vector[i])) >>> 63) ^ 1;
+        }
+      }
+    } else {
+      if (inputColVector.isRepeating) {
+        //All must be selected otherwise size would be zero
+        //Repeating property will not change.
+        if (!nullPos[0]) {
+          outputVector[0] = vector[0] == value ? 1 : 0;
+          outNulls[0] = false;
+        } else {
+          outNulls[0] = true;
+        }
+        outputColVector.isRepeating = true;
+      } else if (batch.selectedInUse) {
+        System.arraycopy(nullPos, 0, outNulls, 0, n);
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = vector[i] == value ? 1 : 0;
+        }
+      } else {
+        System.arraycopy(nullPos, 0, outNulls, 0, n);
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = (((vector[i] - value) ^ (value - vector[i])) >>> 63) ^ 1;
+        }
+      }
+    }
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return outputColumn;
+  }
+
+  @Override
+  public String getOutputType() {
+    return "long";
+  }
+
+  public int getColNum() {
+    return colNum;
+  }
+
+  public void setColNum(int colNum) {
+    this.colNum = colNum;
+  }
+
+  public long getValue() {
+    return value;
+  }
+
+  public void setValue(long value) {
+    this.value = value;
+  }
+
+  public void setOutputColumn(int outputColumn) {
+    this.outputColumn = outputColumn;
+  }
+
+  @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/2529e9bb/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
new file mode 100644
index 0000000..94c5bed
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColGreaterEqualLongColumn.java
@@ -0,0 +1,169 @@
+/**
+ * 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;
+
+public class LongColGreaterEqualLongColumn extends VectorExpression {
+
+  private static final long serialVersionUID = 1L;
+
+  private int colNum1;
+  private int colNum2;
+  private int outputColumn;
+
+  public LongColGreaterEqualLongColumn(int colNum1, int colNum2, int outputColumn) {
+    this.colNum1 = colNum1;
+    this.colNum2 = colNum2;
+    this.outputColumn = outputColumn;
+  }
+
+  public LongColGreaterEqualLongColumn() {
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    LongColumnVector inputColVector1 = (LongColumnVector) batch.cols[colNum1];
+    LongColumnVector inputColVector2 = (LongColumnVector) batch.cols[colNum2];
+    LongColumnVector outputColVector = (LongColumnVector) batch.cols[outputColumn];
+    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;
+    }
+
+    outputColVector.isRepeating =
+        inputColVector1.isRepeating && inputColVector2.isRepeating
+            || inputColVector1.isRepeating && !inputColVector1.noNulls && inputColVector1.isNull[0]
+            || inputColVector2.isRepeating && !inputColVector2.noNulls && inputColVector2.isNull[0];
+
+    // Handle nulls first  
+    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 int getOutputColumn() {
+    return outputColumn;
+  }
+
+  @Override
+  public String getOutputType() {
+    return "long";
+  }
+
+  public int getColNum1() {
+    return colNum1;
+  }
+
+  public void setColNum1(int colNum1) {
+    this.colNum1 = colNum1;
+  }
+
+  public int getColNum2() {
+    return colNum2;
+  }
+
+  public void setColNum2(int colNum2) {
+    this.colNum2 = colNum2;
+  }
+
+  public void setOutputColumn(int outputColumn) {
+    this.outputColumn = outputColumn;
+  }
+
+  @Override
+  public VectorExpressionDescriptor.Descriptor getDescriptor() {
+    return (new VectorExpressionDescriptor.Builder())
+        .setMode(
+            VectorExpressionDescriptor.Mode.PROJECTION)
+        .setNumArguments(2)
+        .setArgumentTypes(
+            VectorExpressionDescriptor.ArgumentType.getType("int_datetime_interval_family"),
+            VectorExpressionDescriptor.ArgumentType.getType("int_datetime_interval_family"))
+        .setInputExpressionTypes(
+            VectorExpressionDescriptor.InputExpressionType.COLUMN,
+            VectorExpressionDescriptor.InputExpressionType.COLUMN).build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/2529e9bb/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
new file mode 100644
index 0000000..433162c
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColGreaterEqualLongScalar.java
@@ -0,0 +1,151 @@
+/**
+ * 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.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+
+public class LongColGreaterEqualLongScalar extends VectorExpression {
+
+  private static final long serialVersionUID = 1L;
+
+  private int colNum;
+  private long value;
+  private int outputColumn;
+
+  public LongColGreaterEqualLongScalar(int colNum, long value, int outputColumn) {
+    this.colNum = colNum;
+    this.value = value;
+    this.outputColumn = outputColumn;
+  }
+
+  public LongColGreaterEqualLongScalar() {
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    LongColumnVector inputColVector = (LongColumnVector) batch.cols[colNum];
+    LongColumnVector outputColVector = (LongColumnVector) batch.cols[outputColumn];
+    int[] sel = batch.selected;
+    boolean[] nullPos = inputColVector.isNull;
+    boolean[] outNulls = outputColVector.isNull;
+    int n = batch.size;
+    long[] vector = inputColVector.vector;
+    long[] outputVector = outputColVector.vector;
+
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+
+    outputColVector.isRepeating = false;
+    outputColVector.noNulls = inputColVector.noNulls;
+    if (inputColVector.noNulls) {
+      if (inputColVector.isRepeating) {
+        //All must be selected otherwise size would be zero
+        //Repeating property will not change.
+        outputVector[0] = vector[0] >= value ? 1 : 0;
+        outputColVector.isRepeating = true;
+      } else if (batch.selectedInUse) {
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = vector[i] >= value ? 1 : 0;
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          // The SIMD optimized form of "a >= b" is "((a - b) >>> 63) ^ 1"
+          outputVector[i] = ((vector[i] - value) >>> 63) ^ 1;
+        }
+      }
+    } else {
+      if (inputColVector.isRepeating) {
+        //All must be selected otherwise size would be zero
+        //Repeating property will not change.
+        if (!nullPos[0]) {
+          outputVector[0] = vector[0] >= value ? 1 : 0;
+          outNulls[0] = false;
+        } else {
+          outNulls[0] = true;
+        }
+        outputColVector.isRepeating = true;
+      } else if (batch.selectedInUse) {
+        System.arraycopy(nullPos, 0, outNulls, 0, n);
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = vector[i] >= value ? 1 : 0;
+        }
+      } else {
+        System.arraycopy(nullPos, 0, outNulls, 0, n);
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = ((vector[i] - value) >>> 63) ^ 1;
+        }
+      }
+    }
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return outputColumn;
+  }
+
+  @Override
+  public String getOutputType() {
+    return "long";
+  }
+
+  public int getColNum() {
+    return colNum;
+  }
+
+  public void setColNum(int colNum) {
+    this.colNum = colNum;
+  }
+
+  public long getValue() {
+    return value;
+  }
+
+  public void setValue(long value) {
+    this.value = value;
+  }
+
+  public void setOutputColumn(int outputColumn) {
+    this.outputColumn = outputColumn;
+  }
+
+  @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/2529e9bb/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
new file mode 100644
index 0000000..dda941e
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColGreaterLongColumn.java
@@ -0,0 +1,169 @@
+/**
+ * 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;
+
+public class LongColGreaterLongColumn extends VectorExpression {
+
+  private static final long serialVersionUID = 1L;
+
+  private int colNum1;
+  private int colNum2;
+  private int outputColumn;
+
+  public LongColGreaterLongColumn(int colNum1, int colNum2, int outputColumn) {
+    this.colNum1 = colNum1;
+    this.colNum2 = colNum2;
+    this.outputColumn = outputColumn;
+  }
+
+  public LongColGreaterLongColumn() {
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    LongColumnVector inputColVector1 = (LongColumnVector) batch.cols[colNum1];
+    LongColumnVector inputColVector2 = (LongColumnVector) batch.cols[colNum2];
+    LongColumnVector outputColVector = (LongColumnVector) batch.cols[outputColumn];
+    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;
+    }
+
+    outputColVector.isRepeating =
+        inputColVector1.isRepeating && inputColVector2.isRepeating
+            || inputColVector1.isRepeating && !inputColVector1.noNulls && inputColVector1.isNull[0]
+            || inputColVector2.isRepeating && !inputColVector2.noNulls && inputColVector2.isNull[0];
+
+    // Handle nulls first  
+    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 int getOutputColumn() {
+    return outputColumn;
+  }
+
+  @Override
+  public String getOutputType() {
+    return "long";
+  }
+
+  public int getColNum1() {
+    return colNum1;
+  }
+
+  public void setColNum1(int colNum1) {
+    this.colNum1 = colNum1;
+  }
+
+  public int getColNum2() {
+    return colNum2;
+  }
+
+  public void setColNum2(int colNum2) {
+    this.colNum2 = colNum2;
+  }
+
+  public void setOutputColumn(int outputColumn) {
+    this.outputColumn = outputColumn;
+  }
+
+  @Override
+  public VectorExpressionDescriptor.Descriptor getDescriptor() {
+    return (new VectorExpressionDescriptor.Builder())
+        .setMode(
+            VectorExpressionDescriptor.Mode.PROJECTION)
+        .setNumArguments(2)
+        .setArgumentTypes(
+            VectorExpressionDescriptor.ArgumentType.getType("int_datetime_interval_family"),
+            VectorExpressionDescriptor.ArgumentType.getType("int_datetime_interval_family"))
+        .setInputExpressionTypes(
+            VectorExpressionDescriptor.InputExpressionType.COLUMN,
+            VectorExpressionDescriptor.InputExpressionType.COLUMN).build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/2529e9bb/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
new file mode 100644
index 0000000..088b265
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColGreaterLongScalar.java
@@ -0,0 +1,151 @@
+/**
+ * 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.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+
+public class LongColGreaterLongScalar extends VectorExpression {
+
+  private static final long serialVersionUID = 1L;
+
+  private int colNum;
+  private long value;
+  private int outputColumn;
+
+  public LongColGreaterLongScalar(int colNum, long value, int outputColumn) {
+    this.colNum = colNum;
+    this.value = value;
+    this.outputColumn = outputColumn;
+  }
+
+  public LongColGreaterLongScalar() {
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    LongColumnVector inputColVector = (LongColumnVector) batch.cols[colNum];
+    LongColumnVector outputColVector = (LongColumnVector) batch.cols[outputColumn];
+    int[] sel = batch.selected;
+    boolean[] nullPos = inputColVector.isNull;
+    boolean[] outNulls = outputColVector.isNull;
+    int n = batch.size;
+    long[] vector = inputColVector.vector;
+    long[] outputVector = outputColVector.vector;
+
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+
+    outputColVector.isRepeating = false;
+    outputColVector.noNulls = inputColVector.noNulls;
+    if (inputColVector.noNulls) {
+      if (inputColVector.isRepeating) {
+        //All must be selected otherwise size would be zero
+        //Repeating property will not change.
+        outputVector[0] = vector[0] > value ? 1 : 0;
+        outputColVector.isRepeating = true;
+      } else if (batch.selectedInUse) {
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = vector[i] > value ? 1 : 0;
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          // The SIMD optimized form of "a > b" is "(b - a) >>> 63"
+          outputVector[i] = (value - vector[i]) >>> 63;
+        }
+      }
+    } else {
+      if (inputColVector.isRepeating) {
+        //All must be selected otherwise size would be zero
+        //Repeating property will not change.
+        if (!nullPos[0]) {
+          outputVector[0] = vector[0] > value ? 1 : 0;
+          outNulls[0] = false;
+        } else {
+          outNulls[0] = true;
+        }
+        outputColVector.isRepeating = true;
+      } else if (batch.selectedInUse) {
+        System.arraycopy(nullPos, 0, outNulls, 0, n);
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = vector[i] > value ? 1 : 0;
+        }
+      } else {
+        System.arraycopy(nullPos, 0, outNulls, 0, n);
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = (value - vector[i]) >>> 63;
+        }
+      }
+    }
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return outputColumn;
+  }
+
+  @Override
+  public String getOutputType() {
+    return "long";
+  }
+
+  public int getColNum() {
+    return colNum;
+  }
+
+  public void setColNum(int colNum) {
+    this.colNum = colNum;
+  }
+
+  public long getValue() {
+    return value;
+  }
+
+  public void setValue(long value) {
+    this.value = value;
+  }
+
+  public void setOutputColumn(int outputColumn) {
+    this.outputColumn = outputColumn;
+  }
+
+  @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/2529e9bb/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
new file mode 100644
index 0000000..aacdfe6
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColLessEqualLongColumn.java
@@ -0,0 +1,169 @@
+/**
+ * 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;
+
+public class LongColLessEqualLongColumn extends VectorExpression {
+
+  private static final long serialVersionUID = 1L;
+
+  private int colNum1;
+  private int colNum2;
+  private int outputColumn;
+
+  public LongColLessEqualLongColumn(int colNum1, int colNum2, int outputColumn) {
+    this.colNum1 = colNum1;
+    this.colNum2 = colNum2;
+    this.outputColumn = outputColumn;
+  }
+
+  public LongColLessEqualLongColumn() {
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    LongColumnVector inputColVector1 = (LongColumnVector) batch.cols[colNum1];
+    LongColumnVector inputColVector2 = (LongColumnVector) batch.cols[colNum2];
+    LongColumnVector outputColVector = (LongColumnVector) batch.cols[outputColumn];
+    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;
+    }
+
+    outputColVector.isRepeating =
+        inputColVector1.isRepeating && inputColVector2.isRepeating
+            || inputColVector1.isRepeating && !inputColVector1.noNulls && inputColVector1.isNull[0]
+            || inputColVector2.isRepeating && !inputColVector2.noNulls && inputColVector2.isNull[0];
+
+    // Handle nulls first  
+    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 int getOutputColumn() {
+    return outputColumn;
+  }
+
+  @Override
+  public String getOutputType() {
+    return "long";
+  }
+
+  public int getColNum1() {
+    return colNum1;
+  }
+
+  public void setColNum1(int colNum1) {
+    this.colNum1 = colNum1;
+  }
+
+  public int getColNum2() {
+    return colNum2;
+  }
+
+  public void setColNum2(int colNum2) {
+    this.colNum2 = colNum2;
+  }
+
+  public void setOutputColumn(int outputColumn) {
+    this.outputColumn = outputColumn;
+  }
+
+  @Override
+  public VectorExpressionDescriptor.Descriptor getDescriptor() {
+    return (new VectorExpressionDescriptor.Builder())
+        .setMode(
+            VectorExpressionDescriptor.Mode.PROJECTION)
+        .setNumArguments(2)
+        .setArgumentTypes(
+            VectorExpressionDescriptor.ArgumentType.getType("int_datetime_interval_family"),
+            VectorExpressionDescriptor.ArgumentType.getType("int_datetime_interval_family"))
+        .setInputExpressionTypes(
+            VectorExpressionDescriptor.InputExpressionType.COLUMN,
+            VectorExpressionDescriptor.InputExpressionType.COLUMN).build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/2529e9bb/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
new file mode 100644
index 0000000..42bd807
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColLessEqualLongScalar.java
@@ -0,0 +1,151 @@
+/**
+ * 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.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+
+public class LongColLessEqualLongScalar extends VectorExpression {
+
+  private static final long serialVersionUID = 1L;
+
+  private int colNum;
+  private long value;
+  private int outputColumn;
+
+  public LongColLessEqualLongScalar(int colNum, long value, int outputColumn) {
+    this.colNum = colNum;
+    this.value = value;
+    this.outputColumn = outputColumn;
+  }
+
+  public LongColLessEqualLongScalar() {
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    LongColumnVector inputColVector = (LongColumnVector) batch.cols[colNum];
+    LongColumnVector outputColVector = (LongColumnVector) batch.cols[outputColumn];
+    int[] sel = batch.selected;
+    boolean[] nullPos = inputColVector.isNull;
+    boolean[] outNulls = outputColVector.isNull;
+    int n = batch.size;
+    long[] vector = inputColVector.vector;
+    long[] outputVector = outputColVector.vector;
+
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+
+    outputColVector.isRepeating = false;
+    outputColVector.noNulls = inputColVector.noNulls;
+    if (inputColVector.noNulls) {
+      if (inputColVector.isRepeating) {
+        //All must be selected otherwise size would be zero
+        //Repeating property will not change.
+        outputVector[0] = vector[0] <= value ? 1 : 0;
+        outputColVector.isRepeating = true;
+      } else if (batch.selectedInUse) {
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = vector[i] <= value ? 1 : 0;
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          // The SIMD optimized form of "a <= b" is "((b - a) >>> 63) ^ 1"
+          outputVector[i] = ((value - vector[i]) >>> 63) ^ 1;
+        }
+      }
+    } else {
+      if (inputColVector.isRepeating) {
+        //All must be selected otherwise size would be zero
+        //Repeating property will not change.
+        if (!nullPos[0]) {
+          outputVector[0] = vector[0] <= value ? 1 : 0;
+          outNulls[0] = false;
+        } else {
+          outNulls[0] = true;
+        }
+        outputColVector.isRepeating = true;
+      } else if (batch.selectedInUse) {
+        System.arraycopy(nullPos, 0, outNulls, 0, n);
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = vector[i] <= value ? 1 : 0;
+        }
+      } else {
+        System.arraycopy(nullPos, 0, outNulls, 0, n);
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = ((value - vector[i]) >>> 63) ^ 1;
+        }
+      }
+    }
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return outputColumn;
+  }
+
+  @Override
+  public String getOutputType() {
+    return "long";
+  }
+
+  public int getColNum() {
+    return colNum;
+  }
+
+  public void setColNum(int colNum) {
+    this.colNum = colNum;
+  }
+
+  public long getValue() {
+    return value;
+  }
+
+  public void setValue(long value) {
+    this.value = value;
+  }
+
+  public void setOutputColumn(int outputColumn) {
+    this.outputColumn = outputColumn;
+  }
+
+  @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/2529e9bb/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
new file mode 100644
index 0000000..b041ab6
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColLessLongColumn.java
@@ -0,0 +1,169 @@
+/**
+ * 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;
+
+public class LongColLessLongColumn extends VectorExpression {
+
+  private static final long serialVersionUID = 1L;
+
+  private int colNum1;
+  private int colNum2;
+  private int outputColumn;
+
+  public LongColLessLongColumn(int colNum1, int colNum2, int outputColumn) {
+    this.colNum1 = colNum1;
+    this.colNum2 = colNum2;
+    this.outputColumn = outputColumn;
+  }
+
+  public LongColLessLongColumn() {
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    LongColumnVector inputColVector1 = (LongColumnVector) batch.cols[colNum1];
+    LongColumnVector inputColVector2 = (LongColumnVector) batch.cols[colNum2];
+    LongColumnVector outputColVector = (LongColumnVector) batch.cols[outputColumn];
+    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;
+    }
+
+    outputColVector.isRepeating =
+        inputColVector1.isRepeating && inputColVector2.isRepeating
+            || inputColVector1.isRepeating && !inputColVector1.noNulls && inputColVector1.isNull[0]
+            || inputColVector2.isRepeating && !inputColVector2.noNulls && inputColVector2.isNull[0];
+
+    // Handle nulls first
+    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 int getOutputColumn() {
+    return outputColumn;
+  }
+
+  @Override
+  public String getOutputType() {
+    return "long";
+  }
+
+  public int getColNum1() {
+    return colNum1;
+  }
+
+  public void setColNum1(int colNum1) {
+    this.colNum1 = colNum1;
+  }
+
+  public int getColNum2() {
+    return colNum2;
+  }
+
+  public void setColNum2(int colNum2) {
+    this.colNum2 = colNum2;
+  }
+
+  public void setOutputColumn(int outputColumn) {
+    this.outputColumn = outputColumn;
+  }
+
+  @Override
+  public VectorExpressionDescriptor.Descriptor getDescriptor() {
+    return (new VectorExpressionDescriptor.Builder())
+        .setMode(
+            VectorExpressionDescriptor.Mode.PROJECTION)
+        .setNumArguments(2)
+        .setArgumentTypes(
+            VectorExpressionDescriptor.ArgumentType.getType("int_datetime_interval_family"),
+            VectorExpressionDescriptor.ArgumentType.getType("int_datetime_interval_family"))
+        .setInputExpressionTypes(
+            VectorExpressionDescriptor.InputExpressionType.COLUMN,
+            VectorExpressionDescriptor.InputExpressionType.COLUMN).build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/2529e9bb/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
new file mode 100644
index 0000000..67fb47e
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColLessLongScalar.java
@@ -0,0 +1,151 @@
+/**
+ * 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.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+
+public class LongColLessLongScalar extends VectorExpression {
+
+  private static final long serialVersionUID = 1L;
+
+  private int colNum;
+  private long value;
+  private int outputColumn;
+
+  public LongColLessLongScalar(int colNum, long value, int outputColumn) {
+    this.colNum = colNum;
+    this.value = value;
+    this.outputColumn = outputColumn;
+  }
+
+  public LongColLessLongScalar() {
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    LongColumnVector inputColVector = (LongColumnVector) batch.cols[colNum];
+    LongColumnVector outputColVector = (LongColumnVector) batch.cols[outputColumn];
+    int[] sel = batch.selected;
+    boolean[] nullPos = inputColVector.isNull;
+    boolean[] outNulls = outputColVector.isNull;
+    int n = batch.size;
+    long[] vector = inputColVector.vector;
+    long[] outputVector = outputColVector.vector;
+
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+
+    outputColVector.isRepeating = false;
+    outputColVector.noNulls = inputColVector.noNulls;
+    if (inputColVector.noNulls) {
+      if (inputColVector.isRepeating) {
+        //All must be selected otherwise size would be zero
+        //Repeating property will not change.
+        outputVector[0] = vector[0] < value ? 1 : 0;
+        outputColVector.isRepeating = true;
+      } else if (batch.selectedInUse) {
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = vector[i] < value ? 1 : 0;
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          // The SIMD optimized form of "a < b" is "(a - b) >>> 63"
+          outputVector[i] = (vector[i] - value) >>> 63;
+        }
+      }
+    } else {
+      if (inputColVector.isRepeating) {
+        //All must be selected otherwise size would be zero
+        //Repeating property will not change.
+        if (!nullPos[0]) {
+          outputVector[0] = vector[0] < value ? 1 : 0;
+          outNulls[0] = false;
+        } else {
+          outNulls[0] = true;
+        }
+        outputColVector.isRepeating = true;
+      } else if (batch.selectedInUse) {
+        System.arraycopy(nullPos, 0, outNulls, 0, n);
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = vector[i] < value ? 1 : 0;
+        }
+      } else {
+        System.arraycopy(nullPos, 0, outNulls, 0, n);
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = (vector[i] - value) >>> 63;
+        }
+      }
+    }
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return outputColumn;
+  }
+
+  @Override
+  public String getOutputType() {
+    return "long";
+  }
+
+  public int getColNum() {
+    return colNum;
+  }
+
+  public void setColNum(int colNum) {
+    this.colNum = colNum;
+  }
+
+  public long getValue() {
+    return value;
+  }
+
+  public void setValue(long value) {
+    this.value = value;
+  }
+
+  public void setOutputColumn(int outputColumn) {
+    this.outputColumn = outputColumn;
+  }
+
+  @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/2529e9bb/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
new file mode 100644
index 0000000..179e493
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColNotEqualLongColumn.java
@@ -0,0 +1,169 @@
+/**
+ * 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;
+
+public class LongColNotEqualLongColumn extends VectorExpression {
+
+  private static final long serialVersionUID = 1L;
+
+  private int colNum1;
+  private int colNum2;
+  private int outputColumn;
+
+  public LongColNotEqualLongColumn(int colNum1, int colNum2, int outputColumn) {
+    this.colNum1 = colNum1;
+    this.colNum2 = colNum2;
+    this.outputColumn = outputColumn;
+  }
+
+  public LongColNotEqualLongColumn() {
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    LongColumnVector inputColVector1 = (LongColumnVector) batch.cols[colNum1];
+    LongColumnVector inputColVector2 = (LongColumnVector) batch.cols[colNum2];
+    LongColumnVector outputColVector = (LongColumnVector) batch.cols[outputColumn];
+    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;
+    }
+
+    outputColVector.isRepeating =
+        inputColVector1.isRepeating && inputColVector2.isRepeating
+            || inputColVector1.isRepeating && !inputColVector1.noNulls && inputColVector1.isNull[0]
+            || inputColVector2.isRepeating && !inputColVector2.noNulls && inputColVector2.isNull[0];
+
+    // Handle nulls first  
+    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 int getOutputColumn() {
+    return outputColumn;
+  }
+
+  @Override
+  public String getOutputType() {
+    return "long";
+  }
+
+  public int getColNum1() {
+    return colNum1;
+  }
+
+  public void setColNum1(int colNum1) {
+    this.colNum1 = colNum1;
+  }
+
+  public int getColNum2() {
+    return colNum2;
+  }
+
+  public void setColNum2(int colNum2) {
+    this.colNum2 = colNum2;
+  }
+
+  public void setOutputColumn(int outputColumn) {
+    this.outputColumn = outputColumn;
+  }
+
+  @Override
+  public VectorExpressionDescriptor.Descriptor getDescriptor() {
+    return (new VectorExpressionDescriptor.Builder())
+        .setMode(
+            VectorExpressionDescriptor.Mode.PROJECTION)
+        .setNumArguments(2)
+        .setArgumentTypes(
+            VectorExpressionDescriptor.ArgumentType.getType("int_datetime_interval_family"),
+            VectorExpressionDescriptor.ArgumentType.getType("int_datetime_interval_family"))
+        .setInputExpressionTypes(
+            VectorExpressionDescriptor.InputExpressionType.COLUMN,
+            VectorExpressionDescriptor.InputExpressionType.COLUMN).build();
+  }
+}