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

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

http://git-wip-us.apache.org/repos/asf/hive/blob/4a479d0c/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncTimestampToLong.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncTimestampToLong.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncTimestampToLong.java
new file mode 100644
index 0000000..b84d9be
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncTimestampToLong.java
@@ -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;
+
+import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+
+/**
+ * This is a superclass for unary timestamp functions and expressions returning long that
+ * operate directly on the input and set the output.
+ */
+public abstract class FuncTimestampToLong extends VectorExpression {
+  private static final long serialVersionUID = 1L;
+  int inputColumn;
+  int outputColumn;
+
+  public FuncTimestampToLong(int inputColumn, int outputColumn) {
+    this.inputColumn = inputColumn;
+    this.outputColumn = outputColumn;
+    this.outputType = "long";
+  }
+
+  public FuncTimestampToLong() {
+    super();
+    this.outputType = "long";
+  }
+
+  abstract protected void func(LongColumnVector outV, TimestampColumnVector inV, int i);
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    TimestampColumnVector inV = (TimestampColumnVector) batch.cols[inputColumn];
+    int[] sel = batch.selected;
+    int n = batch.size;
+    LongColumnVector outV = (LongColumnVector) batch.cols[outputColumn];
+
+    if (n == 0) {
+
+      // Nothing to do
+      return;
+    }
+
+    if (inV.noNulls) {
+      outV.noNulls = true;
+      if (inV.isRepeating) {
+        outV.isRepeating = true;
+        func(outV, inV, 0);
+      } else if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          func(outV, inV, i);
+        }
+        outV.isRepeating = false;
+      } else {
+        for(int i = 0; i != n; i++) {
+          func(outV, inV, i);
+        }
+        outV.isRepeating = false;
+      }
+    } else {
+
+      // Handle case with nulls. Don't do function if the value is null,
+      // because the data may be undefined for a null value.
+      outV.noNulls = false;
+      if (inV.isRepeating) {
+        outV.isRepeating = true;
+        outV.isNull[0] = inV.isNull[0];
+        if (!inV.isNull[0]) {
+          func(outV, inV, 0);
+        }
+      } else if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          outV.isNull[i] = inV.isNull[i];
+          if (!inV.isNull[i]) {
+            func(outV, inV, i);
+          }
+        }
+        outV.isRepeating = false;
+      } else {
+        System.arraycopy(inV.isNull, 0, outV.isNull, 0, n);
+        for(int i = 0; i != n; i++) {
+          if (!inV.isNull[i]) {
+            func(outV, inV, i);
+          }
+        }
+        outV.isRepeating = false;
+      }
+    }
+  }
+
+
+  @Override
+  public int getOutputColumn() {
+    return outputColumn;
+  }
+
+  public void setOutputColumn(int outputColumn) {
+    this.outputColumn = outputColumn;
+  }
+
+  public int getInputColumn() {
+    return inputColumn;
+  }
+
+  public void setInputColumn(int inputColumn) {
+    this.inputColumn = inputColumn;
+  }
+
+  @Override
+  public VectorExpressionDescriptor.Descriptor getDescriptor() {
+    VectorExpressionDescriptor.Builder b = new VectorExpressionDescriptor.Builder();
+    b.setMode(VectorExpressionDescriptor.Mode.PROJECTION)
+        .setNumArguments(1)
+        .setArgumentTypes(
+            VectorExpressionDescriptor.ArgumentType.TIMESTAMP)
+        .setInputExpressionTypes(
+            VectorExpressionDescriptor.InputExpressionType.COLUMN);
+    return b.build();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/4a479d0c/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/ITimestampInExpr.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/ITimestampInExpr.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/ITimestampInExpr.java
new file mode 100644
index 0000000..f6cc971
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/ITimestampInExpr.java
@@ -0,0 +1,25 @@
+/**
+ * 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;
+
+import java.sql.Timestamp;
+
+public interface ITimestampInExpr {
+  void setInListValues(Timestamp[] inVals);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/4a479d0c/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprIntervalDayTimeColumnColumn.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprIntervalDayTimeColumnColumn.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprIntervalDayTimeColumnColumn.java
new file mode 100644
index 0000000..a6f8057
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprIntervalDayTimeColumnColumn.java
@@ -0,0 +1,54 @@
+/*
+ * 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;
+
+import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+
+/**
+ * Compute IF(expr1, expr2, expr3) for 3 input column expressions.
+ * The first is always a boolean (LongColumnVector).
+ * The second and third are long columns or long expression results.
+ */
+public class IfExprIntervalDayTimeColumnColumn extends IfExprTimestampColumnColumnBase {
+
+  private static final long serialVersionUID = 1L;
+
+  public IfExprIntervalDayTimeColumnColumn(int arg1Column, int arg2Column, int arg3Column, int outputColumn) {
+    super(arg1Column, arg2Column, arg3Column, outputColumn);
+  }
+
+  public IfExprIntervalDayTimeColumnColumn() {
+    super();
+  }
+
+  @Override
+  public VectorExpressionDescriptor.Descriptor getDescriptor() {
+    return (new VectorExpressionDescriptor.Builder())
+        .setMode(
+            VectorExpressionDescriptor.Mode.PROJECTION)
+        .setNumArguments(3)
+        .setArgumentTypes(
+            VectorExpressionDescriptor.ArgumentType.getType("int_family"),
+            VectorExpressionDescriptor.ArgumentType.getType("interval_day_time"),
+            VectorExpressionDescriptor.ArgumentType.getType("interval_day_time"))
+        .setInputExpressionTypes(
+            VectorExpressionDescriptor.InputExpressionType.COLUMN,
+            VectorExpressionDescriptor.InputExpressionType.COLUMN,
+            VectorExpressionDescriptor.InputExpressionType.COLUMN).build();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/4a479d0c/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprIntervalDayTimeColumnScalar.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprIntervalDayTimeColumnScalar.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprIntervalDayTimeColumnScalar.java
new file mode 100644
index 0000000..4beb50a
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprIntervalDayTimeColumnScalar.java
@@ -0,0 +1,59 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.exec.vector.expressions;
+
+import org.apache.hadoop.hive.common.type.HiveIntervalDayTime;
+import org.apache.hadoop.hive.common.type.PisaTimestamp;
+import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+
+/**
+ * Compute IF(expr1, expr2, expr3) for 3 input column expressions.
+ * The first is always a boolean (LongColumnVector).
+ * The second is a column or non-constant expression result.
+ * The third is a constant value.
+ */
+public class IfExprIntervalDayTimeColumnScalar extends IfExprTimestampColumnScalarBase {
+
+  private static final long serialVersionUID = 1L;
+
+  public IfExprIntervalDayTimeColumnScalar(int arg1Column, int arg2Column, HiveIntervalDayTime arg3Scalar,
+      int outputColumn) {
+    super(arg1Column, arg2Column, arg3Scalar.pisaTimestampUpdate(new PisaTimestamp()), outputColumn);
+  }
+
+  public IfExprIntervalDayTimeColumnScalar() {
+    super();
+  }
+
+  @Override
+  public VectorExpressionDescriptor.Descriptor getDescriptor() {
+    return (new VectorExpressionDescriptor.Builder())
+        .setMode(
+            VectorExpressionDescriptor.Mode.PROJECTION)
+        .setNumArguments(3)
+        .setArgumentTypes(
+            VectorExpressionDescriptor.ArgumentType.getType("int_family"),
+            VectorExpressionDescriptor.ArgumentType.getType("interval_day_time"),
+            VectorExpressionDescriptor.ArgumentType.getType("interval_day_time"))
+        .setInputExpressionTypes(
+            VectorExpressionDescriptor.InputExpressionType.COLUMN,
+            VectorExpressionDescriptor.InputExpressionType.COLUMN,
+            VectorExpressionDescriptor.InputExpressionType.SCALAR).build();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/4a479d0c/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprIntervalDayTimeScalarColumn.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprIntervalDayTimeScalarColumn.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprIntervalDayTimeScalarColumn.java
new file mode 100644
index 0000000..5463c7c
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprIntervalDayTimeScalarColumn.java
@@ -0,0 +1,59 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.exec.vector.expressions;
+
+import org.apache.hadoop.hive.common.type.HiveIntervalDayTime;
+import org.apache.hadoop.hive.common.type.PisaTimestamp;
+import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+
+/**
+ * Compute IF(expr1, expr2, expr3) for 3 input column expressions.
+ * The first is always a boolean (LongColumnVector).
+ * The second is a column or non-constant expression result.
+ * The third is a constant value.
+ */
+public class IfExprIntervalDayTimeScalarColumn extends IfExprTimestampScalarColumnBase {
+
+  private static final long serialVersionUID = 1L;
+
+  public IfExprIntervalDayTimeScalarColumn(int arg1Column, HiveIntervalDayTime arg2Scalar, int arg3Column,
+      int outputColumn) {
+    super(arg1Column, arg2Scalar.pisaTimestampUpdate(new PisaTimestamp()), arg3Column, outputColumn);
+  }
+
+  public IfExprIntervalDayTimeScalarColumn() {
+    super();
+  }
+
+  @Override
+  public VectorExpressionDescriptor.Descriptor getDescriptor() {
+    return (new VectorExpressionDescriptor.Builder())
+        .setMode(
+            VectorExpressionDescriptor.Mode.PROJECTION)
+        .setNumArguments(3)
+        .setArgumentTypes(
+            VectorExpressionDescriptor.ArgumentType.getType("int_family"),
+            VectorExpressionDescriptor.ArgumentType.getType("interval_day_time"),
+            VectorExpressionDescriptor.ArgumentType.getType("interval_day_time"))
+        .setInputExpressionTypes(
+            VectorExpressionDescriptor.InputExpressionType.COLUMN,
+            VectorExpressionDescriptor.InputExpressionType.SCALAR,
+            VectorExpressionDescriptor.InputExpressionType.COLUMN).build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/4a479d0c/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprIntervalDayTimeScalarScalar.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprIntervalDayTimeScalarScalar.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprIntervalDayTimeScalarScalar.java
new file mode 100644
index 0000000..af2e0c0
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprIntervalDayTimeScalarScalar.java
@@ -0,0 +1,59 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.exec.vector.expressions;
+
+import org.apache.hadoop.hive.common.type.HiveIntervalDayTime;
+import org.apache.hadoop.hive.common.type.PisaTimestamp;
+import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+
+/**
+ * Compute IF(expr1, expr2, expr3) for 3 input  expressions.
+ * The first is always a boolean (LongColumnVector).
+ * The second is a constant value.
+ * The third is a constant value.
+ */
+public class IfExprIntervalDayTimeScalarScalar extends IfExprTimestampScalarScalarBase {
+
+  private static final long serialVersionUID = 1L;
+
+  public IfExprIntervalDayTimeScalarScalar(int arg1Column, HiveIntervalDayTime arg2Scalar, HiveIntervalDayTime arg3Scalar,
+      int outputColumn) {
+    super(arg1Column, arg2Scalar.pisaTimestampUpdate(new PisaTimestamp()), arg3Scalar.pisaTimestampUpdate(new PisaTimestamp()), outputColumn);
+  }
+
+  public IfExprIntervalDayTimeScalarScalar() {
+    super();
+  }
+
+  @Override
+  public VectorExpressionDescriptor.Descriptor getDescriptor() {
+    return (new VectorExpressionDescriptor.Builder())
+        .setMode(
+            VectorExpressionDescriptor.Mode.PROJECTION)
+        .setNumArguments(3)
+        .setArgumentTypes(
+            VectorExpressionDescriptor.ArgumentType.getType("int_family"),
+            VectorExpressionDescriptor.ArgumentType.getType("interval_day_time"),
+            VectorExpressionDescriptor.ArgumentType.getType("interval_day_time"))
+        .setInputExpressionTypes(
+            VectorExpressionDescriptor.InputExpressionType.COLUMN,
+            VectorExpressionDescriptor.InputExpressionType.SCALAR,
+            VectorExpressionDescriptor.InputExpressionType.SCALAR).build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/4a479d0c/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprLongColumnLongColumn.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprLongColumnLongColumn.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprLongColumnLongColumn.java
index a2bb3d0..06ba8f8 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprLongColumnLongColumn.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprLongColumnLongColumn.java
@@ -173,8 +173,8 @@ public class IfExprLongColumnLongColumn extends VectorExpression {
         .setNumArguments(3)
         .setArgumentTypes(
             VectorExpressionDescriptor.ArgumentType.getType("long"),
-            VectorExpressionDescriptor.ArgumentType.getType("int_datetime_interval_family"),
-            VectorExpressionDescriptor.ArgumentType.getType("int_datetime_interval_family"))
+            VectorExpressionDescriptor.ArgumentType.getType("int_date_interval_year_month"),
+            VectorExpressionDescriptor.ArgumentType.getType("int_date_interval_year_month"))
         .setInputExpressionTypes(
             VectorExpressionDescriptor.InputExpressionType.COLUMN,
             VectorExpressionDescriptor.InputExpressionType.COLUMN,

http://git-wip-us.apache.org/repos/asf/hive/blob/4a479d0c/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprTimestampColumnColumn.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprTimestampColumnColumn.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprTimestampColumnColumn.java
new file mode 100644
index 0000000..a1e489b
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprTimestampColumnColumn.java
@@ -0,0 +1,54 @@
+/*
+ * 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;
+
+import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+
+/**
+ * Compute IF(expr1, expr2, expr3) for 3 input column expressions.
+ * The first is always a boolean (LongColumnVector).
+ * The second and third are long columns or long expression results.
+ */
+public class IfExprTimestampColumnColumn extends IfExprTimestampColumnColumnBase {
+
+  private static final long serialVersionUID = 1L;
+
+  public IfExprTimestampColumnColumn(int arg1Column, int arg2Column, int arg3Column, int outputColumn) {
+    super(arg1Column, arg2Column, arg3Column, outputColumn);
+  }
+
+  public IfExprTimestampColumnColumn() {
+    super();
+  }
+
+  @Override
+  public VectorExpressionDescriptor.Descriptor getDescriptor() {
+    return (new VectorExpressionDescriptor.Builder())
+        .setMode(
+            VectorExpressionDescriptor.Mode.PROJECTION)
+        .setNumArguments(3)
+        .setArgumentTypes(
+            VectorExpressionDescriptor.ArgumentType.getType("int_family"),
+            VectorExpressionDescriptor.ArgumentType.getType("timestamp"),
+            VectorExpressionDescriptor.ArgumentType.getType("timestamp"))
+        .setInputExpressionTypes(
+            VectorExpressionDescriptor.InputExpressionType.COLUMN,
+            VectorExpressionDescriptor.InputExpressionType.COLUMN,
+            VectorExpressionDescriptor.InputExpressionType.COLUMN).build();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/4a479d0c/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprTimestampColumnColumnBase.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprTimestampColumnColumnBase.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprTimestampColumnColumnBase.java
new file mode 100644
index 0000000..d3dd67d
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprTimestampColumnColumnBase.java
@@ -0,0 +1,136 @@
+/*
+ * 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;
+
+import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+
+/**
+ * Compute IF(expr1, expr2, expr3) for 3 input column expressions.
+ * The first is always a boolean (LongColumnVector).
+ * The second and third are long columns or long expression results.
+ */
+public abstract class IfExprTimestampColumnColumnBase extends VectorExpression {
+
+  private static final long serialVersionUID = 1L;
+
+  private int arg1Column, arg2Column, arg3Column;
+  private int outputColumn;
+
+  public IfExprTimestampColumnColumnBase(int arg1Column, int arg2Column, int arg3Column, int outputColumn) {
+    this.arg1Column = arg1Column;
+    this.arg2Column = arg2Column;
+    this.arg3Column = arg3Column;
+    this.outputColumn = outputColumn;
+  }
+
+  public IfExprTimestampColumnColumnBase() {
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    LongColumnVector arg1ColVector = (LongColumnVector) batch.cols[arg1Column];
+    TimestampColumnVector arg2ColVector = (TimestampColumnVector) batch.cols[arg2Column];
+    TimestampColumnVector arg3ColVector = (TimestampColumnVector) batch.cols[arg3Column];
+    TimestampColumnVector outputColVector = (TimestampColumnVector) batch.cols[outputColumn];
+    int[] sel = batch.selected;
+    boolean[] outputIsNull = outputColVector.isNull;
+    outputColVector.noNulls = arg2ColVector.noNulls && arg3ColVector.noNulls;
+    outputColVector.isRepeating = false; // may override later
+    int n = batch.size;
+    long[] vector1 = arg1ColVector.vector;
+
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+
+    /* All the code paths below propagate nulls even if neither arg2 nor arg3
+     * have nulls. This is to reduce the number of code paths and shorten the
+     * code, at the expense of maybe doing unnecessary work if neither input
+     * has nulls. This could be improved in the future by expanding the number
+     * of code paths.
+     */
+    if (arg1ColVector.isRepeating) {
+      if (vector1[0] == 1) {
+        arg2ColVector.copySelected(batch.selectedInUse, sel, n, outputColVector);
+      } else {
+        arg3ColVector.copySelected(batch.selectedInUse, sel, n, outputColVector);
+      }
+      return;
+    }
+
+    // extend any repeating values and noNulls indicator in the inputs
+    arg2ColVector.flatten(batch.selectedInUse, sel, n);
+    arg3ColVector.flatten(batch.selectedInUse, sel, n);
+
+    if (arg1ColVector.noNulls) {
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          outputColVector.set(i, vector1[i] == 1 ? arg2ColVector.asScratchPisaTimestamp(i) : arg3ColVector.asScratchPisaTimestamp(i));
+          outputIsNull[i] = (vector1[i] == 1 ?
+              arg2ColVector.isNull[i] : arg3ColVector.isNull[i]);
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputColVector.set(i, vector1[i] == 1 ? arg2ColVector.asScratchPisaTimestamp(i) : arg3ColVector.asScratchPisaTimestamp(i));
+          outputIsNull[i] = (vector1[i] == 1 ?
+              arg2ColVector.isNull[i] : arg3ColVector.isNull[i]);
+        }
+      }
+    } else /* there are nulls */ {
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          outputColVector.set(i, !arg1ColVector.isNull[i] && vector1[i] == 1 ?
+              arg2ColVector.asScratchPisaTimestamp(i) : arg3ColVector.asScratchPisaTimestamp(i));
+          outputIsNull[i] = (!arg1ColVector.isNull[i] && vector1[i] == 1 ?
+              arg2ColVector.isNull[i] : arg3ColVector.isNull[i]);
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputColVector.set(i, !arg1ColVector.isNull[i] && vector1[i] == 1 ?
+              arg2ColVector.asScratchPisaTimestamp(i) : arg3ColVector.asScratchPisaTimestamp(i));
+          outputIsNull[i] = (!arg1ColVector.isNull[i] && vector1[i] == 1 ?
+              arg2ColVector.isNull[i] : arg3ColVector.isNull[i]);
+        }
+      }
+    }
+
+    // restore repeating and no nulls indicators
+    arg2ColVector.unFlatten();
+    arg3ColVector.unFlatten();
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return outputColumn;
+  }
+
+  @Override
+  public String getOutputType() {
+    return "long";
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/4a479d0c/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprTimestampColumnScalar.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprTimestampColumnScalar.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprTimestampColumnScalar.java
new file mode 100644
index 0000000..0660038
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprTimestampColumnScalar.java
@@ -0,0 +1,60 @@
+/**
+ * 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;
+
+import java.sql.Timestamp;
+
+import org.apache.hadoop.hive.common.type.PisaTimestamp;
+import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+
+/**
+ * Compute IF(expr1, expr2, expr3) for 3 input column expressions.
+ * The first is always a boolean (LongColumnVector).
+ * The second is a column or non-constant expression result.
+ * The third is a constant value.
+ */
+public class IfExprTimestampColumnScalar extends IfExprTimestampColumnScalarBase {
+
+  private static final long serialVersionUID = 1L;
+
+  public IfExprTimestampColumnScalar(int arg1Column, int arg2Column, Timestamp arg3Scalar,
+      int outputColumn) {
+    super(arg1Column, arg2Column, new PisaTimestamp(arg3Scalar), outputColumn);
+  }
+
+  public IfExprTimestampColumnScalar() {
+    super();
+  }
+
+  @Override
+  public VectorExpressionDescriptor.Descriptor getDescriptor() {
+    return (new VectorExpressionDescriptor.Builder())
+        .setMode(
+            VectorExpressionDescriptor.Mode.PROJECTION)
+        .setNumArguments(3)
+        .setArgumentTypes(
+            VectorExpressionDescriptor.ArgumentType.getType("int_family"),
+            VectorExpressionDescriptor.ArgumentType.getType("timestamp"),
+            VectorExpressionDescriptor.ArgumentType.getType("timestamp"))
+        .setInputExpressionTypes(
+            VectorExpressionDescriptor.InputExpressionType.COLUMN,
+            VectorExpressionDescriptor.InputExpressionType.COLUMN,
+            VectorExpressionDescriptor.InputExpressionType.SCALAR).build();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/4a479d0c/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprTimestampColumnScalarBase.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprTimestampColumnScalarBase.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprTimestampColumnScalarBase.java
new file mode 100644
index 0000000..8aaad3f
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprTimestampColumnScalarBase.java
@@ -0,0 +1,133 @@
+/**
+ * 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;
+
+import org.apache.hadoop.hive.common.type.PisaTimestamp;
+
+import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.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;
+
+/**
+ * Compute IF(expr1, expr2, expr3) for 3 input column expressions.
+ * The first is always a boolean (LongColumnVector).
+ * The second is a column or non-constant expression result.
+ * The third is a constant value.
+ */
+public abstract class IfExprTimestampColumnScalarBase extends VectorExpression {
+
+  private static final long serialVersionUID = 1L;
+
+  private int arg1Column, arg2Column;
+  private PisaTimestamp arg3Scalar;
+  private int outputColumn;
+
+  public IfExprTimestampColumnScalarBase(int arg1Column, int arg2Column, PisaTimestamp arg3Scalar,
+      int outputColumn) {
+    this.arg1Column = arg1Column;
+    this.arg2Column = arg2Column;
+    this.arg3Scalar = arg3Scalar;
+    this.outputColumn = outputColumn;
+  }
+
+  public IfExprTimestampColumnScalarBase() {
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    LongColumnVector arg1ColVector = (LongColumnVector) batch.cols[arg1Column];
+    TimestampColumnVector arg2ColVector = (TimestampColumnVector) batch.cols[arg2Column];
+    TimestampColumnVector outputColVector = (TimestampColumnVector) batch.cols[outputColumn];
+    int[] sel = batch.selected;
+    boolean[] outputIsNull = outputColVector.isNull;
+    outputColVector.noNulls = arg2ColVector.noNulls; // nulls can only come from arg2
+    outputColVector.isRepeating = false; // may override later
+    int n = batch.size;
+    long[] vector1 = arg1ColVector.vector;
+
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+
+    if (arg1ColVector.isRepeating) {
+      if (vector1[0] == 1) {
+        arg2ColVector.copySelected(batch.selectedInUse, sel, n, outputColVector);
+      } else {
+        outputColVector.fill(arg3Scalar);
+      }
+      return;
+    }
+
+    // Extend any repeating values and noNulls indicator in the inputs to
+    // reduce the number of code paths needed below.
+    arg2ColVector.flatten(batch.selectedInUse, sel, n);
+
+    if (arg1ColVector.noNulls) {
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          outputColVector.set(i, vector1[i] == 1 ? arg2ColVector.asScratchPisaTimestamp(i) : arg3Scalar);
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputColVector.set(i, vector1[i] == 1 ? arg2ColVector.asScratchPisaTimestamp(i) : arg3Scalar);
+        }
+      }
+    } else /* there are nulls */ {
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          outputColVector.set(i, !arg1ColVector.isNull[i] && vector1[i] == 1 ?
+              arg2ColVector.asScratchPisaTimestamp(i) : arg3Scalar);
+          outputIsNull[i] = (!arg1ColVector.isNull[i] && vector1[i] == 1 ?
+              arg2ColVector.isNull[i] : false);
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputColVector.set(i, !arg1ColVector.isNull[i] && vector1[i] == 1 ?
+              arg2ColVector.asScratchPisaTimestamp(i) : arg3Scalar);
+          outputIsNull[i] = (!arg1ColVector.isNull[i] && vector1[i] == 1 ?
+              arg2ColVector.isNull[i] : false);
+        }
+      }
+    }
+
+    // restore repeating and no nulls indicators
+    arg2ColVector.unFlatten();
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return outputColumn;
+  }
+
+  @Override
+  public String getOutputType() {
+    return "timestamp";
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/4a479d0c/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprTimestampScalarColumn.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprTimestampScalarColumn.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprTimestampScalarColumn.java
new file mode 100644
index 0000000..7f618cb
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprTimestampScalarColumn.java
@@ -0,0 +1,60 @@
+/**
+ * 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;
+
+import java.sql.Timestamp;
+
+import org.apache.hadoop.hive.common.type.PisaTimestamp;
+import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+
+/**
+ * Compute IF(expr1, expr2, expr3) for 3 input column expressions.
+ * The first is always a boolean (LongColumnVector).
+ * The second is a column or non-constant expression result.
+ * The third is a constant value.
+ */
+public class IfExprTimestampScalarColumn extends IfExprTimestampScalarColumnBase {
+
+  private static final long serialVersionUID = 1L;
+
+  public IfExprTimestampScalarColumn(int arg1Column, Timestamp arg2Scalar, int arg3Column,
+      int outputColumn) {
+    super(arg1Column, new PisaTimestamp(arg2Scalar), arg3Column, outputColumn);
+  }
+
+  public IfExprTimestampScalarColumn() {
+    super();
+  }
+
+  @Override
+  public VectorExpressionDescriptor.Descriptor getDescriptor() {
+    return (new VectorExpressionDescriptor.Builder())
+        .setMode(
+            VectorExpressionDescriptor.Mode.PROJECTION)
+        .setNumArguments(3)
+        .setArgumentTypes(
+            VectorExpressionDescriptor.ArgumentType.getType("int_family"),
+            VectorExpressionDescriptor.ArgumentType.getType("timestamp"),
+            VectorExpressionDescriptor.ArgumentType.getType("timestamp"))
+        .setInputExpressionTypes(
+            VectorExpressionDescriptor.InputExpressionType.COLUMN,
+            VectorExpressionDescriptor.InputExpressionType.SCALAR,
+            VectorExpressionDescriptor.InputExpressionType.COLUMN).build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/4a479d0c/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprTimestampScalarColumnBase.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprTimestampScalarColumnBase.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprTimestampScalarColumnBase.java
new file mode 100644
index 0000000..84d7655
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprTimestampScalarColumnBase.java
@@ -0,0 +1,133 @@
+/**
+ * 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;
+
+import org.apache.hadoop.hive.common.type.PisaTimestamp;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+
+/**
+ * Compute IF(expr1, expr2, expr3) for 3 input column expressions.
+ * The first is always a boolean (LongColumnVector).
+ * The second is a column or non-constant expression result.
+ * The third is a constant value.
+ */
+public abstract class IfExprTimestampScalarColumnBase extends VectorExpression {
+
+  private static final long serialVersionUID = 1L;
+
+  private int arg1Column, arg3Column;
+  private PisaTimestamp arg2Scalar;
+  private int outputColumn;
+
+  public IfExprTimestampScalarColumnBase(int arg1Column, PisaTimestamp arg2Scalar, int arg3Column,
+      int outputColumn) {
+    this.arg1Column = arg1Column;
+    this.arg2Scalar = arg2Scalar;
+    this.arg3Column = arg3Column;
+    this.outputColumn = outputColumn;
+  }
+
+  public IfExprTimestampScalarColumnBase() {
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    LongColumnVector arg1ColVector = (LongColumnVector) batch.cols[arg1Column];
+    TimestampColumnVector arg3ColVector = (TimestampColumnVector) batch.cols[arg3Column];
+    TimestampColumnVector outputColVector = (TimestampColumnVector) batch.cols[outputColumn];
+    int[] sel = batch.selected;
+    boolean[] outputIsNull = outputColVector.isNull;
+    outputColVector.noNulls = arg3ColVector.noNulls; // nulls can only come from arg3 column vector
+    outputColVector.isRepeating = false; // may override later
+    int n = batch.size;
+    long[] vector1 = arg1ColVector.vector;
+
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+
+    if (arg1ColVector.isRepeating) {
+      if (vector1[0] == 1) {
+        outputColVector.fill(arg2Scalar);
+      } else {
+        arg3ColVector.copySelected(batch.selectedInUse, sel, n, outputColVector);
+      }
+      return;
+    }
+
+    // Extend any repeating values and noNulls indicator in the inputs to
+    // reduce the number of code paths needed below.
+    // This could be optimized in the future by having separate paths
+    // for when arg3ColVector is repeating or has no nulls.
+    arg3ColVector.flatten(batch.selectedInUse, sel, n);
+
+    if (arg1ColVector.noNulls) {
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          outputColVector.set(i, vector1[i] == 1 ? arg2Scalar : arg3ColVector.asScratchPisaTimestamp(i));
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputColVector.set(i, vector1[i] == 1 ? arg2Scalar : arg3ColVector.asScratchPisaTimestamp(i));
+        }
+      }
+    } else /* there are nulls */ {
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          outputColVector.set(i, !arg1ColVector.isNull[i] && vector1[i] == 1 ?
+              arg2Scalar : arg3ColVector.asScratchPisaTimestamp(i));
+          outputIsNull[i] = (!arg1ColVector.isNull[i] && vector1[i] == 1 ?
+              false : arg3ColVector.isNull[i]);
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputColVector.set(i, !arg1ColVector.isNull[i] && vector1[i] == 1 ?
+              arg2Scalar : arg3ColVector.asScratchPisaTimestamp(i));
+          outputIsNull[i] = (!arg1ColVector.isNull[i] && vector1[i] == 1 ?
+              false : arg3ColVector.isNull[i]);
+        }
+      }
+    }
+
+    // restore repeating and no nulls indicators
+    arg3ColVector.unFlatten();
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return outputColumn;
+  }
+
+  @Override
+  public String getOutputType() {
+    return "timestamp";
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/4a479d0c/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprTimestampScalarScalar.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprTimestampScalarScalar.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprTimestampScalarScalar.java
new file mode 100644
index 0000000..5286ea3
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprTimestampScalarScalar.java
@@ -0,0 +1,60 @@
+/**
+ * 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;
+
+import org.apache.hadoop.hive.common.type.PisaTimestamp;
+import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+
+import java.sql.Timestamp;
+
+/**
+ * Compute IF(expr1, expr2, expr3) for 3 input  expressions.
+ * The first is always a boolean (LongColumnVector).
+ * The second is a constant value.
+ * The third is a constant value.
+ */
+public class IfExprTimestampScalarScalar extends IfExprTimestampScalarScalarBase {
+
+  private static final long serialVersionUID = 1L;
+
+  public IfExprTimestampScalarScalar(int arg1Column, Timestamp arg2Scalar, Timestamp arg3Scalar,
+      int outputColumn) {
+    super(arg1Column, new PisaTimestamp(arg2Scalar), new PisaTimestamp(arg3Scalar), outputColumn);
+  }
+
+  public IfExprTimestampScalarScalar() {
+    super();
+  }
+
+  @Override
+  public VectorExpressionDescriptor.Descriptor getDescriptor() {
+    return (new VectorExpressionDescriptor.Builder())
+        .setMode(
+            VectorExpressionDescriptor.Mode.PROJECTION)
+        .setNumArguments(3)
+        .setArgumentTypes(
+            VectorExpressionDescriptor.ArgumentType.getType("int_family"),
+            VectorExpressionDescriptor.ArgumentType.getType("timestamp"),
+            VectorExpressionDescriptor.ArgumentType.getType("timestamp"))
+        .setInputExpressionTypes(
+            VectorExpressionDescriptor.InputExpressionType.COLUMN,
+            VectorExpressionDescriptor.InputExpressionType.SCALAR,
+            VectorExpressionDescriptor.InputExpressionType.SCALAR).build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/4a479d0c/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprTimestampScalarScalarBase.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprTimestampScalarScalarBase.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprTimestampScalarScalarBase.java
new file mode 100644
index 0000000..1aeabfc
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprTimestampScalarScalarBase.java
@@ -0,0 +1,123 @@
+/**
+ * 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;
+
+import org.apache.hadoop.hive.common.type.PisaTimestamp;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import java.util.Arrays;
+
+/**
+ * Compute IF(expr1, expr2, expr3) for 3 input  expressions.
+ * The first is always a boolean (LongColumnVector).
+ * The second is a constant value.
+ * The third is a constant value.
+ */
+public abstract class IfExprTimestampScalarScalarBase extends VectorExpression {
+
+  private static final long serialVersionUID = 1L;
+
+  private int arg1Column;
+  private PisaTimestamp arg2Scalar;
+  private PisaTimestamp arg3Scalar;
+  private int outputColumn;
+
+  public IfExprTimestampScalarScalarBase(int arg1Column, PisaTimestamp arg2Scalar, PisaTimestamp arg3Scalar,
+      int outputColumn) {
+    this.arg1Column = arg1Column;
+    this.arg2Scalar = arg2Scalar;
+    this.arg3Scalar = arg3Scalar;
+    this.outputColumn = outputColumn;
+  }
+
+  public IfExprTimestampScalarScalarBase() {
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    LongColumnVector arg1ColVector = (LongColumnVector) batch.cols[arg1Column];
+    TimestampColumnVector outputColVector = (TimestampColumnVector) batch.cols[outputColumn];
+    int[] sel = batch.selected;
+    boolean[] outputIsNull = outputColVector.isNull;
+    outputColVector.noNulls = false; // output is a scalar which we know is non null
+    outputColVector.isRepeating = false; // may override later
+    int n = batch.size;
+    long[] vector1 = arg1ColVector.vector;
+
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+
+    if (arg1ColVector.isRepeating) {
+      if (vector1[0] == 1) {
+        outputColVector.fill(arg2Scalar);
+      } else {
+        outputColVector.fill(arg3Scalar);
+      }
+    } else if (arg1ColVector.noNulls) {
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          outputColVector.set(i, vector1[i] == 1 ? arg2Scalar : arg3Scalar);
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputColVector.set(i, vector1[i] == 1 ? arg2Scalar : arg3Scalar);
+        }
+      }
+    } else /* there are nulls */ {
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          outputColVector.set(i, !arg1ColVector.isNull[i] && vector1[i] == 1 ?
+              arg2Scalar : arg3Scalar);
+          outputIsNull[i] = false;
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputColVector.set(i, !arg1ColVector.isNull[i] && vector1[i] == 1 ?
+              arg2Scalar : arg3Scalar);
+        }
+        Arrays.fill(outputIsNull, 0, n, false);
+      }
+    }
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return outputColumn;
+  }
+
+  @Override
+  public String getOutputType() {
+    return "timestamp";
+  }
+
+  public int getArg1Column() {
+    return arg1Column;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/4a479d0c/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColEqualLongColumn.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColEqualLongColumn.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColEqualLongColumn.java
index 9b9f15e..a77d41a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColEqualLongColumn.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColEqualLongColumn.java
@@ -160,8 +160,8 @@ public class LongColEqualLongColumn extends VectorExpression {
             VectorExpressionDescriptor.Mode.PROJECTION)
         .setNumArguments(2)
         .setArgumentTypes(
-            VectorExpressionDescriptor.ArgumentType.getType("int_datetime_interval_family"),
-            VectorExpressionDescriptor.ArgumentType.getType("int_datetime_interval_family"))
+            VectorExpressionDescriptor.ArgumentType.getType("int_date_interval_year_month"),
+            VectorExpressionDescriptor.ArgumentType.getType("int_date_interval_year_month"))
         .setInputExpressionTypes(
             VectorExpressionDescriptor.InputExpressionType.COLUMN,
             VectorExpressionDescriptor.InputExpressionType.COLUMN).build();

http://git-wip-us.apache.org/repos/asf/hive/blob/4a479d0c/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColGreaterEqualLongColumn.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColGreaterEqualLongColumn.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColGreaterEqualLongColumn.java
index 94c5bed..6ee5daf 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColGreaterEqualLongColumn.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColGreaterEqualLongColumn.java
@@ -160,8 +160,8 @@ public class LongColGreaterEqualLongColumn extends VectorExpression {
             VectorExpressionDescriptor.Mode.PROJECTION)
         .setNumArguments(2)
         .setArgumentTypes(
-            VectorExpressionDescriptor.ArgumentType.getType("int_datetime_interval_family"),
-            VectorExpressionDescriptor.ArgumentType.getType("int_datetime_interval_family"))
+            VectorExpressionDescriptor.ArgumentType.getType("int_date_interval_year_month"),
+            VectorExpressionDescriptor.ArgumentType.getType("int_date_interval_year_month"))
         .setInputExpressionTypes(
             VectorExpressionDescriptor.InputExpressionType.COLUMN,
             VectorExpressionDescriptor.InputExpressionType.COLUMN).build();

http://git-wip-us.apache.org/repos/asf/hive/blob/4a479d0c/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColGreaterLongColumn.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColGreaterLongColumn.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColGreaterLongColumn.java
index dda941e..053ced9 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColGreaterLongColumn.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColGreaterLongColumn.java
@@ -160,8 +160,8 @@ public class LongColGreaterLongColumn extends VectorExpression {
             VectorExpressionDescriptor.Mode.PROJECTION)
         .setNumArguments(2)
         .setArgumentTypes(
-            VectorExpressionDescriptor.ArgumentType.getType("int_datetime_interval_family"),
-            VectorExpressionDescriptor.ArgumentType.getType("int_datetime_interval_family"))
+            VectorExpressionDescriptor.ArgumentType.getType("int_date_interval_year_month"),
+            VectorExpressionDescriptor.ArgumentType.getType("int_date_interval_year_month"))
         .setInputExpressionTypes(
             VectorExpressionDescriptor.InputExpressionType.COLUMN,
             VectorExpressionDescriptor.InputExpressionType.COLUMN).build();

http://git-wip-us.apache.org/repos/asf/hive/blob/4a479d0c/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColLessEqualLongColumn.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColLessEqualLongColumn.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColLessEqualLongColumn.java
index aacdfe6..25d52b3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColLessEqualLongColumn.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColLessEqualLongColumn.java
@@ -160,8 +160,8 @@ public class LongColLessEqualLongColumn extends VectorExpression {
             VectorExpressionDescriptor.Mode.PROJECTION)
         .setNumArguments(2)
         .setArgumentTypes(
-            VectorExpressionDescriptor.ArgumentType.getType("int_datetime_interval_family"),
-            VectorExpressionDescriptor.ArgumentType.getType("int_datetime_interval_family"))
+            VectorExpressionDescriptor.ArgumentType.getType("int_date_interval_year_month"),
+            VectorExpressionDescriptor.ArgumentType.getType("int_date_interval_year_month"))
         .setInputExpressionTypes(
             VectorExpressionDescriptor.InputExpressionType.COLUMN,
             VectorExpressionDescriptor.InputExpressionType.COLUMN).build();

http://git-wip-us.apache.org/repos/asf/hive/blob/4a479d0c/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColLessLongColumn.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColLessLongColumn.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColLessLongColumn.java
index b041ab6..e6e54e9 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColLessLongColumn.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColLessLongColumn.java
@@ -160,8 +160,8 @@ public class LongColLessLongColumn extends VectorExpression {
             VectorExpressionDescriptor.Mode.PROJECTION)
         .setNumArguments(2)
         .setArgumentTypes(
-            VectorExpressionDescriptor.ArgumentType.getType("int_datetime_interval_family"),
-            VectorExpressionDescriptor.ArgumentType.getType("int_datetime_interval_family"))
+            VectorExpressionDescriptor.ArgumentType.getType("int_date_interval_year_month"),
+            VectorExpressionDescriptor.ArgumentType.getType("int_date_interval_year_month"))
         .setInputExpressionTypes(
             VectorExpressionDescriptor.InputExpressionType.COLUMN,
             VectorExpressionDescriptor.InputExpressionType.COLUMN).build();

http://git-wip-us.apache.org/repos/asf/hive/blob/4a479d0c/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColNotEqualLongColumn.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColNotEqualLongColumn.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColNotEqualLongColumn.java
index 179e493..865fdb9 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColNotEqualLongColumn.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColNotEqualLongColumn.java
@@ -160,8 +160,8 @@ public class LongColNotEqualLongColumn extends VectorExpression {
             VectorExpressionDescriptor.Mode.PROJECTION)
         .setNumArguments(2)
         .setArgumentTypes(
-            VectorExpressionDescriptor.ArgumentType.getType("int_datetime_interval_family"),
-            VectorExpressionDescriptor.ArgumentType.getType("int_datetime_interval_family"))
+            VectorExpressionDescriptor.ArgumentType.getType("int_date_interval_year_month"),
+            VectorExpressionDescriptor.ArgumentType.getType("int_date_interval_year_month"))
         .setInputExpressionTypes(
             VectorExpressionDescriptor.InputExpressionType.COLUMN,
             VectorExpressionDescriptor.InputExpressionType.COLUMN).build();

http://git-wip-us.apache.org/repos/asf/hive/blob/4a479d0c/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/NullUtil.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/NullUtil.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/NullUtil.java
index 2eb48fb..3c6824d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/NullUtil.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/NullUtil.java
@@ -25,6 +25,7 @@ import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
 
 /**
  * Utility functions to handle null propagation.
@@ -81,6 +82,31 @@ public class NullUtil {
     }
   }
 
+  /**
+   * Set the data value for all NULL entries to the designated NULL_VALUE.
+   */
+  public static void setNullDataEntriesTimestamp(
+      TimestampColumnVector v, boolean selectedInUse, int[] sel, int n) {
+    if (v.noNulls) {
+      return;
+    } else if (v.isRepeating && v.isNull[0]) {
+      v.setNullValue(0);
+    } else if (selectedInUse) {
+      for (int j = 0; j != n; j++) {
+        int i = sel[j];
+        if(v.isNull[i]) {
+          v.setNullValue(i);
+        }
+      }
+    } else {
+      for (int i = 0; i != n; i++) {
+        if(v.isNull[i]) {
+          v.setNullValue(i);
+        }
+      }
+    }
+  }
+
   // for use by Column-Scalar and Scalar-Column arithmetic for null propagation
   public static void setNullOutputEntriesColScalar(
       ColumnVector v, boolean selectedInUse, int[] sel, int n) {
@@ -89,8 +115,11 @@ public class NullUtil {
       // No need to set null data entries because the input NaN values
       // will automatically propagate to the output.
       return;
+    } else if (v instanceof LongColumnVector) {
+      setNullDataEntriesLong((LongColumnVector) v, selectedInUse, sel, n);
+    } else if (v instanceof TimestampColumnVector){
+      setNullDataEntriesTimestamp((TimestampColumnVector) v, selectedInUse, sel, n);
     }
-    setNullDataEntriesLong((LongColumnVector) v, selectedInUse, sel, n);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hive/blob/4a479d0c/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/TimestampColumnInList.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/TimestampColumnInList.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/TimestampColumnInList.java
new file mode 100644
index 0000000..2d7d0c2
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/TimestampColumnInList.java
@@ -0,0 +1,164 @@
+/**
+ * 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;
+
+import java.sql.Timestamp;
+import java.util.HashSet;
+
+import org.apache.hadoop.hive.common.type.PisaTimestamp;
+import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor.Descriptor;
+import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+
+/**
+ * Output a boolean value indicating if a column is IN a list of constants.
+ */
+public class TimestampColumnInList extends VectorExpression implements ITimestampInExpr {
+  private static final long serialVersionUID = 1L;
+  private int inputCol;
+  private Timestamp[] inListValues;
+  private int outputColumn;
+
+  private transient PisaTimestamp scratchTimestamp;
+
+
+  // The set object containing the IN list.
+  private transient HashSet<PisaTimestamp> inSet;
+
+  public TimestampColumnInList() {
+    super();
+    inSet = null;
+  }
+
+  /**
+   * After construction you must call setInListValues() to add the values to the IN set.
+   */
+  public TimestampColumnInList(int colNum, int outputColumn) {
+    this.inputCol = colNum;
+    this.outputColumn = outputColumn;
+    inSet = null;
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    if (inSet == null) {
+      inSet = new HashSet<PisaTimestamp>(inListValues.length);
+      for (Timestamp val : inListValues) {
+        inSet.add(new PisaTimestamp(val));
+      }
+      scratchTimestamp = new PisaTimestamp();
+    }
+
+    TimestampColumnVector inputColVector = (TimestampColumnVector) batch.cols[inputCol];
+    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.
+        inputColVector.pisaTimestampUpdate(scratchTimestamp, 0);
+        outputVector[0] = inSet.contains(scratchTimestamp) ? 1 : 0;
+        outputColVector.isRepeating = true;
+      } else if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          inputColVector.pisaTimestampUpdate(scratchTimestamp, i);
+          outputVector[i] = inSet.contains(scratchTimestamp) ? 1 : 0;
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          inputColVector.pisaTimestampUpdate(scratchTimestamp, i);
+          outputVector[i] = inSet.contains(scratchTimestamp) ? 1 : 0;
+        }
+      }
+    } else {
+      if (inputColVector.isRepeating) {
+
+        //All must be selected otherwise size would be zero
+        //Repeating property will not change.
+        if (!nullPos[0]) {
+          inputColVector.pisaTimestampUpdate(scratchTimestamp, 0);
+          outputVector[0] = inSet.contains(scratchTimestamp) ? 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];
+          outNulls[i] = nullPos[i];
+          if (!nullPos[i]) {
+            inputColVector.pisaTimestampUpdate(scratchTimestamp, i);
+            outputVector[i] = inSet.contains(scratchTimestamp) ? 1 : 0;
+          }
+        }
+      } else {
+        System.arraycopy(nullPos, 0, outNulls, 0, n);
+        for(int i = 0; i != n; i++) {
+          if (!nullPos[i]) {
+            inputColVector.pisaTimestampUpdate(scratchTimestamp, i);
+            outputVector[i] = inSet.contains(scratchTimestamp) ? 1 : 0;
+          }
+        }
+      }
+    }
+  }
+
+
+  @Override
+  public String getOutputType() {
+    return "boolean";
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return outputColumn;
+  }
+
+  @Override
+  public Descriptor getDescriptor() {
+
+    // This VectorExpression (IN) is a special case, so don't return a descriptor.
+    return null;
+  }
+
+  public void setInListValues(Timestamp[] a) {
+    this.inListValues = a;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/4a479d0c/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/TimestampToStringUnaryUDF.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/TimestampToStringUnaryUDF.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/TimestampToStringUnaryUDF.java
new file mode 100644
index 0000000..052d57c
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/TimestampToStringUnaryUDF.java
@@ -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;
+
+import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+
+/**
+ * This is a superclass for unary long functions returning strings that operate directly on the
+ * input and set the output.
+ */
+abstract public class TimestampToStringUnaryUDF extends VectorExpression {
+  private static final long serialVersionUID = 1L;
+  int inputColumn;
+  int outputColumn;
+
+  public TimestampToStringUnaryUDF(int inputColumn, int outputColumn) {
+    this.inputColumn = inputColumn;
+    this.outputColumn = outputColumn;
+  }
+
+  public TimestampToStringUnaryUDF() {
+    super();
+  }
+
+  abstract protected void func(BytesColumnVector outV, TimestampColumnVector inV, int i);
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    TimestampColumnVector inputColVector = (TimestampColumnVector) batch.cols[inputColumn];
+    int[] sel = batch.selected;
+    int n = batch.size;
+    BytesColumnVector outV = (BytesColumnVector) batch.cols[outputColumn];
+    outV.initBuffer();
+
+    if (n == 0) {
+      //Nothing to do
+      return;
+    }
+
+    if (inputColVector.noNulls) {
+      outV.noNulls = true;
+      if (inputColVector.isRepeating) {
+        outV.isRepeating = true;
+        func(outV, inputColVector, 0);
+      } else if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          func(outV, inputColVector, i);
+        }
+        outV.isRepeating = false;
+      } else {
+        for(int i = 0; i != n; i++) {
+          func(outV, inputColVector, i);
+        }
+        outV.isRepeating = false;
+      }
+    } else {
+
+      // Handle case with nulls. Don't do function if the value is null,
+      // because the data may be undefined for a null value.
+      outV.noNulls = false;
+      if (inputColVector.isRepeating) {
+        outV.isRepeating = true;
+        outV.isNull[0] = inputColVector.isNull[0];
+        if (!inputColVector.isNull[0]) {
+          func(outV, inputColVector, 0);
+        }
+      } else if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          outV.isNull[i] = inputColVector.isNull[i];
+          if (!inputColVector.isNull[i]) {
+            func(outV, inputColVector, i);
+          }
+        }
+        outV.isRepeating = false;
+      } else {
+        System.arraycopy(inputColVector.isNull, 0, outV.isNull, 0, n);
+        for(int i = 0; i != n; i++) {
+          if (!inputColVector.isNull[i]) {
+            func(outV, inputColVector, i);
+          }
+        }
+        outV.isRepeating = false;
+      }
+    }
+  }
+
+
+  @Override
+  public int getOutputColumn() {
+    return outputColumn;
+  }
+
+  public void setOutputColumn(int outputColumn) {
+    this.outputColumn = outputColumn;
+  }
+
+  public int getInputColumn() {
+    return inputColumn;
+  }
+
+  public void setInputColumn(int inputColumn) {
+    this.inputColumn = inputColumn;
+  }
+
+  @Override
+  public String getOutputType() {
+    return "String";
+  }
+
+  @Override
+  public VectorExpressionDescriptor.Descriptor getDescriptor() {
+    VectorExpressionDescriptor.Builder b = new VectorExpressionDescriptor.Builder();
+    b.setMode(VectorExpressionDescriptor.Mode.PROJECTION)
+        .setNumArguments(1)
+        .setArgumentTypes(
+            VectorExpressionDescriptor.ArgumentType.TIMESTAMP)
+        .setInputExpressionTypes(
+            VectorExpressionDescriptor.InputExpressionType.COLUMN);
+    return b.build();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/4a479d0c/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorExpression.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorExpression.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorExpression.java
index c0e4cf0..8fca8a1 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorExpression.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorExpression.java
@@ -144,6 +144,8 @@ public abstract class VectorExpression implements Serializable {
     b.append(this.getClass().getSimpleName());
     b.append("[");
     b.append(this.getOutputColumn());
+    b.append(":");
+    b.append(this.getOutputType());
     b.append("]");
     if (childExpressions != null) {
       b.append("(");

http://git-wip-us.apache.org/repos/asf/hive/blob/4a479d0c/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorExpressionWriter.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorExpressionWriter.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorExpressionWriter.java
index d91b880..326bfb9 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorExpressionWriter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorExpressionWriter.java
@@ -18,12 +18,15 @@
 
 package org.apache.hadoop.hive.ql.exec.vector.expressions;
 
+import java.sql.Timestamp;
+
 import org.apache.hadoop.hive.common.type.HiveDecimal;
+import org.apache.hadoop.hive.common.type.PisaTimestamp;
 import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.hadoop.hive.serde2.io.TimestampWritable;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
-import org.apache.hadoop.io.Writable;
 
 /**
  * Interface used to create Writable objects from vector expression primitives.
@@ -37,6 +40,9 @@ public interface VectorExpressionWriter {
   Object writeValue(byte[] value, int start, int length) throws HiveException;
   Object writeValue(HiveDecimalWritable value) throws HiveException;
   Object writeValue(HiveDecimal value) throws HiveException;
+  Object writeValue(TimestampWritable value) throws HiveException;
+  Object writeValue(Timestamp value) throws HiveException;
+  Object writeValue(PisaTimestamp value) throws HiveException;
   Object setValue(Object row, ColumnVector column, int columnRow) throws HiveException;
   Object initValue(Object ost) throws HiveException;
 }