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 06:57:23 UTC

[4/4] hive git commit: HIVE-19384: Vectorization: IfExprTimestamp* do not handle NULLs correctly (Matt McCline, reviewed by Teddy Choi)

HIVE-19384: Vectorization: IfExprTimestamp* do not handle NULLs correctly (Matt McCline, reviewed by Teddy Choi)


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

Branch: refs/heads/master
Commit: f3beace2ebfa989cc1d8a4f491cb705bf58ecd82
Parents: f327624
Author: Matt McCline <mm...@hortonworks.com>
Authored: Mon May 14 01:57:12 2018 -0500
Committer: Matt McCline <mm...@hortonworks.com>
Committed: Mon May 14 01:57:12 2018 -0500

----------------------------------------------------------------------
 .../IfExprObjectColumnColumn.txt                | 217 +++++++++
 .../IfExprObjectColumnScalar.txt                | 194 ++++++++
 .../IfExprObjectScalarColumn.txt                | 196 ++++++++
 .../IfExprObjectScalarScalar.txt                | 166 +++++++
 .../ql/exec/vector/VectorizationContext.java    |   5 -
 .../expressions/IfExprLongColumnLongColumn.java |   7 +-
 .../hive/ql/udf/generic/GenericUDFIf.java       |  16 +-
 .../exec/vector/TestVectorizationContext.java   |   8 +-
 .../ql/exec/vector/VectorRandomBatchSource.java | 311 +++++++++++++
 .../ql/exec/vector/VectorRandomRowSource.java   | 312 +++++++++----
 .../expressions/TestVectorIfStatement.java      | 444 +++++++++++++++++++
 .../clientpositive/vectorized_timestamp_funcs.q |  49 +-
 .../llap/vectorized_timestamp_funcs.q.out       | 384 ++++++++++++----
 .../spark/vectorized_timestamp_funcs.q.out      | 384 ++++++++++++----
 .../vectorized_timestamp_funcs.q.out            | 382 ++++++++++++----
 .../hive/ql/exec/vector/VectorizedRowBatch.java |  91 ++--
 .../ql/exec/vector/TestStructColumnVector.java  |   4 +-
 .../apache/hadoop/hive/tools/GenVectorCode.java |  55 +++
 18 files changed, 2814 insertions(+), 411 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/f3beace2/ql/src/gen/vectorization/ExpressionTemplates/IfExprObjectColumnColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/IfExprObjectColumnColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/IfExprObjectColumnColumn.txt
new file mode 100644
index 0000000..e8ef279
--- /dev/null
+++ b/ql/src/gen/vectorization/ExpressionTemplates/IfExprObjectColumnColumn.txt
@@ -0,0 +1,217 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.ql.exec.vector.expressions.gen;
+
+import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.<ColumnVectorType>;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.ql.exec.vector.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 <ClassName> extends VectorExpression {
+
+  private static final long serialVersionUID = 1L;
+
+  private final int arg1Column;
+  private final int arg2Column;
+  private final int arg3Column;
+
+  public <ClassName>(int arg1Column, int arg2Column, int arg3Column,
+      int outputColumnNum) {
+    super(outputColumnNum);
+    this.arg1Column = arg1Column;
+    this.arg2Column = arg2Column;
+    this.arg3Column = arg3Column;
+  }
+
+  public <ClassName>() {
+    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];
+    <ColumnVectorType> arg2ColVector = (<ColumnVectorType>) batch.cols[arg2Column];
+    boolean[] arg2IsNull = arg2ColVector.isNull;
+    <ColumnVectorType> arg3ColVector = (<ColumnVectorType>) batch.cols[arg3Column];
+    boolean[] arg3IsNull = arg3ColVector.isNull;
+    <ColumnVectorType> outputColVector = (<ColumnVectorType>) 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);
+
+    if (arg1ColVector.noNulls) {
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          if (vector1[i] == 1) {
+            if (!arg2IsNull[i]) {
+              outputIsNull[i] = false;
+              outputColVector.set(i, arg2ColVector.asScratch<ObjectName>(i));
+            } else {
+              outputIsNull[i] = true;
+              outputColVector.noNulls = false;
+            }
+          } else {
+            if (!arg3IsNull[i]) {
+              outputIsNull[i] = false;
+              outputColVector.set(i, arg3ColVector.asScratch<ObjectName>(i));
+            } else {
+              outputIsNull[i] = true;
+              outputColVector.noNulls = false;
+            }
+          }
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          if (vector1[i] == 1) {
+            if (!arg2IsNull[i]) {
+              outputIsNull[i] = false;
+              outputColVector.set(i, arg2ColVector.asScratch<ObjectName>(i));
+            } else {
+              outputIsNull[i] = true;
+              outputColVector.noNulls = false;
+            }
+          } else {
+            if (!arg3IsNull[i]) {
+              outputIsNull[i] = false;
+              outputColVector.set(i, arg3ColVector.asScratch<ObjectName>(i));
+            } else {
+              outputIsNull[i] = true;
+              outputColVector.noNulls = false;
+            }
+          }
+        }
+      }
+    } else /* there are nulls */ {
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          if (!arg1ColVector.isNull[i] && vector1[i] == 1) {
+            if (!arg2IsNull[i]) {
+              outputIsNull[i] = false;
+              outputColVector.set(i, arg2ColVector.asScratch<ObjectName>(i));
+            } else {
+              outputIsNull[i] = true;
+              outputColVector.noNulls = false;
+            }
+          } else {
+            if (!arg3IsNull[i]) {
+              outputIsNull[i] = false;
+              outputColVector.set(i, arg3ColVector.asScratch<ObjectName>(i));
+            } else {
+              outputIsNull[i] = true;
+              outputColVector.noNulls = false;
+            }
+          }
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          if (!arg1ColVector.isNull[i] && vector1[i] == 1) {
+            if (!arg2IsNull[i]) {
+              outputIsNull[i] = false;
+              outputColVector.set(i, arg2ColVector.asScratch<ObjectName>(i));
+            } else {
+              outputIsNull[i] = true;
+              outputColVector.noNulls = false;
+            }
+          } else {
+            if (!arg3IsNull[i]) {
+              outputIsNull[i] = false;
+              outputColVector.set(i, arg3ColVector.asScratch<ObjectName>(i));
+            } else {
+              outputIsNull[i] = true;
+              outputColVector.noNulls = false;
+            }
+          }
+        }
+      }
+    }
+
+    // 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("<TypeName>"),
+            VectorExpressionDescriptor.ArgumentType.getType("<TypeName>"))
+        .setInputExpressionTypes(
+            VectorExpressionDescriptor.InputExpressionType.COLUMN,
+            VectorExpressionDescriptor.InputExpressionType.COLUMN,
+            VectorExpressionDescriptor.InputExpressionType.COLUMN).build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/f3beace2/ql/src/gen/vectorization/ExpressionTemplates/IfExprObjectColumnScalar.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/IfExprObjectColumnScalar.txt b/ql/src/gen/vectorization/ExpressionTemplates/IfExprObjectColumnScalar.txt
new file mode 100644
index 0000000..56ae2ca
--- /dev/null
+++ b/ql/src/gen/vectorization/ExpressionTemplates/IfExprObjectColumnScalar.txt
@@ -0,0 +1,194 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.exec.vector.expressions.gen;
+
+import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.<ColumnVectorType>;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import <ScalarImport>;
+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 <ClassName> extends VectorExpression {
+
+  private static final long serialVersionUID = 1L;
+
+  private final int arg1Column;
+  private final int arg2Column;
+  private final <ScalarType> arg3Scalar;
+
+  public <ClassName>(int arg1Column, int arg2Column, <ScalarType> arg3Scalar,
+      int outputColumnNum) {
+    super(outputColumnNum);
+    this.arg1Column = arg1Column;
+    this.arg2Column = arg2Column;
+    this.arg3Scalar = arg3Scalar;
+  }
+
+  public <ClassName>() {
+    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];
+    <ColumnVectorType> arg2ColVector = (<ColumnVectorType>) batch.cols[arg2Column];
+    boolean[] arg2IsNull = arg2ColVector.isNull;
+    <ColumnVectorType> outputColVector = (<ColumnVectorType>) 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) {
+
+      // FUTURE: We could check arg2ColVector.noNulls and optimize these loops.
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          if (vector1[i] == 1) {
+            if (!arg2IsNull[i]) {
+              outputIsNull[i] = false;
+              outputColVector.set(i, arg2ColVector.asScratch<ObjectName>(i));
+            } else {
+              outputIsNull[i] = true;
+              outputColVector.noNulls = false;
+            }
+          } else {
+            outputIsNull[i] = false;
+            outputColVector.set(i, arg3Scalar);
+          }
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          if (vector1[i] == 1) {
+            if (!arg2IsNull[i]) {
+              outputIsNull[i] = false;
+              outputColVector.set(i, arg2ColVector.asScratch<ObjectName>(i));
+            } else {
+              outputIsNull[i] = true;
+              outputColVector.noNulls = false;
+            }
+          } else {
+            outputIsNull[i] = false;
+            outputColVector.set(i, arg3Scalar);
+          }
+        }
+      }
+    } else /* there are nulls */ {
+
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          if (!arg1ColVector.isNull[i] && vector1[i] == 1) {
+            if (!arg2IsNull[i]) {
+              outputIsNull[i] = false;
+              outputColVector.set(i, arg2ColVector.asScratch<ObjectName>(i));
+            } else {
+              outputIsNull[i] = true;
+              outputColVector.noNulls = false;
+            }
+          } else {
+            outputIsNull[i] = false;
+            outputColVector.set(i, arg3Scalar);
+          }
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          if (!arg1ColVector.isNull[i] && vector1[i] == 1) {
+            if (!arg2IsNull[i]) {
+              outputIsNull[i] = false;
+              outputColVector.set(i, arg2ColVector.asScratch<ObjectName>(i));
+            } else {
+              outputIsNull[i] = true;
+              outputColVector.noNulls = false;
+            }
+          } else {
+            outputIsNull[i] = false;
+            outputColVector.set(i, arg3Scalar);
+          }
+        }
+      }
+    }
+
+    // restore repeating and no nulls indicators
+    arg2ColVector.unFlatten();
+  }
+
+  @Override
+  public String vectorExpressionParameters() {
+    return getColumnParamString(0, arg1Column) + ", " + getColumnParamString(1, arg2Column) +
+        ", 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("<TypeName>"),
+            VectorExpressionDescriptor.ArgumentType.getType("<TypeName>"))
+        .setInputExpressionTypes(
+            VectorExpressionDescriptor.InputExpressionType.COLUMN,
+            VectorExpressionDescriptor.InputExpressionType.COLUMN,
+            VectorExpressionDescriptor.InputExpressionType.SCALAR).build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/f3beace2/ql/src/gen/vectorization/ExpressionTemplates/IfExprObjectScalarColumn.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/IfExprObjectScalarColumn.txt b/ql/src/gen/vectorization/ExpressionTemplates/IfExprObjectScalarColumn.txt
new file mode 100644
index 0000000..271b589
--- /dev/null
+++ b/ql/src/gen/vectorization/ExpressionTemplates/IfExprObjectScalarColumn.txt
@@ -0,0 +1,196 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.exec.vector.expressions.gen;
+
+import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.<ColumnVectorType>;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import <ScalarImport>;
+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 <ClassName> extends VectorExpression {
+
+  private static final long serialVersionUID = 1L;
+
+  private final int arg1Column;
+  private <ScalarType> arg2Scalar;
+  private final int arg3Column;
+
+  public <ClassName>(int arg1Column, <ScalarType> arg2Scalar, int arg3Column,
+      int outputColumnNum) {
+    super(outputColumnNum);
+    this.arg1Column = arg1Column;
+    this.arg2Scalar = arg2Scalar;
+    this.arg3Column = arg3Column;
+  }
+
+  public <ClassName>() {
+    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];
+    <ColumnVectorType> arg3ColVector = (<ColumnVectorType>) batch.cols[arg3Column];
+    boolean[] arg3IsNull = arg3ColVector.isNull;
+    <ColumnVectorType> outputColVector = (<ColumnVectorType>) 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) {
+
+      // FUTURE: We could check arg3ColVector.noNulls and optimize these loops.
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          if (vector1[i] == 1) {
+            outputIsNull[i] = false;
+            outputColVector.set(i, arg2Scalar);
+          } else {
+            if (!arg3IsNull[i]) {
+              outputIsNull[i] = false;
+              outputColVector.set(i, arg3ColVector.asScratch<ObjectName>(i));
+            } else {
+              outputIsNull[i] = true;
+              outputColVector.noNulls = false;
+            }
+          }
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          if (vector1[i] == 1) {
+            outputIsNull[i] = false;
+            outputColVector.set(i, arg2Scalar);
+          } else {
+            if (!arg3IsNull[i]) {
+              outputIsNull[i] = false;
+              outputColVector.set(i, arg3ColVector.asScratch<ObjectName>(i));
+            } else {
+              outputIsNull[i] = true;
+              outputColVector.noNulls = false;
+            }
+          }
+        }
+      }
+    } else /* there are nulls */ {
+
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          if (!arg1ColVector.isNull[i] && vector1[i] == 1) {
+            outputIsNull[i] = false;
+            outputColVector.set(i, arg2Scalar);
+          } else {
+            if (!arg3IsNull[i]) {
+              outputIsNull[i] = false;
+              outputColVector.set(i, arg3ColVector.asScratch<ObjectName>(i));
+            } else {
+              outputIsNull[i] = true;
+              outputColVector.noNulls = false;
+            }
+          }
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          if (!arg1ColVector.isNull[i] && vector1[i] == 1) {
+            outputIsNull[i] = false;
+            outputColVector.set(i, arg2Scalar);
+          } else {
+            if (!arg3IsNull[i]) {
+              outputIsNull[i] = false;
+              outputColVector.set(i, arg3ColVector.asScratch<ObjectName>(i));
+            } else {
+              outputIsNull[i] = true;
+              outputColVector.noNulls = false;
+            }
+          }
+        }
+      }
+    }
+
+    // restore repeating and no nulls indicators
+    arg3ColVector.unFlatten();
+  }
+
+  @Override
+  public String vectorExpressionParameters() {
+    return getColumnParamString(0, arg1Column) + ", val "+ arg2Scalar + ", " +
+        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("<TypeName>"),
+            VectorExpressionDescriptor.ArgumentType.getType("<TypeName>"))
+        .setInputExpressionTypes(
+            VectorExpressionDescriptor.InputExpressionType.COLUMN,
+            VectorExpressionDescriptor.InputExpressionType.SCALAR,
+            VectorExpressionDescriptor.InputExpressionType.COLUMN).build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/f3beace2/ql/src/gen/vectorization/ExpressionTemplates/IfExprObjectScalarScalar.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/ExpressionTemplates/IfExprObjectScalarScalar.txt b/ql/src/gen/vectorization/ExpressionTemplates/IfExprObjectScalarScalar.txt
new file mode 100644
index 0000000..10f97df
--- /dev/null
+++ b/ql/src/gen/vectorization/ExpressionTemplates/IfExprObjectScalarScalar.txt
@@ -0,0 +1,166 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.exec.vector.expressions.gen;
+
+import java.util.Arrays;
+
+import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.<ColumnVectorType>;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import <ScalarImport>;
+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 <ClassName> extends VectorExpression {
+
+  private static final long serialVersionUID = 1L;
+
+  private final int arg1Column;
+  private final <ScalarType> arg2Scalar;
+  private final <ScalarType> arg3Scalar;
+
+  public <ClassName>(int arg1Column, <ScalarType> arg2Scalar, <ScalarType> arg3Scalar,
+      int outputColumnNum) {
+    super(outputColumnNum);
+    this.arg1Column = arg1Column;
+    this.arg2Scalar = arg2Scalar;
+    this.arg3Scalar = arg3Scalar;
+  }
+
+  public <ClassName>() {
+    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];
+    <ColumnVectorType> outputColVector = (<ColumnVectorType>) 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;
+  }
+
+  @Override
+  public VectorExpressionDescriptor.Descriptor getDescriptor() {
+    return (new VectorExpressionDescriptor.Builder())
+        .setMode(
+            VectorExpressionDescriptor.Mode.PROJECTION)
+        .setNumArguments(3)
+        .setArgumentTypes(
+            VectorExpressionDescriptor.ArgumentType.getType("int_family"),
+            VectorExpressionDescriptor.ArgumentType.getType("<TypeName>"),
+            VectorExpressionDescriptor.ArgumentType.getType("<TypeName>"))
+        .setInputExpressionTypes(
+            VectorExpressionDescriptor.InputExpressionType.COLUMN,
+            VectorExpressionDescriptor.InputExpressionType.SCALAR,
+            VectorExpressionDescriptor.InputExpressionType.SCALAR).build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/f3beace2/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java
index 2cad04b..491a6b1 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java
@@ -390,9 +390,6 @@ public class VectorizationContext {
     if (initialDataTypePhysicalVariations == null) {
       return null;
     }
-    if (columnNum < 0) {
-      fake++;
-    }
     if (columnNum < initialDataTypePhysicalVariations.size()) {
       return initialDataTypePhysicalVariations.get(columnNum);
     }
@@ -1682,8 +1679,6 @@ public class VectorizationContext {
     return vectorExpression;
   }
 
-  static int fake = 0;
-
   private VectorExpression getVectorExpressionForUdf(GenericUDF genericUdf,
       Class<?> udfClass, List<ExprNodeDesc> childExpr, VectorExpressionDescriptor.Mode mode,
       TypeInfo returnType) throws HiveException {

http://git-wip-us.apache.org/repos/asf/hive/blob/f3beace2/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 75de7a0..2a10e29 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
@@ -60,6 +60,7 @@ public class IfExprLongColumnLongColumn extends VectorExpression {
     }
 
     LongColumnVector arg1ColVector = (LongColumnVector) batch.cols[arg1Column];
+    boolean[] arg1IsNull = arg1ColVector.isNull;
     LongColumnVector arg2ColVector = (LongColumnVector) batch.cols[arg2Column];
     LongColumnVector arg3ColVector = (LongColumnVector) batch.cols[arg3Column];
     LongColumnVector outputColVector = (LongColumnVector) batch.cols[outputColumnNum];
@@ -87,7 +88,7 @@ public class IfExprLongColumnLongColumn extends VectorExpression {
      * of code paths.
      */
     if (arg1ColVector.isRepeating) {
-      if (vector1[0] == 1) {
+      if ((arg1ColVector.noNulls || !arg1IsNull[0]) && vector1[0] == 1) {
         arg2ColVector.copySelected(batch.selectedInUse, sel, n, outputColVector);
       } else {
         arg3ColVector.copySelected(batch.selectedInUse, sel, n, outputColVector);
@@ -121,14 +122,14 @@ public class IfExprLongColumnLongColumn extends VectorExpression {
       if (batch.selectedInUse) {
         for(int j = 0; j != n; j++) {
           int i = sel[j];
-          outputVector[i] = (!arg1ColVector.isNull[i] && vector1[i] == 1 ?
+          outputVector[i] = (!arg1IsNull[i] && vector1[i] == 1 ?
               vector2[i] : vector3[i]);
           outputIsNull[i] = (!arg1ColVector.isNull[i] && vector1[i] == 1 ?
               arg2ColVector.isNull[i] : arg3ColVector.isNull[i]);
         }
       } else {
         for(int i = 0; i != n; i++) {
-          outputVector[i] = (!arg1ColVector.isNull[i] && vector1[i] == 1 ?
+          outputVector[i] = (!arg1IsNull[i] && vector1[i] == 1 ?
               vector2[i] : vector3[i]);
           outputIsNull[i] = (!arg1ColVector.isNull[i] && vector1[i] == 1 ?
               arg2ColVector.isNull[i] : arg3ColVector.isNull[i]);

http://git-wip-us.apache.org/repos/asf/hive/blob/f3beace2/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFIf.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFIf.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFIf.java
index 90ff765..142dd1b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFIf.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFIf.java
@@ -41,10 +41,10 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.IfExprDoubleScalarD
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.IfExprDoubleScalarLongScalar;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.IfExprLongScalarDoubleScalar;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.IfExprDoubleColumnDoubleColumn;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.IfExprIntervalDayTimeColumnColumn;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.IfExprIntervalDayTimeColumnScalar;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.IfExprIntervalDayTimeScalarColumn;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.IfExprIntervalDayTimeScalarScalar;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.IfExprIntervalDayTimeColumnColumn;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.IfExprIntervalDayTimeColumnScalar;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.IfExprIntervalDayTimeScalarColumn;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.IfExprIntervalDayTimeScalarScalar;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.IfExprLongColumnLongColumn;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.IfExprStringGroupColumnStringGroupColumn;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.IfExprStringGroupColumnStringScalar;
@@ -52,10 +52,10 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.IfExprStringGroupColumn
 import org.apache.hadoop.hive.ql.exec.vector.expressions.IfExprStringGroupColumnVarCharScalar;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.IfExprStringScalarStringGroupColumn;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.IfExprCharScalarStringGroupColumn;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.IfExprTimestampColumnColumn;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.IfExprTimestampColumnScalar;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.IfExprTimestampScalarColumn;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.IfExprTimestampScalarScalar;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.IfExprTimestampColumnColumn;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.IfExprTimestampColumnScalar;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.IfExprTimestampScalarColumn;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.IfExprTimestampScalarScalar;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.IfExprVarCharScalarStringGroupColumn;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.IfExprStringScalarStringScalar;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.IfExprStringScalarCharScalar;

http://git-wip-us.apache.org/repos/asf/hive/blob/f3beace2/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorizationContext.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorizationContext.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorizationContext.java
index 551bb9e..791ac82 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorizationContext.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorizationContext.java
@@ -47,10 +47,10 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.IfExprStringGroupColumn
 import org.apache.hadoop.hive.ql.exec.vector.expressions.IfExprStringGroupColumnVarCharScalar;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.IfExprStringScalarStringGroupColumn;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.IfExprStringScalarStringScalar;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.IfExprTimestampColumnColumn;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.IfExprTimestampColumnScalar;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.IfExprTimestampScalarColumn;
-import org.apache.hadoop.hive.ql.exec.vector.expressions.IfExprTimestampScalarScalar;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.IfExprTimestampColumnColumn;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.IfExprTimestampColumnScalar;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.IfExprTimestampScalarColumn;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.IfExprTimestampScalarScalar;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.IfExprVarCharScalarStringGroupColumn;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.IsNotNull;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.IsNull;

http://git-wip-us.apache.org/repos/asf/hive/blob/f3beace2/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/VectorRandomBatchSource.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/VectorRandomBatchSource.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/VectorRandomBatchSource.java
new file mode 100644
index 0000000..8de247c
--- /dev/null
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/VectorRandomBatchSource.java
@@ -0,0 +1,311 @@
+/*
+ * 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;
+
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.List;
+import java.util.Random;
+
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+
+
+/**
+ * Generate random batch source from a random Object[] row source (VectorRandomRowSource).
+ */
+public class VectorRandomBatchSource {
+
+  // Divide up rows array into different sized batches.
+  // Modify the rows array for isRepeating / NULL patterns.
+  // Provide iterator that will fill up a VRB with the divided up rows.
+
+  private final VectorRandomRowSource vectorRandomRowSource;
+
+  private final Object[][] randomRows;
+
+  private final int rowCount;
+  private final int columnCount;
+
+  private final VectorBatchPatterns vectorBatchPatterns;
+
+  private VectorAssignRow vectorAssignRow;
+
+  private int nextRowIndex;
+  private int batchCount;
+
+  private VectorRandomBatchSource(
+      VectorRandomRowSource vectorRandomRowSource,
+      Object[][] randomRows,
+      VectorBatchPatterns vectorBatchPatterns,
+      VectorAssignRow vectorAssignRow) {
+    this.vectorRandomRowSource = vectorRandomRowSource;
+    this.randomRows = randomRows;
+    rowCount = randomRows.length;
+    Object[] firstRow = randomRows[0];
+    columnCount = firstRow.length;
+    this.vectorBatchPatterns = vectorBatchPatterns;
+    this.vectorAssignRow = vectorAssignRow;
+  }
+
+  public static class VectorRandomBatchParameters {
+  }
+
+  private static class VectorBatchPatterns {
+
+    private final List<VectorBatchPattern> vectorBatchPatternList;
+
+    VectorBatchPatterns(List<VectorBatchPattern> vectorBatchPatternList) {
+      this.vectorBatchPatternList = vectorBatchPatternList;
+    }
+
+    List<VectorBatchPattern> getTectorBatchPatternList() {
+      return vectorBatchPatternList;
+    }
+  }
+
+  private static class VectorBatchPattern {
+
+    final int batchSize;
+    final BitSet bitSet;
+
+    private VectorBatchPattern(int batchSize, BitSet bitSet) {
+      this.batchSize = batchSize;
+      this.bitSet = bitSet;
+    }
+
+    public static VectorBatchPattern createRegularBatch(int batchSize) {
+      return new VectorBatchPattern(batchSize, null);
+    }
+
+    public static VectorBatchPattern createRepeatedBatch(int batchSize, BitSet bitSet) {
+      return new VectorBatchPattern(batchSize, bitSet);
+    }
+
+    public int getBatchSize() {
+      return batchSize;
+    }
+
+    public BitSet getBitSet() {
+      return bitSet;
+    }
+
+    public String toString() {
+      String batchSizeString = "batchSize " + Integer.toString(batchSize);
+      if (bitSet == null) {
+        return batchSizeString;
+      }
+      long bitMask = bitSet.toLongArray()[0];
+      return batchSizeString + " repeating 0x" + Long.toHexString(bitMask);
+    }
+  }
+
+  private static VectorBatchPatterns chooseBatchPatterns(
+      Random random,
+      VectorRandomRowSource vectorRandomRowSource,
+      Object[][] randomRows) {
+
+    List<VectorBatchPattern> vectorBatchPatternList = new ArrayList<VectorBatchPattern>();
+    final int rowCount = randomRows.length;
+    int rowIndex = 0;
+
+    if (rowCount > 0) {
+
+      final int columnCount = randomRows[0].length;
+
+      // Choose first up to a full batch.
+      final int regularBatchSize = Math.min(rowCount - rowIndex, VectorizedRowBatch.DEFAULT_SIZE);
+      vectorBatchPatternList.add(VectorBatchPattern.createRegularBatch(regularBatchSize));
+      rowIndex += regularBatchSize;
+
+      // Have a non-NULL value on hand.
+      Object[] nonNullRow = new Object[columnCount];
+      for (int c = 0; c < columnCount; c++) {
+        for (int r = 0; r < rowCount; r++) {
+          Object object = randomRows[r][c];
+          if (object != null) {
+            nonNullRow[c] = object;
+            break;
+          }
+        }
+      }
+
+      int columnPermutationLimit = Math.min(columnCount, Long.SIZE);
+
+      // Repeated NULL permutations.
+      long columnPermutation = 1;
+      while (true) {
+        if (columnPermutation > columnPermutationLimit) {
+          break;
+        }
+        final int maximumRowCount = Math.min(rowCount - rowIndex, VectorizedRowBatch.DEFAULT_SIZE);
+        if (maximumRowCount == 0) {
+          break;
+        }
+        int randomRowCount = 1 + random.nextInt(maximumRowCount);
+        final int rowLimit = rowIndex + randomRowCount;
+
+        BitSet bitSet = BitSet.valueOf(new long[]{columnPermutation});
+
+        for (int columnNum = bitSet.nextSetBit(0);
+             columnNum >= 0;
+             columnNum = bitSet.nextSetBit(columnNum + 1)) {
+
+          // Repeated NULL fill down column.
+          for (int r = rowIndex; r < rowLimit; r++) {
+            randomRows[r][columnNum] = null;
+          }
+        }
+        vectorBatchPatternList.add(VectorBatchPattern.createRepeatedBatch(randomRowCount, bitSet));
+        columnPermutation++;
+        rowIndex = rowLimit;
+      }
+
+      // Repeated non-NULL permutations.
+      columnPermutation = 1;
+      while (true) {
+        if (columnPermutation > columnPermutationLimit) {
+          break;
+        }
+        final int maximumRowCount = Math.min(rowCount - rowIndex, VectorizedRowBatch.DEFAULT_SIZE);
+        if (maximumRowCount == 0) {
+          break;
+        }
+        int randomRowCount = 1 + random.nextInt(maximumRowCount);
+        final int rowLimit = rowIndex + randomRowCount;
+
+        BitSet bitSet = BitSet.valueOf(new long[]{columnPermutation});
+
+        for (int columnNum = bitSet.nextSetBit(0);
+             columnNum >= 0;
+             columnNum = bitSet.nextSetBit(columnNum + 1)) {
+
+          // Repeated non-NULL fill down column.
+          Object repeatedObject = randomRows[rowIndex][columnNum];
+          if (repeatedObject == null) {
+            repeatedObject = nonNullRow[columnNum];
+          }
+          for (int r = rowIndex; r < rowLimit; r++) {
+            randomRows[r][columnNum] = repeatedObject;
+          }
+        }
+        vectorBatchPatternList.add(VectorBatchPattern.createRepeatedBatch(randomRowCount, bitSet));
+        columnPermutation++;
+        rowIndex = rowLimit;
+      }
+
+      // Remaining batches.
+      while (true) {
+        final int maximumRowCount = Math.min(rowCount - rowIndex, VectorizedRowBatch.DEFAULT_SIZE);
+        if (maximumRowCount == 0) {
+          break;
+        }
+        int randomRowCount = 1 + random.nextInt(maximumRowCount);
+        vectorBatchPatternList.add(VectorBatchPattern.createRegularBatch(randomRowCount));
+        rowIndex += randomRowCount;
+      }
+    }
+
+    // System.out.println("*DEBUG* vectorBatchPatternList" + vectorBatchPatternList.toString());
+
+    return new VectorBatchPatterns(vectorBatchPatternList);
+  }
+
+  public static VectorRandomBatchSource createInterestingBatches(
+      Random random,
+      VectorRandomRowSource vectorRandomRowSource,
+      Object[][] randomRows,
+      VectorRandomBatchParameters vectorRandomBatchParameters)
+          throws HiveException {
+
+    VectorAssignRow vectorAssignRow = new VectorAssignRow();
+    vectorAssignRow.init(vectorRandomRowSource.typeNames());
+
+    VectorBatchPatterns vectorBatchPatterns =
+        chooseBatchPatterns(random, vectorRandomRowSource, randomRows);
+
+    return new VectorRandomBatchSource(
+        vectorRandomRowSource, randomRows, vectorBatchPatterns, vectorAssignRow);
+  }
+
+  public VectorRandomRowSource getRowSource() {
+    return vectorRandomRowSource;
+  }
+
+  public Object[][] getRandomRows() {
+    return randomRows;
+  }
+
+  public void resetBatchIteration() {
+    nextRowIndex = 0;
+    batchCount = 0;
+  }
+
+  public int getBatchCount() {
+    return batchCount;
+  }
+
+  public int getRowCount() {
+    return rowCount;
+  }
+
+  /*
+   * Patterns of isRepeating columns
+   * For boolean: tri-state: null, 0, 1
+   * For others: null, some-value
+   * noNulls: sometimes false and there are no NULLs.
+   * Random selectedInUse, too.
+   */
+  public boolean fillNextBatch(VectorizedRowBatch batch) {
+    if (nextRowIndex >= rowCount) {
+      return false;
+    }
+
+    VectorBatchPattern vectorBatchPattern =
+        vectorBatchPatterns.getTectorBatchPatternList().get(batchCount);
+    final int batchSize = vectorBatchPattern.getBatchSize();
+
+    for (int c = 0; c < columnCount; c++) {
+      batch.cols[c].reset();
+    }
+
+    BitSet bitSet = vectorBatchPattern.getBitSet();
+    if (bitSet != null) {
+      for (int columnNum = bitSet.nextSetBit(0);
+           columnNum >= 0;
+           columnNum = bitSet.nextSetBit(columnNum + 1)) {
+        batch.cols[columnNum].isRepeating = true;
+      }
+    }
+
+    int rowIndex = nextRowIndex;
+    for (int batchIndex = 0; batchIndex < batchSize; batchIndex++) {
+      for (int c = 0; c < columnCount; c++) {
+        if (batch.cols[c].isRepeating && batchIndex > 0) {
+          continue;
+        }
+        vectorAssignRow.assignRowColumn(batch, batchIndex, c, randomRows[rowIndex][c]);
+      }
+      rowIndex++;
+    }
+    batch.size = batchSize;
+    batchCount++;
+    nextRowIndex += batchSize;
+    return true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/f3beace2/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/VectorRandomRowSource.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/VectorRandomRowSource.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/VectorRandomRowSource.java
index 3f99328..fa5c775 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/VectorRandomRowSource.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/VectorRandomRowSource.java
@@ -24,7 +24,9 @@ import java.util.ArrayList;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Random;
+import java.util.Set;
 
+import org.apache.hadoop.hive.common.type.DataTypePhysicalVariation;
 import org.apache.hadoop.hive.common.type.HiveChar;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
 import org.apache.hadoop.hive.common.type.HiveIntervalDayTime;
@@ -32,6 +34,7 @@ import org.apache.hadoop.hive.common.type.HiveIntervalYearMonth;
 import org.apache.hadoop.hive.common.type.HiveVarchar;
 import org.apache.hadoop.hive.common.type.RandomTypeUtil;
 import org.apache.hadoop.hive.serde2.io.HiveCharWritable;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 import org.apache.hadoop.hive.serde2.io.HiveVarcharWritable;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category;
@@ -71,6 +74,7 @@ import org.apache.hadoop.hive.serde2.typeinfo.MapTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.StructTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
 import org.apache.hadoop.hive.serde2.typeinfo.UnionTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.VarcharTypeInfo;
@@ -96,6 +100,8 @@ public class VectorRandomRowSource {
 
   private TypeInfo[] typeInfos;
 
+  private DataTypePhysicalVariation[] dataTypePhysicalVariations;
+
   private List<ObjectInspector> objectInspectorList;
 
   // Primitive.
@@ -127,6 +133,10 @@ public class VectorRandomRowSource {
     return typeInfos;
   }
 
+  public DataTypePhysicalVariation[] dataTypePhysicalVariations() {
+    return dataTypePhysicalVariations;
+  }
+
   public PrimitiveCategory[] primitiveCategories() {
     return primitiveCategories;
   }
@@ -163,7 +173,22 @@ public class VectorRandomRowSource {
   public void init(Random r, SupportedTypes supportedTypes, int maxComplexDepth, boolean allowNull) {
     this.r = r;
     this.allowNull = allowNull;
-    chooseSchema(supportedTypes, maxComplexDepth);
+    chooseSchema(supportedTypes, null, null, null, maxComplexDepth);
+  }
+
+  public void init(Random r, Set<String> allowedTypeNameSet, int maxComplexDepth, boolean allowNull) {
+    this.r = r;
+    this.allowNull = allowNull;
+    chooseSchema(SupportedTypes.ALL, allowedTypeNameSet, null, null, maxComplexDepth);
+  }
+
+  public void initExplicitSchema(Random r, List<String> explicitTypeNameList, int maxComplexDepth,
+      boolean allowNull, List<DataTypePhysicalVariation> explicitDataTypePhysicalVariationList) {
+    this.r = r;
+    this.allowNull = allowNull;
+    chooseSchema(
+        SupportedTypes.ALL, null, explicitTypeNameList, explicitDataTypePhysicalVariationList,
+        maxComplexDepth);
   }
 
   /*
@@ -180,7 +205,7 @@ public class VectorRandomRowSource {
       "float",
       "double",
       "string",
-//    "char",
+      "char",
       "varchar",
       "binary",
       "date",
@@ -197,27 +222,30 @@ public class VectorRandomRowSource {
       "map"
   };
 
-  private String getRandomTypeName(SupportedTypes supportedTypes) {
+  private String getRandomTypeName(SupportedTypes supportedTypes, Set<String> allowedTypeNameSet) {
     String typeName = null;
-    if (r.nextInt(10 ) != 0) {
-      typeName = possibleHivePrimitiveTypeNames[r.nextInt(possibleHivePrimitiveTypeNames.length)];
-    } else {
-      switch (supportedTypes) {
-      case PRIMITIVES:
+    do {
+      if (r.nextInt(10 ) != 0) {
         typeName = possibleHivePrimitiveTypeNames[r.nextInt(possibleHivePrimitiveTypeNames.length)];
-        break;
-      case ALL_EXCEPT_MAP:
-        typeName = possibleHiveComplexTypeNames[r.nextInt(possibleHiveComplexTypeNames.length - 1)];
-        break;
-      case ALL:
-        typeName = possibleHiveComplexTypeNames[r.nextInt(possibleHiveComplexTypeNames.length)];
-        break;
+      } else {
+        switch (supportedTypes) {
+        case PRIMITIVES:
+          typeName = possibleHivePrimitiveTypeNames[r.nextInt(possibleHivePrimitiveTypeNames.length)];
+          break;
+        case ALL_EXCEPT_MAP:
+          typeName = possibleHiveComplexTypeNames[r.nextInt(possibleHiveComplexTypeNames.length - 1)];
+          break;
+        case ALL:
+          typeName = possibleHiveComplexTypeNames[r.nextInt(possibleHiveComplexTypeNames.length)];
+          break;
+        }
       }
-    }
+    } while (allowedTypeNameSet != null && !allowedTypeNameSet.contains(typeName));
     return typeName;
   }
 
-  private String getDecoratedTypeName(String typeName, SupportedTypes supportedTypes, int depth, int maxDepth) {
+  private String getDecoratedTypeName(String typeName, SupportedTypes supportedTypes,
+      Set<String> allowedTypeNameSet, int depth, int maxDepth) {
     depth++;
     if (depth < maxDepth) {
       supportedTypes = SupportedTypes.PRIMITIVES;
@@ -229,23 +257,32 @@ public class VectorRandomRowSource {
       final int maxLength = 1 + r.nextInt(100);
       typeName = String.format("varchar(%d)", maxLength);
     } else if (typeName.equals("decimal")) {
-      typeName = String.format("decimal(%d,%d)", HiveDecimal.SYSTEM_DEFAULT_PRECISION, HiveDecimal.SYSTEM_DEFAULT_SCALE);
+      typeName =
+          String.format(
+              "decimal(%d,%d)",
+              HiveDecimal.SYSTEM_DEFAULT_PRECISION,
+              HiveDecimal.SYSTEM_DEFAULT_SCALE);
     } else if (typeName.equals("array")) {
-      String elementTypeName = getRandomTypeName(supportedTypes);
-      elementTypeName = getDecoratedTypeName(elementTypeName, supportedTypes, depth, maxDepth);
+      String elementTypeName = getRandomTypeName(supportedTypes, allowedTypeNameSet);
+      elementTypeName =
+          getDecoratedTypeName(elementTypeName, supportedTypes, allowedTypeNameSet, depth, maxDepth);
       typeName = String.format("array<%s>", elementTypeName);
     } else if (typeName.equals("map")) {
-      String keyTypeName = getRandomTypeName(SupportedTypes.PRIMITIVES);
-      keyTypeName = getDecoratedTypeName(keyTypeName, supportedTypes, depth, maxDepth);
-      String valueTypeName = getRandomTypeName(supportedTypes);
-      valueTypeName = getDecoratedTypeName(valueTypeName, supportedTypes, depth, maxDepth);
+      String keyTypeName = getRandomTypeName(SupportedTypes.PRIMITIVES, allowedTypeNameSet);
+      keyTypeName =
+          getDecoratedTypeName(keyTypeName, supportedTypes, allowedTypeNameSet, depth, maxDepth);
+      String valueTypeName = getRandomTypeName(supportedTypes, allowedTypeNameSet);
+      valueTypeName =
+          getDecoratedTypeName(valueTypeName, supportedTypes, allowedTypeNameSet, depth, maxDepth);
       typeName = String.format("map<%s,%s>", keyTypeName, valueTypeName);
     } else if (typeName.equals("struct")) {
       final int fieldCount = 1 + r.nextInt(10);
       final StringBuilder sb = new StringBuilder();
       for (int i = 0; i < fieldCount; i++) {
-        String fieldTypeName = getRandomTypeName(supportedTypes);
-        fieldTypeName = getDecoratedTypeName(fieldTypeName, supportedTypes, depth, maxDepth);
+        String fieldTypeName = getRandomTypeName(supportedTypes, allowedTypeNameSet);
+        fieldTypeName =
+            getDecoratedTypeName(
+                fieldTypeName, supportedTypes, allowedTypeNameSet, depth, maxDepth);
         if (i > 0) {
           sb.append(",");
         }
@@ -260,8 +297,10 @@ public class VectorRandomRowSource {
       final int fieldCount = 1 + r.nextInt(10);
       final StringBuilder sb = new StringBuilder();
       for (int i = 0; i < fieldCount; i++) {
-        String fieldTypeName = getRandomTypeName(supportedTypes);
-        fieldTypeName = getDecoratedTypeName(fieldTypeName, supportedTypes, depth, maxDepth);
+        String fieldTypeName = getRandomTypeName(supportedTypes, allowedTypeNameSet);
+        fieldTypeName =
+            getDecoratedTypeName(
+                fieldTypeName, supportedTypes, allowedTypeNameSet, depth, maxDepth);
         if (i > 0) {
           sb.append(",");
         }
@@ -273,14 +312,29 @@ public class VectorRandomRowSource {
   }
 
   private ObjectInspector getObjectInspector(TypeInfo typeInfo) {
+    return getObjectInspector(typeInfo, DataTypePhysicalVariation.NONE);
+  }
+
+  private ObjectInspector getObjectInspector(TypeInfo typeInfo,
+      DataTypePhysicalVariation dataTypePhysicalVariation) {
+
     final ObjectInspector objectInspector;
     switch (typeInfo.getCategory()) {
     case PRIMITIVE:
       {
-        final PrimitiveTypeInfo primitiveType = (PrimitiveTypeInfo) typeInfo;
-        objectInspector =
-            PrimitiveObjectInspectorFactory.
-                getPrimitiveWritableObjectInspector(primitiveType);
+        final PrimitiveTypeInfo primitiveTypeInfo = (PrimitiveTypeInfo) typeInfo;
+        if (primitiveTypeInfo instanceof DecimalTypeInfo &&
+            dataTypePhysicalVariation == DataTypePhysicalVariation.DECIMAL_64) {
+          objectInspector =
+              PrimitiveObjectInspectorFactory.
+                  getPrimitiveWritableObjectInspector(
+                      TypeInfoFactory.longTypeInfo);
+        } else {
+          objectInspector =
+              PrimitiveObjectInspectorFactory.
+                  getPrimitiveWritableObjectInspector(
+                      primitiveTypeInfo);
+        }
       }
       break;
     case MAP:
@@ -341,35 +395,50 @@ public class VectorRandomRowSource {
     return objectInspector;
   }
 
-  private void chooseSchema(SupportedTypes supportedTypes, int maxComplexDepth) {
-    HashSet hashSet = null;
+  private void chooseSchema(SupportedTypes supportedTypes, Set<String> allowedTypeNameSet,
+      List<String> explicitTypeNameList,
+      List<DataTypePhysicalVariation> explicitDataTypePhysicalVariationList,
+      int maxComplexDepth) {
+    HashSet<Integer> hashSet = null;
     final boolean allTypes;
-    final boolean onlyOne = (r.nextInt(100) == 7);
-    if (onlyOne) {
-      columnCount = 1;
+    final boolean onlyOne;
+    if (explicitTypeNameList != null) {
+      columnCount = explicitTypeNameList.size();
+      allTypes = false;
+      onlyOne = false;
+    } else if (allowedTypeNameSet != null) {
+      columnCount = 1 + r.nextInt(20);
       allTypes = false;
+      onlyOne = false;
     } else {
-      allTypes = r.nextBoolean();
-      if (allTypes) {
-        switch (supportedTypes) {
-        case ALL:
-          columnCount = possibleHivePrimitiveTypeNames.length + possibleHiveComplexTypeNames.length;
-          break;
-        case ALL_EXCEPT_MAP:
-          columnCount = possibleHivePrimitiveTypeNames.length + possibleHiveComplexTypeNames.length - 1;
-          break;
-        case PRIMITIVES:
-          columnCount = possibleHivePrimitiveTypeNames.length;
-          break;
-        }
-        hashSet = new HashSet<Integer>();
+      onlyOne = (r.nextInt(100) == 7);
+      if (onlyOne) {
+        columnCount = 1;
+        allTypes = false;
       } else {
-        columnCount = 1 + r.nextInt(20);
+        allTypes = r.nextBoolean();
+        if (allTypes) {
+          switch (supportedTypes) {
+          case ALL:
+            columnCount = possibleHivePrimitiveTypeNames.length + possibleHiveComplexTypeNames.length;
+            break;
+          case ALL_EXCEPT_MAP:
+            columnCount = possibleHivePrimitiveTypeNames.length + possibleHiveComplexTypeNames.length - 1;
+            break;
+          case PRIMITIVES:
+            columnCount = possibleHivePrimitiveTypeNames.length;
+            break;
+          }
+          hashSet = new HashSet<Integer>();
+        } else {
+          columnCount = 1 + r.nextInt(20);
+        }
       }
     }
     typeNames = new ArrayList<String>(columnCount);
     categories = new Category[columnCount];
     typeInfos = new TypeInfo[columnCount];
+    dataTypePhysicalVariations = new DataTypePhysicalVariation[columnCount];
     objectInspectorList = new ArrayList<ObjectInspector>(columnCount);
 
     primitiveCategories = new PrimitiveCategory[columnCount];
@@ -379,9 +448,13 @@ public class VectorRandomRowSource {
     for (int c = 0; c < columnCount; c++) {
       columnNames.add(String.format("col%d", c));
       final String typeName;
+      DataTypePhysicalVariation dataTypePhysicalVariation = DataTypePhysicalVariation.NONE;
 
-      if (onlyOne) {
-        typeName = getRandomTypeName(supportedTypes);
+      if (explicitTypeNameList != null) {
+        typeName = explicitTypeNameList.get(c);
+        dataTypePhysicalVariation = explicitDataTypePhysicalVariationList.get(c);
+      } else if (onlyOne || allowedTypeNameSet != null) {
+        typeName = getRandomTypeName(supportedTypes, allowedTypeNameSet);
       } else {
         int typeNum;
         if (allTypes) {
@@ -425,7 +498,8 @@ public class VectorRandomRowSource {
 
       }
 
-      String decoratedTypeName = getDecoratedTypeName(typeName, supportedTypes, 0, maxComplexDepth);
+      String decoratedTypeName =
+          getDecoratedTypeName(typeName, supportedTypes, allowedTypeNameSet, 0, maxComplexDepth);
 
       final TypeInfo typeInfo;
       try {
@@ -435,15 +509,14 @@ public class VectorRandomRowSource {
       }
 
       typeInfos[c] = typeInfo;
+      dataTypePhysicalVariations[c] = dataTypePhysicalVariation;
       final Category category = typeInfo.getCategory();
       categories[c] = category;
-      ObjectInspector objectInspector = getObjectInspector(typeInfo);
+      ObjectInspector objectInspector = getObjectInspector(typeInfo, dataTypePhysicalVariation);
       switch (category) {
       case PRIMITIVE:
         {
           final PrimitiveTypeInfo primitiveTypeInfo = (PrimitiveTypeInfo) typeInfo;
-          objectInspector = PrimitiveObjectInspectorFactory.
-              getPrimitiveWritableObjectInspector(primitiveTypeInfo);
           primitiveTypeInfos[c] = primitiveTypeInfo;
           PrimitiveCategory primitiveCategory = primitiveTypeInfo.getPrimitiveCategory();
           primitiveCategories[c] = primitiveCategory;
@@ -498,27 +571,46 @@ public class VectorRandomRowSource {
   }
 
   public Object[] randomPrimitiveRow(int columnCount) {
-    return randomPrimitiveRow(columnCount, r, primitiveTypeInfos);
+    return randomPrimitiveRow(columnCount, r, primitiveTypeInfos, dataTypePhysicalVariations);
   }
 
   public static Object[] randomPrimitiveRow(int columnCount, Random r,
-      PrimitiveTypeInfo[] primitiveTypeInfos) {
+      PrimitiveTypeInfo[] primitiveTypeInfos,
+      DataTypePhysicalVariation[] dataTypePhysicalVariations) {
 
     final Object row[] = new Object[columnCount];
     for (int c = 0; c < columnCount; c++) {
-      row[c] = randomPrimitiveObject(r, primitiveTypeInfos[c]);
+      row[c] = randomPrimitiveObject(r, primitiveTypeInfos[c], dataTypePhysicalVariations[c]);
     }
     return row;
   }
 
   public static Object[] randomWritablePrimitiveRow(int columnCount, Random r,
       PrimitiveTypeInfo[] primitiveTypeInfos) {
+    return randomWritablePrimitiveRow(columnCount, r, primitiveTypeInfos, null);
+  }
+
+  public static Object[] randomWritablePrimitiveRow(int columnCount, Random r,
+      PrimitiveTypeInfo[] primitiveTypeInfos,
+      DataTypePhysicalVariation[] dataTypePhysicalVariations) {
 
     final Object row[] = new Object[columnCount];
     for (int c = 0; c < columnCount; c++) {
       final PrimitiveTypeInfo primitiveTypeInfo = primitiveTypeInfos[c];
-      final ObjectInspector objectInspector =
-          PrimitiveObjectInspectorFactory.getPrimitiveWritableObjectInspector(primitiveTypeInfo);
+      final DataTypePhysicalVariation dataTypePhysicalVariation =
+          (dataTypePhysicalVariations != null ?
+              dataTypePhysicalVariations[c] : DataTypePhysicalVariation.NONE);
+      final ObjectInspector objectInspector;
+      if (primitiveTypeInfo instanceof DecimalTypeInfo &&
+          dataTypePhysicalVariation == DataTypePhysicalVariation.DECIMAL_64) {
+        objectInspector =
+            PrimitiveObjectInspectorFactory.getPrimitiveWritableObjectInspector(
+                TypeInfoFactory.longTypeInfo);
+      } else {
+        objectInspector =
+            PrimitiveObjectInspectorFactory.getPrimitiveWritableObjectInspector(
+                primitiveTypeInfo);
+      }
       final Object object = randomPrimitiveObject(r, primitiveTypeInfo);
       row[c] = getWritablePrimitiveObject(primitiveTypeInfo, objectInspector, object);
     }
@@ -575,6 +667,14 @@ public class VectorRandomRowSource {
 
   public static Object getWritablePrimitiveObject(PrimitiveTypeInfo primitiveTypeInfo,
       ObjectInspector objectInspector, Object object) {
+    return
+        getWritablePrimitiveObject(
+            primitiveTypeInfo, objectInspector, DataTypePhysicalVariation.NONE, object);
+  }
+
+  public static Object getWritablePrimitiveObject(PrimitiveTypeInfo primitiveTypeInfo,
+      ObjectInspector objectInspector, DataTypePhysicalVariation dataTypePhysicalVariation,
+      Object object) {
 
     switch (primitiveTypeInfo.getPrimitiveCategory()) {
     case BOOLEAN:
@@ -596,17 +696,17 @@ public class VectorRandomRowSource {
     case STRING:
       return ((WritableStringObjectInspector) objectInspector).create((String) object);
     case CHAR:
-    {
-      WritableHiveCharObjectInspector writableCharObjectInspector =
-          new WritableHiveCharObjectInspector( (CharTypeInfo) primitiveTypeInfo);
-      return writableCharObjectInspector.create((HiveChar) object);
-    }
+      {
+        WritableHiveCharObjectInspector writableCharObjectInspector =
+            new WritableHiveCharObjectInspector( (CharTypeInfo) primitiveTypeInfo);
+        return writableCharObjectInspector.create((HiveChar) object);
+      }
     case VARCHAR:
-    {
-      WritableHiveVarcharObjectInspector writableVarcharObjectInspector =
-          new WritableHiveVarcharObjectInspector( (VarcharTypeInfo) primitiveTypeInfo);
-      return writableVarcharObjectInspector.create((HiveVarchar) object);
-    }
+      {
+        WritableHiveVarcharObjectInspector writableVarcharObjectInspector =
+            new WritableHiveVarcharObjectInspector( (VarcharTypeInfo) primitiveTypeInfo);
+        return writableVarcharObjectInspector.create((HiveVarchar) object);
+      }
     case BINARY:
       return PrimitiveObjectInspectorFactory.writableBinaryObjectInspector.create((byte[]) object);
     case TIMESTAMP:
@@ -616,31 +716,55 @@ public class VectorRandomRowSource {
     case INTERVAL_DAY_TIME:
       return ((WritableHiveIntervalDayTimeObjectInspector) objectInspector).create((HiveIntervalDayTime) object);
     case DECIMAL:
-    {
-      WritableHiveDecimalObjectInspector writableDecimalObjectInspector =
-          new WritableHiveDecimalObjectInspector((DecimalTypeInfo) primitiveTypeInfo);
-      return writableDecimalObjectInspector.create((HiveDecimal) object);
-    }
+      {
+        if (dataTypePhysicalVariation == dataTypePhysicalVariation.DECIMAL_64) {
+          final long value;
+          if (object instanceof HiveDecimal) {
+            DecimalTypeInfo decimalTypeInfo = (DecimalTypeInfo) primitiveTypeInfo;
+            value = new HiveDecimalWritable((HiveDecimal) object).serialize64(
+                decimalTypeInfo.getScale());
+          } else {
+            value = (long) object;
+          }
+          return ((WritableLongObjectInspector) objectInspector).create(value);
+        } else {
+          WritableHiveDecimalObjectInspector writableDecimalObjectInspector =
+              new WritableHiveDecimalObjectInspector((DecimalTypeInfo) primitiveTypeInfo);
+          return writableDecimalObjectInspector.create((HiveDecimal) object);
+        }
+      }
     default:
       throw new Error("Unknown primitive category " + primitiveTypeInfo.getPrimitiveCategory());
     }
   }
 
   public Object randomWritable(int column) {
-    return randomWritable(typeInfos[column], objectInspectorList.get(column));
+    return randomWritable(
+        typeInfos[column], objectInspectorList.get(column), dataTypePhysicalVariations[column],
+        allowNull);
   }
 
   public Object randomWritable(TypeInfo typeInfo, ObjectInspector objectInspector) {
-    return randomWritable(typeInfo, objectInspector, allowNull);
+    return randomWritable(typeInfo, objectInspector, DataTypePhysicalVariation.NONE, allowNull);
+  }
+
+  public Object randomWritable(TypeInfo typeInfo, ObjectInspector objectInspector,
+      boolean allowNull) {
+    return randomWritable(typeInfo, objectInspector, DataTypePhysicalVariation.NONE, allowNull);
   }
 
-  public Object randomWritable(TypeInfo typeInfo, ObjectInspector objectInspector, boolean allowNull) {
+  public Object randomWritable(TypeInfo typeInfo, ObjectInspector objectInspector,
+      DataTypePhysicalVariation dataTypePhysicalVariation, boolean allowNull) {
 
     switch (typeInfo.getCategory()) {
     case PRIMITIVE:
       {
+        if (allowNull && r.nextInt(20) == 0) {
+          return null;
+        }
         final Object object = randomPrimitiveObject(r, (PrimitiveTypeInfo) typeInfo);
-        return getWritablePrimitiveObject((PrimitiveTypeInfo) typeInfo, objectInspector, object);
+        return getWritablePrimitiveObject(
+            (PrimitiveTypeInfo) typeInfo, objectInspector, dataTypePhysicalVariation, object);
       }
     case LIST:
       {
@@ -780,6 +904,11 @@ public class VectorRandomRowSource {
   }
 
   public static Object randomPrimitiveObject(Random r, PrimitiveTypeInfo primitiveTypeInfo) {
+    return randomPrimitiveObject(r, primitiveTypeInfo, DataTypePhysicalVariation.NONE);
+  }
+
+  public static Object randomPrimitiveObject(Random r, PrimitiveTypeInfo primitiveTypeInfo,
+      DataTypePhysicalVariation dataTypePhysicalVariation) {
 
     switch (primitiveTypeInfo.getPrimitiveCategory()) {
     case BOOLEAN:
@@ -813,9 +942,14 @@ public class VectorRandomRowSource {
     case INTERVAL_DAY_TIME:
       return getRandIntervalDayTime(r);
     case DECIMAL:
-    {
-      return getRandHiveDecimal(r, (DecimalTypeInfo) primitiveTypeInfo);
-    }
+      {
+        DecimalTypeInfo decimalTypeInfo = (DecimalTypeInfo) primitiveTypeInfo;
+        HiveDecimal hiveDecimal = getRandHiveDecimal(r, decimalTypeInfo);
+        if (dataTypePhysicalVariation == DataTypePhysicalVariation.DECIMAL_64) {
+          return new HiveDecimalWritable(hiveDecimal).serialize64(decimalTypeInfo.getScale());
+        }
+        return hiveDecimal;
+      }
     default:
       throw new Error("Unknown primitive category " + primitiveTypeInfo.getCategory());
     }
@@ -869,7 +1003,13 @@ public class VectorRandomRowSource {
         sb.append(RandomTypeUtil.getRandString(r, DECIMAL_CHARS, scale));
       }
 
-      return HiveDecimal.create(sb.toString());
+      HiveDecimal dec = HiveDecimal.create(sb.toString());
+      dec =
+          HiveDecimal.enforcePrecisionScale(
+              dec, decimalTypeInfo.getPrecision(), decimalTypeInfo.getScale());
+      if (dec != null) {
+        return dec;
+      }
     }
   }