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 2018/05/14 07:10:13 UTC

hive git commit: Addendum to HIVE-19384: Vectorization: IfExprTimestamp* do not handle NULLs correctly

Repository: hive
Updated Branches:
  refs/heads/master f3beace2e -> 1542c88d5


Addendum to HIVE-19384: Vectorization: IfExprTimestamp* do not handle NULLs correctly


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/1542c88d
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/1542c88d
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/1542c88d

Branch: refs/heads/master
Commit: 1542c88d5ff9bb6a81491031918561d24dee59c6
Parents: f3beace
Author: Matt McCline <mm...@hortonworks.com>
Authored: Mon May 14 02:10:00 2018 -0500
Committer: Matt McCline <mm...@hortonworks.com>
Committed: Mon May 14 02:10:00 2018 -0500

----------------------------------------------------------------------
 .../IfExprIntervalDayTimeColumnColumn.java      | 178 ------------------
 .../IfExprIntervalDayTimeColumnScalar.java      | 164 ----------------
 .../IfExprIntervalDayTimeScalarColumn.java      | 185 -------------------
 .../IfExprIntervalDayTimeScalarScalar.java      | 170 -----------------
 .../IfExprTimestampColumnColumn.java            |  54 ------
 .../IfExprTimestampColumnColumnBase.java        | 148 ---------------
 .../IfExprTimestampColumnScalar.java            |  59 ------
 .../IfExprTimestampColumnScalarBase.java        | 143 --------------
 .../IfExprTimestampScalarColumn.java            |  59 ------
 .../IfExprTimestampScalarColumnBase.java        | 169 -----------------
 .../IfExprTimestampScalarScalar.java            |  59 ------
 .../IfExprTimestampScalarScalarBase.java        | 149 ---------------
 12 files changed, 1537 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/1542c88d/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
deleted file mode 100644
index 13e5fff..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprIntervalDayTimeColumnColumn.java
+++ /dev/null
@@ -1,178 +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;
-
-import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.IntervalDayTimeColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
-import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
-import org.apache.hadoop.hive.ql.metadata.HiveException;
-
-/**
- * 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 VectorExpression {
-
-  private static final long serialVersionUID = 1L;
-
-  private final int arg1Column;
-  private final int arg2Column;
-  private final int arg3Column;
-
-  public IfExprIntervalDayTimeColumnColumn(int arg1Column, int arg2Column, int arg3Column,
-      int outputColumnNum) {
-    super(outputColumnNum);
-    this.arg1Column = arg1Column;
-    this.arg2Column = arg2Column;
-    this.arg3Column = arg3Column;
-  }
-
-  public IfExprIntervalDayTimeColumnColumn() {
-    super();
-
-    // Dummy final assignments.
-    arg1Column = -1;
-    arg2Column = -1;
-    arg3Column = -1;
-  }
-
-  @Override
-  public void evaluate(VectorizedRowBatch batch) throws HiveException {
-
-    if (childExpressions != null) {
-      super.evaluateChildren(batch);
-    }
-
-    LongColumnVector arg1ColVector = (LongColumnVector) batch.cols[arg1Column];
-    IntervalDayTimeColumnVector arg2ColVector = (IntervalDayTimeColumnVector) batch.cols[arg2Column];
-    IntervalDayTimeColumnVector arg3ColVector = (IntervalDayTimeColumnVector) batch.cols[arg3Column];
-    IntervalDayTimeColumnVector outputColVector = (IntervalDayTimeColumnVector) batch.cols[outputColumnNum];
-    int[] sel = batch.selected;
-    boolean[] outputIsNull = outputColVector.isNull;
-
-    int n = batch.size;
-    long[] vector1 = arg1ColVector.vector;
-
-    // return immediately if batch is empty
-    if (n == 0) {
-      return;
-    }
-
-    // We do not need to do a column reset since we are carefully changing the output.
-    outputColVector.isRepeating = false;
-
-    /* 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 ((arg1ColVector.noNulls || !arg1ColVector.isNull[0]) && 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) {
-
-      // Carefully handle NULLs...
-
-      /*
-       * For better performance on LONG/DOUBLE we don't want the conditional
-       * statements inside the for loop.
-       */
-      outputColVector.noNulls = false;
-
-      if (batch.selectedInUse) {
-        for(int j = 0; j != n; j++) {
-          int i = sel[j];
-          outputIsNull[i] = (vector1[i] == 1 ?
-              arg2ColVector.isNull[i] : arg3ColVector.isNull[i]);
-          outputColVector.set(i, vector1[i] == 1 ? arg2ColVector.asScratchIntervalDayTime(i) : arg3ColVector.asScratchIntervalDayTime(i));
-        }
-      } else {
-        for(int i = 0; i != n; i++) {
-          outputIsNull[i] = (vector1[i] == 1 ?
-              arg2ColVector.isNull[i] : arg3ColVector.isNull[i]);
-          outputColVector.set(i, vector1[i] == 1 ? arg2ColVector.asScratchIntervalDayTime(i) : arg3ColVector.asScratchIntervalDayTime(i));
-        }
-      }
-    } else /* there are NULLs in the inputColVector */ {
-
-      // Carefully handle NULLs...
-
-      /*
-       * For better performance on LONG/DOUBLE we don't want the conditional
-       * statements inside the for loop.
-       */
-      outputColVector.noNulls = false;
-
-      if (batch.selectedInUse) {
-        for(int j = 0; j != n; j++) {
-          int i = sel[j];
-          outputColVector.set(i, !arg1ColVector.isNull[i] && vector1[i] == 1 ?
-              arg2ColVector.asScratchIntervalDayTime(i) : arg3ColVector.asScratchIntervalDayTime(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.asScratchIntervalDayTime(i) : arg3ColVector.asScratchIntervalDayTime(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 String vectorExpressionParameters() {
-    return getColumnParamString(0, arg1Column) + ", " + getColumnParamString(1, arg2Column) +
-        getColumnParamString(2, arg3Column);
-  }
-
-  @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/1542c88d/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
deleted file mode 100644
index d5fb6a0..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprIntervalDayTimeColumnScalar.java
+++ /dev/null
@@ -1,164 +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;
-
-import java.util.Arrays;
-
-import org.apache.hadoop.hive.common.type.HiveIntervalDayTime;
-import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.IntervalDayTimeColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
-import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
-import org.apache.hadoop.hive.ql.metadata.HiveException;
-
-/**
- * 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 VectorExpression {
-
-  private static final long serialVersionUID = 1L;
-
-  private final int arg1Column;
-  private final int arg2Column;
-  private final HiveIntervalDayTime arg3Scalar;
-
-  public IfExprIntervalDayTimeColumnScalar(int arg1Column, int arg2Column, HiveIntervalDayTime arg3Scalar,
-      int outputColumnNum) {
-    super(outputColumnNum);
-    this.arg1Column = arg1Column;
-    this.arg2Column = arg2Column;
-    this.arg3Scalar = arg3Scalar;
-  }
-
-  public IfExprIntervalDayTimeColumnScalar() {
-    super();
-
-    // Dummy final assignments.
-    arg1Column = -1;
-    arg2Column = -1;
-    arg3Scalar = null;
-  }
-
-  @Override
-  public void evaluate(VectorizedRowBatch batch) throws HiveException {
-
-    if (childExpressions != null) {
-      super.evaluateChildren(batch);
-    }
-
-    LongColumnVector arg1ColVector = (LongColumnVector) batch.cols[arg1Column];
-    IntervalDayTimeColumnVector arg2ColVector = (IntervalDayTimeColumnVector) batch.cols[arg2Column];
-    IntervalDayTimeColumnVector outputColVector = (IntervalDayTimeColumnVector) batch.cols[outputColumnNum];
-    int[] sel = batch.selected;
-    boolean[] outputIsNull = outputColVector.isNull;
-
-    // We do not need to do a column reset since we are carefully changing the output.
-    outputColVector.isRepeating = false;
-
-    int n = batch.size;
-    long[] vector1 = arg1ColVector.vector;
-
-    // return immediately if batch is empty
-    if (n == 0) {
-      return;
-    }
-
-    if (arg1ColVector.isRepeating) {
-      if ((arg1ColVector.noNulls || !arg1ColVector.isNull[0]) && 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];
-          outputIsNull[i] = false;
-          outputColVector.set(i, vector1[i] == 1 ? arg2ColVector.asScratchIntervalDayTime(i) : arg3Scalar);
-        }
-      } else {
-        Arrays.fill(outputIsNull, 0, n, false);
-        for(int i = 0; i != n; i++) {
-          outputColVector.set(i, vector1[i] == 1 ? arg2ColVector.asScratchIntervalDayTime(i) : arg3Scalar);
-        }
-      }
-    } else /* there are nulls */ {
-
-      // Carefully handle NULLs...
-
-      /*
-       * For better performance on LONG/DOUBLE we don't want the conditional
-       * statements inside the for loop.
-       */
-      outputColVector.noNulls = false;
-
-      if (batch.selectedInUse) {
-        for(int j = 0; j != n; j++) {
-          int i = sel[j];
-          outputColVector.set(i, !arg1ColVector.isNull[i] && vector1[i] == 1 ?
-              arg2ColVector.asScratchIntervalDayTime(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.asScratchIntervalDayTime(i) : arg3Scalar);
-          outputIsNull[i] = (!arg1ColVector.isNull[i] && vector1[i] == 1 ?
-              arg2ColVector.isNull[i] : false);
-        }
-      }
-    }
-
-    // restore repeating and no nulls indicators
-    arg2ColVector.unFlatten();
-  }
-
-  @Override
-  public String vectorExpressionParameters() {
-    return getColumnParamString(0, arg1Column) + ", " + getColumnParamString(1, arg2Column) +
-        ", val "+ arg3Scalar.toString();
-  }
-
-  @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/1542c88d/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
deleted file mode 100644
index 53466e5..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprIntervalDayTimeScalarColumn.java
+++ /dev/null
@@ -1,185 +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;
-
-import java.util.Arrays;
-
-import org.apache.hadoop.hive.common.type.HiveIntervalDayTime;
-import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.IntervalDayTimeColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
-import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
-import org.apache.hadoop.hive.ql.metadata.HiveException;
-
-/**
- * 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 VectorExpression {
-
-  private static final long serialVersionUID = 1L;
-
-  private final int arg1Column;
-  private final HiveIntervalDayTime arg2Scalar;
-  private final int arg3Column;
-
-  public IfExprIntervalDayTimeScalarColumn(int arg1Column, HiveIntervalDayTime arg2Scalar,
-      int arg3Column, int outputColumnNum) {
-    super(outputColumnNum);
-    this.arg1Column = arg1Column;
-    this.arg2Scalar = arg2Scalar;
-    this.arg3Column = arg3Column;
-  }
-
-  public IfExprIntervalDayTimeScalarColumn() {
-    super();
-
-    // Dummy final assignments.
-    arg1Column = -1;
-    arg2Scalar = null;
-    arg3Column = -1;
-  }
-
-  @Override
-  public void evaluate(VectorizedRowBatch batch) throws HiveException {
-
-    if (childExpressions != null) {
-      super.evaluateChildren(batch);
-    }
-
-    LongColumnVector arg1ColVector = (LongColumnVector) batch.cols[arg1Column];
-    IntervalDayTimeColumnVector arg3ColVector = (IntervalDayTimeColumnVector) batch.cols[arg3Column];
-    IntervalDayTimeColumnVector outputColVector = (IntervalDayTimeColumnVector) batch.cols[outputColumnNum];
-    int[] sel = batch.selected;
-    boolean[] outputIsNull = outputColVector.isNull;
-
-    // We do not need to do a column reset since we are carefully changing the output.
-    outputColVector.isRepeating = false;
-
-    int n = batch.size;
-    long[] vector1 = arg1ColVector.vector;
-
-    // return immediately if batch is empty
-    if (n == 0) {
-      return;
-    }
-
-    if (arg1ColVector.isRepeating) {
-      if ((arg1ColVector.noNulls || !arg1ColVector.isNull[0]) && 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) {
-
-        // CONSIDER: For large n, fill n or all of isNull array and use the tighter ELSE loop.
-
-        if (!outputColVector.noNulls) {
-          for(int j = 0; j != n; j++) {
-           final int i = sel[j];
-           // Set isNull before call in case it changes it mind.
-           outputIsNull[i] = false;
-           outputColVector.set(
-               i, vector1[i] == 1 ? arg2Scalar : arg3ColVector.asScratchIntervalDayTime(i));
-         }
-        } else {
-          for(int j = 0; j != n; j++) {
-            final int i = sel[j];
-            outputColVector.set(
-                i, vector1[i] == 1 ? arg2Scalar : arg3ColVector.asScratchIntervalDayTime(i));
-          }
-        }
-      } else {
-        if (!outputColVector.noNulls) {
-
-          // Assume it is almost always a performance win to fill all of isNull so we can
-          // safely reset noNulls.
-          Arrays.fill(outputIsNull, false);
-          outputColVector.noNulls = true;
-        }
-        for(int i = 0; i != n; i++) {
-          outputColVector.set(
-              i, vector1[i] == 1 ? arg2Scalar : arg3ColVector.asScratchIntervalDayTime(i));
-        }
-      }
-    } else /* there are nulls */ {
-
-      // Carefully handle NULLs...
-
-      /*
-       * For better performance on LONG/DOUBLE we don't want the conditional
-       * statements inside the for loop.
-       */
-      outputColVector.noNulls = false;
-
-      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.asScratchIntervalDayTime(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.asScratchIntervalDayTime(i));
-          outputIsNull[i] = (!arg1ColVector.isNull[i] && vector1[i] == 1 ?
-              false : arg3ColVector.isNull[i]);
-        }
-      }
-    }
-
-    // restore repeating and no nulls indicators
-    arg3ColVector.unFlatten();
-  }
-
-  @Override
-  public String vectorExpressionParameters() {
-    return getColumnParamString(0, arg1Column) + ", val "+ arg2Scalar + ", col "+ arg3Column;
-  }
-
-  @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/1542c88d/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
deleted file mode 100644
index 177f358..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprIntervalDayTimeScalarScalar.java
+++ /dev/null
@@ -1,170 +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;
-
-import java.util.Arrays;
-
-import org.apache.hadoop.hive.common.type.HiveIntervalDayTime;
-import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.IntervalDayTimeColumnVector;
-import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
-import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
-import org.apache.hadoop.hive.ql.metadata.HiveException;
-
-/**
- * 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 VectorExpression {
-
-  private static final long serialVersionUID = 1L;
-
-  private final int arg1Column;
-  private final HiveIntervalDayTime arg2Scalar;
-  private final HiveIntervalDayTime arg3Scalar;
-
-  public IfExprIntervalDayTimeScalarScalar(int arg1Column, HiveIntervalDayTime arg2Scalar,
-      HiveIntervalDayTime arg3Scalar, int outputColumnNum) {
-    super(outputColumnNum);
-    this.arg1Column = arg1Column;
-    this.arg2Scalar = arg2Scalar;
-    this.arg3Scalar = arg3Scalar;
-  }
-
-  public IfExprIntervalDayTimeScalarScalar() {
-    super();
-
-    // Dummy final assignments.
-    arg1Column = -1;
-    arg2Scalar = null;
-    arg3Scalar = null;
-  }
-
-  @Override
-  public void evaluate(VectorizedRowBatch batch) throws HiveException {
-
-    if (childExpressions != null) {
-      super.evaluateChildren(batch);
-    }
-
-    LongColumnVector arg1ColVector = (LongColumnVector) batch.cols[arg1Column];
-    IntervalDayTimeColumnVector outputColVector = (IntervalDayTimeColumnVector) batch.cols[outputColumnNum];
-    int[] sel = batch.selected;
-    boolean[] outputIsNull = outputColVector.isNull;
-
-    // We do not need to do a column reset since we are carefully changing the output.
-    outputColVector.isRepeating = false;
-
-    int n = batch.size;
-    long[] vector1 = arg1ColVector.vector;
-
-    // return immediately if batch is empty
-    if (n == 0) {
-      return;
-    }
-
-    if (arg1ColVector.isRepeating) {
-      if ((arg1ColVector.noNulls || !arg1ColVector.isNull[0]) && vector1[0] == 1) {
-        outputColVector.fill(arg2Scalar);
-      } else {
-        outputColVector.fill(arg3Scalar);
-      }
-      return;
-    }
-
-    if (arg1ColVector.noNulls) {
-      if (batch.selectedInUse) {
-
-        // CONSIDER: For large n, fill n or all of isNull array and use the tighter ELSE loop.
-
-        if (!outputColVector.noNulls) {
-          for(int j = 0; j != n; j++) {
-           final int i = sel[j];
-           // Set isNull before call in case it changes it mind.
-           outputIsNull[i] = false;
-           outputColVector.set(i, vector1[i] == 1 ? arg2Scalar : arg3Scalar);
-         }
-        } else {
-          for(int j = 0; j != n; j++) {
-            final int i = sel[j];
-            outputColVector.set(i, vector1[i] == 1 ? arg2Scalar : arg3Scalar);
-          }
-        }
-      } else {
-        if (!outputColVector.noNulls) {
-
-          // Assume it is almost always a performance win to fill all of isNull so we can
-          // safely reset noNulls.
-          Arrays.fill(outputIsNull, false);
-          outputColVector.noNulls = true;
-        }
-        for(int i = 0; i != n; i++) {
-          outputColVector.set(i, vector1[i] == 1 ? arg2Scalar : arg3Scalar);
-        }
-      }
-    } else /* there are nulls */ {
-
-      // Carefully handle NULLs...
-
-      /*
-       * For better performance on LONG/DOUBLE we don't want the conditional
-       * statements inside the for loop.
-       */
-      outputColVector.noNulls = false;
-
-      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 String vectorExpressionParameters() {
-    return getColumnParamString(0, arg1Column) + ", val "+ arg2Scalar + ", val "+ arg3Scalar;
-  }
-
-  @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/1542c88d/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
deleted file mode 100644
index 579eead..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprTimestampColumnColumn.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hive.ql.exec.vector.expressions;
-
-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 outputColumnNum) {
-    super(arg1Column, arg2Column, arg3Column, outputColumnNum);
-  }
-
-  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/1542c88d/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
deleted file mode 100644
index fc7a859..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprTimestampColumnColumnBase.java
+++ /dev/null
@@ -1,148 +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;
-
-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.metadata.HiveException;
-
-/**
- * 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 final int arg1Column;
-  private final int arg2Column;
-  private final int arg3Column;
-
-  public IfExprTimestampColumnColumnBase(int arg1Column, int arg2Column, int arg3Column,
-      int outputColumnNum) {
-    super(outputColumnNum);
-    this.arg1Column = arg1Column;
-    this.arg2Column = arg2Column;
-    this.arg3Column = arg3Column;
-  }
-
-  public IfExprTimestampColumnColumnBase() {
-    super();
-
-    // Dummy final assignments.
-    arg1Column = -1;
-    arg2Column = -1;
-    arg3Column = -1;
-  }
-
-  @Override
-  public void evaluate(VectorizedRowBatch batch) throws HiveException {
-
-    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[outputColumnNum];
-    int[] sel = batch.selected;
-    boolean[] outputIsNull = outputColVector.isNull;
-
-    // We do not need to do a column reset since we are carefully changing the output.
-    outputColVector.isRepeating = false;
-
-    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 ((arg1ColVector.noNulls || !arg1ColVector.isNull[0]) && 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);
-
-    /*
-     * Do careful maintenance of NULLs.
-     */
-    outputColVector.noNulls = false;
-
-    if (arg1ColVector.noNulls) {
-      if (batch.selectedInUse) {
-        for(int j = 0; j != n; j++) {
-          int i = sel[j];
-          outputColVector.set(i, vector1[i] == 1 ? arg2ColVector.asScratchTimestamp(i) : arg3ColVector.asScratchTimestamp(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.asScratchTimestamp(i) : arg3ColVector.asScratchTimestamp(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.asScratchTimestamp(i) : arg3ColVector.asScratchTimestamp(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.asScratchTimestamp(i) : arg3ColVector.asScratchTimestamp(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 String vectorExpressionParameters() {
-    return getColumnParamString(0, arg1Column) + ", " + getColumnParamString(1, arg2Column) +
-        getColumnParamString(2, arg3Column);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/1542c88d/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
deleted file mode 100644
index fcd7ca4..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprTimestampColumnScalar.java
+++ /dev/null
@@ -1,59 +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;
-
-import java.sql.Timestamp;
-
-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 outputColumnNum) {
-    super(arg1Column, arg2Column, arg3Scalar, outputColumnNum);
-  }
-
-  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/1542c88d/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
deleted file mode 100644
index 64add36..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprTimestampColumnScalarBase.java
+++ /dev/null
@@ -1,143 +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;
-
-import java.sql.Timestamp;
-import java.util.Arrays;
-
-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.metadata.HiveException;
-
-/**
- * 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 final int arg1Column;
-  private final int arg2Column;
-  private final Timestamp arg3Scalar;
-
-  public IfExprTimestampColumnScalarBase(int arg1Column, int arg2Column, Timestamp arg3Scalar,
-      int outputColumnNum) {
-    super(outputColumnNum);
-    this.arg1Column = arg1Column;
-    this.arg2Column = arg2Column;
-    this.arg3Scalar = arg3Scalar;
-  }
-
-  public IfExprTimestampColumnScalarBase() {
-    super();
-
-    // Dummy final assignments.
-    arg1Column = -1;
-    arg2Column = -1;
-    arg3Scalar = null;
-  }
-
-  @Override
-  public void evaluate(VectorizedRowBatch batch) throws HiveException {
-
-    if (childExpressions != null) {
-      super.evaluateChildren(batch);
-    }
-
-    LongColumnVector arg1ColVector = (LongColumnVector) batch.cols[arg1Column];
-    TimestampColumnVector arg2ColVector = (TimestampColumnVector) batch.cols[arg2Column];
-    TimestampColumnVector outputColVector = (TimestampColumnVector) batch.cols[outputColumnNum];
-    int[] sel = batch.selected;
-    boolean[] outputIsNull = outputColVector.isNull;
-
-    // We do not need to do a column reset since we are carefully changing the output.
-    outputColVector.isRepeating = false;
-
-    int n = batch.size;
-    long[] vector1 = arg1ColVector.vector;
-
-    // return immediately if batch is empty
-    if (n == 0) {
-      return;
-    }
-
-    if (arg1ColVector.isRepeating) {
-      if ((arg1ColVector.noNulls || !arg1ColVector.isNull[0]) && 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);
-
-    /*
-     * Since we always set a value, make sure all isNull entries are set to false.
-     */
-
-    if (arg1ColVector.noNulls) {
-      if (batch.selectedInUse) {
-        for(int j = 0; j != n; j++) {
-          int i = sel[j];
-          outputIsNull[i] = false;
-          outputColVector.set(i, vector1[i] == 1 ? arg2ColVector.asScratchTimestamp(i) : arg3Scalar);
-        }
-      } else {
-        Arrays.fill(outputIsNull, 0, n, false);
-        for(int i = 0; i != n; i++) {
-          outputColVector.set(i, vector1[i] == 1 ? arg2ColVector.asScratchTimestamp(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.asScratchTimestamp(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.asScratchTimestamp(i) : arg3Scalar);
-          outputIsNull[i] = (!arg1ColVector.isNull[i] && vector1[i] == 1 ?
-              arg2ColVector.isNull[i] : false);
-        }
-      }
-    }
-
-    // restore repeating and no nulls indicators
-    arg2ColVector.unFlatten();
-  }
-
-  @Override
-  public String vectorExpressionParameters() {
-    return getColumnParamString(0, arg1Column) + ", " + getColumnParamString(1, arg2Column) +
-        ", val "+ arg3Scalar;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/1542c88d/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
deleted file mode 100644
index 510774a..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprTimestampScalarColumn.java
+++ /dev/null
@@ -1,59 +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;
-
-import java.sql.Timestamp;
-
-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 outputColumnNum) {
-    super(arg1Column, arg2Scalar, arg3Column, outputColumnNum);
-  }
-
-  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/1542c88d/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
deleted file mode 100644
index 73044ad..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprTimestampScalarColumnBase.java
+++ /dev/null
@@ -1,169 +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;
-
-import java.sql.Timestamp;
-import java.util.Arrays;
-
-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;
-import org.apache.hadoop.hive.ql.metadata.HiveException;
-
-/**
- * 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 final int arg1Column;
-  private Timestamp arg2Scalar;
-  private final int arg3Column;
-
-  public IfExprTimestampScalarColumnBase(int arg1Column, Timestamp arg2Scalar, int arg3Column,
-      int outputColumnNum) {
-    super(outputColumnNum);
-    this.arg1Column = arg1Column;
-    this.arg2Scalar = arg2Scalar;
-    this.arg3Column = arg3Column;
-  }
-
-  public IfExprTimestampScalarColumnBase() {
-    super();
-
-    // Dummy final assignments.
-    arg1Column = -1;
-    arg2Scalar = null;
-    arg3Column = -1;
-  }
-
-  @Override
-  public void evaluate(VectorizedRowBatch batch) throws HiveException {
-
-    if (childExpressions != null) {
-      super.evaluateChildren(batch);
-    }
-
-    LongColumnVector arg1ColVector = (LongColumnVector) batch.cols[arg1Column];
-    TimestampColumnVector arg3ColVector = (TimestampColumnVector) batch.cols[arg3Column];
-    TimestampColumnVector outputColVector = (TimestampColumnVector) batch.cols[outputColumnNum];
-    int[] sel = batch.selected;
-    boolean[] outputIsNull = outputColVector.isNull;
-
-    // We do not need to do a column reset since we are carefully changing the output.
-    outputColVector.isRepeating = false;
-
-    int n = batch.size;
-    long[] vector1 = arg1ColVector.vector;
-
-    // return immediately if batch is empty
-    if (n == 0) {
-      return;
-    }
-
-    if (arg1ColVector.isRepeating) {
-      if ((arg1ColVector.noNulls || !arg1ColVector.isNull[0]) && 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) {
-
-        // CONSIDER: For large n, fill n or all of isNull array and use the tighter ELSE loop.
-
-        if (!outputColVector.noNulls) {
-          for(int j = 0; j != n; j++) {
-           final int i = sel[j];
-           // Set isNull before call in case it changes it mind.
-           outputIsNull[i] = false;
-           outputColVector.set(
-               i, vector1[i] == 1 ? arg2Scalar : arg3ColVector.asScratchTimestamp(i));
-         }
-        } else {
-          for(int j = 0; j != n; j++) {
-            final int i = sel[j];
-            outputColVector.set(
-                i, vector1[i] == 1 ? arg2Scalar : arg3ColVector.asScratchTimestamp(i));
-          }
-        }
-      } else {
-        if (!outputColVector.noNulls) {
-
-          // Assume it is almost always a performance win to fill all of isNull so we can
-          // safely reset noNulls.
-          Arrays.fill(outputIsNull, false);
-          outputColVector.noNulls = true;
-        }
-        for(int i = 0; i != n; i++) {
-          outputColVector.set(
-              i, vector1[i] == 1 ? arg2Scalar : arg3ColVector.asScratchTimestamp(i));
-        }
-      }
-    } else /* there are nulls */ {
-
-      /*
-       * Do careful maintenance of NULLs.
-       */
-      outputColVector.noNulls = false;
-
-      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.asScratchTimestamp(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.asScratchTimestamp(i));
-          outputIsNull[i] = (!arg1ColVector.isNull[i] && vector1[i] == 1 ?
-              false : arg3ColVector.isNull[i]);
-        }
-      }
-    }
-
-    // restore repeating and no nulls indicators
-    arg3ColVector.unFlatten();
-  }
-
-  @Override
-  public String vectorExpressionParameters() {
-    return getColumnParamString(0, arg1Column) + ", val "+ arg2Scalar + ", " +
-        getColumnParamString(2, arg3Column);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/1542c88d/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
deleted file mode 100644
index 0c4cce0..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprTimestampScalarScalar.java
+++ /dev/null
@@ -1,59 +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;
-
-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 outputColumnNum) {
-    super(arg1Column, arg2Scalar, arg3Scalar, outputColumnNum);
-  }
-
-  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/1542c88d/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
deleted file mode 100644
index df1418e..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/IfExprTimestampScalarScalarBase.java
+++ /dev/null
@@ -1,149 +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;
-
-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.metadata.HiveException;
-
-import java.sql.Timestamp;
-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 final int arg1Column;
-  private final Timestamp arg2Scalar;
-  private final Timestamp arg3Scalar;
-
-  public IfExprTimestampScalarScalarBase(int arg1Column, Timestamp arg2Scalar, Timestamp arg3Scalar,
-      int outputColumnNum) {
-    super(outputColumnNum);
-    this.arg1Column = arg1Column;
-    this.arg2Scalar = arg2Scalar;
-    this.arg3Scalar = arg3Scalar;
-  }
-
-  public IfExprTimestampScalarScalarBase() {
-    super();
-
-    // Dummy final assignments.
-    arg1Column = -1;
-    arg2Scalar = null;
-    arg3Scalar = null;
-  }
-
-  @Override
-  public void evaluate(VectorizedRowBatch batch) throws HiveException {
-
-    if (childExpressions != null) {
-      super.evaluateChildren(batch);
-    }
-
-    LongColumnVector arg1ColVector = (LongColumnVector) batch.cols[arg1Column];
-    TimestampColumnVector outputColVector = (TimestampColumnVector) batch.cols[outputColumnNum];
-    int[] sel = batch.selected;
-    boolean[] outputIsNull = outputColVector.isNull;
-
-    // We do not need to do a column reset since we are carefully changing the output.
-    outputColVector.isRepeating = false;
-
-    int n = batch.size;
-    long[] vector1 = arg1ColVector.vector;
-
-    // return immediately if batch is empty
-    if (n == 0) {
-      return;
-    }
-
-    if (arg1ColVector.isRepeating) {
-      if ((arg1ColVector.noNulls || !arg1ColVector.isNull[0]) && vector1[0] == 1) {
-        outputColVector.fill(arg2Scalar);
-      } else {
-        outputColVector.fill(arg3Scalar);
-      }
-      return;
-    }
-
-    /*
-     * Since we always set a value, make sure all isNull entries are set to false.
-     */
-
-    if (arg1ColVector.noNulls) {
-      if (batch.selectedInUse) {
-
-        // CONSIDER: For large n, fill n or all of isNull array and use the tighter ELSE loop.
-
-        if (!outputColVector.noNulls) {
-          for(int j = 0; j != n; j++) {
-           final int i = sel[j];
-           // Set isNull before call in case it changes it mind.
-           outputIsNull[i] = false;
-           outputColVector.set(i, vector1[i] == 1 ? arg2Scalar : arg3Scalar);
-         }
-        } else {
-          for(int j = 0; j != n; j++) {
-            final int i = sel[j];
-            outputColVector.set(i, vector1[i] == 1 ? arg2Scalar : arg3Scalar);
-          }
-        }
-      } else {
-        if (!outputColVector.noNulls) {
-
-          // Assume it is almost always a performance win to fill all of isNull so we can
-          // safely reset noNulls.
-          Arrays.fill(outputIsNull, false);
-          outputColVector.noNulls = true;
-        }
-        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];
-          outputIsNull[i] = false;
-          outputColVector.set(i, !arg1ColVector.isNull[i] && vector1[i] == 1 ?
-              arg2Scalar : arg3Scalar);
-        }
-      } else {
-        Arrays.fill(outputIsNull, 0, n, false);
-        for(int i = 0; i != n; i++) {
-          outputColVector.set(i, !arg1ColVector.isNull[i] && vector1[i] == 1 ?
-              arg2Scalar : arg3Scalar);
-        }
-      }
-    }
-  }
-
-  @Override
-  public String vectorExpressionParameters() {
-    return getColumnParamString(0, arg1Column) + ", val "+ arg2Scalar + ", val "+ arg3Scalar;
-  }
-}