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/19 12:13:09 UTC

[15/20] hive git commit: HIVE-9862 Vectorized execution corrupts timestamp values (Matt McCline, reviewed by Jason Dere) 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/130293e5/ql/src/gen/vectorization/ExpressionTemplates/IntervalYearMonthScalarArithmeticDateColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/IntervalYearMonthScalarArithmeticDateColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/IntervalYearMonthScalarArithmeticDateColumn.txt
new file mode 100644
index 0000000..ddde913
--- /dev/null
+++ b/ql/src/gen/vectorization/ExpressionTemplates/IntervalYearMonthScalarArithmeticDateColumn.txt
@@ -0,0 +1,170 @@
+/**
+ * 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.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.*;
+
+
+/*
+ * 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.ql.util.DateTimeMath;
+import org.apache.hadoop.hive.serde2.io.DateWritable;
+
+/**
+ * Generated from template DateTimeScalarArithmeticIntervalYearMonthColumn.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 VectorExpression {
+
+  private static final long serialVersionUID = 1L;
+
+  private int colNum;
+  private HiveIntervalYearMonth value;
+  private int outputColumn;
+  private Date scratchDate2;
+  private Date outputDate;
+  private DateTimeMath dtm = new DateTimeMath();
+
+  public <ClassName>(long value, int colNum, int outputColumn) {
+    this.colNum = colNum;
+    this.value = new HiveIntervalYearMonth((int) value);
+    this.outputColumn = outputColumn;
+    scratchDate2 = new Date(0);
+    outputDate = new Date(0);
+  }
+
+  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 date.
+    LongColumnVector inputColVector2 = (LongColumnVector) batch.cols[colNum];
+
+    // Output is type Date.
+    LongColumnVector outputColVector = (LongColumnVector) 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;
+    long[] vector2 = inputColVector2.vector;
+    long[] outputVector = outputColVector.vector;
+
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+
+    if (inputColVector2.isRepeating) {
+      scratchDate2.setTime(DateWritable.daysToMillis((int) vector2[0]));
+      dtm.<OperatorMethod>(
+          value, scratchDate2, 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 (inputColVector2.noNulls) {
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          scratchDate2.setTime(DateWritable.daysToMillis((int) vector2[i]));
+          dtm.<OperatorMethod>(
+              value, scratchDate2, outputDate);
+          outputVector[i] = DateWritable.dateToDays(outputDate);
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          scratchDate2.setTime(DateWritable.daysToMillis((int) vector2[i]));
+          dtm.<OperatorMethod>(
+              value, scratchDate2, outputDate);
+          outputVector[i] = DateWritable.dateToDays(outputDate);
+        }
+      }
+    } else {                         /* there are nulls */
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          scratchDate2.setTime(DateWritable.daysToMillis((int) vector2[i]));
+          dtm.<OperatorMethod>(
+              value, scratchDate2, outputDate);
+          outputVector[i] = DateWritable.dateToDays(outputDate);
+          outputIsNull[i] = inputIsNull[i];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          scratchDate2.setTime(DateWritable.daysToMillis((int) vector2[i]));
+          dtm.<OperatorMethod>(
+              value, scratchDate2, outputDate);
+          outputVector[i] = DateWritable.dateToDays(outputDate);
+        }
+        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 "long";
+  }
+
+  @Override
+  public VectorExpressionDescriptor.Descriptor getDescriptor() {
+    return (new VectorExpressionDescriptor.Builder())
+        .setMode(
+            VectorExpressionDescriptor.Mode.PROJECTION)
+        .setNumArguments(2)
+        .setArgumentTypes(
+            VectorExpressionDescriptor.ArgumentType.getType("interval_year_month"),
+            VectorExpressionDescriptor.ArgumentType.getType("date"))
+        .setInputExpressionTypes(
+            VectorExpressionDescriptor.InputExpressionType.SCALAR,
+            VectorExpressionDescriptor.InputExpressionType.COLUMN).build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/130293e5/ql/src/gen/vectorization/ExpressionTemplates/IntervalYearMonthScalarArithmeticTimestampColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/IntervalYearMonthScalarArithmeticTimestampColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/IntervalYearMonthScalarArithmeticTimestampColumn.txt
new file mode 100644
index 0000000..cbb7021
--- /dev/null
+++ b/ql/src/gen/vectorization/ExpressionTemplates/IntervalYearMonthScalarArithmeticTimestampColumn.txt
@@ -0,0 +1,158 @@
+/**
+ * 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.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.*;
+
+/*
+ * 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.ql.util.DateTimeMath;
+
+/**
+ * Generated from template TimestampScalarArithmeticIntervalYearMonthColumn.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 VectorExpression {
+
+  private static final long serialVersionUID = 1L;
+
+  private int colNum;
+  private HiveIntervalYearMonth value;
+  private int outputColumn;
+  private DateTimeMath dtm = new DateTimeMath();
+
+  public <ClassName>(long value, int colNum, int outputColumn) {
+    this.colNum = colNum;
+    this.value = new HiveIntervalYearMonth((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 timestamp.
+    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) {
+      dtm.<OperatorMethod>(
+          value, inputColVector2.asScratchTimestamp(0), outputColVector.getScratchTimestamp());
+      outputColVector.setFromScratchTimestamp(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.asScratchTimestamp(i), outputColVector.getScratchTimestamp());
+          outputColVector.setFromScratchTimestamp(i);
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          dtm.<OperatorMethod>(
+              value, inputColVector2.asScratchTimestamp(i), outputColVector.getScratchTimestamp());
+          outputColVector.setFromScratchTimestamp(i);
+        }
+      }
+    } else {                         /* there are nulls */
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          dtm.<OperatorMethod>(
+              value, inputColVector2.asScratchTimestamp(i), outputColVector.getScratchTimestamp());
+          outputColVector.setFromScratchTimestamp(i);
+          outputIsNull[i] = inputIsNull[i];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          dtm.<OperatorMethod>(
+              value, inputColVector2.asScratchTimestamp(i), outputColVector.getScratchTimestamp());
+          outputColVector.setFromScratchTimestamp(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";
+  }
+
+  @Override
+  public VectorExpressionDescriptor.Descriptor getDescriptor() {
+    return (new VectorExpressionDescriptor.Builder())
+        .setMode(
+            VectorExpressionDescriptor.Mode.PROJECTION)
+        .setNumArguments(2)
+        .setArgumentTypes(
+            VectorExpressionDescriptor.ArgumentType.getType("interval_year_month"),
+            VectorExpressionDescriptor.ArgumentType.getType("timestamp"))
+        .setInputExpressionTypes(
+            VectorExpressionDescriptor.InputExpressionType.SCALAR,
+            VectorExpressionDescriptor.InputExpressionType.COLUMN).build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/130293e5/ql/src/gen/vectorization/ExpressionTemplates/LongDoubleColumnCompareTimestampColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/LongDoubleColumnCompareTimestampColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/LongDoubleColumnCompareTimestampColumn.txt
new file mode 100644
index 0000000..9ccfaac
--- /dev/null
+++ b/ql/src/gen/vectorization/ExpressionTemplates/LongDoubleColumnCompareTimestampColumn.txt
@@ -0,0 +1,154 @@
+/**
+ * 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.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.NullUtil;
+import org.apache.hadoop.hive.ql.exec.vector.*;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+
+/**
+ * Generated from template ColumnCompareTimestampColumn.txt, which covers binary arithmetic
+ * expressions between columns.
+ */
+public class <ClassName> extends VectorExpression {
+
+  private static final long serialVersionUID = 1L;
+
+  private int colNum1;
+  private int colNum2;
+  private int outputColumn;
+
+  public <ClassName>(int colNum1, int colNum2, int outputColumn) {
+    this.colNum1 = colNum1;
+    this.colNum2 = colNum2;
+    this.outputColumn = outputColumn;
+  }
+
+  public <ClassName>() {
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    <InputColumnVectorType1> inputColVector1 = (<InputColumnVectorType1>) batch.cols[colNum1];
+    TimestampColumnVector inputColVector2 = (TimestampColumnVector) batch.cols[colNum2];
+    LongColumnVector outputColVector = (LongColumnVector) batch.cols[outputColumn];
+    int[] sel = batch.selected;
+    int n = batch.size;
+    <OperandType>[] vector1 = inputColVector1.vector;
+    long[] outputVector = outputColVector.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) {
+      outputVector[0] = vector1[0] <OperatorSymbol> inputColVector2.<GetTimestampLongDoubleMethod>(0) ? 1 : 0;
+    } else if (inputColVector1.isRepeating) {
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = vector1[0] <OperatorSymbol> inputColVector2.<GetTimestampLongDoubleMethod>(i) ? 1 : 0;
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = vector1[0] <OperatorSymbol> inputColVector2.<GetTimestampLongDoubleMethod>(i) ? 1 : 0;
+        }
+      }
+    } else if (inputColVector2.isRepeating) {
+      <OperandType> value2 = inputColVector2.<GetTimestampLongDoubleMethod>(0);
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = vector1[i] <OperatorSymbol> value2 ? 1 : 0;
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = vector1[i] <OperatorSymbol> value2 ? 1 : 0;
+        }
+      }
+    } else {
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = vector1[i] <OperatorSymbol> inputColVector2.<GetTimestampLongDoubleMethod>(i) ? 1 : 0;
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = vector1[i] <OperatorSymbol> inputColVector2.<GetTimestampLongDoubleMethod>(i) ? 1 : 0;
+        }
+      }
+    }
+
+    /* For the case when the output can have null values, follow
+     * the convention that the data values must be 1 for long and
+     * NaN for double. This is to prevent possible later zero-divide errors
+     * in complex arithmetic expressions like col2 / (col1 - 1)
+     * in the case when some col1 entries are null.
+     */
+    NullUtil.setNullDataEntriesLong(outputColVector, batch.selectedInUse, sel, n);
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return outputColumn;
+  }
+
+  @Override
+  public String getOutputType() {
+    return "long";
+  }
+
+  @Override
+  public VectorExpressionDescriptor.Descriptor getDescriptor() {
+    return (new VectorExpressionDescriptor.Builder())
+        .setMode(
+            VectorExpressionDescriptor.Mode.PROJECTION)
+        .setNumArguments(2)
+        .setArgumentTypes(
+            VectorExpressionDescriptor.ArgumentType.getType("<OperandType>"),
+            VectorExpressionDescriptor.ArgumentType.getType("timestamp"))
+        .setInputExpressionTypes(
+            VectorExpressionDescriptor.InputExpressionType.COLUMN,
+            VectorExpressionDescriptor.InputExpressionType.COLUMN).build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/130293e5/ql/src/gen/vectorization/ExpressionTemplates/LongDoubleColumnCompareTimestampScalar.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/LongDoubleColumnCompareTimestampScalar.txt b/ql/src/gen/vectorization/ExpressionTemplates/LongDoubleColumnCompareTimestampScalar.txt
new file mode 100644
index 0000000..c7d8c65
--- /dev/null
+++ b/ql/src/gen/vectorization/ExpressionTemplates/LongDoubleColumnCompareTimestampScalar.txt
@@ -0,0 +1,146 @@
+/**
+ * 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.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.exec.vector.*;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+
+/**
+ * Generated from template ColumnCompareTimestampScalar.txt, which covers binary comparison
+ * expressions between a column and a scalar. The boolean output is stored in a
+ * separate boolean column.
+ */
+public class <ClassName> extends VectorExpression {
+
+  private static final long serialVersionUID = 1L;
+
+  private int colNum;
+  private <OperandType> value;
+  private int outputColumn;
+
+  public <ClassName>(int colNum, Timestamp value, int outputColumn) {
+    this.colNum = colNum;
+    this.value = TimestampColumnVector.<GetTimestampLongDoubleMethod>(value);
+    this.outputColumn = outputColumn;
+  }
+
+  public <ClassName>() {
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    <InputColumnVectorType1> inputColVector1 = (<InputColumnVectorType1>) batch.cols[colNum];
+    LongColumnVector outputColVector = (LongColumnVector) batch.cols[outputColumn];
+    int[] sel = batch.selected;
+    boolean[] nullPos = inputColVector1.isNull;
+    boolean[] outNulls = outputColVector.isNull;
+    int n = batch.size;
+    <OperandType>[] vector1 = inputColVector1.vector;
+    long[] outputVector = outputColVector.vector;
+
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+
+    outputColVector.isRepeating = false;
+    outputColVector.noNulls = inputColVector1.noNulls;
+    if (inputColVector1.noNulls) {
+      if (inputColVector1.isRepeating) {
+        //All must be selected otherwise size would be zero
+        //Repeating property will not change.
+        outputVector[0] = vector1[0] <OperatorSymbol> value ? 1 : 0;
+        outputColVector.isRepeating = true;
+      } else if (batch.selectedInUse) {
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = vector1[i] <OperatorSymbol> value ? 1 : 0;
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = vector1[i] <OperatorSymbol> value ? 1 : 0;
+        }
+      }
+    } else {
+      if (inputColVector1.isRepeating) {
+        //All must be selected otherwise size would be zero
+        //Repeating property will not change.
+        if (!nullPos[0]) {
+          outputVector[0] = vector1[0] <OperatorSymbol> value ? 1 : 0;
+          outNulls[0] = false;
+        } else {
+          outNulls[0] = true;
+        }
+        outputColVector.isRepeating = true;
+      } else if (batch.selectedInUse) {
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          if (!nullPos[i]) {
+            outputVector[i] = vector1[i] <OperatorSymbol> value ? 1 : 0;
+            outNulls[i] = false;
+          } else {
+            //comparison with null is null
+            outNulls[i] = true;
+          }
+        }
+      } else {
+        System.arraycopy(nullPos, 0, outNulls, 0, n);
+        for(int i = 0; i != n; i++) {
+          if (!nullPos[i]) {
+            outputVector[i] = vector1[i] <OperatorSymbol> value ? 1 : 0;
+          }
+        }
+      }
+    }
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return outputColumn;
+  }
+
+  @Override
+  public String getOutputType() {
+    return "long";
+  }
+
+
+  @Override
+  public VectorExpressionDescriptor.Descriptor getDescriptor() {
+    return (new VectorExpressionDescriptor.Builder())
+        .setMode(
+            VectorExpressionDescriptor.Mode.PROJECTION)
+        .setNumArguments(2)
+        .setArgumentTypes(
+            VectorExpressionDescriptor.ArgumentType.getType("<OperandType>"),
+            VectorExpressionDescriptor.ArgumentType.getType("timestamp"))
+        .setInputExpressionTypes(
+            VectorExpressionDescriptor.InputExpressionType.COLUMN,
+            VectorExpressionDescriptor.InputExpressionType.SCALAR).build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/130293e5/ql/src/gen/vectorization/ExpressionTemplates/LongDoubleScalarCompareTimestampColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/LongDoubleScalarCompareTimestampColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/LongDoubleScalarCompareTimestampColumn.txt
new file mode 100644
index 0000000..d47bc10
--- /dev/null
+++ b/ql/src/gen/vectorization/ExpressionTemplates/LongDoubleScalarCompareTimestampColumn.txt
@@ -0,0 +1,144 @@
+/**
+ * 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.LongColumnVector;
+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 ScalarCompareTimestamp.txt, which covers comparison
+ * expressions between a long/double scalar and a column. The boolean output is stored in a
+ * separate boolean column.
+ */
+public class <ClassName> extends VectorExpression {
+
+  private static final long serialVersionUID = 1L;
+
+  private int colNum;
+  private <OperandType> value;
+  private int outputColumn;
+
+  public <ClassName>(<OperandType> value, int colNum, int outputColumn) {
+    this.colNum = colNum;
+    this.value = value;
+    this.outputColumn = outputColumn;
+  }
+
+  public <ClassName>() {
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    TimestampColumnVector inputColVector = (TimestampColumnVector) batch.cols[colNum];
+    LongColumnVector outputColVector = (LongColumnVector) batch.cols[outputColumn];
+    int[] sel = batch.selected;
+    boolean[] nullPos = inputColVector.isNull;
+    boolean[] outNulls = outputColVector.isNull;
+    int n = batch.size;
+    long[] outputVector = outputColVector.vector;
+
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+
+    outputColVector.isRepeating = false;
+    outputColVector.noNulls = inputColVector.noNulls;
+    if (inputColVector.noNulls) {
+      if (inputColVector.isRepeating) {
+        //All must be selected otherwise size would be zero
+        //Repeating property will not change.
+        outputVector[0] = value <OperatorSymbol> inputColVector.<GetTimestampLongDoubleMethod>(0) ? 1 : 0;
+        outputColVector.isRepeating = true;
+      } else if (batch.selectedInUse) {
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = value <OperatorSymbol> inputColVector.<GetTimestampLongDoubleMethod>(i) ? 1 : 0;
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = value <OperatorSymbol> inputColVector.<GetTimestampLongDoubleMethod>(i) ? 1 : 0;
+        }
+      }
+    } else {
+      if (inputColVector.isRepeating) {
+        //All must be selected otherwise size would be zero
+        //Repeating property will not change.
+        if (!nullPos[0]) {
+          outputVector[0] = value <OperatorSymbol> inputColVector.<GetTimestampLongDoubleMethod>(0) ? 1 : 0;
+          outNulls[0] = false;
+        } else {
+          outNulls[0] = true;
+        }
+        outputColVector.isRepeating = true;
+      } else if (batch.selectedInUse) {
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          if (!nullPos[i]) {
+            outputVector[i] = value <OperatorSymbol> inputColVector.<GetTimestampLongDoubleMethod>(i) ? 1 : 0;
+            outNulls[i] = false;
+          } else {
+            //comparison with null is null
+            outNulls[i] = true;
+          }
+        }
+      } else {
+        System.arraycopy(nullPos, 0, outNulls, 0, n);
+        for(int i = 0; i != n; i++) {
+          if (!nullPos[i]) {
+            outputVector[i] = value <OperatorSymbol> inputColVector.<GetTimestampLongDoubleMethod>(i) ? 1 : 0;
+          }
+        }
+      }
+    }
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return outputColumn;
+  }
+
+  @Override
+  public String getOutputType() {
+    return "long";
+  }
+
+  @Override
+  public VectorExpressionDescriptor.Descriptor getDescriptor() {
+    return (new VectorExpressionDescriptor.Builder())
+        .setMode(
+            VectorExpressionDescriptor.Mode.PROJECTION)
+        .setNumArguments(2)
+        .setArgumentTypes(
+            VectorExpressionDescriptor.ArgumentType.getType("<OperandType>"),
+            VectorExpressionDescriptor.ArgumentType.getType("timestamp"))
+        .setInputExpressionTypes(
+            VectorExpressionDescriptor.InputExpressionType.SCALAR,
+            VectorExpressionDescriptor.InputExpressionType.COLUMN).build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/130293e5/ql/src/gen/vectorization/ExpressionTemplates/ScalarCompareTimestampColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/ScalarCompareTimestampColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/ScalarCompareTimestampColumn.txt
deleted file mode 100644
index 7867610..0000000
--- a/ql/src/gen/vectorization/ExpressionTemplates/ScalarCompareTimestampColumn.txt
+++ /dev/null
@@ -1,63 +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.udf.UDFToString;
-import org.apache.hadoop.hive.serde2.io.TimestampWritable;
-import java.sql.Timestamp;
-import org.apache.hadoop.hive.ql.exec.vector.TimestampUtils;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.StringExpr;
-import org.apache.hadoop.io.LongWritable;
-
-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.VectorExpressionDescriptor;
-
-/**
- * Generated from template ScalarCompareTimestampColumn.txt, which covers comparison 
- * expressions between a timestamp column and a long or double scalar. The boolean output
- * is stored in a separate boolean column.
- * Note: For timestamp and long or double we implicitly interpret the long as the number
- * of seconds or double as seconds and fraction since the epoch.
- */
-public class <ClassName> extends <BaseClassName> {
-
-  public <ClassName>(<OperandType> value, int colNum, int outputColumn) {
-    super(TimestampUtils.<TimestampScalarConversion>(value), colNum, outputColumn);
-  }
-
-  public <ClassName>() {
-    super();
-  }
-
-  @Override
-  public VectorExpressionDescriptor.Descriptor getDescriptor() {
-    return (new VectorExpressionDescriptor.Builder())
-        .setMode(
-            VectorExpressionDescriptor.Mode.PROJECTION)
-        .setNumArguments(2)
-        .setArgumentTypes(
-            VectorExpressionDescriptor.ArgumentType.getType("<OperandType>"),
-            VectorExpressionDescriptor.ArgumentType.getType("timestamp"))
-        .setInputExpressionTypes(
-            VectorExpressionDescriptor.InputExpressionType.SCALAR,
-            VectorExpressionDescriptor.InputExpressionType.COLUMN).build();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/130293e5/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnArithmeticDateColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnArithmeticDateColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnArithmeticDateColumn.txt
new file mode 100644
index 0000000..27e083d
--- /dev/null
+++ b/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnArithmeticDateColumn.txt
@@ -0,0 +1,187 @@
+/**
+ * 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.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 TimestampColumnArithmeticDateColumn.txt, which covers binary arithmetic
+ * expressions between columns.
+ */
+public class <ClassName> extends VectorExpression {
+
+  private static final long serialVersionUID = 1L;
+
+  private int colNum1;
+  private int colNum2;
+  private int outputColumn;
+  private Timestamp scratchTimestamp2;
+  private DateTimeMath dtm = new DateTimeMath();
+
+  public <ClassName>(int colNum1, int colNum2, int outputColumn) {
+    this.colNum1 = colNum1;
+    this.colNum2 = colNum2;
+    this.outputColumn = outputColumn;
+    scratchTimestamp2 = new Timestamp(0);
+  }
+
+  public <ClassName>() {
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+     // Input #1 is type <OperandType1>.
+    <InputColumnVectorType1> inputColVector1 = (<InputColumnVectorType1>) batch.cols[colNum1];
+
+    // Input #2 is type date (days).  For the math we convert it to a timestamp.
+    LongColumnVector inputColVector2 = (LongColumnVector) batch.cols[colNum2];
+
+    // Output is type <ReturnType>.
+    <OutputColumnVectorType> outputColVector = (<OutputColumnVectorType>) batch.cols[outputColumn];
+
+    int[] sel = batch.selected;
+    int n = batch.size;
+
+    long[] vector2 = inputColVector2.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) {
+      scratchTimestamp2.setTime(DateWritable.daysToMillis((int) vector2[0]));
+      dtm.<OperatorMethod>(
+          inputColVector1.asScratch<CamelOperandType1>(0), scratchTimestamp2, outputColVector.getScratch<CamelReturnType>());
+      outputColVector.setFromScratch<CamelReturnType>(0);
+    } else if (inputColVector1.isRepeating) {
+      <HiveOperandType1> value1 = inputColVector1.asScratch<CamelOperandType1>(0);
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          scratchTimestamp2.setTime(DateWritable.daysToMillis((int) vector2[i]));
+          dtm.<OperatorMethod>(
+              value1, scratchTimestamp2, outputColVector.getScratch<CamelReturnType>());
+          outputColVector.setFromScratch<CamelReturnType>(i);
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          scratchTimestamp2.setTime(DateWritable.daysToMillis((int) vector2[i]));
+          dtm.<OperatorMethod>(
+              value1, scratchTimestamp2, outputColVector.getScratch<CamelReturnType>());
+          outputColVector.setFromScratch<CamelReturnType>(i);
+        }
+      }
+    } else if (inputColVector2.isRepeating) {
+      scratchTimestamp2.setTime(DateWritable.daysToMillis((int) vector2[0]));
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          dtm.<OperatorMethod>(
+              inputColVector1.asScratch<CamelOperandType1>(i), scratchTimestamp2, outputColVector.getScratch<CamelReturnType>());
+          outputColVector.setFromScratch<CamelReturnType>(i);
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          dtm.<OperatorMethod>(
+              inputColVector1.asScratch<CamelOperandType1>(i), scratchTimestamp2, outputColVector.getScratch<CamelReturnType>());
+          outputColVector.setFromScratch<CamelReturnType>(i);
+        }
+      }
+    } else {
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          scratchTimestamp2.setTime(DateWritable.daysToMillis((int) vector2[i]));
+          dtm.<OperatorMethod>(
+              inputColVector1.asScratch<CamelOperandType1>(i), scratchTimestamp2, outputColVector.getScratch<CamelReturnType>());
+          outputColVector.setFromScratch<CamelReturnType>(i);
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          scratchTimestamp2.setTime(DateWritable.daysToMillis((int) vector2[i]));
+          dtm.<OperatorMethod>(
+              inputColVector1.asScratch<CamelOperandType1>(i), scratchTimestamp2, 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 "interval_day_time";
+  }
+
+  @Override
+  public VectorExpressionDescriptor.Descriptor getDescriptor() {
+    return (new VectorExpressionDescriptor.Builder())
+        .setMode(
+            VectorExpressionDescriptor.Mode.PROJECTION)
+        .setNumArguments(2)
+        .setArgumentTypes(
+            VectorExpressionDescriptor.ArgumentType.getType("<OperandType1>"),
+            VectorExpressionDescriptor.ArgumentType.getType("date"))
+        .setInputExpressionTypes(
+            VectorExpressionDescriptor.InputExpressionType.COLUMN,
+            VectorExpressionDescriptor.InputExpressionType.COLUMN).build();
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/hive/blob/130293e5/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnArithmeticDateScalar.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnArithmeticDateScalar.txt b/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnArithmeticDateScalar.txt
new file mode 100644
index 0000000..8b91a4a
--- /dev/null
+++ b/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnArithmeticDateScalar.txt
@@ -0,0 +1,147 @@
+/**
+ * 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.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;
+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 TimestampColumnArithmeticDateScalar.txt, which covers binary arithmetic
+ * expressions between a column and a scalar.
+ */
+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>(int colNum, long value, int outputColumn) {
+    this.colNum = colNum;
+    this.value = new Timestamp(0);
+    this.value.setTime(DateWritable.daysToMillis((int) value));
+    this.outputColumn = outputColumn;
+  }
+
+  public <ClassName>() {
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+     // Input #1 is type <OperandType1>.
+    <InputColumnVectorType1> inputColVector1 = (<InputColumnVectorType1>) 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;
+
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+
+    if (inputColVector1.isRepeating) {
+      dtm.<OperatorMethod>(
+          inputColVector1.asScratch<CamelOperandType1>(0), 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];
+          dtm.<OperatorMethod>(
+              inputColVector1.asScratch<CamelOperandType1>(i), value, outputColVector.getScratch<CamelReturnType>());
+          outputColVector.setFromScratch<CamelReturnType>(i);
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          dtm.<OperatorMethod>(
+              inputColVector1.asScratch<CamelOperandType1>(i), 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];
+          dtm.<OperatorMethod>(
+              inputColVector1.asScratch<CamelOperandType1>(i), value, outputColVector.getScratch<CamelReturnType>());
+          outputColVector.setFromScratch<CamelReturnType>(i);
+          outputIsNull[i] = inputIsNull[i];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          dtm.<OperatorMethod>(
+              inputColVector1.asScratch<CamelOperandType1>(i), 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
+  public VectorExpressionDescriptor.Descriptor getDescriptor() {
+    return (new VectorExpressionDescriptor.Builder())
+        .setMode(
+            VectorExpressionDescriptor.Mode.PROJECTION)
+        .setNumArguments(2)
+        .setArgumentTypes(
+            VectorExpressionDescriptor.ArgumentType.getType("<OperandType1>"),
+            VectorExpressionDescriptor.ArgumentType.getType("date"))
+        .setInputExpressionTypes(
+            VectorExpressionDescriptor.InputExpressionType.COLUMN,
+            VectorExpressionDescriptor.InputExpressionType.SCALAR).build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/130293e5/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnArithmeticIntervalYearMonthColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnArithmeticIntervalYearMonthColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnArithmeticIntervalYearMonthColumn.txt
new file mode 100644
index 0000000..4ac2174
--- /dev/null
+++ b/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnArithmeticIntervalYearMonthColumn.txt
@@ -0,0 +1,186 @@
+/**
+ * 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.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;
+
+/**
+ * Generated from template TimestampColumnArithmeticIntervalYearMonthColumn.txt, which covers binary arithmetic
+ * expressions between columns.
+ */
+public class <ClassName> extends VectorExpression {
+
+  private static final long serialVersionUID = 1L;
+
+  private int colNum1;
+  private int colNum2;
+  private int outputColumn;
+  private HiveIntervalYearMonth scratchIntervalYearMonth2;
+  private DateTimeMath dtm = new DateTimeMath();
+
+  public <ClassName>(int colNum1, int colNum2, int outputColumn) {
+    this.colNum1 = colNum1;
+    this.colNum2 = colNum2;
+    this.outputColumn = outputColumn;
+    scratchIntervalYearMonth2 = new HiveIntervalYearMonth();
+  }
+
+  public <ClassName>() {
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    // Input #1 is type Timestamp.
+    TimestampColumnVector inputColVector1 = (TimestampColumnVector) batch.cols[colNum1];
+
+    // Input #2 is type Interval_Year_Month (months).
+    LongColumnVector inputColVector2 = (LongColumnVector) batch.cols[colNum2];
+
+    // Output is type Timestamp.
+    TimestampColumnVector outputColVector = (TimestampColumnVector) batch.cols[outputColumn];
+
+    int[] sel = batch.selected;
+    int n = batch.size;
+
+    long[] vector2 = inputColVector2.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) {
+      scratchIntervalYearMonth2.set((int) vector2[0]);
+      dtm.<OperatorMethod>(
+          inputColVector1.asScratchTimestamp(0), scratchIntervalYearMonth2, outputColVector.getScratchTimestamp());
+      outputColVector.setFromScratchTimestamp(0);
+    } else if (inputColVector1.isRepeating) {
+      Timestamp value1 = inputColVector1.asScratchTimestamp(0);
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          scratchIntervalYearMonth2.set((int) vector2[i]);
+          dtm.<OperatorMethod>(
+              value1, scratchIntervalYearMonth2, outputColVector.getScratchTimestamp());
+          outputColVector.setFromScratchTimestamp(i);
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+         scratchIntervalYearMonth2.set((int) vector2[i]);
+          dtm.<OperatorMethod>(
+              value1, scratchIntervalYearMonth2, outputColVector.getScratchTimestamp());
+          outputColVector.setFromScratchTimestamp(i);
+        }
+      }
+    } else if (inputColVector2.isRepeating) {
+      scratchIntervalYearMonth2.set((int) vector2[0]);
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          dtm.<OperatorMethod>(
+              inputColVector1.asScratchTimestamp(i), scratchIntervalYearMonth2, outputColVector.getScratchTimestamp());
+          outputColVector.setFromScratchTimestamp(i);
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          dtm.<OperatorMethod>(
+              inputColVector1.asScratchTimestamp(i), scratchIntervalYearMonth2, outputColVector.getScratchTimestamp());
+          outputColVector.setFromScratchTimestamp(i);
+        }
+      }
+    } else {
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          scratchIntervalYearMonth2.set((int) vector2[i]);
+          dtm.<OperatorMethod>(
+              inputColVector1.asScratchTimestamp(i), scratchIntervalYearMonth2, outputColVector.getScratchTimestamp());
+          outputColVector.setFromScratchTimestamp(i);
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          scratchIntervalYearMonth2.set((int) vector2[i]);
+          dtm.<OperatorMethod>(
+              inputColVector1.asScratchTimestamp(i), scratchIntervalYearMonth2, outputColVector.getScratchTimestamp());
+          outputColVector.setFromScratchTimestamp(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";
+  }
+
+  @Override
+  public VectorExpressionDescriptor.Descriptor getDescriptor() {
+    return (new VectorExpressionDescriptor.Builder())
+        .setMode(
+            VectorExpressionDescriptor.Mode.PROJECTION)
+        .setNumArguments(2)
+        .setArgumentTypes(
+            VectorExpressionDescriptor.ArgumentType.getType("timestamp"),
+            VectorExpressionDescriptor.ArgumentType.getType("interval_year_month"))
+        .setInputExpressionTypes(
+            VectorExpressionDescriptor.InputExpressionType.COLUMN,
+            VectorExpressionDescriptor.InputExpressionType.COLUMN).build();
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/hive/blob/130293e5/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnArithmeticIntervalYearMonthScalar.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnArithmeticIntervalYearMonthScalar.txt b/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnArithmeticIntervalYearMonthScalar.txt
new file mode 100644
index 0000000..9382aca
--- /dev/null
+++ b/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnArithmeticIntervalYearMonthScalar.txt
@@ -0,0 +1,143 @@
+/**
+ * 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.HiveIntervalYearMonth;
+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.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;
+
+/**
+ * Generated from template TimestampColumnArithmeticIntervalYearMonthScalar.txt, which covers binary arithmetic
+ * expressions between a column and a scalar.
+ */
+public class <ClassName> extends VectorExpression {
+
+  private static final long serialVersionUID = 1L;
+
+  private int colNum;
+  private HiveIntervalYearMonth value;
+  private int outputColumn;
+  private DateTimeMath dtm = new DateTimeMath();
+
+  public <ClassName>(int colNum, long value, int outputColumn) {
+    this.colNum = colNum;
+    this.value = new HiveIntervalYearMonth((int) value);
+    this.outputColumn = outputColumn;
+  }
+
+  public <ClassName>() {
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    // Input #1 is type Timestamp.
+    TimestampColumnVector inputColVector1 = (TimestampColumnVector) 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;
+
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+
+    if (inputColVector1.isRepeating) {
+      dtm.<OperatorMethod>(
+          inputColVector1.asScratchTimestamp(0), value, outputColVector.getScratchTimestamp());
+      outputColVector.setFromScratchTimestamp(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];
+          dtm.<OperatorMethod>(
+              inputColVector1.asScratchTimestamp(i), value, outputColVector.getScratchTimestamp());
+          outputColVector.setFromScratchTimestamp(i);
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          dtm.<OperatorMethod>(
+              inputColVector1.asScratchTimestamp(i), value, outputColVector.getScratchTimestamp());
+          outputColVector.setFromScratchTimestamp(i);
+        }
+      }
+    } else /* there are nulls */ {
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          dtm.<OperatorMethod>(
+              inputColVector1.asScratchTimestamp(i), value, outputColVector.getScratchTimestamp());
+          outputColVector.setFromScratchTimestamp(i);
+          outputIsNull[i] = inputIsNull[i];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          dtm.<OperatorMethod>(
+              inputColVector1.asScratchTimestamp(i), value, outputColVector.getScratchTimestamp());
+          outputColVector.setFromScratchTimestamp(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";
+  }
+
+  @Override
+  public VectorExpressionDescriptor.Descriptor getDescriptor() {
+    return (new VectorExpressionDescriptor.Builder())
+        .setMode(
+            VectorExpressionDescriptor.Mode.PROJECTION)
+        .setNumArguments(2)
+        .setArgumentTypes(
+            VectorExpressionDescriptor.ArgumentType.getType("timestamp"),
+            VectorExpressionDescriptor.ArgumentType.getType("interval_year_month"))
+        .setInputExpressionTypes(
+            VectorExpressionDescriptor.InputExpressionType.COLUMN,
+            VectorExpressionDescriptor.InputExpressionType.SCALAR).build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/130293e5/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnArithmeticTimestampColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnArithmeticTimestampColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnArithmeticTimestampColumn.txt
new file mode 100644
index 0000000..5eaa450
--- /dev/null
+++ b/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnArithmeticTimestampColumn.txt
@@ -0,0 +1,177 @@
+/**
+ * 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.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;
+
+/**
+ * Generated from template TimestampColumnArithmeticTimestampColumnBase.txt, which covers binary arithmetic
+ * expressions between columns.
+ */
+public class <ClassName> extends VectorExpression {
+
+  private static final long serialVersionUID = 1L;
+
+  private int colNum1;
+  private int colNum2;
+  private int outputColumn;
+  private DateTimeMath dtm = new DateTimeMath();
+
+  public <ClassName>(int colNum1, int colNum2, int outputColumn) {
+    this.colNum1 = colNum1;
+    this.colNum2 = colNum2;
+    this.outputColumn = outputColumn;
+  }
+
+  public <ClassName>() {
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    // Input #1 is type <OperandType1>.
+    <InputColumnVectorType1> inputColVector1 = (<InputColumnVectorType1>) 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;
+
+    // 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) {
+      dtm.<OperatorMethod>(
+          inputColVector1.asScratch<CamelOperandType1>(0), inputColVector2.asScratch<CamelOperandType2>(0), outputColVector.getScratch<CamelReturnType>());
+      outputColVector.setFromScratch<CamelReturnType>(0);
+    } else if (inputColVector1.isRepeating) {
+      <HiveOperandType1> value1 = inputColVector1.asScratch<CamelOperandType1>(0);
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          dtm.<OperatorMethod>(
+              value1, inputColVector2.asScratch<CamelOperandType2>(i), outputColVector.getScratch<CamelReturnType>());
+          outputColVector.setFromScratch<CamelReturnType>(i);
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          dtm.<OperatorMethod>(
+              value1, 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];
+          dtm.<OperatorMethod>(
+              inputColVector1.asScratch<CamelOperandType1>(i), value2, outputColVector.getScratch<CamelReturnType>());
+          outputColVector.setFromScratch<CamelReturnType>(i);
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          dtm.<OperatorMethod>(
+              inputColVector1.asScratch<CamelOperandType1>(i), value2, outputColVector.getScratch<CamelReturnType>());
+          outputColVector.setFromScratch<CamelReturnType>(i);
+        }
+      }
+    } else {
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          dtm.<OperatorMethod>(
+              inputColVector1.asScratch<CamelOperandType1>(i), inputColVector2.asScratch<CamelOperandType2>(i), outputColVector.getScratch<CamelReturnType>());
+          outputColVector.setFromScratch<CamelReturnType>(i);
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          dtm.<OperatorMethod>(
+              inputColVector1.asScratch<CamelOperandType1>(i), 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
+  public VectorExpressionDescriptor.Descriptor getDescriptor() {
+    return (new VectorExpressionDescriptor.Builder())
+        .setMode(
+            VectorExpressionDescriptor.Mode.PROJECTION)
+        .setNumArguments(2)
+        .setArgumentTypes(
+            VectorExpressionDescriptor.ArgumentType.getType("<OperandType1>"),
+            VectorExpressionDescriptor.ArgumentType.getType("<OperandType2>"))
+        .setInputExpressionTypes(
+            VectorExpressionDescriptor.InputExpressionType.COLUMN,
+            VectorExpressionDescriptor.InputExpressionType.COLUMN).build();
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/hive/blob/130293e5/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnArithmeticTimestampScalar.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnArithmeticTimestampScalar.txt b/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnArithmeticTimestampScalar.txt
new file mode 100644
index 0000000..c6c872f
--- /dev/null
+++ b/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnArithmeticTimestampScalar.txt
@@ -0,0 +1,145 @@
+/**
+ * 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.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;
+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;
+
+/**
+ * Generated from template TimestampColumnArithmeticTimestampScalar.txt, which covers binary arithmetic
+ * expressions between a column and a scalar.
+ */
+public class <ClassName> extends VectorExpression {
+
+  private static final long serialVersionUID = 1L;
+
+  private int colNum;
+  private <HiveOperandType2> value;
+  private int outputColumn;
+  private DateTimeMath dtm = new DateTimeMath();
+
+  public <ClassName>(int colNum, <HiveOperandType2> value, int outputColumn) {
+    this.colNum = colNum;
+    this.value = value;
+    this.outputColumn = outputColumn;
+  }
+
+  public <ClassName>() {
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    // Input #1 is type <OperandType1>.
+    <InputColumnVectorType1> inputColVector1 = (<InputColumnVectorType1>) 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;
+
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+
+    if (inputColVector1.isRepeating) {
+      dtm.<OperatorMethod>(
+          inputColVector1.asScratch<CamelOperandType1>(0), 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];
+          dtm.<OperatorMethod>(
+              inputColVector1.asScratch<CamelOperandType1>(i), value, outputColVector.getScratch<CamelReturnType>());
+          outputColVector.setFromScratch<CamelReturnType>(i);
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          dtm.<OperatorMethod>(
+              inputColVector1.asScratch<CamelOperandType1>(i), 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];
+          dtm.<OperatorMethod>(
+              inputColVector1.asScratch<CamelOperandType1>(i), value, outputColVector.getScratch<CamelReturnType>());
+          outputColVector.setFromScratch<CamelReturnType>(i);
+          outputIsNull[i] = inputIsNull[i];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          dtm.<OperatorMethod>(
+              inputColVector1.asScratch<CamelOperandType1>(i), 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
+  public VectorExpressionDescriptor.Descriptor getDescriptor() {
+    return (new VectorExpressionDescriptor.Builder())
+        .setMode(
+            VectorExpressionDescriptor.Mode.PROJECTION)
+        .setNumArguments(2)
+        .setArgumentTypes(
+            VectorExpressionDescriptor.ArgumentType.getType("<OperandType1>"),
+            VectorExpressionDescriptor.ArgumentType.getType("<OperandType2>"))
+        .setInputExpressionTypes(
+            VectorExpressionDescriptor.InputExpressionType.COLUMN,
+            VectorExpressionDescriptor.InputExpressionType.SCALAR).build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/130293e5/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnCompareLongDoubleColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnCompareLongDoubleColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnCompareLongDoubleColumn.txt
new file mode 100644
index 0000000..0fc402d
--- /dev/null
+++ b/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnCompareLongDoubleColumn.txt
@@ -0,0 +1,153 @@
+/**
+ * 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.expressions.NullUtil;
+import org.apache.hadoop.hive.ql.exec.vector.*;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+
+/**
+ * Generated from template ColumnArithmeticColumn.txt, which covers binary arithmetic
+ * expressions between columns.
+ */
+public class <ClassName> extends VectorExpression {
+
+  private static final long serialVersionUID = 1L;
+
+  private int colNum1;
+  private int colNum2;
+  private int outputColumn;
+
+  public <ClassName>(int colNum1, int colNum2, int outputColumn) {
+    this.colNum1 = colNum1;
+    this.colNum2 = colNum2;
+    this.outputColumn = outputColumn;
+  }
+
+  public <ClassName>() {
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    TimestampColumnVector inputColVector1 = (TimestampColumnVector) batch.cols[colNum1];
+    <InputColumnVectorType2> inputColVector2 = (<InputColumnVectorType2>) batch.cols[colNum2];
+    LongColumnVector outputColVector = (LongColumnVector) batch.cols[outputColumn];
+    int[] sel = batch.selected;
+    int n = batch.size;
+    <OperandType>[] vector2 = inputColVector2.vector;
+    long[] outputVector = outputColVector.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) {
+      outputVector[0] = inputColVector1.<GetTimestampLongDoubleMethod>(0) <OperatorSymbol> vector2[0] ? 1 : 0;
+    } else if (inputColVector1.isRepeating) {
+      <OperandType> value1 = inputColVector1.<GetTimestampLongDoubleMethod>(0);
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = value1 <OperatorSymbol> vector2[i] ? 1 : 0;
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = value1 <OperatorSymbol> vector2[i] ? 1 : 0;
+        }
+      }
+    } else if (inputColVector2.isRepeating) {
+      <OperandType> value2 = vector2[0];
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = inputColVector1.<GetTimestampLongDoubleMethod>(i) <OperatorSymbol> vector2[0] ? 1 : 0;
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = inputColVector1.<GetTimestampLongDoubleMethod>(i) <OperatorSymbol> vector2[0] ? 1 : 0;
+        }
+      }
+    } else {
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = inputColVector1.<GetTimestampLongDoubleMethod>(i) <OperatorSymbol> vector2[i] ? 1 : 0;
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = inputColVector1.<GetTimestampLongDoubleMethod>(i) <OperatorSymbol> vector2[i] ? 1 : 0;
+        }
+      }
+    }
+
+    /* For the case when the output can have null values, follow
+     * the convention that the data values must be 1 for long and
+     * NaN for double. This is to prevent possible later zero-divide errors
+     * in complex arithmetic expressions like col2 / (col1 - 1)
+     * in the case when some col1 entries are null.
+     */
+    NullUtil.setNullDataEntriesLong(outputColVector, batch.selectedInUse, sel, n);
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return outputColumn;
+  }
+
+  @Override
+  public String getOutputType() {
+    return "long";
+  }
+
+  @Override
+  public VectorExpressionDescriptor.Descriptor getDescriptor() {
+    return (new VectorExpressionDescriptor.Builder())
+        .setMode(
+            VectorExpressionDescriptor.Mode.PROJECTION)
+        .setNumArguments(2)
+        .setArgumentTypes(
+            VectorExpressionDescriptor.ArgumentType.getType("timestamp"),
+            VectorExpressionDescriptor.ArgumentType.getType("<OperandType>"))
+        .setInputExpressionTypes(
+            VectorExpressionDescriptor.InputExpressionType.COLUMN,
+            VectorExpressionDescriptor.InputExpressionType.COLUMN).build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/130293e5/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnCompareLongDoubleScalar.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnCompareLongDoubleScalar.txt b/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnCompareLongDoubleScalar.txt
new file mode 100644
index 0000000..e0ae206
--- /dev/null
+++ b/ql/src/gen/vectorization/ExpressionTemplates/TimestampColumnCompareLongDoubleScalar.txt
@@ -0,0 +1,144 @@
+/**
+ * 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.LongColumnVector;
+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 TimestampColumnCompareScalar.txt, which covers comparison
+ * expressions between a Timestamp column and a long/double scalar. The boolean output is stored
+ * in a separate boolean column.
+ */
+public class <ClassName> extends VectorExpression {
+
+  private static final long serialVersionUID = 1L;
+
+  private int colNum;
+  private <OperandType> value;
+  private int outputColumn;
+
+  public <ClassName>(int colNum, <OperandType> value, int outputColumn) {
+    this.colNum = colNum;
+    this.value = value;
+    this.outputColumn = outputColumn;
+  }
+
+  public <ClassName>() {
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    TimestampColumnVector inputColVector = (TimestampColumnVector) batch.cols[colNum];
+    LongColumnVector outputColVector = (LongColumnVector) batch.cols[outputColumn];
+    int[] sel = batch.selected;
+    boolean[] nullPos = inputColVector.isNull;
+    boolean[] outNulls = outputColVector.isNull;
+    int n = batch.size;
+    long[] outputVector = outputColVector.vector;
+
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+
+    outputColVector.isRepeating = false;
+    outputColVector.noNulls = inputColVector.noNulls;
+    if (inputColVector.noNulls) {
+      if (inputColVector.isRepeating) {
+        //All must be selected otherwise size would be zero
+        //Repeating property will not change.
+        outputVector[0] = inputColVector.<GetTimestampLongDoubleMethod>(0) <OperatorSymbol> value ? 1 : 0;
+        outputColVector.isRepeating = true;
+      } else if (batch.selectedInUse) {
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = inputColVector.<GetTimestampLongDoubleMethod>(i) <OperatorSymbol> value ? 1 : 0;
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = inputColVector.<GetTimestampLongDoubleMethod>(i) <OperatorSymbol> value ? 1 : 0;
+        }
+      }
+    } else {
+      if (inputColVector.isRepeating) {
+        //All must be selected otherwise size would be zero
+        //Repeating property will not change.
+        if (!nullPos[0]) {
+          outputVector[0] = inputColVector.<GetTimestampLongDoubleMethod>(0) <OperatorSymbol> value ? 1 : 0;
+          outNulls[0] = false;
+        } else {
+          outNulls[0] = true;
+        }
+        outputColVector.isRepeating = true;
+      } else if (batch.selectedInUse) {
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          if (!nullPos[i]) {
+            outputVector[i] = inputColVector.<GetTimestampLongDoubleMethod>(i) <OperatorSymbol> value ? 1 : 0;
+            outNulls[i] = false;
+          } else {
+            //comparison with null is null
+            outNulls[i] = true;
+          }
+        }
+      } else {
+        System.arraycopy(nullPos, 0, outNulls, 0, n);
+        for(int i = 0; i != n; i++) {
+          if (!nullPos[i]) {
+            outputVector[i] = inputColVector.<GetTimestampLongDoubleMethod>(i) <OperatorSymbol> value ? 1 : 0;
+          }
+        }
+      }
+    }
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return outputColumn;
+  }
+
+  @Override
+  public String getOutputType() {
+    return "long";
+  }
+
+  @Override
+  public VectorExpressionDescriptor.Descriptor getDescriptor() {
+    return (new VectorExpressionDescriptor.Builder())
+        .setMode(
+            VectorExpressionDescriptor.Mode.PROJECTION)
+        .setNumArguments(2)
+        .setArgumentTypes(
+            VectorExpressionDescriptor.ArgumentType.getType("timestamp"),
+            VectorExpressionDescriptor.ArgumentType.getType("<OperandType>"))
+        .setInputExpressionTypes(
+            VectorExpressionDescriptor.InputExpressionType.COLUMN,
+            VectorExpressionDescriptor.InputExpressionType.SCALAR).build();
+  }
+}