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:59:42 UTC

[10/12] hive git commit: HIVE-13111: Fix timestamp / interval_day_time wrong results with HIVE-9862 (Matt McCline, reviewed by Jason Dere)

http://git-wip-us.apache.org/repos/asf/hive/blob/ca11c393/ql/src/gen/vectorization/ExpressionTemplates/DateScalarArithmeticTimestampColumnBase.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/DateScalarArithmeticTimestampColumnBase.txt b/ql/src/gen/vectorization/ExpressionTemplates/DateScalarArithmeticTimestampColumnBase.txt
deleted file mode 100644
index a1f4e6f..0000000
--- a/ql/src/gen/vectorization/ExpressionTemplates/DateScalarArithmeticTimestampColumnBase.txt
+++ /dev/null
@@ -1,147 +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.VectorExpressionDescriptor;
-import org.apache.hadoop.hive.ql.exec.vector.*;
-
-import org.apache.hadoop.hive.common.type.PisaTimestamp;
-/*
- * Because of the templatized nature of the code, either or both
- * of these ColumnVector imports may be needed. Listing both of them
- * rather than using ....vectorization.*;
- */
-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.expressions.NullUtil;
-import org.apache.hadoop.hive.serde2.io.DateWritable;
-
-/**
- * Generated from template DateTimeScalarArithmeticTimestampColumnBase.txt.
- * Implements a vectorized arithmetic operator with a scalar on the left and a
- * column vector on the right. The result is output to an output column vector.
- */
-public abstract class <BaseClassName> extends VectorExpression {
-
-  private static final long serialVersionUID = 1L;
-
-  private int colNum;
-  private PisaTimestamp value;
-  private int outputColumn;
-
-  public <BaseClassName>(long value, int colNum, int outputColumn) {
-    this.colNum = colNum;
-    this.value = new PisaTimestamp().updateFromTimestampMilliseconds(DateWritable.daysToMillis((int) value));
-    this.outputColumn = outputColumn;
-  }
-
-  public <BaseClassName>() {
-  }
-
-  @Override
-  /**
-   * Method to evaluate scalar-column operation in vectorized fashion.
-   *
-   * @batch a package of rows with each column stored in a vector
-   */
-  public void evaluate(VectorizedRowBatch batch) {
-
-    if (childExpressions != null) {
-      super.evaluateChildren(batch);
-    }
-
-    // Input #2 is type timestamp/interval_day_time.
-    TimestampColumnVector inputColVector2 = (TimestampColumnVector) batch.cols[colNum];
-
-    // Output is type timestamp.
-    TimestampColumnVector outputColVector = (TimestampColumnVector) batch.cols[outputColumn];
-
-    int[] sel = batch.selected;
-    boolean[] inputIsNull = inputColVector2.isNull;
-    boolean[] outputIsNull = outputColVector.isNull;
-    outputColVector.noNulls = inputColVector2.noNulls;
-    outputColVector.isRepeating = inputColVector2.isRepeating;
-    int n = batch.size;
-
-    // return immediately if batch is empty
-    if (n == 0) {
-      return;
-    }
-
-    if (inputColVector2.isRepeating) {
-      outputColVector.<OperatorMethod>(
-          value,
-          inputColVector2.asScratchPisaTimestamp(0),
-          0);
-
-      // Even if there are no nulls, we always copy over entry 0. Simplifies code.
-      outputIsNull[0] = inputIsNull[0];
-    } else if (inputColVector2.noNulls) {
-      if (batch.selectedInUse) {
-        for(int j = 0; j != n; j++) {
-          int i = sel[j];
-          outputColVector.<OperatorMethod>(
-            value,
-            inputColVector2.asScratchPisaTimestamp(i),
-            i);
-        }
-      } else {
-        for(int i = 0; i != n; i++) {
-          outputColVector.<OperatorMethod>(
-            value,
-            inputColVector2.asScratchPisaTimestamp(i),
-            i);
-        }
-      }
-    } else {                         /* there are nulls */
-      if (batch.selectedInUse) {
-        for(int j = 0; j != n; j++) {
-          int i = sel[j];
-          outputColVector.<OperatorMethod>(
-            value,
-            inputColVector2.asScratchPisaTimestamp(i),
-            i);
-          outputIsNull[i] = inputIsNull[i];
-        }
-      } else {
-        for(int i = 0; i != n; i++) {
-          outputColVector.<OperatorMethod>(
-            value,
-            inputColVector2.asScratchPisaTimestamp(i),
-            i);
-        }
-        System.arraycopy(inputIsNull, 0, outputIsNull, 0, n);
-      }
-    }
-
-    NullUtil.setNullOutputEntriesColScalar(outputColVector, batch.selectedInUse, sel, n);
-  }
-
-  @Override
-  public int getOutputColumn() {
-    return outputColumn;
-  }
-
-  @Override
-  public String getOutputType() {
-    return "timestamp";
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/ca11c393/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
deleted file mode 100644
index 8d9bdf1..0000000
--- a/ql/src/gen/vectorization/ExpressionTemplates/FilterIntervalDayTimeColumnCompareIntervalDayTimeColumn.txt
+++ /dev/null
@@ -1,52 +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.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/ca11c393/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
deleted file mode 100644
index 7022b4f..0000000
--- a/ql/src/gen/vectorization/ExpressionTemplates/FilterIntervalDayTimeColumnCompareIntervalDayTimeScalar.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.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/ca11c393/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
deleted file mode 100644
index d227bf0..0000000
--- a/ql/src/gen/vectorization/ExpressionTemplates/FilterIntervalDayTimeScalarCompareIntervalDayTimeColumn.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.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/ca11c393/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
index 0c8321f..57caf7e 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/FilterLongDoubleColumnCompareTimestampColumn.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/FilterLongDoubleColumnCompareTimestampColumn.txt
@@ -19,8 +19,8 @@
 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.common.type.HiveIntervalDayTime;
 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.*;

http://git-wip-us.apache.org/repos/asf/hive/blob/ca11c393/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
index 7e4d55e..1b86691 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/FilterLongDoubleColumnCompareTimestampScalar.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/FilterLongDoubleColumnCompareTimestampScalar.txt
@@ -19,8 +19,8 @@
 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.common.type.HiveIntervalDayTime;
 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;
@@ -36,7 +36,7 @@ public class <ClassName> extends <BaseClassName> {
   private static final long serialVersionUID = 1L;
 
   public <ClassName>(int colNum, Timestamp value) {
-    super(colNum, new PisaTimestamp(value).<GetTimestampLongDoubleMethod>());
+    super(colNum, TimestampColumnVector.<GetTimestampLongDoubleMethod>(value));
   }
 
   public <ClassName>() {

http://git-wip-us.apache.org/repos/asf/hive/blob/ca11c393/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
index ba6ca66..f5f59c2 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/FilterLongDoubleScalarCompareTimestampColumn.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/FilterLongDoubleScalarCompareTimestampColumn.txt
@@ -18,6 +18,10 @@
 
 package org.apache.hadoop.hive.ql.exec.vector.expressions.gen;
 
+import java.sql.Timestamp;
+
+import org.apache.hadoop.hive.common.type.HiveIntervalDayTime;
+
 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;

http://git-wip-us.apache.org/repos/asf/hive/blob/ca11c393/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
index 12f73da..4298d79 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/FilterTimestampColumnBetween.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/FilterTimestampColumnBetween.txt
@@ -20,7 +20,6 @@ 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;
@@ -39,14 +38,14 @@ public class <ClassName> extends VectorExpression {
   private int colNum;
 
   // The comparison is of the form "column BETWEEN leftValue AND rightValue"
-  private PisaTimestamp leftValue;
-  private PisaTimestamp rightValue;
-  private PisaTimestamp scratchValue;
+  private Timestamp leftValue;
+  private Timestamp rightValue;
+  private Timestamp scratchValue;
 
   public <ClassName>(int colNum, Timestamp leftValue, Timestamp rightValue) {
     this.colNum = colNum;
-    this.leftValue = new PisaTimestamp(leftValue);
-    this.rightValue = new PisaTimestamp(rightValue);
+    this.leftValue = leftValue;
+    this.rightValue = rightValue;
   }
 
   public <ClassName>() {

http://git-wip-us.apache.org/repos/asf/hive/blob/ca11c393/ql/src/gen/vectorization/ExpressionTemplates/FilterTimestampColumnCompareTimestampColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/FilterTimestampColumnCompareTimestampColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/FilterTimestampColumnCompareTimestampColumn.txt
index 746b297..31dce1c 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/FilterTimestampColumnCompareTimestampColumn.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/FilterTimestampColumnCompareTimestampColumn.txt
@@ -18,22 +18,421 @@
 
 package org.apache.hadoop.hive.ql.exec.vector.expressions.gen;
 
+import java.sql.Timestamp;
+
+import org.apache.hadoop.hive.common.type.HiveIntervalDayTime;
+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.*;
+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 FilterTimestampColumnCompareTimestampColumn.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.
+ * Generated from template FilterTimestampColumnCompareColumn.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 <BaseClassName> {
+public class <ClassName> extends VectorExpression {
+
+  private static final long serialVersionUID = 1L;
 
-  public <ClassName>(int colNum1, int colNum2) { 
-    super(colNum1, colNum2);
+  private int colNum1;
+  private int colNum2;
+
+  public <ClassName>(int colNum1, int colNum2) {
+    this.colNum1 = colNum1;
+    this.colNum2 = colNum2;
   }
 
   public <ClassName>() {
-    super();
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+     // Input #1 is type <OperandType>.
+    <InputColumnVectorType> inputColVector1 = (<InputColumnVectorType>) batch.cols[colNum1];
+
+     // Input #2 is type <OperandType>.
+    <InputColumnVectorType> inputColVector2 = (<InputColumnVectorType>) batch.cols[colNum2];
+
+    int[] sel = batch.selected;
+    boolean[] nullPos1 = inputColVector1.isNull;
+    boolean[] nullPos2 = inputColVector2.isNull;
+    int n = batch.size;
+
+    // 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 (!(inputColVector1.compareTo(0, inputColVector2, 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 (inputColVector1.compareTo(0, inputColVector2, i) <OperatorSymbol> 0) {
+              sel[newSize++] = i;
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (inputColVector1.compareTo(0, inputColVector2, 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 (inputColVector1.compareTo(i, inputColVector2, 0) <OperatorSymbol> 0) {
+              sel[newSize++] = i;
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (inputColVector1.compareTo(i, inputColVector2, 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 (inputColVector1.compareTo(i, inputColVector2, i) <OperatorSymbol> 0) {
+            sel[newSize++] = i;
+          }
+        }
+        batch.size = newSize;
+      } else {
+        int newSize = 0;
+        for(int i = 0; i != n; i++) {
+          if (inputColVector1.compareTo(i, inputColVector2, 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] ||
+            !(inputColVector1.compareTo(0, inputColVector2, 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 (inputColVector1.compareTo(0, inputColVector2, i) <OperatorSymbol> 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (!nullPos2[i]) {
+              if (inputColVector1.compareTo(0, inputColVector2, 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 (inputColVector1.compareTo(i, inputColVector2, 0) <OperatorSymbol> 0) {
+              sel[newSize++] = i;
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (inputColVector1.compareTo(i, inputColVector2, 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 (inputColVector1.compareTo(i, inputColVector2, i) <OperatorSymbol> 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (!nullPos2[i]) {
+              if (inputColVector1.compareTo(i, inputColVector2, 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] ||
+            !(inputColVector1.compareTo(0, inputColVector2, 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 (inputColVector1.compareTo(0, inputColVector2, i) <OperatorSymbol> 0) {
+              sel[newSize++] = i;
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (inputColVector1.compareTo(0, inputColVector2, 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 (inputColVector1.compareTo(i, inputColVector2, 0) <OperatorSymbol> 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (!nullPos1[i]) {
+              if (inputColVector1.compareTo(i, inputColVector2, 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 (inputColVector1.compareTo(i, inputColVector2, i) <OperatorSymbol> 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (!nullPos1[i]) {
+              if (inputColVector1.compareTo(i, inputColVector2, 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] ||
+            !(inputColVector1.compareTo(0, inputColVector2, 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 (inputColVector1.compareTo(0, inputColVector2, i) <OperatorSymbol> 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (!nullPos2[i]) {
+              if (inputColVector1.compareTo(0, inputColVector2, 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 (inputColVector1.compareTo(i, inputColVector2, 0) <OperatorSymbol> 0) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (!nullPos1[i]) {
+              if (inputColVector1.compareTo(i, inputColVector2, 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 (inputColVector1.compareTo(i, inputColVector2, 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 (inputColVector1.compareTo(i, inputColVector2, 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
@@ -43,8 +442,8 @@ public class <ClassName> extends <BaseClassName> {
             VectorExpressionDescriptor.Mode.FILTER)
         .setNumArguments(2)
         .setArgumentTypes(
-            VectorExpressionDescriptor.ArgumentType.getType("timestamp"),
-            VectorExpressionDescriptor.ArgumentType.getType("timestamp"))
+            VectorExpressionDescriptor.ArgumentType.getType("<OperandType>"),
+            VectorExpressionDescriptor.ArgumentType.getType("<OperandType>"))
         .setInputExpressionTypes(
             VectorExpressionDescriptor.InputExpressionType.COLUMN,
             VectorExpressionDescriptor.InputExpressionType.COLUMN).build();

http://git-wip-us.apache.org/repos/asf/hive/blob/ca11c393/ql/src/gen/vectorization/ExpressionTemplates/FilterTimestampColumnCompareTimestampColumnBase.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/FilterTimestampColumnCompareTimestampColumnBase.txt b/ql/src/gen/vectorization/ExpressionTemplates/FilterTimestampColumnCompareTimestampColumnBase.txt
deleted file mode 100644
index b5a7a7a..0000000
--- a/ql/src/gen/vectorization/ExpressionTemplates/FilterTimestampColumnCompareTimestampColumnBase.txt
+++ /dev/null
@@ -1,429 +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.TimestampColumnVector;
-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 FilterTimestampColumnCompareColumn.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 abstract 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);
-    }
-
-    TimestampColumnVector inputColVector1 = (TimestampColumnVector) 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;
-
-    // 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 (!(inputColVector1.compareTo(0, inputColVector2, 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 (inputColVector1.compareTo(0, inputColVector2, i) <OperatorSymbol> 0) {
-              sel[newSize++] = i;
-            }
-          }
-          batch.size = newSize;
-        } else {
-          int newSize = 0;
-          for(int i = 0; i != n; i++) {
-            if (inputColVector1.compareTo(0, inputColVector2, 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 (inputColVector1.compareTo(i, inputColVector2, 0) <OperatorSymbol> 0) {
-              sel[newSize++] = i;
-            }
-          }
-          batch.size = newSize;
-        } else {
-          int newSize = 0;
-          for(int i = 0; i != n; i++) {
-            if (inputColVector1.compareTo(i, inputColVector2, 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 (inputColVector1.compareTo(i, inputColVector2, i) <OperatorSymbol> 0) {
-            sel[newSize++] = i;
-          }
-        }
-        batch.size = newSize;
-      } else {
-        int newSize = 0;
-        for(int i = 0; i != n; i++) {
-          if (inputColVector1.compareTo(i, inputColVector2, 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] ||
-            !(inputColVector1.compareTo(0, inputColVector2, 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 (inputColVector1.compareTo(0, inputColVector2, i) <OperatorSymbol> 0) {
-                sel[newSize++] = i;
-              }
-            }
-          }
-          batch.size = newSize;
-        } else {
-          int newSize = 0;
-          for(int i = 0; i != n; i++) {
-            if (!nullPos2[i]) {
-              if (inputColVector1.compareTo(0, inputColVector2, 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 (inputColVector1.compareTo(i, inputColVector2, 0) <OperatorSymbol> 0) {
-              sel[newSize++] = i;
-            }
-          }
-          batch.size = newSize;
-        } else {
-          int newSize = 0;
-          for(int i = 0; i != n; i++) {
-            if (inputColVector1.compareTo(i, inputColVector2, 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 (inputColVector1.compareTo(i, inputColVector2, i) <OperatorSymbol> 0) {
-                sel[newSize++] = i;
-              }
-            }
-          }
-          batch.size = newSize;
-        } else {
-          int newSize = 0;
-          for(int i = 0; i != n; i++) {
-            if (!nullPos2[i]) {
-              if (inputColVector1.compareTo(i, inputColVector2, 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] ||
-            !(inputColVector1.compareTo(0, inputColVector2, 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 (inputColVector1.compareTo(0, inputColVector2, i) <OperatorSymbol> 0) {
-              sel[newSize++] = i;
-            }
-          }
-          batch.size = newSize;
-        } else {
-          int newSize = 0;
-          for(int i = 0; i != n; i++) {
-            if (inputColVector1.compareTo(0, inputColVector2, 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 (inputColVector1.compareTo(i, inputColVector2, 0) <OperatorSymbol> 0) {
-                sel[newSize++] = i;
-              }
-            }
-          }
-          batch.size = newSize;
-        } else {
-          int newSize = 0;
-          for(int i = 0; i != n; i++) {
-            if (!nullPos1[i]) {
-              if (inputColVector1.compareTo(i, inputColVector2, 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 (inputColVector1.compareTo(i, inputColVector2, i) <OperatorSymbol> 0) {
-                sel[newSize++] = i;
-              }
-            }
-          }
-          batch.size = newSize;
-        } else {
-          int newSize = 0;
-          for(int i = 0; i != n; i++) {
-            if (!nullPos1[i]) {
-              if (inputColVector1.compareTo(i, inputColVector2, 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] ||
-            !(inputColVector1.compareTo(0, inputColVector2, 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 (inputColVector1.compareTo(0, inputColVector2, i) <OperatorSymbol> 0) {
-                sel[newSize++] = i;
-              }
-            }
-          }
-          batch.size = newSize;
-        } else {
-          int newSize = 0;
-          for(int i = 0; i != n; i++) {
-            if (!nullPos2[i]) {
-              if (inputColVector1.compareTo(0, inputColVector2, 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 (inputColVector1.compareTo(i, inputColVector2, 0) <OperatorSymbol> 0) {
-                sel[newSize++] = i;
-              }
-            }
-          }
-          batch.size = newSize;
-        } else {
-          int newSize = 0;
-          for(int i = 0; i != n; i++) {
-            if (!nullPos1[i]) {
-              if (inputColVector1.compareTo(i, inputColVector2, 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 (inputColVector1.compareTo(i, inputColVector2, 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 (inputColVector1.compareTo(i, inputColVector2, 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;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/ca11c393/ql/src/gen/vectorization/ExpressionTemplates/FilterTimestampColumnCompareTimestampScalar.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/FilterTimestampColumnCompareTimestampScalar.txt b/ql/src/gen/vectorization/ExpressionTemplates/FilterTimestampColumnCompareTimestampScalar.txt
index f744d9b..bab8508 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/FilterTimestampColumnCompareTimestampScalar.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/FilterTimestampColumnCompareTimestampScalar.txt
@@ -20,24 +20,130 @@ 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.common.type.HiveIntervalDayTime;
+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.*;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
 
 /**
- * Generated from template FilterTimestampColumnCompareScalar.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.
+ * 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> {
+public class <ClassName> extends VectorExpression {
+
+  private static final long serialVersionUID = 1L;
+
+  private int colNum;
+  private <HiveOperandType> value;
 
-  public <ClassName>(int colNum, Timestamp value) {
-    super(colNum, new PisaTimestamp(value));
+  public <ClassName>(int colNum, <HiveOperandType> value) {
+    this.colNum = colNum;
+    this.value = value;
   }
 
   public <ClassName>() {
-    super();
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+     // Input #1 is type <OperandType>.
+    <InputColumnVectorType> inputColVector1 = (<InputColumnVectorType>) batch.cols[colNum];
+
+    int[] sel = batch.selected;
+    boolean[] nullPos = inputColVector1.isNull;
+    int n = batch.size;
+
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+
+    if (inputColVector1.noNulls) {
+      if (inputColVector1.isRepeating) {
+        //All must be selected otherwise size would be zero
+        //Repeating property will not change.
+        if (!(inputColVector1.compareTo(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 (inputColVector1.compareTo(i, value) <OperatorSymbol> 0) {
+            sel[newSize++] = i;
+          }
+        }
+        batch.size = newSize;
+      } else {
+        int newSize = 0;
+        for(int i = 0; i != n; i++) {
+          if (inputColVector1.compareTo(i, value) <OperatorSymbol> 0) {
+            sel[newSize++] = i;
+          }
+        }
+        if (newSize < n) {
+          batch.size = newSize;
+          batch.selectedInUse = true;
+        }
+      }
+    } else {
+      if (inputColVector1.isRepeating) {
+        //All must be selected otherwise size would be zero
+        //Repeating property will not change.
+        if (!nullPos[0]) {
+          if (!(inputColVector1.compareTo(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 (inputColVector1.compareTo(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 (inputColVector1.compareTo(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
@@ -47,8 +153,8 @@ public class <ClassName> extends <BaseClassName> {
             VectorExpressionDescriptor.Mode.FILTER)
         .setNumArguments(2)
         .setArgumentTypes(
-            VectorExpressionDescriptor.ArgumentType.getType("timestamp"),
-            VectorExpressionDescriptor.ArgumentType.getType("timestamp"))
+            VectorExpressionDescriptor.ArgumentType.getType("<OperandType>"),
+            VectorExpressionDescriptor.ArgumentType.getType("<OperandType>"))
         .setInputExpressionTypes(
             VectorExpressionDescriptor.InputExpressionType.COLUMN,
             VectorExpressionDescriptor.InputExpressionType.SCALAR).build();

http://git-wip-us.apache.org/repos/asf/hive/blob/ca11c393/ql/src/gen/vectorization/ExpressionTemplates/FilterTimestampColumnCompareTimestampScalarBase.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/FilterTimestampColumnCompareTimestampScalarBase.txt b/ql/src/gen/vectorization/ExpressionTemplates/FilterTimestampColumnCompareTimestampScalarBase.txt
deleted file mode 100644
index c84b4bf..0000000
--- a/ql/src/gen/vectorization/ExpressionTemplates/FilterTimestampColumnCompareTimestampScalarBase.txt
+++ /dev/null
@@ -1,145 +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 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.TimestampColumnVector;
-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 abstract class <ClassName> extends VectorExpression {
-
-  private static final long serialVersionUID = 1L;
-
-  private int colNum;
-  private PisaTimestamp value;
-
-  public <ClassName>(int colNum, PisaTimestamp value) {
-    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 (!(inputColVector.compareTo(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 (inputColVector.compareTo(i, value) <OperatorSymbol> 0) {
-            sel[newSize++] = i;
-          }
-        }
-        batch.size = newSize;
-      } else {
-        int newSize = 0;
-        for(int i = 0; i != n; i++) {
-          if (inputColVector.compareTo(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 (!(inputColVector.compareTo(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 (inputColVector.compareTo(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 (inputColVector.compareTo(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";
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/ca11c393/ql/src/gen/vectorization/ExpressionTemplates/FilterTimestampScalarCompareLongDoubleColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/FilterTimestampScalarCompareLongDoubleColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/FilterTimestampScalarCompareLongDoubleColumn.txt
index c3cd3b4..5e418de 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/FilterTimestampScalarCompareLongDoubleColumn.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/FilterTimestampScalarCompareLongDoubleColumn.txt
@@ -19,7 +19,6 @@
 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.*;
@@ -36,7 +35,7 @@ public class <ClassName> extends <BaseClassName> {
   private static final long serialVersionUID = 1L;
 
   public <ClassName>(Timestamp value, int colNum) {
-    super(new PisaTimestamp(value).<GetTimestampLongDoubleMethod>(), colNum);
+    super(TimestampColumnVector.<GetTimestampLongDoubleMethod>(value), colNum);
   }
 
   public <ClassName>() {

http://git-wip-us.apache.org/repos/asf/hive/blob/ca11c393/ql/src/gen/vectorization/ExpressionTemplates/FilterTimestampScalarCompareTimestampColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/FilterTimestampScalarCompareTimestampColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/FilterTimestampScalarCompareTimestampColumn.txt
index 05ab310..ff5d11e 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/FilterTimestampScalarCompareTimestampColumn.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/FilterTimestampScalarCompareTimestampColumn.txt
@@ -20,24 +20,132 @@ 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.common.type.HiveIntervalDayTime;
+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.*;
+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;
 
 /**
- * Generated from template FilterTimestampScalarCompareTimestampColumn.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.
+ * This is a generated class to evaluate a <OperatorSymbol> comparison on a vector of timestamp
+ * values.
  */
-public class <ClassName> extends <BaseClassName> {
+public class <ClassName> extends VectorExpression {
+
+  private static final long serialVersionUID = 1L;
+
+  private int colNum;
+  private <HiveOperandType> value;
 
-  public <ClassName>(Timestamp value, int colNum) {
-    super(new PisaTimestamp(value), colNum);
+  public <ClassName>(<HiveOperandType> value, int colNum) {
+    this.colNum = colNum;
+    this.value = value;
   }
 
   public <ClassName>() {
-    super();
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+     // Input #2 is type <OperandType>.
+    <InputColumnVectorType> inputColVector2 = (<InputColumnVectorType>) batch.cols[colNum];
+
+    int[] sel = batch.selected;
+    boolean[] nullPos = inputColVector2.isNull;
+    int n = batch.size;
+
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+
+    if (inputColVector2.noNulls) {
+      if (inputColVector2.isRepeating) {
+
+        // All must be selected otherwise size would be zero. Repeating property will not change.
+        if (!(inputColVector2.compareTo(value, 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 (inputColVector2.compareTo(value, i) <OperatorSymbol> 0) {
+            sel[newSize++] = i;
+          }
+        }
+        batch.size = newSize;
+      } else {
+        int newSize = 0;
+        for(int i = 0; i != n; i++) {
+          if (inputColVector2.compareTo(value, i) <OperatorSymbol> 0) {
+            sel[newSize++] = i;
+          }
+        }
+        if (newSize < n) {
+          batch.size = newSize;
+          batch.selectedInUse = true;
+        }
+      }
+    } else {
+      if (inputColVector2.isRepeating) {
+
+        // All must be selected otherwise size would be zero. Repeating property will not change.
+        if (!nullPos[0]) {
+          if (!(inputColVector2.compareTo(value, 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 (inputColVector2.compareTo(value, 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 (inputColVector2.compareTo(value, 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
@@ -47,8 +155,8 @@ public class <ClassName> extends <BaseClassName> {
             VectorExpressionDescriptor.Mode.FILTER)
         .setNumArguments(2)
         .setArgumentTypes(
-            VectorExpressionDescriptor.ArgumentType.getType("timestamp"),
-            VectorExpressionDescriptor.ArgumentType.getType("timestamp"))
+            VectorExpressionDescriptor.ArgumentType.getType("<OperandType>"),
+            VectorExpressionDescriptor.ArgumentType.getType("<OperandType>"))
         .setInputExpressionTypes(
             VectorExpressionDescriptor.InputExpressionType.SCALAR,
             VectorExpressionDescriptor.InputExpressionType.COLUMN).build();

http://git-wip-us.apache.org/repos/asf/hive/blob/ca11c393/ql/src/gen/vectorization/ExpressionTemplates/FilterTimestampScalarCompareTimestampColumnBase.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/FilterTimestampScalarCompareTimestampColumnBase.txt b/ql/src/gen/vectorization/ExpressionTemplates/FilterTimestampScalarCompareTimestampColumnBase.txt
deleted file mode 100644
index 608faef..0000000
--- a/ql/src/gen/vectorization/ExpressionTemplates/FilterTimestampScalarCompareTimestampColumnBase.txt
+++ /dev/null
@@ -1,147 +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 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.TimestampColumnVector;
-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 timestamp
- * values.
- */
-public abstract class <ClassName> extends VectorExpression {
-
-  private static final long serialVersionUID = 1L;
-
-  private int colNum;
-  private PisaTimestamp value;
-
-  public <ClassName>(PisaTimestamp 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 (!(inputColVector.compareTo(value, 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 (inputColVector.compareTo(value, i) <OperatorSymbol> 0) {
-            sel[newSize++] = i;
-          }
-        }
-        batch.size = newSize;
-      } else {
-        int newSize = 0;
-        for(int i = 0; i != n; i++) {
-          if (inputColVector.compareTo(value, 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 (!(inputColVector.compareTo(value, 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 (inputColVector.compareTo(value, 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 (inputColVector.compareTo(value, 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";
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/ca11c393/ql/src/gen/vectorization/ExpressionTemplates/IntervalDayTimeColumnCompareIntervalDayTimeColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/IntervalDayTimeColumnCompareIntervalDayTimeColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/IntervalDayTimeColumnCompareIntervalDayTimeColumn.txt
deleted file mode 100644
index bf62b78..0000000
--- a/ql/src/gen/vectorization/ExpressionTemplates/IntervalDayTimeColumnCompareIntervalDayTimeColumn.txt
+++ /dev/null
@@ -1,54 +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.VectorExpressionDescriptor;
-
-
-/**
- * Generated from template IntervalDayTimeColumnCompareColumn.txt, which covers comparison
- * expressions between a datetime/interval column and a scalar of the same type. The boolean output
- * is stored in a separate boolean column.
- */
-public class <ClassName> extends <BaseClassName> {
-
-  private static final long serialVersionUID = 1L;
-
-  public <ClassName>(int colNum1, int colNum2, int outputColumn) {
-    super(colNum1, colNum2, outputColumn);
-  }
-
-  public <ClassName>() {
-    super();
-  }
-
-  @Override
-  public VectorExpressionDescriptor.Descriptor getDescriptor() {
-    return (new VectorExpressionDescriptor.Builder())
-        .setMode(
-            VectorExpressionDescriptor.Mode.PROJECTION)
-        .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/ca11c393/ql/src/gen/vectorization/ExpressionTemplates/IntervalDayTimeColumnCompareIntervalDayTimeScalar.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/IntervalDayTimeColumnCompareIntervalDayTimeScalar.txt b/ql/src/gen/vectorization/ExpressionTemplates/IntervalDayTimeColumnCompareIntervalDayTimeScalar.txt
deleted file mode 100644
index 1abb4a3..0000000
--- a/ql/src/gen/vectorization/ExpressionTemplates/IntervalDayTimeColumnCompareIntervalDayTimeScalar.txt
+++ /dev/null
@@ -1,57 +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.common.type.HiveIntervalDayTime;
-import org.apache.hadoop.hive.common.type.PisaTimestamp;
-
-import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
-
-
-/**
- * Generated from template IntervalDayTimeColumnCompareScalar.txt, which covers comparison
- * expressions between a datetime/interval column and a scalar of the same type. The boolean output
- * is stored in a separate boolean column.
- */
-public class <ClassName> extends <BaseClassName> {
-
-  private static final long serialVersionUID = 1L;
-
-  public <ClassName>(int colNum, HiveIntervalDayTime value, int outputColumn) {
-    super(colNum, value.pisaTimestampUpdate(new PisaTimestamp()), outputColumn);
-  }
-
-  public <ClassName>() {
-    super();
-  }
-
-  @Override
-  public VectorExpressionDescriptor.Descriptor getDescriptor() {
-    return (new VectorExpressionDescriptor.Builder())
-        .setMode(
-            VectorExpressionDescriptor.Mode.PROJECTION)
-        .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/ca11c393/ql/src/gen/vectorization/ExpressionTemplates/IntervalDayTimeScalarCompareIntervalDayTimeColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/IntervalDayTimeScalarCompareIntervalDayTimeColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/IntervalDayTimeScalarCompareIntervalDayTimeColumn.txt
deleted file mode 100644
index 26762ff..0000000
--- a/ql/src/gen/vectorization/ExpressionTemplates/IntervalDayTimeScalarCompareIntervalDayTimeColumn.txt
+++ /dev/null
@@ -1,57 +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.common.type.HiveIntervalDayTime;
-import org.apache.hadoop.hive.common.type.PisaTimestamp;
-
-import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
-
-
-/**
- * Generated from template IntervalDayTimeColumnCompareScalar.txt, which covers comparison
- * expressions between a datetime/interval column and a scalar of the same type. The boolean output
- * is stored in a separate boolean column.
- */
-public class <ClassName> extends <BaseClassName> {
-
-  private static final long serialVersionUID = 1L;
-
-  public <ClassName>(HiveIntervalDayTime value, int colNum, int outputColumn) {
-    super(value.pisaTimestampUpdate(new PisaTimestamp()), colNum, outputColumn);
-  }
-
-  public <ClassName>() {
-    super();
-  }
-
-  @Override
-  public VectorExpressionDescriptor.Descriptor getDescriptor() {
-    return (new VectorExpressionDescriptor.Builder())
-        .setMode(
-            VectorExpressionDescriptor.Mode.PROJECTION)
-        .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/ca11c393/ql/src/gen/vectorization/ExpressionTemplates/IntervalYearMonthColumnArithmeticDateColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/IntervalYearMonthColumnArithmeticDateColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/IntervalYearMonthColumnArithmeticDateColumn.txt
index 7ae84b7..8e3a419 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/IntervalYearMonthColumnArithmeticDateColumn.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/IntervalYearMonthColumnArithmeticDateColumn.txt
@@ -18,12 +18,15 @@
 
 package org.apache.hadoop.hive.ql.exec.vector.expressions.gen;
 
+import java.sql.Date;
+import org.apache.hadoop.hive.common.type.HiveIntervalYearMonth;
 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;
 import org.apache.hadoop.hive.ql.util.DateTimeMath;
+import org.apache.hadoop.hive.serde2.io.DateWritable;
 
 /**
  * Generated from template DateColumnArithmeticIntervalYearMonthColumn.txt, which covers binary arithmetic
@@ -36,12 +39,18 @@ public class <ClassName> extends VectorExpression {
   private int colNum1;
   private int colNum2;
   private int outputColumn;
+  private HiveIntervalYearMonth scratchIntervalYearMonth1;
+  private Date scratchDate2;
+  private Date outputDate;
   private DateTimeMath dtm = new DateTimeMath();
 
   public <ClassName>(int colNum1, int colNum2, int outputColumn) {
     this.colNum1 = colNum1;
     this.colNum2 = colNum2;
     this.outputColumn = outputColumn;
+    scratchIntervalYearMonth1 = new HiveIntervalYearMonth();
+    scratchDate2 = new Date(0);
+    outputDate = new Date(0);
   }
 
   public <ClassName>() {
@@ -54,10 +63,10 @@ public class <ClassName> extends VectorExpression {
       super.evaluateChildren(batch);
     }
 
-    // Input #1 is type interval_year_month (months).
+    // Input #1 is type interval_year_month.
     LongColumnVector inputColVector1 = (LongColumnVector) batch.cols[colNum1];
 
-    // Input #2 is type date (epochDays).
+    // Input #2 is type date.
     LongColumnVector inputColVector2 = (LongColumnVector) batch.cols[colNum2];
 
     // Output is type date.
@@ -89,40 +98,64 @@ public class <ClassName> extends VectorExpression {
      * conditional checks in the inner loop.
      */
     if (inputColVector1.isRepeating && inputColVector2.isRepeating) {
-      outputVector[0] = dtm.addMonthsToDays(vector2[0], <OperatorSymbol> (int) vector1[0]);
+      scratchIntervalYearMonth1.set((int) vector1[0]);
+      scratchDate2.setTime(DateWritable.daysToMillis((int) vector2[0]));
+      dtm.<OperatorMethod>(
+          scratchIntervalYearMonth1, scratchDate2, outputDate);
+      outputVector[0] = DateWritable.dateToDays(outputDate);
     } else if (inputColVector1.isRepeating) {
-      long value1 = vector1[0];
+      scratchIntervalYearMonth1.set((int) vector1[0]);
       if (batch.selectedInUse) {
         for(int j = 0; j != n; j++) {
           int i = sel[j];
-          outputVector[i] = dtm.addMonthsToDays(vector2[i], <OperatorSymbol> (int) value1);
+          scratchDate2.setTime(DateWritable.daysToMillis((int) vector2[i]));
+          dtm.<OperatorMethod>(
+              scratchIntervalYearMonth1, scratchDate2, outputDate);
+          outputVector[i] = DateWritable.dateToDays(outputDate);
         }
       } else {
         for(int i = 0; i != n; i++) {
-          outputVector[i] = dtm.addMonthsToDays(vector2[i], <OperatorSymbol> (int) value1);
+          scratchDate2.setTime(DateWritable.daysToMillis((int) vector2[i]));
+          dtm.<OperatorMethod>(
+              scratchIntervalYearMonth1, scratchDate2, outputDate);
+          outputVector[i] = DateWritable.dateToDays(outputDate);
         }
       }
     } else if (inputColVector2.isRepeating) {
-      long value2 = vector2[0];
+      scratchDate2.setTime(DateWritable.daysToMillis((int) vector2[0]));
       if (batch.selectedInUse) {
         for(int j = 0; j != n; j++) {
           int i = sel[j];
-          outputVector[i] = dtm.addMonthsToDays(value2, <OperatorSymbol> (int) vector1[i]);
+          scratchIntervalYearMonth1.set((int) vector1[i]);
+          dtm.<OperatorMethod>(
+              scratchIntervalYearMonth1, scratchDate2, outputDate);
+          outputVector[i] = DateWritable.dateToDays(outputDate);
         }
       } else {
         for(int i = 0; i != n; i++) {
-          outputVector[i] = dtm.addMonthsToDays(value2, <OperatorSymbol> (int) vector1[i]);
+          scratchIntervalYearMonth1.set((int) vector1[i]);
+          dtm.<OperatorMethod>(
+              scratchIntervalYearMonth1, scratchDate2, outputDate);
+          outputVector[i] = DateWritable.dateToDays(outputDate);
         }
       }
     } else {
       if (batch.selectedInUse) {
         for(int j = 0; j != n; j++) {
           int i = sel[j];
-          outputVector[i] = dtm.addMonthsToDays(vector2[i], <OperatorSymbol> (int) vector1[i]);
+          scratchIntervalYearMonth1.set((int) vector1[i]);
+          scratchDate2.setTime(DateWritable.daysToMillis((int) vector2[i]));
+          dtm.<OperatorMethod>(
+              scratchIntervalYearMonth1, scratchDate2, outputDate);
+          outputVector[i] = DateWritable.dateToDays(outputDate);
         }
       } else {
         for(int i = 0; i != n; i++) {
-          outputVector[i] = dtm.addMonthsToDays(vector2[i], <OperatorSymbol> (int) vector1[i]);
+          scratchIntervalYearMonth1.set((int) vector1[i]);
+          scratchDate2.setTime(DateWritable.daysToMillis((int) vector2[i]));
+          dtm.<OperatorMethod>(
+              scratchIntervalYearMonth1, scratchDate2, outputDate);
+          outputVector[i] = DateWritable.dateToDays(outputDate);
         }
       }
     }