You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by jd...@apache.org on 2016/04/04 22:37:09 UTC

[34/50] [abbrv] 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/52016296/ql/src/gen/vectorization/ExpressionTemplates/DateColumnArithmeticTimestampColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/DateColumnArithmeticTimestampColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/DateColumnArithmeticTimestampColumn.txt
index 6241ee2..63cebaf 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/DateColumnArithmeticTimestampColumn.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/DateColumnArithmeticTimestampColumn.txt
@@ -18,28 +18,155 @@
 
 package org.apache.hadoop.hive.ql.exec.vector.expressions.gen;
 
-import org.apache.hadoop.hive.common.type.PisaTimestamp;
+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.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 DateColumnArithmeticTimestampColumn.txt, which covers binary arithmetic
- * expressions between a date column and a timestamp column.
+ * Generated from template DateColumnArithmeticTimestampColumn.txt, a class
+ * which covers binary arithmetic expressions between a date column and timestamp column.
  */
-public class <ClassName> extends <BaseClassName> {
+public class <ClassName> extends VectorExpression {
 
   private static final long serialVersionUID = 1L;
 
+  private int colNum1;
+  private int colNum2;
+  private int outputColumn;
+  private Timestamp scratchTimestamp1;
+  private DateTimeMath dtm = new DateTimeMath();
+
   public <ClassName>(int colNum1, int colNum2, int outputColumn) {
-    super(colNum1, colNum2, outputColumn);
+    this.colNum1 = colNum1;
+    this.colNum2 = colNum2;
+    this.outputColumn = outputColumn;
+    scratchTimestamp1 = new Timestamp(0);
   }
 
   public <ClassName>() {
-    super();
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    // Input #1 is type Date (days).  For the math we convert it to a timestamp.
+    LongColumnVector inputColVector1 = (LongColumnVector) batch.cols[colNum1];
+
+    // Input #2 is type <OperandType2>.
+    <InputColumnVectorType2> inputColVector2 = (<InputColumnVectorType2>) batch.cols[colNum2];
+
+    // Output is type <ReturnType>.
+    <OutputColumnVectorType> outputColVector = (<OutputColumnVectorType>) batch.cols[outputColumn];
+
+    int[] sel = batch.selected;
+    int n = batch.size;
+    long[] vector1 = inputColVector1.vector;
+
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+
+    outputColVector.isRepeating =
+         inputColVector1.isRepeating && inputColVector2.isRepeating
+      || inputColVector1.isRepeating && !inputColVector1.noNulls && inputColVector1.isNull[0]
+      || inputColVector2.isRepeating && !inputColVector2.noNulls && inputColVector2.isNull[0];
+
+    // Handle nulls first
+    NullUtil.propagateNullsColCol(
+      inputColVector1, inputColVector2, outputColVector, sel, n, batch.selectedInUse);
+
+    /* Disregard nulls for processing. In other words,
+     * the arithmetic operation is performed even if one or
+     * more inputs are null. This is to improve speed by avoiding
+     * conditional checks in the inner loop.
+     */
+    if (inputColVector1.isRepeating && inputColVector2.isRepeating) {
+      scratchTimestamp1.setTime(DateWritable.daysToMillis((int) vector1[0]));
+      dtm.<OperatorMethod>(
+          scratchTimestamp1, inputColVector2.asScratch<CamelOperandType2>(0), outputColVector.getScratch<CamelReturnType>());
+      outputColVector.setFromScratch<CamelReturnType>(0);
+    } else if (inputColVector1.isRepeating) {
+      scratchTimestamp1.setTime(DateWritable.daysToMillis((int) vector1[0]));
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          dtm.<OperatorMethod>(
+              scratchTimestamp1, inputColVector2.asScratch<CamelOperandType2>(i), outputColVector.getScratch<CamelReturnType>());
+          outputColVector.setFromScratch<CamelReturnType>(i);
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          dtm.<OperatorMethod>(
+              scratchTimestamp1, inputColVector2.asScratch<CamelOperandType2>(i), outputColVector.getScratch<CamelReturnType>());
+          outputColVector.setFromScratch<CamelReturnType>(i);
+        }
+      }
+    } else if (inputColVector2.isRepeating) {
+      <HiveOperandType2> value2 = inputColVector2.asScratch<CamelOperandType2>(0);
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          scratchTimestamp1.setTime(DateWritable.daysToMillis((int) vector1[i]));
+          dtm.<OperatorMethod>(
+              scratchTimestamp1, value2, outputColVector.getScratch<CamelReturnType>());
+          outputColVector.setFromScratch<CamelReturnType>(i);
+         }
+      } else {
+        for(int i = 0; i != n; i++) {
+          scratchTimestamp1.setTime(DateWritable.daysToMillis((int) vector1[i]));
+          dtm.<OperatorMethod>(
+              scratchTimestamp1, value2, outputColVector.getScratch<CamelReturnType>());
+          outputColVector.setFromScratch<CamelReturnType>(i);
+        }
+      }
+    } else {
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          scratchTimestamp1.setTime(DateWritable.daysToMillis((int) vector1[i]));
+         dtm.<OperatorMethod>(
+              scratchTimestamp1, inputColVector2.asScratch<CamelOperandType2>(i), outputColVector.getScratch<CamelReturnType>());
+          outputColVector.setFromScratch<CamelReturnType>(i);
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          scratchTimestamp1.setTime(DateWritable.daysToMillis((int) vector1[i]));
+          dtm.<OperatorMethod>(
+              scratchTimestamp1, inputColVector2.asScratch<CamelOperandType2>(i), outputColVector.getScratch<CamelReturnType>());
+          outputColVector.setFromScratch<CamelReturnType>(i);
+        }
+      }
+    }
+
+    /* For the case when the output can have null values, follow
+     * the convention that the data values must be 1 for long and
+     * NaN for double. This is to prevent possible later zero-divide errors
+     * in complex arithmetic expressions like col2 / (col1 - 1)
+     * in the case when some col1 entries are null.
+     */
+    NullUtil.setNullDataEntries<CamelReturnType>(outputColVector, batch.selectedInUse, sel, n);
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return outputColumn;
+  }
+
+  @Override
+  public String getOutputType() {
+    return "<ReturnType>";
   }
 
   @Override
@@ -49,7 +176,7 @@ public class <ClassName> extends <BaseClassName> {
             VectorExpressionDescriptor.Mode.PROJECTION)
         .setNumArguments(2)
         .setArgumentTypes(
-            VectorExpressionDescriptor.ArgumentType.getType("<OperandType1>"),
+            VectorExpressionDescriptor.ArgumentType.getType("date"),
             VectorExpressionDescriptor.ArgumentType.getType("<OperandType2>"))
         .setInputExpressionTypes(
             VectorExpressionDescriptor.InputExpressionType.COLUMN,

http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/ql/src/gen/vectorization/ExpressionTemplates/DateColumnArithmeticTimestampColumnBase.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/DateColumnArithmeticTimestampColumnBase.txt b/ql/src/gen/vectorization/ExpressionTemplates/DateColumnArithmeticTimestampColumnBase.txt
deleted file mode 100644
index a61b769..0000000
--- a/ql/src/gen/vectorization/ExpressionTemplates/DateColumnArithmeticTimestampColumnBase.txt
+++ /dev/null
@@ -1,171 +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.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;
-import org.apache.hadoop.hive.serde2.io.DateWritable;
-
-/**
- * Generated from template DateColumnArithmeticTimestampColumnBase.txt, a base class
- * which covers binary arithmetic expressions between a date column and timestamp column.
- */
-public abstract class <BaseClassName> extends VectorExpression {
-
-  private static final long serialVersionUID = 1L;
-
-  private int colNum1;
-  private int colNum2;
-  private int outputColumn;
-  private PisaTimestamp scratchPisaTimestamp;
-
-  public <BaseClassName>(int colNum1, int colNum2, int outputColumn) {
-    this.colNum1 = colNum1;
-    this.colNum2 = colNum2;
-    this.outputColumn = outputColumn;
-    scratchPisaTimestamp = new PisaTimestamp();
-  }
-
-  public <BaseClassName>() {
-  }
-
-  @Override
-  public void evaluate(VectorizedRowBatch batch) {
-
-    if (childExpressions != null) {
-      super.evaluateChildren(batch);
-    }
-
-    // Input #1 is type Date (epochDays).
-    LongColumnVector inputColVector1 = (LongColumnVector) batch.cols[colNum1];
-
-    // Input #2 is type timestamp/interval_day_time.
-    TimestampColumnVector inputColVector2 = (TimestampColumnVector) batch.cols[colNum2];
-
-    // Output is type timestamp.
-    TimestampColumnVector outputColVector = (TimestampColumnVector) batch.cols[outputColumn];
-
-    int[] sel = batch.selected;
-    int n = batch.size;
-    long[] vector1 = inputColVector1.vector;
-
-    // return immediately if batch is empty
-    if (n == 0) {
-      return;
-    }
-
-    outputColVector.isRepeating =
-         inputColVector1.isRepeating && inputColVector2.isRepeating
-      || inputColVector1.isRepeating && !inputColVector1.noNulls && inputColVector1.isNull[0]
-      || inputColVector2.isRepeating && !inputColVector2.noNulls && inputColVector2.isNull[0];
-
-    // Handle nulls first  
-    NullUtil.propagateNullsColCol(
-      inputColVector1, inputColVector2, outputColVector, sel, n, batch.selectedInUse);
-
-    /* Disregard nulls for processing. In other words,
-     * the arithmetic operation is performed even if one or
-     * more inputs are null. This is to improve speed by avoiding
-     * conditional checks in the inner loop.
-     */
-    if (inputColVector1.isRepeating && inputColVector2.isRepeating) {
-      outputColVector.<OperatorMethod>(
-          scratchPisaTimestamp.updateFromTimestampMilliseconds(DateWritable.daysToMillis((int) vector1[0])),
-          inputColVector2.asScratchPisaTimestamp(0),
-          0);
-    } else if (inputColVector1.isRepeating) {
-        PisaTimestamp value1 =
-            scratchPisaTimestamp.updateFromTimestampMilliseconds(DateWritable.daysToMillis((int) vector1[0]));
-      if (batch.selectedInUse) {
-        for(int j = 0; j != n; j++) {
-          int i = sel[j];
-          outputColVector.<OperatorMethod>(
-              value1,
-              inputColVector2.asScratchPisaTimestamp(i),
-              i);
-        }
-      } else {
-        for(int i = 0; i != n; i++) {
-          outputColVector.<OperatorMethod>(
-              value1,
-              inputColVector2.asScratchPisaTimestamp(i),
-              i);
-        }
-      }
-    } else if (inputColVector2.isRepeating) {
-      PisaTimestamp value2 = inputColVector2.asScratchPisaTimestamp(0);
-      if (batch.selectedInUse) {
-        for(int j = 0; j != n; j++) {
-          int i = sel[j];
-          outputColVector.<OperatorMethod>(
-              scratchPisaTimestamp.updateFromTimestampMilliseconds(DateWritable.daysToMillis((int) vector1[i])),
-              value2,
-              i);
-         }
-      } else {
-        for(int i = 0; i != n; i++) {
-          outputColVector.<OperatorMethod>(
-              scratchPisaTimestamp.updateFromTimestampMilliseconds(DateWritable.daysToMillis((int) vector1[i])),
-              value2,
-              i);
-        }
-      }
-    } else {
-      if (batch.selectedInUse) {
-        for(int j = 0; j != n; j++) {
-          int i = sel[j];
-          outputColVector.<OperatorMethod>(
-              scratchPisaTimestamp.updateFromTimestampMilliseconds(DateWritable.daysToMillis((int) vector1[i])),
-              inputColVector2.asScratchPisaTimestamp(i),
-              i);
-        }
-      } else {
-        for(int i = 0; i != n; i++) {
-          outputColVector.<OperatorMethod>(
-              scratchPisaTimestamp.updateFromTimestampMilliseconds(DateWritable.daysToMillis((int) vector1[i])),
-              inputColVector2.asScratchPisaTimestamp(i),
-              i);
-        }
-      }
-    }
-
-    /* For the case when the output can have null values, follow
-     * the convention that the data values must be 1 for long and
-     * NaN for double. This is to prevent possible later zero-divide errors
-     * in complex arithmetic expressions like col2 / (col1 - 1)
-     * in the case when some col1 entries are null.
-     */
-    NullUtil.setNullDataEntriesTimestamp(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/52016296/ql/src/gen/vectorization/ExpressionTemplates/DateColumnArithmeticTimestampScalar.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/DateColumnArithmeticTimestampScalar.txt b/ql/src/gen/vectorization/ExpressionTemplates/DateColumnArithmeticTimestampScalar.txt
index b813d11..7aee529 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/DateColumnArithmeticTimestampScalar.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/DateColumnArithmeticTimestampScalar.txt
@@ -19,32 +19,123 @@
 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.hive.common.util.DateUtils;
 
+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.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.ql.exec.vector.VectorExpressionDescriptor;
 import org.apache.hadoop.hive.ql.exec.vector.*;
 import org.apache.hadoop.hive.ql.util.DateTimeMath;
+import org.apache.hadoop.hive.serde2.io.DateWritable;
 
 /**
- * Generated from template DateColumnArithmeticTimestampScalar.txt, which covers binary arithmetic
- * expressions between a date column and a timestamp scalar.
+ * Generated from template DateColumnArithmeticTimestampScalarBase.txt, a base class
+ * which covers binary arithmetic expressions between a date column and a timestamp scalar.
  */
-public class <ClassName> extends <BaseClassName> {
+public class <ClassName> extends VectorExpression {
 
   private static final long serialVersionUID = 1L;
 
-  public <ClassName>(int colNum, <ScalarHiveTimestampType2> value, int outputColumn) {
-    super(colNum, <PisaTimestampConversion2>, outputColumn);
+  private int colNum;
+  private <HiveOperandType2> value;
+  private int outputColumn;
+  private Timestamp scratchTimestamp1;
+  private DateTimeMath dtm = new DateTimeMath();
+
+  public <ClassName>(int colNum, <HiveOperandType2> value, int outputColumn) {
+    this.colNum = colNum;
+    this.value = value;
+    this.outputColumn = outputColumn;
+    scratchTimestamp1 = new Timestamp(0);
   }
 
   public <ClassName>() {
-    super();
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    // Input #1 is type date (days).  For the math we convert it to a timestamp.
+    LongColumnVector inputColVector1 = (LongColumnVector) batch.cols[colNum];
+
+    // Output is type <ReturnType>.
+    <OutputColumnVectorType> outputColVector = (<OutputColumnVectorType>) batch.cols[outputColumn];
+
+    int[] sel = batch.selected;
+    boolean[] inputIsNull = inputColVector1.isNull;
+    boolean[] outputIsNull = outputColVector.isNull;
+    outputColVector.noNulls = inputColVector1.noNulls;
+    outputColVector.isRepeating = inputColVector1.isRepeating;
+    int n = batch.size;
+    long[] vector1 = inputColVector1.vector;
+
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+
+    if (inputColVector1.isRepeating) {
+      scratchTimestamp1.setTime(DateWritable.daysToMillis((int) vector1[0]));
+      dtm.<OperatorMethod>(
+          scratchTimestamp1, value, outputColVector.getScratch<CamelReturnType>());
+      outputColVector.setFromScratch<CamelReturnType>(0);
+      // Even if there are no nulls, we always copy over entry 0. Simplifies code.
+      outputIsNull[0] = inputIsNull[0];
+    } else if (inputColVector1.noNulls) {
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          scratchTimestamp1.setTime(DateWritable.daysToMillis((int) vector1[i]));
+          dtm.<OperatorMethod>(
+             scratchTimestamp1, value, outputColVector.getScratch<CamelReturnType>());
+          outputColVector.setFromScratch<CamelReturnType>(i);
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          scratchTimestamp1.setTime(DateWritable.daysToMillis((int) vector1[i]));
+          dtm.<OperatorMethod>(
+             scratchTimestamp1, value, outputColVector.getScratch<CamelReturnType>());
+          outputColVector.setFromScratch<CamelReturnType>(i);
+        }
+      }
+    } else /* there are nulls */ {
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          scratchTimestamp1.setTime(DateWritable.daysToMillis((int) vector1[i]));
+          dtm.<OperatorMethod>(
+             scratchTimestamp1, value, outputColVector.getScratch<CamelReturnType>());
+          outputColVector.setFromScratch<CamelReturnType>(i);
+          outputIsNull[i] = inputIsNull[i];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          scratchTimestamp1.setTime(DateWritable.daysToMillis((int) vector1[i]));
+          dtm.<OperatorMethod>(
+             scratchTimestamp1, value, outputColVector.getScratch<CamelReturnType>());
+          outputColVector.setFromScratch<CamelReturnType>(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 "<ReturnType>";
   }
 
   @Override
@@ -54,7 +145,7 @@ public class <ClassName> extends <BaseClassName> {
             VectorExpressionDescriptor.Mode.PROJECTION)
         .setNumArguments(2)
         .setArgumentTypes(
-            VectorExpressionDescriptor.ArgumentType.getType("<OperandType1>"),
+            VectorExpressionDescriptor.ArgumentType.getType("date"),
             VectorExpressionDescriptor.ArgumentType.getType("<OperandType2>"))
         .setInputExpressionTypes(
             VectorExpressionDescriptor.InputExpressionType.COLUMN,

http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/ql/src/gen/vectorization/ExpressionTemplates/DateColumnArithmeticTimestampScalarBase.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/DateColumnArithmeticTimestampScalarBase.txt b/ql/src/gen/vectorization/ExpressionTemplates/DateColumnArithmeticTimestampScalarBase.txt
deleted file mode 100644
index d64fba0..0000000
--- a/ql/src/gen/vectorization/ExpressionTemplates/DateColumnArithmeticTimestampScalarBase.txt
+++ /dev/null
@@ -1,137 +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.PisaTimestamp;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
-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.ql.exec.vector.VectorExpressionDescriptor;
-import org.apache.hadoop.hive.ql.exec.vector.*;
-import org.apache.hadoop.hive.serde2.io.DateWritable;
-
-/**
- * Generated from template DateColumnArithmeticTimestampScalarBase.txt, a base class
- * which covers binary arithmetic expressions between a date column and a timestamp scalar.
- */
-public abstract class <BaseClassName> extends VectorExpression {
-
-  private static final long serialVersionUID = 1L;
-
-  private int colNum;
-  private PisaTimestamp value;
-  private int outputColumn;
-  private PisaTimestamp scratchPisaTimestamp;
-
-  public <BaseClassName>(int colNum, PisaTimestamp value, int outputColumn) {
-    this.colNum = colNum;
-    this.value = value;
-    this.outputColumn = outputColumn;
-    scratchPisaTimestamp = new PisaTimestamp();
-  }
-
-  public <BaseClassName>() {
-  }
-
-  @Override
-  public void evaluate(VectorizedRowBatch batch) {
-
-    if (childExpressions != null) {
-      super.evaluateChildren(batch);
-    }
-
-    // Input #1 is type date (epochDays).
-    LongColumnVector inputColVector1 = (LongColumnVector) batch.cols[colNum];
-
-    // Output is type timestamp.
-    TimestampColumnVector outputColVector = (TimestampColumnVector) batch.cols[outputColumn];
-
-    int[] sel = batch.selected;
-    boolean[] inputIsNull = inputColVector1.isNull;
-    boolean[] outputIsNull = outputColVector.isNull;
-    outputColVector.noNulls = inputColVector1.noNulls;
-    outputColVector.isRepeating = inputColVector1.isRepeating;
-    int n = batch.size;
-    long[] vector1 = inputColVector1.vector;
-
-    // return immediately if batch is empty
-    if (n == 0) {
-      return;
-    }
-
-    if (inputColVector1.isRepeating) {
-        outputColVector.<OperatorMethod>(
-          scratchPisaTimestamp.updateFromTimestampMilliseconds(DateWritable.daysToMillis((int) vector1[0])),
-          value,
-          0);
-
-      // Even if there are no nulls, we always copy over entry 0. Simplifies code.
-      outputIsNull[0] = inputIsNull[0];
-    } else if (inputColVector1.noNulls) {
-      if (batch.selectedInUse) {
-        for(int j = 0; j != n; j++) {
-          int i = sel[j];
-          outputColVector.<OperatorMethod>(
-            scratchPisaTimestamp.updateFromTimestampMilliseconds(DateWritable.daysToMillis((int) vector1[i])),
-            value,
-            i);
-        }
-      } else {
-        for(int i = 0; i != n; i++) {
-          outputColVector.<OperatorMethod>(
-            scratchPisaTimestamp.updateFromTimestampMilliseconds(DateWritable.daysToMillis((int) vector1[i])),
-            value,
-            i);
-        }
-      }
-    } else /* there are nulls */ {
-      if (batch.selectedInUse) {
-        for(int j = 0; j != n; j++) {
-          int i = sel[j];
-          outputColVector.<OperatorMethod>(
-            scratchPisaTimestamp.updateFromTimestampMilliseconds(DateWritable.daysToMillis((int) vector1[i])),
-            value,
-            i);
-          outputIsNull[i] = inputIsNull[i];
-        }
-      } else {
-        for(int i = 0; i != n; i++) {
-          outputColVector.<OperatorMethod>(
-            scratchPisaTimestamp.updateFromTimestampMilliseconds(DateWritable.daysToMillis((int) vector1[i])),
-            value,
-            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/52016296/ql/src/gen/vectorization/ExpressionTemplates/DateScalarArithmeticIntervalYearMonthColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/DateScalarArithmeticIntervalYearMonthColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/DateScalarArithmeticIntervalYearMonthColumn.txt
index 653565e..c68ac34 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/DateScalarArithmeticIntervalYearMonthColumn.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/DateScalarArithmeticIntervalYearMonthColumn.txt
@@ -18,6 +18,8 @@
 
 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.VectorExpressionDescriptor;
 import org.apache.hadoop.hive.ql.exec.vector.*;
@@ -33,6 +35,7 @@ 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.ql.util.DateTimeMath;
+import org.apache.hadoop.hive.serde2.io.DateWritable;
 
 /**
  * Generated from template DateTimeScalarArithmeticIntervalYearMonthColumn.txt.
@@ -44,14 +47,18 @@ public class <ClassName> extends VectorExpression {
   private static final long serialVersionUID = 1L;
 
   private int colNum;
-  private long value;
+  private Date value;
   private int outputColumn;
+  private HiveIntervalYearMonth scratchIntervalYearMonth2;
+  private Date outputDate;
   private DateTimeMath dtm = new DateTimeMath();
 
   public <ClassName>(long value, int colNum, int outputColumn) {
     this.colNum = colNum;
-    this.value = value;
+    this.value = new Date(DateWritable.daysToMillis((int) value));
     this.outputColumn = outputColumn;
+    scratchIntervalYearMonth2 = new HiveIntervalYearMonth();
+    outputDate = new Date(0);
   }
 
   public <ClassName>() {
@@ -70,18 +77,18 @@ public class <ClassName> extends VectorExpression {
     }
 
     // Input #2 is type Interval_Year_Month (months).
-    LongColumnVector inputColVector = (LongColumnVector) batch.cols[colNum];
+    LongColumnVector inputColVector2 = (LongColumnVector) batch.cols[colNum];
 
     // Output is type Date.
     LongColumnVector outputColVector = (LongColumnVector) batch.cols[outputColumn];
 
     int[] sel = batch.selected;
-    boolean[] inputIsNull = inputColVector.isNull;
+    boolean[] inputIsNull = inputColVector2.isNull;
     boolean[] outputIsNull = outputColVector.isNull;
-    outputColVector.noNulls = inputColVector.noNulls;
-    outputColVector.isRepeating = inputColVector.isRepeating;
+    outputColVector.noNulls = inputColVector2.noNulls;
+    outputColVector.isRepeating = inputColVector2.isRepeating;
     int n = batch.size;
-    long[] vector = inputColVector.vector;
+    long[] vector2 = inputColVector2.vector;
     long[] outputVector = outputColVector.vector;
 
     // return immediately if batch is empty
@@ -89,32 +96,46 @@ public class <ClassName> extends VectorExpression {
       return;
     }
 
-    if (inputColVector.isRepeating) {
-      outputVector[0] = dtm.addMonthsToDays(value, <OperatorSymbol> (int) vector[0]);
-
-      // Even if there are no nulls, we always copy over entry 0. Simplifies code.
+    if (inputColVector2.isRepeating) {
+      scratchIntervalYearMonth2.set((int) vector2[0]);
+      dtm.<OperatorMethod>(
+          value, scratchIntervalYearMonth2, outputDate);
+      outputVector[0] = DateWritable.dateToDays(outputDate);
+       // Even if there are no nulls, we always copy over entry 0. Simplifies code.
       outputIsNull[0] = inputIsNull[0];
-    } else if (inputColVector.noNulls) {
+    } else if (inputColVector2.noNulls) {
       if (batch.selectedInUse) {
         for(int j = 0; j != n; j++) {
           int i = sel[j];
-          outputVector[i] = dtm.addMonthsToDays(value, <OperatorSymbol> (int) vector[i]);
+          scratchIntervalYearMonth2.set((int) vector2[i]);
+          dtm.<OperatorMethod>(
+              value, scratchIntervalYearMonth2, outputDate);
+          outputVector[i] = DateWritable.dateToDays(outputDate);
         }
       } else {
         for(int i = 0; i != n; i++) {
-          outputVector[i] = dtm.addMonthsToDays(value, <OperatorSymbol> (int) vector[i]);
+          scratchIntervalYearMonth2.set((int) vector2[i]);
+          dtm.<OperatorMethod>(
+              value, scratchIntervalYearMonth2, outputDate);
+          outputVector[i] = DateWritable.dateToDays(outputDate);
         }
       }
     } else {                         /* there are nulls */
       if (batch.selectedInUse) {
         for(int j = 0; j != n; j++) {
           int i = sel[j];
-          outputVector[i] = dtm.addMonthsToDays(value, <OperatorSymbol> (int) vector[i]);
+          scratchIntervalYearMonth2.set((int) vector2[i]);
+          dtm.<OperatorMethod>(
+              value, scratchIntervalYearMonth2, outputDate);
+          outputVector[i] = DateWritable.dateToDays(outputDate);
           outputIsNull[i] = inputIsNull[i];
         }
       } else {
         for(int i = 0; i != n; i++) {
-          outputVector[i] = dtm.addMonthsToDays(value, <OperatorSymbol> (int) vector[i]);
+          scratchIntervalYearMonth2.set((int) vector2[i]);
+          dtm.<OperatorMethod>(
+              value, scratchIntervalYearMonth2, outputDate);
+          outputVector[i] = DateWritable.dateToDays(outputDate);
         }
         System.arraycopy(inputIsNull, 0, outputIsNull, 0, n);
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/ql/src/gen/vectorization/ExpressionTemplates/DateScalarArithmeticTimestampColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/DateScalarArithmeticTimestampColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/DateScalarArithmeticTimestampColumn.txt
index e93bed5..cb6b750 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/DateScalarArithmeticTimestampColumn.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/DateScalarArithmeticTimestampColumn.txt
@@ -18,45 +18,141 @@
 
 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.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.TimestampColumnVector;
+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.ql.util.DateTimeMath;
+import org.apache.hadoop.hive.serde2.io.DateWritable;
 
 /**
- * Generated from template DateScalarArithmeticTimestampColumn.txt.
+ * 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 class <ClassName> extends <BaseClassName> {
+public class <ClassName> extends VectorExpression {
 
   private static final long serialVersionUID = 1L;
 
+  private int colNum;
+  private Timestamp value;
+  private int outputColumn;
+  private DateTimeMath dtm = new DateTimeMath();
+
   public <ClassName>(long value, int colNum, int outputColumn) {
-    super(value, colNum, outputColumn);
+    this.colNum = colNum;
+    // Scalar input #1 is type date (days).  For the math we convert it to a timestamp.
+    this.value = new Timestamp(0);
+    this.value.setTime(DateWritable.daysToMillis((int) value));
+    this.outputColumn = outputColumn;
   }
 
   public <ClassName>() {
   }
 
   @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 <OperandType2>.
+    <InputColumnVectorType2> inputColVector2 = (<InputColumnVectorType2>) batch.cols[colNum];
+
+    // Output is type <ReturnType>.
+    <OutputColumnVectorType> outputColVector = (<OutputColumnVectorType>) 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) {
+      dtm.<OperatorMethod>(
+          value, inputColVector2.asScratch<CamelOperandType2>(0), outputColVector.getScratch<CamelReturnType>());
+      outputColVector.setFromScratch<CamelReturnType>(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];
+          dtm.<OperatorMethod>(
+              value, inputColVector2.asScratch<CamelOperandType2>(i), outputColVector.getScratch<CamelReturnType>());
+          outputColVector.setFromScratch<CamelReturnType>(i);
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          dtm.<OperatorMethod>(
+              value, inputColVector2.asScratch<CamelOperandType2>(i), outputColVector.getScratch<CamelReturnType>());
+          outputColVector.setFromScratch<CamelReturnType>(i);
+        }
+      }
+    } else {                         /* there are nulls */
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          dtm.<OperatorMethod>(
+              value, inputColVector2.asScratch<CamelOperandType2>(i), outputColVector.getScratch<CamelReturnType>());
+          outputColVector.setFromScratch<CamelReturnType>(i);
+          outputIsNull[i] = inputIsNull[i];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          dtm.<OperatorMethod>(
+              value, inputColVector2.asScratch<CamelOperandType2>(i), outputColVector.getScratch<CamelReturnType>());
+          outputColVector.setFromScratch<CamelReturnType>(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 "<ReturnType>";
+  }
+
+  @Override
   public VectorExpressionDescriptor.Descriptor getDescriptor() {
     return (new VectorExpressionDescriptor.Builder())
         .setMode(
             VectorExpressionDescriptor.Mode.PROJECTION)
         .setNumArguments(2)
         .setArgumentTypes(
-            VectorExpressionDescriptor.ArgumentType.getType("<OperandType1>"),
+            VectorExpressionDescriptor.ArgumentType.getType("date"),
             VectorExpressionDescriptor.ArgumentType.getType("<OperandType2>"))
         .setInputExpressionTypes(
             VectorExpressionDescriptor.InputExpressionType.SCALAR,

http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/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/52016296/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/52016296/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/52016296/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/52016296/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/52016296/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/52016296/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/52016296/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/52016296/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/52016296/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;
-  }
-}