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:12 UTC

[18/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/common/src/java/org/apache/hadoop/hive/common/type/HiveIntervalDayTime.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/common/type/HiveIntervalDayTime.java b/common/src/java/org/apache/hadoop/hive/common/type/HiveIntervalDayTime.java
index e8dc21b..b891e27 100644
--- a/common/src/java/org/apache/hadoop/hive/common/type/HiveIntervalDayTime.java
+++ b/common/src/java/org/apache/hadoop/hive/common/type/HiveIntervalDayTime.java
@@ -18,12 +18,16 @@
 package org.apache.hadoop.hive.common.type;
 
 import java.math.BigDecimal;
+import java.sql.Timestamp;
+import java.util.Date;
 import java.util.concurrent.TimeUnit;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
 import org.apache.commons.lang.builder.HashCodeBuilder;
-import org.apache.hive.common.util.DateUtils;
+import org.apache.hive.common.util.IntervalDayTimeUtils;
+
+import sun.util.calendar.BaseCalendar;
 
 /**
  * Day-time interval type representing an offset in days/hours/minutes/seconds,
@@ -85,15 +89,23 @@ public class HiveIntervalDayTime implements Comparable<HiveIntervalDayTime> {
   }
 
   /**
+   *
+   * @return double representation of the interval day time, accurate to nanoseconds
+   */
+  public double getDouble() {
+    return totalSeconds + nanos / 1000000000;
+  }
+
+  /**
    * Ensures that the seconds and nanoseconds fields have consistent sign
    */
   protected void normalizeSecondsAndNanos() {
     if (totalSeconds > 0 && nanos < 0) {
       --totalSeconds;
-      nanos += DateUtils.NANOS_PER_SEC;
+      nanos += IntervalDayTimeUtils.NANOS_PER_SEC;
     } else if (totalSeconds < 0 && nanos > 0) {
       ++totalSeconds;
-      nanos -= DateUtils.NANOS_PER_SEC;
+      nanos -= IntervalDayTimeUtils.NANOS_PER_SEC;
     }
   }
 
@@ -103,7 +115,7 @@ public class HiveIntervalDayTime implements Comparable<HiveIntervalDayTime> {
     totalSeconds += TimeUnit.HOURS.toSeconds(hours);
     totalSeconds += TimeUnit.MINUTES.toSeconds(minutes);
     totalSeconds += TimeUnit.NANOSECONDS.toSeconds(nanos);
-    nanos = nanos % DateUtils.NANOS_PER_SEC;
+    nanos = nanos % IntervalDayTimeUtils.NANOS_PER_SEC;
 
     this.totalSeconds = totalSeconds;
     this.nanos = nanos;
@@ -120,7 +132,7 @@ public class HiveIntervalDayTime implements Comparable<HiveIntervalDayTime> {
   public void set(BigDecimal totalSecondsBd) {
     long totalSeconds = totalSecondsBd.longValue();
     BigDecimal fractionalSecs = totalSecondsBd.remainder(BigDecimal.ONE);
-    int nanos = fractionalSecs.multiply(DateUtils.NANOS_PER_SEC_BD).intValue();
+    int nanos = fractionalSecs.multiply(IntervalDayTimeUtils.NANOS_PER_SEC_BD).intValue();
     set(totalSeconds, nanos);
   }
 
@@ -155,6 +167,13 @@ public class HiveIntervalDayTime implements Comparable<HiveIntervalDayTime> {
     return 0 == compareTo((HiveIntervalDayTime) obj);
   }
 
+  /**
+   * Return a copy of this object.
+   */
+  public Object clone() {
+      return new HiveIntervalDayTime(totalSeconds, nanos);
+  }
+
   @Override
   public int hashCode() {
     return new HashCodeBuilder().append(totalSeconds).append(nanos).toHashCode();
@@ -190,23 +209,23 @@ public class HiveIntervalDayTime implements Comparable<HiveIntervalDayTime> {
           sign = -1;
         }
         int days = sign *
-            DateUtils.parseNumericValueWithRange("day", patternMatcher.group(2),
+            IntervalDayTimeUtils.parseNumericValueWithRange("day", patternMatcher.group(2),
                 0, Integer.MAX_VALUE);
         byte hours = (byte) (sign *
-            DateUtils.parseNumericValueWithRange("hour", patternMatcher.group(3), 0, 23));
+            IntervalDayTimeUtils.parseNumericValueWithRange("hour", patternMatcher.group(3), 0, 23));
         byte minutes = (byte) (sign *
-            DateUtils.parseNumericValueWithRange("minute", patternMatcher.group(4), 0, 59));
+            IntervalDayTimeUtils.parseNumericValueWithRange("minute", patternMatcher.group(4), 0, 59));
         int seconds = 0;
         int nanos = 0;
         field = patternMatcher.group(5);
         if (field != null) {
           BigDecimal bdSeconds = new BigDecimal(field);
-          if (bdSeconds.compareTo(DateUtils.MAX_INT_BD) > 0) {
+          if (bdSeconds.compareTo(IntervalDayTimeUtils.MAX_INT_BD) > 0) {
             throw new IllegalArgumentException("seconds value of " + bdSeconds + " too large");
           }
           seconds = sign * bdSeconds.intValue();
           nanos = sign * bdSeconds.subtract(new BigDecimal(bdSeconds.toBigInteger()))
-              .multiply(DateUtils.NANOS_PER_SEC_BD).intValue();
+              .multiply(IntervalDayTimeUtils.NANOS_PER_SEC_BD).intValue();
         }
 
         result = new HiveIntervalDayTime(days, hours, minutes, seconds, nanos);

http://git-wip-us.apache.org/repos/asf/hive/blob/130293e5/common/src/java/org/apache/hadoop/hive/common/type/RandomTypeUtil.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/common/type/RandomTypeUtil.java b/common/src/java/org/apache/hadoop/hive/common/type/RandomTypeUtil.java
new file mode 100644
index 0000000..3fb0cfd
--- /dev/null
+++ b/common/src/java/org/apache/hadoop/hive/common/type/RandomTypeUtil.java
@@ -0,0 +1,115 @@
+/**
+ * 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.common.type;
+
+import java.sql.Timestamp;
+import java.text.DateFormat;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.util.Random;
+import java.util.concurrent.TimeUnit;
+
+public class RandomTypeUtil {
+
+  public static final long NANOSECONDS_PER_SECOND = TimeUnit.SECONDS.toNanos(1);
+  public static final long MILLISECONDS_PER_SECOND = TimeUnit.SECONDS.toMillis(1);
+  public static final long NANOSECONDS_PER_MILLISSECOND = TimeUnit.MILLISECONDS.toNanos(1);
+
+  private static ThreadLocal<DateFormat> DATE_FORMAT =
+      new ThreadLocal<DateFormat>() {
+        @Override
+        protected DateFormat initialValue() {
+          return new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
+        }
+      };
+
+  // We've switched to Joda/Java Calendar which has a more limited time range....
+  public static int MIN_YEAR = 1900;
+  public static int MAX_YEAR = 3000;
+  private static long MIN_FOUR_DIGIT_YEAR_MILLIS = parseToMillis("1900-01-01 00:00:00");
+  private static long MAX_FOUR_DIGIT_YEAR_MILLIS = parseToMillis("3000-01-01 00:00:00");
+
+  private static long parseToMillis(String s) {
+    try {
+      return DATE_FORMAT.get().parse(s).getTime();
+    } catch (ParseException ex) {
+      throw new RuntimeException(ex);
+    }
+  }
+
+  public static Timestamp getRandTimestamp(Random r) {
+    return getRandTimestamp(r, MIN_YEAR, MAX_YEAR);
+  }
+
+  public static Timestamp getRandTimestamp(Random r, int minYear, int maxYear) {
+    String optionalNanos = "";
+    switch (r.nextInt(4)) {
+    case 0:
+      // No nanos.
+      break;
+    case 1:
+      optionalNanos = String.format(".%09d",
+          Integer.valueOf(r.nextInt((int) NANOSECONDS_PER_SECOND)));
+      break;
+    case 2:
+      // Limit to milliseconds only...
+      optionalNanos = String.format(".%09d",
+          Integer.valueOf(r.nextInt((int) MILLISECONDS_PER_SECOND)) * NANOSECONDS_PER_MILLISSECOND);
+      break;
+    case 3:
+      // Limit to below milliseconds only...
+      optionalNanos = String.format(".%09d",
+          Integer.valueOf(r.nextInt((int) NANOSECONDS_PER_MILLISSECOND)));
+      break;
+    }
+    String timestampStr = String.format("%04d-%02d-%02d %02d:%02d:%02d%s",
+        Integer.valueOf(minYear + r.nextInt(maxYear - minYear + 1)),  // year
+        Integer.valueOf(1 + r.nextInt(12)),      // month
+        Integer.valueOf(1 + r.nextInt(28)),      // day
+        Integer.valueOf(0 + r.nextInt(24)),      // hour
+        Integer.valueOf(0 + r.nextInt(60)),      // minute
+        Integer.valueOf(0 + r.nextInt(60)),      // second
+        optionalNanos);
+    Timestamp timestampVal;
+    try {
+      timestampVal = Timestamp.valueOf(timestampStr);
+    } catch (Exception e) {
+      System.err.println("Timestamp string " + timestampStr + " did not parse");
+      throw e;
+    }
+    return timestampVal;
+  }
+
+  public static long randomMillis(long minMillis, long maxMillis, Random rand) {
+    return minMillis + (long) ((maxMillis - minMillis) * rand.nextDouble());
+  }
+
+  public static long randomMillis(Random rand) {
+    return randomMillis(MIN_FOUR_DIGIT_YEAR_MILLIS, MAX_FOUR_DIGIT_YEAR_MILLIS, rand);
+  }
+
+  public static int randomNanos(Random rand, int decimalDigits) {
+    // Only keep the most significant decimalDigits digits.
+    int nanos = rand.nextInt((int) NANOSECONDS_PER_SECOND);
+    return nanos - nanos % (int) Math.pow(10, 9 - decimalDigits);
+  }
+
+  public static int randomNanos(Random rand) {
+    return randomNanos(rand, 9);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/130293e5/common/src/java/org/apache/hive/common/util/DateUtils.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hive/common/util/DateUtils.java b/common/src/java/org/apache/hive/common/util/DateUtils.java
index c749bcb..c5a1c50 100644
--- a/common/src/java/org/apache/hive/common/util/DateUtils.java
+++ b/common/src/java/org/apache/hive/common/util/DateUtils.java
@@ -56,21 +56,4 @@ public class DateUtils {
     }
     return result;
   }
-
-  public static long getIntervalDayTimeTotalNanos(HiveIntervalDayTime intervalDayTime) {
-    return intervalDayTime.getTotalSeconds() * NANOS_PER_SEC + intervalDayTime.getNanos();
-  }
-
-  public static void setIntervalDayTimeTotalNanos(HiveIntervalDayTime intervalDayTime,
-      long totalNanos) {
-    intervalDayTime.set(totalNanos / NANOS_PER_SEC, (int) (totalNanos % NANOS_PER_SEC));
-  }
-
-  public static long getIntervalDayTimeTotalSecondsFromTotalNanos(long totalNanos) {
-    return totalNanos / NANOS_PER_SEC;
-  }
-
-  public static int getIntervalDayTimeNanosFromTotalNanos(long totalNanos) {
-    return (int) (totalNanos % NANOS_PER_SEC);
-  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/130293e5/common/src/java/org/apache/hive/common/util/IntervalDayTimeUtils.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hive/common/util/IntervalDayTimeUtils.java b/common/src/java/org/apache/hive/common/util/IntervalDayTimeUtils.java
new file mode 100644
index 0000000..727c1e6
--- /dev/null
+++ b/common/src/java/org/apache/hive/common/util/IntervalDayTimeUtils.java
@@ -0,0 +1,77 @@
+/**
+ * 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.hive.common.util;
+
+import java.math.BigDecimal;
+import java.text.SimpleDateFormat;
+
+import org.apache.hadoop.hive.common.type.HiveIntervalDayTime;
+
+
+/**
+ * DateUtils. Thread-safe class
+ *
+ */
+public class IntervalDayTimeUtils {
+
+  private static final ThreadLocal<SimpleDateFormat> dateFormatLocal = new ThreadLocal<SimpleDateFormat>() {
+    @Override
+    protected SimpleDateFormat initialValue() {
+      return new SimpleDateFormat("yyyy-MM-dd");
+    }
+  };
+
+  public static SimpleDateFormat getDateFormat() {
+    return dateFormatLocal.get();
+  }
+
+  public static final int NANOS_PER_SEC = 1000000000;
+  public static final BigDecimal MAX_INT_BD = new BigDecimal(Integer.MAX_VALUE);
+  public static final BigDecimal NANOS_PER_SEC_BD = new BigDecimal(NANOS_PER_SEC);
+
+  public static int parseNumericValueWithRange(String fieldName,
+      String strVal, int minValue, int maxValue) throws IllegalArgumentException {
+    int result = 0;
+    if (strVal != null) {
+      result = Integer.parseInt(strVal);
+      if (result < minValue || result > maxValue) {
+        throw new IllegalArgumentException(String.format("%s value %d outside range [%d, %d]",
+            fieldName, result, minValue, maxValue));
+      }
+    }
+    return result;
+  }
+
+  public static long getIntervalDayTimeTotalNanos(HiveIntervalDayTime intervalDayTime) {
+    return intervalDayTime.getTotalSeconds() * NANOS_PER_SEC + intervalDayTime.getNanos();
+  }
+
+  public static void setIntervalDayTimeTotalNanos(HiveIntervalDayTime intervalDayTime,
+      long totalNanos) {
+    intervalDayTime.set(totalNanos / NANOS_PER_SEC, (int) (totalNanos % NANOS_PER_SEC));
+  }
+
+  public static long getIntervalDayTimeTotalSecondsFromTotalNanos(long totalNanos) {
+    return totalNanos / NANOS_PER_SEC;
+  }
+
+  public static int getIntervalDayTimeNanosFromTotalNanos(long totalNanos) {
+    return (int) (totalNanos % NANOS_PER_SEC);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/130293e5/data/files/timestamps.txt
----------------------------------------------------------------------
diff --git a/data/files/timestamps.txt b/data/files/timestamps.txt
new file mode 100644
index 0000000..36ffd23
--- /dev/null
+++ b/data/files/timestamps.txt
@@ -0,0 +1,50 @@
+6631-11-13 16:31:29.702202248
+6731-02-12 08:12:48.287783702
+6705-09-28 18:27:28.000845672
+5397-07-13 07:12:32.000896438
+9209-11-11 04:08:58.223768453
+9403-01-09 18:12:33.547
+6482-04-27 12:07:38.073915413
+7503-06-23 23:14:17.486
+1883-04-17 04:14:34.647766229
+0004-09-22 18:26:29.519542222
+7160-12-02 06:00:24.81200852
+8422-07-22 03:21:45.745036084
+4143-07-08 10:53:27.252802259
+5344-10-04 18:40:08.165
+5966-07-09 03:30:50.597
+9075-06-13 16:20:09.218517797
+1815-05-06 00:12:37.543584705
+7409-09-07 23:33:32.459349602
+5339-02-01 14:10:01.085678691
+4966-12-04 09:30:55.202
+1319-02-02 16:31:57.778
+1404-07-23 15:32:16.059185026
+6229-06-28 02:54:28.970117179
+0528-10-27 08:15:18.941718273
+8521-01-16 20:42:05.668832388
+1976-05-06 00:42:30.910786948
+2003-09-23 22:33:17.00003252
+2007-02-09 05:17:29.368756876
+1998-10-16 20:05:29.397591987
+1976-03-03 04:54:33.000895162
+1985-07-20 09:30:11.0
+2021-09-24 03:18:32.413655165
+2013-04-07 02:44:43.00086821
+2002-05-10 05:29:48.990818073
+1973-04-17 06:30:38.596784156
+1987-02-21 19:48:29.0
+1981-11-15 23:03:10.999338387
+2000-12-18 08:42:30.000595596
+1999-10-03 16:59:10.396903939
+2024-11-11 16:42:41.101
+2013-04-10 00:43:46.854731546
+2010-04-08 02:43:35.861742727
+2004-03-07 20:14:13.0
+1987-05-28 13:52:07.900916635
+1978-08-05 14:41:05.501
+1966-08-16 13:36:50.183618031
+2009-01-21 10:49:07.108
+1981-04-25 09:01:12.077192689
+1985-11-18 16:37:54.0
+1974-10-04 17:21:03.989

http://git-wip-us.apache.org/repos/asf/hive/blob/130293e5/itests/src/test/resources/testconfiguration.properties
----------------------------------------------------------------------
diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties
index 3229c44..2d9cab8 100644
--- a/itests/src/test/resources/testconfiguration.properties
+++ b/itests/src/test/resources/testconfiguration.properties
@@ -263,6 +263,8 @@ minitez.query.files.shared=acid_globallimit.q,\
   vector_inner_join.q,\
   vector_interval_1.q,\
   vector_interval_2.q,\
+  vector_interval_arithmetic.q,\
+  vector_interval_mapjoin.q,\
   vector_join30.q,\
   vector_join_filters.q,\
   vector_join_nulls.q,\
@@ -287,6 +289,7 @@ minitez.query.files.shared=acid_globallimit.q,\
   vector_partitioned_date_time.q,\
   vector_reduce_groupby_decimal.q,\
   vector_string_concat.q,\
+  vectorized_timestamp.q,\
   vector_varchar_4.q,\
   vector_varchar_mapjoin1.q,\
   vector_varchar_simple.q,\
@@ -333,6 +336,7 @@ minitez.query.files.shared=acid_globallimit.q,\
   vectorized_shufflejoin.q,\
   vectorized_string_funcs.q,\
   vectorized_timestamp_funcs.q,\
+  vectorized_timestamp_ints_casts.q,\
   auto_sortmerge_join_1.q,\
   auto_sortmerge_join_10.q,\
   auto_sortmerge_join_11.q,\

http://git-wip-us.apache.org/repos/asf/hive/blob/130293e5/ql/src/gen/vectorization/ExpressionTemplates/DTIColumnArithmeticDTIColumnNoConvert.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/DTIColumnArithmeticDTIColumnNoConvert.txt b/ql/src/gen/vectorization/ExpressionTemplates/DTIColumnArithmeticDTIColumnNoConvert.txt
index f2ec645..fe8f535 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/DTIColumnArithmeticDTIColumnNoConvert.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/DTIColumnArithmeticDTIColumnNoConvert.txt
@@ -34,6 +34,8 @@ import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
  */
 public class <ClassName> extends LongCol<OperatorName>LongColumn {
 
+  private static final long serialVersionUID = 1L;
+
   public <ClassName>(int colNum1, int colNum2, int outputColumn) {
     super(colNum1, colNum2, outputColumn);
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/130293e5/ql/src/gen/vectorization/ExpressionTemplates/DTIColumnArithmeticDTIScalarNoConvert.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/DTIColumnArithmeticDTIScalarNoConvert.txt b/ql/src/gen/vectorization/ExpressionTemplates/DTIColumnArithmeticDTIScalarNoConvert.txt
index 1a360b8..293369f 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/DTIColumnArithmeticDTIScalarNoConvert.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/DTIColumnArithmeticDTIScalarNoConvert.txt
@@ -29,6 +29,8 @@ import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
  */
 public class <ClassName> extends LongCol<OperatorName>LongScalar {
 
+  private static final long serialVersionUID = 1L;
+
   public <ClassName>(int colNum, long value, int outputColumn) {
     super(colNum, value, outputColumn);
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/130293e5/ql/src/gen/vectorization/ExpressionTemplates/DTIColumnCompareScalar.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/DTIColumnCompareScalar.txt b/ql/src/gen/vectorization/ExpressionTemplates/DTIColumnCompareScalar.txt
index 9d692cb..60884cd 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/DTIColumnCompareScalar.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/DTIColumnCompareScalar.txt
@@ -29,6 +29,8 @@ import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
  */
 public class <ClassName> extends <BaseClassName> {
 
+  private static final long serialVersionUID = 1L;
+
   public <ClassName>(int colNum, long value, int outputColumn) {
     super(colNum, value, outputColumn);
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/130293e5/ql/src/gen/vectorization/ExpressionTemplates/DTIScalarArithmeticDTIColumnNoConvert.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/DTIScalarArithmeticDTIColumnNoConvert.txt b/ql/src/gen/vectorization/ExpressionTemplates/DTIScalarArithmeticDTIColumnNoConvert.txt
index 753ea71..04607f6 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/DTIScalarArithmeticDTIColumnNoConvert.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/DTIScalarArithmeticDTIColumnNoConvert.txt
@@ -34,6 +34,8 @@ import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
  */
 public class <ClassName> extends LongScalar<OperatorName>LongColumn {
 
+  private static final long serialVersionUID = 1L;
+
   public <ClassName>(long value, int colNum, int outputColumn) {
     super(value, colNum, outputColumn);
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/130293e5/ql/src/gen/vectorization/ExpressionTemplates/DTIScalarCompareColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/DTIScalarCompareColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/DTIScalarCompareColumn.txt
index fdd453a..d518c44 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/DTIScalarCompareColumn.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/DTIScalarCompareColumn.txt
@@ -34,6 +34,8 @@ import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
  */
 public class <ClassName> extends <BaseClassName> {
 
+  private static final long serialVersionUID = 1L;
+
   public <ClassName>(long value, int colNum, int outputColumn) {
     super(value, colNum, outputColumn);
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/130293e5/ql/src/gen/vectorization/ExpressionTemplates/DateColumnArithmeticIntervalYearMonthColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/DateColumnArithmeticIntervalYearMonthColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/DateColumnArithmeticIntervalYearMonthColumn.txt
new file mode 100644
index 0000000..c3d8d7e
--- /dev/null
+++ b/ql/src/gen/vectorization/ExpressionTemplates/DateColumnArithmeticIntervalYearMonthColumn.txt
@@ -0,0 +1,197 @@
+/**
+ * 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.expressions.NullUtil;
+import org.apache.hadoop.hive.ql.exec.vector.*;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.ql.util.DateTimeMath;
+import org.apache.hadoop.hive.serde2.io.DateWritable;
+
+/**
+ * Generated from template DateColumnArithmeticIntervalYearMonthColumn.txt, which covers binary arithmetic
+ * expressions between date and interval year month columns.
+ */
+public class <ClassName> extends VectorExpression {
+
+  private static final long serialVersionUID = 1L;
+
+  private int colNum1;
+  private int colNum2;
+  private int outputColumn;
+  private Date scratchDate1;
+  private HiveIntervalYearMonth scratchIntervalYearMonth2;
+  private Date outputDate;
+  private DateTimeMath dtm = new DateTimeMath();
+
+  public <ClassName>(int colNum1, int colNum2, int outputColumn) {
+    this.colNum1 = colNum1;
+    this.colNum2 = colNum2;
+    this.outputColumn = outputColumn;
+    scratchDate1 = new Date(0);
+    scratchIntervalYearMonth2 = new HiveIntervalYearMonth();
+    outputDate = new Date(0);
+  }
+
+  public <ClassName>() {
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    // Input #1 is type date.
+    LongColumnVector inputColVector1 = (LongColumnVector) batch.cols[colNum1];
+
+    // Input #2 is type interval_year_month.
+    LongColumnVector inputColVector2 = (LongColumnVector) batch.cols[colNum2];
+
+    // Output is type date.
+    LongColumnVector outputColVector = (LongColumnVector) batch.cols[outputColumn];
+
+    int[] sel = batch.selected;
+    int n = batch.size;
+    long[] vector1 = inputColVector1.vector;
+    long[] 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) {
+      scratchDate1.setTime(DateWritable.daysToMillis((int) vector1[0]));
+      scratchIntervalYearMonth2.set((int) vector2[0]);
+      dtm.<OperatorMethod>(
+          scratchDate1, scratchIntervalYearMonth2,  outputDate);
+      outputVector[0] = DateWritable.dateToDays(outputDate);
+    } else if (inputColVector1.isRepeating) {
+      scratchDate1.setTime(DateWritable.daysToMillis((int) vector1[0]));
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          scratchIntervalYearMonth2.set((int) vector2[i]);
+          dtm.<OperatorMethod>(
+              scratchDate1, scratchIntervalYearMonth2,  outputDate);
+          outputVector[i] = DateWritable.dateToDays(outputDate);
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          scratchIntervalYearMonth2.set((int) vector2[i]);
+          dtm.<OperatorMethod>(
+              scratchDate1, scratchIntervalYearMonth2,  outputDate);
+          outputVector[i] = DateWritable.dateToDays(outputDate);
+        }
+      }
+    } else if (inputColVector2.isRepeating) {
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          scratchDate1.setTime(DateWritable.daysToMillis((int) vector1[i]));
+          scratchIntervalYearMonth2.set((int) vector2[i]);
+          dtm.<OperatorMethod>(
+              scratchDate1, scratchIntervalYearMonth2,  outputDate);
+          outputVector[i] = DateWritable.dateToDays(outputDate);
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          scratchDate1.setTime(DateWritable.daysToMillis((int) vector1[i]));
+          scratchIntervalYearMonth2.set((int) vector2[i]);
+          dtm.<OperatorMethod>(
+              scratchDate1, scratchIntervalYearMonth2,  outputDate);
+          outputVector[i] = DateWritable.dateToDays(outputDate);
+        }
+      }
+    } else {
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          scratchDate1.setTime(DateWritable.daysToMillis((int) vector1[i]));
+          scratchIntervalYearMonth2.set((int) vector2[i]);
+          dtm.<OperatorMethod>(
+              scratchDate1, scratchIntervalYearMonth2,  outputDate);
+          outputVector[i] = DateWritable.dateToDays(outputDate);
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          scratchDate1.setTime(DateWritable.daysToMillis((int) vector1[i]));
+          scratchIntervalYearMonth2.set((int) vector2[i]);
+          dtm.<OperatorMethod>(
+              scratchDate1, scratchIntervalYearMonth2,  outputDate);
+          outputVector[i] = DateWritable.dateToDays(outputDate);
+        }
+      }
+    }
+
+    /* 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("date"),
+            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/DateColumnArithmeticIntervalYearMonthScalar.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/DateColumnArithmeticIntervalYearMonthScalar.txt b/ql/src/gen/vectorization/ExpressionTemplates/DateColumnArithmeticIntervalYearMonthScalar.txt
new file mode 100644
index 0000000..d1474fb
--- /dev/null
+++ b/ql/src/gen/vectorization/ExpressionTemplates/DateColumnArithmeticIntervalYearMonthScalar.txt
@@ -0,0 +1,156 @@
+/**
+ * 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.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 DateColumnArithmeticIntervalYearMonthScalar.txt, which covers binary arithmetic
+ * expressions between a date column and a interval year month scalar.
+ */
+public class <ClassName> extends VectorExpression {
+
+  private static final long serialVersionUID = 1L;
+
+  private int colNum;
+  private HiveIntervalYearMonth value;
+  private int outputColumn;
+  private Date scratchDate1;
+  private Date outputDate;
+  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;
+    scratchDate1 = new Date(0);
+    outputDate = new Date(0);
+  }
+
+  public <ClassName>() {
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    // Input #1 is type date.
+    LongColumnVector inputColVector1 = (LongColumnVector) batch.cols[colNum];
+
+    // Output is type date.
+    LongColumnVector outputColVector = (LongColumnVector) 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;
+    long[] outputVector = outputColVector.vector;
+
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+
+    if (inputColVector1.isRepeating) {
+      scratchDate1.setTime(DateWritable.daysToMillis((int) vector1[0]));
+      dtm.<OperatorMethod>(
+          scratchDate1, value, 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 (inputColVector1.noNulls) {
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          scratchDate1.setTime(DateWritable.daysToMillis((int) vector1[i]));
+          dtm.<OperatorMethod>(
+              scratchDate1, value, outputDate);
+          outputVector[i] = DateWritable.dateToDays(outputDate);
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          scratchDate1.setTime(DateWritable.daysToMillis((int) vector1[i]));
+          dtm.<OperatorMethod>(
+              scratchDate1, value, outputDate);
+          outputVector[i] = DateWritable.dateToDays(outputDate);
+        }
+      }
+    } else /* there are nulls */ {
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          scratchDate1.setTime(DateWritable.daysToMillis((int) vector1[i]));
+          dtm.<OperatorMethod>(
+              scratchDate1, value, outputDate);
+          outputVector[i] = DateWritable.dateToDays(outputDate);
+          outputIsNull[i] = inputIsNull[i];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          scratchDate1.setTime(DateWritable.daysToMillis((int) vector1[i]));
+          dtm.<OperatorMethod>(
+              scratchDate1, value, 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("date"),
+            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/DateColumnArithmeticTimestampColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/DateColumnArithmeticTimestampColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/DateColumnArithmeticTimestampColumn.txt
new file mode 100644
index 0000000..63cebaf
--- /dev/null
+++ b/ql/src/gen/vectorization/ExpressionTemplates/DateColumnArithmeticTimestampColumn.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.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, a class
+ * which covers binary arithmetic expressions between a date column and timestamp column.
+ */
+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) {
+    this.colNum1 = colNum1;
+    this.colNum2 = colNum2;
+    this.outputColumn = outputColumn;
+    scratchTimestamp1 = new Timestamp(0);
+  }
+
+  public <ClassName>() {
+  }
+
+  @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
+  public VectorExpressionDescriptor.Descriptor getDescriptor() {
+    return (new VectorExpressionDescriptor.Builder())
+        .setMode(
+            VectorExpressionDescriptor.Mode.PROJECTION)
+        .setNumArguments(2)
+        .setArgumentTypes(
+            VectorExpressionDescriptor.ArgumentType.getType("date"),
+            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/DateColumnArithmeticTimestampScalar.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/DateColumnArithmeticTimestampScalar.txt b/ql/src/gen/vectorization/ExpressionTemplates/DateColumnArithmeticTimestampScalar.txt
new file mode 100644
index 0000000..7aee529
--- /dev/null
+++ b/ql/src/gen/vectorization/ExpressionTemplates/DateColumnArithmeticTimestampScalar.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.common.type.HiveIntervalDayTime;
+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.ql.util.DateTimeMath;
+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 class <ClassName> extends VectorExpression {
+
+  private static final long serialVersionUID = 1L;
+
+  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>() {
+  }
+
+  @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
+  public VectorExpressionDescriptor.Descriptor getDescriptor() {
+    return (new VectorExpressionDescriptor.Builder())
+        .setMode(
+            VectorExpressionDescriptor.Mode.PROJECTION)
+        .setNumArguments(2)
+        .setArgumentTypes(
+            VectorExpressionDescriptor.ArgumentType.getType("date"),
+            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/DateScalarArithmeticIntervalYearMonthColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/DateScalarArithmeticIntervalYearMonthColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/DateScalarArithmeticIntervalYearMonthColumn.txt
new file mode 100644
index 0000000..c68ac34
--- /dev/null
+++ b/ql/src/gen/vectorization/ExpressionTemplates/DateScalarArithmeticIntervalYearMonthColumn.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 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 = new Date(DateWritable.daysToMillis((int) value));
+    this.outputColumn = outputColumn;
+    scratchIntervalYearMonth2 = new HiveIntervalYearMonth();
+    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 Interval_Year_Month (months).
+    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) {
+      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 (inputColVector2.noNulls) {
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          scratchIntervalYearMonth2.set((int) vector2[i]);
+          dtm.<OperatorMethod>(
+              value, scratchIntervalYearMonth2, outputDate);
+          outputVector[i] = DateWritable.dateToDays(outputDate);
+        }
+      } else {
+        for(int i = 0; i != n; 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];
+          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++) {
+          scratchIntervalYearMonth2.set((int) vector2[i]);
+          dtm.<OperatorMethod>(
+              value, scratchIntervalYearMonth2, 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("date"),
+            VectorExpressionDescriptor.ArgumentType.getType("interval_year_month"))
+        .setInputExpressionTypes(
+            VectorExpressionDescriptor.InputExpressionType.SCALAR,
+            VectorExpressionDescriptor.InputExpressionType.COLUMN).build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/130293e5/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
new file mode 100644
index 0000000..cb6b750
--- /dev/null
+++ b/ql/src/gen/vectorization/ExpressionTemplates/DateScalarArithmeticTimestampColumn.txt
@@ -0,0 +1,161 @@
+/**
+ * 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.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 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 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) {
+    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("date"),
+            VectorExpressionDescriptor.ArgumentType.getType("<OperandType2>"))
+        .setInputExpressionTypes(
+            VectorExpressionDescriptor.InputExpressionType.SCALAR,
+            VectorExpressionDescriptor.InputExpressionType.COLUMN).build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/130293e5/ql/src/gen/vectorization/ExpressionTemplates/DateTimeColumnArithmeticIntervalColumnWithConvert.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/DateTimeColumnArithmeticIntervalColumnWithConvert.txt b/ql/src/gen/vectorization/ExpressionTemplates/DateTimeColumnArithmeticIntervalColumnWithConvert.txt
deleted file mode 100644
index cd7a1e7..0000000
--- a/ql/src/gen/vectorization/ExpressionTemplates/DateTimeColumnArithmeticIntervalColumnWithConvert.txt
+++ /dev/null
@@ -1,175 +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.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 DateTimeColumnArithmeticIntervalColumnWithConvert.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);
-    }
-
-    <InputColumnVectorType1> inputColVector1 = (<InputColumnVectorType1>) batch.cols[colNum1];
-    <InputColumnVectorType2> inputColVector2 = (<InputColumnVectorType2>) batch.cols[colNum2];
-    <OutputColumnVectorType> outputColVector = (<OutputColumnVectorType>) batch.cols[outputColumn];
-    int[] sel = batch.selected;
-    int n = batch.size;
-    <VectorOperandType1>[] vector1 = inputColVector1.vector;
-    <VectorOperandType2>[] vector2 = inputColVector2.vector;
-    <VectorReturnType>[] 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] = <OperatorFunction>(<TypeConversionToMillis>(vector1[0]), <OperatorSymbol> (int) vector2[0]);
-    } else if (inputColVector1.isRepeating) {
-      if (batch.selectedInUse) {
-        for(int j = 0; j != n; j++) {
-          int i = sel[j];
-          outputVector[i] = <OperatorFunction>(<TypeConversionToMillis>(vector1[0]), <OperatorSymbol> (int) vector2[i]);
-        }
-      } else {
-        for(int i = 0; i != n; i++) {
-          outputVector[i] = <OperatorFunction>(<TypeConversionToMillis>(vector1[0]), <OperatorSymbol> (int) vector2[i]);
-        }
-      }
-    } else if (inputColVector2.isRepeating) {
-      if (batch.selectedInUse) {
-        for(int j = 0; j != n; j++) {
-          int i = sel[j];
-          outputVector[i] = <OperatorFunction>(<TypeConversionToMillis>(vector1[i]), <OperatorSymbol> (int) vector2[0]);
-        }
-      } else {
-        for(int i = 0; i != n; i++) {
-          outputVector[i] = <OperatorFunction>(<TypeConversionToMillis>(vector1[i]), <OperatorSymbol> (int) vector2[0]);
-        }
-      }
-    } else {
-      if (batch.selectedInUse) {
-        for(int j = 0; j != n; j++) {
-          int i = sel[j];
-          outputVector[i] = <OperatorFunction>(<TypeConversionToMillis>(vector1[i]), <OperatorSymbol> (int) vector2[i]);
-        }
-      } else {
-        for(int i = 0; i != n; i++) {
-          outputVector[i] = <OperatorFunction>(<TypeConversionToMillis>(vector1[i]), <OperatorSymbol> (int) vector2[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 "<VectorReturnType>";
-  }
-  
-  public int getColNum1() {
-    return colNum1;
-  }
-
-  public void setColNum1(int colNum1) {
-    this.colNum1 = colNum1;
-  }
-
-  public int getColNum2() {
-    return colNum2;
-  }
-
-  public void setColNum2(int colNum2) {
-    this.colNum2 = colNum2;
-  }
-
-  public void setOutputColumn(int outputColumn) {
-    this.outputColumn = outputColumn;
-  }
-  
-  @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/DateTimeColumnArithmeticIntervalScalarWithConvert.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/DateTimeColumnArithmeticIntervalScalarWithConvert.txt b/ql/src/gen/vectorization/ExpressionTemplates/DateTimeColumnArithmeticIntervalScalarWithConvert.txt
deleted file mode 100644
index abee249..0000000
--- a/ql/src/gen/vectorization/ExpressionTemplates/DateTimeColumnArithmeticIntervalScalarWithConvert.txt
+++ /dev/null
@@ -1,152 +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.<InputColumnVectorType>;
-import org.apache.hadoop.hive.ql.exec.vector.<OutputColumnVectorType>;
-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 ColumnArithmeticScalarWithConvert.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 <VectorOperandType2> value;
-  private int outputColumn;
-  private DateTimeMath dtm = new DateTimeMath();
-
-  public <ClassName>(int colNum, <VectorOperandType2> 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);
-    }
-
-    <InputColumnVectorType> inputColVector = (<InputColumnVectorType>) batch.cols[colNum];
-    <OutputColumnVectorType> outputColVector = (<OutputColumnVectorType>) batch.cols[outputColumn];
-    int[] sel = batch.selected;
-    boolean[] inputIsNull = inputColVector.isNull;
-    boolean[] outputIsNull = outputColVector.isNull;
-    outputColVector.noNulls = inputColVector.noNulls;
-    outputColVector.isRepeating = inputColVector.isRepeating;
-    int n = batch.size;
-    <VectorOperandType1>[] vector = inputColVector.vector;
-    <VectorReturnType>[] outputVector = outputColVector.vector;
-    
-    // return immediately if batch is empty
-    if (n == 0) {
-      return;
-    }
-
-    if (inputColVector.isRepeating) {
-      outputVector[0] = <OperatorFunction>(<TypeConversionToMillis>(vector[0]), <OperatorSymbol> (int) value);
-      
-      // Even if there are no nulls, we always copy over entry 0. Simplifies code.
-      outputIsNull[0] = inputIsNull[0]; 
-    } else if (inputColVector.noNulls) {
-      if (batch.selectedInUse) {
-        for(int j = 0; j != n; j++) {
-          int i = sel[j];
-          outputVector[i] = <OperatorFunction>(<TypeConversionToMillis>(vector[i]), <OperatorSymbol> (int) value);
-        }
-      } else {
-        for(int i = 0; i != n; i++) {
-          outputVector[i] = <OperatorFunction>(<TypeConversionToMillis>(vector[i]), <OperatorSymbol> (int) value);
-        }
-      }
-    } else /* there are nulls */ {
-      if (batch.selectedInUse) {
-        for(int j = 0; j != n; j++) {
-          int i = sel[j];
-          outputVector[i] = <OperatorFunction>(<TypeConversionToMillis>(vector[i]), <OperatorSymbol> (int) value);
-          outputIsNull[i] = inputIsNull[i];
-        }
-      } else {
-        for(int i = 0; i != n; i++) {
-          outputVector[i] = <OperatorFunction>(<TypeConversionToMillis>(vector[i]), <OperatorSymbol> (int) value);
-        }
-        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 "<VectorReturnType>";
-  }
-  
-  public int getColNum() {
-    return colNum;
-  }
-  
-  public void setColNum(int colNum) {
-    this.colNum = colNum;
-  }
-
-  public <VectorOperandType2> getValue() {
-    return value;
-  }
-
-  public void setValue(<VectorOperandType2> value) {
-    this.value = value;
-  }
-
-  public void setOutputColumn(int outputColumn) {
-    this.outputColumn = outputColumn;
-  }
-
-  @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/DateTimeScalarArithmeticIntervalColumnWithConvert.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/DateTimeScalarArithmeticIntervalColumnWithConvert.txt b/ql/src/gen/vectorization/ExpressionTemplates/DateTimeScalarArithmeticIntervalColumnWithConvert.txt
deleted file mode 100644
index 93a441a..0000000
--- a/ql/src/gen/vectorization/ExpressionTemplates/DateTimeScalarArithmeticIntervalColumnWithConvert.txt
+++ /dev/null
@@ -1,165 +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.*;
-
-
-/*
- * 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 DateTimeScalarArithmeticIntervalColumnWithConvert.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 <VectorOperandType1> value;
-  private int outputColumn;
-  private DateTimeMath dtm = new DateTimeMath();
-
-  public <ClassName>(<VectorOperandType1> value, int colNum, int outputColumn) {
-    this.colNum = colNum;
-    this.value = <TypeConversionToMillis>(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);
-    }
-
-    <InputColumnVectorType> inputColVector = (<InputColumnVectorType>) batch.cols[colNum];
-    <OutputColumnVectorType> outputColVector = (<OutputColumnVectorType>) batch.cols[outputColumn];
-    int[] sel = batch.selected;
-    boolean[] inputIsNull = inputColVector.isNull;
-    boolean[] outputIsNull = outputColVector.isNull;
-    outputColVector.noNulls = inputColVector.noNulls;
-    outputColVector.isRepeating = inputColVector.isRepeating;
-    int n = batch.size;
-    <VectorOperandType2>[] vector = inputColVector.vector;
-    <VectorReturnType>[] outputVector = outputColVector.vector;
-    
-    // return immediately if batch is empty
-    if (n == 0) {
-      return;
-    }
-
-    if (inputColVector.isRepeating) {
-      outputVector[0] = <OperatorFunction>(value, <OperatorSymbol> (int) vector[0]);
-      
-      // Even if there are no nulls, we always copy over entry 0. Simplifies code.
-      outputIsNull[0] = inputIsNull[0]; 
-    } else if (inputColVector.noNulls) {
-      if (batch.selectedInUse) {
-        for(int j = 0; j != n; j++) {
-          int i = sel[j];
-          outputVector[i] = <OperatorFunction>(value, <OperatorSymbol> (int) vector[i]);
-        }
-      } else {
-        for(int i = 0; i != n; i++) {
-          outputVector[i] = <OperatorFunction>(value, <OperatorSymbol> (int) vector[i]);
-        }
-      }
-    } else {                         /* there are nulls */ 
-      if (batch.selectedInUse) {
-        for(int j = 0; j != n; j++) {
-          int i = sel[j];
-          outputVector[i] = <OperatorFunction>(value, <OperatorSymbol> (int) vector[i]);
-          outputIsNull[i] = inputIsNull[i];
-        }
-      } else {
-        for(int i = 0; i != n; i++) {
-          outputVector[i] = <OperatorFunction>(value, <OperatorSymbol> (int) vector[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 "<VectorReturnType>";
-  }
-  
-  public int getColNum() {
-    return colNum;
-  }
-
-  public void setColNum(int colNum) {
-    this.colNum = colNum;
-  }
-
-  public <VectorOperandType1> getValue() {
-    return value;
-  }
-
-  public void setValue(<VectorOperandType1> value) {
-    this.value = value;
-  }
-
-  public void setOutputColumn(int outputColumn) {
-    this.outputColumn = outputColumn;
-  }
-
-  @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.SCALAR,
-            VectorExpressionDescriptor.InputExpressionType.COLUMN).build();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/130293e5/ql/src/gen/vectorization/ExpressionTemplates/FilterDTIColumnCompareScalar.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/FilterDTIColumnCompareScalar.txt b/ql/src/gen/vectorization/ExpressionTemplates/FilterDTIColumnCompareScalar.txt
index 55193ac..2351230 100644
--- a/ql/src/gen/vectorization/ExpressionTemplates/FilterDTIColumnCompareScalar.txt
+++ b/ql/src/gen/vectorization/ExpressionTemplates/FilterDTIColumnCompareScalar.txt
@@ -18,8 +18,6 @@
  
 package org.apache.hadoop.hive.ql.exec.vector.expressions.gen;
 
-import org.apache.hadoop.hive.ql.exec.vector.TimestampUtils;
-
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
 
 /**