You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by om...@apache.org on 2013/05/22 23:04:35 UTC

svn commit: r1485421 [4/6] - in /hive/branches/vectorization/ql/src: java/org/apache/hadoop/hive/ql/exec/vector/expressions/templates/ test/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/ test/org/apache/hadoop/hive/ql/exec/vector/util/

Added: hive/branches/vectorization/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/TestColumnScalarFilterVectorExpressionEvaluation.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/TestColumnScalarFilterVectorExpressionEvaluation.java?rev=1485421&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/TestColumnScalarFilterVectorExpressionEvaluation.java (added)
+++ hive/branches/vectorization/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/TestColumnScalarFilterVectorExpressionEvaluation.java Wed May 22 21:04:35 2013
@@ -0,0 +1,5900 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.exec.vector.expressions.gen;
+
+import static org.junit.Assert.assertEquals;
+import java.util.Random;
+import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
+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.util.VectorizedRowGroupGenUtil;
+import org.junit.Test;
+
+
+/**
+ *
+ * TestColumnScalarFilterVectorExpressionEvaluation.
+ *
+ */
+public class TestColumnScalarFilterVectorExpressionEvaluation{
+
+  private static final int BATCH_SIZE = 100;
+  private static final long SEED = 0xfa57;
+
+  
+  @Test
+  public void testFilterLongColEqualDoubleScalarColNullsRepeats() {
+
+    Random rand = new Random(SEED);
+
+    LongColumnVector inputColumnVector =
+      VectorizedRowGroupGenUtil.generateLongColumnVector(true,
+      true, BATCH_SIZE, rand);
+
+    VectorizedRowBatch rowBatch = new VectorizedRowBatch(1, BATCH_SIZE);
+    rowBatch.cols[0] = inputColumnVector;
+
+    double scalarValue = 0;
+    do {
+      scalarValue = rand.nextDouble();
+    } while(scalarValue == 0);
+
+    FilterLongColEqualDoubleScalar vectorExpression =
+      new FilterLongColEqualDoubleScalar(0, scalarValue);
+
+    vectorExpression.evaluate(rowBatch);
+
+    int selectedIndex = 0;
+    //check for isRepeating optimization
+    if(inputColumnVector.isRepeating) {
+      //null vector is safe to check, as it is always initialized to match the data vector
+      selectedIndex =
+        !inputColumnVector.isNull[0] && inputColumnVector.vector[0] == scalarValue
+          ? BATCH_SIZE : 0;
+    } else {
+      for(int i = 0; i < BATCH_SIZE; i++) {
+        if(!inputColumnVector.isNull[i]) {
+          if(inputColumnVector.vector[i] == scalarValue) {
+            assertEquals(
+              "Vector index that passes filter "
+              + inputColumnVector.vector[i] + "=="
+              + scalarValue + " is not in rowBatch selected index",
+              i,
+              rowBatch.selected[selectedIndex]);
+            selectedIndex++;
+          }
+        }
+      }
+    }
+
+    assertEquals("Row batch size not set to number of selected rows: " + selectedIndex,
+      selectedIndex, rowBatch.size);
+
+    if(selectedIndex > 0 && selectedIndex < BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should be set when > 0 and < entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        true, rowBatch.selectedInUse);
+    } else if(selectedIndex == BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should not be set when entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        false, rowBatch.selectedInUse);
+    }
+  }
+
+  @Test
+  public void testFilterLongColEqualDoubleScalarColNulls() {
+
+    Random rand = new Random(SEED);
+
+    LongColumnVector inputColumnVector =
+      VectorizedRowGroupGenUtil.generateLongColumnVector(true,
+      false, BATCH_SIZE, rand);
+
+    VectorizedRowBatch rowBatch = new VectorizedRowBatch(1, BATCH_SIZE);
+    rowBatch.cols[0] = inputColumnVector;
+
+    double scalarValue = 0;
+    do {
+      scalarValue = rand.nextDouble();
+    } while(scalarValue == 0);
+
+    FilterLongColEqualDoubleScalar vectorExpression =
+      new FilterLongColEqualDoubleScalar(0, scalarValue);
+
+    vectorExpression.evaluate(rowBatch);
+
+    int selectedIndex = 0;
+    //check for isRepeating optimization
+    if(inputColumnVector.isRepeating) {
+      //null vector is safe to check, as it is always initialized to match the data vector
+      selectedIndex =
+        !inputColumnVector.isNull[0] && inputColumnVector.vector[0] == scalarValue
+          ? BATCH_SIZE : 0;
+    } else {
+      for(int i = 0; i < BATCH_SIZE; i++) {
+        if(!inputColumnVector.isNull[i]) {
+          if(inputColumnVector.vector[i] == scalarValue) {
+            assertEquals(
+              "Vector index that passes filter "
+              + inputColumnVector.vector[i] + "=="
+              + scalarValue + " is not in rowBatch selected index",
+              i,
+              rowBatch.selected[selectedIndex]);
+            selectedIndex++;
+          }
+        }
+      }
+    }
+
+    assertEquals("Row batch size not set to number of selected rows: " + selectedIndex,
+      selectedIndex, rowBatch.size);
+
+    if(selectedIndex > 0 && selectedIndex < BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should be set when > 0 and < entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        true, rowBatch.selectedInUse);
+    } else if(selectedIndex == BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should not be set when entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        false, rowBatch.selectedInUse);
+    }
+  }
+
+  @Test
+  public void testFilterLongColEqualDoubleScalar() {
+
+    Random rand = new Random(SEED);
+
+    LongColumnVector inputColumnVector =
+      VectorizedRowGroupGenUtil.generateLongColumnVector(false,
+      false, BATCH_SIZE, rand);
+
+    VectorizedRowBatch rowBatch = new VectorizedRowBatch(1, BATCH_SIZE);
+    rowBatch.cols[0] = inputColumnVector;
+
+    double scalarValue = 0;
+    do {
+      scalarValue = rand.nextDouble();
+    } while(scalarValue == 0);
+
+    FilterLongColEqualDoubleScalar vectorExpression =
+      new FilterLongColEqualDoubleScalar(0, scalarValue);
+
+    vectorExpression.evaluate(rowBatch);
+
+    int selectedIndex = 0;
+    //check for isRepeating optimization
+    if(inputColumnVector.isRepeating) {
+      //null vector is safe to check, as it is always initialized to match the data vector
+      selectedIndex =
+        !inputColumnVector.isNull[0] && inputColumnVector.vector[0] == scalarValue
+          ? BATCH_SIZE : 0;
+    } else {
+      for(int i = 0; i < BATCH_SIZE; i++) {
+        if(!inputColumnVector.isNull[i]) {
+          if(inputColumnVector.vector[i] == scalarValue) {
+            assertEquals(
+              "Vector index that passes filter "
+              + inputColumnVector.vector[i] + "=="
+              + scalarValue + " is not in rowBatch selected index",
+              i,
+              rowBatch.selected[selectedIndex]);
+            selectedIndex++;
+          }
+        }
+      }
+    }
+
+    assertEquals("Row batch size not set to number of selected rows: " + selectedIndex,
+      selectedIndex, rowBatch.size);
+
+    if(selectedIndex > 0 && selectedIndex < BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should be set when > 0 and < entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        true, rowBatch.selectedInUse);
+    } else if(selectedIndex == BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should not be set when entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        false, rowBatch.selectedInUse);
+    }
+  }
+
+  @Test
+  public void testFilterLongColEqualDoubleScalarColRepeats() {
+
+    Random rand = new Random(SEED);
+
+    LongColumnVector inputColumnVector =
+      VectorizedRowGroupGenUtil.generateLongColumnVector(false,
+      true, BATCH_SIZE, rand);
+
+    VectorizedRowBatch rowBatch = new VectorizedRowBatch(1, BATCH_SIZE);
+    rowBatch.cols[0] = inputColumnVector;
+
+    double scalarValue = 0;
+    do {
+      scalarValue = rand.nextDouble();
+    } while(scalarValue == 0);
+
+    FilterLongColEqualDoubleScalar vectorExpression =
+      new FilterLongColEqualDoubleScalar(0, scalarValue);
+
+    vectorExpression.evaluate(rowBatch);
+
+    int selectedIndex = 0;
+    //check for isRepeating optimization
+    if(inputColumnVector.isRepeating) {
+      //null vector is safe to check, as it is always initialized to match the data vector
+      selectedIndex =
+        !inputColumnVector.isNull[0] && inputColumnVector.vector[0] == scalarValue
+          ? BATCH_SIZE : 0;
+    } else {
+      for(int i = 0; i < BATCH_SIZE; i++) {
+        if(!inputColumnVector.isNull[i]) {
+          if(inputColumnVector.vector[i] == scalarValue) {
+            assertEquals(
+              "Vector index that passes filter "
+              + inputColumnVector.vector[i] + "=="
+              + scalarValue + " is not in rowBatch selected index",
+              i,
+              rowBatch.selected[selectedIndex]);
+            selectedIndex++;
+          }
+        }
+      }
+    }
+
+    assertEquals("Row batch size not set to number of selected rows: " + selectedIndex,
+      selectedIndex, rowBatch.size);
+
+    if(selectedIndex > 0 && selectedIndex < BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should be set when > 0 and < entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        true, rowBatch.selectedInUse);
+    } else if(selectedIndex == BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should not be set when entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        false, rowBatch.selectedInUse);
+    }
+  }
+
+  @Test
+  public void testFilterDoubleColEqualDoubleScalarColNullsRepeats() {
+
+    Random rand = new Random(SEED);
+
+    DoubleColumnVector inputColumnVector =
+      VectorizedRowGroupGenUtil.generateDoubleColumnVector(true,
+      true, BATCH_SIZE, rand);
+
+    VectorizedRowBatch rowBatch = new VectorizedRowBatch(1, BATCH_SIZE);
+    rowBatch.cols[0] = inputColumnVector;
+
+    double scalarValue = 0;
+    do {
+      scalarValue = rand.nextDouble();
+    } while(scalarValue == 0);
+
+    FilterDoubleColEqualDoubleScalar vectorExpression =
+      new FilterDoubleColEqualDoubleScalar(0, scalarValue);
+
+    vectorExpression.evaluate(rowBatch);
+
+    int selectedIndex = 0;
+    //check for isRepeating optimization
+    if(inputColumnVector.isRepeating) {
+      //null vector is safe to check, as it is always initialized to match the data vector
+      selectedIndex =
+        !inputColumnVector.isNull[0] && inputColumnVector.vector[0] == scalarValue
+          ? BATCH_SIZE : 0;
+    } else {
+      for(int i = 0; i < BATCH_SIZE; i++) {
+        if(!inputColumnVector.isNull[i]) {
+          if(inputColumnVector.vector[i] == scalarValue) {
+            assertEquals(
+              "Vector index that passes filter "
+              + inputColumnVector.vector[i] + "=="
+              + scalarValue + " is not in rowBatch selected index",
+              i,
+              rowBatch.selected[selectedIndex]);
+            selectedIndex++;
+          }
+        }
+      }
+    }
+
+    assertEquals("Row batch size not set to number of selected rows: " + selectedIndex,
+      selectedIndex, rowBatch.size);
+
+    if(selectedIndex > 0 && selectedIndex < BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should be set when > 0 and < entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        true, rowBatch.selectedInUse);
+    } else if(selectedIndex == BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should not be set when entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        false, rowBatch.selectedInUse);
+    }
+  }
+
+  @Test
+  public void testFilterDoubleColEqualDoubleScalarColNulls() {
+
+    Random rand = new Random(SEED);
+
+    DoubleColumnVector inputColumnVector =
+      VectorizedRowGroupGenUtil.generateDoubleColumnVector(true,
+      false, BATCH_SIZE, rand);
+
+    VectorizedRowBatch rowBatch = new VectorizedRowBatch(1, BATCH_SIZE);
+    rowBatch.cols[0] = inputColumnVector;
+
+    double scalarValue = 0;
+    do {
+      scalarValue = rand.nextDouble();
+    } while(scalarValue == 0);
+
+    FilterDoubleColEqualDoubleScalar vectorExpression =
+      new FilterDoubleColEqualDoubleScalar(0, scalarValue);
+
+    vectorExpression.evaluate(rowBatch);
+
+    int selectedIndex = 0;
+    //check for isRepeating optimization
+    if(inputColumnVector.isRepeating) {
+      //null vector is safe to check, as it is always initialized to match the data vector
+      selectedIndex =
+        !inputColumnVector.isNull[0] && inputColumnVector.vector[0] == scalarValue
+          ? BATCH_SIZE : 0;
+    } else {
+      for(int i = 0; i < BATCH_SIZE; i++) {
+        if(!inputColumnVector.isNull[i]) {
+          if(inputColumnVector.vector[i] == scalarValue) {
+            assertEquals(
+              "Vector index that passes filter "
+              + inputColumnVector.vector[i] + "=="
+              + scalarValue + " is not in rowBatch selected index",
+              i,
+              rowBatch.selected[selectedIndex]);
+            selectedIndex++;
+          }
+        }
+      }
+    }
+
+    assertEquals("Row batch size not set to number of selected rows: " + selectedIndex,
+      selectedIndex, rowBatch.size);
+
+    if(selectedIndex > 0 && selectedIndex < BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should be set when > 0 and < entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        true, rowBatch.selectedInUse);
+    } else if(selectedIndex == BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should not be set when entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        false, rowBatch.selectedInUse);
+    }
+  }
+
+  @Test
+  public void testFilterDoubleColEqualDoubleScalar() {
+
+    Random rand = new Random(SEED);
+
+    DoubleColumnVector inputColumnVector =
+      VectorizedRowGroupGenUtil.generateDoubleColumnVector(false,
+      false, BATCH_SIZE, rand);
+
+    VectorizedRowBatch rowBatch = new VectorizedRowBatch(1, BATCH_SIZE);
+    rowBatch.cols[0] = inputColumnVector;
+
+    double scalarValue = 0;
+    do {
+      scalarValue = rand.nextDouble();
+    } while(scalarValue == 0);
+
+    FilterDoubleColEqualDoubleScalar vectorExpression =
+      new FilterDoubleColEqualDoubleScalar(0, scalarValue);
+
+    vectorExpression.evaluate(rowBatch);
+
+    int selectedIndex = 0;
+    //check for isRepeating optimization
+    if(inputColumnVector.isRepeating) {
+      //null vector is safe to check, as it is always initialized to match the data vector
+      selectedIndex =
+        !inputColumnVector.isNull[0] && inputColumnVector.vector[0] == scalarValue
+          ? BATCH_SIZE : 0;
+    } else {
+      for(int i = 0; i < BATCH_SIZE; i++) {
+        if(!inputColumnVector.isNull[i]) {
+          if(inputColumnVector.vector[i] == scalarValue) {
+            assertEquals(
+              "Vector index that passes filter "
+              + inputColumnVector.vector[i] + "=="
+              + scalarValue + " is not in rowBatch selected index",
+              i,
+              rowBatch.selected[selectedIndex]);
+            selectedIndex++;
+          }
+        }
+      }
+    }
+
+    assertEquals("Row batch size not set to number of selected rows: " + selectedIndex,
+      selectedIndex, rowBatch.size);
+
+    if(selectedIndex > 0 && selectedIndex < BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should be set when > 0 and < entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        true, rowBatch.selectedInUse);
+    } else if(selectedIndex == BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should not be set when entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        false, rowBatch.selectedInUse);
+    }
+  }
+
+  @Test
+  public void testFilterDoubleColEqualDoubleScalarColRepeats() {
+
+    Random rand = new Random(SEED);
+
+    DoubleColumnVector inputColumnVector =
+      VectorizedRowGroupGenUtil.generateDoubleColumnVector(false,
+      true, BATCH_SIZE, rand);
+
+    VectorizedRowBatch rowBatch = new VectorizedRowBatch(1, BATCH_SIZE);
+    rowBatch.cols[0] = inputColumnVector;
+
+    double scalarValue = 0;
+    do {
+      scalarValue = rand.nextDouble();
+    } while(scalarValue == 0);
+
+    FilterDoubleColEqualDoubleScalar vectorExpression =
+      new FilterDoubleColEqualDoubleScalar(0, scalarValue);
+
+    vectorExpression.evaluate(rowBatch);
+
+    int selectedIndex = 0;
+    //check for isRepeating optimization
+    if(inputColumnVector.isRepeating) {
+      //null vector is safe to check, as it is always initialized to match the data vector
+      selectedIndex =
+        !inputColumnVector.isNull[0] && inputColumnVector.vector[0] == scalarValue
+          ? BATCH_SIZE : 0;
+    } else {
+      for(int i = 0; i < BATCH_SIZE; i++) {
+        if(!inputColumnVector.isNull[i]) {
+          if(inputColumnVector.vector[i] == scalarValue) {
+            assertEquals(
+              "Vector index that passes filter "
+              + inputColumnVector.vector[i] + "=="
+              + scalarValue + " is not in rowBatch selected index",
+              i,
+              rowBatch.selected[selectedIndex]);
+            selectedIndex++;
+          }
+        }
+      }
+    }
+
+    assertEquals("Row batch size not set to number of selected rows: " + selectedIndex,
+      selectedIndex, rowBatch.size);
+
+    if(selectedIndex > 0 && selectedIndex < BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should be set when > 0 and < entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        true, rowBatch.selectedInUse);
+    } else if(selectedIndex == BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should not be set when entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        false, rowBatch.selectedInUse);
+    }
+  }
+
+  @Test
+  public void testFilterLongColNotEqualDoubleScalarColNullsRepeats() {
+
+    Random rand = new Random(SEED);
+
+    LongColumnVector inputColumnVector =
+      VectorizedRowGroupGenUtil.generateLongColumnVector(true,
+      true, BATCH_SIZE, rand);
+
+    VectorizedRowBatch rowBatch = new VectorizedRowBatch(1, BATCH_SIZE);
+    rowBatch.cols[0] = inputColumnVector;
+
+    double scalarValue = 0;
+    do {
+      scalarValue = rand.nextDouble();
+    } while(scalarValue == 0);
+
+    FilterLongColNotEqualDoubleScalar vectorExpression =
+      new FilterLongColNotEqualDoubleScalar(0, scalarValue);
+
+    vectorExpression.evaluate(rowBatch);
+
+    int selectedIndex = 0;
+    //check for isRepeating optimization
+    if(inputColumnVector.isRepeating) {
+      //null vector is safe to check, as it is always initialized to match the data vector
+      selectedIndex =
+        !inputColumnVector.isNull[0] && inputColumnVector.vector[0] != scalarValue
+          ? BATCH_SIZE : 0;
+    } else {
+      for(int i = 0; i < BATCH_SIZE; i++) {
+        if(!inputColumnVector.isNull[i]) {
+          if(inputColumnVector.vector[i] != scalarValue) {
+            assertEquals(
+              "Vector index that passes filter "
+              + inputColumnVector.vector[i] + "!="
+              + scalarValue + " is not in rowBatch selected index",
+              i,
+              rowBatch.selected[selectedIndex]);
+            selectedIndex++;
+          }
+        }
+      }
+    }
+
+    assertEquals("Row batch size not set to number of selected rows: " + selectedIndex,
+      selectedIndex, rowBatch.size);
+
+    if(selectedIndex > 0 && selectedIndex < BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should be set when > 0 and < entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        true, rowBatch.selectedInUse);
+    } else if(selectedIndex == BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should not be set when entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        false, rowBatch.selectedInUse);
+    }
+  }
+
+  @Test
+  public void testFilterLongColNotEqualDoubleScalarColNulls() {
+
+    Random rand = new Random(SEED);
+
+    LongColumnVector inputColumnVector =
+      VectorizedRowGroupGenUtil.generateLongColumnVector(true,
+      false, BATCH_SIZE, rand);
+
+    VectorizedRowBatch rowBatch = new VectorizedRowBatch(1, BATCH_SIZE);
+    rowBatch.cols[0] = inputColumnVector;
+
+    double scalarValue = 0;
+    do {
+      scalarValue = rand.nextDouble();
+    } while(scalarValue == 0);
+
+    FilterLongColNotEqualDoubleScalar vectorExpression =
+      new FilterLongColNotEqualDoubleScalar(0, scalarValue);
+
+    vectorExpression.evaluate(rowBatch);
+
+    int selectedIndex = 0;
+    //check for isRepeating optimization
+    if(inputColumnVector.isRepeating) {
+      //null vector is safe to check, as it is always initialized to match the data vector
+      selectedIndex =
+        !inputColumnVector.isNull[0] && inputColumnVector.vector[0] != scalarValue
+          ? BATCH_SIZE : 0;
+    } else {
+      for(int i = 0; i < BATCH_SIZE; i++) {
+        if(!inputColumnVector.isNull[i]) {
+          if(inputColumnVector.vector[i] != scalarValue) {
+            assertEquals(
+              "Vector index that passes filter "
+              + inputColumnVector.vector[i] + "!="
+              + scalarValue + " is not in rowBatch selected index",
+              i,
+              rowBatch.selected[selectedIndex]);
+            selectedIndex++;
+          }
+        }
+      }
+    }
+
+    assertEquals("Row batch size not set to number of selected rows: " + selectedIndex,
+      selectedIndex, rowBatch.size);
+
+    if(selectedIndex > 0 && selectedIndex < BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should be set when > 0 and < entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        true, rowBatch.selectedInUse);
+    } else if(selectedIndex == BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should not be set when entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        false, rowBatch.selectedInUse);
+    }
+  }
+
+  @Test
+  public void testFilterLongColNotEqualDoubleScalar() {
+
+    Random rand = new Random(SEED);
+
+    LongColumnVector inputColumnVector =
+      VectorizedRowGroupGenUtil.generateLongColumnVector(false,
+      false, BATCH_SIZE, rand);
+
+    VectorizedRowBatch rowBatch = new VectorizedRowBatch(1, BATCH_SIZE);
+    rowBatch.cols[0] = inputColumnVector;
+
+    double scalarValue = 0;
+    do {
+      scalarValue = rand.nextDouble();
+    } while(scalarValue == 0);
+
+    FilterLongColNotEqualDoubleScalar vectorExpression =
+      new FilterLongColNotEqualDoubleScalar(0, scalarValue);
+
+    vectorExpression.evaluate(rowBatch);
+
+    int selectedIndex = 0;
+    //check for isRepeating optimization
+    if(inputColumnVector.isRepeating) {
+      //null vector is safe to check, as it is always initialized to match the data vector
+      selectedIndex =
+        !inputColumnVector.isNull[0] && inputColumnVector.vector[0] != scalarValue
+          ? BATCH_SIZE : 0;
+    } else {
+      for(int i = 0; i < BATCH_SIZE; i++) {
+        if(!inputColumnVector.isNull[i]) {
+          if(inputColumnVector.vector[i] != scalarValue) {
+            assertEquals(
+              "Vector index that passes filter "
+              + inputColumnVector.vector[i] + "!="
+              + scalarValue + " is not in rowBatch selected index",
+              i,
+              rowBatch.selected[selectedIndex]);
+            selectedIndex++;
+          }
+        }
+      }
+    }
+
+    assertEquals("Row batch size not set to number of selected rows: " + selectedIndex,
+      selectedIndex, rowBatch.size);
+
+    if(selectedIndex > 0 && selectedIndex < BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should be set when > 0 and < entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        true, rowBatch.selectedInUse);
+    } else if(selectedIndex == BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should not be set when entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        false, rowBatch.selectedInUse);
+    }
+  }
+
+  @Test
+  public void testFilterLongColNotEqualDoubleScalarColRepeats() {
+
+    Random rand = new Random(SEED);
+
+    LongColumnVector inputColumnVector =
+      VectorizedRowGroupGenUtil.generateLongColumnVector(false,
+      true, BATCH_SIZE, rand);
+
+    VectorizedRowBatch rowBatch = new VectorizedRowBatch(1, BATCH_SIZE);
+    rowBatch.cols[0] = inputColumnVector;
+
+    double scalarValue = 0;
+    do {
+      scalarValue = rand.nextDouble();
+    } while(scalarValue == 0);
+
+    FilterLongColNotEqualDoubleScalar vectorExpression =
+      new FilterLongColNotEqualDoubleScalar(0, scalarValue);
+
+    vectorExpression.evaluate(rowBatch);
+
+    int selectedIndex = 0;
+    //check for isRepeating optimization
+    if(inputColumnVector.isRepeating) {
+      //null vector is safe to check, as it is always initialized to match the data vector
+      selectedIndex =
+        !inputColumnVector.isNull[0] && inputColumnVector.vector[0] != scalarValue
+          ? BATCH_SIZE : 0;
+    } else {
+      for(int i = 0; i < BATCH_SIZE; i++) {
+        if(!inputColumnVector.isNull[i]) {
+          if(inputColumnVector.vector[i] != scalarValue) {
+            assertEquals(
+              "Vector index that passes filter "
+              + inputColumnVector.vector[i] + "!="
+              + scalarValue + " is not in rowBatch selected index",
+              i,
+              rowBatch.selected[selectedIndex]);
+            selectedIndex++;
+          }
+        }
+      }
+    }
+
+    assertEquals("Row batch size not set to number of selected rows: " + selectedIndex,
+      selectedIndex, rowBatch.size);
+
+    if(selectedIndex > 0 && selectedIndex < BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should be set when > 0 and < entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        true, rowBatch.selectedInUse);
+    } else if(selectedIndex == BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should not be set when entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        false, rowBatch.selectedInUse);
+    }
+  }
+
+  @Test
+  public void testFilterDoubleColNotEqualDoubleScalarColNullsRepeats() {
+
+    Random rand = new Random(SEED);
+
+    DoubleColumnVector inputColumnVector =
+      VectorizedRowGroupGenUtil.generateDoubleColumnVector(true,
+      true, BATCH_SIZE, rand);
+
+    VectorizedRowBatch rowBatch = new VectorizedRowBatch(1, BATCH_SIZE);
+    rowBatch.cols[0] = inputColumnVector;
+
+    double scalarValue = 0;
+    do {
+      scalarValue = rand.nextDouble();
+    } while(scalarValue == 0);
+
+    FilterDoubleColNotEqualDoubleScalar vectorExpression =
+      new FilterDoubleColNotEqualDoubleScalar(0, scalarValue);
+
+    vectorExpression.evaluate(rowBatch);
+
+    int selectedIndex = 0;
+    //check for isRepeating optimization
+    if(inputColumnVector.isRepeating) {
+      //null vector is safe to check, as it is always initialized to match the data vector
+      selectedIndex =
+        !inputColumnVector.isNull[0] && inputColumnVector.vector[0] != scalarValue
+          ? BATCH_SIZE : 0;
+    } else {
+      for(int i = 0; i < BATCH_SIZE; i++) {
+        if(!inputColumnVector.isNull[i]) {
+          if(inputColumnVector.vector[i] != scalarValue) {
+            assertEquals(
+              "Vector index that passes filter "
+              + inputColumnVector.vector[i] + "!="
+              + scalarValue + " is not in rowBatch selected index",
+              i,
+              rowBatch.selected[selectedIndex]);
+            selectedIndex++;
+          }
+        }
+      }
+    }
+
+    assertEquals("Row batch size not set to number of selected rows: " + selectedIndex,
+      selectedIndex, rowBatch.size);
+
+    if(selectedIndex > 0 && selectedIndex < BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should be set when > 0 and < entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        true, rowBatch.selectedInUse);
+    } else if(selectedIndex == BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should not be set when entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        false, rowBatch.selectedInUse);
+    }
+  }
+
+  @Test
+  public void testFilterDoubleColNotEqualDoubleScalarColNulls() {
+
+    Random rand = new Random(SEED);
+
+    DoubleColumnVector inputColumnVector =
+      VectorizedRowGroupGenUtil.generateDoubleColumnVector(true,
+      false, BATCH_SIZE, rand);
+
+    VectorizedRowBatch rowBatch = new VectorizedRowBatch(1, BATCH_SIZE);
+    rowBatch.cols[0] = inputColumnVector;
+
+    double scalarValue = 0;
+    do {
+      scalarValue = rand.nextDouble();
+    } while(scalarValue == 0);
+
+    FilterDoubleColNotEqualDoubleScalar vectorExpression =
+      new FilterDoubleColNotEqualDoubleScalar(0, scalarValue);
+
+    vectorExpression.evaluate(rowBatch);
+
+    int selectedIndex = 0;
+    //check for isRepeating optimization
+    if(inputColumnVector.isRepeating) {
+      //null vector is safe to check, as it is always initialized to match the data vector
+      selectedIndex =
+        !inputColumnVector.isNull[0] && inputColumnVector.vector[0] != scalarValue
+          ? BATCH_SIZE : 0;
+    } else {
+      for(int i = 0; i < BATCH_SIZE; i++) {
+        if(!inputColumnVector.isNull[i]) {
+          if(inputColumnVector.vector[i] != scalarValue) {
+            assertEquals(
+              "Vector index that passes filter "
+              + inputColumnVector.vector[i] + "!="
+              + scalarValue + " is not in rowBatch selected index",
+              i,
+              rowBatch.selected[selectedIndex]);
+            selectedIndex++;
+          }
+        }
+      }
+    }
+
+    assertEquals("Row batch size not set to number of selected rows: " + selectedIndex,
+      selectedIndex, rowBatch.size);
+
+    if(selectedIndex > 0 && selectedIndex < BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should be set when > 0 and < entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        true, rowBatch.selectedInUse);
+    } else if(selectedIndex == BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should not be set when entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        false, rowBatch.selectedInUse);
+    }
+  }
+
+  @Test
+  public void testFilterDoubleColNotEqualDoubleScalar() {
+
+    Random rand = new Random(SEED);
+
+    DoubleColumnVector inputColumnVector =
+      VectorizedRowGroupGenUtil.generateDoubleColumnVector(false,
+      false, BATCH_SIZE, rand);
+
+    VectorizedRowBatch rowBatch = new VectorizedRowBatch(1, BATCH_SIZE);
+    rowBatch.cols[0] = inputColumnVector;
+
+    double scalarValue = 0;
+    do {
+      scalarValue = rand.nextDouble();
+    } while(scalarValue == 0);
+
+    FilterDoubleColNotEqualDoubleScalar vectorExpression =
+      new FilterDoubleColNotEqualDoubleScalar(0, scalarValue);
+
+    vectorExpression.evaluate(rowBatch);
+
+    int selectedIndex = 0;
+    //check for isRepeating optimization
+    if(inputColumnVector.isRepeating) {
+      //null vector is safe to check, as it is always initialized to match the data vector
+      selectedIndex =
+        !inputColumnVector.isNull[0] && inputColumnVector.vector[0] != scalarValue
+          ? BATCH_SIZE : 0;
+    } else {
+      for(int i = 0; i < BATCH_SIZE; i++) {
+        if(!inputColumnVector.isNull[i]) {
+          if(inputColumnVector.vector[i] != scalarValue) {
+            assertEquals(
+              "Vector index that passes filter "
+              + inputColumnVector.vector[i] + "!="
+              + scalarValue + " is not in rowBatch selected index",
+              i,
+              rowBatch.selected[selectedIndex]);
+            selectedIndex++;
+          }
+        }
+      }
+    }
+
+    assertEquals("Row batch size not set to number of selected rows: " + selectedIndex,
+      selectedIndex, rowBatch.size);
+
+    if(selectedIndex > 0 && selectedIndex < BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should be set when > 0 and < entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        true, rowBatch.selectedInUse);
+    } else if(selectedIndex == BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should not be set when entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        false, rowBatch.selectedInUse);
+    }
+  }
+
+  @Test
+  public void testFilterDoubleColNotEqualDoubleScalarColRepeats() {
+
+    Random rand = new Random(SEED);
+
+    DoubleColumnVector inputColumnVector =
+      VectorizedRowGroupGenUtil.generateDoubleColumnVector(false,
+      true, BATCH_SIZE, rand);
+
+    VectorizedRowBatch rowBatch = new VectorizedRowBatch(1, BATCH_SIZE);
+    rowBatch.cols[0] = inputColumnVector;
+
+    double scalarValue = 0;
+    do {
+      scalarValue = rand.nextDouble();
+    } while(scalarValue == 0);
+
+    FilterDoubleColNotEqualDoubleScalar vectorExpression =
+      new FilterDoubleColNotEqualDoubleScalar(0, scalarValue);
+
+    vectorExpression.evaluate(rowBatch);
+
+    int selectedIndex = 0;
+    //check for isRepeating optimization
+    if(inputColumnVector.isRepeating) {
+      //null vector is safe to check, as it is always initialized to match the data vector
+      selectedIndex =
+        !inputColumnVector.isNull[0] && inputColumnVector.vector[0] != scalarValue
+          ? BATCH_SIZE : 0;
+    } else {
+      for(int i = 0; i < BATCH_SIZE; i++) {
+        if(!inputColumnVector.isNull[i]) {
+          if(inputColumnVector.vector[i] != scalarValue) {
+            assertEquals(
+              "Vector index that passes filter "
+              + inputColumnVector.vector[i] + "!="
+              + scalarValue + " is not in rowBatch selected index",
+              i,
+              rowBatch.selected[selectedIndex]);
+            selectedIndex++;
+          }
+        }
+      }
+    }
+
+    assertEquals("Row batch size not set to number of selected rows: " + selectedIndex,
+      selectedIndex, rowBatch.size);
+
+    if(selectedIndex > 0 && selectedIndex < BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should be set when > 0 and < entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        true, rowBatch.selectedInUse);
+    } else if(selectedIndex == BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should not be set when entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        false, rowBatch.selectedInUse);
+    }
+  }
+
+  @Test
+  public void testFilterLongColLessDoubleScalarColNullsRepeats() {
+
+    Random rand = new Random(SEED);
+
+    LongColumnVector inputColumnVector =
+      VectorizedRowGroupGenUtil.generateLongColumnVector(true,
+      true, BATCH_SIZE, rand);
+
+    VectorizedRowBatch rowBatch = new VectorizedRowBatch(1, BATCH_SIZE);
+    rowBatch.cols[0] = inputColumnVector;
+
+    double scalarValue = 0;
+    do {
+      scalarValue = rand.nextDouble();
+    } while(scalarValue == 0);
+
+    FilterLongColLessDoubleScalar vectorExpression =
+      new FilterLongColLessDoubleScalar(0, scalarValue);
+
+    vectorExpression.evaluate(rowBatch);
+
+    int selectedIndex = 0;
+    //check for isRepeating optimization
+    if(inputColumnVector.isRepeating) {
+      //null vector is safe to check, as it is always initialized to match the data vector
+      selectedIndex =
+        !inputColumnVector.isNull[0] && inputColumnVector.vector[0] < scalarValue
+          ? BATCH_SIZE : 0;
+    } else {
+      for(int i = 0; i < BATCH_SIZE; i++) {
+        if(!inputColumnVector.isNull[i]) {
+          if(inputColumnVector.vector[i] < scalarValue) {
+            assertEquals(
+              "Vector index that passes filter "
+              + inputColumnVector.vector[i] + "<"
+              + scalarValue + " is not in rowBatch selected index",
+              i,
+              rowBatch.selected[selectedIndex]);
+            selectedIndex++;
+          }
+        }
+      }
+    }
+
+    assertEquals("Row batch size not set to number of selected rows: " + selectedIndex,
+      selectedIndex, rowBatch.size);
+
+    if(selectedIndex > 0 && selectedIndex < BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should be set when > 0 and < entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        true, rowBatch.selectedInUse);
+    } else if(selectedIndex == BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should not be set when entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        false, rowBatch.selectedInUse);
+    }
+  }
+
+  @Test
+  public void testFilterLongColLessDoubleScalarColNulls() {
+
+    Random rand = new Random(SEED);
+
+    LongColumnVector inputColumnVector =
+      VectorizedRowGroupGenUtil.generateLongColumnVector(true,
+      false, BATCH_SIZE, rand);
+
+    VectorizedRowBatch rowBatch = new VectorizedRowBatch(1, BATCH_SIZE);
+    rowBatch.cols[0] = inputColumnVector;
+
+    double scalarValue = 0;
+    do {
+      scalarValue = rand.nextDouble();
+    } while(scalarValue == 0);
+
+    FilterLongColLessDoubleScalar vectorExpression =
+      new FilterLongColLessDoubleScalar(0, scalarValue);
+
+    vectorExpression.evaluate(rowBatch);
+
+    int selectedIndex = 0;
+    //check for isRepeating optimization
+    if(inputColumnVector.isRepeating) {
+      //null vector is safe to check, as it is always initialized to match the data vector
+      selectedIndex =
+        !inputColumnVector.isNull[0] && inputColumnVector.vector[0] < scalarValue
+          ? BATCH_SIZE : 0;
+    } else {
+      for(int i = 0; i < BATCH_SIZE; i++) {
+        if(!inputColumnVector.isNull[i]) {
+          if(inputColumnVector.vector[i] < scalarValue) {
+            assertEquals(
+              "Vector index that passes filter "
+              + inputColumnVector.vector[i] + "<"
+              + scalarValue + " is not in rowBatch selected index",
+              i,
+              rowBatch.selected[selectedIndex]);
+            selectedIndex++;
+          }
+        }
+      }
+    }
+
+    assertEquals("Row batch size not set to number of selected rows: " + selectedIndex,
+      selectedIndex, rowBatch.size);
+
+    if(selectedIndex > 0 && selectedIndex < BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should be set when > 0 and < entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        true, rowBatch.selectedInUse);
+    } else if(selectedIndex == BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should not be set when entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        false, rowBatch.selectedInUse);
+    }
+  }
+
+  @Test
+  public void testFilterLongColLessDoubleScalar() {
+
+    Random rand = new Random(SEED);
+
+    LongColumnVector inputColumnVector =
+      VectorizedRowGroupGenUtil.generateLongColumnVector(false,
+      false, BATCH_SIZE, rand);
+
+    VectorizedRowBatch rowBatch = new VectorizedRowBatch(1, BATCH_SIZE);
+    rowBatch.cols[0] = inputColumnVector;
+
+    double scalarValue = 0;
+    do {
+      scalarValue = rand.nextDouble();
+    } while(scalarValue == 0);
+
+    FilterLongColLessDoubleScalar vectorExpression =
+      new FilterLongColLessDoubleScalar(0, scalarValue);
+
+    vectorExpression.evaluate(rowBatch);
+
+    int selectedIndex = 0;
+    //check for isRepeating optimization
+    if(inputColumnVector.isRepeating) {
+      //null vector is safe to check, as it is always initialized to match the data vector
+      selectedIndex =
+        !inputColumnVector.isNull[0] && inputColumnVector.vector[0] < scalarValue
+          ? BATCH_SIZE : 0;
+    } else {
+      for(int i = 0; i < BATCH_SIZE; i++) {
+        if(!inputColumnVector.isNull[i]) {
+          if(inputColumnVector.vector[i] < scalarValue) {
+            assertEquals(
+              "Vector index that passes filter "
+              + inputColumnVector.vector[i] + "<"
+              + scalarValue + " is not in rowBatch selected index",
+              i,
+              rowBatch.selected[selectedIndex]);
+            selectedIndex++;
+          }
+        }
+      }
+    }
+
+    assertEquals("Row batch size not set to number of selected rows: " + selectedIndex,
+      selectedIndex, rowBatch.size);
+
+    if(selectedIndex > 0 && selectedIndex < BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should be set when > 0 and < entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        true, rowBatch.selectedInUse);
+    } else if(selectedIndex == BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should not be set when entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        false, rowBatch.selectedInUse);
+    }
+  }
+
+  @Test
+  public void testFilterLongColLessDoubleScalarColRepeats() {
+
+    Random rand = new Random(SEED);
+
+    LongColumnVector inputColumnVector =
+      VectorizedRowGroupGenUtil.generateLongColumnVector(false,
+      true, BATCH_SIZE, rand);
+
+    VectorizedRowBatch rowBatch = new VectorizedRowBatch(1, BATCH_SIZE);
+    rowBatch.cols[0] = inputColumnVector;
+
+    double scalarValue = 0;
+    do {
+      scalarValue = rand.nextDouble();
+    } while(scalarValue == 0);
+
+    FilterLongColLessDoubleScalar vectorExpression =
+      new FilterLongColLessDoubleScalar(0, scalarValue);
+
+    vectorExpression.evaluate(rowBatch);
+
+    int selectedIndex = 0;
+    //check for isRepeating optimization
+    if(inputColumnVector.isRepeating) {
+      //null vector is safe to check, as it is always initialized to match the data vector
+      selectedIndex =
+        !inputColumnVector.isNull[0] && inputColumnVector.vector[0] < scalarValue
+          ? BATCH_SIZE : 0;
+    } else {
+      for(int i = 0; i < BATCH_SIZE; i++) {
+        if(!inputColumnVector.isNull[i]) {
+          if(inputColumnVector.vector[i] < scalarValue) {
+            assertEquals(
+              "Vector index that passes filter "
+              + inputColumnVector.vector[i] + "<"
+              + scalarValue + " is not in rowBatch selected index",
+              i,
+              rowBatch.selected[selectedIndex]);
+            selectedIndex++;
+          }
+        }
+      }
+    }
+
+    assertEquals("Row batch size not set to number of selected rows: " + selectedIndex,
+      selectedIndex, rowBatch.size);
+
+    if(selectedIndex > 0 && selectedIndex < BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should be set when > 0 and < entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        true, rowBatch.selectedInUse);
+    } else if(selectedIndex == BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should not be set when entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        false, rowBatch.selectedInUse);
+    }
+  }
+
+  @Test
+  public void testFilterDoubleColLessDoubleScalarColNullsRepeats() {
+
+    Random rand = new Random(SEED);
+
+    DoubleColumnVector inputColumnVector =
+      VectorizedRowGroupGenUtil.generateDoubleColumnVector(true,
+      true, BATCH_SIZE, rand);
+
+    VectorizedRowBatch rowBatch = new VectorizedRowBatch(1, BATCH_SIZE);
+    rowBatch.cols[0] = inputColumnVector;
+
+    double scalarValue = 0;
+    do {
+      scalarValue = rand.nextDouble();
+    } while(scalarValue == 0);
+
+    FilterDoubleColLessDoubleScalar vectorExpression =
+      new FilterDoubleColLessDoubleScalar(0, scalarValue);
+
+    vectorExpression.evaluate(rowBatch);
+
+    int selectedIndex = 0;
+    //check for isRepeating optimization
+    if(inputColumnVector.isRepeating) {
+      //null vector is safe to check, as it is always initialized to match the data vector
+      selectedIndex =
+        !inputColumnVector.isNull[0] && inputColumnVector.vector[0] < scalarValue
+          ? BATCH_SIZE : 0;
+    } else {
+      for(int i = 0; i < BATCH_SIZE; i++) {
+        if(!inputColumnVector.isNull[i]) {
+          if(inputColumnVector.vector[i] < scalarValue) {
+            assertEquals(
+              "Vector index that passes filter "
+              + inputColumnVector.vector[i] + "<"
+              + scalarValue + " is not in rowBatch selected index",
+              i,
+              rowBatch.selected[selectedIndex]);
+            selectedIndex++;
+          }
+        }
+      }
+    }
+
+    assertEquals("Row batch size not set to number of selected rows: " + selectedIndex,
+      selectedIndex, rowBatch.size);
+
+    if(selectedIndex > 0 && selectedIndex < BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should be set when > 0 and < entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        true, rowBatch.selectedInUse);
+    } else if(selectedIndex == BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should not be set when entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        false, rowBatch.selectedInUse);
+    }
+  }
+
+  @Test
+  public void testFilterDoubleColLessDoubleScalarColNulls() {
+
+    Random rand = new Random(SEED);
+
+    DoubleColumnVector inputColumnVector =
+      VectorizedRowGroupGenUtil.generateDoubleColumnVector(true,
+      false, BATCH_SIZE, rand);
+
+    VectorizedRowBatch rowBatch = new VectorizedRowBatch(1, BATCH_SIZE);
+    rowBatch.cols[0] = inputColumnVector;
+
+    double scalarValue = 0;
+    do {
+      scalarValue = rand.nextDouble();
+    } while(scalarValue == 0);
+
+    FilterDoubleColLessDoubleScalar vectorExpression =
+      new FilterDoubleColLessDoubleScalar(0, scalarValue);
+
+    vectorExpression.evaluate(rowBatch);
+
+    int selectedIndex = 0;
+    //check for isRepeating optimization
+    if(inputColumnVector.isRepeating) {
+      //null vector is safe to check, as it is always initialized to match the data vector
+      selectedIndex =
+        !inputColumnVector.isNull[0] && inputColumnVector.vector[0] < scalarValue
+          ? BATCH_SIZE : 0;
+    } else {
+      for(int i = 0; i < BATCH_SIZE; i++) {
+        if(!inputColumnVector.isNull[i]) {
+          if(inputColumnVector.vector[i] < scalarValue) {
+            assertEquals(
+              "Vector index that passes filter "
+              + inputColumnVector.vector[i] + "<"
+              + scalarValue + " is not in rowBatch selected index",
+              i,
+              rowBatch.selected[selectedIndex]);
+            selectedIndex++;
+          }
+        }
+      }
+    }
+
+    assertEquals("Row batch size not set to number of selected rows: " + selectedIndex,
+      selectedIndex, rowBatch.size);
+
+    if(selectedIndex > 0 && selectedIndex < BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should be set when > 0 and < entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        true, rowBatch.selectedInUse);
+    } else if(selectedIndex == BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should not be set when entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        false, rowBatch.selectedInUse);
+    }
+  }
+
+  @Test
+  public void testFilterDoubleColLessDoubleScalar() {
+
+    Random rand = new Random(SEED);
+
+    DoubleColumnVector inputColumnVector =
+      VectorizedRowGroupGenUtil.generateDoubleColumnVector(false,
+      false, BATCH_SIZE, rand);
+
+    VectorizedRowBatch rowBatch = new VectorizedRowBatch(1, BATCH_SIZE);
+    rowBatch.cols[0] = inputColumnVector;
+
+    double scalarValue = 0;
+    do {
+      scalarValue = rand.nextDouble();
+    } while(scalarValue == 0);
+
+    FilterDoubleColLessDoubleScalar vectorExpression =
+      new FilterDoubleColLessDoubleScalar(0, scalarValue);
+
+    vectorExpression.evaluate(rowBatch);
+
+    int selectedIndex = 0;
+    //check for isRepeating optimization
+    if(inputColumnVector.isRepeating) {
+      //null vector is safe to check, as it is always initialized to match the data vector
+      selectedIndex =
+        !inputColumnVector.isNull[0] && inputColumnVector.vector[0] < scalarValue
+          ? BATCH_SIZE : 0;
+    } else {
+      for(int i = 0; i < BATCH_SIZE; i++) {
+        if(!inputColumnVector.isNull[i]) {
+          if(inputColumnVector.vector[i] < scalarValue) {
+            assertEquals(
+              "Vector index that passes filter "
+              + inputColumnVector.vector[i] + "<"
+              + scalarValue + " is not in rowBatch selected index",
+              i,
+              rowBatch.selected[selectedIndex]);
+            selectedIndex++;
+          }
+        }
+      }
+    }
+
+    assertEquals("Row batch size not set to number of selected rows: " + selectedIndex,
+      selectedIndex, rowBatch.size);
+
+    if(selectedIndex > 0 && selectedIndex < BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should be set when > 0 and < entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        true, rowBatch.selectedInUse);
+    } else if(selectedIndex == BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should not be set when entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        false, rowBatch.selectedInUse);
+    }
+  }
+
+  @Test
+  public void testFilterDoubleColLessDoubleScalarColRepeats() {
+
+    Random rand = new Random(SEED);
+
+    DoubleColumnVector inputColumnVector =
+      VectorizedRowGroupGenUtil.generateDoubleColumnVector(false,
+      true, BATCH_SIZE, rand);
+
+    VectorizedRowBatch rowBatch = new VectorizedRowBatch(1, BATCH_SIZE);
+    rowBatch.cols[0] = inputColumnVector;
+
+    double scalarValue = 0;
+    do {
+      scalarValue = rand.nextDouble();
+    } while(scalarValue == 0);
+
+    FilterDoubleColLessDoubleScalar vectorExpression =
+      new FilterDoubleColLessDoubleScalar(0, scalarValue);
+
+    vectorExpression.evaluate(rowBatch);
+
+    int selectedIndex = 0;
+    //check for isRepeating optimization
+    if(inputColumnVector.isRepeating) {
+      //null vector is safe to check, as it is always initialized to match the data vector
+      selectedIndex =
+        !inputColumnVector.isNull[0] && inputColumnVector.vector[0] < scalarValue
+          ? BATCH_SIZE : 0;
+    } else {
+      for(int i = 0; i < BATCH_SIZE; i++) {
+        if(!inputColumnVector.isNull[i]) {
+          if(inputColumnVector.vector[i] < scalarValue) {
+            assertEquals(
+              "Vector index that passes filter "
+              + inputColumnVector.vector[i] + "<"
+              + scalarValue + " is not in rowBatch selected index",
+              i,
+              rowBatch.selected[selectedIndex]);
+            selectedIndex++;
+          }
+        }
+      }
+    }
+
+    assertEquals("Row batch size not set to number of selected rows: " + selectedIndex,
+      selectedIndex, rowBatch.size);
+
+    if(selectedIndex > 0 && selectedIndex < BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should be set when > 0 and < entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        true, rowBatch.selectedInUse);
+    } else if(selectedIndex == BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should not be set when entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        false, rowBatch.selectedInUse);
+    }
+  }
+
+  @Test
+  public void testFilterLongColLessEqualDoubleScalarColNullsRepeats() {
+
+    Random rand = new Random(SEED);
+
+    LongColumnVector inputColumnVector =
+      VectorizedRowGroupGenUtil.generateLongColumnVector(true,
+      true, BATCH_SIZE, rand);
+
+    VectorizedRowBatch rowBatch = new VectorizedRowBatch(1, BATCH_SIZE);
+    rowBatch.cols[0] = inputColumnVector;
+
+    double scalarValue = 0;
+    do {
+      scalarValue = rand.nextDouble();
+    } while(scalarValue == 0);
+
+    FilterLongColLessEqualDoubleScalar vectorExpression =
+      new FilterLongColLessEqualDoubleScalar(0, scalarValue);
+
+    vectorExpression.evaluate(rowBatch);
+
+    int selectedIndex = 0;
+    //check for isRepeating optimization
+    if(inputColumnVector.isRepeating) {
+      //null vector is safe to check, as it is always initialized to match the data vector
+      selectedIndex =
+        !inputColumnVector.isNull[0] && inputColumnVector.vector[0] <= scalarValue
+          ? BATCH_SIZE : 0;
+    } else {
+      for(int i = 0; i < BATCH_SIZE; i++) {
+        if(!inputColumnVector.isNull[i]) {
+          if(inputColumnVector.vector[i] <= scalarValue) {
+            assertEquals(
+              "Vector index that passes filter "
+              + inputColumnVector.vector[i] + "<="
+              + scalarValue + " is not in rowBatch selected index",
+              i,
+              rowBatch.selected[selectedIndex]);
+            selectedIndex++;
+          }
+        }
+      }
+    }
+
+    assertEquals("Row batch size not set to number of selected rows: " + selectedIndex,
+      selectedIndex, rowBatch.size);
+
+    if(selectedIndex > 0 && selectedIndex < BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should be set when > 0 and < entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        true, rowBatch.selectedInUse);
+    } else if(selectedIndex == BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should not be set when entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        false, rowBatch.selectedInUse);
+    }
+  }
+
+  @Test
+  public void testFilterLongColLessEqualDoubleScalarColNulls() {
+
+    Random rand = new Random(SEED);
+
+    LongColumnVector inputColumnVector =
+      VectorizedRowGroupGenUtil.generateLongColumnVector(true,
+      false, BATCH_SIZE, rand);
+
+    VectorizedRowBatch rowBatch = new VectorizedRowBatch(1, BATCH_SIZE);
+    rowBatch.cols[0] = inputColumnVector;
+
+    double scalarValue = 0;
+    do {
+      scalarValue = rand.nextDouble();
+    } while(scalarValue == 0);
+
+    FilterLongColLessEqualDoubleScalar vectorExpression =
+      new FilterLongColLessEqualDoubleScalar(0, scalarValue);
+
+    vectorExpression.evaluate(rowBatch);
+
+    int selectedIndex = 0;
+    //check for isRepeating optimization
+    if(inputColumnVector.isRepeating) {
+      //null vector is safe to check, as it is always initialized to match the data vector
+      selectedIndex =
+        !inputColumnVector.isNull[0] && inputColumnVector.vector[0] <= scalarValue
+          ? BATCH_SIZE : 0;
+    } else {
+      for(int i = 0; i < BATCH_SIZE; i++) {
+        if(!inputColumnVector.isNull[i]) {
+          if(inputColumnVector.vector[i] <= scalarValue) {
+            assertEquals(
+              "Vector index that passes filter "
+              + inputColumnVector.vector[i] + "<="
+              + scalarValue + " is not in rowBatch selected index",
+              i,
+              rowBatch.selected[selectedIndex]);
+            selectedIndex++;
+          }
+        }
+      }
+    }
+
+    assertEquals("Row batch size not set to number of selected rows: " + selectedIndex,
+      selectedIndex, rowBatch.size);
+
+    if(selectedIndex > 0 && selectedIndex < BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should be set when > 0 and < entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        true, rowBatch.selectedInUse);
+    } else if(selectedIndex == BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should not be set when entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        false, rowBatch.selectedInUse);
+    }
+  }
+
+  @Test
+  public void testFilterLongColLessEqualDoubleScalar() {
+
+    Random rand = new Random(SEED);
+
+    LongColumnVector inputColumnVector =
+      VectorizedRowGroupGenUtil.generateLongColumnVector(false,
+      false, BATCH_SIZE, rand);
+
+    VectorizedRowBatch rowBatch = new VectorizedRowBatch(1, BATCH_SIZE);
+    rowBatch.cols[0] = inputColumnVector;
+
+    double scalarValue = 0;
+    do {
+      scalarValue = rand.nextDouble();
+    } while(scalarValue == 0);
+
+    FilterLongColLessEqualDoubleScalar vectorExpression =
+      new FilterLongColLessEqualDoubleScalar(0, scalarValue);
+
+    vectorExpression.evaluate(rowBatch);
+
+    int selectedIndex = 0;
+    //check for isRepeating optimization
+    if(inputColumnVector.isRepeating) {
+      //null vector is safe to check, as it is always initialized to match the data vector
+      selectedIndex =
+        !inputColumnVector.isNull[0] && inputColumnVector.vector[0] <= scalarValue
+          ? BATCH_SIZE : 0;
+    } else {
+      for(int i = 0; i < BATCH_SIZE; i++) {
+        if(!inputColumnVector.isNull[i]) {
+          if(inputColumnVector.vector[i] <= scalarValue) {
+            assertEquals(
+              "Vector index that passes filter "
+              + inputColumnVector.vector[i] + "<="
+              + scalarValue + " is not in rowBatch selected index",
+              i,
+              rowBatch.selected[selectedIndex]);
+            selectedIndex++;
+          }
+        }
+      }
+    }
+
+    assertEquals("Row batch size not set to number of selected rows: " + selectedIndex,
+      selectedIndex, rowBatch.size);
+
+    if(selectedIndex > 0 && selectedIndex < BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should be set when > 0 and < entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        true, rowBatch.selectedInUse);
+    } else if(selectedIndex == BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should not be set when entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        false, rowBatch.selectedInUse);
+    }
+  }
+
+  @Test
+  public void testFilterLongColLessEqualDoubleScalarColRepeats() {
+
+    Random rand = new Random(SEED);
+
+    LongColumnVector inputColumnVector =
+      VectorizedRowGroupGenUtil.generateLongColumnVector(false,
+      true, BATCH_SIZE, rand);
+
+    VectorizedRowBatch rowBatch = new VectorizedRowBatch(1, BATCH_SIZE);
+    rowBatch.cols[0] = inputColumnVector;
+
+    double scalarValue = 0;
+    do {
+      scalarValue = rand.nextDouble();
+    } while(scalarValue == 0);
+
+    FilterLongColLessEqualDoubleScalar vectorExpression =
+      new FilterLongColLessEqualDoubleScalar(0, scalarValue);
+
+    vectorExpression.evaluate(rowBatch);
+
+    int selectedIndex = 0;
+    //check for isRepeating optimization
+    if(inputColumnVector.isRepeating) {
+      //null vector is safe to check, as it is always initialized to match the data vector
+      selectedIndex =
+        !inputColumnVector.isNull[0] && inputColumnVector.vector[0] <= scalarValue
+          ? BATCH_SIZE : 0;
+    } else {
+      for(int i = 0; i < BATCH_SIZE; i++) {
+        if(!inputColumnVector.isNull[i]) {
+          if(inputColumnVector.vector[i] <= scalarValue) {
+            assertEquals(
+              "Vector index that passes filter "
+              + inputColumnVector.vector[i] + "<="
+              + scalarValue + " is not in rowBatch selected index",
+              i,
+              rowBatch.selected[selectedIndex]);
+            selectedIndex++;
+          }
+        }
+      }
+    }
+
+    assertEquals("Row batch size not set to number of selected rows: " + selectedIndex,
+      selectedIndex, rowBatch.size);
+
+    if(selectedIndex > 0 && selectedIndex < BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should be set when > 0 and < entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        true, rowBatch.selectedInUse);
+    } else if(selectedIndex == BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should not be set when entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        false, rowBatch.selectedInUse);
+    }
+  }
+
+  @Test
+  public void testFilterDoubleColLessEqualDoubleScalarColNullsRepeats() {
+
+    Random rand = new Random(SEED);
+
+    DoubleColumnVector inputColumnVector =
+      VectorizedRowGroupGenUtil.generateDoubleColumnVector(true,
+      true, BATCH_SIZE, rand);
+
+    VectorizedRowBatch rowBatch = new VectorizedRowBatch(1, BATCH_SIZE);
+    rowBatch.cols[0] = inputColumnVector;
+
+    double scalarValue = 0;
+    do {
+      scalarValue = rand.nextDouble();
+    } while(scalarValue == 0);
+
+    FilterDoubleColLessEqualDoubleScalar vectorExpression =
+      new FilterDoubleColLessEqualDoubleScalar(0, scalarValue);
+
+    vectorExpression.evaluate(rowBatch);
+
+    int selectedIndex = 0;
+    //check for isRepeating optimization
+    if(inputColumnVector.isRepeating) {
+      //null vector is safe to check, as it is always initialized to match the data vector
+      selectedIndex =
+        !inputColumnVector.isNull[0] && inputColumnVector.vector[0] <= scalarValue
+          ? BATCH_SIZE : 0;
+    } else {
+      for(int i = 0; i < BATCH_SIZE; i++) {
+        if(!inputColumnVector.isNull[i]) {
+          if(inputColumnVector.vector[i] <= scalarValue) {
+            assertEquals(
+              "Vector index that passes filter "
+              + inputColumnVector.vector[i] + "<="
+              + scalarValue + " is not in rowBatch selected index",
+              i,
+              rowBatch.selected[selectedIndex]);
+            selectedIndex++;
+          }
+        }
+      }
+    }
+
+    assertEquals("Row batch size not set to number of selected rows: " + selectedIndex,
+      selectedIndex, rowBatch.size);
+
+    if(selectedIndex > 0 && selectedIndex < BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should be set when > 0 and < entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        true, rowBatch.selectedInUse);
+    } else if(selectedIndex == BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should not be set when entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        false, rowBatch.selectedInUse);
+    }
+  }
+
+  @Test
+  public void testFilterDoubleColLessEqualDoubleScalarColNulls() {
+
+    Random rand = new Random(SEED);
+
+    DoubleColumnVector inputColumnVector =
+      VectorizedRowGroupGenUtil.generateDoubleColumnVector(true,
+      false, BATCH_SIZE, rand);
+
+    VectorizedRowBatch rowBatch = new VectorizedRowBatch(1, BATCH_SIZE);
+    rowBatch.cols[0] = inputColumnVector;
+
+    double scalarValue = 0;
+    do {
+      scalarValue = rand.nextDouble();
+    } while(scalarValue == 0);
+
+    FilterDoubleColLessEqualDoubleScalar vectorExpression =
+      new FilterDoubleColLessEqualDoubleScalar(0, scalarValue);
+
+    vectorExpression.evaluate(rowBatch);
+
+    int selectedIndex = 0;
+    //check for isRepeating optimization
+    if(inputColumnVector.isRepeating) {
+      //null vector is safe to check, as it is always initialized to match the data vector
+      selectedIndex =
+        !inputColumnVector.isNull[0] && inputColumnVector.vector[0] <= scalarValue
+          ? BATCH_SIZE : 0;
+    } else {
+      for(int i = 0; i < BATCH_SIZE; i++) {
+        if(!inputColumnVector.isNull[i]) {
+          if(inputColumnVector.vector[i] <= scalarValue) {
+            assertEquals(
+              "Vector index that passes filter "
+              + inputColumnVector.vector[i] + "<="
+              + scalarValue + " is not in rowBatch selected index",
+              i,
+              rowBatch.selected[selectedIndex]);
+            selectedIndex++;
+          }
+        }
+      }
+    }
+
+    assertEquals("Row batch size not set to number of selected rows: " + selectedIndex,
+      selectedIndex, rowBatch.size);
+
+    if(selectedIndex > 0 && selectedIndex < BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should be set when > 0 and < entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        true, rowBatch.selectedInUse);
+    } else if(selectedIndex == BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should not be set when entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        false, rowBatch.selectedInUse);
+    }
+  }
+
+  @Test
+  public void testFilterDoubleColLessEqualDoubleScalar() {
+
+    Random rand = new Random(SEED);
+
+    DoubleColumnVector inputColumnVector =
+      VectorizedRowGroupGenUtil.generateDoubleColumnVector(false,
+      false, BATCH_SIZE, rand);
+
+    VectorizedRowBatch rowBatch = new VectorizedRowBatch(1, BATCH_SIZE);
+    rowBatch.cols[0] = inputColumnVector;
+
+    double scalarValue = 0;
+    do {
+      scalarValue = rand.nextDouble();
+    } while(scalarValue == 0);
+
+    FilterDoubleColLessEqualDoubleScalar vectorExpression =
+      new FilterDoubleColLessEqualDoubleScalar(0, scalarValue);
+
+    vectorExpression.evaluate(rowBatch);
+
+    int selectedIndex = 0;
+    //check for isRepeating optimization
+    if(inputColumnVector.isRepeating) {
+      //null vector is safe to check, as it is always initialized to match the data vector
+      selectedIndex =
+        !inputColumnVector.isNull[0] && inputColumnVector.vector[0] <= scalarValue
+          ? BATCH_SIZE : 0;
+    } else {
+      for(int i = 0; i < BATCH_SIZE; i++) {
+        if(!inputColumnVector.isNull[i]) {
+          if(inputColumnVector.vector[i] <= scalarValue) {
+            assertEquals(
+              "Vector index that passes filter "
+              + inputColumnVector.vector[i] + "<="
+              + scalarValue + " is not in rowBatch selected index",
+              i,
+              rowBatch.selected[selectedIndex]);
+            selectedIndex++;
+          }
+        }
+      }
+    }
+
+    assertEquals("Row batch size not set to number of selected rows: " + selectedIndex,
+      selectedIndex, rowBatch.size);
+
+    if(selectedIndex > 0 && selectedIndex < BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should be set when > 0 and < entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        true, rowBatch.selectedInUse);
+    } else if(selectedIndex == BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should not be set when entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        false, rowBatch.selectedInUse);
+    }
+  }
+
+  @Test
+  public void testFilterDoubleColLessEqualDoubleScalarColRepeats() {
+
+    Random rand = new Random(SEED);
+
+    DoubleColumnVector inputColumnVector =
+      VectorizedRowGroupGenUtil.generateDoubleColumnVector(false,
+      true, BATCH_SIZE, rand);
+
+    VectorizedRowBatch rowBatch = new VectorizedRowBatch(1, BATCH_SIZE);
+    rowBatch.cols[0] = inputColumnVector;
+
+    double scalarValue = 0;
+    do {
+      scalarValue = rand.nextDouble();
+    } while(scalarValue == 0);
+
+    FilterDoubleColLessEqualDoubleScalar vectorExpression =
+      new FilterDoubleColLessEqualDoubleScalar(0, scalarValue);
+
+    vectorExpression.evaluate(rowBatch);
+
+    int selectedIndex = 0;
+    //check for isRepeating optimization
+    if(inputColumnVector.isRepeating) {
+      //null vector is safe to check, as it is always initialized to match the data vector
+      selectedIndex =
+        !inputColumnVector.isNull[0] && inputColumnVector.vector[0] <= scalarValue
+          ? BATCH_SIZE : 0;
+    } else {
+      for(int i = 0; i < BATCH_SIZE; i++) {
+        if(!inputColumnVector.isNull[i]) {
+          if(inputColumnVector.vector[i] <= scalarValue) {
+            assertEquals(
+              "Vector index that passes filter "
+              + inputColumnVector.vector[i] + "<="
+              + scalarValue + " is not in rowBatch selected index",
+              i,
+              rowBatch.selected[selectedIndex]);
+            selectedIndex++;
+          }
+        }
+      }
+    }
+
+    assertEquals("Row batch size not set to number of selected rows: " + selectedIndex,
+      selectedIndex, rowBatch.size);
+
+    if(selectedIndex > 0 && selectedIndex < BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should be set when > 0 and < entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        true, rowBatch.selectedInUse);
+    } else if(selectedIndex == BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should not be set when entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        false, rowBatch.selectedInUse);
+    }
+  }
+
+  @Test
+  public void testFilterLongColGreaterDoubleScalarColNullsRepeats() {
+
+    Random rand = new Random(SEED);
+
+    LongColumnVector inputColumnVector =
+      VectorizedRowGroupGenUtil.generateLongColumnVector(true,
+      true, BATCH_SIZE, rand);
+
+    VectorizedRowBatch rowBatch = new VectorizedRowBatch(1, BATCH_SIZE);
+    rowBatch.cols[0] = inputColumnVector;
+
+    double scalarValue = 0;
+    do {
+      scalarValue = rand.nextDouble();
+    } while(scalarValue == 0);
+
+    FilterLongColGreaterDoubleScalar vectorExpression =
+      new FilterLongColGreaterDoubleScalar(0, scalarValue);
+
+    vectorExpression.evaluate(rowBatch);
+
+    int selectedIndex = 0;
+    //check for isRepeating optimization
+    if(inputColumnVector.isRepeating) {
+      //null vector is safe to check, as it is always initialized to match the data vector
+      selectedIndex =
+        !inputColumnVector.isNull[0] && inputColumnVector.vector[0] > scalarValue
+          ? BATCH_SIZE : 0;
+    } else {
+      for(int i = 0; i < BATCH_SIZE; i++) {
+        if(!inputColumnVector.isNull[i]) {
+          if(inputColumnVector.vector[i] > scalarValue) {
+            assertEquals(
+              "Vector index that passes filter "
+              + inputColumnVector.vector[i] + ">"
+              + scalarValue + " is not in rowBatch selected index",
+              i,
+              rowBatch.selected[selectedIndex]);
+            selectedIndex++;
+          }
+        }
+      }
+    }
+
+    assertEquals("Row batch size not set to number of selected rows: " + selectedIndex,
+      selectedIndex, rowBatch.size);
+
+    if(selectedIndex > 0 && selectedIndex < BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should be set when > 0 and < entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        true, rowBatch.selectedInUse);
+    } else if(selectedIndex == BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should not be set when entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        false, rowBatch.selectedInUse);
+    }
+  }
+
+  @Test
+  public void testFilterLongColGreaterDoubleScalarColNulls() {
+
+    Random rand = new Random(SEED);
+
+    LongColumnVector inputColumnVector =
+      VectorizedRowGroupGenUtil.generateLongColumnVector(true,
+      false, BATCH_SIZE, rand);
+
+    VectorizedRowBatch rowBatch = new VectorizedRowBatch(1, BATCH_SIZE);
+    rowBatch.cols[0] = inputColumnVector;
+
+    double scalarValue = 0;
+    do {
+      scalarValue = rand.nextDouble();
+    } while(scalarValue == 0);
+
+    FilterLongColGreaterDoubleScalar vectorExpression =
+      new FilterLongColGreaterDoubleScalar(0, scalarValue);
+
+    vectorExpression.evaluate(rowBatch);
+
+    int selectedIndex = 0;
+    //check for isRepeating optimization
+    if(inputColumnVector.isRepeating) {
+      //null vector is safe to check, as it is always initialized to match the data vector
+      selectedIndex =
+        !inputColumnVector.isNull[0] && inputColumnVector.vector[0] > scalarValue
+          ? BATCH_SIZE : 0;
+    } else {
+      for(int i = 0; i < BATCH_SIZE; i++) {
+        if(!inputColumnVector.isNull[i]) {
+          if(inputColumnVector.vector[i] > scalarValue) {
+            assertEquals(
+              "Vector index that passes filter "
+              + inputColumnVector.vector[i] + ">"
+              + scalarValue + " is not in rowBatch selected index",
+              i,
+              rowBatch.selected[selectedIndex]);
+            selectedIndex++;
+          }
+        }
+      }
+    }
+
+    assertEquals("Row batch size not set to number of selected rows: " + selectedIndex,
+      selectedIndex, rowBatch.size);
+
+    if(selectedIndex > 0 && selectedIndex < BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should be set when > 0 and < entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        true, rowBatch.selectedInUse);
+    } else if(selectedIndex == BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should not be set when entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        false, rowBatch.selectedInUse);
+    }
+  }
+
+  @Test
+  public void testFilterLongColGreaterDoubleScalar() {
+
+    Random rand = new Random(SEED);
+
+    LongColumnVector inputColumnVector =
+      VectorizedRowGroupGenUtil.generateLongColumnVector(false,
+      false, BATCH_SIZE, rand);
+
+    VectorizedRowBatch rowBatch = new VectorizedRowBatch(1, BATCH_SIZE);
+    rowBatch.cols[0] = inputColumnVector;
+
+    double scalarValue = 0;
+    do {
+      scalarValue = rand.nextDouble();
+    } while(scalarValue == 0);
+
+    FilterLongColGreaterDoubleScalar vectorExpression =
+      new FilterLongColGreaterDoubleScalar(0, scalarValue);
+
+    vectorExpression.evaluate(rowBatch);
+
+    int selectedIndex = 0;
+    //check for isRepeating optimization
+    if(inputColumnVector.isRepeating) {
+      //null vector is safe to check, as it is always initialized to match the data vector
+      selectedIndex =
+        !inputColumnVector.isNull[0] && inputColumnVector.vector[0] > scalarValue
+          ? BATCH_SIZE : 0;
+    } else {
+      for(int i = 0; i < BATCH_SIZE; i++) {
+        if(!inputColumnVector.isNull[i]) {
+          if(inputColumnVector.vector[i] > scalarValue) {
+            assertEquals(
+              "Vector index that passes filter "
+              + inputColumnVector.vector[i] + ">"
+              + scalarValue + " is not in rowBatch selected index",
+              i,
+              rowBatch.selected[selectedIndex]);
+            selectedIndex++;
+          }
+        }
+      }
+    }
+
+    assertEquals("Row batch size not set to number of selected rows: " + selectedIndex,
+      selectedIndex, rowBatch.size);
+
+    if(selectedIndex > 0 && selectedIndex < BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should be set when > 0 and < entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        true, rowBatch.selectedInUse);
+    } else if(selectedIndex == BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should not be set when entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        false, rowBatch.selectedInUse);
+    }
+  }
+
+  @Test
+  public void testFilterLongColGreaterDoubleScalarColRepeats() {
+
+    Random rand = new Random(SEED);
+
+    LongColumnVector inputColumnVector =
+      VectorizedRowGroupGenUtil.generateLongColumnVector(false,
+      true, BATCH_SIZE, rand);
+
+    VectorizedRowBatch rowBatch = new VectorizedRowBatch(1, BATCH_SIZE);
+    rowBatch.cols[0] = inputColumnVector;
+
+    double scalarValue = 0;
+    do {
+      scalarValue = rand.nextDouble();
+    } while(scalarValue == 0);
+
+    FilterLongColGreaterDoubleScalar vectorExpression =
+      new FilterLongColGreaterDoubleScalar(0, scalarValue);
+
+    vectorExpression.evaluate(rowBatch);
+
+    int selectedIndex = 0;
+    //check for isRepeating optimization
+    if(inputColumnVector.isRepeating) {
+      //null vector is safe to check, as it is always initialized to match the data vector
+      selectedIndex =
+        !inputColumnVector.isNull[0] && inputColumnVector.vector[0] > scalarValue
+          ? BATCH_SIZE : 0;
+    } else {
+      for(int i = 0; i < BATCH_SIZE; i++) {
+        if(!inputColumnVector.isNull[i]) {
+          if(inputColumnVector.vector[i] > scalarValue) {
+            assertEquals(
+              "Vector index that passes filter "
+              + inputColumnVector.vector[i] + ">"
+              + scalarValue + " is not in rowBatch selected index",
+              i,
+              rowBatch.selected[selectedIndex]);
+            selectedIndex++;
+          }
+        }
+      }
+    }
+
+    assertEquals("Row batch size not set to number of selected rows: " + selectedIndex,
+      selectedIndex, rowBatch.size);
+
+    if(selectedIndex > 0 && selectedIndex < BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should be set when > 0 and < entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        true, rowBatch.selectedInUse);
+    } else if(selectedIndex == BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should not be set when entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        false, rowBatch.selectedInUse);
+    }
+  }
+
+  @Test
+  public void testFilterDoubleColGreaterDoubleScalarColNullsRepeats() {
+
+    Random rand = new Random(SEED);
+
+    DoubleColumnVector inputColumnVector =
+      VectorizedRowGroupGenUtil.generateDoubleColumnVector(true,
+      true, BATCH_SIZE, rand);
+
+    VectorizedRowBatch rowBatch = new VectorizedRowBatch(1, BATCH_SIZE);
+    rowBatch.cols[0] = inputColumnVector;
+
+    double scalarValue = 0;
+    do {
+      scalarValue = rand.nextDouble();
+    } while(scalarValue == 0);
+
+    FilterDoubleColGreaterDoubleScalar vectorExpression =
+      new FilterDoubleColGreaterDoubleScalar(0, scalarValue);
+
+    vectorExpression.evaluate(rowBatch);
+
+    int selectedIndex = 0;
+    //check for isRepeating optimization
+    if(inputColumnVector.isRepeating) {
+      //null vector is safe to check, as it is always initialized to match the data vector
+      selectedIndex =
+        !inputColumnVector.isNull[0] && inputColumnVector.vector[0] > scalarValue
+          ? BATCH_SIZE : 0;
+    } else {
+      for(int i = 0; i < BATCH_SIZE; i++) {
+        if(!inputColumnVector.isNull[i]) {
+          if(inputColumnVector.vector[i] > scalarValue) {
+            assertEquals(
+              "Vector index that passes filter "
+              + inputColumnVector.vector[i] + ">"
+              + scalarValue + " is not in rowBatch selected index",
+              i,
+              rowBatch.selected[selectedIndex]);
+            selectedIndex++;
+          }
+        }
+      }
+    }
+
+    assertEquals("Row batch size not set to number of selected rows: " + selectedIndex,
+      selectedIndex, rowBatch.size);
+
+    if(selectedIndex > 0 && selectedIndex < BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should be set when > 0 and < entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        true, rowBatch.selectedInUse);
+    } else if(selectedIndex == BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should not be set when entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        false, rowBatch.selectedInUse);
+    }
+  }
+
+  @Test
+  public void testFilterDoubleColGreaterDoubleScalarColNulls() {
+
+    Random rand = new Random(SEED);
+
+    DoubleColumnVector inputColumnVector =
+      VectorizedRowGroupGenUtil.generateDoubleColumnVector(true,
+      false, BATCH_SIZE, rand);
+
+    VectorizedRowBatch rowBatch = new VectorizedRowBatch(1, BATCH_SIZE);
+    rowBatch.cols[0] = inputColumnVector;
+
+    double scalarValue = 0;
+    do {
+      scalarValue = rand.nextDouble();
+    } while(scalarValue == 0);
+
+    FilterDoubleColGreaterDoubleScalar vectorExpression =
+      new FilterDoubleColGreaterDoubleScalar(0, scalarValue);
+
+    vectorExpression.evaluate(rowBatch);
+
+    int selectedIndex = 0;
+    //check for isRepeating optimization
+    if(inputColumnVector.isRepeating) {
+      //null vector is safe to check, as it is always initialized to match the data vector
+      selectedIndex =
+        !inputColumnVector.isNull[0] && inputColumnVector.vector[0] > scalarValue
+          ? BATCH_SIZE : 0;
+    } else {
+      for(int i = 0; i < BATCH_SIZE; i++) {
+        if(!inputColumnVector.isNull[i]) {
+          if(inputColumnVector.vector[i] > scalarValue) {
+            assertEquals(
+              "Vector index that passes filter "
+              + inputColumnVector.vector[i] + ">"
+              + scalarValue + " is not in rowBatch selected index",
+              i,
+              rowBatch.selected[selectedIndex]);
+            selectedIndex++;
+          }
+        }
+      }
+    }
+
+    assertEquals("Row batch size not set to number of selected rows: " + selectedIndex,
+      selectedIndex, rowBatch.size);
+
+    if(selectedIndex > 0 && selectedIndex < BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should be set when > 0 and < entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        true, rowBatch.selectedInUse);
+    } else if(selectedIndex == BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should not be set when entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        false, rowBatch.selectedInUse);
+    }
+  }
+
+  @Test
+  public void testFilterDoubleColGreaterDoubleScalar() {
+
+    Random rand = new Random(SEED);
+
+    DoubleColumnVector inputColumnVector =
+      VectorizedRowGroupGenUtil.generateDoubleColumnVector(false,
+      false, BATCH_SIZE, rand);
+
+    VectorizedRowBatch rowBatch = new VectorizedRowBatch(1, BATCH_SIZE);
+    rowBatch.cols[0] = inputColumnVector;
+
+    double scalarValue = 0;
+    do {
+      scalarValue = rand.nextDouble();
+    } while(scalarValue == 0);
+
+    FilterDoubleColGreaterDoubleScalar vectorExpression =
+      new FilterDoubleColGreaterDoubleScalar(0, scalarValue);
+
+    vectorExpression.evaluate(rowBatch);
+
+    int selectedIndex = 0;
+    //check for isRepeating optimization
+    if(inputColumnVector.isRepeating) {
+      //null vector is safe to check, as it is always initialized to match the data vector
+      selectedIndex =
+        !inputColumnVector.isNull[0] && inputColumnVector.vector[0] > scalarValue
+          ? BATCH_SIZE : 0;
+    } else {
+      for(int i = 0; i < BATCH_SIZE; i++) {
+        if(!inputColumnVector.isNull[i]) {
+          if(inputColumnVector.vector[i] > scalarValue) {
+            assertEquals(
+              "Vector index that passes filter "
+              + inputColumnVector.vector[i] + ">"
+              + scalarValue + " is not in rowBatch selected index",
+              i,
+              rowBatch.selected[selectedIndex]);
+            selectedIndex++;
+          }
+        }
+      }
+    }
+
+    assertEquals("Row batch size not set to number of selected rows: " + selectedIndex,
+      selectedIndex, rowBatch.size);
+
+    if(selectedIndex > 0 && selectedIndex < BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should be set when > 0 and < entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        true, rowBatch.selectedInUse);
+    } else if(selectedIndex == BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should not be set when entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        false, rowBatch.selectedInUse);
+    }
+  }
+
+  @Test
+  public void testFilterDoubleColGreaterDoubleScalarColRepeats() {
+
+    Random rand = new Random(SEED);
+
+    DoubleColumnVector inputColumnVector =
+      VectorizedRowGroupGenUtil.generateDoubleColumnVector(false,
+      true, BATCH_SIZE, rand);
+
+    VectorizedRowBatch rowBatch = new VectorizedRowBatch(1, BATCH_SIZE);
+    rowBatch.cols[0] = inputColumnVector;
+
+    double scalarValue = 0;
+    do {
+      scalarValue = rand.nextDouble();
+    } while(scalarValue == 0);
+
+    FilterDoubleColGreaterDoubleScalar vectorExpression =
+      new FilterDoubleColGreaterDoubleScalar(0, scalarValue);
+
+    vectorExpression.evaluate(rowBatch);
+
+    int selectedIndex = 0;
+    //check for isRepeating optimization
+    if(inputColumnVector.isRepeating) {
+      //null vector is safe to check, as it is always initialized to match the data vector
+      selectedIndex =
+        !inputColumnVector.isNull[0] && inputColumnVector.vector[0] > scalarValue
+          ? BATCH_SIZE : 0;
+    } else {
+      for(int i = 0; i < BATCH_SIZE; i++) {
+        if(!inputColumnVector.isNull[i]) {
+          if(inputColumnVector.vector[i] > scalarValue) {
+            assertEquals(
+              "Vector index that passes filter "
+              + inputColumnVector.vector[i] + ">"
+              + scalarValue + " is not in rowBatch selected index",
+              i,
+              rowBatch.selected[selectedIndex]);
+            selectedIndex++;
+          }
+        }
+      }
+    }
+
+    assertEquals("Row batch size not set to number of selected rows: " + selectedIndex,
+      selectedIndex, rowBatch.size);
+
+    if(selectedIndex > 0 && selectedIndex < BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should be set when > 0 and < entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        true, rowBatch.selectedInUse);
+    } else if(selectedIndex == BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should not be set when entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        false, rowBatch.selectedInUse);
+    }
+  }
+
+  @Test
+  public void testFilterLongColGreaterEqualDoubleScalarColNullsRepeats() {
+
+    Random rand = new Random(SEED);
+
+    LongColumnVector inputColumnVector =
+      VectorizedRowGroupGenUtil.generateLongColumnVector(true,
+      true, BATCH_SIZE, rand);
+
+    VectorizedRowBatch rowBatch = new VectorizedRowBatch(1, BATCH_SIZE);
+    rowBatch.cols[0] = inputColumnVector;
+
+    double scalarValue = 0;
+    do {
+      scalarValue = rand.nextDouble();
+    } while(scalarValue == 0);
+
+    FilterLongColGreaterEqualDoubleScalar vectorExpression =
+      new FilterLongColGreaterEqualDoubleScalar(0, scalarValue);
+
+    vectorExpression.evaluate(rowBatch);
+
+    int selectedIndex = 0;
+    //check for isRepeating optimization
+    if(inputColumnVector.isRepeating) {
+      //null vector is safe to check, as it is always initialized to match the data vector
+      selectedIndex =
+        !inputColumnVector.isNull[0] && inputColumnVector.vector[0] >= scalarValue
+          ? BATCH_SIZE : 0;
+    } else {
+      for(int i = 0; i < BATCH_SIZE; i++) {
+        if(!inputColumnVector.isNull[i]) {
+          if(inputColumnVector.vector[i] >= scalarValue) {
+            assertEquals(
+              "Vector index that passes filter "
+              + inputColumnVector.vector[i] + ">="
+              + scalarValue + " is not in rowBatch selected index",
+              i,
+              rowBatch.selected[selectedIndex]);
+            selectedIndex++;
+          }
+        }
+      }
+    }
+
+    assertEquals("Row batch size not set to number of selected rows: " + selectedIndex,
+      selectedIndex, rowBatch.size);
+
+    if(selectedIndex > 0 && selectedIndex < BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should be set when > 0 and < entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        true, rowBatch.selectedInUse);
+    } else if(selectedIndex == BATCH_SIZE) {
+      assertEquals(
+        "selectedInUse should not be set when entire batch(" + BATCH_SIZE + ") is selected: "
+        + selectedIndex,
+        false, rowBatch.selectedInUse);
+    }
+  }
+
+  @Test
+  public void testFilterLongColGreaterEqualDoubleScalarColNulls() {
+
+    Random rand = new Random(SEED);
+
+    LongColumnVector inputColumnVector =
+      VectorizedRowGroupGenUtil.generateLongColumnVector(true,
+      false, BATCH_SIZE, rand);
+
+    VectorizedRowBatch rowBatch = new VectorizedRowBatch(1, BATCH_SIZE);
+    rowBatch.cols[0] = inputColumnVector;
+
+    double scalarValue = 0;
+    do {
+      scalarValue = rand.nextDouble();
+    } while(scalarValue == 0);
+
+    FilterLongColGreaterEqualDoubleScalar vectorExpression =
+      new FilterLongColGreaterEqualDoubleScalar(0, scalarValue);
+
+    vectorExpression.evaluate(rowBatch);
+
+    int selectedIndex = 0;
+    //check for isRepeating optimization
+    if(inputColumnVector.isRepeating) {
+      //null vector is safe to check, as it is always initialized to match the data vector
+      selectedIndex =
+        !inputColumnVector.isNull[0] && inputColumnVector.vector[0] >= scalarValue
+          ? BATCH_SIZE : 0;
+    } else {

[... 3330 lines stripped ...]