You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by mm...@apache.org on 2016/04/10 08:02:36 UTC

[13/16] hive git commit: HIVE-9862 Vectorized execution corrupts timestamp values (Matt McCline, reviewed by Jason Dere)

http://git-wip-us.apache.org/repos/asf/hive/blob/4a479d0c/ql/src/gen/vectorization/ExpressionTemplates/FilterDecimalColumnCompareColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/FilterDecimalColumnCompareColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/FilterDecimalColumnCompareColumn.txt
deleted file mode 100644
index 353e849..0000000
--- a/ql/src/gen/vectorization/ExpressionTemplates/FilterDecimalColumnCompareColumn.txt
+++ /dev/null
@@ -1,445 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hive.ql.exec.vector.expressions.gen;
-
-import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
-import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
-import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
-import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
-
-/**
- * Generated from template FilterDecimalColumnCompareColumn.txt, which covers binary comparison 
- * filter expressions between two columns. Output is not produced in a separate column. 
- * The selected vector of the input {@link VectorizedRowBatch} is updated for in-place filtering.
- */
-public class <ClassName> extends VectorExpression {
-
-  private static final long serialVersionUID = 1L;
-
-  private int colNum1;
-  private int colNum2;
-
-  public <ClassName>(int colNum1, int colNum2) { 
-    this.colNum1 = colNum1;
-    this.colNum2 = colNum2;
-  }
-
-  public <ClassName>() {
-  }
-
-  @Override
-  public void evaluate(VectorizedRowBatch batch) {
-
-    if (childExpressions != null) {
-      super.evaluateChildren(batch);
-    }
-
-    DecimalColumnVector inputColVector1 = (DecimalColumnVector) batch.cols[colNum1];
-    DecimalColumnVector inputColVector2 = (DecimalColumnVector) batch.cols[colNum2];
-    int[] sel = batch.selected;
-    boolean[] nullPos1 = inputColVector1.isNull;
-    boolean[] nullPos2 = inputColVector2.isNull;
-    int n = batch.size;
-    HiveDecimalWritable[] vector1 = inputColVector1.vector;
-    HiveDecimalWritable[] vector2 = inputColVector2.vector;
-
-    // return immediately if batch is empty
-    if (n == 0) {
-      return;
-    }
-
-    // handle case where neither input has nulls
-    if (inputColVector1.noNulls && inputColVector2.noNulls) {
-      if (inputColVector1.isRepeating && inputColVector2.isRepeating) {
-
-        /* Either all must remain selected or all will be eliminated.
-         * Repeating property will not change.
-         */
-        if (!(vector1[0].compareTo(vector2[0]) <OperatorSymbol> 0)) {
-          batch.size = 0;
-        }
-      } else if (inputColVector1.isRepeating) {
-        if (batch.selectedInUse) {
-          int newSize = 0;
-          for(int j = 0; j != n; j++) {
-            int i = sel[j];
-            if (vector1[0].compareTo(vector2[i]) <OperatorSymbol> 0) {
-              sel[newSize++] = i;
-            }
-          }
-          batch.size = newSize;
-        } else {
-          int newSize = 0;
-          for(int i = 0; i != n; i++) {
-            if (vector1[0].compareTo(vector2[i]) <OperatorSymbol> 0) {
-              sel[newSize++] = i;
-            }
-          }
-          if (newSize < batch.size) {
-            batch.size = newSize;
-            batch.selectedInUse = true;
-          }
-        }
-      } else if (inputColVector2.isRepeating) {
-        if (batch.selectedInUse) {
-          int newSize = 0;
-          for(int j = 0; j != n; j++) {
-            int i = sel[j];
-            if (vector1[i].compareTo(vector2[0]) <OperatorSymbol> 0) {
-              sel[newSize++] = i;
-            }
-          }
-          batch.size = newSize;
-        } else {
-          int newSize = 0;
-          for(int i = 0; i != n; i++) {
-            if (vector1[i].compareTo(vector2[0]) <OperatorSymbol> 0) {
-              sel[newSize++] = i;
-            }
-          }
-          if (newSize < batch.size) {
-            batch.size = newSize;
-            batch.selectedInUse = true;
-          }
-        }
-      } else if (batch.selectedInUse) {
-        int newSize = 0;
-        for(int j = 0; j != n; j++) {
-          int i = sel[j];
-          if (vector1[i].compareTo(vector2[i]) <OperatorSymbol> 0) {
-            sel[newSize++] = i;
-          }
-        }
-        batch.size = newSize;
-      } else {
-        int newSize = 0;
-        for(int i = 0; i != n; i++) {
-          if (vector1[i].compareTo(vector2[i]) <OperatorSymbol> 0) {
-            sel[newSize++] = i;
-          }
-        }
-        if (newSize < batch.size) {
-          batch.size = newSize;
-          batch.selectedInUse = true;
-        }
-      }
-
-    // handle case where only input 2 has nulls
-    } else if (inputColVector1.noNulls) {
-      if (inputColVector1.isRepeating && inputColVector2.isRepeating) {
-        if (nullPos2[0] ||
-            !(vector1[0].compareTo(vector2[0]) <OperatorSymbol> 0)) {
-          batch.size = 0; 
-        } 
-      } else if (inputColVector1.isRepeating) {
-
-         // no need to check for nulls in input 1
-         if (batch.selectedInUse) {
-          int newSize = 0;
-          for(int j = 0; j != n; j++) {
-            int i = sel[j];
-            if (!nullPos2[i]) {
-              if (vector1[0].compareTo(vector2[i]) <OperatorSymbol> 0) {
-                sel[newSize++] = i;
-              }
-            }
-          }
-          batch.size = newSize;
-        } else {
-          int newSize = 0;
-          for(int i = 0; i != n; i++) {
-            if (!nullPos2[i]) {
-              if (vector1[0].compareTo(vector2[i]) <OperatorSymbol> 0) {
-                sel[newSize++] = i;
-              }
-            }
-          }
-          if (newSize < batch.size) {
-            batch.size = newSize;
-            batch.selectedInUse = true;
-          }
-        }
-      } else if (inputColVector2.isRepeating) {
-        if (nullPos2[0]) {
-
-          // no values will qualify because every comparison will be with NULL
-          batch.size = 0;
-          return;
-        }
-        if (batch.selectedInUse) {
-          int newSize = 0;
-          for(int j = 0; j != n; j++) {
-            int i = sel[j];
-            if (vector1[i].compareTo(vector2[0]) <OperatorSymbol> 0) {
-              sel[newSize++] = i;
-            }
-          }
-          batch.size = newSize;
-        } else {
-          int newSize = 0;
-          for(int i = 0; i != n; i++) {
-            if (vector1[i].compareTo(vector2[0]) <OperatorSymbol> 0) {
-              sel[newSize++] = i;
-            }
-          }
-          if (newSize < batch.size) {
-            batch.size = newSize;
-            batch.selectedInUse = true;
-          }
-        }
-      } else { // neither input is repeating
-        if (batch.selectedInUse) {
-          int newSize = 0;
-          for(int j = 0; j != n; j++) {
-            int i = sel[j];
-            if (!nullPos2[i]) {
-              if (vector1[i].compareTo(vector2[i]) <OperatorSymbol> 0) {
-                sel[newSize++] = i;
-              }
-            }
-          }
-          batch.size = newSize;
-        } else {
-          int newSize = 0;
-          for(int i = 0; i != n; i++) {
-            if (!nullPos2[i]) {
-              if (vector1[i].compareTo(vector2[i]) <OperatorSymbol> 0) {
-                sel[newSize++] = i;
-              }
-            }
-          }
-          if (newSize < batch.size) {
-            batch.size = newSize;
-            batch.selectedInUse = true;
-          }
-        }
-      }
-
-    // handle case where only input 1 has nulls
-    } else if (inputColVector2.noNulls) {
-      if (inputColVector1.isRepeating && inputColVector2.isRepeating) {
-        if (nullPos1[0] ||
-            !(vector1[0].compareTo(vector2[0]) <OperatorSymbol> 0)) {
-          batch.size = 0; 
-          return;
-        } 
-      } else if (inputColVector1.isRepeating) {
-        if (nullPos1[0]) {
-
-          // if repeating value is null then every comparison will fail so nothing qualifies
-          batch.size = 0;
-          return; 
-        }
-        if (batch.selectedInUse) {
-          int newSize = 0;
-          for(int j = 0; j != n; j++) {
-            int i = sel[j];
-            if (vector1[0].compareTo(vector2[i]) <OperatorSymbol> 0) {
-              sel[newSize++] = i;
-            }
-          }
-          batch.size = newSize;
-        } else {
-          int newSize = 0;
-          for(int i = 0; i != n; i++) {
-            if (vector1[0].compareTo(vector2[i]) <OperatorSymbol> 0) {
-              sel[newSize++] = i;
-            }
-          }
-          if (newSize < batch.size) {
-            batch.size = newSize;
-            batch.selectedInUse = true;
-          }
-        }
-      } else if (inputColVector2.isRepeating) {
-         if (batch.selectedInUse) {
-          int newSize = 0;
-          for(int j = 0; j != n; j++) {
-            int i = sel[j];
-            if (!nullPos1[i]) {
-              if (vector1[i].compareTo(vector2[0]) <OperatorSymbol> 0) {
-                sel[newSize++] = i;
-              }
-            }
-          }
-          batch.size = newSize;
-        } else {
-          int newSize = 0;
-          for(int i = 0; i != n; i++) {
-            if (!nullPos1[i]) {
-              if (vector1[i].compareTo(vector2[0]) <OperatorSymbol> 0) {
-                sel[newSize++] = i;
-              }
-            }
-          }
-          if (newSize < batch.size) {
-            batch.size = newSize;
-            batch.selectedInUse = true;
-          }
-        }
-      } else { // neither input is repeating
-         if (batch.selectedInUse) {
-          int newSize = 0;
-          for(int j = 0; j != n; j++) {
-            int i = sel[j];
-            if (!nullPos1[i]) {
-              if (vector1[i].compareTo(vector2[i]) <OperatorSymbol> 0) {
-                sel[newSize++] = i;
-              }
-            }
-          }
-          batch.size = newSize;
-        } else {
-          int newSize = 0;
-          for(int i = 0; i != n; i++) {
-            if (!nullPos1[i]) {
-              if (vector1[i].compareTo(vector2[i]) <OperatorSymbol> 0) {
-                sel[newSize++] = i;
-              }
-            }
-          }
-          if (newSize < batch.size) {
-            batch.size = newSize;
-            batch.selectedInUse = true;
-          }
-        }
-      }
-
-    // handle case where both inputs have nulls
-    } else {
-      if (inputColVector1.isRepeating && inputColVector2.isRepeating) {
-        if (nullPos1[0] || nullPos2[0] ||
-            !(vector1[0].compareTo(vector2[0]) <OperatorSymbol> 0)) {
-          batch.size = 0; 
-        } 
-      } else if (inputColVector1.isRepeating) {
-         if (nullPos1[0]) {
-           batch.size = 0;
-           return;
-         }
-         if (batch.selectedInUse) {
-          int newSize = 0;
-          for(int j = 0; j != n; j++) {
-            int i = sel[j];
-            if (!nullPos2[i]) {
-              if (vector1[0].compareTo(vector2[i]) <OperatorSymbol> 0) {
-                sel[newSize++] = i;
-              }
-            }
-          }
-          batch.size = newSize;
-        } else {
-          int newSize = 0;
-          for(int i = 0; i != n; i++) {
-            if (!nullPos2[i]) {
-              if (vector1[0].compareTo(vector2[i]) <OperatorSymbol> 0) {
-                sel[newSize++] = i;
-              }
-            }
-          }
-          if (newSize < batch.size) {
-            batch.size = newSize;
-            batch.selectedInUse = true;
-          }
-        }
-      } else if (inputColVector2.isRepeating) {
-        if (nullPos2[0]) {
-          batch.size = 0;
-          return;
-        }
-        if (batch.selectedInUse) {
-          int newSize = 0;
-          for(int j = 0; j != n; j++) {
-            int i = sel[j];
-            if (!nullPos1[i]) {
-              if (vector1[i].compareTo(vector2[0]) <OperatorSymbol> 0) {
-                sel[newSize++] = i;
-              }
-            }
-          }
-          batch.size = newSize;
-        } else {
-          int newSize = 0;
-          for(int i = 0; i != n; i++) {
-            if (!nullPos1[i]) {
-              if (vector1[i].compareTo(vector2[0]) <OperatorSymbol> 0) {
-                sel[newSize++] = i;
-              }
-            }
-          }
-          if (newSize < batch.size) {
-            batch.size = newSize;
-            batch.selectedInUse = true;
-          }
-        }
-      } else { // neither input is repeating
-         if (batch.selectedInUse) {
-          int newSize = 0;
-          for(int j = 0; j != n; j++) {
-            int i = sel[j];
-            if (!nullPos1[i] && !nullPos2[i]) {
-              if (vector1[i].compareTo(vector2[i]) <OperatorSymbol> 0) {
-                sel[newSize++] = i;
-              }
-            }
-          }
-          batch.size = newSize;
-        } else {
-          int newSize = 0;
-          for(int i = 0; i != n; i++) {
-            if (!nullPos1[i] && !nullPos2[i]) {
-              if (vector1[i].compareTo(vector2[i]) <OperatorSymbol> 0) {
-                sel[newSize++] = i;
-              }
-            }
-          }
-          if (newSize < batch.size) {
-            batch.size = newSize;
-            batch.selectedInUse = true;
-          }
-        }
-      } 
-    }
-  }
-
-  @Override
-  public String getOutputType() {
-    return "boolean";
-  }
-
-  @Override
-  public int getOutputColumn() {
-    return -1;
-  }
-
-  @Override
-  public VectorExpressionDescriptor.Descriptor getDescriptor() {
-    return (new VectorExpressionDescriptor.Builder())
-        .setMode(
-            VectorExpressionDescriptor.Mode.FILTER)
-        .setNumArguments(2)
-        .setArgumentTypes(
-            VectorExpressionDescriptor.ArgumentType.getType("decimal"),
-            VectorExpressionDescriptor.ArgumentType.getType("decimal"))
-        .setInputExpressionTypes(
-            VectorExpressionDescriptor.InputExpressionType.COLUMN,
-            VectorExpressionDescriptor.InputExpressionType.COLUMN).build();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/4a479d0c/ql/src/gen/vectorization/ExpressionTemplates/FilterDecimalColumnCompareDecimalColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/FilterDecimalColumnCompareDecimalColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/FilterDecimalColumnCompareDecimalColumn.txt
new file mode 100644
index 0000000..a2352c6
--- /dev/null
+++ b/ql/src/gen/vectorization/ExpressionTemplates/FilterDecimalColumnCompareDecimalColumn.txt
@@ -0,0 +1,445 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.exec.vector.expressions.gen;
+
+import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+
+/**
+ * Generated from template FilterDecimalColumnCompareColumn.txt, which covers binary comparison
+ * filter expressions between two columns. Output is not produced in a separate column.
+ * The selected vector of the input {@link VectorizedRowBatch} is updated for in-place filtering.
+ */
+public class <ClassName> extends VectorExpression {
+
+  private static final long serialVersionUID = 1L;
+
+  private int colNum1;
+  private int colNum2;
+
+  public <ClassName>(int colNum1, int colNum2) {
+    this.colNum1 = colNum1;
+    this.colNum2 = colNum2;
+  }
+
+  public <ClassName>() {
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    DecimalColumnVector inputColVector1 = (DecimalColumnVector) batch.cols[colNum1];
+    DecimalColumnVector inputColVector2 = (DecimalColumnVector) batch.cols[colNum2];
+    int[] sel = batch.selected;
+    boolean[] nullPos1 = inputColVector1.isNull;
+    boolean[] nullPos2 = inputColVector2.isNull;
+    int n = batch.size;
+    HiveDecimalWritable[] vector1 = inputColVector1.vector;
+    HiveDecimalWritable[] vector2 = inputColVector2.vector;
+
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+
+    // handle case where neither input has nulls
+    if (inputColVector1.noNulls && inputColVector2.noNulls) {
+      if (inputColVector1.isRepeating && inputColVector2.isRepeating) {
+
+        /* Either all must remain selected or all will be eliminated.
+         * Repeating property will not change.
+         */
+        if (!(vector1[0].compareTo(vector2[0]) <OperatorSymbol> 0)) {
+          batch.size = 0;
+        }
+      } else if (inputColVector1.isRepeating) {
+        if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            if (vector1[0].compareTo(vector2[i]) <OperatorSymbol> 0) {
+              sel[newSize++] = i;
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (vector1[0].compareTo(vector2[i]) <OperatorSymbol> 0) {
+              sel[newSize++] = i;
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }
+      } else if (inputColVector2.isRepeating) {
+        if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            if (vector1[i].compareTo(vector2[0]) <OperatorSymbol> 0) {
+              sel[newSize++] = i;
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (vector1[i].compareTo(vector2[0]) <OperatorSymbol> 0) {
+              sel[newSize++] = i;
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }
+      } else if (batch.selectedInUse) {
+        int newSize = 0;
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          if (vector1[i].compareTo(vector2[i]) <OperatorSymbol> 0) {
+            sel[newSize++] = i;
+          }
+        }
+        batch.size = newSize;
+      } else {
+        int newSize = 0;
+        for(int i = 0; i != n; i++) {
+          if (vector1[i].compareTo(vector2[i]) <OperatorSymbol> 0) {
+            sel[newSize++] = i;
+          }
+        }
+        if (newSize < batch.size) {
+          batch.size = newSize;
+          batch.selectedInUse = true;
+        }
+      }
+
+    // handle case where only input 2 has nulls
+    } else if (inputColVector1.noNulls) {
+      if (inputColVector1.isRepeating && inputColVector2.isRepeating) {
+        if (nullPos2[0] ||
+            !(vector1[0].compareTo(vector2[0]) <OperatorSymbol> 0)) {
+          batch.size = 0;
+        }
+      } else if (inputColVector1.isRepeating) {
+
+         // no need to check for nulls in input 1
+         if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            if (!nullPos2[i]) {
+              if (vector1[0].compareTo(vector2[i]) <OperatorSymbol> 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (!nullPos2[i]) {
+              if (vector1[0].compareTo(vector2[i]) <OperatorSymbol> 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }
+      } else if (inputColVector2.isRepeating) {
+        if (nullPos2[0]) {
+
+          // no values will qualify because every comparison will be with NULL
+          batch.size = 0;
+          return;
+        }
+        if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            if (vector1[i].compareTo(vector2[0]) <OperatorSymbol> 0) {
+              sel[newSize++] = i;
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (vector1[i].compareTo(vector2[0]) <OperatorSymbol> 0) {
+              sel[newSize++] = i;
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }
+      } else { // neither input is repeating
+        if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            if (!nullPos2[i]) {
+              if (vector1[i].compareTo(vector2[i]) <OperatorSymbol> 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (!nullPos2[i]) {
+              if (vector1[i].compareTo(vector2[i]) <OperatorSymbol> 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }
+      }
+
+    // handle case where only input 1 has nulls
+    } else if (inputColVector2.noNulls) {
+      if (inputColVector1.isRepeating && inputColVector2.isRepeating) {
+        if (nullPos1[0] ||
+            !(vector1[0].compareTo(vector2[0]) <OperatorSymbol> 0)) {
+          batch.size = 0;
+          return;
+        }
+      } else if (inputColVector1.isRepeating) {
+        if (nullPos1[0]) {
+
+          // if repeating value is null then every comparison will fail so nothing qualifies
+          batch.size = 0;
+          return;
+        }
+        if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            if (vector1[0].compareTo(vector2[i]) <OperatorSymbol> 0) {
+              sel[newSize++] = i;
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (vector1[0].compareTo(vector2[i]) <OperatorSymbol> 0) {
+              sel[newSize++] = i;
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }
+      } else if (inputColVector2.isRepeating) {
+         if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            if (!nullPos1[i]) {
+              if (vector1[i].compareTo(vector2[0]) <OperatorSymbol> 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (!nullPos1[i]) {
+              if (vector1[i].compareTo(vector2[0]) <OperatorSymbol> 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }
+      } else { // neither input is repeating
+         if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            if (!nullPos1[i]) {
+              if (vector1[i].compareTo(vector2[i]) <OperatorSymbol> 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (!nullPos1[i]) {
+              if (vector1[i].compareTo(vector2[i]) <OperatorSymbol> 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }
+      }
+
+    // handle case where both inputs have nulls
+    } else {
+      if (inputColVector1.isRepeating && inputColVector2.isRepeating) {
+        if (nullPos1[0] || nullPos2[0] ||
+            !(vector1[0].compareTo(vector2[0]) <OperatorSymbol> 0)) {
+          batch.size = 0;
+        }
+      } else if (inputColVector1.isRepeating) {
+         if (nullPos1[0]) {
+           batch.size = 0;
+           return;
+         }
+         if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            if (!nullPos2[i]) {
+              if (vector1[0].compareTo(vector2[i]) <OperatorSymbol> 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (!nullPos2[i]) {
+              if (vector1[0].compareTo(vector2[i]) <OperatorSymbol> 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }
+      } else if (inputColVector2.isRepeating) {
+        if (nullPos2[0]) {
+          batch.size = 0;
+          return;
+        }
+        if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            if (!nullPos1[i]) {
+              if (vector1[i].compareTo(vector2[0]) <OperatorSymbol> 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (!nullPos1[i]) {
+              if (vector1[i].compareTo(vector2[0]) <OperatorSymbol> 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }
+      } else { // neither input is repeating
+         if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            if (!nullPos1[i] && !nullPos2[i]) {
+              if (vector1[i].compareTo(vector2[i]) <OperatorSymbol> 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (!nullPos1[i] && !nullPos2[i]) {
+              if (vector1[i].compareTo(vector2[i]) <OperatorSymbol> 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }
+      }
+    }
+  }
+
+  @Override
+  public String getOutputType() {
+    return "boolean";
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return -1;
+  }
+
+  @Override
+  public VectorExpressionDescriptor.Descriptor getDescriptor() {
+    return (new VectorExpressionDescriptor.Builder())
+        .setMode(
+            VectorExpressionDescriptor.Mode.FILTER)
+        .setNumArguments(2)
+        .setArgumentTypes(
+            VectorExpressionDescriptor.ArgumentType.getType("decimal"),
+            VectorExpressionDescriptor.ArgumentType.getType("decimal"))
+        .setInputExpressionTypes(
+            VectorExpressionDescriptor.InputExpressionType.COLUMN,
+            VectorExpressionDescriptor.InputExpressionType.COLUMN).build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/4a479d0c/ql/src/gen/vectorization/ExpressionTemplates/FilterDecimalColumnCompareDecimalScalar.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/FilterDecimalColumnCompareDecimalScalar.txt b/ql/src/gen/vectorization/ExpressionTemplates/FilterDecimalColumnCompareDecimalScalar.txt
new file mode 100644
index 0000000..bdd39b9
--- /dev/null
+++ b/ql/src/gen/vectorization/ExpressionTemplates/FilterDecimalColumnCompareDecimalScalar.txt
@@ -0,0 +1,160 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.exec.vector.expressions.gen;
+
+import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.DecimalUtil;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.hadoop.hive.common.type.HiveDecimal;
+
+/**
+ * This is a generated class to evaluate a <OperatorSymbol> comparison on a vector of decimal
+ * values.
+ */
+public class <ClassName> extends VectorExpression {
+
+  private static final long serialVersionUID = 1L;
+
+  private int colNum;
+  private HiveDecimal value;
+
+  public <ClassName>(int colNum, HiveDecimal value) {
+    this.colNum = colNum;
+    this.value = value;
+  }
+
+  public <ClassName>() {
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+    DecimalColumnVector inputColVector = (DecimalColumnVector) batch.cols[colNum];
+    int[] sel = batch.selected;
+    boolean[] nullPos = inputColVector.isNull;
+    int n = batch.size;
+    HiveDecimalWritable[] vector = inputColVector.vector;
+
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+
+    if (inputColVector.noNulls) {
+      if (inputColVector.isRepeating) {
+
+        // All must be selected otherwise size would be zero. Repeating property will not change.
+        if (!(DecimalUtil.compare(vector[0], value) <OperatorSymbol> 0)) {
+
+          // Entire batch is filtered out.
+          batch.size = 0;
+        }
+      } else if (batch.selectedInUse) {
+        int newSize = 0;
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          if (DecimalUtil.compare(vector[i], value) <OperatorSymbol> 0) {
+            sel[newSize++] = i;
+          }
+        }
+        batch.size = newSize;
+      } else {
+        int newSize = 0;
+        for(int i = 0; i != n; i++) {
+          if (DecimalUtil.compare(vector[i], value) <OperatorSymbol> 0) {
+            sel[newSize++] = i;
+          }
+        }
+        if (newSize < n) {
+          batch.size = newSize;
+          batch.selectedInUse = true;
+        }
+      }
+    } else {
+      if (inputColVector.isRepeating) {
+
+        // All must be selected otherwise size would be zero. Repeating property will not change.
+        if (!nullPos[0]) {
+          if (!(DecimalUtil.compare(vector[0], value) <OperatorSymbol> 0)) {
+
+            // Entire batch is filtered out.
+            batch.size = 0;
+          }
+        } else {
+          batch.size = 0;
+        }
+      } else if (batch.selectedInUse) {
+        int newSize = 0;
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          if (!nullPos[i]) {
+           if (DecimalUtil.compare(vector[i], value) <OperatorSymbol> 0) {
+             sel[newSize++] = i;
+           }
+          }
+        }
+
+        // Change the selected vector
+        batch.size = newSize;
+      } else {
+        int newSize = 0;
+        for(int i = 0; i != n; i++) {
+          if (!nullPos[i]) {
+            if (DecimalUtil.compare(vector[i], value) <OperatorSymbol> 0) {
+              sel[newSize++] = i;
+            }
+          }
+        }
+        if (newSize < n) {
+          batch.size = newSize;
+          batch.selectedInUse = true;
+        }
+      }
+    }
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return -1;
+  }
+
+  @Override
+  public String getOutputType() {
+    return "boolean";
+  }
+
+  @Override
+  public VectorExpressionDescriptor.Descriptor getDescriptor() {
+    return (new VectorExpressionDescriptor.Builder())
+        .setMode(
+            VectorExpressionDescriptor.Mode.FILTER)
+        .setNumArguments(2)
+        .setArgumentTypes(
+            VectorExpressionDescriptor.ArgumentType.getType("decimal"),
+            VectorExpressionDescriptor.ArgumentType.getType("decimal"))
+        .setInputExpressionTypes(
+            VectorExpressionDescriptor.InputExpressionType.COLUMN,
+            VectorExpressionDescriptor.InputExpressionType.SCALAR).build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/4a479d0c/ql/src/gen/vectorization/ExpressionTemplates/FilterDecimalColumnCompareScalar.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/FilterDecimalColumnCompareScalar.txt b/ql/src/gen/vectorization/ExpressionTemplates/FilterDecimalColumnCompareScalar.txt
deleted file mode 100644
index bdd39b9..0000000
--- a/ql/src/gen/vectorization/ExpressionTemplates/FilterDecimalColumnCompareScalar.txt
+++ /dev/null
@@ -1,160 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hive.ql.exec.vector.expressions.gen;
-
-import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
-import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.DecimalUtil;
-import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
-import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
-import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
-import org.apache.hadoop.hive.common.type.HiveDecimal;
-
-/**
- * This is a generated class to evaluate a <OperatorSymbol> comparison on a vector of decimal
- * values.
- */
-public class <ClassName> extends VectorExpression {
-
-  private static final long serialVersionUID = 1L;
-
-  private int colNum;
-  private HiveDecimal value;
-
-  public <ClassName>(int colNum, HiveDecimal value) {
-    this.colNum = colNum;
-    this.value = value;
-  }
-
-  public <ClassName>() {
-  }
-
-  @Override
-  public void evaluate(VectorizedRowBatch batch) {
-    if (childExpressions != null) {
-      super.evaluateChildren(batch);
-    }
-    DecimalColumnVector inputColVector = (DecimalColumnVector) batch.cols[colNum];
-    int[] sel = batch.selected;
-    boolean[] nullPos = inputColVector.isNull;
-    int n = batch.size;
-    HiveDecimalWritable[] vector = inputColVector.vector;
-
-    // return immediately if batch is empty
-    if (n == 0) {
-      return;
-    }
-
-    if (inputColVector.noNulls) {
-      if (inputColVector.isRepeating) {
-
-        // All must be selected otherwise size would be zero. Repeating property will not change.
-        if (!(DecimalUtil.compare(vector[0], value) <OperatorSymbol> 0)) {
-
-          // Entire batch is filtered out.
-          batch.size = 0;
-        }
-      } else if (batch.selectedInUse) {
-        int newSize = 0;
-        for(int j = 0; j != n; j++) {
-          int i = sel[j];
-          if (DecimalUtil.compare(vector[i], value) <OperatorSymbol> 0) {
-            sel[newSize++] = i;
-          }
-        }
-        batch.size = newSize;
-      } else {
-        int newSize = 0;
-        for(int i = 0; i != n; i++) {
-          if (DecimalUtil.compare(vector[i], value) <OperatorSymbol> 0) {
-            sel[newSize++] = i;
-          }
-        }
-        if (newSize < n) {
-          batch.size = newSize;
-          batch.selectedInUse = true;
-        }
-      }
-    } else {
-      if (inputColVector.isRepeating) {
-
-        // All must be selected otherwise size would be zero. Repeating property will not change.
-        if (!nullPos[0]) {
-          if (!(DecimalUtil.compare(vector[0], value) <OperatorSymbol> 0)) {
-
-            // Entire batch is filtered out.
-            batch.size = 0;
-          }
-        } else {
-          batch.size = 0;
-        }
-      } else if (batch.selectedInUse) {
-        int newSize = 0;
-        for(int j = 0; j != n; j++) {
-          int i = sel[j];
-          if (!nullPos[i]) {
-           if (DecimalUtil.compare(vector[i], value) <OperatorSymbol> 0) {
-             sel[newSize++] = i;
-           }
-          }
-        }
-
-        // Change the selected vector
-        batch.size = newSize;
-      } else {
-        int newSize = 0;
-        for(int i = 0; i != n; i++) {
-          if (!nullPos[i]) {
-            if (DecimalUtil.compare(vector[i], value) <OperatorSymbol> 0) {
-              sel[newSize++] = i;
-            }
-          }
-        }
-        if (newSize < n) {
-          batch.size = newSize;
-          batch.selectedInUse = true;
-        }
-      }
-    }
-  }
-
-  @Override
-  public int getOutputColumn() {
-    return -1;
-  }
-
-  @Override
-  public String getOutputType() {
-    return "boolean";
-  }
-
-  @Override
-  public VectorExpressionDescriptor.Descriptor getDescriptor() {
-    return (new VectorExpressionDescriptor.Builder())
-        .setMode(
-            VectorExpressionDescriptor.Mode.FILTER)
-        .setNumArguments(2)
-        .setArgumentTypes(
-            VectorExpressionDescriptor.ArgumentType.getType("decimal"),
-            VectorExpressionDescriptor.ArgumentType.getType("decimal"))
-        .setInputExpressionTypes(
-            VectorExpressionDescriptor.InputExpressionType.COLUMN,
-            VectorExpressionDescriptor.InputExpressionType.SCALAR).build();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/4a479d0c/ql/src/gen/vectorization/ExpressionTemplates/FilterDecimalScalarCompareColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/FilterDecimalScalarCompareColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/FilterDecimalScalarCompareColumn.txt
deleted file mode 100644
index 0608016..0000000
--- a/ql/src/gen/vectorization/ExpressionTemplates/FilterDecimalScalarCompareColumn.txt
+++ /dev/null
@@ -1,160 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hive.ql.exec.vector.expressions.gen;
-
-import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
-import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.DecimalUtil;
-import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
-import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
-import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
-import org.apache.hadoop.hive.common.type.HiveDecimal;
-
-/**
- * This is a generated class to evaluate a <OperatorSymbol> comparison on a vector of decimal
- * values.
- */
-public class <ClassName> extends VectorExpression {
-
-  private static final long serialVersionUID = 1L;
-
-  private int colNum;
-  private HiveDecimal value;
-
-  public <ClassName>(HiveDecimal value, int colNum) {
-    this.colNum = colNum;
-    this.value = value;
-  }
-
-  public <ClassName>() {
-  }
-
-  @Override
-  public void evaluate(VectorizedRowBatch batch) {
-    if (childExpressions != null) {
-      super.evaluateChildren(batch);
-    }
-    DecimalColumnVector inputColVector = (DecimalColumnVector) batch.cols[colNum];
-    int[] sel = batch.selected;
-    boolean[] nullPos = inputColVector.isNull;
-    int n = batch.size;
-    HiveDecimalWritable[] vector = inputColVector.vector;
-
-    // return immediately if batch is empty
-    if (n == 0) {
-      return;
-    }
-
-    if (inputColVector.noNulls) {
-      if (inputColVector.isRepeating) {
-
-        // All must be selected otherwise size would be zero. Repeating property will not change.
-        if (!(DecimalUtil.compare(value, vector[0]) <OperatorSymbol> 0)) {
-
-          // Entire batch is filtered out.
-          batch.size = 0;
-        }
-      } else if (batch.selectedInUse) {
-        int newSize = 0;
-        for(int j = 0; j != n; j++) {
-          int i = sel[j];
-          if (DecimalUtil.compare(value, vector[i]) <OperatorSymbol> 0) {
-            sel[newSize++] = i;
-          }
-        }
-        batch.size = newSize;
-      } else {
-        int newSize = 0;
-        for(int i = 0; i != n; i++) {
-          if (DecimalUtil.compare(value, vector[i]) <OperatorSymbol> 0) {
-            sel[newSize++] = i;
-          }
-        }
-        if (newSize < n) {
-          batch.size = newSize;
-          batch.selectedInUse = true;
-        }
-      }
-    } else {
-      if (inputColVector.isRepeating) {
-
-        // All must be selected otherwise size would be zero. Repeating property will not change.
-        if (!nullPos[0]) {
-          if (!(DecimalUtil.compare(value, vector[0]) <OperatorSymbol> 0)) {
-
-            // Entire batch is filtered out.
-            batch.size = 0;
-          }
-        } else {
-          batch.size = 0;
-        }
-      } else if (batch.selectedInUse) {
-        int newSize = 0;
-        for(int j = 0; j != n; j++) {
-          int i = sel[j];
-          if (!nullPos[i]) {
-           if (DecimalUtil.compare(value, vector[i]) <OperatorSymbol> 0) {
-             sel[newSize++] = i;
-           }
-          }
-        }
-
-        // Change the selected vector
-        batch.size = newSize;
-      } else {
-        int newSize = 0;
-        for(int i = 0; i != n; i++) {
-          if (!nullPos[i]) {
-            if (DecimalUtil.compare(value, vector[i]) <OperatorSymbol> 0) {
-              sel[newSize++] = i;
-            }
-          }
-        }
-        if (newSize < n) {
-          batch.size = newSize;
-          batch.selectedInUse = true;
-        }
-      }
-    }
-  }
-
-  @Override
-  public int getOutputColumn() {
-    return -1;
-  }
-
-  @Override
-  public String getOutputType() {
-    return "boolean";
-  }
-
-  @Override
-  public VectorExpressionDescriptor.Descriptor getDescriptor() {
-    return (new VectorExpressionDescriptor.Builder())
-        .setMode(
-            VectorExpressionDescriptor.Mode.FILTER)
-        .setNumArguments(2)
-        .setArgumentTypes(
-            VectorExpressionDescriptor.ArgumentType.getType("decimal"),
-            VectorExpressionDescriptor.ArgumentType.getType("decimal"))
-        .setInputExpressionTypes(
-            VectorExpressionDescriptor.InputExpressionType.SCALAR,
-            VectorExpressionDescriptor.InputExpressionType.COLUMN).build();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/4a479d0c/ql/src/gen/vectorization/ExpressionTemplates/FilterDecimalScalarCompareDecimalColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/FilterDecimalScalarCompareDecimalColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/FilterDecimalScalarCompareDecimalColumn.txt
new file mode 100644
index 0000000..0608016
--- /dev/null
+++ b/ql/src/gen/vectorization/ExpressionTemplates/FilterDecimalScalarCompareDecimalColumn.txt
@@ -0,0 +1,160 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.exec.vector.expressions.gen;
+
+import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.DecimalUtil;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.hadoop.hive.common.type.HiveDecimal;
+
+/**
+ * This is a generated class to evaluate a <OperatorSymbol> comparison on a vector of decimal
+ * values.
+ */
+public class <ClassName> extends VectorExpression {
+
+  private static final long serialVersionUID = 1L;
+
+  private int colNum;
+  private HiveDecimal value;
+
+  public <ClassName>(HiveDecimal value, int colNum) {
+    this.colNum = colNum;
+    this.value = value;
+  }
+
+  public <ClassName>() {
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+    DecimalColumnVector inputColVector = (DecimalColumnVector) batch.cols[colNum];
+    int[] sel = batch.selected;
+    boolean[] nullPos = inputColVector.isNull;
+    int n = batch.size;
+    HiveDecimalWritable[] vector = inputColVector.vector;
+
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+
+    if (inputColVector.noNulls) {
+      if (inputColVector.isRepeating) {
+
+        // All must be selected otherwise size would be zero. Repeating property will not change.
+        if (!(DecimalUtil.compare(value, vector[0]) <OperatorSymbol> 0)) {
+
+          // Entire batch is filtered out.
+          batch.size = 0;
+        }
+      } else if (batch.selectedInUse) {
+        int newSize = 0;
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          if (DecimalUtil.compare(value, vector[i]) <OperatorSymbol> 0) {
+            sel[newSize++] = i;
+          }
+        }
+        batch.size = newSize;
+      } else {
+        int newSize = 0;
+        for(int i = 0; i != n; i++) {
+          if (DecimalUtil.compare(value, vector[i]) <OperatorSymbol> 0) {
+            sel[newSize++] = i;
+          }
+        }
+        if (newSize < n) {
+          batch.size = newSize;
+          batch.selectedInUse = true;
+        }
+      }
+    } else {
+      if (inputColVector.isRepeating) {
+
+        // All must be selected otherwise size would be zero. Repeating property will not change.
+        if (!nullPos[0]) {
+          if (!(DecimalUtil.compare(value, vector[0]) <OperatorSymbol> 0)) {
+
+            // Entire batch is filtered out.
+            batch.size = 0;
+          }
+        } else {
+          batch.size = 0;
+        }
+      } else if (batch.selectedInUse) {
+        int newSize = 0;
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          if (!nullPos[i]) {
+           if (DecimalUtil.compare(value, vector[i]) <OperatorSymbol> 0) {
+             sel[newSize++] = i;
+           }
+          }
+        }
+
+        // Change the selected vector
+        batch.size = newSize;
+      } else {
+        int newSize = 0;
+        for(int i = 0; i != n; i++) {
+          if (!nullPos[i]) {
+            if (DecimalUtil.compare(value, vector[i]) <OperatorSymbol> 0) {
+              sel[newSize++] = i;
+            }
+          }
+        }
+        if (newSize < n) {
+          batch.size = newSize;
+          batch.selectedInUse = true;
+        }
+      }
+    }
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return -1;
+  }
+
+  @Override
+  public String getOutputType() {
+    return "boolean";
+  }
+
+  @Override
+  public VectorExpressionDescriptor.Descriptor getDescriptor() {
+    return (new VectorExpressionDescriptor.Builder())
+        .setMode(
+            VectorExpressionDescriptor.Mode.FILTER)
+        .setNumArguments(2)
+        .setArgumentTypes(
+            VectorExpressionDescriptor.ArgumentType.getType("decimal"),
+            VectorExpressionDescriptor.ArgumentType.getType("decimal"))
+        .setInputExpressionTypes(
+            VectorExpressionDescriptor.InputExpressionType.SCALAR,
+            VectorExpressionDescriptor.InputExpressionType.COLUMN).build();
+  }
+}

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

http://git-wip-us.apache.org/repos/asf/hive/blob/4a479d0c/ql/src/gen/vectorization/ExpressionTemplates/FilterIntervalDayTimeColumnCompareIntervalDayTimeScalar.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/FilterIntervalDayTimeColumnCompareIntervalDayTimeScalar.txt b/ql/src/gen/vectorization/ExpressionTemplates/FilterIntervalDayTimeColumnCompareIntervalDayTimeScalar.txt
new file mode 100644
index 0000000..7022b4f
--- /dev/null
+++ b/ql/src/gen/vectorization/ExpressionTemplates/FilterIntervalDayTimeColumnCompareIntervalDayTimeScalar.txt
@@ -0,0 +1,55 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.exec.vector.expressions.gen;
+
+import org.apache.hadoop.hive.common.type.HiveIntervalDayTime;
+import org.apache.hadoop.hive.common.type.PisaTimestamp;
+
+import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+
+/**
+ * Generated from template FilterIntervalDayTimeColumnCompareScalar.txt, which covers comparison
+ * expressions between a datetime/interval column and a scalar of the same type, however output is not
+ * produced in a separate column.
+ * The selected vector of the input {@link VectorizedRowBatch} is updated for in-place filtering.
+ */
+public class <ClassName> extends <BaseClassName> {
+
+  public <ClassName>(int colNum, HiveIntervalDayTime value) {
+    super(colNum, value.pisaTimestampUpdate(new PisaTimestamp()));
+  }
+
+  public <ClassName>() {
+    super();
+  }
+
+  @Override
+  public VectorExpressionDescriptor.Descriptor getDescriptor() {
+    return (new VectorExpressionDescriptor.Builder())
+        .setMode(
+            VectorExpressionDescriptor.Mode.FILTER)
+        .setNumArguments(2)
+        .setArgumentTypes(
+            VectorExpressionDescriptor.ArgumentType.getType("interval_day_time"),
+            VectorExpressionDescriptor.ArgumentType.getType("interval_day_time"))
+        .setInputExpressionTypes(
+            VectorExpressionDescriptor.InputExpressionType.COLUMN,
+            VectorExpressionDescriptor.InputExpressionType.SCALAR).build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/4a479d0c/ql/src/gen/vectorization/ExpressionTemplates/FilterIntervalDayTimeScalarCompareIntervalDayTimeColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/FilterIntervalDayTimeScalarCompareIntervalDayTimeColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/FilterIntervalDayTimeScalarCompareIntervalDayTimeColumn.txt
new file mode 100644
index 0000000..d227bf0
--- /dev/null
+++ b/ql/src/gen/vectorization/ExpressionTemplates/FilterIntervalDayTimeScalarCompareIntervalDayTimeColumn.txt
@@ -0,0 +1,55 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.exec.vector.expressions.gen;
+
+import org.apache.hadoop.hive.common.type.HiveIntervalDayTime;
+import org.apache.hadoop.hive.common.type.PisaTimestamp;
+
+import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+
+/**
+ * Generated from template FilterIntervalDayTimeScalarCompareColumn.txt, which covers comparison
+ * expressions between a datetime/interval column and a scalar of the same type, however output is not
+ * produced in a separate column.
+ * The selected vector of the input {@link VectorizedRowBatch} is updated for in-place filtering.
+ */
+public class <ClassName> extends <BaseClassName> {
+
+  public <ClassName>(HiveIntervalDayTime value, int colNum) {
+    super(value.pisaTimestampUpdate(new PisaTimestamp()), colNum);
+  }
+
+  public <ClassName>() {
+    super();
+  }
+
+  @Override
+  public VectorExpressionDescriptor.Descriptor getDescriptor() {
+    return (new VectorExpressionDescriptor.Builder())
+        .setMode(
+            VectorExpressionDescriptor.Mode.FILTER)
+        .setNumArguments(2)
+        .setArgumentTypes(
+            VectorExpressionDescriptor.ArgumentType.getType("interval_day_time"),
+            VectorExpressionDescriptor.ArgumentType.getType("interval_day_time"))
+        .setInputExpressionTypes(
+            VectorExpressionDescriptor.InputExpressionType.SCALAR,
+            VectorExpressionDescriptor.InputExpressionType.COLUMN).build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/4a479d0c/ql/src/gen/vectorization/ExpressionTemplates/FilterLongDoubleColumnCompareTimestampColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/FilterLongDoubleColumnCompareTimestampColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/FilterLongDoubleColumnCompareTimestampColumn.txt
new file mode 100644
index 0000000..0c8321f
--- /dev/null
+++ b/ql/src/gen/vectorization/ExpressionTemplates/FilterLongDoubleColumnCompareTimestampColumn.txt
@@ -0,0 +1,185 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.exec.vector.expressions.gen;
+
+import java.sql.Timestamp;
+import org.apache.hadoop.hive.common.type.PisaTimestamp;
+
+import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.NullUtil;
+import org.apache.hadoop.hive.ql.exec.vector.*;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+
+/**
+ * Generated from template FilterColumnCompareColumn.txt, which covers binary comparison
+ * expressions between two columns, however output is not produced in a separate column.
+ * The selected vector of the input {@link VectorizedRowBatch} is updated for in-place filtering.
+ */
+public class <ClassName> extends VectorExpression {
+
+  private static final long serialVersionUID = 1L;
+
+  private int colNum1;
+  private int colNum2;
+
+  public <ClassName>(int colNum1, int colNum2) {
+    this.colNum1 = colNum1;
+    this.colNum2 = colNum2;
+  }
+
+  public <ClassName>() {
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    <InputColumnVectorType1> inputColVector1 = (<InputColumnVectorType1>) batch.cols[colNum1];
+    TimestampColumnVector inputColVector2 = (TimestampColumnVector) batch.cols[colNum2];
+    int[] sel = batch.selected;
+    boolean[] nullPos1 = inputColVector1.isNull;
+    boolean[] nullPos2 = inputColVector2.isNull;
+    int n = batch.size;
+    <OperandType>[] vector1 = inputColVector1.vector;
+
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+
+    // filter rows with NULL on left input
+    int newSize;
+    newSize = NullUtil.filterNulls(batch.cols[colNum1], batch.selectedInUse, sel, n);
+    if (newSize < n) {
+      n = batch.size = newSize;
+      batch.selectedInUse = true;
+    }
+
+    // filter rows with NULL on right input
+    newSize = NullUtil.filterNulls(batch.cols[colNum2], batch.selectedInUse, sel, n);
+    if (newSize < n) {
+      n = batch.size = newSize;
+      batch.selectedInUse = true;
+    }
+
+    // All rows with nulls have been filtered out, so just do normal filter for non-null case
+    if (n != 0 && inputColVector1.isRepeating && inputColVector2.isRepeating) {
+
+      // All must be selected otherwise size would be zero
+      // Repeating property will not change.
+      if (!(vector1[0] <OperatorSymbol> inputColVector2.<GetTimestampLongDoubleMethod>(0))) {
+        batch.size = 0;
+      }
+    } else if (inputColVector1.isRepeating) {
+      <OperandType> value1 = vector1[0];
+      if (batch.selectedInUse) {
+        newSize = 0;
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          if (value1 <OperatorSymbol> inputColVector2.<GetTimestampLongDoubleMethod>(i)) {
+            sel[newSize++] = i;
+          }
+        }
+        batch.size = newSize;
+      } else {
+        newSize = 0;
+        for(int i = 0; i != n; i++) {
+          if (value1 <OperatorSymbol> inputColVector2.<GetTimestampLongDoubleMethod>(i)) {
+            sel[newSize++] = i;
+          }
+        }
+        if (newSize < batch.size) {
+          batch.size = newSize;
+          batch.selectedInUse = true;
+        }
+      }
+    } else if (inputColVector2.isRepeating) {
+      <OperandType> value2 = inputColVector2.<GetTimestampLongDoubleMethod>(0);
+      if (batch.selectedInUse) {
+        newSize = 0;
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          if (vector1[i] <OperatorSymbol> value2) {
+            sel[newSize++] = i;
+          }
+        }
+        batch.size = newSize;
+      } else {
+        newSize = 0;
+        for(int i = 0; i != n; i++) {
+          if (vector1[i] <OperatorSymbol> value2) {
+            sel[newSize++] = i;
+          }
+        }
+        if (newSize < batch.size) {
+          batch.size = newSize;
+          batch.selectedInUse = true;
+        }
+      }
+    } else if (batch.selectedInUse) {
+      newSize = 0;
+      for(int j = 0; j != n; j++) {
+        int i = sel[j];
+        if (vector1[i] <OperatorSymbol> inputColVector2.<GetTimestampLongDoubleMethod>(i)) {
+          sel[newSize++] = i;
+        }
+      }
+      batch.size = newSize;
+    } else {
+      newSize = 0;
+      for(int i = 0; i != n; i++) {
+        if (vector1[i] <OperatorSymbol>  inputColVector2.<GetTimestampLongDoubleMethod>(i)) {
+          sel[newSize++] = i;
+        }
+      }
+      if (newSize < batch.size) {
+        batch.size = newSize;
+        batch.selectedInUse = true;
+      }
+    }
+  }
+
+  @Override
+  public String getOutputType() {
+    return "boolean";
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return -1;
+  }
+
+  @Override
+  public VectorExpressionDescriptor.Descriptor getDescriptor() {
+    return (new VectorExpressionDescriptor.Builder())
+        .setMode(
+            VectorExpressionDescriptor.Mode.FILTER)
+        .setNumArguments(2)
+        .setArgumentTypes(
+            VectorExpressionDescriptor.ArgumentType.getType("<OperandType>"),
+            VectorExpressionDescriptor.ArgumentType.getType("timestamp"))
+        .setInputExpressionTypes(
+            VectorExpressionDescriptor.InputExpressionType.COLUMN,
+            VectorExpressionDescriptor.InputExpressionType.COLUMN).build();
+  }
+}

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

http://git-wip-us.apache.org/repos/asf/hive/blob/4a479d0c/ql/src/gen/vectorization/ExpressionTemplates/FilterLongDoubleScalarCompareTimestampColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/FilterLongDoubleScalarCompareTimestampColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/FilterLongDoubleScalarCompareTimestampColumn.txt
new file mode 100644
index 0000000..ba6ca66
--- /dev/null
+++ b/ql/src/gen/vectorization/ExpressionTemplates/FilterLongDoubleScalarCompareTimestampColumn.txt
@@ -0,0 +1,165 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.exec.vector.expressions.gen;
+
+import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+
+/**
+ * Generated from template FilterScalarCompareTimestampColumn.txt, which covers comparison
+ * expressions between a long/double scalar and a timestamp column, however output is not produced
+ * in a separate column. The selected vector of the input {@link VectorizedRowBatch} is updated
+ * for in-place filtering.
+ */
+public class <ClassName> extends VectorExpression {
+
+  private static final long serialVersionUID = 1L;
+
+  private int colNum;
+  private <OperandType> value;
+
+  public <ClassName>(<OperandType> value, int colNum) {
+    this.colNum = colNum;
+    this.value = value;
+  }
+
+  public <ClassName>() {
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    TimestampColumnVector inputColVector = (TimestampColumnVector) batch.cols[colNum];
+    int[] sel = batch.selected;
+    boolean[] nullPos = inputColVector.isNull;
+    int n = batch.size;
+
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+
+    if (inputColVector.noNulls) {
+      if (inputColVector.isRepeating) {
+        //All must be selected otherwise size would be zero
+        //Repeating property will not change.
+        if (!(value <OperatorSymbol> inputColVector.<GetTimestampLongDoubleMethod>(0))) {
+          //Entire batch is filtered out.
+          batch.size = 0;
+        }
+      } else if (batch.selectedInUse) {
+        int newSize = 0;
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          if (value <OperatorSymbol> inputColVector.<GetTimestampLongDoubleMethod>(i)) {
+            sel[newSize++] = i;
+          }
+        }
+        batch.size = newSize;
+      } else {
+        int newSize = 0;
+        for(int i = 0; i != n; i++) {
+          if (value <OperatorSymbol> inputColVector.<GetTimestampLongDoubleMethod>(i)) {
+            sel[newSize++] = i;
+          }
+        }
+        if (newSize < n) {
+          batch.size = newSize;
+          batch.selectedInUse = true;
+        }
+      }
+    } else {
+      if (inputColVector.isRepeating) {
+        //All must be selected otherwise size would be zero
+        //Repeating property will not change.
+        if (!nullPos[0]) {
+          if (!(value <OperatorSymbol> inputColVector.<GetTimestampLongDoubleMethod>(0))) {
+            //Entire batch is filtered out.
+            batch.size = 0;
+          }
+        } else {
+          batch.size = 0;
+        }
+      } else if (batch.selectedInUse) {
+        int newSize = 0;
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          if (!nullPos[i]) {
+           if (value <OperatorSymbol> inputColVector.<GetTimestampLongDoubleMethod>(i)) {
+             sel[newSize++] = i;
+           }
+          }
+        }
+        //Change the selected vector
+        batch.size = newSize;
+      } else {
+        int newSize = 0;
+        for(int i = 0; i != n; i++) {
+          if (!nullPos[i]) {
+            if (value <OperatorSymbol> inputColVector.<GetTimestampLongDoubleMethod>(i)) {
+              sel[newSize++] = i;
+            }
+          }
+        }
+        if (newSize < n) {
+          batch.size = newSize;
+          batch.selectedInUse = true;
+        }
+      }
+    }
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return -1;
+  }
+
+  @Override
+  public String getOutputType() {
+    return "boolean";
+  }
+
+  public int getColNum() {
+    return colNum;
+  }
+
+  public void setColNum(int colNum) {
+    this.colNum = colNum;
+  }
+
+  @Override
+  public VectorExpressionDescriptor.Descriptor getDescriptor() {
+    return (new VectorExpressionDescriptor.Builder())
+        .setMode(
+            VectorExpressionDescriptor.Mode.FILTER)
+        .setNumArguments(2)
+        .setArgumentTypes(
+            VectorExpressionDescriptor.ArgumentType.getType("<OperandType>"),
+            VectorExpressionDescriptor.ArgumentType.getType("timestamp"))
+        .setInputExpressionTypes(
+            VectorExpressionDescriptor.InputExpressionType.SCALAR,
+            VectorExpressionDescriptor.InputExpressionType.COLUMN).build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/4a479d0c/ql/src/gen/vectorization/ExpressionTemplates/FilterScalarCompareTimestampColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/FilterScalarCompareTimestampColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/FilterScalarCompareTimestampColumn.txt
deleted file mode 100644
index e0e5022..0000000
--- a/ql/src/gen/vectorization/ExpressionTemplates/FilterScalarCompareTimestampColumn.txt
+++ /dev/null
@@ -1,55 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
- 
-package org.apache.hadoop.hive.ql.exec.vector.expressions.gen;
-
-import org.apache.hadoop.hive.ql.exec.vector.TimestampUtils;
-
-import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
-
-/**
- * Generated from template FilterScalarCompareTimestampColumn.txt, which covers comparison 
- * expressions between a long or double scalar and a column, however output is not produced in a separate column. 
- * The selected vector of the input {@link VectorizedRowBatch} is updated for in-place filtering.
- * Note: For timestamp and long or double we implicitly interpret the long as the number
- * of seconds or double as seconds and fraction since the epoch.
- */
-public class <ClassName> extends <BaseClassName> {
-
-  public <ClassName>(<OperandType> value, int colNum) { 
-    super(TimestampUtils.<TimestampScalarConversion>(value), colNum);
-  }
-
-  public <ClassName>() {
-    super();
-  }
-
-  @Override
-  public VectorExpressionDescriptor.Descriptor getDescriptor() {
-    return (new VectorExpressionDescriptor.Builder())
-        .setMode(
-            VectorExpressionDescriptor.Mode.FILTER)
-        .setNumArguments(2)
-        .setArgumentTypes(
-            VectorExpressionDescriptor.ArgumentType.getType("<OperandType>"),
-            VectorExpressionDescriptor.ArgumentType.getType("timestamp"))
-        .setInputExpressionTypes(
-            VectorExpressionDescriptor.InputExpressionType.SCALAR,
-            VectorExpressionDescriptor.InputExpressionType.COLUMN).build();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/4a479d0c/ql/src/gen/vectorization/ExpressionTemplates/FilterTimestampColumnBetween.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/FilterTimestampColumnBetween.txt b/ql/src/gen/vectorization/ExpressionTemplates/FilterTimestampColumnBetween.txt
new file mode 100644
index 0000000..12f73da
--- /dev/null
+++ b/ql/src/gen/vectorization/ExpressionTemplates/FilterTimestampColumnBetween.txt
@@ -0,0 +1,172 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.exec.vector.expressions.gen;
+
+import java.sql.Timestamp;
+
+import org.apache.hadoop.hive.common.type.PisaTimestamp;
+import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+
+/**
+ * Generated from template FilterTimestampColumnBetween.txt, which covers [NOT] BETWEEN filter
+ * expressions where a column is [NOT] between one scalar and another.
+ * Output is not produced in a separate column.  The selected vector of the input
+ * {@link VectorizedRowBatch} is updated for in-place filtering.
+ */
+public class <ClassName> extends VectorExpression {
+
+  private static final long serialVersionUID = 1L;
+
+  private int colNum;
+
+  // The comparison is of the form "column BETWEEN leftValue AND rightValue"
+  private PisaTimestamp leftValue;
+  private PisaTimestamp rightValue;
+  private PisaTimestamp scratchValue;
+
+  public <ClassName>(int colNum, Timestamp leftValue, Timestamp rightValue) {
+    this.colNum = colNum;
+    this.leftValue = new PisaTimestamp(leftValue);
+    this.rightValue = new PisaTimestamp(rightValue);
+  }
+
+  public <ClassName>() {
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    TimestampColumnVector inputColVector = (TimestampColumnVector) batch.cols[colNum];
+    int[] sel = batch.selected;
+    boolean[] nullPos = inputColVector.isNull;
+    int n = batch.size;
+
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+
+    if (inputColVector.noNulls) {
+      if (inputColVector.isRepeating) {
+
+        // All must be selected otherwise size would be zero.
+        // Repeating property will not change.
+        if (<OptionalNot>(inputColVector.compareTo(0, leftValue) < 0 || inputColVector.compareTo(0, rightValue) > 0)) {
+
+          // Entire batch is filtered out.
+          batch.size = 0;
+        }
+      } else if (batch.selectedInUse) {
+        int newSize = 0;
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          if (<OptionalNot>(inputColVector.compareTo(leftValue, i) <= 0 && inputColVector.compareTo(i, rightValue) <= 0)) {
+            sel[newSize++] = i;
+          }
+        }
+        batch.size = newSize;
+      } else {
+        int newSize = 0;
+        for(int i = 0; i != n; i++) {
+          if (<OptionalNot>(inputColVector.compareTo(leftValue, i) <= 0 && inputColVector.compareTo(i, rightValue) <= 0)) {
+            sel[newSize++] = i;
+          }
+        }
+        if (newSize < n) {
+          batch.size = newSize;
+          batch.selectedInUse = true;
+        }
+      }
+    } else {
+      if (inputColVector.isRepeating) {
+
+        // All must be selected otherwise size would be zero.
+        // Repeating property will not change.
+        if (!nullPos[0]) {
+          if (<OptionalNot>(inputColVector.compareTo(0, leftValue) < 0 || inputColVector.compareTo(0, rightValue) > 0)) {
+
+            // Entire batch is filtered out.
+            batch.size = 0;
+          }
+        } else {
+          batch.size = 0;
+        }
+      } else if (batch.selectedInUse) {
+        int newSize = 0;
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          if (!nullPos[i]) {
+            if (<OptionalNot>(inputColVector.compareTo(leftValue, i) <= 0 && inputColVector.compareTo(i, rightValue) <= 0)) {
+             sel[newSize++] = i;
+            }
+          }
+        }
+
+        // Change the selected vector
+        batch.size = newSize;
+      } else {
+        int newSize = 0;
+        for(int i = 0; i != n; i++) {
+          if (!nullPos[i]) {
+            if (<OptionalNot>(inputColVector.compareTo(leftValue, i) <= 0 && inputColVector.compareTo(i, rightValue) <= 0)) {
+              sel[newSize++] = i;
+            }
+          }
+        }
+        if (newSize < n) {
+          batch.size = newSize;
+          batch.selectedInUse = true;
+        }
+      }
+    }
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return -1;
+  }
+
+  @Override
+  public String getOutputType() {
+    return "boolean";
+  }
+
+  @Override
+  public VectorExpressionDescriptor.Descriptor getDescriptor() {
+    return (new VectorExpressionDescriptor.Builder())
+        .setMode(
+            VectorExpressionDescriptor.Mode.FILTER)
+        .setNumArguments(3)
+        .setArgumentTypes(
+            VectorExpressionDescriptor.ArgumentType.getType("timestamp"),
+            VectorExpressionDescriptor.ArgumentType.getType("timestamp"),
+            VectorExpressionDescriptor.ArgumentType.getType("timestamp"))
+        .setInputExpressionTypes(
+            VectorExpressionDescriptor.InputExpressionType.COLUMN,
+            VectorExpressionDescriptor.InputExpressionType.SCALAR,
+            VectorExpressionDescriptor.InputExpressionType.SCALAR).build();
+  }
+}